From 4fa711ca2b4e9276284071c8f1f195603cfec6c7 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 11 Oct 2024 14:32:42 +0200 Subject: [PATCH 01/40] feat: TryRecovery broadcast and replica impls --- synevi_core/src/lib.rs | 6 +++++ synevi_core/src/replica.rs | 25 +++++++++++++++++-- synevi_network/src/network.rs | 25 +++++++++++++++++-- .../src/protos/consensus_transport.proto | 7 ++++++ synevi_network/src/replica.rs | 15 +++++++++++ tests/maelstrom/network.rs | 3 +++ 6 files changed, 77 insertions(+), 4 deletions(-) diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 4baec48..6afd80c 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -16,6 +16,7 @@ pub mod tests { use synevi_types::Executor; use synevi_types::SyneviError; use synevi_types::T; + use synevi_types::T0; use tokio::sync::mpsc::Receiver; use tokio::sync::Mutex; use ulid::Ulid; @@ -40,6 +41,11 @@ pub mod tests { self.got_requests.lock().await.push(request); Ok(vec![]) } + async fn broadcast_recovery( + &self, _t0: T0 + ) -> Result<(), SyneviError> { + Ok(()) + } } #[async_trait::async_trait] diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 24fd15e..b118963 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -1,3 +1,4 @@ +use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; use crate::wait_handler::WaitAction; @@ -12,7 +13,8 @@ use synevi_network::configure_transport::{ }; use synevi_network::consensus_transport::{ AcceptRequest, AcceptResponse, ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, - PreAcceptRequest, PreAcceptResponse, RecoverRequest, RecoverResponse, + PreAcceptRequest, PreAcceptResponse, RecoverRequest, RecoverResponse, TryRecoverRequest, + TryRecoverResponse, }; use synevi_network::network::Network; use synevi_network::reconfiguration::{BufferedMessage, Reconfiguration, Report}; @@ -413,6 +415,21 @@ where nack: Ballot::default().into(), }) } + + #[instrument(level = "trace", skip(self))] + async fn try_recover( + &self, + request: TryRecoverRequest, + ready: bool, + ) -> Result { + if ready { + let t0 = T0::try_from(request.timestamp_zero.as_slice())?; + if self.node.event_store.get_event(t0).await?.is_some() { + Coordinator::recover(self.node.clone(), t0).await?; + } + } + Ok(TryRecoverResponse {}) + } } #[async_trait::async_trait] @@ -469,7 +486,11 @@ where let (sdx, rcv) = tokio::sync::mpsc::channel(200); let event_id = u128::from_be_bytes(request.self_event.as_slice().try_into()?); let last_applied = T::try_from(request.last_applied.as_slice())?; - let mut store_rcv = self.node.event_store.get_events_after(last_applied, event_id).await?; + let mut store_rcv = self + .node + .event_store + .get_events_after(last_applied, event_id) + .await?; tokio::spawn(async move { while let Some(Ok(event)) = store_rcv.recv().await { let response = { diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index fdcbbdf..7b59855 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -7,7 +7,7 @@ use crate::configure_transport::{ Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, ReadyElectorateRequest, ReportLastAppliedRequest, }; -use crate::consensus_transport::{RecoverRequest, RecoverResponse}; +use crate::consensus_transport::{RecoverRequest, RecoverResponse, TryRecoverRequest}; use crate::latency_service::get_latency; use crate::reconfiguration::Reconfiguration; use crate::{ @@ -23,7 +23,7 @@ use std::collections::HashMap; use std::sync::atomic::{AtomicI64, AtomicU64, Ordering}; use std::{net::SocketAddr, sync::Arc}; use synevi_types::error::SyneviError; -use synevi_types::T; +use synevi_types::{T, T0}; use tokio::sync::{Mutex, RwLock}; use tokio::task::JoinSet; use tonic::metadata::{AsciiMetadataKey, AsciiMetadataValue}; @@ -36,6 +36,7 @@ pub trait NetworkInterface: Send + Sync { &self, request: BroadcastRequest, ) -> Result, SyneviError>; + async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError>; // All members } #[async_trait::async_trait] @@ -156,6 +157,9 @@ where ) -> Result, SyneviError> { self.as_ref().broadcast(request).await } + async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError> { + self.as_ref().broadcast_recovery(t0).await + } } #[derive(Clone, Debug, Default)] @@ -340,6 +344,7 @@ impl Network for GrpcNetwork { Ok((response.majority, response.self_event)) } + async fn report_config( &self, last_applied: T, @@ -616,4 +621,20 @@ impl NetworkInterface for GrpcNetworkSet { } Ok(result) } + + async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError> { + let mut responses: JoinSet> = JoinSet::new(); + let inner_request = TryRecoverRequest { timestamp_zero: t0.into() }; + for replica in &self.members { + let channel = replica.channel.clone(); + let request = tonic::Request::new(inner_request.clone()); + responses.spawn(async move { + let mut client = ConsensusTransportClient::new(channel); + client.try_recover(request).await?; + Ok(()) + }); + } + tokio::spawn(async move { responses.join_all().await }); + Ok(()) + } } diff --git a/synevi_network/src/protos/consensus_transport.proto b/synevi_network/src/protos/consensus_transport.proto index 5447781..157a761 100644 --- a/synevi_network/src/protos/consensus_transport.proto +++ b/synevi_network/src/protos/consensus_transport.proto @@ -8,6 +8,7 @@ service ConsensusTransport { rpc Accept(AcceptRequest) returns (AcceptResponse) {} rpc Apply(ApplyRequest) returns (ApplyResponse) {} rpc Recover(RecoverRequest) returns (RecoverResponse) {} + rpc TryRecover(TryRecoverRequest) returns (TryRecoverResponse) {} } message PreAcceptRequest { @@ -86,3 +87,9 @@ message RecoverResponse { bytes timestamp = 5; bytes nack = 6; } + +message TryRecoverRequest { + bytes timestamp_zero = 1; +} + +message TryRecoverResponse {} diff --git a/synevi_network/src/replica.rs b/synevi_network/src/replica.rs index 7d33dfd..3f81977 100644 --- a/synevi_network/src/replica.rs +++ b/synevi_network/src/replica.rs @@ -46,6 +46,12 @@ pub trait Replica: Send + Sync { ready: bool, ) -> Result; + async fn try_recover( + &self, + request: TryRecoverRequest, + ready: bool, + ) -> Result; + fn is_ready(&self) -> bool; } @@ -189,6 +195,15 @@ where .map_err(|e| tonic::Status::internal(e.to_string()))?, )) } + + async fn try_recover(&self, request: Request) -> Result, Status> { + Ok(Response::new( + self.inner + .try_recover(request.into_inner(), self.inner.is_ready()) + .await + .map_err(|e| tonic::Status::internal(e.to_string()))?, + )) + } } #[async_trait::async_trait] diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index 17ed147..85e2574 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -418,6 +418,9 @@ impl NetworkInterface for MaelstromNetwork { Ok(result) } + async fn broadcast_recovery(&self, _t0: T0) -> Result<(), SyneviError> { + todo!() + } } pub(crate) async fn replica_dispatch( From 652e932b3993d40bc3b6feaadcc0a0540f3cb871 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Mon, 14 Oct 2024 09:27:28 +0200 Subject: [PATCH 02/40] feat: Prepared Store for unknown event recovery --- synevi_core/src/coordinator.rs | 15 ++++------- synevi_core/src/replica.rs | 9 +++++-- synevi_core/src/wait_handler.rs | 41 +++++++++++++++++++++-------- synevi_persistence/src/database.rs | 12 ++++----- synevi_persistence/src/mem_store.rs | 12 ++++----- synevi_types/src/traits.rs | 2 +- 6 files changed, 55 insertions(+), 36 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 95f9b4b..e521525 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -15,7 +15,7 @@ use synevi_network::network::{BroadcastRequest, Network, NetworkInterface}; use synevi_network::utils::IntoInner; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalExecution, RecoveryState, SyneviResult, TransactionPayload, + ExecutorResult, Hashes, InternalExecution, RecoverEvent, RecoveryState, SyneviResult, TransactionPayload }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; use tracing::{instrument, trace}; @@ -363,14 +363,9 @@ where } #[instrument(level = "trace", skip(node))] - pub async fn recover(node: Arc>, t0_recover: T0) -> SyneviResult { + pub async fn recover(node: Arc>, recover_event: RecoverEvent) -> SyneviResult { loop { let node = node.clone(); - let recover_event = node - .event_store - .recover_event(&t0_recover, node.get_info().serial) - .await; - let recover_event = recover_event?; let network_interface = node.network.get_interface().await; let recover_responses = network_interface @@ -378,20 +373,20 @@ where id: recover_event.id.to_be_bytes().to_vec(), ballot: recover_event.ballot.into(), event: recover_event.transaction.clone(), - timestamp_zero: t0_recover.into(), + timestamp_zero: recover_event.t_zero.into(), })) .await?; let mut recover_coordinator = Coordinator:: { node, transaction: TransactionStateMachine { - transaction: TransactionPayload::from_bytes(recover_event.transaction)?, + transaction: TransactionPayload::from_bytes(recover_event.transaction.clone())?, t_zero: recover_event.t_zero, t: recover_event.t, ballot: recover_event.ballot, state: recover_event.state, id: recover_event.id, - dependencies: recover_event.dependencies, + dependencies: recover_event.dependencies.clone(), }, }; diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index b118963..b3acfcb 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -424,8 +424,13 @@ where ) -> Result { if ready { let t0 = T0::try_from(request.timestamp_zero.as_slice())?; - if self.node.event_store.get_event(t0).await?.is_some() { - Coordinator::recover(self.node.clone(), t0).await?; + if let Some(recover_event) = self + .node + .event_store + .recover_event(&t0, self.node.get_info().serial) + .await? + { + Coordinator::recover(self.node.clone(), recover_event).await?; } } Ok(TryRecoverResponse {}) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index b8b7a37..c0f6c0f 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -9,7 +9,7 @@ use std::{ sync::Arc, time::{Duration, Instant}, }; -use synevi_network::network::Network; +use synevi_network::network::{Network, NetworkInterface}; use synevi_types::traits::Store; use synevi_types::types::UpsertEvent; use synevi_types::{Executor, State, SyneviError, T, T0}; @@ -100,7 +100,6 @@ where } pub async fn run(self: Arc) -> Result<(), SyneviError> { - let mut waiter_state = WaiterState::new(); let mut recovering = BTreeSet::new(); @@ -201,7 +200,11 @@ where recovering.insert(t0_recover); let recover_t0 = recovering.pop_first().unwrap_or(t0_recover); let wait_handler = self.clone(); - wait_handler.recover(recover_t0, &mut waiter_state).await; + if let Err(err) = wait_handler.recover(recover_t0, &mut waiter_state).await + { + tracing::error!("Error recovering event: {:?}", err); + println!("Error recovering event: {:?}", err); + }; } } } @@ -265,17 +268,33 @@ where Ok(()) } - async fn recover(self: Arc, t0_recover: T0, waiter_state: &mut WaiterState) { + async fn recover( + self: Arc, + t0_recover: T0, + waiter_state: &mut WaiterState, + ) -> Result<(), SyneviError> { if let Some(event) = waiter_state.events.get_mut(&t0_recover) { event.started_at = Instant::now(); } - let node = self.node.clone(); - tokio::spawn(async move { - if let Err(err) = Coordinator::recover(node, t0_recover).await { - tracing::error!("Error recovering event: {:?}", err); - println!("Error recovering event: {:?}", err); - } - }); + if let Some(recover_event) = self + .node + .event_store + .recover_event(&t0_recover, self.node.get_info().serial) + .await? + { + let node = self.node.clone(); + tokio::spawn(async move { + if let Err(err) = Coordinator::recover(node, recover_event).await { + tracing::error!("Error recovering event: {:?}", err); + println!("Error recovering event: {:?}", err); + } + }); + } else { + let interface = self.node.network.get_interface().await; + interface.broadcast_recovery(t0_recover).await?; + todo!() + }; + Ok(()) } fn check_recovery(&self, waiter_state: &mut WaiterState) -> Option { diff --git a/synevi_persistence/src/database.rs b/synevi_persistence/src/database.rs index c47150e..6939401 100644 --- a/synevi_persistence/src/database.rs +++ b/synevi_persistence/src/database.rs @@ -161,7 +161,7 @@ impl Store for PersistentStore { &self, t_zero_recover: &T0, node_serial: u16, - ) -> Result { + ) -> Result, SyneviError> { self.data .lock() .await @@ -493,9 +493,9 @@ impl MutableData { &self, t_zero_recover: &T0, node_serial: u16, - ) -> Result { + ) -> Result, SyneviError> { let Some(state) = self.get_event_state(t_zero_recover).await else { - return Err(SyneviError::EventNotFound(t_zero_recover.get_inner())); + return Ok(None); }; if matches!(state, synevi_types::State::Undefined) { return Err(SyneviError::UndefinedRecovery); @@ -513,7 +513,7 @@ impl MutableData { db.put(&mut write_txn, &t_zero_recover.get_inner(), &event)?; write_txn.commit()?; - Ok(RecoverEvent { + Ok(Some(RecoverEvent { id: event.id, t_zero: event.t_zero, t: event.t, @@ -521,9 +521,9 @@ impl MutableData { transaction: event.transaction.clone(), dependencies: event.dependencies.clone(), ballot: event.ballot, - }) + })) } else { - Err(SyneviError::EventNotFound(t_zero_recover.get_inner())) + Ok(None) } } diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index f4e80d8..6cc31d0 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -72,7 +72,7 @@ impl Store for MemStore { &self, t_zero_recover: &T0, node_serial: u16, - ) -> Result { + ) -> Result, SyneviError> { self.store .lock() .await @@ -381,9 +381,9 @@ impl InternalStore { &mut self, t_zero_recover: &T0, node_serial: u16, - ) -> Result { + ) -> Result, SyneviError> { let Some(state) = self.get_event_state(t_zero_recover) else { - return Err(SyneviError::EventNotFound(t_zero_recover.get_inner())); + return Ok(None); }; if matches!(state, synevi_types::State::Undefined) { return Err(SyneviError::UndefinedRecovery); @@ -392,7 +392,7 @@ impl InternalStore { if let Some(event) = self.events.get_mut(t_zero_recover) { event.ballot = Ballot(event.ballot.next_with_node(node_serial).into_time()); - Ok(RecoverEvent { + Ok(Some(RecoverEvent { id: event.id, t_zero: event.t_zero, t: event.t, @@ -400,9 +400,9 @@ impl InternalStore { transaction: event.transaction.clone(), dependencies: event.dependencies.clone(), ballot: event.ballot, - }) + })) } else { - Err(SyneviError::EventNotFound(t_zero_recover.get_inner())) + Ok(None) } } diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index dd6f70f..d0643a9 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -89,7 +89,7 @@ pub trait Store: Send + Sync + Sized + 'static { &self, t_zero_recover: &T0, node_serial: u16, - ) -> Result; + ) -> Result, SyneviError>; // Check and update the ballot for a transaction // Returns true if the ballot was accepted (current <= ballot) async fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option; From 7e8bfa664da2c8e0ef2a02083c0144ac55a780ed Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 14 Oct 2024 14:47:46 +0200 Subject: [PATCH 03/40] chore: Fix naming add docs for recovery --- docs/reconfiguration.md | 131 ------------------ docs/recovery.md | 30 ++++ synevi_core/src/replica.rs | 18 ++- synevi_network/Cargo.toml | 4 +- synevi_network/src/network.rs | 39 ++++-- .../src/protos/configure_transport.proto | 22 --- .../src/protos/consensus_transport.proto | 8 +- synevi_network/src/replica.rs | 6 +- synevi_types/src/error.rs | 3 + 9 files changed, 84 insertions(+), 177 deletions(-) create mode 100644 docs/recovery.md diff --git a/docs/reconfiguration.md b/docs/reconfiguration.md index 0a6caac..505d62d 100644 --- a/docs/reconfiguration.md +++ b/docs/reconfiguration.md @@ -1,134 +1,3 @@ # Reconfiguration Protocol -Each node has an epoch u32. - -# Initialisation (0 Nodes) - -The first node starts with `epoch = 0`. It adds itself to the config (address etc.). And initializes with serial 0; - -# Join cluster as new node - -TODO: Electorate Networkinterface pre-accept -Electorate @ pre-accept = Set of all members from current epoch -If pre-accept returns a new epoch... Fetch configuration. -Accept to all members of old epoch and all members of new epoch - -In our impl: -When the coordinator receives info about a new epoch in PA responses. --> Fetch config of epoch from appropriate node -Await fetch && PA to all (old + new) - -1. Update Config request (UpdateConfigRequest) on node N1 -2. Wait for the current epoch to be ready ? (ReadyEpoch = e) -3. If node was part of e-1 - --------------------------------- - -### Onboarding revision 2 - -ArunaCtx { - 1. New node starts NewNodeCtx - 2. New node requests config + all_applieds from ExistingNodeCtx - 3. Existing node 1 sends special transaction (NewConfig + Epoch) to other nodes with new config -} - -NewNodeCtx { - 1. Init ReplicaBuffer and collect Commits/Applies - 2. Apply all_applieds - 3. Apply/Commit ReplicaBuffer - 4. Send JoinElectorate && Init FullReplica -} - -ExistingNodeCtx { - 1. Receive UpdateConfig - 2. Send all applieds - 4. Update Config - 5. Send NewConfig to all -} - -ElectorateCtx { - 1. Rcv & Apply NewConfig + Epoch - 2. Broadcast to updated config - 3. Keep old electorate until RcvJoinElectorate -} - -OldElectorateCtx { - 1. if new epoch start ArunaCtx ? -} - - - --------------------------------- - -### Onboarding Phase 1 - -- New node gets created -> Send Join request to an existing node -- Wait for the current epoch (e) to be ready - - Responds with current config (e) -- Existing node creates a transaction (to the old electorate) with the new epoch and asynchronously starts the reconfiguration protocol. - - Replicas notice the new epoch and initialize the reconfiguration protocol after fetching from node - -X | | | X -New epoch +1 (ts) - -- Reconfiguration protocol: - -- Wait for the current epoch to be ready -- If node was part of e-1 (previous epoch) - - Wait until the ReadyElectorate is true - - Send join electorate to all new nodes -- If node is a "new" one (Is part of e+1 but not e) - - ReadyElectorate = false -- else: ReadyElectorate = true - -- Increase epoch e = e+1 -- - - - - - - - - -------- -## Old ideas -0. Accept all consensus requests and put them to a buffer. - -1. Ask all node for last applied + hash + current configuration -2. Ask all other nodes if max last applied + hash is correct ? -3. If yes OK else ask another node 1. -4. Download from any node that responded OK all data from your last_applied to the last_applied from 1. (Check if your last applied is correct ?!) - -If your node was existing: - -1. No consensus requests received --> Check if you are - -Existing nodes can check if they are already a member of the current epoch and/or their state is correct (yet outdated). -- If state is correct: Recover the latest states and go ahead - - You are still member of an epoch and will receive Consensus requests. -> Go to buffer - - The first Apply Consensus you will receive is the last that needs to be synced out of consensus. - - Sync ! -- If state is incorrect: Create a new epoch that replaces the old node with a new one - -### Onboarding Phase 2 - -Fetch all applied of node-x based on my last applied + last applied hash; - -Node sends a `FetchConfiguration` request to a running node (possibly the initial one). I receives the current member configuration and the current epoch e. - -Reponse: Current config from the node (possible outdated) - -Node sends a `JoinCluster` request to all current members with e == e+1 and its own configuration and optionally its last applied t0. - -Response: -Majority(Old) -> OK (Last applied / Hash) - -> Replica start - - -NoMajority because someone else already got majority for this epoch. - - -# Re-join cluster as existing node - diff --git a/docs/recovery.md b/docs/recovery.md new file mode 100644 index 0000000..366d2e7 --- /dev/null +++ b/docs/recovery.md @@ -0,0 +1,30 @@ +# Recovery + +Transactions are recovered after a specific time has passed. + +If the node has data about the transaction (is either the coordinator or has witnessed a previous request). +Send recover request to all replicas. This has the function as an explicit pre-accept request and all replicas answer with their current state of the transaction (Tx). + +## Cases + +- No other has witnessed Tx -> Recover functions as Pre-Accept, continue with Accept +- If others have only witnessed a Pre-Accept: + - Do superseding transactions exist ? -> Continue with Accept + - Are there "waiting" transactions that should execute before ? -> Wait and restart recovery + - Otherwise continue with Accept + - For Accept: Enforce the fast-path if the original request got a majority (original T0 as T) otherwise use max(T) +- Accepted / Commited / Applied -> Continue with last known state + +## Synevi specific implementation details + +Unlike the original Accord implementation, synevi only collects T0s of dependencies. +This was done to mitigate the effects of larger transaction bodys, which would otherwise result in drastically increased message sizes. + +One downside of this is that nodes can witness the existence of transactions via dependencies without the actual transaction body. +If it is necessary to recover that transaction to satisfy the dependencies, this creates a problem of "unknown recoveries". + +To fix this: The waiting node can request a recovery from other nodes. These nodes will respond with a status information if they have witnessed the Tx and can recover it. As long as one node has the ability to recover the Tx and has witnessed the request the Tx will be recovered. + +If a majority responds with an unable to recover status and the rest do not respond, the dependency can be removed: + +The reason for this is that because the original transaction has not yet received a majority, it cannot have taken the fast path and must be recovered using a slow path. This also implies that a node that responds with a false state cannot allow the transaction to take the fast path. \ No newline at end of file diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index b3acfcb..c606167 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -13,8 +13,8 @@ use synevi_network::configure_transport::{ }; use synevi_network::consensus_transport::{ AcceptRequest, AcceptResponse, ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, - PreAcceptRequest, PreAcceptResponse, RecoverRequest, RecoverResponse, TryRecoverRequest, - TryRecoverResponse, + PreAcceptRequest, PreAcceptResponse, RecoverRequest, RecoverResponse, TryRecoveryRequest, + TryRecoveryResponse, }; use synevi_network::network::Network; use synevi_network::reconfiguration::{BufferedMessage, Reconfiguration, Report}; @@ -419,9 +419,9 @@ where #[instrument(level = "trace", skip(self))] async fn try_recover( &self, - request: TryRecoverRequest, + request: TryRecoveryRequest, ready: bool, - ) -> Result { + ) -> Result { if ready { let t0 = T0::try_from(request.timestamp_zero.as_slice())?; if let Some(recover_event) = self @@ -430,10 +430,16 @@ where .recover_event(&t0, self.node.get_info().serial) .await? { - Coordinator::recover(self.node.clone(), recover_event).await?; + tokio::spawn(Coordinator::recover(self.node.clone(), recover_event)); + return Ok(TryRecoveryResponse { + accepted: true, + }) } } - Ok(TryRecoverResponse {}) + + Ok(TryRecoveryResponse { + accepted: false, + }) } } diff --git a/synevi_network/Cargo.toml b/synevi_network/Cargo.toml index 8ffaebd..2613aa8 100644 --- a/synevi_network/Cargo.toml +++ b/synevi_network/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true description.workspace = true [dependencies] -prost = "0.13.0" +prost = "0.13.3" tokio = {workspace = true} tokio-stream = {workspace = true} futures-util = "0.3.30" @@ -23,4 +23,4 @@ synevi_types = { path = "../synevi_types", version = "0.1.0"} ahash = {workspace = true} [build-dependencies] -tonic-build = "0.12.0" +tonic-build = "0.12.3" diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 7b59855..b536759 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -7,7 +7,7 @@ use crate::configure_transport::{ Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, ReadyElectorateRequest, ReportLastAppliedRequest, }; -use crate::consensus_transport::{RecoverRequest, RecoverResponse, TryRecoverRequest}; +use crate::consensus_transport::{RecoverRequest, RecoverResponse, TryRecoveryRequest}; use crate::latency_service::get_latency; use crate::reconfiguration::Reconfiguration; use crate::{ @@ -36,7 +36,7 @@ pub trait NetworkInterface: Send + Sync { &self, request: BroadcastRequest, ) -> Result, SyneviError>; - async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError>; // All members + async fn broadcast_recovery(&self, t0: T0) -> Result; // All members } #[async_trait::async_trait] @@ -157,7 +157,7 @@ where ) -> Result, SyneviError> { self.as_ref().broadcast(request).await } - async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError> { + async fn broadcast_recovery(&self, t0: T0) -> Result { self.as_ref().broadcast_recovery(t0).await } } @@ -622,19 +622,38 @@ impl NetworkInterface for GrpcNetworkSet { Ok(result) } - async fn broadcast_recovery(&self, t0: T0) -> Result<(), SyneviError> { - let mut responses: JoinSet> = JoinSet::new(); - let inner_request = TryRecoverRequest { timestamp_zero: t0.into() }; + async fn broadcast_recovery(&self, t0: T0) -> Result { + let mut responses: JoinSet> = JoinSet::new(); + let inner_request = TryRecoveryRequest { timestamp_zero: t0.into() }; for replica in &self.members { let channel = replica.channel.clone(); let request = tonic::Request::new(inner_request.clone()); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); - client.try_recover(request).await?; - Ok(()) + let result = client.try_recovery(request).await?.into_inner().accepted; + Ok(result) }); } - tokio::spawn(async move { responses.join_all().await }); - Ok(()) + + let mut counter = 0; + while let Some(result) = responses.join_next().await { + match result { + Ok(Ok(true)) => return Ok(true), + Ok(Ok(false)) => { + counter += 1; + continue + }, + errors => { + tracing::error!("Error in broadcast try_recovery: {:?}", errors); + continue + }, + } + } + + if counter > (self.members.len() / 2) { + Ok(false) + }else{ + Err(SyneviError::UnrecoverableTransaction) + } } } diff --git a/synevi_network/src/protos/configure_transport.proto b/synevi_network/src/protos/configure_transport.proto index 34307c5..f713d5b 100644 --- a/synevi_network/src/protos/configure_transport.proto +++ b/synevi_network/src/protos/configure_transport.proto @@ -22,28 +22,6 @@ message GetTimeResponse { // Skew ~ LocalTimestamp - (Timestamp + RoundTripLatency / 2) -> Between Node Get and Node Respond -service BroadcastService { - rpc RestartNode(RestartNodeRequest) returns (RestartNodeResponse) {} - rpc CustomBroadcast(CustomBroadcastRequest) returns (CustomBroadcastResponse) {} -} - -message RestartNodeRequest { - bytes last_applied = 1; // if new T0::default, else last_applied - bytes last_applied_hash = 2; -} - -message RestartNodeResponse { - bytes event = 1; // encoded events -} - -message CustomBroadcastRequest { - bytes message = 1; -} - -message CustomBroadcastResponse {} - - - service ReconfigurationService { rpc JoinElectorate(JoinElectorateRequest) returns (JoinElectorateResponse) {} rpc GetEvents(GetEventRequest) returns (stream GetEventResponse) {} diff --git a/synevi_network/src/protos/consensus_transport.proto b/synevi_network/src/protos/consensus_transport.proto index 157a761..08356ae 100644 --- a/synevi_network/src/protos/consensus_transport.proto +++ b/synevi_network/src/protos/consensus_transport.proto @@ -8,7 +8,7 @@ service ConsensusTransport { rpc Accept(AcceptRequest) returns (AcceptResponse) {} rpc Apply(ApplyRequest) returns (ApplyResponse) {} rpc Recover(RecoverRequest) returns (RecoverResponse) {} - rpc TryRecover(TryRecoverRequest) returns (TryRecoverResponse) {} + rpc TryRecovery(TryRecoveryRequest) returns (TryRecoveryResponse) {} } message PreAcceptRequest { @@ -88,8 +88,10 @@ message RecoverResponse { bytes nack = 6; } -message TryRecoverRequest { +message TryRecoveryRequest { bytes timestamp_zero = 1; } -message TryRecoverResponse {} +message TryRecoveryResponse { + bool accepted = 1; // True if recovery can be initiated else false +} diff --git a/synevi_network/src/replica.rs b/synevi_network/src/replica.rs index 3f81977..4103582 100644 --- a/synevi_network/src/replica.rs +++ b/synevi_network/src/replica.rs @@ -48,9 +48,9 @@ pub trait Replica: Send + Sync { async fn try_recover( &self, - request: TryRecoverRequest, + request: TryRecoveryRequest, ready: bool, - ) -> Result; + ) -> Result; fn is_ready(&self) -> bool; } @@ -196,7 +196,7 @@ where )) } - async fn try_recover(&self, request: Request) -> Result, Status> { + async fn try_recovery(&self, request: Request) -> Result, Status> { Ok(Response::new( self.inner .try_recover(request.into_inner(), self.inner.is_ready()) diff --git a/synevi_types/src/error.rs b/synevi_types/src/error.rs index 8d9e20a..8112c3a 100644 --- a/synevi_types/src/error.rs +++ b/synevi_types/src/error.rs @@ -68,6 +68,9 @@ pub enum SyneviError { NotReady, #[error("Mismatched hashes")] MismatchedHashes, + + #[error("Unrecoverable transaction")] + UnrecoverableTransaction, } impl Serialize for SyneviError { From c23310c82cb66d1a843d9799d500335f0eefef48 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 14 Oct 2024 15:29:29 +0200 Subject: [PATCH 04/40] feat: Added/Update latest_time handling --- synevi_core/src/lib.rs | 4 +- synevi_persistence/src/lib.rs | 2 +- .../src/{database.rs => lmdb_store.rs} | 62 +++++++++---------- synevi_persistence/src/mem_store.rs | 50 ++++++++------- synevi_types/src/traits.rs | 4 ++ 5 files changed, 66 insertions(+), 56 deletions(-) rename synevi_persistence/src/{database.rs => lmdb_store.rs} (94%) diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 6afd80c..23293c2 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -43,8 +43,8 @@ pub mod tests { } async fn broadcast_recovery( &self, _t0: T0 - ) -> Result<(), SyneviError> { - Ok(()) + ) -> Result { + Ok(true) } } diff --git a/synevi_persistence/src/lib.rs b/synevi_persistence/src/lib.rs index 3800e03..896d3a7 100644 --- a/synevi_persistence/src/lib.rs +++ b/synevi_persistence/src/lib.rs @@ -1,2 +1,2 @@ -pub mod database; +pub mod lmdb_store; pub mod mem_store; diff --git a/synevi_persistence/src/database.rs b/synevi_persistence/src/lmdb_store.rs similarity index 94% rename from synevi_persistence/src/database.rs rename to synevi_persistence/src/lmdb_store.rs index 6939401..9f59cb8 100644 --- a/synevi_persistence/src/database.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -28,7 +28,7 @@ struct MutableData { db: Env, pub(crate) mappings: BTreeMap, // Key: t, value t0 pub last_applied: T, // t of last applied entry - pub(crate) latest_t0: T0, // last created or recognized t0 + pub(crate) latest_time: MonoTime, // last created or recognized t0 pub node_serial: u16, latest_hash: [u8; 32], } @@ -59,7 +59,7 @@ impl PersistentStore { let mut mappings = BTreeMap::default(); let mut last_applied = T::default(); - let mut latest_t0 = T0::default(); + let mut latest_time = MonoTime::default(); let mut latest_hash: [u8; 32] = [0; 32]; for event in result { mappings.insert(event.t, event.t_zero); @@ -71,8 +71,8 @@ impl PersistentStore { return Err(SyneviError::MissingTransactionHash); }; } - if event.t_zero > latest_t0 { - latest_t0 = event.t_zero; + if *event.t > latest_time { + latest_time = *event.t; } } write_txn.commit()?; @@ -82,7 +82,7 @@ impl PersistentStore { db: env_clone, mappings, last_applied, - latest_t0, + latest_time, node_serial, latest_hash, }), @@ -96,7 +96,7 @@ impl PersistentStore { db: env_clone, mappings: BTreeMap::default(), last_applied: T::default(), - latest_t0: T0::default(), + latest_time: MonoTime::default(), node_serial, latest_hash: [0; 32], }), @@ -213,13 +213,19 @@ impl Store for PersistentStore { async fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { self.data.lock().await.last_applied_hash().await } + + async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { + let mut lock = self.data.lock().await; + lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); + Ok(()) + } } impl MutableData { #[instrument(level = "trace")] async fn init_t_zero(&mut self, node_serial: u16) -> T0 { - let t0 = T0(self.latest_t0.next_with_node(node_serial).into_time()); - self.latest_t0 = t0; - t0 + let next_time = self.latest_time.next_with_node(node_serial).into_time(); + self.latest_time = next_time; + T0(next_time) } #[instrument(level = "trace")] @@ -230,18 +236,17 @@ impl MutableData { transaction: Vec, ) -> Result<(T, HashSet), SyneviError> { let (t, deps) = { - let t = T(if let Some((last_t, _)) = self.mappings.last_key_value() { - if **last_t > *t_zero { - t_zero - .next_with_guard_and_node(last_t, self.node_serial) - .into_time() - } else { - *t_zero - } - } else { - // No entries in the map -> insert the new event - *t_zero - }); + let t = if self.latest_time > *t_zero { + let new_time_t = t_zero + .next_with_guard_and_node(&self.latest_time, self.node_serial) + .into_time(); + + self.latest_time = new_time_t; + T(new_time_t) + }else{ + T(*t_zero) + }; + // This might not be necessary to re-use the write lock here let deps = self.get_tx_dependencies(&t, &t_zero).await; (t, deps) }; @@ -307,6 +312,11 @@ impl MutableData { async fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { //let db = self.db.clone(); + // Update the latest time + if self.latest_time < *upsert_event.t { + self.latest_time = *upsert_event.t; + } + let mut write_txn = self.db.write_txn()?; let events_db: EventDb = self .db @@ -317,11 +327,6 @@ impl MutableData { let Some(mut event) = event else { let mut event = Event::from(upsert_event.clone()); - // Update the latest t0 - if self.latest_t0 < event.t_zero { - self.latest_t0 = event.t_zero; - } - if matches!(event.state, State::Applied) { self.mappings.insert(event.t, event.t_zero); if let Some(deps) = upsert_event.dependencies { @@ -357,11 +362,6 @@ impl MutableData { return Ok(()); }; - // Update the latest t0 - if self.latest_t0 < event.t_zero { - self.latest_t0 = event.t_zero; - } - // Do not update to a "lower" state if upsert_event.state < event.state { write_txn.commit()?; diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 6cc31d0..982be67 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -1,4 +1,5 @@ use ahash::RandomState; +use monotime::MonoTime; use std::collections::{BTreeMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; @@ -17,7 +18,7 @@ pub struct InternalStore { pub events: BTreeMap, // Key: t0, value: Event pub(crate) mappings: BTreeMap, // Key: t, value t0 pub last_applied: T, // t of last applied entry - pub(crate) latest_t0: T0, // last created or recognized t0 + pub(crate) latest_time: MonoTime, // last created or recognized time pub node_serial: u16, latest_hash: [u8; 32], } @@ -34,7 +35,7 @@ impl MemStore { events: BTreeMap::default(), mappings: BTreeMap::default(), last_applied: T::default(), - latest_t0: T0::default(), + latest_time: MonoTime::default(), node_serial, latest_hash: [0; 32], })); @@ -156,14 +157,20 @@ impl Store for MemStore { .ok_or_else(|| SyneviError::MissingExecutionHash)?; Ok((last, hash.execution_hash)) } + + async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { + let mut lock = self.store.lock().await; + lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); + Ok(()) + } } impl InternalStore { #[instrument(level = "trace")] fn init_t_zero(&mut self, node_serial: u16) -> T0 { - let t0 = T0(self.latest_t0.next_with_node(node_serial).into_time()); - self.latest_t0 = t0; - t0 + let next_time = self.latest_time.next_with_node(node_serial).into_time(); + self.latest_time = next_time; + T0(next_time) } #[instrument(level = "trace")] @@ -174,18 +181,16 @@ impl InternalStore { transaction: Vec, ) -> Result<(T, HashSet), SyneviError> { let (t, deps) = { - let t = T(if let Some((last_t, _)) = self.mappings.last_key_value() { - if **last_t > *t_zero { - t_zero - .next_with_guard_and_node(last_t, self.node_serial) - .into_time() - } else { - *t_zero - } - } else { - // No entries in the map -> insert the new event - *t_zero - }); + let t = if self.latest_time > *t_zero { + let new_time_t = t_zero + .next_with_guard_and_node(&self.latest_time, self.node_serial) + .into_time(); + + self.latest_time = new_time_t; + T(new_time_t) + }else{ + T(*t_zero) + }; // This might not be necessary to re-use the write lock here let deps = self.get_tx_dependencies(&t, &t_zero); (t, deps) @@ -238,6 +243,12 @@ impl InternalStore { #[instrument(level = "trace")] fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + + // Update the latest time + if self.latest_time < *upsert_event.t { + self.latest_time = *upsert_event.t; + } + let Some(event) = self.events.get_mut(&upsert_event.t_zero) else { let mut event = Event::from(upsert_event.clone()); if matches!(event.state, State::Applied) { @@ -270,11 +281,6 @@ impl InternalStore { return Ok(()); }; - // Update the latest t0 - if self.latest_t0 < event.t_zero { - self.latest_t0 = event.t_zero; - } - // Do not update to a "lower" state if upsert_event.state < event.state { return Ok(()); diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index d0643a9..9f241c9 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -110,4 +110,8 @@ pub trait Store: Send + Sync + Sized + 'static { t_zero: T0, execution_hash: [u8; 32], ) -> Result; + + // Increases the max time to be above the specified guard + // Ensures that the guards t0 will not get a fast path afterwards + async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError>; } From 2460650ce43967bdd7b330131cf4bf7d05b257ce Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 14 Oct 2024 16:03:12 +0200 Subject: [PATCH 05/40] feat: Added wait_handler forget feature --- docs/waithandler_refactor.md | 0 synevi_core/src/replica.rs | 6 +++++- synevi_core/src/wait_handler.rs | 33 +++++++++++++++++++++++++++++++-- tests/consensus_e2e.rs | 2 +- tests/maelstrom/network.rs | 2 +- 5 files changed, 38 insertions(+), 5 deletions(-) create mode 100644 docs/waithandler_refactor.md diff --git a/docs/waithandler_refactor.md b/docs/waithandler_refactor.md new file mode 100644 index 0000000..e69de29 diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index c606167..c43c3ab 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -422,8 +422,10 @@ where request: TryRecoveryRequest, ready: bool, ) -> Result { + + let t0 = T0::try_from(request.timestamp_zero.as_slice())?; + if ready { - let t0 = T0::try_from(request.timestamp_zero.as_slice())?; if let Some(recover_event) = self .node .event_store @@ -437,6 +439,8 @@ where } } + // This ensures that this t0 will not get a fast path in the future + self.node.event_store.inc_time_with_guard(t0).await?; Ok(TryRecoveryResponse { accepted: false, }) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index c0f6c0f..f208a8c 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -291,8 +291,37 @@ where }); } else { let interface = self.node.network.get_interface().await; - interface.broadcast_recovery(t0_recover).await?; - todo!() + match interface.broadcast_recovery(t0_recover).await { + Ok(true) => (), + Ok(false) => { + + // Remove from + // SAFETY: No majority has witnessed this Tx and no fast path is possible + // -> Can be removed from waiter_state because a recovery will enforce a slow path / new t + let mut to_apply = BTreeMap::new(); + waiter_state.remove_from_waiter_apply(&t0_recover, &mut to_apply); + while let Some(mut apply) = to_apply.pop_first() { + apply.1.action = WaitAction::ApplyAfter; + if let Err(e) = self.upsert_event(&apply.1).await { + tracing::error!("Error upserting event: {:?}", e); + println!("Error upserting event: {:?}", e); + continue; + }; + waiter_state.applied.insert(apply.1.t_zero); + if let Some(notify) = apply.1.notify.take() { + let _ = notify.send(()); + } + waiter_state + .remove_from_waiter_apply(&apply.1.t_zero, &mut to_apply); + } + + } + Err(err) => { + tracing::error!("Error broadcasting recovery: {:?}", err); + // TODO: Graceful restart ? + panic!("Error broadcasting recovery: {:?}", err); + } + } }; Ok(()) } diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index 7040f68..3aed41b 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -10,7 +10,7 @@ mod tests { use synevi_core::node::Node; use synevi_core::tests::DummyExecutor; use synevi_network::network::GrpcNetwork; - use synevi_persistence::database::PersistentStore; + use synevi_persistence::lmdb_store::PersistentStore; use synevi_persistence::mem_store::MemStore; use tokio::fs; use tokio::runtime::Builder; diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index 85e2574..49affd1 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -418,7 +418,7 @@ impl NetworkInterface for MaelstromNetwork { Ok(result) } - async fn broadcast_recovery(&self, _t0: T0) -> Result<(), SyneviError> { + async fn broadcast_recovery(&self, _t0: T0) -> Result { todo!() } } From 52b75ef7873803ff46c391859e698cc095376a18 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Tue, 15 Oct 2024 09:01:58 +0200 Subject: [PATCH 06/40] feat: Draft for new wait handler --- synevi_core/src/replica.rs | 3 ++ synevi_persistence/src/wait_handler.rs | 41 ++++++++++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 synevi_persistence/src/wait_handler.rs diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index c43c3ab..72e6afd 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -274,6 +274,8 @@ where let deps = from_dependency(request.dependencies)?; let (sx, rx) = tokio::sync::oneshot::channel(); + // self.node.event_store.get_waiter(&t_zero, APPLY).await?; + self.node .get_wait_handler() .await? @@ -291,6 +293,7 @@ where rx.await .map_err(|_| SyneviError::ReceiveError("Wait receiver closed".to_string()))?; + // TODO: Refactor in execute function let result = match transaction { TransactionPayload::None => { return Err(SyneviError::TransactionNotFound); diff --git a/synevi_persistence/src/wait_handler.rs b/synevi_persistence/src/wait_handler.rs new file mode 100644 index 0000000..79b0dc8 --- /dev/null +++ b/synevi_persistence/src/wait_handler.rs @@ -0,0 +1,41 @@ + + +// Tx1 = dep[Tx0] + +// -> Tx0 commit +// -> for each waiter: is tx0 in deps? +// -> if yes! -> is t(tx0) > t(tx1) +// -> y -> do nothing +// -> n -> increase dep_state +1 +// -> if dep_state == dep.len() -> send signal to waiter +// + + + +pub struct Waiter { + waited_since: Instant, + dependency_states: u64, + sender: Vec>, +} + +//loop { +// if waiter.waited_since > 10s -> Find inital tx everyone is waiting for -> +// +//} + + + + +pub struct WaitHandler { + waiters: HashMap, +} + + + +impl WaitHandler { + + pub fn run() { + + for + } +} \ No newline at end of file From 7e009dfb2945231ffe20c3b95fa685500bcd65ea Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Tue, 15 Oct 2024 12:38:47 +0200 Subject: [PATCH 07/40] feat: Waithandler refactor boilerplate traits for Store --- synevi_core/src/coordinator.rs | 44 +++++------------- synevi_core/src/lib.rs | 2 +- synevi_core/src/node.rs | 27 +---------- synevi_core/src/replica.rs | 62 +++++++++++--------------- synevi_persistence/src/lib.rs | 1 + synevi_persistence/src/lmdb_store.rs | 16 ++++++- synevi_persistence/src/mem_store.rs | 16 ++++++- synevi_persistence/src/wait_handler.rs | 48 ++++++++++---------- synevi_types/src/traits.rs | 4 +- 9 files changed, 98 insertions(+), 122 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index e521525..52a0a33 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -1,6 +1,6 @@ use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; -use crate::wait_handler::WaitAction; +//use crate::wait_handler::WaitAction; use ahash::RandomState; use serde::Serialize; use sha3::{Digest, Sha3_256}; @@ -15,7 +15,8 @@ use synevi_network::network::{BroadcastRequest, Network, NetworkInterface}; use synevi_network::utils::IntoInner; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalExecution, RecoverEvent, RecoveryState, SyneviResult, TransactionPayload + ExecutorResult, Hashes, InternalExecution, RecoverEvent, RecoveryState, SyneviResult, + TransactionPayload, }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; use tracing::{instrument, trace}; @@ -256,22 +257,10 @@ where async fn commit_consensus(&mut self) -> Result<(), SyneviError> { self.transaction.state = State::Commited; - let (sx, rx) = tokio::sync::oneshot::channel(); - self.node - .wait_handler - .read() - .await - .as_ref() - .ok_or_else(|| SyneviError::MissingWaitHandler)? - .send_msg( - self.transaction.t_zero, - self.transaction.t, - self.transaction.dependencies.clone(), - self.transaction.get_transaction_bytes(), - WaitAction::CommitBefore, - sx, - self.transaction.id, - ) + let rx = self + .node + .event_store + .waiter_commit((&self.transaction).into()) .await?; let _ = rx.await; @@ -305,19 +294,10 @@ where #[instrument(level = "trace", skip(self))] async fn execute_consensus(&mut self) -> Result<(SyneviResult, Hashes), SyneviError> { self.transaction.state = State::Applied; - let (sx, rx) = tokio::sync::oneshot::channel(); - self.node - .get_wait_handler() - .await? - .send_msg( - self.transaction.t_zero, - self.transaction.t, - self.transaction.dependencies.clone(), - self.transaction.get_transaction_bytes(), - WaitAction::ApplyAfter, - sx, - self.transaction.id, - ) + let rx = self + .node + .event_store + .waiter_apply((&self.transaction).into()) .await?; rx.await @@ -363,7 +343,7 @@ where } #[instrument(level = "trace", skip(node))] - pub async fn recover(node: Arc>, recover_event: RecoverEvent) -> SyneviResult { + pub async fn recover(node: Arc>, recover_event: RecoverEvent) -> SyneviResult { loop { let node = node.clone(); diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 23293c2..3764510 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -3,7 +3,7 @@ pub mod node; pub mod reorder_buffer; pub mod replica; pub mod utils; -mod wait_handler; +//mod wait_handler; pub mod tests { use std::sync::Arc; diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 59548f7..447ae7c 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -1,6 +1,5 @@ use crate::coordinator::Coordinator; use crate::replica::ReplicaConfig; -use crate::wait_handler::WaitHandler; use std::fmt::Debug; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{atomic::AtomicU64, Arc}; @@ -15,7 +14,6 @@ use synevi_types::traits::Store; use synevi_types::types::{SyneviResult, TransactionPayload}; use synevi_types::{Executor, State, SyneviError, T}; use tokio::sync::mpsc::Receiver; -use tokio::sync::RwLock; use tokio::task::JoinSet; use tracing::instrument; use ulid::Ulid; @@ -43,7 +41,7 @@ where pub executor: E, pub event_store: Arc, pub stats: Stats, - pub wait_handler: RwLock>>>, + //pub wait_handler: RwLock>>>, semaphore: Arc, has_members: AtomicBool, is_ready: Arc, @@ -101,17 +99,10 @@ where stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, - wait_handler: RwLock::new(None), has_members: AtomicBool::new(false), is_ready: Arc::new(AtomicBool::new(true)), }); - let wait_handler = WaitHandler::new(node.clone()); - let wait_handler_clone = wait_handler.clone(); - tokio::spawn(async move { - wait_handler_clone.run().await.unwrap(); - }); - *node.wait_handler.write().await = Some(wait_handler); let ready = Arc::new(AtomicBool::new(true)); let (replica, _) = ReplicaConfig::new(node.clone(), ready); @@ -150,17 +141,10 @@ where stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, - wait_handler: RwLock::new(None), has_members: AtomicBool::new(false), is_ready: ready.clone(), }); - let wait_handler = WaitHandler::new(node.clone()); - let wait_handler_clone = wait_handler.clone(); - tokio::spawn(async move { - wait_handler_clone.run().await.unwrap(); - }); - *node.wait_handler.write().await = Some(wait_handler); let (replica, config_receiver) = ReplicaConfig::new(node.clone(), ready.clone()); node.network.spawn_server(replica.clone()).await?; @@ -219,15 +203,6 @@ where coordinator.run().await } - pub async fn get_wait_handler(&self) -> Result>, SyneviError> { - let lock = self.wait_handler.read().await; - let handler = lock - .as_ref() - .ok_or_else(|| SyneviError::MissingWaitHandler)? - .clone(); - Ok(handler) - } - pub fn get_stats(&self) -> (u64, u64, u64) { ( self.stats diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 72e6afd..eaffedc 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -1,7 +1,7 @@ use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; -use crate::wait_handler::WaitAction; +//use crate::wait_handler::WaitAction; use sha3::{Digest, Sha3_256}; use std::collections::BTreeMap; use std::sync::atomic::{AtomicBool, Ordering}; @@ -232,19 +232,19 @@ where trace!(?request_id, "Replica: Commit"); let deps = from_dependency(request.dependencies)?; - let (sx, rx) = tokio::sync::oneshot::channel(); - self.node - .get_wait_handler() - .await? - .send_msg( + let rx = self + .node + .event_store + .waiter_commit(UpsertEvent { + id: request_id, t_zero, t, - deps, - request.event, - WaitAction::CommitBefore, - sx, - request_id, - ) + state: State::Commited, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }) .await?; let _ = rx.await; Ok(CommitResponse {}) @@ -269,25 +269,22 @@ where trace!(?request_id, "Replica: Apply"); let transaction: TransactionPayload<::Tx> = - TransactionPayload::from_bytes(request.event)?; - - let deps = from_dependency(request.dependencies)?; - let (sx, rx) = tokio::sync::oneshot::channel(); - - // self.node.event_store.get_waiter(&t_zero, APPLY).await?; + TransactionPayload::from_bytes(request.event.clone())?; - self.node - .get_wait_handler() - .await? - .send_msg( + let deps = from_dependency(request.dependencies.clone())?; + let rx = self + .node + .event_store + .waiter_apply(UpsertEvent { + id: request_id, t_zero, t, - deps.clone(), - transaction.as_bytes(), - WaitAction::ApplyAfter, - sx, - request_id, - ) + state: State::Applied, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }) .await?; rx.await @@ -425,7 +422,6 @@ where request: TryRecoveryRequest, ready: bool, ) -> Result { - let t0 = T0::try_from(request.timestamp_zero.as_slice())?; if ready { @@ -436,17 +432,13 @@ where .await? { tokio::spawn(Coordinator::recover(self.node.clone(), recover_event)); - return Ok(TryRecoveryResponse { - accepted: true, - }) + return Ok(TryRecoveryResponse { accepted: true }); } } // This ensures that this t0 will not get a fast path in the future self.node.event_store.inc_time_with_guard(t0).await?; - Ok(TryRecoveryResponse { - accepted: false, - }) + Ok(TryRecoveryResponse { accepted: false }) } } diff --git a/synevi_persistence/src/lib.rs b/synevi_persistence/src/lib.rs index 896d3a7..af9ba17 100644 --- a/synevi_persistence/src/lib.rs +++ b/synevi_persistence/src/lib.rs @@ -1,2 +1,3 @@ pub mod lmdb_store; pub mod mem_store; +pub mod wait_handler; diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 9f59cb8..92f3385 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -12,7 +12,7 @@ use synevi_types::{ types::{Event, Hashes, RecoverDependencies, RecoverEvent, UpsertEvent}, Ballot, State, T, T0, }; -use tokio::sync::{mpsc::Receiver, Mutex}; +use tokio::sync::{mpsc::Receiver, oneshot, Mutex}; use tracing::instrument; const EVENT_DB_NAME: &str = "events"; @@ -219,6 +219,20 @@ impl Store for PersistentStore { lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); Ok(()) } + + async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { + let (sdx, rcv) = oneshot::channel(); + todo!(); + + Ok(rcv) + } + + async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { + let (sdx, rcv) = oneshot::channel(); + todo!(); + + Ok(rcv) + } } impl MutableData { #[instrument(level = "trace")] diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 982be67..202cbdf 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -10,7 +10,7 @@ use synevi_types::types::{Event, Hashes, RecoverDependencies, UpsertEvent}; use synevi_types::State; use synevi_types::{Ballot, T, T0}; use tokio::sync::mpsc::{Receiver, Sender}; -use tokio::sync::Mutex; +use tokio::sync::{oneshot, Mutex}; use tracing::instrument; #[derive(Debug)] @@ -163,6 +163,20 @@ impl Store for MemStore { lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); Ok(()) } + + async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { + let (sdx, rcv) = oneshot::channel(); + todo!(); + + Ok(rcv) + } + + async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { + let (sdx, rcv) = oneshot::channel(); + todo!(); + + Ok(rcv) + } } impl InternalStore { diff --git a/synevi_persistence/src/wait_handler.rs b/synevi_persistence/src/wait_handler.rs index 79b0dc8..53a5d83 100644 --- a/synevi_persistence/src/wait_handler.rs +++ b/synevi_persistence/src/wait_handler.rs @@ -1,16 +1,7 @@ - - -// Tx1 = dep[Tx0] - -// -> Tx0 commit -// -> for each waiter: is tx0 in deps? -// -> if yes! -> is t(tx0) > t(tx1) -// -> y -> do nothing -// -> n -> increase dep_state +1 -// -> if dep_state == dep.len() -> send signal to waiter -// - - +use std::collections::HashMap; +use ahash::RandomState; +use synevi_types::T0; +use tokio::{sync::oneshot, time::Instant}; pub struct Waiter { waited_since: Instant, @@ -18,24 +9,31 @@ pub struct Waiter { sender: Vec>, } -//loop { -// if waiter.waited_since > 10s -> Find inital tx everyone is waiting for -> -// -//} - - - - pub struct WaitHandler { - waiters: HashMap, + waiters: HashMap, } - impl WaitHandler { pub fn run() { - for + loop {} } -} \ No newline at end of file +} + +// Tx1 = dep[Tx0] + +// -> Tx0 commit +// -> for each waiter: is tx0 in deps? +// -> if yes! -> is t(tx0) > t(tx1) +// -> y -> do nothing +// -> n -> increase dep_state +1 +// -> if dep_state == dep.len() -> send signal to waiter +// +// +//loop { +// if waiter.waited_since > 10s -> Find inital tx everyone is waiting for -> +// +//} + diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index 9f241c9..c6079d1 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -4,7 +4,7 @@ use std::{ collections::{BTreeMap, HashSet}, sync::{Arc, Weak}, }; -use tokio::sync::mpsc::Receiver; +use tokio::sync::{mpsc::Receiver, oneshot}; use crate::{ types::{Event, Hashes, RecoverDependencies, RecoverEvent, SyneviResult, UpsertEvent}, @@ -114,4 +114,6 @@ pub trait Store: Send + Sync + Sized + 'static { // Increases the max time to be above the specified guard // Ensures that the guards t0 will not get a fast path afterwards async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError>; + async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError>; + async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError>; } From 71d44c579a1cdcd9a72b0ea9a73541e3fcd162c3 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Tue, 15 Oct 2024 14:44:23 +0200 Subject: [PATCH 08/40] feat: Refactor waithandler --- synevi_core/src/coordinator.rs | 44 +- synevi_core/src/lib.rs | 6 +- synevi_core/src/node.rs | 92 ++-- synevi_core/src/reorder_buffer.rs | 14 +- synevi_core/src/replica.rs | 90 ++-- synevi_core/src/wait_handler.rs | 667 +++---------------------- synevi_kv/src/kv_store.rs | 5 +- synevi_network/src/latency_service.rs | 10 +- synevi_network/src/network.rs | 15 +- synevi_network/src/reconfiguration.rs | 10 +- synevi_network/src/replica.rs | 5 +- synevi_persistence/src/lib.rs | 1 - synevi_persistence/src/lmdb_store.rs | 178 ++++--- synevi_persistence/src/mem_store.rs | 140 +++--- synevi_persistence/src/wait_handler.rs | 39 -- synevi_types/src/traits.rs | 43 +- synevi_types/src/types.rs | 13 +- tests/consensus_e2e.rs | 53 +- 18 files changed, 419 insertions(+), 1006 deletions(-) delete mode 100644 synevi_persistence/src/wait_handler.rs diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 52a0a33..16100b3 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -1,6 +1,5 @@ use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; -//use crate::wait_handler::WaitAction; use ahash::RandomState; use serde::Serialize; use sha3::{Digest, Sha3_256}; @@ -67,7 +66,7 @@ where id: u128, ) -> Self { trace!(?id, "Coordinator: New"); - let t0 = node.event_store.init_t_zero(node.info.serial).await; + let t0 = node.event_store.init_t_zero(node.info.serial); Coordinator { node, transaction: TransactionStateMachine { @@ -97,7 +96,7 @@ where .fetch_add(1, Ordering::Relaxed); let last_applied = { - let (t, _) = self.node.event_store.last_applied().await; + let (t, _) = self.node.event_store.last_applied(); t.into() }; @@ -140,7 +139,7 @@ where responses: &[PreAcceptResponse], ) -> Result<(), SyneviError> { // Collect deps by t_zero and only keep the max t - let (_, last_applied_t0) = self.node.event_store.last_applied().await; + let (_, last_applied_t0) = self.node.event_store.last_applied(); if last_applied_t0 != T0::default() { self.transaction.dependencies.insert(last_applied_t0); } @@ -157,8 +156,7 @@ where // Upsert store self.node .event_store - .upsert_tx((&self.transaction).into()) - .await?; + .upsert_tx((&self.transaction).into())?; Ok(()) } @@ -176,7 +174,7 @@ where .total_accepts .fetch_add(1, Ordering::Relaxed); let last_applied = { - let (t, _) = self.node.event_store.last_applied().await; + let (t, _) = self.node.event_store.last_applied(); t.into() }; let accepted_request = AcceptRequest { @@ -224,8 +222,7 @@ where self.transaction.state = State::Accepted; self.node .event_store - .upsert_tx((&self.transaction).into()) - .await?; + .upsert_tx((&self.transaction).into())?; Ok(()) } @@ -256,14 +253,7 @@ where #[instrument(level = "trace", skip(self))] async fn commit_consensus(&mut self) -> Result<(), SyneviError> { self.transaction.state = State::Commited; - - let rx = self - .node - .event_store - .waiter_commit((&self.transaction).into()) - .await?; - let _ = rx.await; - + self.node.commit((&self.transaction).into()).await?; Ok(()) } @@ -294,14 +284,8 @@ where #[instrument(level = "trace", skip(self))] async fn execute_consensus(&mut self) -> Result<(SyneviResult, Hashes), SyneviError> { self.transaction.state = State::Applied; - let rx = self - .node - .event_store - .waiter_apply((&self.transaction).into()) - .await?; - - rx.await - .map_err(|_| SyneviError::ReceiveError("Wait handle sender closed".to_string()))?; + + self.node.apply((&self.transaction).into()).await?; let result = match &self.transaction.transaction { TransactionPayload::None => Err(SyneviError::TransactionNotFound), @@ -313,7 +297,7 @@ where .node .add_member(*id, *serial, host.clone(), false) .await; - let (t, hash) = self.node.event_store.last_applied_hash().await?; // TODO: Remove ? + let (t, hash) = self.node.event_store.last_applied_hash()?; // TODO: Remove ? self.node .network .report_config(t, hash, host.clone()) @@ -337,8 +321,7 @@ where let hashes = self .node .event_store - .get_and_update_hash(self.transaction.t_zero, hash.into()) - .await?; + .get_and_update_hash(self.transaction.t_zero, hash.into())?; Ok((result, hashes)) } @@ -479,8 +462,7 @@ where if let Some(ballot) = highest_ballot { self.node .event_store - .accept_tx_ballot(&self.transaction.t_zero, ballot) - .await; + .accept_tx_ballot(&self.transaction.t_zero, ballot); return Ok(RecoveryState::CompetingCoordinator); } @@ -554,7 +536,7 @@ pub mod tests { .fetch_add(1, Ordering::Relaxed); let last_applied = { - let (t, _) = self.node.event_store.last_applied().await; + let (t, _) = self.node.event_store.last_applied(); t.into() }; diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 3764510..0193f1b 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -3,7 +3,7 @@ pub mod node; pub mod reorder_buffer; pub mod replica; pub mod utils; -//mod wait_handler; +mod wait_handler; pub mod tests { use std::sync::Arc; @@ -41,9 +41,7 @@ pub mod tests { self.got_requests.lock().await.push(request); Ok(vec![]) } - async fn broadcast_recovery( - &self, _t0: T0 - ) -> Result { + async fn broadcast_recovery(&self, _t0: T0) -> Result { Ok(true) } } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 447ae7c..cfd2efd 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -1,5 +1,6 @@ use crate::coordinator::Coordinator; use crate::replica::ReplicaConfig; +use crate::wait_handler::WaitHandler; use std::fmt::Debug; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{atomic::AtomicU64, Arc}; @@ -11,7 +12,7 @@ use synevi_network::reconfiguration::{BufferedMessage, Report}; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; -use synevi_types::types::{SyneviResult, TransactionPayload}; +use synevi_types::types::{SyneviResult, TransactionPayload, UpsertEvent}; use synevi_types::{Executor, State, SyneviError, T}; use tokio::sync::mpsc::Receiver; use tokio::task::JoinSet; @@ -41,7 +42,7 @@ where pub executor: E, pub event_store: Arc, pub stats: Stats, - //pub wait_handler: RwLock>>>, + pub wait_handler: WaitHandler, semaphore: Arc, has_members: AtomicBool, is_ready: Arc, @@ -92,9 +93,14 @@ where // reorder_clone.run().await.unwrap(); // }); + let arc_store = Arc::new(store); + + let wait_handler = WaitHandler::new(arc_store.clone()); + let node = Arc::new(Node { info: node_name, - event_store: Arc::new(store), + event_store: arc_store, + wait_handler, network, stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), @@ -103,7 +109,6 @@ where is_ready: Arc::new(AtomicBool::new(true)), }); - let ready = Arc::new(AtomicBool::new(true)); let (replica, _) = ReplicaConfig::new(node.clone(), ready); node.network.spawn_server(replica).await?; @@ -133,11 +138,15 @@ where total_recovers: AtomicU64::new(0), }; + let arc_store = Arc::new(store); + let wait_handler = WaitHandler::new(arc_store.clone()); + let ready = Arc::new(AtomicBool::new(false)); let node = Arc::new(Node { info: node_name, - event_store: Arc::new(store), + event_store: arc_store, network, + wait_handler, stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, @@ -145,7 +154,6 @@ where is_ready: ready.clone(), }); - let (replica, config_receiver) = ReplicaConfig::new(node.clone(), ready.clone()); node.network.spawn_server(replica.clone()).await?; node.reconfigure(replica, member_host, config_receiver, ready) @@ -198,8 +206,7 @@ where return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; - let mut coordinator = - Coordinator::new(self.clone(), transaction, id).await; + let mut coordinator = Coordinator::new(self.clone(), transaction, id).await; coordinator.run().await } @@ -221,6 +228,49 @@ where self.info.clone() } + #[instrument(level = "trace", skip(self))] + pub async fn commit(&self, event: UpsertEvent) -> Result<(), SyneviError> { + let t0_commit = event.t_zero.clone(); + let t_commit = event.t.clone(); + + let prev_event = self + .event_store + .get_event(t0_commit)? + .ok_or_else(|| SyneviError::EventNotFound(event.id))?; + + if prev_event.state < State::Commited { + self.event_store.upsert_tx(event)?; + let waiter = self.wait_handler.get_waiter(&t0_commit); + waiter.await.map_err(|e| { + tracing::error!("Error waiting for commit: {:?}", e); + SyneviError::ReceiveError(format!("Error waiting for commit")) + })?; + } + self.wait_handler.commit(&t0_commit, &t_commit); + Ok(()) + } + + #[instrument(level = "trace", skip(self))] + pub async fn apply(&self, event: UpsertEvent) -> Result<(), SyneviError> { + let t0_apply = event.t_zero.clone(); + + let prev_event = self + .event_store + .get_event(t0_apply)? + .ok_or_else(|| SyneviError::EventNotFound(event.id))?; + + if prev_event.state < State::Applied { + let waiter = self.wait_handler.get_waiter(&t0_apply); + waiter.await.map_err(|e| { + tracing::error!("Error waiting for commit: {:?}", e); + SyneviError::ReceiveError(format!("Error waiting for commit")) + })?; + self.event_store.upsert_tx(event)?; + } + self.wait_handler.apply(&t0_apply); + Ok(()) + } + #[instrument(level = "trace", skip(self, replica))] async fn reconfigure( &self, @@ -259,7 +309,7 @@ where } } - let (last_applied, _) = self.event_store.last_applied().await; + let (last_applied, _) = self.event_store.last_applied(); // 2.1 if majority replies with 0 events -> skip to 2.4. self.sync_events(last_applied, self_id, &replica).await?; @@ -424,17 +474,14 @@ mod tests { let _result = coordinator .clone() - .transaction( - 2, - Vec::from("F"), - ) + .transaction(2, Vec::from("F")) .await .unwrap(); - let coord = coordinator.event_store.get_event_store().await; + let coord = coordinator.event_store.get_event_store(); for node in nodes { assert_eq!( - node.event_store.get_event_store().await, + node.event_store.get_event_store(), coord, "Node: {:?}", node.get_info() @@ -487,10 +534,7 @@ mod tests { } match coordinator .clone() - .transaction( - 0, - Vec::from("last transaction"), - ) + .transaction(0, Vec::from("last transaction")) .await .unwrap() { @@ -501,14 +545,12 @@ mod tests { let coordinator_store: BTreeMap = coordinator .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, e.t)) .collect(); assert!(coordinator .event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied)); @@ -517,14 +559,12 @@ mod tests { let node_store: BTreeMap = node .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, e.t)) .collect(); assert!(node .event_store .get_event_store() - .await .clone() .iter() .all(|(_, e)| e.state == State::Applied)); @@ -564,11 +604,7 @@ mod tests { .await .unwrap(); - let result = match node - .transaction(0, vec![127u8]) - .await - .unwrap() - { + let result = match node.transaction(0, vec![127u8]).await.unwrap() { ExecutorResult::External(e) => e.unwrap(), _ => panic!(), }; diff --git a/synevi_core/src/reorder_buffer.rs b/synevi_core/src/reorder_buffer.rs index b843426..519043b 100644 --- a/synevi_core/src/reorder_buffer.rs +++ b/synevi_core/src/reorder_buffer.rs @@ -101,11 +101,8 @@ where if entry.key() <= ¤t_transaction.1 { let (t0_buffer, (notify, event, id)) = entry.remove_entry(); - let (t, deps) = self - .node - .event_store - .pre_accept_tx(id, t0_buffer, event) - .await?; + let (t, deps) = + self.node.event_store.pre_accept_tx(id, t0_buffer, event)?; let _ = notify.send((t, into_dependency(&deps))); } else { break; @@ -122,11 +119,8 @@ where if (current_transaction.0.elapsed().as_micros() as u64) > next_latency { while let Some(entry) = buffer.first_entry() { let (t0_buffer, (notify, event, id)) = entry.remove_entry(); - let (t, deps) = self - .node - .event_store - .pre_accept_tx(id, t0_buffer, event) - .await?; + let (t, deps) = + self.node.event_store.pre_accept_tx(id, t0_buffer, event)?; let _ = notify.send((t, into_dependency(&deps))); } } diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index eaffedc..4bf8b59 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -1,7 +1,6 @@ use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; -//use crate::wait_handler::WaitAction; use sha3::{Digest, Sha3_256}; use std::collections::BTreeMap; use std::sync::atomic::{AtomicBool, Ordering}; @@ -126,7 +125,6 @@ where .node .event_store .accept_tx_ballot(&t0, Ballot::default()) - .await { if ballot != Ballot::default() { return Ok(PreAcceptResponse { @@ -143,8 +141,7 @@ where let (t, deps) = self .node .event_store - .pre_accept_tx(request_id, t0, request.event) - .await?; + .pre_accept_tx(request_id, t0, request.event)?; // self.reorder_buffer // .send_msg(t0, sx, request.event, waiting_time) @@ -180,7 +177,6 @@ where .node .event_store .accept_tx_ballot(&t_zero, request_ballot) - .await { if ballot != request_ballot { return Ok(AcceptResponse { @@ -201,10 +197,9 @@ where dependencies: Some(from_dependency(request.dependencies)?), ballot: Some(request_ballot), execution_hash: None, - }) - .await?; + })?; - self.node.event_store.get_tx_dependencies(&t, &t_zero).await + self.node.event_store.get_tx_dependencies(&t, &t_zero) }; Ok(AcceptResponse { dependencies: into_dependency(&dependencies), @@ -232,21 +227,19 @@ where trace!(?request_id, "Replica: Commit"); let deps = from_dependency(request.dependencies)?; - let rx = self - .node - .event_store - .waiter_commit(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Commited, - transaction: Some(request.event), - dependencies: Some(deps), - ballot: None, - execution_hash: None, - }) - .await?; - let _ = rx.await; + + + + self.node.commit(UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Commited, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }).await?; Ok(CommitResponse {}) } @@ -272,23 +265,17 @@ where TransactionPayload::from_bytes(request.event.clone())?; let deps = from_dependency(request.dependencies.clone())?; - let rx = self - .node - .event_store - .waiter_apply(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Applied, - transaction: Some(request.event), - dependencies: Some(deps), - ballot: None, - execution_hash: None, - }) - .await?; - rx.await - .map_err(|_| SyneviError::ReceiveError("Wait receiver closed".to_string()))?; + self.node.apply(UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Applied, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }).await?; // TODO: Refactor in execute function let result = match transaction { @@ -305,7 +292,7 @@ where .node .add_member(*id, *serial, host.clone(), false) .await; - let (t, hash) = self.node.event_store.last_applied_hash().await?; + let (t, hash) = self.node.event_store.last_applied_hash()?; self.node .network .report_config(t, hash, host.clone()) @@ -336,8 +323,7 @@ where let hashes = self .node .event_store - .get_and_update_hash(t_zero, hash.into()) - .await?; + .get_and_update_hash(t_zero, hash.into())?; if request.transaction_hash != hashes.transaction_hash || request.execution_hash != hashes.execution_hash { @@ -363,7 +349,7 @@ where // TODO/WARNING: This was initially in one Mutex lock //let mut event_store = self.node.event_store.lock().await; - if let Some(state) = self.node.event_store.get_event_state(&t_zero).await { + if let Some(state) = self.node.event_store.get_event_state(&t_zero) { // If another coordinator has started recovery with a higher ballot // Return NACK with the higher ballot number let request_ballot = Ballot::try_from(request.ballot.as_slice())?; @@ -371,7 +357,6 @@ where .node .event_store .accept_tx_ballot(&t_zero, request_ballot) - .await { if request_ballot != ballot { return Ok(RecoverResponse { @@ -384,16 +369,14 @@ where if matches!(state, State::Undefined) { self.node .event_store - .pre_accept_tx(request_id, t_zero, request.event) - .await?; + .pre_accept_tx(request_id, t_zero, request.event)?; }; } else { self.node .event_store - .pre_accept_tx(request_id, t_zero, request.event) - .await?; + .pre_accept_tx(request_id, t_zero, request.event)?; } - let recover_deps = self.node.event_store.get_recover_deps(&t_zero).await?; + let recover_deps = self.node.event_store.get_recover_deps(&t_zero)?; self.node .stats @@ -404,7 +387,6 @@ where .node .event_store .get_event_state(&t_zero) - .await .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner()))?; Ok(RecoverResponse { local_state: local_state.into(), @@ -428,8 +410,7 @@ where if let Some(recover_event) = self .node .event_store - .recover_event(&t0, self.node.get_info().serial) - .await? + .recover_event(&t0, self.node.get_info().serial)? { tokio::spawn(Coordinator::recover(self.node.clone(), recover_event)); return Ok(TryRecoveryResponse { accepted: true }); @@ -437,7 +418,7 @@ where } // This ensures that this t0 will not get a fast path in the future - self.node.event_store.inc_time_with_guard(t0).await?; + self.node.event_store.inc_time_with_guard(t0)?; Ok(TryRecoveryResponse { accepted: false }) } } @@ -499,8 +480,7 @@ where let mut store_rcv = self .node .event_store - .get_events_after(last_applied, event_id) - .await?; + .get_events_after(last_applied, event_id)?; tokio::spawn(async move { while let Some(Ok(event)) = store_rcv.recv().await { let response = { diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index f208a8c..cff0816 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -1,427 +1,78 @@ -use crate::coordinator::Coordinator; -use crate::node::Node; use ahash::RandomState; -use async_channel::{Receiver, Sender}; -use std::collections::BTreeMap; -use std::sync::atomic::AtomicU8; use std::{ - collections::{BTreeSet, HashMap, HashSet}, - sync::Arc, - time::{Duration, Instant}, + collections::HashMap, + sync::{Arc, Mutex}, }; -use synevi_network::network::{Network, NetworkInterface}; -use synevi_types::traits::Store; -use synevi_types::types::UpsertEvent; -use synevi_types::{Executor, State, SyneviError, T, T0}; -use tokio::{sync::oneshot, time::timeout}; +use synevi_types::{traits::Store, T, T0}; +use tokio::{sync::oneshot, time::Instant}; -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] -pub enum WaitAction { - CommitBefore, - ApplyAfter, +pub struct Waiter { + waited_since: Instant, + dependency_states: u64, + sender: Vec>, } -#[derive(Debug)] -pub struct WaitMessage { - id: u128, - t_zero: T0, - t: T, - deps: HashSet, - transaction: Vec, - action: WaitAction, - notify: Option>, -} - -#[derive(Clone)] -pub struct WaitHandler +pub struct WaitHandler where - N: Network + Send + Sync, - E: Executor + Send + Sync, - S: Store + Send + Sync, + S: Store, { - sender: Sender, - receiver: Receiver, - node: Arc>, -} - -#[derive(Debug)] -struct WaitDependency { - wait_message: Option, - deps: HashSet, - started_at: Instant, + waiters: Mutex>, + store: Arc, } -struct WaiterState { - events: HashMap, - committed: HashMap, - applied: HashSet, -} - -static _RECOVERY_CYCLE: AtomicU8 = AtomicU8::new(0); - -impl WaitHandler +impl WaitHandler where - N: Network + Send + Sync, - E: Executor + Send + Sync, - S: Store + Send + Sync, + S: Store, { - pub fn new(node: Arc>) -> Arc { - let (sender, receiver) = async_channel::bounded(1000); - Arc::new(Self { - sender, - receiver, - node, - }) - } - - #[allow(clippy::too_many_arguments)] - pub async fn send_msg( - &self, - t_zero: T0, - t: T, - deps: HashSet, - transaction: Vec, - action: WaitAction, - notify: oneshot::Sender<()>, - id: u128, - ) -> Result<(), SyneviError> { - self.sender - .send(WaitMessage { - id, - t_zero, - t, - deps, - transaction, - action, - notify: Some(notify), - }) - .await - .map_err(|e| SyneviError::SendError(e.to_string())) - } - - pub async fn run(self: Arc) -> Result<(), SyneviError> { - let mut waiter_state = WaiterState::new(); - let mut recovering = BTreeSet::new(); - - loop { - match timeout(Duration::from_millis(50), self.receiver.recv()).await { - Ok(Ok(msg)) => match msg.action { - WaitAction::CommitBefore => { - if let Err(err) = self.commit_action(msg, &mut waiter_state).await { - tracing::error!("Error commit event: {:?}", err); - println!("Error commit event: {:?}", err); - continue; - }; - } - WaitAction::ApplyAfter => { - match &self.node.event_store.get_event(msg.t_zero).await? { - Some(event) if event.state < State::Commited => { - if let Err(err) = self - .commit_action( - WaitMessage { - id: msg.id, - t_zero: msg.t_zero, - t: msg.t, - deps: msg.deps.clone(), - transaction: msg.transaction.clone(), - action: WaitAction::CommitBefore, - notify: None, - }, - &mut waiter_state, - ) - .await - { - tracing::error!( - "Error committing event before apply: {:?}", - err - ); - println!("Error committing event bevore apply: {:?}", err); - continue; - }; - } - None => { - if let Err(err) = self - .commit_action( - WaitMessage { - id: msg.id, - t_zero: msg.t_zero, - t: msg.t, - deps: msg.deps.clone(), - transaction: msg.transaction.clone(), - action: WaitAction::CommitBefore, - notify: None, - }, - &mut waiter_state, - ) - .await - { - tracing::error!( - "Error committing event before apply: {:?}", - err - ); - println!("Error committing event before apply: {:?}", err); - continue; - }; - } - _ => (), - } - - if let Some(mut msg) = waiter_state.insert_apply(msg) { - if let Err(e) = self.upsert_event(&msg).await { - tracing::error!("Error upserting event: {:?}", e); - println!("Error upserting event: {:?}", e); - continue; - }; - if let Some(notify) = msg.notify.take() { - let _ = notify.send(()); - } - waiter_state.applied.insert(msg.t_zero); - let mut to_apply = BTreeMap::new(); - waiter_state.remove_from_waiter_apply(&msg.t_zero, &mut to_apply); - while let Some(mut apply) = to_apply.pop_first() { - apply.1.action = WaitAction::ApplyAfter; - if let Err(e) = self.upsert_event(&apply.1).await { - tracing::error!("Error upserting event: {:?}", e); - println!("Error upserting event: {:?}", e); - continue; - }; - waiter_state.applied.insert(apply.1.t_zero); - if let Some(notify) = apply.1.notify.take() { - let _ = notify.send(()); - } - waiter_state - .remove_from_waiter_apply(&apply.1.t_zero, &mut to_apply); - } - } - } - }, - _ => { - if let Some(t0_recover) = self.check_recovery(&mut waiter_state) { - recovering.insert(t0_recover); - let recover_t0 = recovering.pop_first().unwrap_or(t0_recover); - let wait_handler = self.clone(); - if let Err(err) = wait_handler.recover(recover_t0, &mut waiter_state).await - { - tracing::error!("Error recovering event: {:?}", err); - println!("Error recovering event: {:?}", err); - }; - } - } - } - } - } - - async fn commit_action( - &self, - msg: WaitMessage, - waiter_state: &mut WaiterState, - ) -> Result<(), SyneviError> { - self.upsert_event(&msg).await?; - waiter_state.committed.insert(msg.t_zero, msg.t); - let mut to_apply = waiter_state.remove_from_waiter_commit(&msg.t_zero, &msg.t); - while let Some(mut apply) = to_apply.pop_first() { - apply.1.action = WaitAction::ApplyAfter; - if let Err(e) = self.upsert_event(&apply.1).await { - tracing::error!("Error upserting event: {:?}", e); - println!("Error upserting event: {:?}", e); - continue; - }; - waiter_state.applied.insert(apply.1.t_zero); - if let Some(notify) = apply.1.notify.take() { - let _ = notify.send(()); - } - waiter_state.remove_from_waiter_apply(&apply.1.t_zero, &mut to_apply); - } - waiter_state.insert_commit(msg); - Ok(()) - } - - async fn upsert_event( - &self, - WaitMessage { - id, - t_zero, - t, - action, - deps, - transaction, - .. - }: &WaitMessage, - ) -> Result<(), SyneviError> { - let state = match action { - WaitAction::CommitBefore => State::Commited, - WaitAction::ApplyAfter => State::Applied, - }; - self.node - .event_store - .upsert_tx(UpsertEvent { - id: *id, - t_zero: *t_zero, - t: *t, - state, - transaction: Some(transaction.clone()), - dependencies: Some(deps.clone()), - ..Default::default() - }) - .await?; - - Ok(()) - } - - async fn recover( - self: Arc, - t0_recover: T0, - waiter_state: &mut WaiterState, - ) -> Result<(), SyneviError> { - if let Some(event) = waiter_state.events.get_mut(&t0_recover) { - event.started_at = Instant::now(); - } - if let Some(recover_event) = self - .node - .event_store - .recover_event(&t0_recover, self.node.get_info().serial) - .await? - { - let node = self.node.clone(); - tokio::spawn(async move { - if let Err(err) = Coordinator::recover(node, recover_event).await { - tracing::error!("Error recovering event: {:?}", err); - println!("Error recovering event: {:?}", err); - } - }); - } else { - let interface = self.node.network.get_interface().await; - match interface.broadcast_recovery(t0_recover).await { - Ok(true) => (), - Ok(false) => { - - // Remove from - // SAFETY: No majority has witnessed this Tx and no fast path is possible - // -> Can be removed from waiter_state because a recovery will enforce a slow path / new t - let mut to_apply = BTreeMap::new(); - waiter_state.remove_from_waiter_apply(&t0_recover, &mut to_apply); - while let Some(mut apply) = to_apply.pop_first() { - apply.1.action = WaitAction::ApplyAfter; - if let Err(e) = self.upsert_event(&apply.1).await { - tracing::error!("Error upserting event: {:?}", e); - println!("Error upserting event: {:?}", e); - continue; - }; - waiter_state.applied.insert(apply.1.t_zero); - if let Some(notify) = apply.1.notify.take() { - let _ = notify.send(()); - } - waiter_state - .remove_from_waiter_apply(&apply.1.t_zero, &mut to_apply); - } - - } - Err(err) => { - tracing::error!("Error broadcasting recovery: {:?}", err); - // TODO: Graceful restart ? - panic!("Error broadcasting recovery: {:?}", err); - } - } - }; - Ok(()) - } - - fn check_recovery(&self, waiter_state: &mut WaiterState) -> Option { - for ( - _t0, - WaitDependency { - deps, started_at, .. - }, - ) in waiter_state.events.iter_mut() - { - if started_at.elapsed() > Duration::from_secs(1) { - let sorted_deps: BTreeSet = deps.iter().cloned().collect(); - - let mut min_dep = None; - for t0_dep in sorted_deps { - if let Some(t_dep) = waiter_state.committed.get(&t0_dep) { - // Check if lowest t0 is committed - // If yes -> Recover dep with lowest T - if let Some((t0_min, t_min)) = min_dep.as_mut() { - if t_dep < t_min { - *t0_min = t0_dep; - *t_min = *t_dep; - } - } else { - min_dep = Some((t0_dep, *t_dep)); - } - } else { - // Lowest T0 is not commited -> Recover lowest t0 to ensure commit - // Recover t0_dep - *started_at = Instant::now(); - return Some(t0_dep); - } - } - - // Recover min_dep - if let Some((t0_dep, _)) = min_dep { - *started_at = Instant::now(); - return Some(t0_dep); - } - } - } - None - } -} - -impl WaiterState { - fn new() -> Self { + pub fn new(store: Arc) -> Self { Self { - events: HashMap::default(), - committed: HashMap::default(), - applied: HashSet::default(), + waiters: Mutex::new(HashMap::default()), + store, } } - fn remove_from_waiter_commit(&mut self, t0_dep: &T0, t_dep: &T) -> BTreeMap { - let mut apply_deps = BTreeMap::default(); - self.events.retain(|_, event| { - if let Some(msg) = &mut event.wait_message { - if msg.t_zero == *t0_dep { - return true; - } + pub fn get_waiter(&self, t0: &T0) -> oneshot::Receiver<()> { + let (sdx, rcv) = oneshot::channel(); + let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + let waiter = waiter_lock.entry(*t0).or_insert(Waiter { + waited_since: Instant::now(), + dependency_states: 0, + sender: Vec::new(), + }); + waiter.sender.push(sdx); + rcv + } - if t_dep < &msg.t { - // Cannot remove must wait for apply -> retain - return true; - } - event.deps.remove(t0_dep); - if event.deps.is_empty() { - if msg.action != WaitAction::ApplyAfter { - if let Some(sender) = msg.notify.take() { - let _ = sender.send(()); + pub fn commit(&self, t0_commit: &T0, t_commit: &T) { + let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + waiter_lock.retain(|t0_waiting, waiter| { + let event = self.store.get_event(*t0_waiting).unwrap().unwrap(); // TODO: Remove unwrap + if event.dependencies.contains(t0_commit) { + if t_commit > &event.t { + waiter.dependency_states += 1; + waiter.waited_since = Instant::now(); + if waiter.dependency_states >= event.dependencies.len() as u64 { + for sdx in waiter.sender.drain(..) { + sdx.send(()).unwrap(); // TODO: Remove unwrap } - } else if let Some(msg) = event.wait_message.take() { - apply_deps.insert(msg.t, msg); + return false; } - return false; } } true }); - apply_deps } - fn remove_from_waiter_apply(&mut self, t0_dep: &T0, to_apply: &mut BTreeMap) { - self.events.retain(|_, event| { - event.deps.remove(t0_dep); - for wait_dep in to_apply.iter() { - event.deps.remove(&wait_dep.1.t_zero); - } - - if let Some(msg) = &mut event.wait_message { - if event.deps.is_empty() { - if msg.action != WaitAction::ApplyAfter { - if let Some(sender) = msg.notify.take() { - let _ = sender.send(()); - } - } else if let Some(msg) = event.wait_message.take() { - to_apply.insert(msg.t, msg); + pub fn apply(&self, t0_commit: &T0) { + let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + waiter_lock.retain(|t0_waiting, waiter| { + let event = self.store.get_event(*t0_waiting).unwrap().unwrap(); // TODO: Remove unwrap + if event.dependencies.contains(t0_commit) { + waiter.dependency_states += 1; + waiter.waited_since = Instant::now(); + if waiter.dependency_states >= event.dependencies.len() as u64 { + for sdx in waiter.sender.drain(..) { + sdx.send(()).unwrap(); // TODO: Remove unwrap } return false; } @@ -429,209 +80,19 @@ impl WaiterState { true }); } - - fn insert_commit(&mut self, mut wait_message: WaitMessage) { - if self.applied.contains(&wait_message.t_zero) { - if let Some(sender) = wait_message.notify.take() { - let _ = sender.send(()); - } - return; - } - let mut wait_dep = WaitDependency { - wait_message: Some(wait_message), - deps: HashSet::default(), - started_at: Instant::now(), - }; - if let Some(wait_message) = &mut wait_dep.wait_message { - for dep_t0 in wait_message.deps.iter() { - if !self.applied.contains(dep_t0) { - if let Some(stored_t) = self.committed.get(dep_t0) { - // Your T is lower than the dep commited t -> no wait necessary - if &wait_message.t < stored_t { - continue; - } - } - wait_dep.deps.insert(*dep_t0); - } - } - - if wait_dep.deps.is_empty() { - if let Some(sender) = wait_message.notify.take() { - let _ = sender.send(()); - } - return; - } - - if let Some(existing) = self.events.get_mut(&wait_message.t_zero) { - if let Some(existing_wait_message) = &mut existing.wait_message { - if let Some(sender) = existing_wait_message.notify.take() { - let _ = sender.send(()); - return; - } - } - } - self.events.insert(wait_message.t_zero, wait_dep); - } - } - - fn insert_apply(&mut self, mut wait_message: WaitMessage) -> Option { - if self.applied.contains(&wait_message.t_zero) { - if let Some(sender) = wait_message.notify.take() { - let _ = sender.send(()); - } - return None; - } - let mut wait_dep = WaitDependency { - wait_message: Some(wait_message), - deps: HashSet::default(), - started_at: Instant::now(), - }; - - if let Some(wait_message) = &wait_dep.wait_message { - for dep_t0 in wait_message.deps.iter() { - if !self.applied.contains(dep_t0) { - if let Some(stored_t) = self.committed.get(dep_t0) { - // Your T is lower than the dep commited t -> no wait necessary - if &wait_message.t < stored_t { - continue; - } - } - // if not applied and not comitted with lower t - wait_dep.deps.insert(*dep_t0); - } - } - - if wait_dep.deps.is_empty() { - if let Some(wait_msg) = wait_dep.wait_message.take() { - return Some(wait_msg); - } - } else { - self.events.insert(wait_message.t_zero, wait_dep); - } - } - None - } } -#[cfg(test)] -mod tests { - use monotime::MonoTime; - use ulid::Ulid; - - use crate::{ - tests::{DummyExecutor, NetworkMock}, - wait_handler::*, - }; - - #[tokio::test] - async fn test_wait_handler() { - let (sender, receiver): (Sender, Receiver) = - async_channel::unbounded(); - - let node = Node::new_with_network_and_executor( - Ulid::new(), - 1, - NetworkMock::default(), - DummyExecutor, - ) - .await - .unwrap(); - let wait_handler = WaitHandler { - sender, - receiver, - node, - }; - - let (sx11, rx11) = tokio::sync::oneshot::channel(); - let (sx12, rx12) = tokio::sync::oneshot::channel(); - let (sx21, _rx21) = tokio::sync::oneshot::channel(); - - // let notify_2_1_future = notify_2_1.notified(); - // let notify_2_2_future = notify_2_2.notified(); - - let id_1 = u128::from_be_bytes(Ulid::new().to_bytes()); - let _id_2 = u128::from_be_bytes(Ulid::new().to_bytes()); - let t0_1 = T0(MonoTime::new_with_time(1u128, 0, 0)); - let t0_2 = T0(MonoTime::new_with_time(2u128, 0, 0)); - let t_1 = T(MonoTime::new_with_time(1u128, 0, 0)); - let t_2 = T(MonoTime::new_with_time(2u128, 0, 0)); - let deps_2 = HashSet::from_iter([t0_1]); - wait_handler - .send_msg( - t0_2, - t_2, - deps_2.clone(), - Vec::new(), - WaitAction::CommitBefore, - sx11, - id_1, - ) - .await - .unwrap(); - wait_handler - .send_msg( - t0_1, - t_1, - HashSet::default(), - Vec::new(), - WaitAction::CommitBefore, - sx12, - id_1, - ) - .await - .unwrap(); - - wait_handler - .send_msg( - t0_1, - t_1, - HashSet::default(), - Vec::new(), - WaitAction::ApplyAfter, - sx21, - id_1, - ) - .await - .unwrap(); - // wait_handler - // .send_msg( - // t0_2.clone(), - // t_2.clone(), - // deps_2.clone(), - // Bytes::new(), - // WaitAction::CommitBefore, - // notify_2_1.clone(), - // ) - // .await - // .unwrap(); - // wait_handler - // .send_msg( - // t0_1, - // t_1, - // HashMap::new(), - // Bytes::new(), - // WaitAction::ApplyAfter, - // notify_1_2.clone(), - // ) - // .await - // .unwrap(); - - let wait_handler = Arc::new(wait_handler); - - tokio::spawn(async move { wait_handler.run().await.unwrap() }); - timeout(Duration::from_millis(10), rx11) - .await - .unwrap() - .unwrap(); - timeout(Duration::from_millis(10), rx12) - .await - .unwrap() - .unwrap(); - // timeout(Duration::from_millis(10), notify_2_1_future) - // .await - // .unwrap(); - // timeout(Duration::from_millis(10), notify_2_2_future) - // .await - // .unwrap(); - } -} +// Tx1 = dep[Tx0] + +// -> Tx0 commit +// -> for each waiter: is tx0 in deps? +// -> if yes! -> is t(tx0) > t(tx1) +// -> y -> do nothing +// -> n -> increase dep_state +1 +// -> if dep_state == dep.len() -> send signal to waiter +// +// +//loop { +// if waiter.waited_since > 10s -> Find inital tx everyone is waiting for -> +// +//} diff --git a/synevi_kv/src/kv_store.rs b/synevi_kv/src/kv_store.rs index b51d483..abdf532 100644 --- a/synevi_kv/src/kv_store.rs +++ b/synevi_kv/src/kv_store.rs @@ -117,10 +117,7 @@ where async fn transaction(&self, id: Ulid, transaction: Transaction) -> Result { let node = self.node.clone(); match node - .transaction( - u128::from_be_bytes(id.to_bytes()), - transaction, - ) + .transaction(u128::from_be_bytes(id.to_bytes()), transaction) .await? { ExecutorResult::External(result) => result, diff --git a/synevi_network/src/latency_service.rs b/synevi_network/src/latency_service.rs index 78cff30..e557dfc 100644 --- a/synevi_network/src/latency_service.rs +++ b/synevi_network/src/latency_service.rs @@ -1,6 +1,5 @@ -use std::collections::HashMap; use bytes::{BufMut, BytesMut}; -use ulid::Ulid; +use std::collections::HashMap; use std::{ sync::Arc, time::{self, Duration, Instant}, @@ -8,6 +7,7 @@ use std::{ use synevi_types::error::SyneviError; use tokio::sync::RwLock; use tonic::{Request, Response}; +use ulid::Ulid; use crate::{ configure_transport::{ @@ -18,9 +18,11 @@ use crate::{ const LATENCY_INTERVAL: u64 = 10; -pub async fn get_latency(members: Arc>>) -> Result<(), SyneviError> { +pub async fn get_latency( + members: Arc>>, +) -> Result<(), SyneviError> { loop { - for (_ ,member) in members.read().await.iter() { + for (_, member) in members.read().await.iter() { let mut client = TimeServiceClient::new(member.member.channel.clone()); let time = time::SystemTime::now() .duration_since(time::UNIX_EPOCH) diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index b536759..c4675e7 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -344,7 +344,6 @@ impl Network for GrpcNetwork { Ok((response.majority, response.self_event)) } - async fn report_config( &self, last_applied: T, @@ -624,7 +623,9 @@ impl NetworkInterface for GrpcNetworkSet { async fn broadcast_recovery(&self, t0: T0) -> Result { let mut responses: JoinSet> = JoinSet::new(); - let inner_request = TryRecoveryRequest { timestamp_zero: t0.into() }; + let inner_request = TryRecoveryRequest { + timestamp_zero: t0.into(), + }; for replica in &self.members { let channel = replica.channel.clone(); let request = tonic::Request::new(inner_request.clone()); @@ -641,18 +642,18 @@ impl NetworkInterface for GrpcNetworkSet { Ok(Ok(true)) => return Ok(true), Ok(Ok(false)) => { counter += 1; - continue - }, + continue; + } errors => { tracing::error!("Error in broadcast try_recovery: {:?}", errors); - continue - }, + continue; + } } } if counter > (self.members.len() / 2) { Ok(false) - }else{ + } else { Err(SyneviError::UnrecoverableTransaction) } } diff --git a/synevi_network/src/reconfiguration.rs b/synevi_network/src/reconfiguration.rs index b8776b6..2738193 100644 --- a/synevi_network/src/reconfiguration.rs +++ b/synevi_network/src/reconfiguration.rs @@ -1,12 +1,10 @@ use crate::{ configure_transport::{ - GetEventRequest, GetEventResponse, - JoinElectorateRequest, JoinElectorateResponse, ReadyElectorateRequest, - ReadyElectorateResponse, ReportLastAppliedRequest, ReportLastAppliedResponse, - }, - consensus_transport::{ - ApplyRequest, CommitRequest, + GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, + ReadyElectorateRequest, ReadyElectorateResponse, ReportLastAppliedRequest, + ReportLastAppliedResponse, }, + consensus_transport::{ApplyRequest, CommitRequest}, }; use std::{collections::BTreeMap, sync::Arc}; use synevi_types::{SyneviError, T, T0}; diff --git a/synevi_network/src/replica.rs b/synevi_network/src/replica.rs index 4103582..7044d05 100644 --- a/synevi_network/src/replica.rs +++ b/synevi_network/src/replica.rs @@ -196,7 +196,10 @@ where )) } - async fn try_recovery(&self, request: Request) -> Result, Status> { + async fn try_recovery( + &self, + request: Request, + ) -> Result, Status> { Ok(Response::new( self.inner .try_recover(request.into_inner(), self.inner.is_ready()) diff --git a/synevi_persistence/src/lib.rs b/synevi_persistence/src/lib.rs index af9ba17..896d3a7 100644 --- a/synevi_persistence/src/lib.rs +++ b/synevi_persistence/src/lib.rs @@ -1,3 +1,2 @@ pub mod lmdb_store; pub mod mem_store; -pub mod wait_handler; diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 92f3385..f644996 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -5,30 +5,33 @@ use heed::{ Database, Env, EnvOpenOptions, }; use monotime::MonoTime; -use std::collections::{BTreeMap, HashSet}; +use std::{ + collections::{BTreeMap, HashSet}, + sync::{Arc, Mutex}, +}; use synevi_types::{ error::SyneviError, traits::Store, types::{Event, Hashes, RecoverDependencies, RecoverEvent, UpsertEvent}, Ballot, State, T, T0, }; -use tokio::sync::{mpsc::Receiver, oneshot, Mutex}; +use tokio::sync::mpsc::Receiver; use tracing::instrument; const EVENT_DB_NAME: &str = "events"; type EventDb = Database, SerdeBincode>; -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct PersistentStore { - data: Mutex, + data: Arc>, } #[derive(Clone, Debug)] -struct MutableData { +struct InternalData { db: Env, pub(crate) mappings: BTreeMap, // Key: t, value t0 pub last_applied: T, // t of last applied entry - pub(crate) latest_time: MonoTime, // last created or recognized t0 + pub(crate) latest_time: MonoTime, // last created or recognized t0 pub node_serial: u16, latest_hash: [u8; 32], } @@ -78,43 +81,45 @@ impl PersistentStore { write_txn.commit()?; Ok(PersistentStore { //db: env_clone, - data: Mutex::new(MutableData { + data: Arc::new(Mutex::new(InternalData { db: env_clone, mappings, last_applied, latest_time, node_serial, latest_hash, - }), + })), }) } None => { let _: EventDb = env.create_database(&mut write_txn, Some(EVENT_DB_NAME))?; write_txn.commit()?; Ok(PersistentStore { - data: Mutex::new(MutableData { + data: Arc::new(Mutex::new(InternalData { db: env_clone, mappings: BTreeMap::default(), last_applied: T::default(), latest_time: MonoTime::default(), node_serial, latest_hash: [0; 32], - }), + })), }) } } } } -#[async_trait::async_trait] impl Store for PersistentStore { #[instrument(level = "trace")] - async fn init_t_zero(&self, node_serial: u16) -> T0 { - self.data.lock().await.init_t_zero(node_serial).await + fn init_t_zero(&self, node_serial: u16) -> T0 { + self.data + .lock() + .expect("poisoned lock, aborting") + .init_t_zero(node_serial) } #[instrument(level = "trace")] - async fn pre_accept_tx( + fn pre_accept_tx( &self, id: u128, t_zero: T0, @@ -122,128 +127,137 @@ impl Store for PersistentStore { ) -> Result<(T, HashSet), SyneviError> { self.data .lock() - .await + .expect("poisoned lock, aborting") .pre_accept_tx(id, t_zero, transaction) - .await } #[instrument(level = "trace")] - async fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { - self.data.lock().await.get_tx_dependencies(t, t_zero).await + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_tx_dependencies(t, t_zero) } #[instrument(level = "trace")] - async fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { self.data .lock() - .await + .expect("poisoned lock, aborting") .accept_tx_ballot(t_zero, ballot) - .await } #[instrument(level = "trace", skip(self))] - async fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { - self.data.lock().await.upsert_tx(upsert_event).await + fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .upsert_tx(upsert_event) } #[instrument(level = "trace")] - async fn get_recover_deps(&self, t_zero: &T0) -> Result { - self.data.lock().await.get_recover_deps(t_zero).await + fn get_recover_deps(&self, t_zero: &T0) -> Result { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_recover_deps(t_zero) } #[instrument(level = "trace")] - async fn get_event_state(&self, t_zero: &T0) -> Option { - self.data.lock().await.get_event_state(t_zero).await + fn get_event_state(&self, t_zero: &T0) -> Option { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event_state(t_zero) } #[instrument(level = "trace")] - async fn recover_event( + fn recover_event( &self, t_zero_recover: &T0, node_serial: u16, ) -> Result, SyneviError> { self.data .lock() - .await + .expect("poisoned lock, aborting") .recover_event(t_zero_recover, node_serial) - .await } #[instrument(level = "trace")] - async fn get_event_store(&self) -> BTreeMap { - self.data.lock().await.get_event_store().await + fn get_event_store(&self) -> BTreeMap { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event_store() } #[instrument(level = "trace")] - async fn last_applied(&self) -> (T, T0) { - self.data.lock().await.last_applied().await + fn last_applied(&self) -> (T, T0) { + self.data + .lock() + .expect("poisoned lock, aborting") + .last_applied() } #[instrument(level = "trace")] - async fn get_events_after( + fn get_events_after( &self, last_applied: T, self_event: u128, ) -> Result>, SyneviError> { self.data .lock() - .await + .expect("poisoned lock, aborting") .get_events_after(last_applied, self_event) - .await } #[instrument(level = "trace", skip(self))] - async fn get_event(&self, t_zero: T0) -> Result, SyneviError> { - self.data.lock().await.get_event(t_zero).await + fn get_event(&self, t_zero: T0) -> Result, SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event(t_zero) } - async fn get_and_update_hash( + fn get_and_update_hash( &self, t_zero: T0, execution_hash: [u8; 32], ) -> Result { self.data .lock() - .await + .expect("poisoned lock, aborting") .get_and_update_hash(t_zero, execution_hash) - .await } #[instrument(level = "trace", skip(self))] - async fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - self.data.lock().await.last_applied_hash().await + fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .last_applied_hash() } - async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { - let mut lock = self.data.lock().await; - lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { + let mut lock = self.data.lock().expect("poisoned lock, aborting"); + lock.latest_time = lock + .latest_time + .next_with_guard_and_node(&guard, lock.node_serial) + .into_time(); Ok(()) } - - async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { - let (sdx, rcv) = oneshot::channel(); - todo!(); - - Ok(rcv) - } - - async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { - let (sdx, rcv) = oneshot::channel(); - todo!(); - - Ok(rcv) - } } -impl MutableData { + +impl InternalData { #[instrument(level = "trace")] - async fn init_t_zero(&mut self, node_serial: u16) -> T0 { + fn init_t_zero(&mut self, node_serial: u16) -> T0 { let next_time = self.latest_time.next_with_node(node_serial).into_time(); self.latest_time = next_time; T0(next_time) } #[instrument(level = "trace")] - async fn pre_accept_tx( + fn pre_accept_tx( &mut self, id: u128, t_zero: T0, @@ -252,16 +266,16 @@ impl MutableData { let (t, deps) = { let t = if self.latest_time > *t_zero { let new_time_t = t_zero - .next_with_guard_and_node(&self.latest_time, self.node_serial) - .into_time(); + .next_with_guard_and_node(&self.latest_time, self.node_serial) + .into_time(); self.latest_time = new_time_t; T(new_time_t) - }else{ + } else { T(*t_zero) }; // This might not be necessary to re-use the write lock here - let deps = self.get_tx_dependencies(&t, &t_zero).await; + let deps = self.get_tx_dependencies(&t, &t_zero); (t, deps) }; @@ -274,12 +288,12 @@ impl MutableData { dependencies: Some(deps.clone()), ..Default::default() }; - self.upsert_tx(event).await?; + self.upsert_tx(event)?; Ok((t, deps)) } #[instrument(level = "trace")] - async fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { if self.last_applied == *t { return HashSet::default(); } @@ -305,7 +319,7 @@ impl MutableData { } #[instrument(level = "trace")] - async fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { let mut write_txn = self.db.write_txn().ok()?; let events_db: EventDb = self .db @@ -323,7 +337,7 @@ impl MutableData { } #[instrument(level = "trace", skip(self))] - async fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { //let db = self.db.clone(); // Update the latest time @@ -428,7 +442,7 @@ impl MutableData { } #[instrument(level = "trace")] - async fn get_recover_deps(&self, t_zero: &T0) -> Result { + fn get_recover_deps(&self, t_zero: &T0) -> Result { let read_txn = self.db.read_txn()?; let db: EventDb = self .db @@ -487,7 +501,7 @@ impl MutableData { Ok(recover_deps) } - async fn get_event_state(&self, t_zero: &T0) -> Option { + fn get_event_state(&self, t_zero: &T0) -> Option { let read_txn = self.db.read_txn().ok()?; let db: EventDb = self .db @@ -503,12 +517,12 @@ impl MutableData { Some(state) } - async fn recover_event( + fn recover_event( &self, t_zero_recover: &T0, node_serial: u16, ) -> Result, SyneviError> { - let Some(state) = self.get_event_state(t_zero_recover).await else { + let Some(state) = self.get_event_state(t_zero_recover) else { return Ok(None); }; if matches!(state, synevi_types::State::Undefined) { @@ -541,7 +555,7 @@ impl MutableData { } } - async fn get_event_store(&self) -> BTreeMap { + fn get_event_store(&self) -> BTreeMap { // TODO: Remove unwrap and change trait result let read_txn = self.db.read_txn().unwrap(); let events_db: Database, SerdeBincode> = self @@ -565,13 +579,13 @@ impl MutableData { result } - async fn last_applied(&self) -> (T, T0) { + fn last_applied(&self) -> (T, T0) { let t = self.last_applied.clone(); let t0 = self.mappings.get(&t).cloned().unwrap_or(T0::default()); (t, t0) } - async fn get_events_after( + fn get_events_after( &self, last_applied: T, _self_event: u128, @@ -601,7 +615,7 @@ impl MutableData { Ok(rcv) } - async fn get_event(&self, t_zero: T0) -> Result, SyneviError> { + fn get_event(&self, t_zero: T0) -> Result, SyneviError> { let read_txn = self.db.read_txn()?; let db: EventDb = self .db @@ -612,7 +626,7 @@ impl MutableData { Ok(event) } - async fn get_and_update_hash( + fn get_and_update_hash( &self, t_zero: T0, execution_hash: [u8; 32], @@ -637,7 +651,7 @@ impl MutableData { Ok(hashes) } - async fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { let last = self.last_applied; let last_t0 = self .mappings diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 202cbdf..1fe9566 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -2,30 +2,29 @@ use ahash::RandomState; use monotime::MonoTime; use std::collections::{BTreeMap, HashSet}; use std::fmt::Debug; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use synevi_types::error::SyneviError; use synevi_types::traits::{Dependencies, Store}; -use synevi_types::types::RecoverEvent; use synevi_types::types::{Event, Hashes, RecoverDependencies, UpsertEvent}; +use synevi_types::types::RecoverEvent; use synevi_types::State; use synevi_types::{Ballot, T, T0}; use tokio::sync::mpsc::{Receiver, Sender}; -use tokio::sync::{oneshot, Mutex}; use tracing::instrument; -#[derive(Debug)] -pub struct InternalStore { +#[derive(Debug, Clone)] +struct InternalStore { pub events: BTreeMap, // Key: t0, value: Event pub(crate) mappings: BTreeMap, // Key: t, value t0 pub last_applied: T, // t of last applied entry - pub(crate) latest_time: MonoTime, // last created or recognized time + pub(crate) latest_time: MonoTime, // last created or recognized time pub node_serial: u16, latest_hash: [u8; 32], } #[derive(Debug)] pub struct MemStore { - pub store: Arc>, + store: Arc>, } impl MemStore { @@ -43,13 +42,15 @@ impl MemStore { } } -#[async_trait::async_trait] impl Store for MemStore { - async fn init_t_zero(&self, node_serial: u16) -> T0 { - self.store.lock().await.init_t_zero(node_serial) + fn init_t_zero(&self, node_serial: u16) -> T0 { + self.store + .lock() + .expect("poisoned lock, aborting") + .init_t_zero(node_serial) } - async fn pre_accept_tx( + fn pre_accept_tx( &self, id: u128, t_zero: T0, @@ -57,50 +58,71 @@ impl Store for MemStore { ) -> Result<(T, Dependencies), SyneviError> { self.store .lock() - .await + .expect("poisoned lock, aborting") .pre_accept_tx(id, t_zero, transaction) } - async fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> Dependencies { - self.store.lock().await.get_tx_dependencies(t, t_zero) + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> Dependencies { + self.store + .lock() + .expect("poisoned lock, aborting") + .get_tx_dependencies(t, t_zero) } - async fn get_recover_deps(&self, t_zero: &T0) -> Result { - self.store.lock().await.get_recover_deps(t_zero) + fn get_recover_deps(&self, t_zero: &T0) -> Result { + self.store + .lock() + .expect("poisoned lock, aborting") + .get_recover_deps(t_zero) } - async fn recover_event( + fn recover_event( &self, t_zero_recover: &T0, node_serial: u16, ) -> Result, SyneviError> { self.store .lock() - .await + .expect("poisoned lock, aborting") .recover_event(t_zero_recover, node_serial) } - async fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { - self.store.lock().await.accept_tx_ballot(t_zero, ballot) + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { + self.store + .lock() + .expect("poisoned lock, aborting") + .accept_tx_ballot(t_zero, ballot) } - async fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { - self.store.lock().await.upsert_tx(upsert_event) + fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + self.store + .lock() + .expect("poisoned lock, aborting") + .upsert_tx(upsert_event) } - async fn get_event_state(&self, t_zero: &T0) -> Option { - self.store.lock().await.get_event_state(t_zero) + fn get_event_state(&self, t_zero: &T0) -> Option { + self.store + .lock() + .expect("poisoned lock, aborting") + .get_event_state(t_zero) } - async fn get_event_store(&self) -> BTreeMap { - self.store.lock().await.get_event_store() + fn get_event_store(&self) -> BTreeMap { + self.store + .lock() + .expect("poisoned lock, aborting") + .get_event_store() } - async fn last_applied(&self) -> (T, T0) { - self.store.lock().await.last_applied() + fn last_applied(&self) -> (T, T0) { + self.store + .lock() + .expect("poisoned lock, aborting") + .last_applied() } - async fn get_events_after( + fn get_events_after( &self, last_applied: T, self_event: u128, @@ -108,27 +130,32 @@ impl Store for MemStore { let (sdx, rcv) = tokio::sync::mpsc::channel(100); let store = self.store.clone(); - tokio::spawn(async move { + tokio::task::spawn_blocking(move || { store .lock() - .await - .get_events_until(last_applied, self_event, sdx) - .await?; + .expect("poisoned lock, aborting") + .get_events_until(last_applied, self_event, sdx)?; Ok::<(), SyneviError>(()) }); Ok(rcv) } - async fn get_event(&self, t_zero: T0) -> Result, SyneviError> { - Ok(self.store.lock().await.events.get(&t_zero).cloned()) + fn get_event(&self, t_zero: T0) -> Result, SyneviError> { + Ok(self + .store + .lock() + .expect("poisoned lock, aborting") + .events + .get(&t_zero) + .cloned()) } - async fn get_and_update_hash( + fn get_and_update_hash( &self, t_zero: T0, execution_hash: [u8; 32], ) -> Result { - let mut lock = self.store.lock().await; + let mut lock = self.store.lock().expect("poisoned lock, aborting"); if let Some(event) = lock.events.get_mut(&t_zero) { let hashes = event .hashes @@ -141,8 +168,8 @@ impl Store for MemStore { } } - async fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - let lock = self.store.lock().await; + fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + let lock = self.store.lock().expect("poisoned lock, aborting"); let last = lock.last_applied; let last_t0 = lock .mappings @@ -158,25 +185,14 @@ impl Store for MemStore { Ok((last, hash.execution_hash)) } - async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { - let mut lock = self.store.lock().await; - lock.latest_time = lock.latest_time.next_with_guard_and_node(&guard, lock.node_serial).into_time(); + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { + let mut lock = self.store.lock().expect("poisoned lock, aborting"); + lock.latest_time = lock + .latest_time + .next_with_guard_and_node(&guard, lock.node_serial) + .into_time(); Ok(()) } - - async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { - let (sdx, rcv) = oneshot::channel(); - todo!(); - - Ok(rcv) - } - - async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError> { - let (sdx, rcv) = oneshot::channel(); - todo!(); - - Ok(rcv) - } } impl InternalStore { @@ -197,12 +213,12 @@ impl InternalStore { let (t, deps) = { let t = if self.latest_time > *t_zero { let new_time_t = t_zero - .next_with_guard_and_node(&self.latest_time, self.node_serial) - .into_time(); + .next_with_guard_and_node(&self.latest_time, self.node_serial) + .into_time(); self.latest_time = new_time_t; T(new_time_t) - }else{ + } else { T(*t_zero) }; // This might not be necessary to re-use the write lock here @@ -257,7 +273,6 @@ impl InternalStore { #[instrument(level = "trace")] fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { - // Update the latest time if self.latest_time < *upsert_event.t { self.latest_time = *upsert_event.t; @@ -439,7 +454,7 @@ impl InternalStore { (self.last_applied, t0) } - async fn get_events_until( + fn get_events_until( &self, last_applied: T, _self_event: u128, @@ -451,8 +466,7 @@ impl InternalStore { _ => return Err(SyneviError::EventNotFound(last_applied.get_inner())), }; for (_, event) in self.events.range(last_applied_t0..) { - sdx.send(Ok(event.clone())) - .await + sdx.blocking_send(Ok(event.clone())) .map_err(|e| SyneviError::SendError(e.to_string()))?; } Ok(()) diff --git a/synevi_persistence/src/wait_handler.rs b/synevi_persistence/src/wait_handler.rs deleted file mode 100644 index 53a5d83..0000000 --- a/synevi_persistence/src/wait_handler.rs +++ /dev/null @@ -1,39 +0,0 @@ -use std::collections::HashMap; -use ahash::RandomState; -use synevi_types::T0; -use tokio::{sync::oneshot, time::Instant}; - -pub struct Waiter { - waited_since: Instant, - dependency_states: u64, - sender: Vec>, -} - -pub struct WaitHandler { - waiters: HashMap, -} - - -impl WaitHandler { - - pub fn run() { - - loop {} - } -} - -// Tx1 = dep[Tx0] - -// -> Tx0 commit -// -> for each waiter: is tx0 in deps? -// -> if yes! -> is t(tx0) > t(tx1) -// -> y -> do nothing -// -> n -> increase dep_state +1 -// -> if dep_state == dep.len() -> send signal to waiter -// -// -//loop { -// if waiter.waited_since > 10s -> Find inital tx everyone is waiting for -> -// -//} - diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index c6079d1..722a407 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -4,10 +4,12 @@ use std::{ collections::{BTreeMap, HashSet}, sync::{Arc, Weak}, }; -use tokio::sync::{mpsc::Receiver, oneshot}; +use tokio::sync::mpsc::Receiver; use crate::{ - types::{Event, Hashes, RecoverDependencies, RecoverEvent, SyneviResult, UpsertEvent}, + types::{ + Event, Hashes, RecoverDependencies, RecoverEvent, SyneviResult, UpsertEvent, + }, Ballot, State, SyneviError, T, T0, }; @@ -68,44 +70,47 @@ where pub type Dependencies = HashSet; -#[async_trait::async_trait] pub trait Store: Send + Sync + Sized + 'static { // fn new(node_serial: u16) -> Result; // Initialize a new t0 - async fn init_t_zero(&self, node_serial: u16) -> T0; + fn init_t_zero(&self, node_serial: u16) -> T0; // Pre-accept a transaction - async fn pre_accept_tx( + fn pre_accept_tx( &self, id: u128, t_zero: T0, transaction: Vec, ) -> Result<(T, Dependencies), SyneviError>; // Get the dependencies for a transaction - async fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> Dependencies; + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> Dependencies; // Get the recover dependencies for a transaction - async fn get_recover_deps(&self, t_zero: &T0) -> Result; + fn get_recover_deps(&self, t_zero: &T0) -> Result; // Tries to recover an unfinished event from the store - async fn recover_event( + fn recover_event( &self, t_zero_recover: &T0, node_serial: u16, ) -> Result, SyneviError>; // Check and update the ballot for a transaction // Returns true if the ballot was accepted (current <= ballot) - async fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option; + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option; // Update or insert a transaction, returns the hash of the transaction if applied - async fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError>; + fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError>; - async fn get_event_state(&self, t_zero: &T0) -> Option; + fn get_event_state(&self, t_zero: &T0) -> Option; - async fn get_event_store(&self) -> BTreeMap; - async fn last_applied(&self) -> (T, T0); - async fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; + fn get_event_store(&self) -> BTreeMap; + fn last_applied(&self) -> (T, T0); + fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; - async fn get_event(&self, t_zero: T0) -> Result, SyneviError>; - async fn get_events_after(&self, last_applied: T, self_event: u128) -> Result>, SyneviError>; + fn get_event(&self, t_zero: T0) -> Result, SyneviError>; + fn get_events_after( + &self, + last_applied: T, + self_event: u128, + ) -> Result>, SyneviError>; - async fn get_and_update_hash( + fn get_and_update_hash( &self, t_zero: T0, execution_hash: [u8; 32], @@ -113,7 +118,5 @@ pub trait Store: Send + Sync + Sized + 'static { // Increases the max time to be above the specified guard // Ensures that the guards t0 will not get a fast path afterwards - async fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError>; - async fn waiter_commit(&self, upsert_event: UpsertEvent) -> Result, SyneviError>; - async fn waiter_apply(&self, upsert_event: UpsertEvent) -> Result, SyneviError>; + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError>; } diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index 5f7e93e..05ea42f 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -7,8 +7,9 @@ use sha3::{Digest, Sha3_256}; use std::{ collections::HashSet, ops::Deref, - time::{SystemTime, UNIX_EPOCH}, + time::{Instant, SystemTime, UNIX_EPOCH}, }; +use tokio::sync::oneshot; use ulid::Ulid; pub type SyneviResult = Result< @@ -23,6 +24,12 @@ pub enum ExecutorResult { Internal(Result), } +pub struct Waiter { + pub waited_since: Instant, + pub dependency_states: u64, + pub sender: Vec>, +} + #[derive(Default, PartialEq, PartialOrd, Ord, Eq, Clone, Debug, Serialize)] pub enum TransactionPayload { #[default] @@ -349,7 +356,7 @@ impl Event { hasher.update(self.transaction.as_slice()); hasher.update(previous_hash); - let event_hash: [u8;32] = hasher.finalize().into(); + let event_hash: [u8; 32] = hasher.finalize().into(); Hashes { previous_hash, transaction_hash: event_hash, @@ -414,7 +421,7 @@ impl From for Event { ballot: value.ballot.unwrap_or_default(), hashes: value.execution_hash.map(|hash| Hashes { previous_hash: [0; 32], - transaction_hash:[0;32], + transaction_hash: [0; 32], execution_hash: hash, }), last_updated: SystemTime::now() diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index 3aed41b..3d34107 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -57,10 +57,7 @@ mod tests { let coordinator = coordinator.clone(); joinset.spawn(async move { coordinator - .transaction( - i, - Vec::from("This is a transaction"), - ) + .transaction(i, Vec::from("This is a transaction")) .await }); } @@ -89,7 +86,6 @@ mod tests { let coordinator_store: BTreeMap)> = coordinator .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, (e.t, e.get_latest_hash()))) .collect(); @@ -97,7 +93,6 @@ mod tests { assert!(coordinator .event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied)); @@ -106,14 +101,12 @@ mod tests { let node_store: BTreeMap)> = node .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, (e.t, e.get_latest_hash()))) .collect(); assert!( node.event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied), "Not all applied @ {:?}", @@ -194,42 +187,27 @@ mod tests { let coordinator5 = coordinator5.clone(); joinset.spawn(async move { coordinator1 - .transaction( - u128::from_be_bytes(Ulid::new().to_bytes()), - Vec::from("C1"), - ) + .transaction(u128::from_be_bytes(Ulid::new().to_bytes()), Vec::from("C1")) .await }); joinset.spawn(async move { coordinator2 - .transaction( - u128::from_be_bytes(Ulid::new().to_bytes()), - Vec::from("C2"), - ) + .transaction(u128::from_be_bytes(Ulid::new().to_bytes()), Vec::from("C2")) .await }); joinset.spawn(async move { coordinator3 - .transaction( - u128::from_be_bytes(Ulid::new().to_bytes()), - Vec::from("C3"), - ) + .transaction(u128::from_be_bytes(Ulid::new().to_bytes()), Vec::from("C3")) .await }); joinset.spawn(async move { coordinator4 - .transaction( - u128::from_be_bytes(Ulid::new().to_bytes()), - Vec::from("C4"), - ) + .transaction(u128::from_be_bytes(Ulid::new().to_bytes()), Vec::from("C4")) .await }); joinset.spawn(async move { coordinator5 - .transaction( - u128::from_be_bytes(Ulid::new().to_bytes()), - Vec::from("C5"), - ) + .transaction(u128::from_be_bytes(Ulid::new().to_bytes()), Vec::from("C5")) .await }); } @@ -295,7 +273,6 @@ mod tests { let coordinator_store: BTreeMap)> = coordinator1 .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, (e.t, e.get_latest_hash()))) .collect(); @@ -310,14 +287,12 @@ mod tests { let node_store: BTreeMap)> = node .event_store .get_event_store() - .await .into_values() .map(|e| (e.t_zero, (e.t, e.get_latest_hash()))) .collect(); assert!(node .event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied)); assert_eq!(coordinator_store.len(), node_store.len()); @@ -388,10 +363,7 @@ mod tests { for i in 0..1000 { match coordinator .clone() - .transaction( - i, - Vec::from("This is a transaction"), - ) + .transaction(i, Vec::from("This is a transaction")) .await .unwrap() { @@ -502,12 +474,7 @@ mod tests { let coordinator = coordinator.clone(); joinset.spawn(async move { coordinator - .transaction( - i, - Vec::from( - "This is a transaction", - ), - ) + .transaction(i, Vec::from("This is a transaction")) .await }); } @@ -547,7 +514,6 @@ mod tests { let coordinator_store: BTreeMap = coordinator .event_store .get_event_store() - .await .into_values() .map(|e| { ( @@ -560,7 +526,6 @@ mod tests { assert!(coordinator .event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied)); @@ -571,7 +536,6 @@ mod tests { let node_store: BTreeMap = node .event_store .get_event_store() - .await .into_values() .map(|e| { ( @@ -584,7 +548,6 @@ mod tests { assert!( node.event_store .get_event_store() - .await .iter() .all(|(_, e)| e.state == State::Applied), "Not all applied @ {:?}", From 9da1ba8c938d83265b05b9aa04cc41eba5866365 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Tue, 15 Oct 2024 17:09:26 +0200 Subject: [PATCH 09/40] feat: Updated waithandler to keep less state --- synevi_core/src/coordinator.rs | 2 +- synevi_core/src/node.rs | 87 ++++++++++++++++++++++---- synevi_core/src/replica.rs | 68 ++++++++++----------- synevi_core/src/wait_handler.rs | 95 ++++++++++++++++++++++++++++- synevi_persistence/src/mem_store.rs | 2 +- synevi_types/src/traits.rs | 4 +- synevi_types/src/types.rs | 14 +++++ 7 files changed, 219 insertions(+), 53 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 16100b3..b8c0190 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -284,7 +284,7 @@ where #[instrument(level = "trace", skip(self))] async fn execute_consensus(&mut self) -> Result<(SyneviResult, Hashes), SyneviError> { self.transaction.state = State::Applied; - + self.node.apply((&self.transaction).into()).await?; let result = match &self.transaction.transaction { diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index cfd2efd..ebb1004 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -1,13 +1,14 @@ use crate::coordinator::Coordinator; use crate::replica::ReplicaConfig; -use crate::wait_handler::WaitHandler; +use crate::wait_handler::{CheckResult, WaitHandler}; use std::fmt::Debug; use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::RwLock; use std::sync::{atomic::AtomicU64, Arc}; use synevi_network::consensus_transport::{ ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, }; -use synevi_network::network::{Network, NodeInfo}; +use synevi_network::network::{Network, NetworkInterface, NodeInfo}; use synevi_network::reconfiguration::{BufferedMessage, Report}; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; @@ -46,6 +47,7 @@ where semaphore: Arc, has_members: AtomicBool, is_ready: Arc, + self_clone: RwLock>>, } impl Node @@ -107,12 +109,20 @@ where executor, has_members: AtomicBool::new(false), is_ready: Arc::new(AtomicBool::new(true)), + self_clone: RwLock::new(None), }); + node.self_clone + .write() + .expect("Locking self_clone failed") + .replace(node.clone()); let ready = Arc::new(AtomicBool::new(true)); let (replica, _) = ReplicaConfig::new(node.clone(), ready); node.network.spawn_server(replica).await?; + let node_clone = node.clone(); + tokio::spawn(async move { node_clone.run_check_recovery().await }); + // If no config / persistence -> default Ok(node) } @@ -152,10 +162,18 @@ where executor, has_members: AtomicBool::new(false), is_ready: ready.clone(), + self_clone: RwLock::new(None), }); + node.self_clone + .write() + .expect("Locking self_clone failed") + .replace(node.clone()); + let (replica, config_receiver) = ReplicaConfig::new(node.clone(), ready.clone()); node.network.spawn_server(replica.clone()).await?; + let node_clone = node.clone(); + tokio::spawn(async move { node_clone.run_check_recovery().await }); node.reconfigure(replica, member_host, config_receiver, ready) .await?; @@ -233,12 +251,9 @@ where let t0_commit = event.t_zero.clone(); let t_commit = event.t.clone(); - let prev_event = self - .event_store - .get_event(t0_commit)? - .ok_or_else(|| SyneviError::EventNotFound(event.id))?; + let prev_event = self.event_store.get_event(t0_commit)?; - if prev_event.state < State::Commited { + if !prev_event.is_some_and(|e| e.state > State::Commited) { self.event_store.upsert_tx(event)?; let waiter = self.wait_handler.get_waiter(&t0_commit); waiter.await.map_err(|e| { @@ -251,15 +266,19 @@ where } #[instrument(level = "trace", skip(self))] - pub async fn apply(&self, event: UpsertEvent) -> Result<(), SyneviError> { + pub async fn apply(&self, mut event: UpsertEvent) -> Result<(), SyneviError> { let t0_apply = event.t_zero.clone(); - let prev_event = self - .event_store - .get_event(t0_apply)? - .ok_or_else(|| SyneviError::EventNotFound(event.id))?; + let prev_event = self.event_store.get_event(t0_apply)?; - if prev_event.state < State::Applied { + let needs_wait = if let Some(prev_event) = prev_event { + prev_event.state < State::Applied + } else { + event.state = State::Commited; + self.commit(event.clone()).await?; + true + }; + if needs_wait { let waiter = self.wait_handler.get_waiter(&t0_apply); waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); @@ -271,6 +290,48 @@ where Ok(()) } + async fn run_check_recovery(&self) { + let self_clonable = self + .self_clone + .read() + .expect("Locking self_clone failed") + .clone() + .expect("Self clone is None"); + + loop { + match self.wait_handler.check_recovery() { + CheckResult::NoRecovery => (), + CheckResult::RecoverEvent(recover_event) => { + let self_clone = self_clonable.clone(); + match tokio::spawn(Coordinator::recover(self_clone, recover_event)).await { + Ok(Ok(_)) => (), + Ok(Err(e)) => { + tracing::error!("Error recovering event: {:?}", e); + } + Err(e) => { + tracing::error!("JoinError recovering event: {:?}", e); + } + } + } + CheckResult::RecoverUnknown(t0_recover) => { + let interface = self.network.get_interface().await; + match interface.broadcast_recovery(t0_recover).await { + Ok(true) => (), + Ok(false) => { + self.wait_handler.apply(&t0_recover); + } + Err(err) => { + tracing::error!("Error broadcasting recovery: {:?}", err); + panic!("Error broadcasting recovery: {:?}", err); + } + } + } + } + + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; + } + } + #[instrument(level = "trace", skip(self, replica))] async fn reconfigure( &self, diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 4bf8b59..e5572de 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -186,18 +186,16 @@ where } } - self.node - .event_store - .upsert_tx(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Accepted, - transaction: Some(request.event), - dependencies: Some(from_dependency(request.dependencies)?), - ballot: Some(request_ballot), - execution_hash: None, - })?; + self.node.event_store.upsert_tx(UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Accepted, + transaction: Some(request.event), + dependencies: Some(from_dependency(request.dependencies)?), + ballot: Some(request_ballot), + execution_hash: None, + })?; self.node.event_store.get_tx_dependencies(&t, &t_zero) }; @@ -228,18 +226,18 @@ where let deps = from_dependency(request.dependencies)?; - - - self.node.commit(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Commited, - transaction: Some(request.event), - dependencies: Some(deps), - ballot: None, - execution_hash: None, - }).await?; + self.node + .commit(UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Commited, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }) + .await?; Ok(CommitResponse {}) } @@ -266,16 +264,18 @@ where let deps = from_dependency(request.dependencies.clone())?; - self.node.apply(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Applied, - transaction: Some(request.event), - dependencies: Some(deps), - ballot: None, - execution_hash: None, - }).await?; + self.node + .apply(UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Applied, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + execution_hash: None, + }) + .await?; // TODO: Refactor in execute function let result = match transaction { diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index cff0816..ed1744d 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -3,7 +3,7 @@ use std::{ collections::HashMap, sync::{Arc, Mutex}, }; -use synevi_types::{traits::Store, T, T0}; +use synevi_types::{traits::Store, types::RecoverEvent, State, T, T0}; use tokio::{sync::oneshot, time::Instant}; pub struct Waiter { @@ -12,6 +12,50 @@ pub struct Waiter { sender: Vec>, } +pub enum CheckResult { + NoRecovery, + RecoverEvent(RecoverEvent), + RecoverUnknown(T0), +} + +impl CheckResult { + pub fn replace_if_smaller(&mut self, other: CheckResult) { + match (&self, &other) { + (CheckResult::NoRecovery, _) => *self = other, + ( + CheckResult::RecoverEvent(recover_event_existing), + CheckResult::RecoverEvent(recover_event), + ) => { + if recover_event.t_zero < recover_event_existing.t_zero { + *self = other; + } + } + ( + CheckResult::RecoverEvent(recover_event_existing), + CheckResult::RecoverUnknown(t0), + ) => { + if *t0 < recover_event_existing.t_zero { + *self = other; + } + } + ( + CheckResult::RecoverUnknown(t0_existing), + CheckResult::RecoverEvent(recover_event), + ) => { + if recover_event.t_zero < *t0_existing { + *self = other; + } + } + (CheckResult::RecoverUnknown(t0_existing), CheckResult::RecoverUnknown(t0)) => { + if t0 < t0_existing { + *self = other; + } + } + _ => (), + } + } +} + pub struct WaitHandler where S: Store, @@ -80,6 +124,55 @@ where true }); } + + pub fn check_recovery(&self) -> CheckResult { + let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + let mut smallest_hanging_dep = CheckResult::NoRecovery; + for (t0, waiter) in waiter_lock.iter_mut() { + if waiter.waited_since.elapsed().as_millis() > 1000 { + // Get deps and find smallest dep that is not committed / applied + let Some(event) = self.store.get_event(*t0).ok().flatten() else { + tracing::error!( + "Unexpected state in wait_handler: Event timed out, but not found in store" + ); + continue; + }; + for dep in event.dependencies.iter() { + let Some(event_dep) = self.store.get_event(*dep).ok().flatten() else { + smallest_hanging_dep.replace_if_smaller(CheckResult::RecoverUnknown(*dep)); + continue; + }; + if event_dep.t_zero > event.t_zero { + tracing::error!("Error: Dependency is newer than event"); + continue; + } + match event_dep.state { + State::Commited => { + if event_dep.t > event.t { + // Dependency is newer than event (and already commited) + continue; + } + smallest_hanging_dep + .replace_if_smaller(CheckResult::RecoverEvent(event_dep.into())); + } + State::Applied => { + // Already applied (no problem) + continue; + } + _ => { + smallest_hanging_dep + .replace_if_smaller(CheckResult::RecoverEvent(event_dep.into())); + } + } + } + if !matches!(smallest_hanging_dep, CheckResult::NoRecovery) { + waiter.waited_since = Instant::now(); + return smallest_hanging_dep; + } + } + } + CheckResult::NoRecovery + } } // Tx1 = dep[Tx0] diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 1fe9566..54a2868 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -5,8 +5,8 @@ use std::fmt::Debug; use std::sync::{Arc, Mutex}; use synevi_types::error::SyneviError; use synevi_types::traits::{Dependencies, Store}; -use synevi_types::types::{Event, Hashes, RecoverDependencies, UpsertEvent}; use synevi_types::types::RecoverEvent; +use synevi_types::types::{Event, Hashes, RecoverDependencies, UpsertEvent}; use synevi_types::State; use synevi_types::{Ballot, T, T0}; use tokio::sync::mpsc::{Receiver, Sender}; diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index 722a407..a54c69d 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -7,9 +7,7 @@ use std::{ use tokio::sync::mpsc::Receiver; use crate::{ - types::{ - Event, Hashes, RecoverDependencies, RecoverEvent, SyneviResult, UpsertEvent, - }, + types::{Event, Hashes, RecoverDependencies, RecoverEvent, SyneviResult, UpsertEvent}, Ballot, State, SyneviError, T, T0, }; diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index 05ea42f..d9e3da0 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -432,6 +432,20 @@ impl From for Event { } } +impl From for RecoverEvent { + fn from(value: Event) -> Self { + RecoverEvent { + id: value.id, + t_zero: value.t_zero, + t: value.t, + state: value.state, + transaction: value.transaction, + dependencies: value.dependencies, + ballot: value.ballot, + } + } +} + #[cfg(test)] mod test { use crate::{types::TransactionPayload, Transaction}; From 708a4f8b2b74df7d0ac7fc1985efe30c6ade2722 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Tue, 15 Oct 2024 17:12:42 +0200 Subject: [PATCH 10/40] chore: Remove unwraps in wait_handler --- synevi_core/src/wait_handler.rs | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index ed1744d..d0be52a 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -90,14 +90,19 @@ where pub fn commit(&self, t0_commit: &T0, t_commit: &T) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); waiter_lock.retain(|t0_waiting, waiter| { - let event = self.store.get_event(*t0_waiting).unwrap().unwrap(); // TODO: Remove unwrap + let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { + tracing::error!( + "Unexpected state in wait_handler: Event not found in store" + ); + return true; + }; if event.dependencies.contains(t0_commit) { if t_commit > &event.t { waiter.dependency_states += 1; waiter.waited_since = Instant::now(); if waiter.dependency_states >= event.dependencies.len() as u64 { for sdx in waiter.sender.drain(..) { - sdx.send(()).unwrap(); // TODO: Remove unwrap + let _ = sdx.send(()); } return false; } @@ -110,13 +115,18 @@ where pub fn apply(&self, t0_commit: &T0) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); waiter_lock.retain(|t0_waiting, waiter| { - let event = self.store.get_event(*t0_waiting).unwrap().unwrap(); // TODO: Remove unwrap + let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { + tracing::error!( + "Unexpected state in wait_handler: Event not found in store" + ); + return true; + }; if event.dependencies.contains(t0_commit) { waiter.dependency_states += 1; waiter.waited_since = Instant::now(); if waiter.dependency_states >= event.dependencies.len() as u64 { for sdx in waiter.sender.drain(..) { - sdx.send(()).unwrap(); // TODO: Remove unwrap + let _ = sdx.send(()); } return false; } From bbc9f53b33cd86dca4661db6d8d4782f21048a74 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Wed, 16 Oct 2024 16:25:09 +0200 Subject: [PATCH 11/40] fix: Empty deps in waithandler and already applied / commited ones --- synevi_core/src/node.rs | 51 +++++++++++++++++++-------------- synevi_core/src/wait_handler.rs | 49 ++++++++++++++++++++++--------- 2 files changed, 66 insertions(+), 34 deletions(-) diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index ebb1004..d0f2a5a 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -252,21 +252,22 @@ where let t_commit = event.t.clone(); let prev_event = self.event_store.get_event(t0_commit)?; - - if !prev_event.is_some_and(|e| e.state > State::Commited) { - self.event_store.upsert_tx(event)?; - let waiter = self.wait_handler.get_waiter(&t0_commit); - waiter.await.map_err(|e| { - tracing::error!("Error waiting for commit: {:?}", e); - SyneviError::ReceiveError(format!("Error waiting for commit")) - })?; + self.event_store.upsert_tx(event)?; + self.wait_handler.notify_commit(&t0_commit, &t_commit); + + if !prev_event.is_some_and(|e| e.state > State::Commited && e.dependencies.is_empty()) { + if let Some(waiter) = self.wait_handler.get_waiter(&t0_commit) { + waiter.await.map_err(|e| { + tracing::error!("Error waiting for commit: {:?}", e); + SyneviError::ReceiveError(format!("Error waiting for commit")) + })? + }; } - self.wait_handler.commit(&t0_commit, &t_commit); Ok(()) } #[instrument(level = "trace", skip(self))] - pub async fn apply(&self, mut event: UpsertEvent) -> Result<(), SyneviError> { + pub async fn apply(&self, event: UpsertEvent) -> Result<(), SyneviError> { let t0_apply = event.t_zero.clone(); let prev_event = self.event_store.get_event(t0_apply)?; @@ -274,19 +275,26 @@ where let needs_wait = if let Some(prev_event) = prev_event { prev_event.state < State::Applied } else { - event.state = State::Commited; - self.commit(event.clone()).await?; + let mut commit_event = event.clone(); + commit_event.state = State::Commited; + self.commit(commit_event).await?; true }; - if needs_wait { - let waiter = self.wait_handler.get_waiter(&t0_apply); - waiter.await.map_err(|e| { - tracing::error!("Error waiting for commit: {:?}", e); - SyneviError::ReceiveError(format!("Error waiting for commit")) - })?; - self.event_store.upsert_tx(event)?; + if event + .dependencies + .as_ref() + .is_some_and(|deps| !deps.is_empty()) + && needs_wait + { + if let Some(waiter) = self.wait_handler.get_waiter(&t0_apply) { + waiter.await.map_err(|e| { + tracing::error!("Error waiting for commit: {:?}", e); + SyneviError::ReceiveError(format!("Error waiting for commit")) + })?; + } } - self.wait_handler.apply(&t0_apply); + self.event_store.upsert_tx(event)?; + self.wait_handler.notify_apply(&t0_apply); Ok(()) } @@ -318,7 +326,8 @@ where match interface.broadcast_recovery(t0_recover).await { Ok(true) => (), Ok(false) => { - self.wait_handler.apply(&t0_recover); + println!("Unknown recovery failed"); + self.wait_handler.notify_apply(&t0_recover); } Err(err) => { tracing::error!("Error broadcasting recovery: {:?}", err); diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index d0be52a..6edc920 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -75,27 +75,52 @@ where } } - pub fn get_waiter(&self, t0: &T0) -> oneshot::Receiver<()> { + pub fn get_waiter(&self, t0: &T0) -> Option> { let (sdx, rcv) = oneshot::channel(); let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + + let Some(event) = self.store.get_event(*t0).ok().flatten() else { + tracing::error!("Unexpected state in wait_handler: Event not found in store"); + return None; + }; + + let mut counter = 0; + for dep_t0 in event.dependencies.iter() { + let Some(dep_event) = self.store.get_event(*dep_t0).ok().flatten() else { + continue; + }; + + match dep_event.state { + State::Commited if dep_event.t > event.t => { + counter += 1; + } + State::Applied => { + counter += 1; + } + _ => {} + } + } + + if counter >= event.dependencies.len() as u64 { + return None; + } + let waiter = waiter_lock.entry(*t0).or_insert(Waiter { waited_since: Instant::now(), - dependency_states: 0, + dependency_states: counter, sender: Vec::new(), }); waiter.sender.push(sdx); - rcv + Some(rcv) } - pub fn commit(&self, t0_commit: &T0, t_commit: &T) { + pub fn notify_commit(&self, t0_commit: &T0, t_commit: &T) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); waiter_lock.retain(|t0_waiting, waiter| { let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { - tracing::error!( - "Unexpected state in wait_handler: Event not found in store" - ); + tracing::error!("Unexpected state in wait_handler: Event not found in store"); return true; - }; + }; if event.dependencies.contains(t0_commit) { if t_commit > &event.t { waiter.dependency_states += 1; @@ -112,15 +137,13 @@ where }); } - pub fn apply(&self, t0_commit: &T0) { + pub fn notify_apply(&self, t0_commit: &T0) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); waiter_lock.retain(|t0_waiting, waiter| { let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { - tracing::error!( - "Unexpected state in wait_handler: Event not found in store" - ); + tracing::error!("Unexpected state in wait_handler: Event not found in store"); return true; - }; + }; if event.dependencies.contains(t0_commit) { waiter.dependency_states += 1; waiter.waited_since = Instant::now(); From 87617d857453d6002e0afabee3f9a679c6242259 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 09:35:09 +0200 Subject: [PATCH 12/40] feat: started removing the reconfig buffer --- failed.txt | 144 ++++++++++++++++++++++++++ synevi_core/src/coordinator.rs | 4 +- synevi_core/src/node.rs | 87 +++++----------- synevi_core/src/replica.rs | 101 +++--------------- synevi_network/src/reconfiguration.rs | 66 +----------- synevi_network/src/replica.rs | 37 ++----- synevi_persistence/src/lmdb_store.rs | 20 ++++ tests/consensus_e2e.rs | 6 +- tests/maelstrom/network.rs | 75 ++++++-------- 9 files changed, 257 insertions(+), 283 deletions(-) create mode 100644 failed.txt diff --git a/failed.txt b/failed.txt new file mode 100644 index 0000000..b28de12 --- /dev/null +++ b/failed.txt @@ -0,0 +1,144 @@ + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 4 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 4 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 1 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.00s + + +running 0 tests + +test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.00s + + +running 1 test +[0]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +Returning streaming receiver +[1]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +[0]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[1]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[0]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +[1]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +Returning streaming receiver +[2]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +[2]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[2]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +[1]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[2]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[0]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[0]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +[2]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +[1]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +Returning streaming receiver +[3]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +[3]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[3]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +[3]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[3]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +[1]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[3]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[0]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[2]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[0]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +[2]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +[3]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +[1]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +Returning streaming receiver +[4]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +[4]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[4]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +[4]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[4]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +[4]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[4]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +[0]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[4]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[3]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[1]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[2]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[0]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[4]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[3]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[2]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[1]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[0]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[4]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[3]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[2]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[1]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[0]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +[0]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[1]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +[3]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +[4]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +[2]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +[1]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[4]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[2]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[3]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[0]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) +[2]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) +[3]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) +[1]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) +[4]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) +[0]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) +[4]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) +[3]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) +[1]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) +[2]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) +[0]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) +[2]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) +[1]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) +[3]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) +[4]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) +Returning streaming receiver +[6]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) +[6]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) +[6]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) +[6]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) +[6]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) +[6]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) +[6]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) +[6]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) +[6]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) +[6]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) +[6]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) +[6]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) +last_t: T(MonoTime(486696121103699508920252790603776)), event.t: T(MonoTime(486696121103698547401740937068544)) +test tests::reconfiguration ... FAILED + +failures: + +failures: + tests::reconfiguration + +test result: FAILED. 0 passed; 1 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.09s + diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index b8c0190..9019a42 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -71,7 +71,7 @@ where node, transaction: TransactionStateMachine { id, - state: State::Undefined, + state: State::PreAccepted, transaction, t_zero: t0, t: T(*t0), @@ -584,7 +584,7 @@ pub mod tests { ) .await; - assert_eq!(coordinator.transaction.state, State::Undefined); + assert_eq!(coordinator.transaction.state, State::PreAccepted); assert_eq!(*coordinator.transaction.t_zero, *coordinator.transaction.t); assert_eq!(coordinator.transaction.t_zero.0.get_node(), 0); assert_eq!(coordinator.transaction.t_zero.0.get_seq(), 1); diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index d0f2a5a..1c97658 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -5,18 +5,15 @@ use std::fmt::Debug; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::RwLock; use std::sync::{atomic::AtomicU64, Arc}; -use synevi_network::consensus_transport::{ - ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, -}; +use synevi_network::consensus_transport::{ApplyRequest, CommitRequest}; use synevi_network::network::{Network, NetworkInterface, NodeInfo}; -use synevi_network::reconfiguration::{BufferedMessage, Report}; +use synevi_network::reconfiguration::Report; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; use synevi_types::types::{SyneviResult, TransactionPayload, UpsertEvent}; use synevi_types::{Executor, State, SyneviError, T}; use tokio::sync::mpsc::Receiver; -use tokio::task::JoinSet; use tracing::instrument; use ulid::Ulid; @@ -27,11 +24,6 @@ pub struct Stats { pub total_recovers: AtomicU64, } -enum HelperResponse { - Commit(CommitResponse), - Apply(ApplyResponse), -} - pub struct Node where N: Network + Send + Sync, @@ -131,6 +123,10 @@ where self.is_ready.store(true, Ordering::Relaxed); } + pub fn is_ready(&self) -> bool { + self.is_ready.load(Ordering::Relaxed) + } + #[instrument(level = "trace", skip(network, executor, store))] pub async fn new_with_member( id: Ulid, @@ -293,6 +289,11 @@ where })?; } } + println!( + "[{:?}]Applied event: t0: {:?}, t: {:?}", + self.info.serial, event.t_zero, event.t + ); + self.event_store.upsert_tx(event)?; self.wait_handler.notify_apply(&t0_apply); Ok(()) @@ -384,34 +385,6 @@ where // 2.1 if majority replies with 0 events -> skip to 2.4. self.sync_events(last_applied, self_id, &replica).await?; - // 2.4 Apply buffered commits & applies - let mut rcv = replica.send_buffered().await?; - let mut join_set = JoinSet::new(); - while let Some((_t0, _, request)) = rcv - .recv() - .await - .ok_or_else(|| SyneviError::ReceiveError("Channel closed".to_string()))? - { - match request { - BufferedMessage::Commit(req) => { - let clone = replica.clone(); - join_set.spawn(async move { - let res = HelperResponse::Commit(clone.commit(req, true).await?); - Ok::(res) - }); - } - BufferedMessage::Apply(req) => { - let clone = replica.clone(); - join_set.spawn(async move { - let res = HelperResponse::Apply(clone.apply(req, true).await?); - Ok::(res) - }); - } - } - } - for task in join_set.join_all().await { - task?; - } Ok(()) } @@ -431,32 +404,26 @@ where match state { State::Applied => { replica - .apply( - ApplyRequest { - id: event.id, - event: event.transaction, - timestamp_zero: event.t_zero, - timestamp: event.t, - dependencies: event.dependencies, - execution_hash: event.execution_hash, - transaction_hash: event.transaction_hash, - }, - false, - ) + .apply(ApplyRequest { + id: event.id, + event: event.transaction, + timestamp_zero: event.t_zero, + timestamp: event.t, + dependencies: event.dependencies, + execution_hash: event.execution_hash, + transaction_hash: event.transaction_hash, + }) .await?; } State::Commited => { replica - .commit( - CommitRequest { - id: event.id, - event: event.transaction, - timestamp_zero: event.t_zero, - timestamp: event.t, - dependencies: event.dependencies, - }, - false, - ) + .commit(CommitRequest { + id: event.id, + event: event.transaction, + timestamp_zero: event.t_zero, + timestamp: event.t, + dependencies: event.dependencies, + }) .await?; } _ => (), diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index e5572de..939057c 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -2,7 +2,6 @@ use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; use sha3::{Digest, Sha3_256}; -use std::collections::BTreeMap; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use synevi_network::configure_transport::{ @@ -16,14 +15,13 @@ use synevi_network::consensus_transport::{ TryRecoveryResponse, }; use synevi_network::network::Network; -use synevi_network::reconfiguration::{BufferedMessage, Reconfiguration, Report}; +use synevi_network::reconfiguration::{Reconfiguration, Report}; use synevi_network::replica::Replica; use synevi_types::traits::Store; use synevi_types::types::{ExecutorResult, InternalExecution, TransactionPayload, UpsertEvent}; use synevi_types::{Ballot, Executor, State, T, T0}; use synevi_types::{SyneviError, Transaction}; use tokio::sync::mpsc::{channel, Receiver, Sender}; -use tokio::sync::Mutex; use tracing::{instrument, trace}; use ulid::Ulid; @@ -34,7 +32,6 @@ where S: Store, { node: Arc>, - buffer: Arc>>, notifier: Sender, ready: Arc, configuring: Arc, @@ -51,7 +48,6 @@ where ( Self { node, - buffer: Arc::new(Mutex::new(BTreeMap::default())), notifier, ready, configuring: Arc::new(AtomicBool::new(false)), @@ -59,38 +55,6 @@ where receiver, ) } - - pub async fn send_buffered( - &self, - ) -> Result>, SyneviError> { - let (sdx, rcv) = channel(100); - let inner = self.buffer.clone(); - let node = self.node.clone(); - let configure_lock = self.configuring.clone(); - tokio::spawn(async move { - configure_lock.store(true, Ordering::SeqCst); - loop { - let event = inner.lock().await.pop_first(); - if let Some(((t0, state), event)) = event { - sdx.send(Some((t0, state, event))).await.map_err(|_| { - SyneviError::SendError( - "Channel for receiving buffered messages closed".to_string(), - ) - })?; - } else { - node.set_ready(); - sdx.send(None).await.map_err(|_| { - SyneviError::SendError( - "Channel for receiving buffered messages closed".to_string(), - ) - })?; - break; - } - } - Ok::<(), SyneviError>(()) - }); - Ok(rcv) - } } #[async_trait::async_trait] @@ -100,23 +64,15 @@ where E: Executor + Send + Sync, S: Store + Send + Sync, { - fn is_ready(&self) -> bool { - self.ready.load(Ordering::SeqCst) - } #[instrument(level = "trace", skip(self, request))] async fn pre_accept( &self, request: PreAcceptRequest, _node_serial: u16, - ready: bool, ) -> Result { let t0 = T0::try_from(request.timestamp_zero.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); - if !ready { - return Ok(PreAcceptResponse::default()); - } - trace!(?request_id, "Replica: PreAccept"); // TODO(performance): Remove the lock here @@ -149,6 +105,10 @@ where // let (t, deps) = rx.await?; + if !self.node.is_ready() { + return Ok(PreAcceptResponse::default()); + } + Ok(PreAcceptResponse { timestamp: t.into(), dependencies: into_dependency(&deps), @@ -157,19 +117,12 @@ where } #[instrument(level = "trace", skip(self, request))] - async fn accept( - &self, - request: AcceptRequest, - ready: bool, - ) -> Result { + async fn accept(&self, request: AcceptRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); let t = T::try_from(request.timestamp.as_slice())?; let request_ballot = Ballot::try_from(request.ballot.as_slice())?; - if !ready { - return Ok(AcceptResponse::default()); - } trace!(?request_id, "Replica: Accept"); let dependencies = { @@ -199,6 +152,10 @@ where self.node.event_store.get_tx_dependencies(&t, &t_zero) }; + + if !self.node.is_ready() { + return Ok(AcceptResponse::default()); + } Ok(AcceptResponse { dependencies: into_dependency(&dependencies), nack: false, @@ -206,21 +163,10 @@ where } #[instrument(level = "trace", skip(self, request))] - async fn commit( - &self, - request: CommitRequest, - ready: bool, - ) -> Result { + async fn commit(&self, request: CommitRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); - if !self.configuring.load(Ordering::SeqCst) && !ready { - self.buffer - .lock() - .await - .insert((t_zero, State::Commited), BufferedMessage::Commit(request)); - return Ok(CommitResponse {}); - } trace!(?request_id, "Replica: Commit"); @@ -242,21 +188,10 @@ where } #[instrument(level = "trace", skip(self, request))] - async fn apply( - &self, - request: ApplyRequest, - ready: bool, - ) -> Result { + async fn apply(&self, request: ApplyRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); - if !self.configuring.load(Ordering::SeqCst) && !ready { - self.buffer - .lock() - .await - .insert((t_zero, State::Applied), BufferedMessage::Apply(request)); - return Ok(ApplyResponse {}); - } trace!(?request_id, "Replica: Apply"); let transaction: TransactionPayload<::Tx> = @@ -334,12 +269,8 @@ where } #[instrument(level = "trace", skip(self))] - async fn recover( - &self, - request: RecoverRequest, - ready: bool, - ) -> Result { - if !ready { + async fn recover(&self, request: RecoverRequest) -> Result { + if !self.node.is_ready() { return Ok(RecoverResponse::default()); } let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); @@ -402,11 +333,10 @@ where async fn try_recover( &self, request: TryRecoveryRequest, - ready: bool, ) -> Result { let t0 = T0::try_from(request.timestamp_zero.as_slice())?; - if ready { + if !self.node.is_ready() { if let Some(recover_event) = self .node .event_store @@ -595,7 +525,6 @@ where fn clone(&self) -> Self { Self { node: self.node.clone(), - buffer: self.buffer.clone(), notifier: self.notifier.clone(), ready: self.ready.clone(), configuring: self.configuring.clone(), diff --git a/synevi_network/src/reconfiguration.rs b/synevi_network/src/reconfiguration.rs index 2738193..73fb44b 100644 --- a/synevi_network/src/reconfiguration.rs +++ b/synevi_network/src/reconfiguration.rs @@ -1,61 +1,11 @@ -use crate::{ - configure_transport::{ - GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, - ReadyElectorateRequest, ReadyElectorateResponse, ReportLastAppliedRequest, - ReportLastAppliedResponse, - }, - consensus_transport::{ApplyRequest, CommitRequest}, -}; -use std::{collections::BTreeMap, sync::Arc}; -use synevi_types::{SyneviError, T, T0}; -use tokio::sync::{ - mpsc::{channel, Receiver, Sender}, - Mutex, +use crate::configure_transport::{ + GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, + ReadyElectorateRequest, ReadyElectorateResponse, ReportLastAppliedRequest, + ReportLastAppliedResponse, }; +use synevi_types::{SyneviError, T}; use ulid::Ulid; -pub struct ReplicaBuffer { - inner: Arc>>, - _notifier: Sender, -} - -impl ReplicaBuffer { - pub fn new(sdx: Sender) -> Self { - ReplicaBuffer { - inner: Arc::new(Mutex::new(BTreeMap::new())), - _notifier: sdx, - } - } - - pub async fn send_buffered( - &self, - ) -> Result>, SyneviError> { - let (sdx, rcv) = channel(100); - let inner = self.inner.clone(); - tokio::spawn(async move { - loop { - let mut lock = inner.lock().await; - if let Some(event) = lock.pop_first() { - sdx.send(Some(event)).await.map_err(|_| { - SyneviError::SendError( - "Channel for receiving buffered messages closed".to_string(), - ) - })?; - } else { - sdx.send(None).await.map_err(|_| { - SyneviError::SendError( - "Channel for receiving buffered messages closed".to_string(), - ) - })?; - break; - } - } - Ok::<(), SyneviError>(()) - }); - Ok(rcv) - } -} - #[async_trait::async_trait] pub trait Reconfiguration { // Existing nodes @@ -79,12 +29,6 @@ pub trait Reconfiguration { ) -> Result; } -#[derive(Debug, Clone)] -pub enum BufferedMessage { - Commit(CommitRequest), - Apply(ApplyRequest), -} - #[derive(Debug)] pub struct Report { pub node_id: Ulid, diff --git a/synevi_network/src/replica.rs b/synevi_network/src/replica.rs index 7044d05..a850cd1 100644 --- a/synevi_network/src/replica.rs +++ b/synevi_network/src/replica.rs @@ -22,37 +22,20 @@ pub trait Replica: Send + Sync { &self, request: PreAcceptRequest, node_serial: u16, - ready: bool, ) -> Result; - async fn accept( - &self, - request: AcceptRequest, - ready: bool, - ) -> Result; + async fn accept(&self, request: AcceptRequest) -> Result; - async fn commit( - &self, - request: CommitRequest, - ready: bool, - ) -> Result; + async fn commit(&self, request: CommitRequest) -> Result; - async fn apply(&self, request: ApplyRequest, ready: bool) - -> Result; + async fn apply(&self, request: ApplyRequest) -> Result; - async fn recover( - &self, - request: RecoverRequest, - ready: bool, - ) -> Result; + async fn recover(&self, request: RecoverRequest) -> Result; async fn try_recover( &self, request: TryRecoveryRequest, - ready: bool, ) -> Result; - - fn is_ready(&self) -> bool; } pub struct ReplicaBox @@ -142,7 +125,7 @@ where Ok(Response::new( self.inner - .pre_accept(request, serial, self.inner.is_ready()) + .pre_accept(request, serial) .await .map_err(|e| Status::internal(e.to_string()))?, )) @@ -154,7 +137,7 @@ where ) -> Result, Status> { Ok(Response::new( self.inner - .accept(request.into_inner(), self.inner.is_ready()) + .accept(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) @@ -166,7 +149,7 @@ where ) -> Result, Status> { Ok(Response::new( self.inner - .commit(request.into_inner(), self.inner.is_ready()) + .commit(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) @@ -178,7 +161,7 @@ where ) -> Result, Status> { Ok(Response::new( self.inner - .apply(request.into_inner(), self.inner.is_ready()) + .apply(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) @@ -190,7 +173,7 @@ where ) -> Result, Status> { Ok(Response::new( self.inner - .recover(request.into_inner(), self.inner.is_ready()) + .recover(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) @@ -202,7 +185,7 @@ where ) -> Result, Status> { Ok(Response::new( self.inner - .try_recover(request.into_inner(), self.inner.is_ready()) + .try_recover(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index f644996..3efe317 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -373,6 +373,14 @@ impl InternalData { } let last_t = self.last_applied; + + // if last_t > event.t { + // println!( + // "Node: {:?}, store: {:?}", + // self.node_serial, + // self.get_event_store() + // ); + // } // Safeguard assert!(last_t < event.t); @@ -424,6 +432,18 @@ impl InternalData { if event.state == State::Applied { let last_t = self.last_applied; + + if last_t > event.t { + println!("last_t: {:?}, event.t: {:?}", last_t, event.t); + } + // println!( + // "Node: {:?}, last_t: {:?}, event_t: {:?}, store: {:?}", + // self.node_serial, + // last_t, + // event.t, + // self.get_event_store() + // ); + // } // Safeguard assert!(last_t < event.t); diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index 3d34107..15a4346 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -440,9 +440,9 @@ mod tests { let mut joinset = tokio::task::JoinSet::new(); - let random_number = rand::thread_rng().gen_range(0..999); - dbg!(&random_number); - for i in 0..1000 { + let num = 10; + let random_number: u128 = rand::thread_rng().gen_range(0..num - 1); + for i in 0..num { if i == random_number { let id = Ulid::new(); let network = synevi_network::network::GrpcNetwork::new( diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index 49affd1..f5650c5 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -445,7 +445,6 @@ pub(crate) async fn replica_dispatch( last_applied: last_applied.clone(), }, node as u16, - true, ) .await .unwrap(); @@ -473,18 +472,15 @@ pub(crate) async fn replica_dispatch( ref last_applied, } => { let response = replica - .accept( - AcceptRequest { - id: id.clone(), - ballot: ballot.clone(), - event: event.clone(), - timestamp_zero: t0.clone(), - timestamp: t.clone(), - dependencies: deps.clone(), - last_applied: last_applied.clone(), - }, - true, - ) + .accept(AcceptRequest { + id: id.clone(), + ballot: ballot.clone(), + event: event.clone(), + timestamp_zero: t0.clone(), + timestamp: t.clone(), + dependencies: deps.clone(), + last_applied: last_applied.clone(), + }) .await?; let reply = msg.reply(Body { @@ -507,16 +503,13 @@ pub(crate) async fn replica_dispatch( ref deps, } => { replica - .commit( - CommitRequest { - id: id.clone(), - event: event.clone(), - timestamp_zero: t0.clone(), - timestamp: t.clone(), - dependencies: deps.clone(), - }, - true, - ) + .commit(CommitRequest { + id: id.clone(), + event: event.clone(), + timestamp_zero: t0.clone(), + timestamp: t.clone(), + dependencies: deps.clone(), + }) .await?; let reply = msg.reply(Body { @@ -538,18 +531,15 @@ pub(crate) async fn replica_dispatch( } => { eprintln!("Replica dispatch apply {:?}", t0); replica - .apply( - ApplyRequest { - id: id.clone(), - event: event.clone(), - timestamp_zero: t0.clone(), - timestamp: t.clone(), - dependencies: deps.clone(), - transaction_hash: transaction_hash.clone(), - execution_hash: execution_hash.clone(), - }, - true, - ) + .apply(ApplyRequest { + id: id.clone(), + event: event.clone(), + timestamp_zero: t0.clone(), + timestamp: t.clone(), + dependencies: deps.clone(), + transaction_hash: transaction_hash.clone(), + execution_hash: execution_hash.clone(), + }) .await?; let reply = msg.reply(Body { @@ -567,15 +557,12 @@ pub(crate) async fn replica_dispatch( ref t0, } => { let result = replica - .recover( - RecoverRequest { - id: id.clone(), - ballot: ballot.clone(), - event: event.clone(), - timestamp_zero: t0.clone(), - }, - true, - ) + .recover(RecoverRequest { + id: id.clone(), + ballot: ballot.clone(), + event: event.clone(), + timestamp_zero: t0.clone(), + }) .await?; let reply = msg.reply(Body { From b546bdba965b45039c95fe089afaa5a8e12be1c2 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 10:12:16 +0200 Subject: [PATCH 13/40] feat: Update type signature for SyneviResult / Transaction --- benches/performance.rs | 27 +------ failed.txt | 144 --------------------------------- synevi_core/src/coordinator.rs | 32 +++++--- synevi_core/src/lib.rs | 2 +- synevi_core/src/node.rs | 28 ++++--- synevi_core/src/replica.rs | 7 +- synevi_kv/src/kv_store.rs | 20 ++--- synevi_types/src/error.rs | 3 +- synevi_types/src/types.rs | 5 +- tests/Cargo.toml | 5 +- tests/consensus_e2e.rs | 39 ++------- 11 files changed, 65 insertions(+), 247 deletions(-) delete mode 100644 failed.txt diff --git a/benches/performance.rs b/benches/performance.rs index dc4992f..ac798f8 100644 --- a/benches/performance.rs +++ b/benches/performance.rs @@ -57,14 +57,7 @@ async fn parallel_execution(coordinator: Arc { - res.unwrap(); - } - synevi_types::types::ExecutorResult::Internal(res) => { - res.unwrap(); - } - } + res.unwrap().unwrap().unwrap(); } } @@ -82,14 +75,7 @@ async fn contention_execution(coordinators: Vec { - res.unwrap(); - } - synevi_types::types::ExecutorResult::Internal(res) => { - res.unwrap(); - } - } + res.unwrap().unwrap().unwrap(); } } @@ -105,14 +91,7 @@ async fn _bigger_payloads_execution( joinset.spawn(async move { coordinator.transaction(i, payload).await }); } while let Some(res) = joinset.join_next().await { - match res.unwrap().unwrap() { - synevi_types::types::ExecutorResult::External(res) => { - res.unwrap(); - } - synevi_types::types::ExecutorResult::Internal(res) => { - res.unwrap(); - } - }; + res.unwrap().unwrap().unwrap(); } } diff --git a/failed.txt b/failed.txt deleted file mode 100644 index b28de12..0000000 --- a/failed.txt +++ /dev/null @@ -1,144 +0,0 @@ - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 4 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 4 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 1 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.00s - - -running 0 tests - -test result: ok. 0 passed; 0 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.00s - - -running 1 test -[0]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -Returning streaming receiver -[1]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -[0]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[1]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[0]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -[1]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -Returning streaming receiver -[2]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -[2]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[2]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -[1]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[2]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[0]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[0]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -[2]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -[1]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -Returning streaming receiver -[3]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -[3]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[3]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -[3]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[3]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -[1]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[3]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[0]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[2]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[0]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -[2]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -[3]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -[1]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -Returning streaming receiver -[4]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -[4]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[4]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -[4]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[4]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -[4]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[4]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -[0]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[4]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[3]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[1]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[2]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[0]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[4]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[3]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[2]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[1]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[0]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[4]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[3]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[2]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[1]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[0]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -[0]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[1]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -[3]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -[4]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -[2]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -[1]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[4]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[2]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[3]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[0]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) -[2]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) -[3]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) -[1]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) -[4]Applied event: t0: T0(MonoTime(486696121103639053161822102224896)), t: T(MonoTime(486696121103704477798038138650624)) -[0]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) -[4]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) -[3]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) -[1]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) -[2]Applied event: t0: T0(MonoTime(486696121103637285217489087627264)), t: T(MonoTime(486696121103707523638756829822976)) -[0]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) -[2]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) -[1]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) -[3]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) -[4]Applied event: t0: T0(MonoTime(486696121103740599481803711971328)), t: T(MonoTime(486696121103740599481803711971328)) -Returning streaming receiver -[6]Applied event: t0: T0(MonoTime(486696121079619795288778575183872)), t: T(MonoTime(486696121079619795288778575183872)) -[6]Applied event: t0: T0(MonoTime(486696121079868950182767701786624)), t: T(MonoTime(486696121079868950182767701786624)) -[6]Applied event: t0: T0(MonoTime(486696121080077696810575223324672)), t: T(MonoTime(486696121080077696810575223324672)) -[6]Applied event: t0: T0(MonoTime(486696121080498828130259956727808)), t: T(MonoTime(486696121080498828130259956727808)) -[6]Applied event: t0: T0(MonoTime(486696121080696542909305257459712)), t: T(MonoTime(486696121080696542909305257459712)) -[6]Applied event: t0: T0(MonoTime(486696121081161790927989758820352)), t: T(MonoTime(486696121081161790927989758820352)) -[6]Applied event: t0: T0(MonoTime(486696121081357072074386419220480)), t: T(MonoTime(486696121081357072074386419220480)) -[6]Applied event: t0: T0(MonoTime(486696121103489372087531000037376)), t: T(MonoTime(486696121103489372087531000037376)) -[6]Applied event: t0: T0(MonoTime(486696121103636267122494030217216)), t: T(MonoTime(486696121103679170945823446990848)) -[6]Applied event: t0: T0(MonoTime(486696121103634764046109805379584)), t: T(MonoTime(486696121103689131219340740329472)) -[6]Applied event: t0: T0(MonoTime(486696121103632978931803211431936)), t: T(MonoTime(486696121103699508920252790603776)) -[6]Applied event: t0: T0(MonoTime(486696121103635542324424705310720)), t: T(MonoTime(486696121103698547401740937068544)) -last_t: T(MonoTime(486696121103699508920252790603776)), event.t: T(MonoTime(486696121103698547401740937068544)) -test tests::reconfiguration ... FAILED - -failures: - -failures: - tests::reconfiguration - -test result: FAILED. 0 passed; 1 failed; 0 ignored; 0 measured; 3 filtered out; finished in 0.09s - diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 9019a42..bb80627 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -14,13 +14,13 @@ use synevi_network::network::{BroadcastRequest, Network, NetworkInterface}; use synevi_network::utils::IntoInner; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalExecution, RecoverEvent, RecoveryState, SyneviResult, + ExecutorResult, Hashes, InternalExecution, InternalSyneviResult, RecoverEvent, RecoveryState, TransactionPayload, }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; use tracing::{instrument, trace}; -type RecoverySyneviResult = +type RecoveryInternalSyneviResult = Result::Tx>>, SyneviError>; pub struct Coordinator @@ -82,12 +82,12 @@ where } #[instrument(level = "trace", skip(self))] - pub async fn run(&mut self) -> SyneviResult { + pub async fn run(&mut self) -> InternalSyneviResult { self.pre_accept().await } #[instrument(level = "trace", skip(self))] - async fn pre_accept(&mut self) -> SyneviResult { + async fn pre_accept(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Preaccept"); self.node @@ -162,7 +162,7 @@ where } #[instrument(level = "trace", skip(self))] - async fn accept(&mut self) -> SyneviResult { + async fn accept(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Accept"); // Safeguard: T0 <= T @@ -227,7 +227,7 @@ where } #[instrument(level = "trace", skip(self))] - async fn commit(&mut self) -> SyneviResult { + async fn commit(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Commit"); let committed_request = CommitRequest { @@ -258,7 +258,7 @@ where } #[instrument(level = "trace", skip(self))] - async fn apply(&mut self) -> SyneviResult { + async fn apply(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Apply"); let (synevi_result, hashes) = self.execute_consensus().await?; @@ -282,14 +282,21 @@ where } #[instrument(level = "trace", skip(self))] - async fn execute_consensus(&mut self) -> Result<(SyneviResult, Hashes), SyneviError> { + async fn execute_consensus( + &mut self, + ) -> Result<(InternalSyneviResult, Hashes), SyneviError> { self.transaction.state = State::Applied; self.node.apply((&self.transaction).into()).await?; let result = match &self.transaction.transaction { TransactionPayload::None => Err(SyneviError::TransactionNotFound), - TransactionPayload::External(tx) => self.node.executor.execute(tx.clone()).await, + TransactionPayload::External(tx) => self + .node + .executor + .execute(tx.clone()) + .await + .map(|e| ExecutorResult::External(e)), TransactionPayload::Internal(request) => { let result = match request { InternalExecution::JoinElectorate { id, serial, host } => { @@ -326,7 +333,10 @@ where } #[instrument(level = "trace", skip(node))] - pub async fn recover(node: Arc>, recover_event: RecoverEvent) -> SyneviResult { + pub async fn recover( + node: Arc>, + recover_event: RecoverEvent, + ) -> InternalSyneviResult { loop { let node = node.clone(); @@ -380,7 +390,7 @@ where async fn recover_consensus( &mut self, mut responses: Vec, - ) -> RecoverySyneviResult { + ) -> RecoveryInternalSyneviResult { // Keep track of values to replace let mut highest_ballot: Option = None; let mut superseding = false; diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 0193f1b..bde2307 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -114,7 +114,7 @@ pub mod tests { type Tx = Vec; async fn execute(&self, data: Vec) -> SyneviResult { - Ok(synevi_types::types::ExecutorResult::External(Ok(data))) + Ok(Ok(data)) } } } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 1c97658..65224ee 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -11,7 +11,9 @@ use synevi_network::reconfiguration::Report; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; -use synevi_types::types::{SyneviResult, TransactionPayload, UpsertEvent}; +use synevi_types::types::{ + ExecutorResult, InternalSyneviResult, SyneviResult, TransactionPayload, UpsertEvent, +}; use synevi_types::{Executor, State, SyneviError, T}; use tokio::sync::mpsc::Receiver; use tracing::instrument; @@ -206,14 +208,21 @@ where let _permit = self.semaphore.acquire().await?; let mut coordinator = Coordinator::new(self.clone(), TransactionPayload::External(transaction), id).await; - coordinator.run().await + let tx_result = coordinator.run().await?; + + match tx_result { + ExecutorResult::External(e) => Ok(e), + ExecutorResult::Internal(e) => { + Err(SyneviError::InternalTransaction(format!("{:?}", e))) + } + } } pub(super) async fn internal_transaction( self: Arc, id: u128, transaction: TransactionPayload, - ) -> SyneviResult { + ) -> InternalSyneviResult { if !self.has_members.load(std::sync::atomic::Ordering::Relaxed) { tracing::warn!("Consensus omitted: No members in the network"); } else if !self.is_ready.load(Ordering::Relaxed) { @@ -444,7 +453,6 @@ mod tests { use synevi_network::network::GrpcNetwork; use synevi_network::network::Network; use synevi_types::traits::Store; - use synevi_types::types::ExecutorResult; use synevi_types::{Executor, State, SyneviError, T, T0}; use ulid::Ulid; @@ -569,15 +577,12 @@ mod tests { .await .unwrap(); } - match coordinator + coordinator .clone() .transaction(0, Vec::from("last transaction")) .await .unwrap() - { - ExecutorResult::External(e) => e.unwrap(), - _ => panic!(), - }; + .unwrap(); let coordinator_store: BTreeMap = coordinator .event_store @@ -641,10 +646,7 @@ mod tests { .await .unwrap(); - let result = match node.transaction(0, vec![127u8]).await.unwrap() { - ExecutorResult::External(e) => e.unwrap(), - _ => panic!(), - }; + let result = node.transaction(0, vec![127u8]).await.unwrap().unwrap(); assert_eq!(result, vec![127u8]); } diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 939057c..90c84e4 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -217,7 +217,12 @@ where TransactionPayload::None => { return Err(SyneviError::TransactionNotFound); } - TransactionPayload::External(tx) => self.node.executor.execute(tx).await, + TransactionPayload::External(tx) => self + .node + .executor + .execute(tx) + .await + .map(|e| ExecutorResult::<::Tx>::External(e)), TransactionPayload::Internal(request) => { // TODO: Build special execution let result = match &request { diff --git a/synevi_kv/src/kv_store.rs b/synevi_kv/src/kv_store.rs index abdf532..ddc5cc3 100644 --- a/synevi_kv/src/kv_store.rs +++ b/synevi_kv/src/kv_store.rs @@ -6,7 +6,7 @@ use std::fmt::Debug; use std::sync::{Arc, Mutex}; use synevi_core::node::Node; use synevi_network::network::Network; -use synevi_types::types::{ExecutorResult, SyneviResult}; +use synevi_types::types::SyneviResult; use synevi_types::{error::SyneviError, Executor}; use ulid::Ulid; @@ -62,29 +62,29 @@ impl Executor for KVExecutor { Ok(match transaction { Transaction::Read { key } => { let Some(key) = self.store.lock().unwrap().get(&key).cloned() else { - return Ok(ExecutorResult::External(Err(KVError::KeyNotFound))); + return Ok(Err(KVError::KeyNotFound)); }; - ExecutorResult::External(Ok(key)) + Ok(key) } Transaction::Write { key, value } => { self.store .lock() .unwrap() .insert(key.clone(), value.clone()); - ExecutorResult::External(Ok(value)) + Ok(value) } Transaction::Cas { key, from, to } => { let mut store = self.store.lock().unwrap(); let Some(entry) = store.get_mut(&key) else { - return Ok(ExecutorResult::External(Err(KVError::KeyNotFound))); + return Ok(Err(KVError::KeyNotFound)); }; if entry == &from { *entry = to.clone(); - ExecutorResult::External(Ok(to)) + Ok(to) } else { - return Ok(ExecutorResult::External(Err(KVError::MismatchError))); + return Ok(Err(KVError::MismatchError)); } } }) @@ -116,13 +116,9 @@ where async fn transaction(&self, id: Ulid, transaction: Transaction) -> Result { let node = self.node.clone(); - match node + node .transaction(u128::from_be_bytes(id.to_bytes()), transaction) .await? - { - ExecutorResult::External(result) => result, - _ => Err(KVError::MismatchError), // TODO: Make a new error for this case - } } pub async fn read(&self, key: String) -> Result { diff --git a/synevi_types/src/error.rs b/synevi_types/src/error.rs index 8112c3a..1dcb56e 100644 --- a/synevi_types/src/error.rs +++ b/synevi_types/src/error.rs @@ -68,9 +68,10 @@ pub enum SyneviError { NotReady, #[error("Mismatched hashes")] MismatchedHashes, - #[error("Unrecoverable transaction")] UnrecoverableTransaction, + #[error("Expected external transaction: {0}")] + InternalTransaction(String), } impl Serialize for SyneviError { diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index d9e3da0..c46aa8a 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -13,11 +13,12 @@ use tokio::sync::oneshot; use ulid::Ulid; pub type SyneviResult = Result< - ExecutorResult<::Tx>, - //Result<<::Tx as Transaction>::TxOk, <::Tx as Transaction>::TxErr>, + Result<<::Tx as Transaction>::TxOk, <::Tx as Transaction>::TxErr>, SyneviError, >; +pub type InternalSyneviResult = Result::Tx>, SyneviError>; + #[derive(Serialize)] pub enum ExecutorResult { External(Result), diff --git a/tests/Cargo.toml b/tests/Cargo.toml index f707ea4..5962a5c 100644 --- a/tests/Cargo.toml +++ b/tests/Cargo.toml @@ -40,7 +40,4 @@ path = "maelstrom/echo.rs" [[bin]] name = "maelstrom_lin_kv" -path = "maelstrom/lin_kv.rs" - -[profile.release] -panic = "abort" +path = "maelstrom/lin_kv.rs" \ No newline at end of file diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index 15a4346..f0684ad 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -62,14 +62,7 @@ mod tests { }); } while let Some(res) = joinset.join_next().await { - match res.unwrap().unwrap() { - synevi::ExecutorResult::External(res) => { - res.unwrap(); - } - synevi::ExecutorResult::Internal(res) => { - res.unwrap(); - } - }; + res.unwrap().unwrap().unwrap(); } let (total, accepts, recovers) = coordinator.get_stats(); @@ -212,14 +205,7 @@ mod tests { }); } while let Some(res) = joinset.join_next().await { - match res.unwrap().unwrap() { - synevi::ExecutorResult::External(res) => { - res.unwrap(); - } - synevi::ExecutorResult::Internal(res) => { - res.unwrap(); - } - }; + res.unwrap().unwrap().unwrap(); } println!("Time: {:?}", start.elapsed()); @@ -361,19 +347,11 @@ mod tests { } for i in 0..1000 { - match coordinator + coordinator .clone() .transaction(i, Vec::from("This is a transaction")) .await - .unwrap() - { - synevi::ExecutorResult::Internal(res) => { - res.unwrap(); - } - synevi::ExecutorResult::External(res) => { - res.unwrap(); - } - }; + .unwrap().unwrap(); } runtime.shutdown_background(); @@ -480,14 +458,7 @@ mod tests { } } while let Some(res) = joinset.join_next().await { - match res.unwrap().unwrap() { - synevi::ExecutorResult::External(res) => { - res.unwrap(); - } - synevi::ExecutorResult::Internal(res) => { - res.unwrap(); - } - }; + res.unwrap().unwrap().unwrap(); } let (total, accepts, recovers) = coordinator.get_stats(); From 35d63966b29951b7dbd68a13368084175936cc5f Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 12:13:12 +0200 Subject: [PATCH 14/40] refactor: Reconfiguration --- synevi_core/src/coordinator.rs | 7 +- synevi_core/src/lib.rs | 7 +- synevi_core/src/node.rs | 44 ++----- synevi_core/src/replica.rs | 75 +++++------ synevi_kv/src/kv_store.rs | 3 +- synevi_network/src/network.rs | 117 ++++++++---------- .../src/protos/configure_transport.proto | 11 +- synevi_network/src/reconfiguration.rs | 10 +- synevi_network/src/replica.rs | 10 +- synevi_types/src/types.rs | 31 +++-- tests/consensus_e2e.rs | 3 +- tests/maelstrom/network.rs | 2 +- 12 files changed, 139 insertions(+), 181 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index bb80627..74baedb 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -299,15 +299,14 @@ where .map(|e| ExecutorResult::External(e)), TransactionPayload::Internal(request) => { let result = match request { - InternalExecution::JoinElectorate { id, serial, host } => { + InternalExecution::JoinElectorate { id, serial, new_node_host } => { let res = self .node - .add_member(*id, *serial, host.clone(), false) + .add_member(*id, *serial, new_node_host.clone(), false) .await; - let (t, hash) = self.node.event_store.last_applied_hash()?; // TODO: Remove ? self.node .network - .report_config(t, hash, host.clone()) + .report_config(new_node_host.to_string()) .await?; res } diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index bde2307..c761a92 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -15,7 +15,6 @@ pub mod tests { use synevi_types::types::SyneviResult; use synevi_types::Executor; use synevi_types::SyneviError; - use synevi_types::T; use synevi_types::T0; use tokio::sync::mpsc::Receiver; use tokio::sync::Mutex; @@ -78,8 +77,8 @@ pub mod tests { 0 } - async fn broadcast_config(&self, _host: String) -> Result<(u32, Vec), SyneviError> { - Ok((0, vec![0])) + async fn join_electorate(&self, _host: String) -> Result { + Ok(0) } async fn get_stream_events( &self, @@ -99,8 +98,6 @@ pub mod tests { async fn report_config( &self, - _last_applied: T, - _last_applied_hash: [u8; 32], _host: String, ) -> Result<(), SyneviError> { Ok(()) diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 65224ee..dd0d884 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -7,7 +7,6 @@ use std::sync::RwLock; use std::sync::{atomic::AtomicU64, Arc}; use synevi_network::consensus_transport::{ApplyRequest, CommitRequest}; use synevi_network::network::{Network, NetworkInterface, NodeInfo}; -use synevi_network::reconfiguration::Report; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; @@ -15,7 +14,6 @@ use synevi_types::types::{ ExecutorResult, InternalSyneviResult, SyneviResult, TransactionPayload, UpsertEvent, }; use synevi_types::{Executor, State, SyneviError, T}; -use tokio::sync::mpsc::Receiver; use tracing::instrument; use ulid::Ulid; @@ -32,15 +30,12 @@ where E: Executor + Send + Sync, S: Store + Send + Sync, { - pub info: NodeInfo, pub network: N, pub executor: E, pub event_store: Arc, pub stats: Stats, pub wait_handler: WaitHandler, semaphore: Arc, - has_members: AtomicBool, - is_ready: Arc, self_clone: RwLock>>, } @@ -75,7 +70,6 @@ where executor: E, store: S, ) -> Result, SyneviError> { - let node_name = NodeInfo { id, serial }; let stats = Stats { total_requests: AtomicU64::new(0), @@ -94,7 +88,6 @@ where let wait_handler = WaitHandler::new(arc_store.clone()); let node = Arc::new(Node { - info: node_name, event_store: arc_store, wait_handler, network, @@ -102,7 +95,7 @@ where semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, has_members: AtomicBool::new(false), - is_ready: Arc::new(AtomicBool::new(true)), + members_responded: Arc::new(AtomicU64::new(0)), self_clone: RwLock::new(None), }); node.self_clone @@ -356,51 +349,30 @@ where &self, replica: ReplicaConfig, member_host: String, - config_receiver: Receiver, ready: Arc, ) -> Result<(), SyneviError> { // 1. Broadcast self_config to other member - let (all_members, self_id) = self.network.broadcast_config(member_host).await?; + let expected = self.network.join_electorate(member_host).await?; // 2. wait for JoinElectorate responses with expected majority and config from others - self.join_electorate(config_receiver, all_members, self_id, &replica) - .await?; - - // 3. Send ReadyJoinElectorate && set myself to ready - ready.store(true, Ordering::Relaxed); - self.network.ready_electorate().await?; - Ok(()) - } - async fn join_electorate( - &self, - mut receiver: Receiver, - all_members: u32, - self_id: Vec, - replica: &ReplicaConfig, - ) -> Result<(), SyneviError> { - let mut member_count = 0; - while let Some(report) = receiver.recv().await { - self.add_member(report.node_id, report.node_serial, report.node_host, true) - .await?; - member_count += 1; - if member_count >= all_members { - break; - } + while self.members_responded.load(Ordering::Relaxed) < self.network.get_member_len().await as u64 { + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } - let (last_applied, _) = self.event_store.last_applied(); - // 2.1 if majority replies with 0 events -> skip to 2.4. + let (last_applied, _) = self.event_store.last_applied(); self.sync_events(last_applied, self_id, &replica).await?; + // 3. Send ReadyJoinElectorate && set myself to ready + ready.store(true, Ordering::Relaxed); + self.network.ready_electorate().await?; Ok(()) } async fn sync_events( &self, last_applied: T, - self_id: Vec, replica: &ReplicaConfig, ) -> Result<(), SyneviError> { // 2.2 else Request stream with events until last_applied (highest t of JoinElectorate) diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 90c84e4..b8191a0 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -5,9 +5,7 @@ use sha3::{Digest, Sha3_256}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use synevi_network::configure_transport::{ - Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, - ReadyElectorateRequest, ReadyElectorateResponse, ReportLastAppliedRequest, - ReportLastAppliedResponse, + Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, ReadyElectorateRequest, ReadyElectorateResponse, ReportElectorateRequest, ReportElectorateResponse, ReportLastAppliedRequest, ReportLastAppliedResponse }; use synevi_network::consensus_transport::{ AcceptRequest, AcceptResponse, ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, @@ -70,6 +68,10 @@ where request: PreAcceptRequest, _node_serial: u16, ) -> Result { + if !self.node.is_ready() { + return Ok(PreAcceptResponse::default()); + } + let t0 = T0::try_from(request.timestamp_zero.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); @@ -105,10 +107,6 @@ where // let (t, deps) = rx.await?; - if !self.node.is_ready() { - return Ok(PreAcceptResponse::default()); - } - Ok(PreAcceptResponse { timestamp: t.into(), dependencies: into_dependency(&deps), @@ -118,6 +116,9 @@ where #[instrument(level = "trace", skip(self, request))] async fn accept(&self, request: AcceptRequest) -> Result { + if !self.node.is_ready() { + return Ok(AcceptResponse::default()); + } let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); let t = T::try_from(request.timestamp.as_slice())?; @@ -153,9 +154,6 @@ where self.node.event_store.get_tx_dependencies(&t, &t_zero) }; - if !self.node.is_ready() { - return Ok(AcceptResponse::default()); - } Ok(AcceptResponse { dependencies: into_dependency(&dependencies), nack: false, @@ -226,17 +224,17 @@ where TransactionPayload::Internal(request) => { // TODO: Build special execution let result = match &request { - InternalExecution::JoinElectorate { id, serial, host } => { + InternalExecution::JoinElectorate { + id, + serial, + new_node_host, + } => { if id != &self.node.info.id { let res = self .node - .add_member(*id, *serial, host.clone(), false) + .add_member(*id, *serial, new_node_host.clone(), false) .await; - let (t, hash) = self.node.event_store.last_applied_hash()?; - self.node - .network - .report_config(t, hash, host.clone()) - .await?; + self.node.network.report_config(new_node_host.clone()).await?; res } else { Ok(()) @@ -485,38 +483,23 @@ where } // TODO: Move trait to Joining Node -> Rename to receive_config, Ready checks - async fn report_last_applied( + async fn report_electorate( &self, - request: ReportLastAppliedRequest, - ) -> Result { - if self.ready.load(Ordering::SeqCst) { + request: ReportElectorateRequest, + ) -> Result { + if self.ready.load(Ordering::Relaxed) { return Ok(ReportLastAppliedResponse::default()); } - let Some(Config { - node_serial, - node_id, - host, - }) = request.config - else { - return Err(SyneviError::InvalidConversionRequest( - "Invalid config".to_string(), - )); - }; - let report = Report { - node_id: Ulid::from_bytes(node_id.try_into().map_err(|_| { - SyneviError::InvalidConversionFromBytes("Invalid Ulid conversion".to_string()) - })?), - node_serial: node_serial.try_into()?, - node_host: host, - last_applied: request.last_applied.as_slice().try_into()?, - last_applied_hash: request.last_applied_hash.try_into().map_err(|_| { - SyneviError::InvalidConversionFromBytes("Invalid hash conversion".to_string()) - })?, - }; - //dbg!(&report); - self.notifier.send(report).await.map_err(|_| { - SyneviError::SendError("Sender for reporting last applied closed".to_string()) - })?; + for member in request.config { + self.node.add_member( + member.node_id, + member.node_serial, + member.host, + true, + ) + .await?; + } + self.node.members_responded.fetch_add(1, Ordering::Relaxed); Ok(ReportLastAppliedResponse {}) } } diff --git a/synevi_kv/src/kv_store.rs b/synevi_kv/src/kv_store.rs index ddc5cc3..573da55 100644 --- a/synevi_kv/src/kv_store.rs +++ b/synevi_kv/src/kv_store.rs @@ -116,8 +116,7 @@ where async fn transaction(&self, id: Ulid, transaction: Transaction) -> Result { let node = self.node.clone(); - node - .transaction(u128::from_be_bytes(id.to_bytes()), transaction) + node.transaction(u128::from_be_bytes(id.to_bytes()), transaction) .await? } diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index c4675e7..9334287 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -5,7 +5,7 @@ use crate::configure_transport::reconfiguration_service_server::ReconfigurationS use crate::configure_transport::time_service_server::TimeServiceServer; use crate::configure_transport::{ Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, ReadyElectorateRequest, - ReportLastAppliedRequest, + ReportElectorateRequest, }; use crate::consensus_transport::{RecoverRequest, RecoverResponse, TryRecoveryRequest}; use crate::latency_service::get_latency; @@ -20,10 +20,10 @@ use crate::{ replica::{Replica, ReplicaBox}, }; use std::collections::HashMap; -use std::sync::atomic::{AtomicI64, AtomicU64, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicI64, AtomicU64, Ordering}; use std::{net::SocketAddr, sync::Arc}; use synevi_types::error::SyneviError; -use synevi_types::{T, T0}; +use synevi_types::T0; use tokio::sync::{Mutex, RwLock}; use tokio::task::JoinSet; use tonic::metadata::{AsciiMetadataKey, AsciiMetadataValue}; @@ -57,13 +57,8 @@ pub trait Network: Send + Sync + 'static { async fn get_interface(&self) -> Arc; async fn get_waiting_time(&self, node_serial: u16) -> u64; async fn get_member_len(&self) -> u32; - async fn broadcast_config(&self, host: String) -> Result<(u32, Vec), SyneviError>; // All members - async fn report_config( - &self, - last_applied: T, - last_applied_hash: [u8; 32], - host: String, - ) -> Result<(), SyneviError>; + async fn join_electorate(&self, host: String) -> Result; // All members + async fn report_config(&self, host: String) -> Result<(), SyneviError>; async fn get_stream_events( &self, last_applied: Vec, @@ -113,8 +108,8 @@ where async fn get_member_len(&self) -> u32 { self.as_ref().get_member_len().await } - async fn broadcast_config(&self, host: String) -> Result<(u32, Vec), SyneviError> { - self.as_ref().broadcast_config(host).await + async fn join_electorate(&self, host: String) -> Result { + self.as_ref().join_electorate(host).await } async fn get_stream_events( &self, @@ -134,15 +129,8 @@ where self.as_ref().ready_member(id, serial).await } - async fn report_config( - &self, - last_applied: T, - last_applied_hash: [u8; 32], - host: String, - ) -> Result<(), SyneviError> { - self.as_ref() - .report_config(last_applied, last_applied_hash, host) - .await + async fn report_config(&self, host: String) -> Result<(), SyneviError> { + self.as_ref().report_config(host).await } } @@ -166,14 +154,14 @@ where pub struct NodeInfo { pub id: Ulid, pub serial: u16, + pub host: String, + pub ready: Arc, } #[derive(Clone, Debug)] pub struct Member { pub info: NodeInfo, - pub host: String, pub channel: Channel, - pub ready_electorate: bool, } #[derive(Debug)] @@ -204,7 +192,7 @@ pub enum BroadcastResponse { #[derive(Debug)] pub struct GrpcNetwork { pub socket_addr: SocketAddr, - pub self_info: (NodeInfo, String), + pub self_info: NodeInfo, pub members: Arc>>, join_set: Mutex>>, } @@ -215,16 +203,16 @@ pub struct GrpcNetworkSet { } impl GrpcNetwork { - pub fn new(socket_addr: SocketAddr, address: String, node_id: Ulid, node_serial: u16) -> Self { + pub fn new(socket_addr: SocketAddr, host: String, node_id: Ulid, node_serial: u16, ready: Arc) -> Self { Self { socket_addr, - self_info: ( + self_info: NodeInfo { id: node_id, serial: node_serial, + host, + ready, }, - address, - ), members: Arc::new(RwLock::new(HashMap::default())), join_set: Mutex::new(JoinSet::new()), } @@ -267,10 +255,8 @@ impl Network for GrpcNetwork { id, MemberWithLatency { member: Arc::new(Member { - info: NodeInfo { id, serial }, - host, + info: NodeInfo { id, serial, host, ready: Arc::new(AtomicBool::new(ready)) }, channel, - ready_electorate: ready, }), latency: AtomicU64::new(500), skew: AtomicI64::new(0), @@ -328,11 +314,12 @@ impl Network for GrpcNetwork { (self.members.read().await.len() + 1) as u32 } - async fn broadcast_config(&self, host: String) -> Result<(u32, Vec), SyneviError> { + async fn join_electorate(&self, host: String) -> Result { let config = Config { - node_serial: self.self_info.0.serial as u32, - node_id: self.self_info.0.id.to_bytes().to_vec(), - host: self.self_info.1.clone(), + node_serial: self.self_info.serial as u32, + node_id: self.self_info.id.to_bytes().to_vec(), + host: self.self_info.host.clone(), + ready: self.self_info.ready.load(Ordering::Relaxed), }; let channel = Channel::from_shared(host)?.connect().await?; let request = tonic::Request::new(JoinElectorateRequest { @@ -341,28 +328,35 @@ impl Network for GrpcNetwork { let mut client = ReconfigurationServiceClient::new(channel); let response = client.join_electorate(request).await?; let response = response.into_inner(); - Ok((response.majority, response.self_event)) + Ok(response.majority) } - async fn report_config( - &self, - last_applied: T, - last_applied_hash: [u8; 32], - host: String, - ) -> Result<(), SyneviError> { + async fn report_config(&self, host: String) -> Result<(), SyneviError> { let config = Config { - node_serial: self.self_info.0.serial as u32, - node_id: self.self_info.0.id.to_bytes().to_vec(), - host: self.self_info.1.clone(), + node_serial: self.self_info.serial as u32, + node_id: self.self_info.id.to_bytes().to_vec(), + host: self.self_info.host.clone(), + ready: self.self_info.ready.load(Ordering::Relaxed), }; + + let mut configs: Vec<_> = self + .members + .read() + .await + .iter() + .map(|(_, m)| Config { + node_serial: m.member.info.serial as u32, + node_id: m.member.info.id.to_bytes().to_vec(), + host: m.member.info.host.clone(), + ready: m.member.info.ready.load(Ordering::Relaxed), + }) + .collect(); + configs.push(config); + let channel = Channel::from_shared(host)?.connect().await?; - let request = tonic::Request::new(ReportLastAppliedRequest { - config: Some(config), - last_applied: last_applied.into(), - last_applied_hash: last_applied_hash.into(), - }); + let request = tonic::Request::new(ReportElectorateRequest { configs }); let mut client = InitServiceClient::new(channel); - let _res = client.report_last_applied(request).await?; + let _res = client.report_electorate(request).await?; Ok(()) } @@ -373,7 +367,7 @@ impl Network for GrpcNetwork { ) -> Result, SyneviError> { let lock = self.members.read().await; let mut members = lock.iter(); - let Some((_, member)) = members.find(|(_, m)| m.member.ready_electorate) else { + let Some((_, member)) = members.find(|(_, m)| m.member.info.ready.load(Ordering::Relaxed)) else { return Err(SyneviError::NoMembersFound); }; let channel = member.member.channel.clone(); @@ -414,8 +408,8 @@ impl Network for GrpcNetwork { }; let channel = member.member.channel.clone(); let request = tonic::Request::new(ReadyElectorateRequest { - node_id: self.self_info.0.id.to_bytes().to_vec(), - node_serial: self.self_info.0.serial as u32, + node_id: self.self_info.id.to_bytes().to_vec(), + node_serial: self.self_info.serial as u32, }); let mut client = ReconfigurationServiceClient::new(channel); let _res = client.ready_electorate(request).await?.into_inner(); @@ -427,10 +421,9 @@ impl Network for GrpcNetwork { if let Some(member) = lock.get_mut(&id) { let new_member = Member { info: member.member.info.clone(), - host: member.member.host.clone(), channel: member.member.channel.clone(), - ready_electorate: true, }; + new_member.info.ready.store(true, Ordering::Relaxed); member.member = Arc::new(new_member); } else { return Err(SyneviError::NoMembersFound); @@ -456,7 +449,7 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::PreAccept(req, serial) => { // ... and then iterate over every member ... for replica in &self.members { - let ready = replica.ready_electorate; + let ready = replica.info.ready.load(Ordering::Relaxed); let channel = replica.channel.clone(); let inner = req.clone(); let mut request = tonic::Request::new(inner); @@ -478,7 +471,7 @@ impl NetworkInterface for GrpcNetworkSet { } BroadcastRequest::Accept(req) => { for replica in &self.members { - let ready = replica.ready_electorate; + let ready = replica.info.ready.load(Ordering::Relaxed); let channel = replica.channel.clone(); let request = req.clone(); responses.spawn(async move { @@ -492,7 +485,7 @@ impl NetworkInterface for GrpcNetworkSet { } BroadcastRequest::Commit(req) => { for replica in &self.members { - let ready = replica.ready_electorate; + let ready = replica.info.ready.load(Ordering::Relaxed); let channel = replica.channel.clone(); let request = req.clone(); responses.spawn(async move { @@ -507,7 +500,7 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::Apply(req) => { await_majority = false; for replica in &self.members { - let ready = replica.ready_electorate; + let ready = replica.info.ready.load(Ordering::Relaxed); let channel = replica.channel.clone(); let request = req.clone(); responses.spawn(async move { @@ -524,7 +517,7 @@ impl NetworkInterface for GrpcNetworkSet { broadcast_all = true; for replica in &self.members { // TODO: Not sure if neccessary - let ready = replica.ready_electorate; + let ready = replica.info.ready.load(Ordering::Relaxed); let channel = replica.channel.clone(); let request = req.clone(); responses.spawn(async move { @@ -541,7 +534,7 @@ impl NetworkInterface for GrpcNetworkSet { let all = self .members .iter() - .filter(|member| member.ready_electorate) + .filter(|member| member.info.ready.load(Ordering::Relaxed)) .count(); let majority = if all == 0 { 0 } else { (all / 2) + 1 }; let mut counter = 0_usize; diff --git a/synevi_network/src/protos/configure_transport.proto b/synevi_network/src/protos/configure_transport.proto index f713d5b..daa0674 100644 --- a/synevi_network/src/protos/configure_transport.proto +++ b/synevi_network/src/protos/configure_transport.proto @@ -36,6 +36,7 @@ message Config { uint32 node_serial = 1; bytes node_id = 2; string host = 3; + bool ready = 4; } message JoinElectorateResponse { @@ -71,14 +72,12 @@ message ReadyElectorateResponse {} service InitService { - rpc ReportLastApplied(ReportLastAppliedRequest) returns (ReportLastAppliedResponse) {} + rpc ReportElectorate(ReportElectorateRequest) returns (ReportElectorateResponse) {} } -message ReportLastAppliedRequest { - Config config = 1; - bytes last_applied = 2; - bytes last_applied_hash = 3; +message ReportElectorateRequest { + repeated Config configs = 1; } -message ReportLastAppliedResponse {} +message ReportElectorateResponse {} diff --git a/synevi_network/src/reconfiguration.rs b/synevi_network/src/reconfiguration.rs index 73fb44b..0bc4a38 100644 --- a/synevi_network/src/reconfiguration.rs +++ b/synevi_network/src/reconfiguration.rs @@ -1,7 +1,7 @@ use crate::configure_transport::{ GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, - ReadyElectorateRequest, ReadyElectorateResponse, ReportLastAppliedRequest, - ReportLastAppliedResponse, + ReadyElectorateRequest, ReadyElectorateResponse, ReportElectorateRequest, + ReportElectorateResponse, }; use synevi_types::{SyneviError, T}; use ulid::Ulid; @@ -23,10 +23,10 @@ pub trait Reconfiguration { ) -> Result; // Joining node - async fn report_last_applied( + async fn report_electorate( &self, - request: ReportLastAppliedRequest, - ) -> Result; + request: ReportElectorateRequest, + ) -> Result; } #[derive(Debug)] diff --git a/synevi_network/src/replica.rs b/synevi_network/src/replica.rs index a850cd1..0743790 100644 --- a/synevi_network/src/replica.rs +++ b/synevi_network/src/replica.rs @@ -3,7 +3,7 @@ use crate::{ init_service_server::InitService, reconfiguration_service_server::ReconfigurationService, time_service_server::TimeService, GetEventRequest, GetEventResponse, GetTimeRequest, GetTimeResponse, JoinElectorateRequest, JoinElectorateResponse, ReadyElectorateRequest, - ReadyElectorateResponse, ReportLastAppliedRequest, ReportLastAppliedResponse, + ReadyElectorateResponse, ReportElectorateRequest, ReportElectorateResponse, }, consensus_transport::*, reconfiguration::Reconfiguration, @@ -247,13 +247,13 @@ impl ReconfigurationService for ReplicaB } #[async_trait::async_trait] impl InitService for ReplicaBox { - async fn report_last_applied( + async fn report_electorate( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { Ok(Response::new( self.inner - .report_last_applied(request.into_inner()) + .report_electorate(request.into_inner()) .await .map_err(|e| tonic::Status::internal(e.to_string()))?, )) diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index c46aa8a..3bd3f59 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -41,8 +41,15 @@ pub enum TransactionPayload { #[derive(Debug, Clone, Serialize, Eq, PartialEq, PartialOrd, Ord)] pub enum InternalExecution { - JoinElectorate { id: Ulid, serial: u16, host: String }, - ReadyElectorate { id: Ulid, serial: u16 }, + JoinElectorate { + id: Ulid, + serial: u16, + new_node_host: String, + }, + ReadyElectorate { + id: Ulid, + serial: u16, + }, } // #[derive(Debug, Clone, Serialize, Eq, PartialEq, PartialOrd, Ord)] @@ -90,11 +97,15 @@ impl Transaction for InternalExecution { let mut bytes = Vec::new(); match self { - InternalExecution::JoinElectorate { id, serial, host } => { + InternalExecution::JoinElectorate { + id, + serial, + new_node_host, + } => { bytes.push(0); bytes.extend_from_slice(&id.to_bytes()); bytes.extend_from_slice(&serial.to_be_bytes()); - bytes.extend_from_slice(&host.as_bytes()); + bytes.extend_from_slice(&new_node_host.as_bytes()); } InternalExecution::ReadyElectorate { id, serial } => { bytes.push(1); @@ -112,15 +123,19 @@ impl Transaction for InternalExecution { Some(0) => { let (id, rest) = rest.split_at(16); let id = Ulid::from_bytes(id.try_into()?); - let (serial, host) = rest.split_at(2); + let (serial, new_node_host) = rest.split_at(2); let serial = u16::from_be_bytes( serial .try_into() .map_err(|_| SyneviError::InvalidConversionFromBytes(String::new()))?, ); - let host = String::from_utf8(host.to_owned()) + let new_node_host = String::from_utf8(new_node_host.to_owned()) .map_err(|e| SyneviError::InvalidConversionFromBytes(e.to_string()))?; - Ok(InternalExecution::JoinElectorate { id, serial, host }) + Ok(InternalExecution::JoinElectorate { + id, + serial, + new_node_host, + }) } Some(1) => { let (id, serial) = rest.split_at(16); @@ -462,7 +477,7 @@ mod test { TransactionPayload::Internal(crate::types::InternalExecution::JoinElectorate { id: ulid::Ulid::new(), serial: 1, - host: "http://test.org:1234".to_string(), + new_node_host: "http://test.org:1234".to_string(), }); let bytes = internal_join.as_bytes(); assert_eq!( diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index f0684ad..fb032a3 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -351,7 +351,8 @@ mod tests { .clone() .transaction(i, Vec::from("This is a transaction")) .await - .unwrap().unwrap(); + .unwrap() + .unwrap(); } runtime.shutdown_background(); diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index f5650c5..5900f18 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -184,7 +184,7 @@ impl Network for MaelstromNetwork { todo!() } - async fn broadcast_config(&self, _host: String) -> Result<(u32, Vec), SyneviError> { + async fn join_electorate(&self, _host: String) -> Result<(u32, Vec), SyneviError> { todo!() } From def9e81fb03f5960e68d9bc3d83923e7632fb87b Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 14:32:46 +0200 Subject: [PATCH 15/40] refactor: Refactor status/members handling to network --- synevi_core/src/coordinator.rs | 14 +- synevi_core/src/lib.rs | 28 +++- synevi_core/src/node.rs | 88 ++++++----- synevi_core/src/replica.rs | 77 ++++------ synevi_network/src/latency_service.rs | 2 +- synevi_network/src/network.rs | 144 ++++++++++-------- .../src/protos/configure_transport.proto | 4 +- synevi_persistence/src/lmdb_store.rs | 4 +- synevi_persistence/src/mem_store.rs | 6 +- synevi_types/src/traits.rs | 1 - tests/consensus_e2e.rs | 10 +- tests/maelstrom/network.rs | 14 +- 12 files changed, 208 insertions(+), 184 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 74baedb..411cc4e 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -66,7 +66,9 @@ where id: u128, ) -> Self { trace!(?id, "Coordinator: New"); - let t0 = node.event_store.init_t_zero(node.info.serial); + let t0 = node + .event_store + .init_t_zero(node.get_serial()); Coordinator { node, transaction: TransactionStateMachine { @@ -112,7 +114,7 @@ where let pre_accepted_responses = network_interface .broadcast(BroadcastRequest::PreAccept( pre_accepted_request, - self.node.info.serial, + self.node.get_serial(), )) .await?; @@ -299,7 +301,11 @@ where .map(|e| ExecutorResult::External(e)), TransactionPayload::Internal(request) => { let result = match request { - InternalExecution::JoinElectorate { id, serial, new_node_host } => { + InternalExecution::JoinElectorate { + id, + serial, + new_node_host, + } => { let res = self .node .add_member(*id, *serial, new_node_host.clone(), false) @@ -561,7 +567,7 @@ pub mod tests { let pre_accepted_responses = network_interface .broadcast(BroadcastRequest::PreAccept( pre_accepted_request, - self.node.info.serial, + self.node.get_serial(), )) .await?; diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index c761a92..6ced01a 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -6,10 +6,15 @@ pub mod utils; mod wait_handler; pub mod tests { + use std::sync::atomic::AtomicBool; + use std::sync::atomic::AtomicU32; use std::sync::Arc; use synevi_network::configure_transport::GetEventResponse; use synevi_network::network::BroadcastResponse; + use synevi_network::network::MemberWithLatency; use synevi_network::network::NetworkInterface; + use synevi_network::network::NodeInfo; + use synevi_network::network::NodeStatus; use synevi_network::network::{BroadcastRequest, Network}; use synevi_network::replica::Replica; use synevi_types::types::SyneviResult; @@ -73,8 +78,21 @@ pub mod tests { async fn get_waiting_time(&self, _node_serial: u16) -> u64 { 0 } - async fn get_member_len(&self) -> u32 { - 0 + async fn get_members(&self) -> Vec> { + vec![] + } + + fn get_node_status(&self) -> Arc { + Arc::new(NodeStatus { + info: NodeInfo { + id: Ulid::new(), + serial: 0, + host: "localhost".to_string(), + ready: AtomicBool::new(false), + }, + members_responded: AtomicU32::new(0), + has_members: AtomicBool::new(false), + }) } async fn join_electorate(&self, _host: String) -> Result { @@ -82,7 +100,6 @@ pub mod tests { } async fn get_stream_events( &self, - _self_id: Vec, _last_applied: Vec, ) -> Result, SyneviError> { let (_, rcv) = tokio::sync::mpsc::channel(1); @@ -96,10 +113,7 @@ pub mod tests { Ok(()) } - async fn report_config( - &self, - _host: String, - ) -> Result<(), SyneviError> { + async fn report_config(&self, _host: String) -> Result<(), SyneviError> { Ok(()) } } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index dd0d884..992620a 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -2,11 +2,11 @@ use crate::coordinator::Coordinator; use crate::replica::ReplicaConfig; use crate::wait_handler::{CheckResult, WaitHandler}; use std::fmt::Debug; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::Ordering; use std::sync::RwLock; use std::sync::{atomic::AtomicU64, Arc}; use synevi_network::consensus_transport::{ApplyRequest, CommitRequest}; -use synevi_network::network::{Network, NetworkInterface, NodeInfo}; +use synevi_network::network::{Network, NetworkInterface}; use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; @@ -70,7 +70,6 @@ where executor: E, store: S, ) -> Result, SyneviError> { - let stats = Stats { total_requests: AtomicU64::new(0), total_accepts: AtomicU64::new(0), @@ -94,8 +93,6 @@ where stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, - has_members: AtomicBool::new(false), - members_responded: Arc::new(AtomicU64::new(0)), self_clone: RwLock::new(None), }); node.self_clone @@ -103,8 +100,9 @@ where .expect("Locking self_clone failed") .replace(node.clone()); - let ready = Arc::new(AtomicBool::new(true)); - let (replica, _) = ReplicaConfig::new(node.clone(), ready); + node.set_ready(); + + let replica = ReplicaConfig::new(node.clone()); node.network.spawn_server(replica).await?; let node_clone = node.clone(); @@ -115,11 +113,34 @@ where } pub fn set_ready(&self) -> () { - self.is_ready.store(true, Ordering::Relaxed); + self.network + .get_node_status() + .info + .ready + .store(true, Ordering::Relaxed); } pub fn is_ready(&self) -> bool { - self.is_ready.load(Ordering::Relaxed) + self.network + .get_node_status() + .info + .ready + .load(Ordering::Relaxed) + } + + pub fn has_members(&self) -> bool { + self.network + .get_node_status() + .has_members + .load(Ordering::Relaxed) + } + + pub fn get_serial(&self) -> u16 { + self.network.get_node_status().info.serial + } + + pub fn get_ulid(&self) -> Ulid { + self.network.get_node_status().info.id } #[instrument(level = "trace", skip(network, executor, store))] @@ -131,8 +152,6 @@ where store: S, member_host: String, ) -> Result, SyneviError> { - let node_name = NodeInfo { id, serial }; - let stats = Stats { total_requests: AtomicU64::new(0), total_accepts: AtomicU64::new(0), @@ -142,17 +161,13 @@ where let arc_store = Arc::new(store); let wait_handler = WaitHandler::new(arc_store.clone()); - let ready = Arc::new(AtomicBool::new(false)); let node = Arc::new(Node { - info: node_name, event_store: arc_store, network, wait_handler, stats, semaphore: Arc::new(tokio::sync::Semaphore::new(10)), executor, - has_members: AtomicBool::new(false), - is_ready: ready.clone(), self_clone: RwLock::new(None), }); @@ -161,12 +176,11 @@ where .expect("Locking self_clone failed") .replace(node.clone()); - let (replica, config_receiver) = ReplicaConfig::new(node.clone(), ready.clone()); + let replica = ReplicaConfig::new(node.clone()); node.network.spawn_server(replica.clone()).await?; let node_clone = node.clone(); tokio::spawn(async move { node_clone.run_check_recovery().await }); - node.reconfigure(replica, member_host, config_receiver, ready) - .await?; + node.reconfigure(replica, member_host).await?; Ok(node) } @@ -180,8 +194,6 @@ where ready: bool, ) -> Result<(), SyneviError> { self.network.add_member(id, serial, host, ready).await?; - self.has_members - .store(true, std::sync::atomic::Ordering::Relaxed); Ok(()) } @@ -193,9 +205,9 @@ where #[instrument(level = "trace", skip(self, transaction))] pub async fn transaction(self: Arc, id: u128, transaction: E::Tx) -> SyneviResult { - if !self.has_members.load(std::sync::atomic::Ordering::Relaxed) { + if !self.has_members() { tracing::warn!("Consensus omitted: No members in the network"); - } else if !self.is_ready.load(Ordering::Relaxed) { + } else if !self.is_ready() { return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; @@ -216,9 +228,7 @@ where id: u128, transaction: TransactionPayload, ) -> InternalSyneviResult { - if !self.has_members.load(std::sync::atomic::Ordering::Relaxed) { - tracing::warn!("Consensus omitted: No members in the network"); - } else if !self.is_ready.load(Ordering::Relaxed) { + if !self.is_ready() { return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; @@ -240,10 +250,6 @@ where ) } - pub fn get_info(&self) -> NodeInfo { - self.info.clone() - } - #[instrument(level = "trace", skip(self))] pub async fn commit(&self, event: UpsertEvent) -> Result<(), SyneviError> { let t0_commit = event.t_zero.clone(); @@ -293,7 +299,9 @@ where } println!( "[{:?}]Applied event: t0: {:?}, t: {:?}", - self.info.serial, event.t_zero, event.t + self.get_serial(), + event.t_zero, + event.t ); self.event_store.upsert_tx(event)?; @@ -349,23 +357,27 @@ where &self, replica: ReplicaConfig, member_host: String, - ready: Arc, ) -> Result<(), SyneviError> { // 1. Broadcast self_config to other member let expected = self.network.join_electorate(member_host).await?; // 2. wait for JoinElectorate responses with expected majority and config from others - while self.members_responded.load(Ordering::Relaxed) < self.network.get_member_len().await as u64 { + while self + .network + .get_node_status() + .members_responded + .load(Ordering::Relaxed) + < (expected / 2) + 1 + { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } - let (last_applied, _) = self.event_store.last_applied(); - self.sync_events(last_applied, self_id, &replica).await?; + self.sync_events(last_applied, &replica).await?; // 3. Send ReadyJoinElectorate && set myself to ready - ready.store(true, Ordering::Relaxed); + self.set_ready(); self.network.ready_electorate().await?; Ok(()) } @@ -378,7 +390,7 @@ where // 2.2 else Request stream with events until last_applied (highest t of JoinElectorate) let mut rcv = self .network - .get_stream_events(last_applied.into(), self_id) + .get_stream_events(last_applied.into()) .await?; while let Some(event) = rcv.recv().await { let state: State = event.state.into(); @@ -501,7 +513,7 @@ mod tests { node.event_store.get_event_store(), coord, "Node: {:?}", - node.get_info() + node.get_serial() ); } } @@ -584,7 +596,7 @@ mod tests { .all(|(_, e)| e.state == State::Applied)); assert_eq!(coordinator_store.len(), node_store.len()); if coordinator_store != node_store { - println!("Node: {:?}", node.get_info()); + println!("Node: {:?}", node.get_serial()); let mut node_store_iter = node_store.iter(); for (k, v) in coordinator_store.iter() { if let Some(next) = node_store_iter.next() { diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index b8191a0..23fafd2 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -2,10 +2,12 @@ use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; use sha3::{Digest, Sha3_256}; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::Ordering; use std::sync::Arc; use synevi_network::configure_transport::{ - Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, ReadyElectorateRequest, ReadyElectorateResponse, ReportElectorateRequest, ReportElectorateResponse, ReportLastAppliedRequest, ReportLastAppliedResponse + Config, GetEventRequest, GetEventResponse, JoinElectorateRequest, JoinElectorateResponse, + ReadyElectorateRequest, ReadyElectorateResponse, ReportElectorateRequest, + ReportElectorateResponse, }; use synevi_network::consensus_transport::{ AcceptRequest, AcceptResponse, ApplyRequest, ApplyResponse, CommitRequest, CommitResponse, @@ -13,13 +15,13 @@ use synevi_network::consensus_transport::{ TryRecoveryResponse, }; use synevi_network::network::Network; -use synevi_network::reconfiguration::{Reconfiguration, Report}; +use synevi_network::reconfiguration::Reconfiguration; use synevi_network::replica::Replica; use synevi_types::traits::Store; use synevi_types::types::{ExecutorResult, InternalExecution, TransactionPayload, UpsertEvent}; use synevi_types::{Ballot, Executor, State, T, T0}; use synevi_types::{SyneviError, Transaction}; -use tokio::sync::mpsc::{channel, Receiver, Sender}; +use tokio::sync::mpsc::Receiver; use tracing::{instrument, trace}; use ulid::Ulid; @@ -30,9 +32,6 @@ where S: Store, { node: Arc>, - notifier: Sender, - ready: Arc, - configuring: Arc, } impl ReplicaConfig @@ -41,17 +40,8 @@ where E: Executor, S: Store, { - pub fn new(node: Arc>, ready: Arc) -> (Self, Receiver) { - let (notifier, receiver) = channel(10); - ( - Self { - node, - notifier, - ready, - configuring: Arc::new(AtomicBool::new(false)), - }, - receiver, - ) + pub fn new(node: Arc>) -> Self { + Self { node } } } @@ -229,19 +219,22 @@ where serial, new_node_host, } => { - if id != &self.node.info.id { + if id != &self.node.get_ulid() { let res = self .node .add_member(*id, *serial, new_node_host.clone(), false) .await; - self.node.network.report_config(new_node_host.clone()).await?; + self.node + .network + .report_config(new_node_host.clone()) + .await?; res } else { Ok(()) } } InternalExecution::ReadyElectorate { id, serial } => { - if id != &self.node.info.id { + if id != &self.node.get_ulid() { self.node.ready_member(*id, *serial).await } else { Ok(()) @@ -343,7 +336,7 @@ where if let Some(recover_event) = self .node .event_store - .recover_event(&t0, self.node.get_info().serial)? + .recover_event(&t0, self.node.get_serial())? { tokio::spawn(Coordinator::recover(self.node.clone(), recover_event)); return Ok(TryRecoveryResponse { accepted: true }); @@ -367,13 +360,14 @@ where &self, request: JoinElectorateRequest, ) -> Result { - if !self.ready.load(Ordering::SeqCst) { + if !self.node.is_ready() { return Ok(JoinElectorateResponse::default()); } let Some(Config { node_id, node_serial, host, + .. }) = request.config else { return Err(SyneviError::TonicStatusError( @@ -382,7 +376,7 @@ where }; let node = self.node.clone(); - let majority = self.node.network.get_member_len().await; + let member_count = self.node.network.get_members().await.len() as u32; let self_event = Ulid::new(); let _res = node .internal_transaction( @@ -390,13 +384,12 @@ where TransactionPayload::Internal(InternalExecution::JoinElectorate { id: Ulid::from_bytes(node_id.as_slice().try_into()?), serial: node_serial.try_into()?, - host, + new_node_host: host, }), ) .await?; Ok(JoinElectorateResponse { - majority, - self_event: self_event.to_bytes().to_vec(), + member_count, }) } @@ -404,16 +397,15 @@ where &self, request: GetEventRequest, ) -> Result>, SyneviError> { - if !self.ready.load(Ordering::SeqCst) { + if !self.node.is_ready() { return Err(SyneviError::NotReady); } let (sdx, rcv) = tokio::sync::mpsc::channel(200); - let event_id = u128::from_be_bytes(request.self_event.as_slice().try_into()?); let last_applied = T::try_from(request.last_applied.as_slice())?; let mut store_rcv = self .node .event_store - .get_events_after(last_applied, event_id)?; + .get_events_after(last_applied)?; tokio::spawn(async move { while let Some(Ok(event)) = store_rcv.recv().await { let response = { @@ -460,7 +452,7 @@ where &self, request: ReadyElectorateRequest, ) -> Result { - if !self.ready.load(Ordering::SeqCst) { + if !self.node.is_ready() { return Ok(ReadyElectorateResponse::default()); } // Start ready electorate transaction with NewMemberUlid @@ -487,20 +479,16 @@ where &self, request: ReportElectorateRequest, ) -> Result { - if self.ready.load(Ordering::Relaxed) { - return Ok(ReportLastAppliedResponse::default()); + if self.node.is_ready() { + return Ok(ReportElectorateResponse::default()); } - for member in request.config { - self.node.add_member( - member.node_id, - member.node_serial, - member.host, - true, - ) - .await?; + for member in request.configs { + self.node + .add_member(Ulid::from_bytes(member.node_id.as_slice().try_into()?), member.node_serial as u16, member.host, member.ready) + .await?; } - self.node.members_responded.fetch_add(1, Ordering::Relaxed); - Ok(ReportLastAppliedResponse {}) + self.node.network.get_node_status().members_responded.fetch_add(1, Ordering::Relaxed); + Ok(ReportElectorateResponse {}) } } @@ -513,9 +501,6 @@ where fn clone(&self) -> Self { Self { node: self.node.clone(), - notifier: self.notifier.clone(), - ready: self.ready.clone(), - configuring: self.configuring.clone(), } } } diff --git a/synevi_network/src/latency_service.rs b/synevi_network/src/latency_service.rs index e557dfc..fd21a28 100644 --- a/synevi_network/src/latency_service.rs +++ b/synevi_network/src/latency_service.rs @@ -19,7 +19,7 @@ use crate::{ const LATENCY_INTERVAL: u64 = 10; pub async fn get_latency( - members: Arc>>, + members: Arc, ahash::RandomState>>>, ) -> Result<(), SyneviError> { loop { for (_, member) in members.read().await.iter() { diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 9334287..78f6966 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -20,7 +20,7 @@ use crate::{ replica::{Replica, ReplicaBox}, }; use std::collections::HashMap; -use std::sync::atomic::{AtomicBool, AtomicI64, AtomicU64, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicI64, AtomicU32, AtomicU64, Ordering}; use std::{net::SocketAddr, sync::Arc}; use synevi_types::error::SyneviError; use synevi_types::T0; @@ -56,13 +56,13 @@ pub trait Network: Send + Sync + 'static { ) -> Result<(), SyneviError>; async fn get_interface(&self) -> Arc; async fn get_waiting_time(&self, node_serial: u16) -> u64; - async fn get_member_len(&self) -> u32; + async fn get_members(&self) -> Vec>; + fn get_node_status(&self) -> Arc; async fn join_electorate(&self, host: String) -> Result; // All members async fn report_config(&self, host: String) -> Result<(), SyneviError>; async fn get_stream_events( &self, last_applied: Vec, - self_event: Vec, ) -> Result, SyneviError>; async fn ready_electorate(&self) -> Result<(), SyneviError>; async fn ready_member(&self, id: Ulid, serial: u16) -> Result<(), SyneviError>; @@ -105,19 +105,23 @@ where self.as_ref().get_waiting_time(node_serial).await } - async fn get_member_len(&self) -> u32 { - self.as_ref().get_member_len().await + async fn get_members(&self) -> Vec> { + self.as_ref().get_members().await } + + fn get_node_status(&self) -> Arc { + self.as_ref().get_node_status() + } + async fn join_electorate(&self, host: String) -> Result { self.as_ref().join_electorate(host).await } async fn get_stream_events( &self, last_applied: Vec, - self_event: Vec, ) -> Result, SyneviError> { self.as_ref() - .get_stream_events(last_applied, self_event) + .get_stream_events(last_applied) .await } @@ -150,15 +154,22 @@ where } } -#[derive(Clone, Debug, Default)] +#[derive(Debug, Default)] pub struct NodeInfo { pub id: Ulid, pub serial: u16, pub host: String, - pub ready: Arc, + pub ready: AtomicBool, +} + +#[derive(Debug)] +pub struct NodeStatus { + pub info: NodeInfo, + pub members_responded: AtomicU32, + pub has_members: AtomicBool, } -#[derive(Clone, Debug)] +#[derive(Debug)] pub struct Member { pub info: NodeInfo, pub channel: Channel, @@ -166,7 +177,7 @@ pub struct Member { #[derive(Debug)] pub struct MemberWithLatency { - pub member: Arc, + pub member: Member, pub latency: AtomicU64, pub skew: AtomicI64, } @@ -192,27 +203,30 @@ pub enum BroadcastResponse { #[derive(Debug)] pub struct GrpcNetwork { pub socket_addr: SocketAddr, - pub self_info: NodeInfo, - pub members: Arc>>, + pub self_status: Arc, + pub members: Arc, ahash::RandomState>>>, join_set: Mutex>>, } #[derive(Debug)] pub struct GrpcNetworkSet { - members: Vec>, + members: Vec>, } impl GrpcNetwork { - pub fn new(socket_addr: SocketAddr, host: String, node_id: Ulid, node_serial: u16, ready: Arc) -> Self { + pub fn new(socket_addr: SocketAddr, host: String, node_id: Ulid, node_serial: u16) -> Self { Self { socket_addr, - self_info: - NodeInfo { + self_status: Arc::new(NodeStatus { + info: NodeInfo { id: node_id, serial: node_serial, host, - ready, + ready: AtomicBool::new(false), }, + members_responded: AtomicU32::new(0), + has_members: AtomicBool::new(false), + }), members: Arc::new(RwLock::new(HashMap::default())), join_set: Mutex::new(JoinSet::new()), } @@ -220,13 +234,7 @@ impl GrpcNetwork { pub async fn create_network_set(&self) -> Arc { Arc::new(GrpcNetworkSet { - members: self - .members - .read() - .await - .iter() - .map(|(_, e)| e.member.clone()) - .collect(), + members: self.members.read().await.values().cloned().collect(), }) } } @@ -253,16 +261,22 @@ impl Network for GrpcNetwork { if writer.get(&id).is_none() { writer.insert( id, - MemberWithLatency { - member: Arc::new(Member { - info: NodeInfo { id, serial, host, ready: Arc::new(AtomicBool::new(ready)) }, + Arc::new(MemberWithLatency { + member: Member { + info: NodeInfo { + id, + serial, + host, + ready: AtomicBool::new(ready), + }, channel, - }), + }, latency: AtomicU64::new(500), skew: AtomicI64::new(0), - }, + }), ); } + self.self_status.has_members.store(true, Ordering::Relaxed); Ok(()) } @@ -310,16 +324,20 @@ impl Network for GrpcNetwork { (max_latency) - (node_latency / 2) } - async fn get_member_len(&self) -> u32 { - (self.members.read().await.len() + 1) as u32 + async fn get_members(&self) -> Vec> { + self.members.read().await.values().cloned().collect() + } + + fn get_node_status(&self) -> Arc { + self.self_status.clone() } async fn join_electorate(&self, host: String) -> Result { let config = Config { - node_serial: self.self_info.serial as u32, - node_id: self.self_info.id.to_bytes().to_vec(), - host: self.self_info.host.clone(), - ready: self.self_info.ready.load(Ordering::Relaxed), + node_serial: self.self_status.info.serial as u32, + node_id: self.self_status.info.id.to_bytes().to_vec(), + host: self.self_status.info.host.clone(), + ready: self.self_status.info.ready.load(Ordering::Relaxed), }; let channel = Channel::from_shared(host)?.connect().await?; let request = tonic::Request::new(JoinElectorateRequest { @@ -328,15 +346,15 @@ impl Network for GrpcNetwork { let mut client = ReconfigurationServiceClient::new(channel); let response = client.join_electorate(request).await?; let response = response.into_inner(); - Ok(response.majority) + Ok(response.member_count) } async fn report_config(&self, host: String) -> Result<(), SyneviError> { let config = Config { - node_serial: self.self_info.serial as u32, - node_id: self.self_info.id.to_bytes().to_vec(), - host: self.self_info.host.clone(), - ready: self.self_info.ready.load(Ordering::Relaxed), + node_serial: self.self_status.info.serial as u32, + node_id: self.self_status.info.id.to_bytes().to_vec(), + host: self.self_status.info.host.clone(), + ready: self.self_status.info.ready.load(Ordering::Relaxed), }; let mut configs: Vec<_> = self @@ -363,17 +381,16 @@ impl Network for GrpcNetwork { async fn get_stream_events( &self, last_applied: Vec, - self_event: Vec, ) -> Result, SyneviError> { let lock = self.members.read().await; let mut members = lock.iter(); - let Some((_, member)) = members.find(|(_, m)| m.member.info.ready.load(Ordering::Relaxed)) else { + let Some((_, member)) = members.find(|(_, m)| m.member.info.ready.load(Ordering::Relaxed)) + else { return Err(SyneviError::NoMembersFound); }; let channel = member.member.channel.clone(); let request = GetEventRequest { last_applied, - self_event, }; let (sdx, rcv) = tokio::sync::mpsc::channel(200); @@ -408,8 +425,8 @@ impl Network for GrpcNetwork { }; let channel = member.member.channel.clone(); let request = tonic::Request::new(ReadyElectorateRequest { - node_id: self.self_info.id.to_bytes().to_vec(), - node_serial: self.self_info.serial as u32, + node_id: self.self_status.info.id.to_bytes().to_vec(), + node_serial: self.self_status.info.serial as u32, }); let mut client = ReconfigurationServiceClient::new(channel); let _res = client.ready_electorate(request).await?.into_inner(); @@ -417,14 +434,9 @@ impl Network for GrpcNetwork { } async fn ready_member(&self, id: Ulid, _serial: u16) -> Result<(), SyneviError> { - let mut lock = self.members.write().await; - if let Some(member) = lock.get_mut(&id) { - let new_member = Member { - info: member.member.info.clone(), - channel: member.member.channel.clone(), - }; - new_member.info.ready.store(true, Ordering::Relaxed); - member.member = Arc::new(new_member); + let lock = self.members.read().await; + if let Some(member) = lock.get(&id) { + member.member.info.ready.store(true, Ordering::Relaxed); } else { return Err(SyneviError::NoMembersFound); } @@ -449,8 +461,8 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::PreAccept(req, serial) => { // ... and then iterate over every member ... for replica in &self.members { - let ready = replica.info.ready.load(Ordering::Relaxed); - let channel = replica.channel.clone(); + let ready = replica.member.info.ready.load(Ordering::Relaxed); + let channel = replica.member.channel.clone(); let inner = req.clone(); let mut request = tonic::Request::new(inner); request.metadata_mut().append( @@ -471,8 +483,8 @@ impl NetworkInterface for GrpcNetworkSet { } BroadcastRequest::Accept(req) => { for replica in &self.members { - let ready = replica.info.ready.load(Ordering::Relaxed); - let channel = replica.channel.clone(); + let ready = replica.member.info.ready.load(Ordering::Relaxed); + let channel = replica.member.channel.clone(); let request = req.clone(); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); @@ -485,8 +497,8 @@ impl NetworkInterface for GrpcNetworkSet { } BroadcastRequest::Commit(req) => { for replica in &self.members { - let ready = replica.info.ready.load(Ordering::Relaxed); - let channel = replica.channel.clone(); + let ready = replica.member.info.ready.load(Ordering::Relaxed); + let channel = replica.member.channel.clone(); let request = req.clone(); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); @@ -500,8 +512,8 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::Apply(req) => { await_majority = false; for replica in &self.members { - let ready = replica.info.ready.load(Ordering::Relaxed); - let channel = replica.channel.clone(); + let ready = replica.member.info.ready.load(Ordering::Relaxed); + let channel = replica.member.channel.clone(); let request = req.clone(); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); @@ -517,8 +529,8 @@ impl NetworkInterface for GrpcNetworkSet { broadcast_all = true; for replica in &self.members { // TODO: Not sure if neccessary - let ready = replica.info.ready.load(Ordering::Relaxed); - let channel = replica.channel.clone(); + let ready = replica.member.info.ready.load(Ordering::Relaxed); + let channel = replica.member.channel.clone(); let request = req.clone(); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); @@ -534,7 +546,7 @@ impl NetworkInterface for GrpcNetworkSet { let all = self .members .iter() - .filter(|member| member.info.ready.load(Ordering::Relaxed)) + .filter(|member| member.member.info.ready.load(Ordering::Relaxed)) .count(); let majority = if all == 0 { 0 } else { (all / 2) + 1 }; let mut counter = 0_usize; @@ -620,7 +632,7 @@ impl NetworkInterface for GrpcNetworkSet { timestamp_zero: t0.into(), }; for replica in &self.members { - let channel = replica.channel.clone(); + let channel = replica.member.channel.clone(); let request = tonic::Request::new(inner_request.clone()); responses.spawn(async move { let mut client = ConsensusTransportClient::new(channel); diff --git a/synevi_network/src/protos/configure_transport.proto b/synevi_network/src/protos/configure_transport.proto index daa0674..48e41d2 100644 --- a/synevi_network/src/protos/configure_transport.proto +++ b/synevi_network/src/protos/configure_transport.proto @@ -40,13 +40,11 @@ message Config { } message JoinElectorateResponse { - uint32 majority = 1; - bytes self_event = 2; + uint32 member_count = 1; } message GetEventRequest { bytes last_applied = 1; - bytes self_event = 2; } message GetEventResponse { diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 3efe317..e9ec24f 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -203,12 +203,11 @@ impl Store for PersistentStore { fn get_events_after( &self, last_applied: T, - self_event: u128, ) -> Result>, SyneviError> { self.data .lock() .expect("poisoned lock, aborting") - .get_events_after(last_applied, self_event) + .get_events_after(last_applied) } #[instrument(level = "trace", skip(self))] @@ -608,7 +607,6 @@ impl InternalData { fn get_events_after( &self, last_applied: T, - _self_event: u128, ) -> Result>, SyneviError> { let (sdx, rcv) = tokio::sync::mpsc::channel(200); let db = self.db.clone(); diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 54a2868..e8dcd0e 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -125,7 +125,6 @@ impl Store for MemStore { fn get_events_after( &self, last_applied: T, - self_event: u128, ) -> Result>, SyneviError> { let (sdx, rcv) = tokio::sync::mpsc::channel(100); @@ -134,7 +133,7 @@ impl Store for MemStore { store .lock() .expect("poisoned lock, aborting") - .get_events_until(last_applied, self_event, sdx)?; + .get_events_after(last_applied, sdx)?; Ok::<(), SyneviError>(()) }); @@ -454,10 +453,9 @@ impl InternalStore { (self.last_applied, t0) } - fn get_events_until( + fn get_events_after( &self, last_applied: T, - _self_event: u128, sdx: Sender>, ) -> Result<(), SyneviError> { let last_applied_t0 = match self.mappings.get(&last_applied) { diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index a54c69d..a53803e 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -105,7 +105,6 @@ pub trait Store: Send + Sync + Sized + 'static { fn get_events_after( &self, last_applied: T, - self_event: u128, ) -> Result>, SyneviError>; fn get_and_update_hash( diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index fb032a3..efd9adc 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -103,11 +103,11 @@ mod tests { .iter() .all(|(_, e)| e.state == State::Applied), "Not all applied @ {:?}", - node.get_info() + node.get_serial() ); assert_eq!(coordinator_store.len(), node_store.len()); if coordinator_store != node_store { - println!("Node: {:?}", node.get_info()); + println!("Node: {:?}", node.get_serial()); let mut node_store_iter = node_store.iter(); for (k, v) in coordinator_store.iter() { if let Some(next) = node_store_iter.next() { @@ -283,7 +283,7 @@ mod tests { .all(|(_, e)| e.state == State::Applied)); assert_eq!(coordinator_store.len(), node_store.len()); if coordinator_store != node_store { - println!("Node: {:?}", node.get_info()); + println!("Node: {:?}", node.get_serial()); let mut node_store_iter = node_store.iter(); for (k, v) in coordinator_store.iter() { if let Some(next) = node_store_iter.next() { @@ -523,11 +523,11 @@ mod tests { .iter() .all(|(_, e)| e.state == State::Applied), "Not all applied @ {:?}", - node.get_info() + node.get_serial() ); assert_eq!(coordinator_store.len(), node_store.len()); if coordinator_store != node_store { - println!("Node: {:?}", node.get_info()); + println!("Node: {:?}", node.get_serial()); let mut node_store_iter = node_store.iter(); for (k, v) in coordinator_store.iter() { if let Some(next) = node_store_iter.next() { diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index 5900f18..b0196ff 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -1,5 +1,6 @@ use crate::messages::{Body, Message, MessageType}; use async_trait::async_trait; +use synevi_network::network::{MemberWithLatency, NodeStatus}; use std::collections::HashMap; use std::sync::{Arc, RwLock}; use std::time::Duration; @@ -9,7 +10,7 @@ use synevi::network::requests::{ RecoverRequest, RecoverResponse, }; use synevi::network::{Network, NetworkInterface, Replica}; -use synevi::{State, SyneviError, T, T0}; +use synevi::{State, SyneviError, T0}; use synevi_network::configure_transport::GetEventResponse; use tokio::sync::mpsc::{Receiver, Sender}; use tokio::sync::Mutex; @@ -70,6 +71,10 @@ impl Network for MaelstromNetwork { } } + fn get_node_status(&self) -> Arc { + todo!() + } + async fn add_member( &self, _id: Ulid, @@ -179,23 +184,20 @@ impl Network for MaelstromNetwork { async fn get_stream_events( &self, _last_applied: Vec, - _self_event: Vec, ) -> Result, SyneviError> { todo!() } - async fn join_electorate(&self, _host: String) -> Result<(u32, Vec), SyneviError> { + async fn join_electorate(&self, _host: String) -> Result { todo!() } - async fn get_member_len(&self) -> u32 { + async fn get_members(&self) -> Vec> { todo!() } async fn report_config( &self, - _last_applied: T, - _last_applied_hash: [u8; 32], _host: String, ) -> Result<(), SyneviError> { todo!() From 7059cb272534941f48418b1af1b77df1eb3c5833 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 16:08:55 +0200 Subject: [PATCH 16/40] fix: Wait handler incorrect state --- synevi_core/src/coordinator.rs | 4 +-- synevi_core/src/node.rs | 39 +++++++++++++++-------------- synevi_core/src/replica.rs | 22 +++++++++------- synevi_core/src/wait_handler.rs | 10 +++++--- synevi_network/src/network.rs | 8 ++---- synevi_persistence/src/mem_store.rs | 3 +++ tests/maelstrom/network.rs | 7 ++---- 7 files changed, 48 insertions(+), 45 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 411cc4e..1cb4fe8 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -66,9 +66,7 @@ where id: u128, ) -> Self { trace!(?id, "Coordinator: New"); - let t0 = node - .event_store - .init_t_zero(node.get_serial()); + let t0 = node.event_store.init_t_zero(node.get_serial()); Coordinator { node, transaction: TransactionStateMachine { diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 992620a..d6eb465 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -83,8 +83,8 @@ where // }); let arc_store = Arc::new(store); + let wait_handler = WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); - let wait_handler = WaitHandler::new(arc_store.clone()); let node = Arc::new(Node { event_store: arc_store, @@ -95,6 +95,7 @@ where executor, self_clone: RwLock::new(None), }); + node.self_clone .write() .expect("Locking self_clone failed") @@ -159,7 +160,7 @@ where }; let arc_store = Arc::new(store); - let wait_handler = WaitHandler::new(arc_store.clone()); + let wait_handler = WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); let node = Arc::new(Node { event_store: arc_store, @@ -256,10 +257,10 @@ where let t_commit = event.t.clone(); let prev_event = self.event_store.get_event(t0_commit)?; + self.event_store.upsert_tx(event)?; self.wait_handler.notify_commit(&t0_commit, &t_commit); - - if !prev_event.is_some_and(|e| e.state > State::Commited && e.dependencies.is_empty()) { + if !prev_event.is_some_and(|e| e.state > State::Commited || e.dependencies.is_empty()) { if let Some(waiter) = self.wait_handler.get_waiter(&t0_commit) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); @@ -298,10 +299,11 @@ where } } println!( - "[{:?}]Applied event: t0: {:?}, t: {:?}", + "[{:?}]Applied event: t0: {:?}, t: {:?}, deps: {:?}", self.get_serial(), event.t_zero, - event.t + event.t, + event.dependencies, ); self.event_store.upsert_tx(event)?; @@ -310,6 +312,10 @@ where } async fn run_check_recovery(&self) { + while !self.is_ready() { + tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + } + let self_clonable = self .self_clone .read() @@ -322,15 +328,14 @@ where CheckResult::NoRecovery => (), CheckResult::RecoverEvent(recover_event) => { let self_clone = self_clonable.clone(); - match tokio::spawn(Coordinator::recover(self_clone, recover_event)).await { - Ok(Ok(_)) => (), - Ok(Err(e)) => { - tracing::error!("Error recovering event: {:?}", e); + tokio::spawn(async move { + match Coordinator::recover(self_clone, recover_event).await { + Ok(_) => (), + Err(e) => { + tracing::error!("JoinError recovering event: {:?}", e); + } } - Err(e) => { - tracing::error!("JoinError recovering event: {:?}", e); - } - } + }); } CheckResult::RecoverUnknown(t0_recover) => { let interface = self.network.get_interface().await; @@ -347,7 +352,6 @@ where } } } - tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } } @@ -388,10 +392,7 @@ where replica: &ReplicaConfig, ) -> Result<(), SyneviError> { // 2.2 else Request stream with events until last_applied (highest t of JoinElectorate) - let mut rcv = self - .network - .get_stream_events(last_applied.into()) - .await?; + let mut rcv = self.network.get_stream_events(last_applied.into()).await?; while let Some(event) = rcv.recv().await { let state: State = event.state.into(); match state { diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 23fafd2..520b955 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -388,9 +388,7 @@ where }), ) .await?; - Ok(JoinElectorateResponse { - member_count, - }) + Ok(JoinElectorateResponse { member_count }) } async fn get_events( @@ -402,10 +400,7 @@ where } let (sdx, rcv) = tokio::sync::mpsc::channel(200); let last_applied = T::try_from(request.last_applied.as_slice())?; - let mut store_rcv = self - .node - .event_store - .get_events_after(last_applied)?; + let mut store_rcv = self.node.event_store.get_events_after(last_applied)?; tokio::spawn(async move { while let Some(Ok(event)) = store_rcv.recv().await { let response = { @@ -484,10 +479,19 @@ where } for member in request.configs { self.node - .add_member(Ulid::from_bytes(member.node_id.as_slice().try_into()?), member.node_serial as u16, member.host, member.ready) + .add_member( + Ulid::from_bytes(member.node_id.as_slice().try_into()?), + member.node_serial as u16, + member.host, + member.ready, + ) .await?; } - self.node.network.get_node_status().members_responded.fetch_add(1, Ordering::Relaxed); + self.node + .network + .get_node_status() + .members_responded + .fetch_add(1, Ordering::Relaxed); Ok(ReportElectorateResponse {}) } } diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 6edc920..0fb59f7 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -62,16 +62,18 @@ where { waiters: Mutex>, store: Arc, + serial: u16, } impl WaitHandler where S: Store, { - pub fn new(store: Arc) -> Self { + pub fn new(store: Arc, serial: u16) -> Self { Self { waiters: Mutex::new(HashMap::default()), store, + serial, } } @@ -107,9 +109,11 @@ where let waiter = waiter_lock.entry(*t0).or_insert(Waiter { waited_since: Instant::now(), - dependency_states: counter, + dependency_states: 0, sender: Vec::new(), }); + waiter.waited_since = Instant::now(); + waiter.dependency_states = counter; waiter.sender.push(sdx); Some(rcv) } @@ -162,7 +166,7 @@ where let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); let mut smallest_hanging_dep = CheckResult::NoRecovery; for (t0, waiter) in waiter_lock.iter_mut() { - if waiter.waited_since.elapsed().as_millis() > 1000 { + if waiter.waited_since.elapsed().as_millis() > 100 { // Get deps and find smallest dep that is not committed / applied let Some(event) = self.store.get_event(*t0).ok().flatten() else { tracing::error!( diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 78f6966..81f3d92 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -120,9 +120,7 @@ where &self, last_applied: Vec, ) -> Result, SyneviError> { - self.as_ref() - .get_stream_events(last_applied) - .await + self.as_ref().get_stream_events(last_applied).await } async fn ready_electorate(&self) -> Result<(), SyneviError> { @@ -389,9 +387,7 @@ impl Network for GrpcNetwork { return Err(SyneviError::NoMembersFound); }; let channel = member.member.channel.clone(); - let request = GetEventRequest { - last_applied, - }; + let request = GetEventRequest { last_applied }; let (sdx, rcv) = tokio::sync::mpsc::channel(200); tokio::spawn(async move { diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index e8dcd0e..55ead39 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -340,6 +340,9 @@ impl InternalStore { } if event.state == State::Applied { + if self.last_applied > event.t { + println!("Node: {} | {:?} > {:?} | Deps: {:?}", self.node_serial, self.last_applied, event.t, event.dependencies); + } assert!(self.last_applied < event.t); self.last_applied = event.t; let hashes = event.hash_event(self.latest_hash); diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index b0196ff..a3519d6 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -1,6 +1,5 @@ use crate::messages::{Body, Message, MessageType}; use async_trait::async_trait; -use synevi_network::network::{MemberWithLatency, NodeStatus}; use std::collections::HashMap; use std::sync::{Arc, RwLock}; use std::time::Duration; @@ -12,6 +11,7 @@ use synevi::network::requests::{ use synevi::network::{Network, NetworkInterface, Replica}; use synevi::{State, SyneviError, T0}; use synevi_network::configure_transport::GetEventResponse; +use synevi_network::network::{MemberWithLatency, NodeStatus}; use tokio::sync::mpsc::{Receiver, Sender}; use tokio::sync::Mutex; use tokio::task::JoinSet; @@ -196,10 +196,7 @@ impl Network for MaelstromNetwork { todo!() } - async fn report_config( - &self, - _host: String, - ) -> Result<(), SyneviError> { + async fn report_config(&self, _host: String) -> Result<(), SyneviError> { todo!() } } From d4eada52433c7b0ee5d5cb338e149a54355a49e7 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 17:17:04 +0200 Subject: [PATCH 17/40] chore: Debug reconfiguration --- synevi_core/src/coordinator.rs | 8 ++++++ synevi_core/src/node.rs | 31 ++++++++++++++++----- synevi_core/src/replica.rs | 18 ++++++++++++- synevi_network/src/network.rs | 42 ++++++++++++++++++----------- synevi_persistence/src/mem_store.rs | 5 +++- tests/consensus_e2e.rs | 1 - 6 files changed, 80 insertions(+), 25 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 1cb4fe8..b7bd3db 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -109,6 +109,7 @@ where }; let network_interface = self.node.network.get_interface().await; + let pre_accepted_responses = network_interface .broadcast(BroadcastRequest::PreAccept( pre_accepted_request, @@ -260,9 +261,12 @@ where #[instrument(level = "trace", skip(self))] async fn apply(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Apply"); + println!("Coordinator: Apply"); let (synevi_result, hashes) = self.execute_consensus().await?; + println!("Coordinator: Apply after execute"); + let applied_request = ApplyRequest { id: self.transaction.id.to_be_bytes().into(), event: self.transaction.get_transaction_bytes(), @@ -273,11 +277,15 @@ where transaction_hash: hashes.transaction_hash.to_vec(), }; + println!("Coordinator: Apply before broadcast"); + let network_interface = self.node.network.get_interface().await; network_interface .broadcast(BroadcastRequest::Apply(applied_request)) .await?; // TODO: This should not be awaited, but can be used to compare hashes + println!("Coordinator: Apply after broadcast"); + synevi_result } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index d6eb465..2a5ae72 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -83,8 +83,8 @@ where // }); let arc_store = Arc::new(store); - let wait_handler = WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); - + let wait_handler = + WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); let node = Arc::new(Node { event_store: arc_store, @@ -160,7 +160,8 @@ where }; let arc_store = Arc::new(store); - let wait_handler = WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); + let wait_handler = + WaitHandler::new(arc_store.clone(), network.get_node_status().info.serial); let node = Arc::new(Node { event_store: arc_store, @@ -234,7 +235,9 @@ where }; let _permit = self.semaphore.acquire().await?; let mut coordinator = Coordinator::new(self.clone(), transaction, id).await; - coordinator.run().await + let result = coordinator.run().await; + + result } pub fn get_stats(&self) -> (u64, u64, u64) { @@ -257,7 +260,7 @@ where let t_commit = event.t.clone(); let prev_event = self.event_store.get_event(t0_commit)?; - + self.event_store.upsert_tx(event)?; self.wait_handler.notify_commit(&t0_commit, &t_commit); if !prev_event.is_some_and(|e| e.state > State::Commited || e.dependencies.is_empty()) { @@ -327,6 +330,11 @@ where match self.wait_handler.check_recovery() { CheckResult::NoRecovery => (), CheckResult::RecoverEvent(recover_event) => { + println!( + "{}, Recovering event: {:?}", + self.get_serial(), + recover_event + ); let self_clone = self_clonable.clone(); tokio::spawn(async move { match Coordinator::recover(self_clone, recover_event).await { @@ -338,6 +346,11 @@ where }); } CheckResult::RecoverUnknown(t0_recover) => { + println!( + "{}, Recovering unknown: {:?}", + self.get_serial(), + t0_recover + ); let interface = self.network.get_interface().await; match interface.broadcast_recovery(t0_recover).await { Ok(true) => (), @@ -363,10 +376,11 @@ where member_host: String, ) -> Result<(), SyneviError> { // 1. Broadcast self_config to other member + println!("{} Before join", self.get_serial()); let expected = self.network.join_electorate(member_host).await?; - // 2. wait for JoinElectorate responses with expected majority and config from others + println!("{} Waiting for responded", self.get_serial()); while self .network .get_node_status() @@ -377,12 +391,17 @@ where tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } + println!("{} Sync events", self.get_serial()); let (last_applied, _) = self.event_store.last_applied(); self.sync_events(last_applied, &replica).await?; // 3. Send ReadyJoinElectorate && set myself to ready self.set_ready(); + println!("{}, Before electorate", self.get_serial()); self.network.ready_electorate().await?; + + println!("{}, Ready electorate finished", self.get_serial()); + Ok(()) } diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 520b955..59296ee 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -58,11 +58,17 @@ where request: PreAcceptRequest, _node_serial: u16, ) -> Result { + let t0 = T0::try_from(request.timestamp_zero.as_slice())?; + + println!( + "Received pre-accept request for event: {:?} @ {:?}", + t0, _node_serial + ); + if !self.node.is_ready() { return Ok(PreAcceptResponse::default()); } - let t0 = T0::try_from(request.timestamp_zero.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: PreAccept"); @@ -178,6 +184,14 @@ where #[instrument(level = "trace", skip(self, request))] async fn apply(&self, request: ApplyRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; + + if self.node.get_serial() == 2 { + println!( + "Received apply request for event: {:?} @ {:?}", + t_zero, + self.node.get_serial() + ); + } let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: Apply"); @@ -378,6 +392,7 @@ where let node = self.node.clone(); let member_count = self.node.network.get_members().await.len() as u32; let self_event = Ulid::new(); + println!("Before joining transaction"); let _res = node .internal_transaction( self_event.0, @@ -388,6 +403,7 @@ where }), ) .await?; + println!("After joining transaction"); Ok(JoinElectorateResponse { member_count }) } diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 81f3d92..6a31426 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -508,6 +508,7 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::Apply(req) => { await_majority = false; for replica in &self.members { + println!("Replica: {:?}", replica); let ready = replica.member.info.ready.load(Ordering::Relaxed); let channel = replica.member.channel.clone(); let request = req.clone(); @@ -549,6 +550,7 @@ impl NetworkInterface for GrpcNetworkSet { // Poll majority // TODO: Electorates for PA ? + println!("Await majority: {}", await_majority); if await_majority { while let Some(response) = responses.join_next().await { // TODO: Resiliency to network errors @@ -593,22 +595,30 @@ impl NetworkInterface for GrpcNetworkSet { } } } else { - //tokio::spawn(async move { - while let Some(r) = &responses.join_next().await { - match r { - Ok(Err(e)) => { - println!("Apply: Error in response: {:?}", e); - tracing::error!("Apply: Error in response: {:?}", e); - continue; - } - Err(_) => { - println!("Apply: Join error"); - tracing::error!("Apply: Join error"); - continue; - } - _ => {} - }; - } + println!("Waiting for results"); + + tokio::spawn(async move { + while let Some(r) = &responses.join_next().await { + println!("Got result: {:?}", r); + + match r { + Ok(Err(e)) => { + println!("Apply: Error in response: {:?}", e); + tracing::error!("Apply: Error in response: {:?}", e); + continue; + } + Err(_) => { + println!("Apply: Join error"); + tracing::error!("Apply: Join error"); + continue; + } + _ => {} + }; + } + }); + + println!("Waited for results"); + //}); return Ok(result); // No majority needed -> return early } diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 55ead39..7e51557 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -341,7 +341,10 @@ impl InternalStore { if event.state == State::Applied { if self.last_applied > event.t { - println!("Node: {} | {:?} > {:?} | Deps: {:?}", self.node_serial, self.last_applied, event.t, event.dependencies); + println!( + "Node: {} | {:?} > {:?} | Deps: {:?}", + self.node_serial, self.last_applied, event.t, event.dependencies + ); } assert!(self.last_applied < event.t); self.last_applied = event.t; diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index efd9adc..b9a620b 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -397,7 +397,6 @@ mod tests { // Copy & create db let test_path = format!("/dev/shm/{m}/"); fs::create_dir(&test_path).await.unwrap(); - dbg!(&test_path); let store = PersistentStore::new(test_path, i as u16).unwrap(); //let store = MemStore::new(i as u16).unwrap(); let node = Node::new_with_member( From b80b3247f381645a040411b0226ea8a3a0327cba Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Thu, 17 Oct 2024 23:04:17 +0200 Subject: [PATCH 18/40] chore: Added debug for mismatched hashes --- synevi_core/src/replica.rs | 19 ++++++++++++++++--- synevi_core/src/wait_handler.rs | 4 +--- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 59296ee..46d7180 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -269,9 +269,22 @@ where .node .event_store .get_and_update_hash(t_zero, hash.into())?; - if request.transaction_hash != hashes.transaction_hash - || request.execution_hash != hashes.execution_hash - { + if request.transaction_hash != hashes.transaction_hash { + println!( + "{} | Mismatched transaction hash in Tx {:?}", + self.node.get_serial(), + t_zero + ); + Err(SyneviError::MismatchedHashes) + } else if request.execution_hash != hashes.execution_hash { + println!( + "{} | Mismatched execution hash in Tx {:?} | expected: {:?} != {:?}", + self.node.get_serial(), + t_zero, + request.execution_hash, + hashes.execution_hash + ); + Err(SyneviError::MismatchedHashes) } else { Ok(ApplyResponse {}) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 0fb59f7..482f02e 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -62,18 +62,16 @@ where { waiters: Mutex>, store: Arc, - serial: u16, } impl WaitHandler where S: Store, { - pub fn new(store: Arc, serial: u16) -> Self { + pub fn new(store: Arc, _serial: u16) -> Self { Self { waiters: Mutex::new(HashMap::default()), store, - serial, } } From fecbc5c5ff986ea3b60a09f00a020d7cb4ce90ef Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 14:06:41 +0200 Subject: [PATCH 19/40] feat: Execution & Apply as one operation --- synevi_core/src/coordinator.rs | 104 ++++++------- synevi_core/src/node.rs | 137 +++++++++++++++-- synevi_core/src/replica.rs | 220 ++++++++++++++------------- synevi_core/src/utils.rs | 2 +- synevi_core/src/wait_handler.rs | 4 +- synevi_network/src/network.rs | 3 +- synevi_persistence/src/lmdb_store.rs | 136 +++++++++-------- synevi_persistence/src/mem_store.rs | 81 +++++----- synevi_types/src/error.rs | 6 +- synevi_types/src/traits.rs | 14 +- synevi_types/src/types.rs | 14 +- tests/maelstrom/network.rs | 6 +- 12 files changed, 435 insertions(+), 292 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index b7bd3db..de7d169 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -2,7 +2,6 @@ use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; use ahash::RandomState; use serde::Serialize; -use sha3::{Digest, Sha3_256}; use std::collections::HashSet; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -14,7 +13,7 @@ use synevi_network::network::{BroadcastRequest, Network, NetworkInterface}; use synevi_network::utils::IntoInner; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalExecution, InternalSyneviResult, RecoverEvent, RecoveryState, + ExecutorResult, Hashes, InternalSyneviResult, RecoverEvent, RecoveryState, TransactionPayload, }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; @@ -33,7 +32,7 @@ where pub transaction: TransactionStateMachine, } -#[derive(Clone, Debug, Default, PartialEq, Eq)] +#[derive(Clone, Debug, Default)] //, PartialEq, Eq)] pub struct TransactionStateMachine { pub id: u128, pub state: State, @@ -42,6 +41,7 @@ pub struct TransactionStateMachine { pub t: T, pub dependencies: HashSet, pub ballot: Ballot, + pub hashes: Option, } impl TransactionStateMachine @@ -77,6 +77,7 @@ where t: T(*t0), dependencies: HashSet::default(), ballot: Ballot::default(), + hashes: None, }, } } @@ -253,7 +254,7 @@ where #[instrument(level = "trace", skip(self))] async fn commit_consensus(&mut self) -> Result<(), SyneviError> { - self.transaction.state = State::Commited; + self.transaction.state = State::Committed; self.node.commit((&self.transaction).into()).await?; Ok(()) } @@ -295,52 +296,52 @@ where ) -> Result<(InternalSyneviResult, Hashes), SyneviError> { self.transaction.state = State::Applied; - self.node.apply((&self.transaction).into()).await?; - - let result = match &self.transaction.transaction { - TransactionPayload::None => Err(SyneviError::TransactionNotFound), - TransactionPayload::External(tx) => self - .node - .executor - .execute(tx.clone()) - .await - .map(|e| ExecutorResult::External(e)), - TransactionPayload::Internal(request) => { - let result = match request { - InternalExecution::JoinElectorate { - id, - serial, - new_node_host, - } => { - let res = self - .node - .add_member(*id, *serial, new_node_host.clone(), false) - .await; - self.node - .network - .report_config(new_node_host.to_string()) - .await?; - res - } - InternalExecution::ReadyElectorate { id, serial } => { - self.node.ready_member(*id, *serial).await - } - }; - match result { - Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), - Err(err) => Ok(ExecutorResult::Internal(Err(err))), - } - } - }; - - let mut hasher = Sha3_256::new(); - postcard::to_io(&result, &mut hasher)?; - let hash = hasher.finalize(); - let hashes = self - .node - .event_store - .get_and_update_hash(self.transaction.t_zero, hash.into())?; - Ok((result, hashes)) + let (result, hashes) = self.node.apply((&self.transaction).into(), None).await?; + + // let result = match &self.transaction.transaction { + // TransactionPayload::None => Err(SyneviError::TransactionNotFound), + // TransactionPayload::External(tx) => self + // .node + // .executor + // .execute(tx.clone()) + // .await + // .map(|e| ExecutorResult::External(e)), + // TransactionPayload::Internal(request) => { + // let result = match request { + // InternalExecution::JoinElectorate { + // id, + // serial, + // new_node_host, + // } => { + // let res = self + // .node + // .add_member(*id, *serial, new_node_host.clone(), false) + // .await; + // self.node + // .network + // .report_config(new_node_host.to_string()) + // .await?; + // res + // } + // InternalExecution::ReadyElectorate { id, serial } => { + // self.node.ready_member(*id, *serial).await + // } + // }; + // match result { + // Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), + // Err(err) => Ok(ExecutorResult::Internal(Err(err))), + // } + // } + // }; + + // let mut hasher = Sha3_256::new(); + // postcard::to_io(&result, &mut hasher)?; + // let hash = hasher.finalize(); + // let hashes = self + // .node + // .event_store + // .get_and_update_hash(self.transaction.t_zero, hash.into())?; + Ok((result.unwrap(), hashes)) } #[instrument(level = "trace", skip(node))] @@ -371,6 +372,7 @@ where state: recover_event.state, id: recover_event.id, dependencies: recover_event.dependencies.clone(), + hashes: None, }, }; @@ -490,7 +492,7 @@ where // Wait for deps Ok(match self.transaction.state { State::Applied => RecoveryState::Recovered(self.apply().await?), - State::Commited => RecoveryState::Recovered(self.commit().await?), + State::Committed => RecoveryState::Recovered(self.commit().await?), State::Accepted => RecoveryState::Recovered(self.accept().await?), State::PreAccepted => { diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 2a5ae72..cb3f70d 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -1,6 +1,7 @@ use crate::coordinator::Coordinator; use crate::replica::ReplicaConfig; use crate::wait_handler::{CheckResult, WaitHandler}; +use sha3::{Digest, Sha3_256}; use std::fmt::Debug; use std::sync::atomic::Ordering; use std::sync::RwLock; @@ -11,9 +12,10 @@ use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, InternalSyneviResult, SyneviResult, TransactionPayload, UpsertEvent, + ExecutorResult, Hashes, InternalExecution, InternalSyneviResult, SyneviResult, + TransactionPayload, UpsertEvent, }; -use synevi_types::{Executor, State, SyneviError, T}; +use synevi_types::{Executor, State, SyneviError, Transaction, T}; use tracing::instrument; use ulid::Ulid; @@ -263,7 +265,7 @@ where self.event_store.upsert_tx(event)?; self.wait_handler.notify_commit(&t0_commit, &t_commit); - if !prev_event.is_some_and(|e| e.state > State::Commited || e.dependencies.is_empty()) { + if !prev_event.is_some_and(|e| e.state > State::Committed || e.dependencies.is_empty()) { if let Some(waiter) = self.wait_handler.get_waiter(&t0_commit) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); @@ -275,16 +277,22 @@ where } #[instrument(level = "trace", skip(self))] - pub async fn apply(&self, event: UpsertEvent) -> Result<(), SyneviError> { + pub async fn apply( + &self, + mut event: UpsertEvent, + request_hashes: Option, + ) -> Result<(Option>, Hashes), SyneviError> { let t0_apply = event.t_zero.clone(); - let prev_event = self.event_store.get_event(t0_apply)?; - - let needs_wait = if let Some(prev_event) = prev_event { + let needs_wait = if let Some(prev_event) = self.event_store.get_event(t0_apply)? { + if prev_event.state == State::Applied { + println!("Skipped at first apply check"); + return Ok((None, prev_event.hashes.unwrap())); + } prev_event.state < State::Applied } else { let mut commit_event = event.clone(); - commit_event.state = State::Commited; + commit_event.state = State::Committed; self.commit(commit_event).await?; true }; @@ -295,12 +303,20 @@ where && needs_wait { if let Some(waiter) = self.wait_handler.get_waiter(&t0_apply) { + println!("[{}] waiting on {:?}", self.get_serial(), t0_apply); waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); SyneviError::ReceiveError(format!("Error waiting for commit")) })?; } } + + if let Some(prev_event) = self.event_store.get_event(t0_apply)? { + if prev_event.state == State::Applied { + println!("Skipped at second apply check"); + return Ok((None, prev_event.hashes.unwrap())); + } + } println!( "[{:?}]Applied event: t0: {:?}, t: {:?}, deps: {:?}", self.get_serial(), @@ -309,15 +325,108 @@ where event.dependencies, ); + // - Check transaction hash -> SyneviError::MismatchingTransactionHash + let mut node_hashes = self + .event_store + .get_and_check_transaction_hash(event.clone()); + if let Some(hashes) = &request_hashes { + if hashes.transaction_hash != node_hashes.transaction_hash { + println!( + "{}", + format!( + "Node: {} +request: {:?} +got: {:?}", + self.get_serial(), + hashes.transaction_hash, + node_hashes.transaction_hash + ) + ); + return Err(SyneviError::MismatchedTransactionHashes); + } + } + + // - Execute + let transaction = TransactionPayload::from_bytes( + event + .transaction + .clone() + .ok_or_else(|| SyneviError::TransactionNotFound)?, + )?; + let result = self.execute(transaction).await; + + // - Check execution hash -> SyneviError::MismatchingExecutionHash + let mut hasher = Sha3_256::new(); + postcard::to_io(&result, &mut hasher)?; + let execution_hash: [u8; 32] = hasher.finalize().into(); + if let Some(hashes) = request_hashes { + if hashes.execution_hash != execution_hash { + return Err(SyneviError::MismatchedExecutionHashes); + } + } + node_hashes.execution_hash = execution_hash; + event.hashes = Some(node_hashes.clone()); + + // - Upsert self.event_store.upsert_tx(event)?; self.wait_handler.notify_apply(&t0_apply); - Ok(()) + Ok((Some(result), node_hashes)) + } + + async fn execute( + &self, + transaction: TransactionPayload<::Tx>, + ) -> Result::Tx>, SyneviError> { + // TODO: Refactor in execute function + let result = match transaction { + TransactionPayload::None => { + return Err(SyneviError::TransactionNotFound); + } + TransactionPayload::External(tx) => self + .executor + .execute(tx) + .await + .map(|e| ExecutorResult::<::Tx>::External(e)), + TransactionPayload::Internal(request) => { + // TODO: Build special execution + let result = match &request { + InternalExecution::JoinElectorate { + id, + serial, + new_node_host, + } => { + if id != &self.get_ulid() { + let res = self + .add_member(*id, *serial, new_node_host.clone(), false) + .await; + self.network.report_config(new_node_host.clone()).await?; + res + } else { + Ok(()) + } + } + InternalExecution::ReadyElectorate { id, serial } => { + if id != &self.get_ulid() { + self.ready_member(*id, *serial).await + } else { + Ok(()) + } + } + }; + match result { + Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), + Err(err) => Ok(ExecutorResult::Internal(Err(err))), + } + } + }; + result } async fn run_check_recovery(&self) { while !self.is_ready() { tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; } + println!("{} ready", self.get_serial()); let self_clonable = self .self_clone @@ -428,7 +537,7 @@ where }) .await?; } - State::Commited => { + State::Committed => { replica .commit(CommitRequest { id: event.id, @@ -454,6 +563,7 @@ mod tests { use std::net::SocketAddr; use std::str::FromStr; use std::sync::Arc; + use std::time::Duration; use synevi_network::network::GrpcNetwork; use synevi_network::network::Network; use synevi_types::traits::Store; @@ -527,6 +637,9 @@ mod tests { .await .unwrap(); + // This sleep accounts for replicas apply step, that is not neccessarily completed after + // the coordinator returns its result + tokio::time::sleep(Duration::from_secs(1)).await; let coord = coordinator.event_store.get_event_store(); for node in nodes { assert_eq!( @@ -588,6 +701,10 @@ mod tests { .unwrap() .unwrap(); + // This sleep accounts for replicas apply step, that is not neccessarily completed after + // the coordinator returns its result + tokio::time::sleep(Duration::from_secs(1)).await; + let coordinator_store: BTreeMap = coordinator .event_store .get_event_store() diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 46d7180..798bcf2 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -1,7 +1,6 @@ use crate::coordinator::Coordinator; use crate::node::Node; use crate::utils::{from_dependency, into_dependency}; -use sha3::{Digest, Sha3_256}; use std::sync::atomic::Ordering; use std::sync::Arc; use synevi_network::configure_transport::{ @@ -18,9 +17,11 @@ use synevi_network::network::Network; use synevi_network::reconfiguration::Reconfiguration; use synevi_network::replica::Replica; use synevi_types::traits::Store; -use synevi_types::types::{ExecutorResult, InternalExecution, TransactionPayload, UpsertEvent}; +use synevi_types::types::{ + Hashes, InternalExecution, TransactionPayload, UpsertEvent, +}; use synevi_types::{Ballot, Executor, State, T, T0}; -use synevi_types::{SyneviError, Transaction}; +use synevi_types::SyneviError; use tokio::sync::mpsc::Receiver; use tracing::{instrument, trace}; use ulid::Ulid; @@ -60,11 +61,6 @@ where ) -> Result { let t0 = T0::try_from(request.timestamp_zero.as_slice())?; - println!( - "Received pre-accept request for event: {:?} @ {:?}", - t0, _node_serial - ); - if !self.node.is_ready() { return Ok(PreAcceptResponse::default()); } @@ -144,7 +140,7 @@ where transaction: Some(request.event), dependencies: Some(from_dependency(request.dependencies)?), ballot: Some(request_ballot), - execution_hash: None, + hashes: None, })?; self.node.event_store.get_tx_dependencies(&t, &t_zero) @@ -171,11 +167,11 @@ where id: request_id, t_zero, t, - state: State::Commited, + state: State::Committed, transaction: Some(request.event), dependencies: Some(deps), ballot: None, - execution_hash: None, + hashes: None, }) .await?; Ok(CommitResponse {}) @@ -185,110 +181,122 @@ where async fn apply(&self, request: ApplyRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; - if self.node.get_serial() == 2 { - println!( - "Received apply request for event: {:?} @ {:?}", - t_zero, - self.node.get_serial() - ); - } + println!( + "Received apply request for event: {:?} @ {:?}", + t_zero, + self.node.get_serial() + ); let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: Apply"); - let transaction: TransactionPayload<::Tx> = - TransactionPayload::from_bytes(request.event.clone())?; - let deps = from_dependency(request.dependencies.clone())?; - self.node - .apply(UpsertEvent { - id: request_id, - t_zero, - t, - state: State::Applied, - transaction: Some(request.event), - dependencies: Some(deps), - ballot: None, - execution_hash: None, - }) + let _ = self.node + .apply( + UpsertEvent { + id: request_id, + t_zero, + t, + state: State::Applied, + transaction: Some(request.event), + dependencies: Some(deps), + ballot: None, + hashes: None, + }, + Some(Hashes { + transaction_hash: request + .transaction_hash + .try_into() + .map_err(|_e| SyneviError::MissingTransactionHash)?, + execution_hash: request + .execution_hash + .try_into() + .map_err(|_e| SyneviError::MissingExecutionHash)?, + ..Default::default() + }), + ) .await?; + Ok(ApplyResponse {}) // TODO: Refactor in execute function - let result = match transaction { - TransactionPayload::None => { - return Err(SyneviError::TransactionNotFound); - } - TransactionPayload::External(tx) => self - .node - .executor - .execute(tx) - .await - .map(|e| ExecutorResult::<::Tx>::External(e)), - TransactionPayload::Internal(request) => { - // TODO: Build special execution - let result = match &request { - InternalExecution::JoinElectorate { - id, - serial, - new_node_host, - } => { - if id != &self.node.get_ulid() { - let res = self - .node - .add_member(*id, *serial, new_node_host.clone(), false) - .await; - self.node - .network - .report_config(new_node_host.clone()) - .await?; - res - } else { - Ok(()) - } - } - InternalExecution::ReadyElectorate { id, serial } => { - if id != &self.node.get_ulid() { - self.node.ready_member(*id, *serial).await - } else { - Ok(()) - } - } - }; - match result { - Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), - Err(err) => Ok(ExecutorResult::Internal(Err(err))), - } - } - }; - - let mut hasher = Sha3_256::new(); - postcard::to_io(&result, &mut hasher)?; - let hash = hasher.finalize(); - let hashes = self - .node - .event_store - .get_and_update_hash(t_zero, hash.into())?; - if request.transaction_hash != hashes.transaction_hash { - println!( - "{} | Mismatched transaction hash in Tx {:?}", - self.node.get_serial(), - t_zero - ); - Err(SyneviError::MismatchedHashes) - } else if request.execution_hash != hashes.execution_hash { - println!( - "{} | Mismatched execution hash in Tx {:?} | expected: {:?} != {:?}", - self.node.get_serial(), - t_zero, - request.execution_hash, - hashes.execution_hash - ); - - Err(SyneviError::MismatchedHashes) - } else { - Ok(ApplyResponse {}) - } + // + // let transaction: TransactionPayload<::Tx> = + // TransactionPayload::from_bytes(request.event.clone())?; + //let result = match transaction { + // TransactionPayload::None => { + // return Err(SyneviError::TransactionNotFound); + // } + // TransactionPayload::External(tx) => self + // .node + // .executor + // .execute(tx) + // .await + // .map(|e| ExecutorResult::<::Tx>::External(e)), + // TransactionPayload::Internal(request) => { + // // TODO: Build special execution + // let result = match &request { + // InternalExecution::JoinElectorate { + // id, + // serial, + // new_node_host, + // } => { + // if id != &self.node.get_ulid() { + // let res = self + // .node + // .add_member(*id, *serial, new_node_host.clone(), false) + // .await; + // self.node + // .network + // .report_config(new_node_host.clone()) + // .await?; + // res + // } else { + // Ok(()) + // } + // } + // InternalExecution::ReadyElectorate { id, serial } => { + // if id != &self.node.get_ulid() { + // self.node.ready_member(*id, *serial).await + // } else { + // Ok(()) + // } + // } + // }; + // match result { + // Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), + // Err(err) => Ok(ExecutorResult::Internal(Err(err))), + // } + // } + //}; + + //let mut hasher = Sha3_256::new(); + //postcard::to_io(&result, &mut hasher)?; + //let hash = hasher.finalize(); + //let hashes = self + // .node + // .event_store + // .get_and_update_hash(t_zero, hash.into())?; + //if request.transaction_hash != hashes.transaction_hash { + // println!( + // "{} | Mismatched transaction hash in Tx {:?}", + // self.node.get_serial(), + // t_zero + // ); + // Err(SyneviError::MismatchedHashes) + //} else if request.execution_hash != hashes.execution_hash { + // println!( + // "{} | Mismatched execution hash in Tx {:?} | expected: {:?} != {:?}", + // self.node.get_serial(), + // t_zero, + // request.execution_hash, + // hashes.execution_hash + // ); + + // Err(SyneviError::MismatchedHashes) + //} else { + //Ok(ApplyResponse {}) + //} } #[instrument(level = "trace", skip(self))] diff --git a/synevi_core/src/utils.rs b/synevi_core/src/utils.rs index 90f296b..978509d 100644 --- a/synevi_core/src/utils.rs +++ b/synevi_core/src/utils.rs @@ -36,7 +36,7 @@ where transaction: Some(value.transaction.as_bytes()), dependencies: Some(value.dependencies.clone()), ballot: Some(value.ballot), - execution_hash: None, + hashes: value.hashes.clone(), } } } diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 482f02e..3374879 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -91,7 +91,7 @@ where }; match dep_event.state { - State::Commited if dep_event.t > event.t => { + State::Committed if dep_event.t > event.t => { counter += 1; } State::Applied => { @@ -182,7 +182,7 @@ where continue; } match event_dep.state { - State::Commited => { + State::Committed => { if event_dep.t > event.t { // Dependency is newer than event (and already commited) continue; diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 6a31426..e5170cb 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -597,7 +597,8 @@ impl NetworkInterface for GrpcNetworkSet { } else { println!("Waiting for results"); - tokio::spawn(async move { + tokio::spawn( + async move { while let Some(r) = &responses.join_next().await { println!("Got result: {:?}", r); diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index e9ec24f..079e8d5 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -218,24 +218,24 @@ impl Store for PersistentStore { .get_event(t_zero) } - fn get_and_update_hash( - &self, - t_zero: T0, - execution_hash: [u8; 32], - ) -> Result { - self.data - .lock() - .expect("poisoned lock, aborting") - .get_and_update_hash(t_zero, execution_hash) - } - - #[instrument(level = "trace", skip(self))] - fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - self.data - .lock() - .expect("poisoned lock, aborting") - .last_applied_hash() - } +// fn get_and_update_hash( +// &self, +// t_zero: T0, +// execution_hash: [u8; 32], +// ) -> Result { +// self.data +// .lock() +// .expect("poisoned lock, aborting") +// .get_and_update_hash(t_zero, execution_hash) +// } +// +// #[instrument(level = "trace", skip(self))] +// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { +// self.data +// .lock() +// .expect("poisoned lock, aborting") +// .last_applied_hash() +// } fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.data.lock().expect("poisoned lock, aborting"); @@ -245,6 +245,14 @@ impl Store for PersistentStore { .into_time(); Ok(()) } + + fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes { + let lock = self.data.lock().expect("poisoned lock, aborting"); + let mut event = Event::from(event); + event.state = State::Applied; + println!("{:?}, {:?}", event, lock.latest_hash); + event.hash_event(lock.latest_hash) + } } impl InternalData { @@ -384,7 +392,7 @@ impl InternalData { assert!(last_t < event.t); self.last_applied = event.t; - let hashes = event.hash_event(self.latest_hash); + let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes.clone()); @@ -447,7 +455,7 @@ impl InternalData { assert!(last_t < event.t); self.last_applied = event.t; - let hashes = event.hash_event(self.latest_hash); + let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes.clone()); }; @@ -497,7 +505,7 @@ impl InternalData { } } } - State::Commited => { + State::Committed => { if dep_event .dependencies .iter() @@ -644,49 +652,49 @@ impl InternalData { Ok(event) } - fn get_and_update_hash( - &self, - t_zero: T0, - execution_hash: [u8; 32], - ) -> Result { - let t_zero = t_zero.get_inner(); - let mut write_txn = self.db.write_txn()?; - let db: EventDb = self - .db - .open_database(&write_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - let Some(mut event) = db.get(&write_txn, &t_zero)? else { - return Err(SyneviError::EventNotFound(t_zero)); - }; - let Some(mut hashes) = event.hashes else { - return Err(SyneviError::MissingTransactionHash); - }; - hashes.execution_hash = execution_hash; - event.hashes = Some(hashes.clone()); - - db.put(&mut write_txn, &t_zero, &event)?; - write_txn.commit()?; - Ok(hashes) - } - - fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - let last = self.last_applied; - let last_t0 = self - .mappings - .get(&last) - .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; - let read_txn = self.db.read_txn()?; - let db: EventDb = self - .db - .open_database(&read_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - let event = db - .get(&read_txn, &last_t0.get_inner())? - .ok_or_else(|| SyneviError::EventNotFound(last_t0.get_inner()))? - .hashes - .ok_or_else(|| SyneviError::MissingExecutionHash)?; - Ok((last, event.execution_hash)) - } + // fn get_and_update_hash( + // &self, + // t_zero: T0, + // execution_hash: [u8; 32], + // ) -> Result { + // let t_zero = t_zero.get_inner(); + // let mut write_txn = self.db.write_txn()?; + // let db: EventDb = self + // .db + // .open_database(&write_txn, Some(EVENT_DB_NAME))? + // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; + // let Some(mut event) = db.get(&write_txn, &t_zero)? else { + // return Err(SyneviError::EventNotFound(t_zero)); + // }; + // let Some(mut hashes) = event.hashes else { + // return Err(SyneviError::MissingTransactionHash); + // }; + // hashes.execution_hash = execution_hash; + // event.hashes = Some(hashes.clone()); + + // db.put(&mut write_txn, &t_zero, &event)?; + // write_txn.commit()?; + // Ok(hashes) + // } + + // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + // let last = self.last_applied; + // let last_t0 = self + // .mappings + // .get(&last) + // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; + // let read_txn = self.db.read_txn()?; + // let db: EventDb = self + // .db + // .open_database(&read_txn, Some(EVENT_DB_NAME))? + // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; + // let event = db + // .get(&read_txn, &last_t0.get_inner())? + // .ok_or_else(|| SyneviError::EventNotFound(last_t0.get_inner()))? + // .hashes + // .ok_or_else(|| SyneviError::MissingExecutionHash)?; + // Ok((last, event.execution_hash)) + // } } #[cfg(test)] diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 7e51557..1ee4669 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -149,40 +149,40 @@ impl Store for MemStore { .cloned()) } - fn get_and_update_hash( - &self, - t_zero: T0, - execution_hash: [u8; 32], - ) -> Result { - let mut lock = self.store.lock().expect("poisoned lock, aborting"); - if let Some(event) = lock.events.get_mut(&t_zero) { - let hashes = event - .hashes - .as_mut() - .ok_or_else(|| SyneviError::MissingTransactionHash)?; - hashes.execution_hash = execution_hash; - Ok(hashes.clone()) - } else { - Err(SyneviError::EventNotFound(t_zero.get_inner())) - } - } - - fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - let lock = self.store.lock().expect("poisoned lock, aborting"); - let last = lock.last_applied; - let last_t0 = lock - .mappings - .get(&last) - .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; - let hash = lock - .events - .get(last_t0) - .cloned() - .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))? - .hashes - .ok_or_else(|| SyneviError::MissingExecutionHash)?; - Ok((last, hash.execution_hash)) - } +// fn get_and_update_hash( +// &self, +// t_zero: T0, +// execution_hash: [u8; 32], +// ) -> Result { +// let mut lock = self.store.lock().expect("poisoned lock, aborting"); +// if let Some(event) = lock.events.get_mut(&t_zero) { +// let hashes = event +// .hashes +// .as_mut() +// .ok_or_else(|| SyneviError::MissingTransactionHash)?; +// hashes.execution_hash = execution_hash; +// Ok(hashes.clone()) +// } else { +// Err(SyneviError::EventNotFound(t_zero.get_inner())) +// } +// } +// +// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { +// let lock = self.store.lock().expect("poisoned lock, aborting"); +// let last = lock.last_applied; +// let last_t0 = lock +// .mappings +// .get(&last) +// .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; +// let hash = lock +// .events +// .get(last_t0) +// .cloned() +// .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))? +// .hashes +// .ok_or_else(|| SyneviError::MissingExecutionHash)?; +// Ok((last, hash.execution_hash)) +// } fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.store.lock().expect("poisoned lock, aborting"); @@ -192,6 +192,13 @@ impl Store for MemStore { .into_time(); Ok(()) } + + fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes { + let lock = self.store.lock().expect("poisoned lock, aborting"); + let mut event = Event::from(event); + event.state = State::Applied; + event.hash_event(lock.latest_hash) + } } impl InternalStore { @@ -297,7 +304,7 @@ impl InternalStore { } assert!(self.last_applied < event.t); self.last_applied = event.t; - let hashes = event.hash_event(self.latest_hash); + let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes); self.events.insert(upsert_event.t_zero, event); @@ -348,7 +355,7 @@ impl InternalStore { } assert!(self.last_applied < event.t); self.last_applied = event.t; - let hashes = event.hash_event(self.latest_hash); + let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes); }; @@ -391,7 +398,7 @@ impl InternalStore { } } } - State::Commited => { + State::Committed => { if dep_event .dependencies .iter() diff --git a/synevi_types/src/error.rs b/synevi_types/src/error.rs index 1dcb56e..6a8ec74 100644 --- a/synevi_types/src/error.rs +++ b/synevi_types/src/error.rs @@ -66,8 +66,10 @@ pub enum SyneviError { NoMembersFound, #[error("Not ready for transactions")] NotReady, - #[error("Mismatched hashes")] - MismatchedHashes, + #[error("Mismatched execution hashes")] + MismatchedExecutionHashes, + #[error("Mismatched transaction hashes")] + MismatchedTransactionHashes, #[error("Unrecoverable transaction")] UnrecoverableTransaction, #[error("Expected external transaction: {0}")] diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index a53803e..4d93649 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -99,7 +99,7 @@ pub trait Store: Send + Sync + Sized + 'static { fn get_event_store(&self) -> BTreeMap; fn last_applied(&self) -> (T, T0); - fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; +// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; fn get_event(&self, t_zero: T0) -> Result, SyneviError>; fn get_events_after( @@ -107,11 +107,13 @@ pub trait Store: Send + Sync + Sized + 'static { last_applied: T, ) -> Result>, SyneviError>; - fn get_and_update_hash( - &self, - t_zero: T0, - execution_hash: [u8; 32], - ) -> Result; +// fn get_and_update_hash( +// &self, +// t_zero: T0, +// execution_hash: [u8; 32], +// ) -> Result; + + fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes; // Increases the max time to be above the specified guard // Ensures that the guards t0 will not get a fast path afterwards diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index 3bd3f59..c178b7d 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -275,7 +275,7 @@ pub enum State { Undefined = 0, PreAccepted = 1, Accepted = 2, - Commited = 3, + Committed = 3, Applied = 4, } @@ -284,7 +284,7 @@ impl From for State { match value { 1 => Self::PreAccepted, 2 => Self::Accepted, - 3 => Self::Commited, + 3 => Self::Committed, 4 => Self::Applied, _ => Self::Undefined, } @@ -296,7 +296,7 @@ impl From for i32 { match val { State::PreAccepted => 1, State::Accepted => 2, - State::Commited => 3, + State::Committed => 3, State::Applied => 4, _ => 0, } @@ -359,7 +359,7 @@ pub struct UpsertEvent { pub transaction: Option>, pub dependencies: Option>, pub ballot: Option, - pub execution_hash: Option<[u8; 32]>, + pub hashes: Option, } impl Event { @@ -435,11 +435,7 @@ impl From for Event { transaction: value.transaction.unwrap_or_default(), dependencies: value.dependencies.unwrap_or_default(), ballot: value.ballot.unwrap_or_default(), - hashes: value.execution_hash.map(|hash| Hashes { - previous_hash: [0; 32], - transaction_hash: [0; 32], - execution_hash: hash, - }), + hashes: value.hashes, last_updated: SystemTime::now() .duration_since(UNIX_EPOCH) .unwrap() // This must fail if the system clock is before the UNIX_EPOCH diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index a3519d6..3e62ebb 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -282,7 +282,7 @@ impl NetworkInterface for MaelstromNetwork { self.broadcast_responses .lock() .await - .insert((State::Commited, t0), sx); + .insert((State::Committed, t0), sx); for replica in members { if let Err(err) = self .message_sender @@ -308,7 +308,7 @@ impl NetworkInterface for MaelstromNetwork { continue; }; } - (State::Commited, t0) + (State::Committed, t0) } BroadcastRequest::Apply(req) => { let t0 = T0::try_from(req.timestamp_zero.as_slice()).unwrap(); @@ -635,7 +635,7 @@ impl MaelstromNetwork { MessageType::CommitOk { t0 } => { let key = T0::try_from(t0.as_slice())?; let lock = self.broadcast_responses.lock().await; - if let Some(entry) = lock.get(&(State::Commited, key)) { + if let Some(entry) = lock.get(&(State::Committed, key)) { entry .send(BroadcastResponse::Commit(CommitResponse {})) .await?; From 2ba8e37b4eccce8dc510d48bc5f456ea82f02279 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 16:02:05 +0200 Subject: [PATCH 20/40] fix: Get or update transaction hashes --- synevi_core/src/coordinator.rs | 1 + synevi_core/src/lib.rs | 2 +- synevi_core/src/node.rs | 88 +++++++++++++++------------- synevi_core/src/replica.rs | 20 ++++--- synevi_network/src/network.rs | 15 ++--- synevi_persistence/src/lmdb_store.rs | 55 ++++++++++------- synevi_persistence/src/mem_store.rs | 87 +++++++++++++++------------ synevi_types/src/traits.rs | 2 +- tests/consensus_e2e.rs | 9 ++- tests/maelstrom/network.rs | 2 +- 10 files changed, 155 insertions(+), 126 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index de7d169..3a58269 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -341,6 +341,7 @@ where // .node // .event_store // .get_and_update_hash(self.transaction.t_zero, hash.into())?; + Ok((result.unwrap(), hashes)) } diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index 6ced01a..b8e22a3 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -105,7 +105,7 @@ pub mod tests { let (_, rcv) = tokio::sync::mpsc::channel(1); Ok(rcv) } - async fn ready_electorate(&self) -> Result<(), SyneviError> { + async fn ready_electorate(&self, _host: String) -> Result<(), SyneviError> { Ok(()) } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index cb3f70d..984e721 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -16,6 +16,7 @@ use synevi_types::types::{ TransactionPayload, UpsertEvent, }; use synevi_types::{Executor, State, SyneviError, Transaction, T}; +use tokio::task::JoinSet; use tracing::instrument; use ulid::Ulid; @@ -273,6 +274,7 @@ where })? }; } + Ok(()) } @@ -285,10 +287,6 @@ where let t0_apply = event.t_zero.clone(); let needs_wait = if let Some(prev_event) = self.event_store.get_event(t0_apply)? { - if prev_event.state == State::Applied { - println!("Skipped at first apply check"); - return Ok((None, prev_event.hashes.unwrap())); - } prev_event.state < State::Applied } else { let mut commit_event = event.clone(); @@ -311,12 +309,6 @@ where } } - if let Some(prev_event) = self.event_store.get_event(t0_apply)? { - if prev_event.state == State::Applied { - println!("Skipped at second apply check"); - return Ok((None, prev_event.hashes.unwrap())); - } - } println!( "[{:?}]Applied event: t0: {:?}, t: {:?}, deps: {:?}", self.get_serial(), @@ -328,19 +320,17 @@ where // - Check transaction hash -> SyneviError::MismatchingTransactionHash let mut node_hashes = self .event_store - .get_and_check_transaction_hash(event.clone()); + .get_or_update_transaction_hash(event.clone())?; + if let Some(hashes) = &request_hashes { if hashes.transaction_hash != node_hashes.transaction_hash { println!( - "{}", - format!( - "Node: {} + "Node: {} request: {:?} got: {:?}", - self.get_serial(), - hashes.transaction_hash, - node_hashes.transaction_hash - ) + self.get_serial(), + hashes.transaction_hash, + node_hashes.transaction_hash ); return Err(SyneviError::MismatchedTransactionHashes); } @@ -424,7 +414,7 @@ got: {:?}", async fn run_check_recovery(&self) { while !self.is_ready() { - tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } println!("{} ready", self.get_serial()); @@ -486,7 +476,7 @@ got: {:?}", ) -> Result<(), SyneviError> { // 1. Broadcast self_config to other member println!("{} Before join", self.get_serial()); - let expected = self.network.join_electorate(member_host).await?; + let expected = self.network.join_electorate(member_host.clone()).await?; // 2. wait for JoinElectorate responses with expected majority and config from others println!("{} Waiting for responded", self.get_serial()); @@ -507,7 +497,7 @@ got: {:?}", // 3. Send ReadyJoinElectorate && set myself to ready self.set_ready(); println!("{}, Before electorate", self.get_serial()); - self.network.ready_electorate().await?; + self.network.ready_electorate(member_host).await?; println!("{}, Ready electorate finished", self.get_serial()); @@ -521,36 +511,52 @@ got: {:?}", ) -> Result<(), SyneviError> { // 2.2 else Request stream with events until last_applied (highest t of JoinElectorate) let mut rcv = self.network.get_stream_events(last_applied.into()).await?; + let mut join_set = JoinSet::new(); while let Some(event) = rcv.recv().await { let state: State = event.state.into(); match state { State::Applied => { - replica - .apply(ApplyRequest { - id: event.id, - event: event.transaction, - timestamp_zero: event.t_zero, - timestamp: event.t, - dependencies: event.dependencies, - execution_hash: event.execution_hash, - transaction_hash: event.transaction_hash, - }) - .await?; + let clone = replica.clone(); + join_set.spawn(async move { + let _response = clone + .apply(ApplyRequest { + id: event.id, + event: event.transaction, + timestamp_zero: event.t_zero, + timestamp: event.t, + dependencies: event.dependencies, + execution_hash: event.execution_hash, + transaction_hash: event.transaction_hash, + }) + .await?; + //println!("{response}"); + Ok::<(), SyneviError>(()) + }); } State::Committed => { - replica - .commit(CommitRequest { - id: event.id, - event: event.transaction, - timestamp_zero: event.t_zero, - timestamp: event.t, - dependencies: event.dependencies, - }) - .await?; + let clone = replica.clone(); + join_set.spawn(async move { + let _response = clone + .commit(CommitRequest { + id: event.id, + event: event.transaction, + timestamp_zero: event.t_zero, + timestamp: event.t, + dependencies: event.dependencies, + }) + .await?; + //println!("{response}"); + Ok::<(), SyneviError>(()) + }); } _ => (), } } + + while let Some(response) = join_set.join_next().await { + response.unwrap().unwrap() + } + Ok(()) } } diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 798bcf2..f42297d 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -17,11 +17,9 @@ use synevi_network::network::Network; use synevi_network::reconfiguration::Reconfiguration; use synevi_network::replica::Replica; use synevi_types::traits::Store; -use synevi_types::types::{ - Hashes, InternalExecution, TransactionPayload, UpsertEvent, -}; -use synevi_types::{Ballot, Executor, State, T, T0}; +use synevi_types::types::{Hashes, InternalExecution, TransactionPayload, UpsertEvent}; use synevi_types::SyneviError; +use synevi_types::{Ballot, Executor, State, T, T0}; use tokio::sync::mpsc::Receiver; use tracing::{instrument, trace}; use ulid::Ulid; @@ -180,19 +178,23 @@ where #[instrument(level = "trace", skip(self, request))] async fn apply(&self, request: ApplyRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; + let deps = from_dependency(request.dependencies.clone())?; println!( - "Received apply request for event: {:?} @ {:?}", + "[{}] APPLY +t0: {:?} +deps: {:?} +", + self.node.get_serial(), t_zero, - self.node.get_serial() + deps, ); let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: Apply"); - let deps = from_dependency(request.dependencies.clone())?; - - let _ = self.node + let _ = self + .node .apply( UpsertEvent { id: request_id, diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index e5170cb..a7cc987 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -64,7 +64,7 @@ pub trait Network: Send + Sync + 'static { &self, last_applied: Vec, ) -> Result, SyneviError>; - async fn ready_electorate(&self) -> Result<(), SyneviError>; + async fn ready_electorate(&self, host: String) -> Result<(), SyneviError>; async fn ready_member(&self, id: Ulid, serial: u16) -> Result<(), SyneviError>; } @@ -123,8 +123,8 @@ where self.as_ref().get_stream_events(last_applied).await } - async fn ready_electorate(&self) -> Result<(), SyneviError> { - self.as_ref().ready_electorate().await + async fn ready_electorate(&self, host: String) -> Result<(), SyneviError> { + self.as_ref().ready_electorate(host).await } async fn ready_member(&self, id: Ulid, serial: u16) -> Result<(), SyneviError> { @@ -413,13 +413,8 @@ impl Network for GrpcNetwork { Ok(rcv) } - async fn ready_electorate(&self) -> Result<(), SyneviError> { - let lock = self.members.read().await; - let mut members = lock.iter(); - let Some((_, member)) = members.next() else { - return Err(SyneviError::NoMembersFound); - }; - let channel = member.member.channel.clone(); + async fn ready_electorate(&self, host: String) -> Result<(), SyneviError> { + let channel = Channel::from_shared(host.clone())?.connect().await?; let request = tonic::Request::new(ReadyElectorateRequest { node_id: self.self_status.info.id.to_bytes().to_vec(), node_serial: self.self_status.info.serial as u32, diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 079e8d5..85a2773 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -218,24 +218,24 @@ impl Store for PersistentStore { .get_event(t_zero) } -// fn get_and_update_hash( -// &self, -// t_zero: T0, -// execution_hash: [u8; 32], -// ) -> Result { -// self.data -// .lock() -// .expect("poisoned lock, aborting") -// .get_and_update_hash(t_zero, execution_hash) -// } -// -// #[instrument(level = "trace", skip(self))] -// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { -// self.data -// .lock() -// .expect("poisoned lock, aborting") -// .last_applied_hash() -// } + // fn get_and_update_hash( + // &self, + // t_zero: T0, + // execution_hash: [u8; 32], + // ) -> Result { + // self.data + // .lock() + // .expect("poisoned lock, aborting") + // .get_and_update_hash(t_zero, execution_hash) + // } + // + // #[instrument(level = "trace", skip(self))] + // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + // self.data + // .lock() + // .expect("poisoned lock, aborting") + // .last_applied_hash() + // } fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.data.lock().expect("poisoned lock, aborting"); @@ -246,12 +246,19 @@ impl Store for PersistentStore { Ok(()) } - fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes { + fn get_or_update_transaction_hash(&self, event: UpsertEvent) -> Result { let lock = self.data.lock().expect("poisoned lock, aborting"); + if let Some(event) = lock.get_event(event.t_zero)? { + if event.state == State::Applied { + if let Some(hashes) = event.hashes { + return Ok(hashes); + } + } + } let mut event = Event::from(event); event.state = State::Applied; println!("{:?}, {:?}", event, lock.latest_hash); - event.hash_event(lock.latest_hash) + Ok(event.hash_event(lock.latest_hash)) } } @@ -392,7 +399,9 @@ impl InternalData { assert!(last_t < event.t); self.last_applied = event.t; - let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes.clone()); @@ -455,7 +464,9 @@ impl InternalData { assert!(last_t < event.t); self.last_applied = event.t; - let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes.clone()); }; diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 1ee4669..4d1335c 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -149,40 +149,40 @@ impl Store for MemStore { .cloned()) } -// fn get_and_update_hash( -// &self, -// t_zero: T0, -// execution_hash: [u8; 32], -// ) -> Result { -// let mut lock = self.store.lock().expect("poisoned lock, aborting"); -// if let Some(event) = lock.events.get_mut(&t_zero) { -// let hashes = event -// .hashes -// .as_mut() -// .ok_or_else(|| SyneviError::MissingTransactionHash)?; -// hashes.execution_hash = execution_hash; -// Ok(hashes.clone()) -// } else { -// Err(SyneviError::EventNotFound(t_zero.get_inner())) -// } -// } -// -// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { -// let lock = self.store.lock().expect("poisoned lock, aborting"); -// let last = lock.last_applied; -// let last_t0 = lock -// .mappings -// .get(&last) -// .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; -// let hash = lock -// .events -// .get(last_t0) -// .cloned() -// .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))? -// .hashes -// .ok_or_else(|| SyneviError::MissingExecutionHash)?; -// Ok((last, hash.execution_hash)) -// } + // fn get_and_update_hash( + // &self, + // t_zero: T0, + // execution_hash: [u8; 32], + // ) -> Result { + // let mut lock = self.store.lock().expect("poisoned lock, aborting"); + // if let Some(event) = lock.events.get_mut(&t_zero) { + // let hashes = event + // .hashes + // .as_mut() + // .ok_or_else(|| SyneviError::MissingTransactionHash)?; + // hashes.execution_hash = execution_hash; + // Ok(hashes.clone()) + // } else { + // Err(SyneviError::EventNotFound(t_zero.get_inner())) + // } + // } + // + // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + // let lock = self.store.lock().expect("poisoned lock, aborting"); + // let last = lock.last_applied; + // let last_t0 = lock + // .mappings + // .get(&last) + // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; + // let hash = lock + // .events + // .get(last_t0) + // .cloned() + // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))? + // .hashes + // .ok_or_else(|| SyneviError::MissingExecutionHash)?; + // Ok((last, hash.execution_hash)) + // } fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.store.lock().expect("poisoned lock, aborting"); @@ -193,11 +193,18 @@ impl Store for MemStore { Ok(()) } - fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes { + fn get_or_update_transaction_hash(&self, event: UpsertEvent) -> Result { let lock = self.store.lock().expect("poisoned lock, aborting"); + if let Some(event) = lock.events.get(&event.t_zero) { + if event.state == State::Applied { + if let Some(hashes) = &event.hashes { + return Ok(hashes.clone()); + } + } + } let mut event = Event::from(event); event.state = State::Applied; - event.hash_event(lock.latest_hash) + Ok(event.hash_event(lock.latest_hash)) } } @@ -304,7 +311,9 @@ impl InternalStore { } assert!(self.last_applied < event.t); self.last_applied = event.t; - let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes); self.events.insert(upsert_event.t_zero, event); @@ -355,7 +364,9 @@ impl InternalStore { } assert!(self.last_applied < event.t); self.last_applied = event.t; - let hashes = upsert_event.hashes.ok_or_else(|| SyneviError::MissingExecutionHash)?; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; self.latest_hash = hashes.transaction_hash; event.hashes = Some(hashes); }; diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index 4d93649..17acc29 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -113,7 +113,7 @@ pub trait Store: Send + Sync + Sized + 'static { // execution_hash: [u8; 32], // ) -> Result; - fn get_and_check_transaction_hash(&self, event: UpsertEvent) -> Hashes; + fn get_or_update_transaction_hash(&self, event: UpsertEvent) -> Result; // Increases the max time to be above the specified guard // Ensures that the guards t0 will not get a fast path afterwards diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index b9a620b..ca1bb1f 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -6,6 +6,7 @@ mod tests { use std::net::SocketAddr; use std::str::FromStr; use std::sync::Arc; + use std::time::Duration; use synevi::{State, Store, T, T0}; use synevi_core::node::Node; use synevi_core::tests::DummyExecutor; @@ -418,7 +419,7 @@ mod tests { let mut joinset = tokio::task::JoinSet::new(); - let num = 10; + let num = 100; let random_number: u128 = rand::thread_rng().gen_range(0..num - 1); for i in 0..num { if i == random_number { @@ -427,7 +428,7 @@ mod tests { SocketAddr::from_str("0.0.0.0:13006").unwrap(), "http://0.0.0.0:13006".to_string(), id, - 6, + 5, ); // Copy & create db @@ -438,7 +439,7 @@ mod tests { //let store = MemStore::new(6).unwrap(); let node = Node::new_with_member( id, - 6, + 5, network, DummyExecutor, store, @@ -461,6 +462,8 @@ mod tests { res.unwrap().unwrap().unwrap(); } + tokio::time::sleep(Duration::from_secs(1)).await; + let (total, accepts, recovers) = coordinator.get_stats(); println!( "Fast: {:?}, Slow: {:?} Paths / {:?} Total / {:?} Recovers", diff --git a/tests/maelstrom/network.rs b/tests/maelstrom/network.rs index 3e62ebb..4d7f245 100644 --- a/tests/maelstrom/network.rs +++ b/tests/maelstrom/network.rs @@ -177,7 +177,7 @@ impl Network for MaelstromNetwork { todo!() } - async fn ready_electorate(&self) -> Result<(), SyneviError> { + async fn ready_electorate(&self, _host: String) -> Result<(), SyneviError> { todo!() } From a43279ecdf4ba7cafe3d21cdd36ed3ad8e6e9aa6 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 16:23:55 +0200 Subject: [PATCH 21/40] fix: Fixed wait handler dependency handling --- synevi_core/src/node.rs | 11 +-- synevi_core/src/replica.rs | 119 ++++++++------------------- synevi_core/src/wait_handler.rs | 33 ++++---- synevi_network/src/network.rs | 6 -- synevi_persistence/src/lmdb_store.rs | 1 - 5 files changed, 51 insertions(+), 119 deletions(-) diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 984e721..f8185e9 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -301,7 +301,6 @@ where && needs_wait { if let Some(waiter) = self.wait_handler.get_waiter(&t0_apply) { - println!("[{}] waiting on {:?}", self.get_serial(), t0_apply); waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); SyneviError::ReceiveError(format!("Error waiting for commit")) @@ -309,14 +308,6 @@ where } } - println!( - "[{:?}]Applied event: t0: {:?}, t: {:?}, deps: {:?}", - self.get_serial(), - event.t_zero, - event.t, - event.dependencies, - ); - // - Check transaction hash -> SyneviError::MismatchingTransactionHash let mut node_hashes = self .event_store @@ -325,7 +316,7 @@ where if let Some(hashes) = &request_hashes { if hashes.transaction_hash != node_hashes.transaction_hash { println!( - "Node: {} + "MismatchedTransactionHashes @ Node: {} request: {:?} got: {:?}", self.get_serial(), diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index f42297d..d4b699d 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -160,6 +160,16 @@ where let deps = from_dependency(request.dependencies)?; + println!( + "[{}] Try commit +t0: {:?} +deps: {:?} +", + self.node.get_serial(), + t_zero, + deps, + ); + self.node .commit(UpsertEvent { id: request_id, @@ -172,24 +182,32 @@ where hashes: None, }) .await?; + + println!( + "[{}] Committed +t0: {:?} +", + self.node.get_serial(), + t_zero, + ); Ok(CommitResponse {}) } #[instrument(level = "trace", skip(self, request))] async fn apply(&self, request: ApplyRequest) -> Result { let t_zero = T0::try_from(request.timestamp_zero.as_slice())?; + let t = T::try_from(request.timestamp.as_slice())?; let deps = from_dependency(request.dependencies.clone())?; println!( - "[{}] APPLY + "[{}] Try apply t0: {:?} -deps: {:?} +t: {:?} ", self.node.get_serial(), t_zero, - deps, + t, ); - let t = T::try_from(request.timestamp.as_slice())?; let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: Apply"); @@ -218,87 +236,20 @@ deps: {:?} ..Default::default() }), ) - .await?; + .await + .unwrap(); + + println!( + "[{}] Applied +t0: {:?} +t: {:?} +", + self.node.get_serial(), + t_zero, + t, + ); Ok(ApplyResponse {}) - // TODO: Refactor in execute function - // - // let transaction: TransactionPayload<::Tx> = - // TransactionPayload::from_bytes(request.event.clone())?; - //let result = match transaction { - // TransactionPayload::None => { - // return Err(SyneviError::TransactionNotFound); - // } - // TransactionPayload::External(tx) => self - // .node - // .executor - // .execute(tx) - // .await - // .map(|e| ExecutorResult::<::Tx>::External(e)), - // TransactionPayload::Internal(request) => { - // // TODO: Build special execution - // let result = match &request { - // InternalExecution::JoinElectorate { - // id, - // serial, - // new_node_host, - // } => { - // if id != &self.node.get_ulid() { - // let res = self - // .node - // .add_member(*id, *serial, new_node_host.clone(), false) - // .await; - // self.node - // .network - // .report_config(new_node_host.clone()) - // .await?; - // res - // } else { - // Ok(()) - // } - // } - // InternalExecution::ReadyElectorate { id, serial } => { - // if id != &self.node.get_ulid() { - // self.node.ready_member(*id, *serial).await - // } else { - // Ok(()) - // } - // } - // }; - // match result { - // Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), - // Err(err) => Ok(ExecutorResult::Internal(Err(err))), - // } - // } - //}; - - //let mut hasher = Sha3_256::new(); - //postcard::to_io(&result, &mut hasher)?; - //let hash = hasher.finalize(); - //let hashes = self - // .node - // .event_store - // .get_and_update_hash(t_zero, hash.into())?; - //if request.transaction_hash != hashes.transaction_hash { - // println!( - // "{} | Mismatched transaction hash in Tx {:?}", - // self.node.get_serial(), - // t_zero - // ); - // Err(SyneviError::MismatchedHashes) - //} else if request.execution_hash != hashes.execution_hash { - // println!( - // "{} | Mismatched execution hash in Tx {:?} | expected: {:?} != {:?}", - // self.node.get_serial(), - // t_zero, - // request.execution_hash, - // hashes.execution_hash - // ); - - // Err(SyneviError::MismatchedHashes) - //} else { - //Ok(ApplyResponse {}) - //} } #[instrument(level = "trace", skip(self))] @@ -415,7 +366,6 @@ where let node = self.node.clone(); let member_count = self.node.network.get_members().await.len() as u32; let self_event = Ulid::new(); - println!("Before joining transaction"); let _res = node .internal_transaction( self_event.0, @@ -426,7 +376,6 @@ where }), ) .await?; - println!("After joining transaction"); Ok(JoinElectorateResponse { member_count }) } diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 3374879..324dbf9 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -1,6 +1,6 @@ use ahash::RandomState; use std::{ - collections::HashMap, + collections::{HashMap, HashSet}, sync::{Arc, Mutex}, }; use synevi_types::{traits::Store, types::RecoverEvent, State, T, T0}; @@ -8,7 +8,7 @@ use tokio::{sync::oneshot, time::Instant}; pub struct Waiter { waited_since: Instant, - dependency_states: u64, + finished_dependencies: HashSet, sender: Vec>, } @@ -84,7 +84,13 @@ where return None; }; - let mut counter = 0; + let waiter = waiter_lock.entry(*t0).or_insert(Waiter { + waited_since: Instant::now(), + finished_dependencies: HashSet::default(), + sender: Vec::new(), + }); + waiter.waited_since = Instant::now(); + for dep_t0 in event.dependencies.iter() { let Some(dep_event) = self.store.get_event(*dep_t0).ok().flatten() else { continue; @@ -92,26 +98,19 @@ where match dep_event.state { State::Committed if dep_event.t > event.t => { - counter += 1; + waiter.finished_dependencies.insert(*dep_t0); } State::Applied => { - counter += 1; + waiter.finished_dependencies.insert(*dep_t0); } _ => {} } } - if counter >= event.dependencies.len() as u64 { + if waiter.finished_dependencies.len() >= event.dependencies.len() { return None; } - let waiter = waiter_lock.entry(*t0).or_insert(Waiter { - waited_since: Instant::now(), - dependency_states: 0, - sender: Vec::new(), - }); - waiter.waited_since = Instant::now(); - waiter.dependency_states = counter; waiter.sender.push(sdx); Some(rcv) } @@ -125,9 +124,9 @@ where }; if event.dependencies.contains(t0_commit) { if t_commit > &event.t { - waiter.dependency_states += 1; + waiter.finished_dependencies.insert(*t0_commit); waiter.waited_since = Instant::now(); - if waiter.dependency_states >= event.dependencies.len() as u64 { + if waiter.finished_dependencies.len() >= event.dependencies.len() { for sdx in waiter.sender.drain(..) { let _ = sdx.send(()); } @@ -147,9 +146,9 @@ where return true; }; if event.dependencies.contains(t0_commit) { - waiter.dependency_states += 1; + waiter.finished_dependencies.insert(*t0_commit); waiter.waited_since = Instant::now(); - if waiter.dependency_states >= event.dependencies.len() as u64 { + if waiter.finished_dependencies.len() >= event.dependencies.len() { for sdx in waiter.sender.drain(..) { let _ = sdx.send(()); } diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index a7cc987..61f2a91 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -503,7 +503,6 @@ impl NetworkInterface for GrpcNetworkSet { BroadcastRequest::Apply(req) => { await_majority = false; for replica in &self.members { - println!("Replica: {:?}", replica); let ready = replica.member.info.ready.load(Ordering::Relaxed); let channel = replica.member.channel.clone(); let request = req.clone(); @@ -545,7 +544,6 @@ impl NetworkInterface for GrpcNetworkSet { // Poll majority // TODO: Electorates for PA ? - println!("Await majority: {}", await_majority); if await_majority { while let Some(response) = responses.join_next().await { // TODO: Resiliency to network errors @@ -590,12 +588,10 @@ impl NetworkInterface for GrpcNetworkSet { } } } else { - println!("Waiting for results"); tokio::spawn( async move { while let Some(r) = &responses.join_next().await { - println!("Got result: {:?}", r); match r { Ok(Err(e)) => { @@ -613,8 +609,6 @@ impl NetworkInterface for GrpcNetworkSet { } }); - println!("Waited for results"); - //}); return Ok(result); // No majority needed -> return early } diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 85a2773..328a334 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -257,7 +257,6 @@ impl Store for PersistentStore { } let mut event = Event::from(event); event.state = State::Applied; - println!("{:?}, {:?}", event, lock.latest_hash); Ok(event.hash_event(lock.latest_hash)) } } From faa17031f0bd63f2bc6fcee8d155f8ff8d8f903d Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 16:38:13 +0200 Subject: [PATCH 22/40] chore: Removed dbg prints --- synevi_core/src/coordinator.rs | 5 ----- synevi_core/src/node.rs | 1 - synevi_core/src/replica.rs | 36 ---------------------------------- tests/consensus_e2e.rs | 4 ++-- 4 files changed, 2 insertions(+), 44 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 3a58269..41e23d3 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -262,12 +262,9 @@ where #[instrument(level = "trace", skip(self))] async fn apply(&mut self) -> InternalSyneviResult { trace!(id = ?self.transaction.id, "Coordinator: Apply"); - println!("Coordinator: Apply"); let (synevi_result, hashes) = self.execute_consensus().await?; - println!("Coordinator: Apply after execute"); - let applied_request = ApplyRequest { id: self.transaction.id.to_be_bytes().into(), event: self.transaction.get_transaction_bytes(), @@ -278,14 +275,12 @@ where transaction_hash: hashes.transaction_hash.to_vec(), }; - println!("Coordinator: Apply before broadcast"); let network_interface = self.node.network.get_interface().await; network_interface .broadcast(BroadcastRequest::Apply(applied_request)) .await?; // TODO: This should not be awaited, but can be used to compare hashes - println!("Coordinator: Apply after broadcast"); synevi_result } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index f8185e9..ef75e94 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -407,7 +407,6 @@ got: {:?}", while !self.is_ready() { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } - println!("{} ready", self.get_serial()); let self_clonable = self .self_clone diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index d4b699d..3fcb3dc 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -160,16 +160,6 @@ where let deps = from_dependency(request.dependencies)?; - println!( - "[{}] Try commit -t0: {:?} -deps: {:?} -", - self.node.get_serial(), - t_zero, - deps, - ); - self.node .commit(UpsertEvent { id: request_id, @@ -183,13 +173,6 @@ deps: {:?} }) .await?; - println!( - "[{}] Committed -t0: {:?} -", - self.node.get_serial(), - t_zero, - ); Ok(CommitResponse {}) } @@ -199,15 +182,6 @@ t0: {:?} let t = T::try_from(request.timestamp.as_slice())?; let deps = from_dependency(request.dependencies.clone())?; - println!( - "[{}] Try apply -t0: {:?} -t: {:?} -", - self.node.get_serial(), - t_zero, - t, - ); let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: Apply"); @@ -239,16 +213,6 @@ t: {:?} .await .unwrap(); - println!( - "[{}] Applied -t0: {:?} -t: {:?} -", - self.node.get_serial(), - t_zero, - t, - ); - Ok(ApplyResponse {}) } diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index ca1bb1f..ea77f46 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -54,7 +54,7 @@ mod tests { let mut joinset = tokio::task::JoinSet::new(); - for i in 0..1000 { + for i in 0..5000 { let coordinator = coordinator.clone(); joinset.spawn(async move { coordinator @@ -173,7 +173,7 @@ mod tests { let start = std::time::Instant::now(); - for _ in 0..10000 { + for _ in 0..100 { let coordinator1 = coordinator1.clone(); let coordinator2 = coordinator2.clone(); let coordinator3 = coordinator3.clone(); From e65f4a15f4d452e311ead07757ceb373818abb45 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 17:06:44 +0200 Subject: [PATCH 23/40] feat: Performance improvements --- synevi_core/src/node.rs | 6 ++-- synevi_core/src/wait_handler.rs | 53 +++++++++++----------------- synevi_persistence/src/lmdb_store.rs | 19 ---------- synevi_persistence/src/mem_store.rs | 35 ------------------ tests/consensus_e2e.rs | 10 ++++-- 5 files changed, 32 insertions(+), 91 deletions(-) diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index ef75e94..342a16f 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -264,10 +264,10 @@ where let prev_event = self.event_store.get_event(t0_commit)?; - self.event_store.upsert_tx(event)?; + self.event_store.upsert_tx(event.clone())?; self.wait_handler.notify_commit(&t0_commit, &t_commit); if !prev_event.is_some_and(|e| e.state > State::Committed || e.dependencies.is_empty()) { - if let Some(waiter) = self.wait_handler.get_waiter(&t0_commit) { + if let Some(waiter) = self.wait_handler.get_waiter(&event) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); SyneviError::ReceiveError(format!("Error waiting for commit")) @@ -300,7 +300,7 @@ where .is_some_and(|deps| !deps.is_empty()) && needs_wait { - if let Some(waiter) = self.wait_handler.get_waiter(&t0_apply) { + if let Some(waiter) = self.wait_handler.get_waiter(&event) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); SyneviError::ReceiveError(format!("Error waiting for commit")) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 324dbf9..71ad20c 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -3,12 +3,13 @@ use std::{ collections::{HashMap, HashSet}, sync::{Arc, Mutex}, }; -use synevi_types::{traits::Store, types::RecoverEvent, State, T, T0}; +use synevi_types::{traits::Store, types::{RecoverEvent, UpsertEvent}, State, T, T0}; use tokio::{sync::oneshot, time::Instant}; pub struct Waiter { + t: T, waited_since: Instant, - finished_dependencies: HashSet, + dependencies: HashSet, sender: Vec>, } @@ -75,39 +76,35 @@ where } } - pub fn get_waiter(&self, t0: &T0) -> Option> { + pub fn get_waiter(&self, upsert_event: &UpsertEvent) -> Option> { let (sdx, rcv) = oneshot::channel(); let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); - let Some(event) = self.store.get_event(*t0).ok().flatten() else { - tracing::error!("Unexpected state in wait_handler: Event not found in store"); - return None; - }; - - let waiter = waiter_lock.entry(*t0).or_insert(Waiter { + let waiter = waiter_lock.entry(upsert_event.t_zero).or_insert(Waiter { + t: upsert_event.t, waited_since: Instant::now(), - finished_dependencies: HashSet::default(), + dependencies: upsert_event.dependencies.clone().unwrap_or_default(), sender: Vec::new(), }); waiter.waited_since = Instant::now(); - for dep_t0 in event.dependencies.iter() { + for dep_t0 in upsert_event.dependencies.clone().unwrap_or_default().iter() { let Some(dep_event) = self.store.get_event(*dep_t0).ok().flatten() else { continue; }; match dep_event.state { - State::Committed if dep_event.t > event.t => { - waiter.finished_dependencies.insert(*dep_t0); + State::Committed if dep_event.t > upsert_event.t => { + waiter.dependencies.remove(dep_t0); } State::Applied => { - waiter.finished_dependencies.insert(*dep_t0); + waiter.dependencies.remove(dep_t0); } _ => {} } } - if waiter.finished_dependencies.len() >= event.dependencies.len() { + if waiter.dependencies.is_empty() { return None; } @@ -117,16 +114,12 @@ where pub fn notify_commit(&self, t0_commit: &T0, t_commit: &T) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); - waiter_lock.retain(|t0_waiting, waiter| { - let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { - tracing::error!("Unexpected state in wait_handler: Event not found in store"); - return true; - }; - if event.dependencies.contains(t0_commit) { - if t_commit > &event.t { - waiter.finished_dependencies.insert(*t0_commit); + waiter_lock.retain(|_, waiter| { + if waiter.dependencies.contains(t0_commit) { + if t_commit > &waiter.t { + waiter.dependencies.remove(t0_commit); waiter.waited_since = Instant::now(); - if waiter.finished_dependencies.len() >= event.dependencies.len() { + if waiter.dependencies.is_empty() { for sdx in waiter.sender.drain(..) { let _ = sdx.send(()); } @@ -140,15 +133,11 @@ where pub fn notify_apply(&self, t0_commit: &T0) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); - waiter_lock.retain(|t0_waiting, waiter| { - let Some(event) = self.store.get_event(*t0_waiting).ok().flatten() else { - tracing::error!("Unexpected state in wait_handler: Event not found in store"); - return true; - }; - if event.dependencies.contains(t0_commit) { - waiter.finished_dependencies.insert(*t0_commit); + waiter_lock.retain(|_, waiter| { + if waiter.dependencies.contains(t0_commit) { + waiter.dependencies.remove(t0_commit); waiter.waited_since = Instant::now(); - if waiter.finished_dependencies.len() >= event.dependencies.len() { + if waiter.dependencies.is_empty() { for sdx in waiter.sender.drain(..) { let _ = sdx.send(()); } diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 328a334..ffdd0eb 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -218,25 +218,6 @@ impl Store for PersistentStore { .get_event(t_zero) } - // fn get_and_update_hash( - // &self, - // t_zero: T0, - // execution_hash: [u8; 32], - // ) -> Result { - // self.data - // .lock() - // .expect("poisoned lock, aborting") - // .get_and_update_hash(t_zero, execution_hash) - // } - // - // #[instrument(level = "trace", skip(self))] - // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - // self.data - // .lock() - // .expect("poisoned lock, aborting") - // .last_applied_hash() - // } - fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.data.lock().expect("poisoned lock, aborting"); lock.latest_time = lock diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 4d1335c..f672596 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -149,41 +149,6 @@ impl Store for MemStore { .cloned()) } - // fn get_and_update_hash( - // &self, - // t_zero: T0, - // execution_hash: [u8; 32], - // ) -> Result { - // let mut lock = self.store.lock().expect("poisoned lock, aborting"); - // if let Some(event) = lock.events.get_mut(&t_zero) { - // let hashes = event - // .hashes - // .as_mut() - // .ok_or_else(|| SyneviError::MissingTransactionHash)?; - // hashes.execution_hash = execution_hash; - // Ok(hashes.clone()) - // } else { - // Err(SyneviError::EventNotFound(t_zero.get_inner())) - // } - // } - // - // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - // let lock = self.store.lock().expect("poisoned lock, aborting"); - // let last = lock.last_applied; - // let last_t0 = lock - // .mappings - // .get(&last) - // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; - // let hash = lock - // .events - // .get(last_t0) - // .cloned() - // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))? - // .hashes - // .ok_or_else(|| SyneviError::MissingExecutionHash)?; - // Ok((last, hash.execution_hash)) - // } - fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.store.lock().expect("poisoned lock, aborting"); lock.latest_time = lock diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index ea77f46..c776be9 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -173,7 +173,7 @@ mod tests { let start = std::time::Instant::now(); - for _ in 0..100 { + for _ in 0..1000 { let coordinator1 = coordinator1.clone(); let coordinator2 = coordinator2.clone(); let coordinator3 = coordinator3.clone(); @@ -205,7 +205,10 @@ mod tests { .await }); } + let mut counter: u16 = 0; while let Some(res) = joinset.join_next().await { + counter += 1; + println!("Got: {counter}"); res.unwrap().unwrap().unwrap(); } @@ -255,7 +258,10 @@ mod tests { recovers ); - assert_eq!(recovers, 0); + tokio::time::sleep(Duration::from_secs(5)).await; + + //assert_eq!(recovers, 0); + let coordinator_store: BTreeMap)> = coordinator1 .event_store From c9a2db90bb1b53fa90c1bef753dd01fe3376436f Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Fri, 18 Oct 2024 17:32:35 +0200 Subject: [PATCH 24/40] feat: Dynamic recovery timeout --- synevi_core/src/wait_handler.rs | 9 +++++++-- tests/consensus_e2e.rs | 10 +++++++--- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 71ad20c..43c31c5 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -3,7 +3,11 @@ use std::{ collections::{HashMap, HashSet}, sync::{Arc, Mutex}, }; -use synevi_types::{traits::Store, types::{RecoverEvent, UpsertEvent}, State, T, T0}; +use synevi_types::{ + traits::Store, + types::{RecoverEvent, UpsertEvent}, + State, T, T0, +}; use tokio::{sync::oneshot, time::Instant}; pub struct Waiter { @@ -150,9 +154,10 @@ where pub fn check_recovery(&self) -> CheckResult { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); + let len = waiter_lock.len() as u128 + 1; let mut smallest_hanging_dep = CheckResult::NoRecovery; for (t0, waiter) in waiter_lock.iter_mut() { - if waiter.waited_since.elapsed().as_millis() > 100 { + if waiter.waited_since.elapsed().as_millis() > len * 2 { // Get deps and find smallest dep that is not committed / applied let Some(event) = self.store.get_event(*t0).ok().flatten() else { tracing::error!( diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index c776be9..c703d6c 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -20,9 +20,13 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn parallel_execution() { let node_names: Vec<_> = (0..5).map(|_| Ulid::new()).collect(); - let mut nodes: Vec>> = vec![]; + let mut nodes: Vec>> = vec![]; for (i, m) in node_names.iter().enumerate() { + let test_path = format!("/dev/shm/{m}/"); + fs::create_dir(&test_path).await.unwrap(); + dbg!(&test_path); + let store = PersistentStore::new(test_path, i as u16).unwrap(); let socket_addr = SocketAddr::from_str(&format!("0.0.0.0:{}", 10000 + i)).unwrap(); let network = synevi_network::network::GrpcNetwork::new( socket_addr, @@ -30,7 +34,7 @@ mod tests { *m, i as u16, ); - let node = Node::new_with_network_and_executor(*m, i as u16, network, DummyExecutor) + let node = Node::new(*m, i as u16, network, DummyExecutor, store) .await .unwrap(); nodes.push(node); @@ -76,6 +80,7 @@ mod tests { ); //assert_eq!(recovers, 0); + tokio::time::sleep(Duration::from_secs(1)).await; let coordinator_store: BTreeMap)> = coordinator .event_store @@ -262,7 +267,6 @@ mod tests { //assert_eq!(recovers, 0); - let coordinator_store: BTreeMap)> = coordinator1 .event_store .get_event_store() From 07036ebc9fc449e6bea8489dfb5ba6d67b5a534d Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 19 Oct 2024 16:02:45 +0200 Subject: [PATCH 25/40] test: Added redb_store for testing --- synevi_core/src/coordinator.rs | 5 +- synevi_core/src/wait_handler.rs | 2 +- synevi_network/src/network.rs | 5 +- synevi_persistence/Cargo.toml | 3 +- synevi_persistence/src/lib.rs | 1 + synevi_persistence/src/lmdb_store.rs | 3 +- synevi_persistence/src/redb_store.rs | 703 +++++++++++++++++++++++++++ synevi_types/Cargo.toml | 7 +- synevi_types/src/traits.rs | 12 +- tests/consensus_e2e.rs | 2 +- 10 files changed, 722 insertions(+), 21 deletions(-) create mode 100644 synevi_persistence/src/redb_store.rs diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 41e23d3..59932c9 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -13,8 +13,7 @@ use synevi_network::network::{BroadcastRequest, Network, NetworkInterface}; use synevi_network::utils::IntoInner; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalSyneviResult, RecoverEvent, RecoveryState, - TransactionPayload, + ExecutorResult, Hashes, InternalSyneviResult, RecoverEvent, RecoveryState, TransactionPayload, }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; use tracing::{instrument, trace}; @@ -275,13 +274,11 @@ where transaction_hash: hashes.transaction_hash.to_vec(), }; - let network_interface = self.node.network.get_interface().await; network_interface .broadcast(BroadcastRequest::Apply(applied_request)) .await?; // TODO: This should not be awaited, but can be used to compare hashes - synevi_result } diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 43c31c5..31a30ad 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -154,7 +154,7 @@ where pub fn check_recovery(&self) -> CheckResult { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); - let len = waiter_lock.len() as u128 + 1; + let len = waiter_lock.len() as u128 + 10; let mut smallest_hanging_dep = CheckResult::NoRecovery; for (t0, waiter) in waiter_lock.iter_mut() { if waiter.waited_since.elapsed().as_millis() > len * 2 { diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index 61f2a91..c0f0c4d 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -588,11 +588,8 @@ impl NetworkInterface for GrpcNetworkSet { } } } else { - - tokio::spawn( - async move { + tokio::spawn(async move { while let Some(r) = &responses.join_next().await { - match r { Ok(Err(e)) => { println!("Apply: Error in response: {:?}", e); diff --git a/synevi_persistence/Cargo.toml b/synevi_persistence/Cargo.toml index 27aba4e..25cc2af 100644 --- a/synevi_persistence/Cargo.toml +++ b/synevi_persistence/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true description.workspace = true [dependencies] -monotime = { path = "../monotime", version = "0.1.0"} +monotime = { path = "../monotime", version = "0.1.0" } synevi_types = { path = "../synevi_types", version = "0.1.0" } bytes = { workspace = true } tokio = { workspace = true } @@ -26,3 +26,4 @@ thiserror = { workspace = true } async-channel = "2.3.1" ahash = { workspace = true } bincode = "1.3.3" +redb = { version = "2.1.4" } diff --git a/synevi_persistence/src/lib.rs b/synevi_persistence/src/lib.rs index 896d3a7..e287d2e 100644 --- a/synevi_persistence/src/lib.rs +++ b/synevi_persistence/src/lib.rs @@ -1,2 +1,3 @@ pub mod lmdb_store; pub mod mem_store; +pub mod redb_store; diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index ffdd0eb..4102857 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -2,7 +2,7 @@ use ahash::RandomState; use heed::{ byteorder::BigEndian, types::{SerdeBincode, U128}, - Database, Env, EnvOpenOptions, + Database, Env, EnvFlags, EnvOpenOptions, }; use monotime::MonoTime; use std::{ @@ -42,6 +42,7 @@ impl PersistentStore { EnvOpenOptions::new() .map_size(1024 * 1024 * 1024) .max_dbs(16) + .flags(EnvFlags::MAP_ASYNC | EnvFlags::WRITE_MAP) .open(path)? }; let env_clone = env.clone(); diff --git a/synevi_persistence/src/redb_store.rs b/synevi_persistence/src/redb_store.rs new file mode 100644 index 0000000..f83aa7c --- /dev/null +++ b/synevi_persistence/src/redb_store.rs @@ -0,0 +1,703 @@ +use ahash::RandomState; +use monotime::MonoTime; +use redb::{Database, ReadableTable, ReadableTableMetadata, TableDefinition}; +use std::{ + collections::{BTreeMap, HashSet}, + sync::{Arc, Mutex}, +}; +use synevi_types::{ + error::SyneviError, + traits::Store, + types::{Event, Hashes, RecoverDependencies, RecoverEvent, UpsertEvent}, + Ballot, State, T, T0, +}; +use tokio::sync::mpsc::Receiver; +use tracing::instrument; + +const TABLE: TableDefinition = TableDefinition::new("events"); + +#[derive(Clone, Debug)] +pub struct PersistentStore { + data: Arc>, +} + +#[derive(Clone, Debug)] +struct InternalData { + db: Arc, + pub(crate) mappings: BTreeMap, // Key: t, value t0 + pub last_applied: T, // t of last applied entry + pub(crate) latest_time: MonoTime, // last created or recognized t0 + pub node_serial: u16, + latest_hash: [u8; 32], +} + +impl PersistentStore { + pub fn new(path: String, node_serial: u16) -> Result { + let db = Database::create(path).unwrap(); + let read_txn = db.begin_read().unwrap(); + let events_db = read_txn.open_table(TABLE).unwrap(); + + if !events_db.is_empty().unwrap() { + let result = events_db + .range(0..) + .unwrap() + .filter_map(|e| { + if let Ok((_, event)) = e { + Some(bincode::deserialize(event.value()).unwrap()) + } else { + None + } + }) + .collect::>(); + + let mut mappings = BTreeMap::default(); + let mut last_applied = T::default(); + let mut latest_time = MonoTime::default(); + let mut latest_hash: [u8; 32] = [0; 32]; + for event in result { + mappings.insert(event.t, event.t_zero); + if event.state == State::Applied && event.t > last_applied { + last_applied = event.t; + latest_hash = if let Some(hashes) = event.hashes { + hashes.transaction_hash + } else { + return Err(SyneviError::MissingTransactionHash); + }; + } + if *event.t > latest_time { + latest_time = *event.t; + } + } + Ok(PersistentStore { + //db: env_clone, + data: Arc::new(Mutex::new(InternalData { + db: Arc::new(db), + mappings, + last_applied, + latest_time, + node_serial, + latest_hash, + })), + }) + } else { + Ok(PersistentStore { + data: Arc::new(Mutex::new(InternalData { + db: Arc::new(db), + mappings: BTreeMap::default(), + last_applied: T::default(), + latest_time: MonoTime::default(), + node_serial, + latest_hash: [0; 32], + })), + }) + } + } +} + +impl Store for PersistentStore { + #[instrument(level = "trace")] + fn init_t_zero(&self, node_serial: u16) -> T0 { + self.data + .lock() + .expect("poisoned lock, aborting") + .init_t_zero(node_serial) + } + + #[instrument(level = "trace")] + fn pre_accept_tx( + &self, + id: u128, + t_zero: T0, + transaction: Vec, + ) -> Result<(T, HashSet), SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .pre_accept_tx(id, t_zero, transaction) + } + + #[instrument(level = "trace")] + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_tx_dependencies(t, t_zero) + } + + #[instrument(level = "trace")] + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { + self.data + .lock() + .expect("poisoned lock, aborting") + .accept_tx_ballot(t_zero, ballot) + } + + #[instrument(level = "trace", skip(self))] + fn upsert_tx(&self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .upsert_tx(upsert_event) + } + + #[instrument(level = "trace")] + fn get_recover_deps(&self, t_zero: &T0) -> Result { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_recover_deps(t_zero) + } + + #[instrument(level = "trace")] + fn get_event_state(&self, t_zero: &T0) -> Option { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event_state(t_zero) + } + + #[instrument(level = "trace")] + fn recover_event( + &self, + t_zero_recover: &T0, + node_serial: u16, + ) -> Result, SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .recover_event(t_zero_recover, node_serial) + } + + #[instrument(level = "trace")] + fn get_event_store(&self) -> BTreeMap { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event_store() + } + + #[instrument(level = "trace")] + fn last_applied(&self) -> (T, T0) { + self.data + .lock() + .expect("poisoned lock, aborting") + .last_applied() + } + + #[instrument(level = "trace")] + fn get_events_after( + &self, + last_applied: T, + ) -> Result>, SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_events_after(last_applied) + } + + #[instrument(level = "trace", skip(self))] + fn get_event(&self, t_zero: T0) -> Result, SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event(t_zero) + } + + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { + let mut lock = self.data.lock().expect("poisoned lock, aborting"); + lock.latest_time = lock + .latest_time + .next_with_guard_and_node(&guard, lock.node_serial) + .into_time(); + Ok(()) + } + + fn get_or_update_transaction_hash(&self, event: UpsertEvent) -> Result { + let lock = self.data.lock().expect("poisoned lock, aborting"); + if let Some(event) = lock.get_event(event.t_zero)? { + if event.state == State::Applied { + if let Some(hashes) = event.hashes { + return Ok(hashes); + } + } + } + let mut event = Event::from(event); + event.state = State::Applied; + Ok(event.hash_event(lock.latest_hash)) + } +} + +impl InternalData { + #[instrument(level = "trace")] + fn init_t_zero(&mut self, node_serial: u16) -> T0 { + let next_time = self.latest_time.next_with_node(node_serial).into_time(); + self.latest_time = next_time; + T0(next_time) + } + + #[instrument(level = "trace")] + fn pre_accept_tx( + &mut self, + id: u128, + t_zero: T0, + transaction: Vec, + ) -> Result<(T, HashSet), SyneviError> { + let (t, deps) = { + let t = if self.latest_time > *t_zero { + let new_time_t = t_zero + .next_with_guard_and_node(&self.latest_time, self.node_serial) + .into_time(); + + self.latest_time = new_time_t; + T(new_time_t) + } else { + T(*t_zero) + }; + // This might not be necessary to re-use the write lock here + let deps = self.get_tx_dependencies(&t, &t_zero); + (t, deps) + }; + + let event = UpsertEvent { + id, + t_zero, + t, + state: State::PreAccepted, + transaction: Some(transaction), + dependencies: Some(deps.clone()), + ..Default::default() + }; + self.upsert_tx(event)?; + Ok((t, deps)) + } + + #[instrument(level = "trace")] + fn get_tx_dependencies(&self, t: &T, t_zero: &T0) -> HashSet { + if self.last_applied == *t { + return HashSet::default(); + } + assert!(self.last_applied < *t); + let mut deps = HashSet::default(); + if let Some(last_applied_t0) = self.mappings.get(&self.last_applied) { + if last_applied_t0 != &T0::default() { + deps.insert(*last_applied_t0); + } + } + // What about deps with dep_t0 < last_applied_t0 && dep_t > t? + + // Dependencies are where any of these cases match: + // - t_dep < t if not applied + // - t0_dep < t0_last_applied, if t_dep > t0 + // - t_dep > t if t0_dep < t + for (_, t0_dep) in self.mappings.range(self.last_applied..) { + if t0_dep != t_zero && (t0_dep < &T0(**t)) { + deps.insert(*t0_dep); + } + } + deps + } + + #[instrument(level = "trace")] + fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { + let write_txn = self.db.begin_write().ok()?; + let mut event: Event = { + let table = write_txn.open_table(TABLE).ok()?; + let event = table.get(&t_zero.get_inner()).ok()??; + bincode::deserialize(event.value()).ok()? + }; + + if event.ballot < ballot { + event.ballot = ballot; + let mut table = write_txn.open_table(TABLE).ok()?; + + let bytes = bincode::serialize(&event).ok()?; + let _ = table.insert(&t_zero.get_inner(), bytes.as_slice()); + } + write_txn.commit().ok()?; + + Some(event.ballot) + } + + #[instrument(level = "trace", skip(self))] + fn upsert_tx(&mut self, upsert_event: UpsertEvent) -> Result<(), SyneviError> { + //let db = self.db.clone(); + + // Update the latest time + if self.latest_time < *upsert_event.t { + self.latest_time = *upsert_event.t; + } + + let write_txn = self.db.begin_write().ok().unwrap(); + let event: Option = { + let table = write_txn.open_table(TABLE).ok().unwrap(); + table + .get(&upsert_event.t_zero.get_inner()) + .ok() + .unwrap() + .map(|e| bincode::deserialize(e.value()).ok().unwrap()) + }; + + let Some(mut event) = event else { + let mut event = Event::from(upsert_event.clone()); + + if matches!(event.state, State::Applied) { + self.mappings.insert(event.t, event.t_zero); + if let Some(deps) = upsert_event.dependencies { + event.dependencies = deps; + } + if let Some(transaction) = upsert_event.transaction { + if event.transaction.is_empty() && !transaction.is_empty() { + event.transaction = transaction; + } + } + event.state = upsert_event.state; + if let Some(ballot) = upsert_event.ballot { + if event.ballot < ballot { + event.ballot = ballot; + } + } + + let last_t = self.last_applied; + + // if last_t > event.t { + // println!( + // "Node: {:?}, store: {:?}", + // self.node_serial, + // self.get_event_store() + // ); + // } + // Safeguard + assert!(last_t < event.t); + + self.last_applied = event.t; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; + self.latest_hash = hashes.transaction_hash; + event.hashes = Some(hashes.clone()); + { + let mut table = write_txn.open_table(TABLE).ok().unwrap(); + let bytes = bincode::serialize(&event).ok().unwrap(); + let _ = table.insert(&upsert_event.t_zero.get_inner(), bytes.as_slice()); + } + } else { + { + let mut table = write_txn.open_table(TABLE).ok().unwrap(); + let bytes = bincode::serialize(&event).ok().unwrap(); + let _ = table.insert(&upsert_event.t_zero.get_inner(), bytes.as_slice()); + } + self.mappings.insert(upsert_event.t, upsert_event.t_zero); + } + write_txn.commit().unwrap(); + return Ok(()); + }; + + // Do not update to a "lower" state + if upsert_event.state < event.state { + write_txn.commit().unwrap(); + return Ok(()); + } + + // Event is already applied + if event.state == State::Applied { + write_txn.commit().unwrap(); + return Ok(()); + } + + if event.is_update(&upsert_event) { + if let Some(old_t) = event.update_t(upsert_event.t) { + self.mappings.remove(&old_t); + self.mappings.insert(event.t, event.t_zero); + } + if let Some(deps) = upsert_event.dependencies { + event.dependencies = deps; + } + if let Some(transaction) = upsert_event.transaction { + if event.transaction.is_empty() && !transaction.is_empty() { + event.transaction = transaction; + } + } + event.state = upsert_event.state; + if let Some(ballot) = upsert_event.ballot { + if event.ballot < ballot { + event.ballot = ballot; + } + } + + if event.state == State::Applied { + let last_t = self.last_applied; + + if last_t > event.t { + println!("last_t: {:?}, event.t: {:?}", last_t, event.t); + } + // println!( + // "Node: {:?}, last_t: {:?}, event_t: {:?}, store: {:?}", + // self.node_serial, + // last_t, + // event.t, + // self.get_event_store() + // ); + // } + // Safeguard + assert!(last_t < event.t); + + self.last_applied = event.t; + let hashes = upsert_event + .hashes + .ok_or_else(|| SyneviError::MissingExecutionHash)?; + self.latest_hash = hashes.transaction_hash; + event.hashes = Some(hashes.clone()); + }; + { + let mut table = write_txn.open_table(TABLE).ok().unwrap(); + let bytes = bincode::serialize(&event).ok().unwrap(); + let _ = table.insert(&upsert_event.t_zero.get_inner(), bytes.as_slice()); + } + write_txn.commit().unwrap(); + Ok(()) + } else { + write_txn.commit().unwrap(); + Ok(()) + } + } + + #[instrument(level = "trace")] + fn get_recover_deps(&self, t_zero: &T0) -> Result { + let read_txn = self.db.begin_read().unwrap(); + let timestamp = { + let table = read_txn.open_table(TABLE).ok().unwrap(); + table + .get(&t_zero.get_inner()) + .ok() + .unwrap() + .map(|e| bincode::deserialize::(e.value()).ok().unwrap()) + .unwrap() + .t + }; + let mut recover_deps = RecoverDependencies { + timestamp, + ..Default::default() + }; + + for (t_dep, t_zero_dep) in self.mappings.range(self.last_applied..) { + let dep_event = { + let table = read_txn.open_table(TABLE).ok().unwrap(); + table + .get(&t_zero.get_inner()) + .ok() + .unwrap() + .map(|e| bincode::deserialize::(e.value()).ok().unwrap()) + .unwrap() + }; + match dep_event.state { + State::Accepted => { + if dep_event + .dependencies + .iter() + .any(|t_zero_dep_dep| t_zero == t_zero_dep_dep) + { + // Wait -> Accord p19 l7 + l9 + if t_zero_dep < t_zero && **t_dep > **t_zero { + recover_deps.wait.insert(*t_zero_dep); + } + // Superseding -> Accord: p19 l10 + if t_zero_dep > t_zero { + recover_deps.superseding = true; + } + } + } + State::Committed => { + if dep_event + .dependencies + .iter() + .any(|t_zero_dep_dep| t_zero == t_zero_dep_dep) + { + // Superseding -> Accord: p19 l11 + if **t_dep > **t_zero { + recover_deps.superseding = true; + } + } + } + _ => {} + } + // Collect "normal" deps -> Accord: p19 l16 + if t_zero_dep < t_zero { + recover_deps.dependencies.insert(*t_zero_dep); + } + } + Ok(recover_deps) + } + + fn get_event_state(&self, t_zero: &T0) -> Option { + self.get_event(*t_zero) + .map(|event| event.map(|e| e.state)) + .ok() + .flatten() + } + + fn recover_event( + &self, + t_zero_recover: &T0, + node_serial: u16, + ) -> Result, SyneviError> { + let Some(state) = self.get_event_state(t_zero_recover) else { + return Ok(None); + }; + if matches!(state, synevi_types::State::Undefined) { + return Err(SyneviError::UndefinedRecovery); + } + + let write_txn = self.db.begin_write().ok().unwrap(); + let event = { + let table = write_txn.open_table(TABLE).ok().unwrap(); + table + .get(&t_zero_recover.get_inner()) + .ok() + .unwrap() + .map(|e| bincode::deserialize::(e.value()).ok().unwrap()) + }; + + if let Some(mut event) = event { + event.ballot = Ballot(event.ballot.next_with_node(node_serial).into_time()); + { + let mut table = write_txn.open_table(TABLE).ok().unwrap(); + let bytes = bincode::serialize(&event).ok().unwrap(); + let _ = table.insert(&t_zero_recover.get_inner(), bytes.as_slice()); + } + write_txn.commit().unwrap(); + + Ok(Some(RecoverEvent { + id: event.id, + t_zero: event.t_zero, + t: event.t, + state, + transaction: event.transaction.clone(), + dependencies: event.dependencies.clone(), + ballot: event.ballot, + })) + } else { + Ok(None) + } + } + + fn get_event_store(&self) -> BTreeMap { + // TODO: Remove unwrap and change trait result + let read_txn = self.db.begin_read().ok().unwrap(); + let table = read_txn.open_table(TABLE).ok().unwrap(); + let range = table.range(0..).unwrap(); + let result = range + .filter_map(|e| { + if let Ok((t0, event)) = e { + Some(( + T0(MonoTime::from(t0.value())), + bincode::deserialize(event.value()).ok().unwrap(), + )) + } else { + None + } + }) + .collect::>(); + result + } + + fn last_applied(&self) -> (T, T0) { + let t = self.last_applied.clone(); + let t0 = self.mappings.get(&t).cloned().unwrap_or(T0::default()); + (t, t0) + } + + fn get_events_after( + &self, + last_applied: T, + ) -> Result>, SyneviError> { + let (sdx, rcv) = tokio::sync::mpsc::channel(200); + let db = self.db.clone(); + let last_applied_t0 = match self.mappings.get(&last_applied) { + Some(t0) => *t0, + None if last_applied == T::default() => T0::default(), + _ => return Err(SyneviError::EventNotFound(last_applied.get_inner())), + }; + tokio::task::spawn_blocking(move || { + let write_txn = db.begin_read().ok().unwrap(); + let table = write_txn.open_table(TABLE).ok().unwrap(); + for result in table.range(last_applied_t0.get_inner()..).unwrap() { + let event = bincode::deserialize(result.unwrap().1.value()) + .ok() + .unwrap(); + sdx.blocking_send(Ok(event)) + .map_err(|e| SyneviError::SendError(e.to_string()))?; + } + Ok::<(), SyneviError>(()) + }); + Ok(rcv) + } + + fn get_event(&self, t_zero: T0) -> Result, SyneviError> { + let write_txn = self.db.begin_read().ok().unwrap(); + let event = { + let table = write_txn.open_table(TABLE).ok().unwrap(); + table + .get(&t_zero.get_inner()) + .ok() + .unwrap() + .map(|e| bincode::deserialize::(e.value()).ok().unwrap()) + }; + Ok(event) + } + + // fn get_and_update_hash( + // &self, + // t_zero: T0, + // execution_hash: [u8; 32], + // ) -> Result { + // let t_zero = t_zero.get_inner(); + // let mut write_txn = self.db.write_txn()?; + // let db: EventDb = self + // .db + // .open_database(&write_txn, Some(EVENT_DB_NAME))? + // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; + // let Some(mut event) = db.get(&write_txn, &t_zero)? else { + // return Err(SyneviError::EventNotFound(t_zero)); + // }; + // let Some(mut hashes) = event.hashes else { + // return Err(SyneviError::MissingTransactionHash); + // }; + // hashes.execution_hash = execution_hash; + // event.hashes = Some(hashes.clone()); + + // db.put(&mut write_txn, &t_zero, &event)?; + // write_txn.commit()?; + // Ok(hashes) + // } + + // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { + // let last = self.last_applied; + // let last_t0 = self + // .mappings + // .get(&last) + // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; + // let read_txn = self.db.read_txn()?; + // let db: EventDb = self + // .db + // .open_database(&read_txn, Some(EVENT_DB_NAME))? + // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; + // let event = db + // .get(&read_txn, &last_t0.get_inner())? + // .ok_or_else(|| SyneviError::EventNotFound(last_t0.get_inner()))? + // .hashes + // .ok_or_else(|| SyneviError::MissingExecutionHash)?; + // Ok((last, event.execution_hash)) + // } +} + +#[cfg(test)] +mod tests { + + #[test] + fn test_db() { + // TODO + //let db = Database::new("../../tests/database".to_string()).unwrap(); + //db.init(Bytes::from("key"), Bytes::from("value")) + // .unwrap() + } +} diff --git a/synevi_types/Cargo.toml b/synevi_types/Cargo.toml index 38c643e..fe91f31 100644 --- a/synevi_types/Cargo.toml +++ b/synevi_types/Cargo.toml @@ -10,7 +10,7 @@ description.workspace = true [dependencies] bytes = { workspace = true } tokio = { workspace = true } -monotime = { path = "../monotime", version = "0.1.0"} +monotime = { path = "../monotime", version = "0.1.0" } serde = { workspace = true } serde_json = { workspace = true } tonic = { workspace = true } @@ -21,6 +21,7 @@ async-channel = "2.3.1" ahash = { workspace = true } heed = { workspace = true } http = "1.1.0" -postcard = {version = "1.0.10", features = ["use-std"]} +postcard = { version = "1.0.10", features = ["use-std"] } sha3 = "0.10.8" -ulid = {workspace = true} +ulid = { workspace = true } +redb = { version = "2.1.4" } \ No newline at end of file diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index 17acc29..a9bf2c7 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -99,7 +99,7 @@ pub trait Store: Send + Sync + Sized + 'static { fn get_event_store(&self) -> BTreeMap; fn last_applied(&self) -> (T, T0); -// fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; + // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; fn get_event(&self, t_zero: T0) -> Result, SyneviError>; fn get_events_after( @@ -107,11 +107,11 @@ pub trait Store: Send + Sync + Sized + 'static { last_applied: T, ) -> Result>, SyneviError>; -// fn get_and_update_hash( -// &self, -// t_zero: T0, -// execution_hash: [u8; 32], -// ) -> Result; + // fn get_and_update_hash( + // &self, + // t_zero: T0, + // execution_hash: [u8; 32], + // ) -> Result; fn get_or_update_transaction_hash(&self, event: UpsertEvent) -> Result; diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index c703d6c..a0622dd 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -23,7 +23,7 @@ mod tests { let mut nodes: Vec>> = vec![]; for (i, m) in node_names.iter().enumerate() { - let test_path = format!("/dev/shm/{m}/"); + let test_path = format!("/home/beavis/tmp/{m}/"); fs::create_dir(&test_path).await.unwrap(); dbg!(&test_path); let store = PersistentStore::new(test_path, i as u16).unwrap(); From 22a158db4d94c734c5f37e509b489207af7611c3 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 21 Oct 2024 15:51:54 +0200 Subject: [PATCH 26/40] chore: Rename stores --- synevi_persistence/src/lmdb_store.rs | 14 ++++++++------ synevi_persistence/src/redb_store.rs | 18 ++++++++++++------ tests/consensus_e2e.rs | 16 ++++++++-------- 3 files changed, 28 insertions(+), 20 deletions(-) diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 4102857..0ab4d73 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -22,7 +22,7 @@ const EVENT_DB_NAME: &str = "events"; type EventDb = Database, SerdeBincode>; #[derive(Clone, Debug)] -pub struct PersistentStore { +pub struct LmdbStore { data: Arc>, } @@ -36,8 +36,8 @@ struct InternalData { latest_hash: [u8; 32], } -impl PersistentStore { - pub fn new(path: String, node_serial: u16) -> Result { +impl LmdbStore { + pub fn new(path: String, node_serial: u16) -> Result { let env = unsafe { EnvOpenOptions::new() .map_size(1024 * 1024 * 1024) @@ -80,7 +80,7 @@ impl PersistentStore { } } write_txn.commit()?; - Ok(PersistentStore { + Ok(LmdbStore { //db: env_clone, data: Arc::new(Mutex::new(InternalData { db: env_clone, @@ -95,7 +95,7 @@ impl PersistentStore { None => { let _: EventDb = env.create_database(&mut write_txn, Some(EVENT_DB_NAME))?; write_txn.commit()?; - Ok(PersistentStore { + Ok(LmdbStore { data: Arc::new(Mutex::new(InternalData { db: env_clone, mappings: BTreeMap::default(), @@ -110,7 +110,7 @@ impl PersistentStore { } } -impl Store for PersistentStore { +impl Store for LmdbStore { #[instrument(level = "trace")] fn init_t_zero(&self, node_serial: u16) -> T0 { self.data @@ -284,6 +284,7 @@ impl InternalData { ..Default::default() }; self.upsert_tx(event)?; + //self.db.force_sync()?; Ok((t, deps)) } @@ -327,6 +328,7 @@ impl InternalData { let _ = events_db.put(&mut write_txn, &t_zero.get_inner(), &event); } write_txn.commit().ok()?; + //self.db.force_sync().ok()?; Some(event.ballot) } diff --git a/synevi_persistence/src/redb_store.rs b/synevi_persistence/src/redb_store.rs index f83aa7c..b65452b 100644 --- a/synevi_persistence/src/redb_store.rs +++ b/synevi_persistence/src/redb_store.rs @@ -17,7 +17,7 @@ use tracing::instrument; const TABLE: TableDefinition = TableDefinition::new("events"); #[derive(Clone, Debug)] -pub struct PersistentStore { +pub struct RedbStore { data: Arc>, } @@ -31,10 +31,16 @@ struct InternalData { latest_hash: [u8; 32], } -impl PersistentStore { - pub fn new(path: String, node_serial: u16) -> Result { +impl RedbStore { + pub fn new(path: String, node_serial: u16) -> Result { let db = Database::create(path).unwrap(); + { + let write_txn = db.begin_write().unwrap(); + let _ = write_txn.open_table(TABLE).unwrap(); + write_txn.commit().unwrap(); + } let read_txn = db.begin_read().unwrap(); + let events_db = read_txn.open_table(TABLE).unwrap(); if !events_db.is_empty().unwrap() { @@ -68,7 +74,7 @@ impl PersistentStore { latest_time = *event.t; } } - Ok(PersistentStore { + Ok(RedbStore { //db: env_clone, data: Arc::new(Mutex::new(InternalData { db: Arc::new(db), @@ -80,7 +86,7 @@ impl PersistentStore { })), }) } else { - Ok(PersistentStore { + Ok(RedbStore { data: Arc::new(Mutex::new(InternalData { db: Arc::new(db), mappings: BTreeMap::default(), @@ -94,7 +100,7 @@ impl PersistentStore { } } -impl Store for PersistentStore { +impl Store for RedbStore { #[instrument(level = "trace")] fn init_t_zero(&self, node_serial: u16) -> T0 { self.data diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index a0622dd..752c816 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -11,7 +11,7 @@ mod tests { use synevi_core::node::Node; use synevi_core::tests::DummyExecutor; use synevi_network::network::GrpcNetwork; - use synevi_persistence::lmdb_store::PersistentStore; + use synevi_persistence::lmdb_store::LmdbStore; use synevi_persistence::mem_store::MemStore; use tokio::fs; use tokio::runtime::Builder; @@ -20,13 +20,13 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn parallel_execution() { let node_names: Vec<_> = (0..5).map(|_| Ulid::new()).collect(); - let mut nodes: Vec>> = vec![]; + let mut nodes: Vec>> = vec![]; for (i, m) in node_names.iter().enumerate() { - let test_path = format!("/home/beavis/tmp/{m}/"); + let test_path = format!("dev/shm/{m}"); fs::create_dir(&test_path).await.unwrap(); dbg!(&test_path); - let store = PersistentStore::new(test_path, i as u16).unwrap(); + let store = LmdbStore::new(test_path, i as u16).unwrap(); let socket_addr = SocketAddr::from_str(&format!("0.0.0.0:{}", 10000 + i)).unwrap(); let network = synevi_network::network::GrpcNetwork::new( socket_addr, @@ -373,7 +373,7 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn reconfiguration() { let node_names: Vec<_> = (0..5).map(|_| Ulid::new()).collect(); - let mut nodes: Vec>> = vec![]; + let mut nodes: Vec>> = vec![]; //let mut nodes: Vec>> = vec![]; for (i, m) in node_names.iter().enumerate() { @@ -390,7 +390,7 @@ mod tests { let test_path = format!("/dev/shm/{m}/"); fs::create_dir(&test_path).await.unwrap(); dbg!(&test_path); - let store = PersistentStore::new(test_path, i as u16).unwrap(); + let store = LmdbStore::new(test_path, i as u16).unwrap(); //let store = MemStore::new(i as u16).unwrap(); let node = Node::new(*m, i as u16, network, DummyExecutor, store) .await @@ -408,7 +408,7 @@ mod tests { // Copy & create db let test_path = format!("/dev/shm/{m}/"); fs::create_dir(&test_path).await.unwrap(); - let store = PersistentStore::new(test_path, i as u16).unwrap(); + let store = LmdbStore::new(test_path, i as u16).unwrap(); //let store = MemStore::new(i as u16).unwrap(); let node = Node::new_with_member( *m, @@ -445,7 +445,7 @@ mod tests { let test_path = format!("/dev/shm/{id}/"); fs::create_dir(&test_path).await.unwrap(); dbg!(&test_path); - let store = PersistentStore::new(test_path, 6).unwrap(); + let store = LmdbStore::new(test_path, 6).unwrap(); //let store = MemStore::new(6).unwrap(); let node = Node::new_with_member( id, From d57bfb9fc56ccc8619db0d42d30a8285cafd6d09 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 26 Oct 2024 14:50:42 +0200 Subject: [PATCH 27/40] chore: coordinator put blocking IO ops into spawn blocking --- rust-toolchain.toml | 3 + synevi_core/src/coordinator.rs | 471 ++++++++++++++++----------------- synevi_core/src/node.rs | 101 +++---- 3 files changed, 284 insertions(+), 291 deletions(-) create mode 100644 rust-toolchain.toml diff --git a/rust-toolchain.toml b/rust-toolchain.toml new file mode 100644 index 0000000..7d27fca --- /dev/null +++ b/rust-toolchain.toml @@ -0,0 +1,3 @@ +[toolchain] +channel = "1.82.0" +components = ["clippy"] diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 59932c9..76e7278 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -16,7 +16,7 @@ use synevi_types::types::{ ExecutorResult, Hashes, InternalSyneviResult, RecoverEvent, RecoveryState, TransactionPayload, }; use synevi_types::{Ballot, Executor, State, SyneviError, Transaction, T, T0}; -use tracing::{instrument, trace}; +use tracing::{error, instrument, trace}; type RecoveryInternalSyneviResult = Result::Tx>>, SyneviError>; @@ -28,7 +28,6 @@ where S: Store, { pub node: Arc>, - pub transaction: TransactionStateMachine, } #[derive(Clone, Debug, Default)] //, PartialEq, Eq)] @@ -58,53 +57,48 @@ where E: Executor, S: Store, { - #[instrument(level = "trace", skip(node, transaction))] - pub async fn new( - node: Arc>, - transaction: TransactionPayload, + #[instrument(level = "trace", skip(node))] + pub fn new(node: Arc>) -> Self { + trace!("Coordinator: New"); + Coordinator { node } + } + + #[instrument(level = "trace", skip(self))] + pub async fn pre_accept( + &self, id: u128, - ) -> Self { - trace!(?id, "Coordinator: New"); - let t0 = node.event_store.init_t_zero(node.get_serial()); - Coordinator { - node, - transaction: TransactionStateMachine { + transaction: TransactionPayload, + ) -> InternalSyneviResult { + trace!(id = ?id, "Coordinator: Preaccept"); + + let node_clone = self.node.clone(); + let (state_machine, last_applied) = tokio::task::spawn_blocking(move || { + let t_zero = node_clone.event_store.init_t_zero(node_clone.get_serial()); + let state_machine = TransactionStateMachine { id, state: State::PreAccepted, transaction, - t_zero: t0, - t: T(*t0), + t_zero: t_zero, + t: T(*t_zero), dependencies: HashSet::default(), ballot: Ballot::default(), hashes: None, - }, - } - } - - #[instrument(level = "trace", skip(self))] - pub async fn run(&mut self) -> InternalSyneviResult { - self.pre_accept().await - } - - #[instrument(level = "trace", skip(self))] - async fn pre_accept(&mut self) -> InternalSyneviResult { - trace!(id = ?self.transaction.id, "Coordinator: Preaccept"); + }; + let (t, _) = node_clone.event_store.last_applied(); + (state_machine, t.into()) + }) + .await?; self.node .stats .total_requests .fetch_add(1, Ordering::Relaxed); - let last_applied = { - let (t, _) = self.node.event_store.last_applied(); - t.into() - }; - // Create the PreAccepted msg let pre_accepted_request = PreAcceptRequest { - id: self.transaction.id.to_be_bytes().into(), - event: self.transaction.get_transaction_bytes(), - timestamp_zero: (*self.transaction.t_zero).into(), + id: state_machine.id.to_be_bytes().into(), + event: state_machine.get_transaction_bytes(), + timestamp_zero: (*state_machine.t_zero).into(), last_applied, }; @@ -129,62 +123,74 @@ where return Err(SyneviError::CompetingCoordinator); } - self.pre_accept_consensus(&pa_responses).await?; + let state_machine = self + .pre_accept_consensus(pa_responses, state_machine) + .await?; - self.accept().await + self.accept(state_machine).await } #[instrument(level = "trace", skip(self))] async fn pre_accept_consensus( - &mut self, - responses: &[PreAcceptResponse], - ) -> Result<(), SyneviError> { + &self, + responses: Vec, + mut state_machine: TransactionStateMachine, + ) -> Result, SyneviError> { // Collect deps by t_zero and only keep the max t - let (_, last_applied_t0) = self.node.event_store.last_applied(); - if last_applied_t0 != T0::default() { - self.transaction.dependencies.insert(last_applied_t0); - } - for response in responses { - let t_response = T::try_from(response.timestamp.as_slice())?; - if t_response > self.transaction.t { - self.transaction.t = t_response; - } - self.transaction - .dependencies - .extend(from_dependency(response.dependencies.clone())?); - } - // Upsert store - self.node - .event_store - .upsert_tx((&self.transaction).into())?; + let node_clone = self.node.clone(); + tokio::task::spawn_blocking(move || { + let (_, last_applied_t0) = node_clone.event_store.last_applied(); + if last_applied_t0 != T0::default() { + state_machine.dependencies.insert(last_applied_t0); + } + for response in responses { + let t_response = T::try_from(response.timestamp.as_slice())?; + if t_response > state_machine.t { + state_machine.t = t_response; + } + state_machine + .dependencies + .extend(from_dependency(response.dependencies.clone())?); + } - Ok(()) + // Upsert store + node_clone.event_store.upsert_tx((&state_machine).into())?; + Ok::<_, SyneviError>(state_machine) + }) + .await? } #[instrument(level = "trace", skip(self))] - async fn accept(&mut self) -> InternalSyneviResult { - trace!(id = ?self.transaction.id, "Coordinator: Accept"); + async fn accept( + &self, + mut state_machine: TransactionStateMachine, + ) -> InternalSyneviResult { + trace!(id = ?state_machine.id, "Coordinator: Accept"); // Safeguard: T0 <= T - assert!(*self.transaction.t_zero <= *self.transaction.t); + assert!(*state_machine.t_zero <= *state_machine.t); - if *self.transaction.t_zero != *self.transaction.t { + if *state_machine.t_zero != *state_machine.t { self.node .stats .total_accepts .fetch_add(1, Ordering::Relaxed); - let last_applied = { - let (t, _) = self.node.event_store.last_applied(); + + let node_clone = self.node.clone(); + + let last_applied = tokio::task::spawn_blocking(move || { + let (t, _) = node_clone.event_store.last_applied(); t.into() - }; + }) + .await?; let accepted_request = AcceptRequest { - id: self.transaction.id.to_be_bytes().into(), - ballot: self.transaction.ballot.into(), - event: self.transaction.get_transaction_bytes(), - timestamp_zero: (*self.transaction.t_zero).into(), - timestamp: (*self.transaction.t).into(), - dependencies: into_dependency(&self.transaction.dependencies), + id: state_machine.id.to_be_bytes().into(), + ballot: state_machine.ballot.into(), + event: state_machine.get_transaction_bytes(), + timestamp_zero: (*state_machine.t_zero).into(), + timestamp: (*state_machine.t).into(), + dependencies: into_dependency(&state_machine.dependencies), last_applied, }; @@ -202,74 +208,89 @@ where return Err(SyneviError::CompetingCoordinator); } - self.accept_consensus(&pa_responses).await?; + state_machine = self.accept_consensus(pa_responses, state_machine).await?; } - self.commit().await + self.commit(state_machine).await } #[instrument(level = "trace", skip(self))] - async fn accept_consensus(&mut self, responses: &[AcceptResponse]) -> Result<(), SyneviError> { + async fn accept_consensus( + &self, + responses: Vec, + mut state_machine: TransactionStateMachine, + ) -> Result, SyneviError> { // A little bit redundant, but I think the alternative to create a common behavior between responses may be even worse // Handle returned dependencies - for response in responses { - for dep in from_dependency(response.dependencies.clone())?.iter() { - if !self.transaction.dependencies.contains(dep) { - self.transaction.dependencies.insert(*dep); + let node_clone = self.node.clone(); + let state_machine = tokio::task::spawn_blocking(move || { + for response in responses { + for dep in from_dependency(response.dependencies.clone())?.iter() { + if !state_machine.dependencies.contains(dep) { + state_machine.dependencies.insert(*dep); + } } } - } - - // Mut state and update entry - self.transaction.state = State::Accepted; - self.node - .event_store - .upsert_tx((&self.transaction).into())?; - Ok(()) + // Mut state and update entry + state_machine.state = State::Accepted; + node_clone.event_store.upsert_tx((&state_machine).into())?; + Ok::<_, SyneviError>(state_machine) + }) + .await??; + Ok(state_machine) } #[instrument(level = "trace", skip(self))] - async fn commit(&mut self) -> InternalSyneviResult { - trace!(id = ?self.transaction.id, "Coordinator: Commit"); + async fn commit( + &self, + mut state_machine: TransactionStateMachine, + ) -> InternalSyneviResult { + trace!(id = ?state_machine.id, "Coordinator: Commit"); let committed_request = CommitRequest { - id: self.transaction.id.to_be_bytes().into(), - event: self.transaction.get_transaction_bytes(), - timestamp_zero: (*self.transaction.t_zero).into(), - timestamp: (*self.transaction.t).into(), - dependencies: into_dependency(&self.transaction.dependencies), + id: state_machine.id.to_be_bytes().into(), + event: state_machine.get_transaction_bytes(), + timestamp_zero: (*state_machine.t_zero).into(), + timestamp: (*state_machine.t).into(), + dependencies: into_dependency(&state_machine.dependencies), }; let network_interface = self.node.network.get_interface().await; let (committed_result, broadcast_result) = tokio::join!( - self.commit_consensus(), + self.commit_consensus(state_machine), network_interface.broadcast(BroadcastRequest::Commit(committed_request)) ); - committed_result.unwrap(); - broadcast_result.unwrap(); // TODO: Recovery + state_machine = committed_result?; + broadcast_result?; - self.apply().await + self.apply(state_machine).await } #[instrument(level = "trace", skip(self))] - async fn commit_consensus(&mut self) -> Result<(), SyneviError> { - self.transaction.state = State::Committed; - self.node.commit((&self.transaction).into()).await?; - Ok(()) + async fn commit_consensus( + &self, + mut state_machine: TransactionStateMachine, + ) -> Result, SyneviError> { + state_machine.state = State::Committed; + self.node.commit((&state_machine).into()).await?; + Ok(state_machine) } #[instrument(level = "trace", skip(self))] - async fn apply(&mut self) -> InternalSyneviResult { - trace!(id = ?self.transaction.id, "Coordinator: Apply"); + async fn apply( + &self, + state_machine: TransactionStateMachine, + ) -> InternalSyneviResult { + trace!(id = ?state_machine.id, "Coordinator: Apply"); - let (synevi_result, hashes) = self.execute_consensus().await?; + let (synevi_result, hashes, state_machine) = self.execute_consensus(state_machine).await?; let applied_request = ApplyRequest { - id: self.transaction.id.to_be_bytes().into(), - event: self.transaction.get_transaction_bytes(), - timestamp: (*self.transaction.t).into(), - timestamp_zero: (*self.transaction.t_zero).into(), - dependencies: into_dependency(&self.transaction.dependencies), + id: state_machine.id.to_be_bytes().into(), + event: state_machine.get_transaction_bytes(), + timestamp: (*state_machine.t).into(), + timestamp_zero: (*state_machine.t_zero).into(), + dependencies: into_dependency(&state_machine.dependencies), execution_hash: hashes.execution_hash.to_vec(), transaction_hash: hashes.transaction_hash.to_vec(), }; @@ -284,57 +305,21 @@ where #[instrument(level = "trace", skip(self))] async fn execute_consensus( - &mut self, - ) -> Result<(InternalSyneviResult, Hashes), SyneviError> { - self.transaction.state = State::Applied; - - let (result, hashes) = self.node.apply((&self.transaction).into(), None).await?; - - // let result = match &self.transaction.transaction { - // TransactionPayload::None => Err(SyneviError::TransactionNotFound), - // TransactionPayload::External(tx) => self - // .node - // .executor - // .execute(tx.clone()) - // .await - // .map(|e| ExecutorResult::External(e)), - // TransactionPayload::Internal(request) => { - // let result = match request { - // InternalExecution::JoinElectorate { - // id, - // serial, - // new_node_host, - // } => { - // let res = self - // .node - // .add_member(*id, *serial, new_node_host.clone(), false) - // .await; - // self.node - // .network - // .report_config(new_node_host.to_string()) - // .await?; - // res - // } - // InternalExecution::ReadyElectorate { id, serial } => { - // self.node.ready_member(*id, *serial).await - // } - // }; - // match result { - // Ok(_) => Ok(ExecutorResult::Internal(Ok(request.clone()))), - // Err(err) => Ok(ExecutorResult::Internal(Err(err))), - // } - // } - // }; - - // let mut hasher = Sha3_256::new(); - // postcard::to_io(&result, &mut hasher)?; - // let hash = hasher.finalize(); - // let hashes = self - // .node - // .event_store - // .get_and_update_hash(self.transaction.t_zero, hash.into())?; - - Ok((result.unwrap(), hashes)) + &self, + mut state_machine: TransactionStateMachine, + ) -> Result< + ( + InternalSyneviResult, + Hashes, + TransactionStateMachine, + ), + SyneviError, + > { + state_machine.state = State::Applied; + + let (result, hashes) = self.node.apply((&state_machine).into(), None).await?; + + Ok((result, hashes, state_machine)) } #[instrument(level = "trace", skip(node))] @@ -355,30 +340,28 @@ where })) .await?; - let mut recover_coordinator = Coordinator:: { - node, - transaction: TransactionStateMachine { - transaction: TransactionPayload::from_bytes(recover_event.transaction.clone())?, - t_zero: recover_event.t_zero, - t: recover_event.t, - ballot: recover_event.ballot, - state: recover_event.state, - id: recover_event.id, - dependencies: recover_event.dependencies.clone(), - hashes: None, - }, + let state_machine = TransactionStateMachine:: { + transaction: TransactionPayload::from_bytes(recover_event.transaction.clone())?, + t_zero: recover_event.t_zero, + t: recover_event.t, + ballot: recover_event.ballot, + state: recover_event.state, + id: recover_event.id, + dependencies: recover_event.dependencies.clone(), + hashes: None, }; - let recover_result = recover_coordinator + let recover_result = Coordinator::::new(node) .recover_consensus( recover_responses .into_iter() .map(|res| res.into_inner()) .collect::, SyneviError>>()?, + state_machine, ) .await; if let Err(err) = &recover_result { - dbg!(&err); + error!(?err); } match recover_result? { RecoveryState::Recovered(result) => return Ok(result), @@ -394,8 +377,9 @@ where #[instrument(level = "trace", skip(self))] async fn recover_consensus( - &mut self, + &self, mut responses: Vec, + mut state_machine: TransactionStateMachine, ) -> RecoveryInternalSyneviResult { // Keep track of values to replace let mut highest_ballot: Option = None; @@ -431,11 +415,11 @@ where let replica_state = State::from(response.local_state() as i32); match replica_state { - State::PreAccepted if self.transaction.state <= State::PreAccepted => { - if replica_t > self.transaction.t { + State::PreAccepted if state_machine.state <= State::PreAccepted => { + if replica_t > state_machine.t { // Slow path - self.transaction.t = replica_t; - self.transaction + state_machine.t = replica_t; + state_machine .dependencies .extend(from_dependency(response.dependencies.clone())?); } else { @@ -443,25 +427,24 @@ where fast_path_counter += 1; fast_path_deps.extend(from_dependency(response.dependencies.clone())?); } - self.transaction.state = State::PreAccepted; + state_machine.state = State::PreAccepted; } - State::Accepted if self.transaction.state < State::Accepted => { - self.transaction.t = replica_t; - self.transaction.state = State::Accepted; - self.transaction.dependencies = from_dependency(response.dependencies.clone())?; + State::Accepted if state_machine.state < State::Accepted => { + state_machine.t = replica_t; + state_machine.state = State::Accepted; + state_machine.dependencies = from_dependency(response.dependencies.clone())?; } State::Accepted - if self.transaction.state == State::Accepted - && replica_t > self.transaction.t => + if state_machine.state == State::Accepted && replica_t > state_machine.t => { - self.transaction.t = replica_t; - self.transaction.dependencies = from_dependency(response.dependencies.clone())?; + state_machine.t = replica_t; + state_machine.dependencies = from_dependency(response.dependencies.clone())?; } - any_state if any_state > self.transaction.state => { - self.transaction.state = any_state; - self.transaction.t = replica_t; - if self.transaction.state >= State::Accepted { - self.transaction.dependencies = + any_state if any_state > state_machine.state => { + state_machine.state = any_state; + state_machine.t = replica_t; + if state_machine.state >= State::Accepted { + state_machine.dependencies = from_dependency(response.dependencies.clone())?; } } @@ -471,32 +454,34 @@ where if fast_path_counter > (responses.len() / 2) { // Enforce the fast path -> Slow path was minority - self.transaction.t = T(*self.transaction.t_zero); - self.transaction.dependencies = fast_path_deps; + state_machine.t = T(*state_machine.t_zero); + state_machine.dependencies = fast_path_deps; } if let Some(ballot) = highest_ballot { - self.node - .event_store - .accept_tx_ballot(&self.transaction.t_zero, ballot); + let event_store = self.node.event_store.clone(); + tokio::task::spawn_blocking(move || { + event_store.accept_tx_ballot(&state_machine.t_zero, ballot); + }) + .await?; return Ok(RecoveryState::CompetingCoordinator); } // Wait for deps - Ok(match self.transaction.state { - State::Applied => RecoveryState::Recovered(self.apply().await?), - State::Committed => RecoveryState::Recovered(self.commit().await?), - State::Accepted => RecoveryState::Recovered(self.accept().await?), + Ok(match state_machine.state { + State::Applied => RecoveryState::Recovered(self.apply(state_machine).await?), + State::Committed => RecoveryState::Recovered(self.commit(state_machine).await?), + State::Accepted => RecoveryState::Recovered(self.accept(state_machine).await?), State::PreAccepted => { if superseding { - RecoveryState::Recovered(self.accept().await?) + RecoveryState::Recovered(self.accept(state_machine).await?) } else if !waiting.is_empty() { // We will wait anyway if RestartRecovery is returned return Ok(RecoveryState::RestartRecovery); } else { - self.transaction.t = T(*self.transaction.t_zero); - RecoveryState::Recovered(self.accept().await?) + state_machine.t = T(*state_machine.t_zero); + RecoveryState::Recovered(self.accept(state_machine).await?) } } _ => { @@ -510,19 +495,21 @@ where #[cfg(test)] pub mod tests { use super::Coordinator; - use crate::node::Node; - use crate::tests::DummyExecutor; - use crate::tests::NetworkMock; + use super::TransactionStateMachine; + use std::collections::HashSet; use std::sync::atomic::Ordering; use synevi_network::consensus_transport::PreAcceptRequest; use synevi_network::network::Network; use synevi_network::network::{BroadcastRequest, NetworkInterface}; use synevi_network::utils::IntoInner; - use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; + use synevi_types::types::TransactionPayload; + use synevi_types::Ballot; + use synevi_types::State; use synevi_types::SyneviError; - use synevi_types::{Executor, State, Transaction}; - use ulid::Ulid; + use synevi_types::T; + use synevi_types::{Executor, Transaction}; + use tracing::trace; #[derive(Default, Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] #[allow(dead_code)] @@ -545,22 +532,40 @@ pub mod tests { E: Executor, S: Store, { - pub async fn failing_pre_accept(&mut self) -> Result<(), SyneviError> { + pub async fn failing_pre_accept( + &self, + id: u128, + transaction: TransactionPayload, + ) -> Result<(), SyneviError> { + trace!(id = ?id, "Coordinator: Failing Preaccept"); + + let node_clone = self.node.clone(); + let (state_machine, last_applied) = tokio::task::spawn_blocking(move || { + let t_zero = node_clone.event_store.init_t_zero(node_clone.get_serial()); + let state_machine = TransactionStateMachine { + id, + state: State::PreAccepted, + transaction, + t_zero: t_zero, + t: T(*t_zero), + dependencies: HashSet::default(), + ballot: Ballot::default(), + hashes: None, + }; + let (t, _) = node_clone.event_store.last_applied(); + (state_machine, t.into()) + }) + .await?; self.node .stats .total_requests .fetch_add(1, Ordering::Relaxed); - let last_applied = { - let (t, _) = self.node.event_store.last_applied(); - t.into() - }; - // Create the PreAccepted msg let pre_accepted_request = PreAcceptRequest { - id: self.transaction.id.to_be_bytes().into(), - event: self.transaction.get_transaction_bytes(), - timestamp_zero: (*self.transaction.t_zero).into(), + id: state_machine.id.to_be_bytes().into(), + event: state_machine.get_transaction_bytes(), + timestamp_zero: (*state_machine.t_zero).into(), last_applied, }; @@ -577,33 +582,9 @@ pub mod tests { .map(|res| res.into_inner()) .collect::, SyneviError>>()?; - self.pre_accept_consensus(&pa_responses).await?; + self.pre_accept_consensus(pa_responses, state_machine) + .await?; Ok(()) } } - - #[tokio::test] - async fn init_test() { - let node = Node::::new_with_network_and_executor( - Ulid::new(), - 0, - NetworkMock::default(), - DummyExecutor, - ) - .await - .unwrap(); - - let coordinator = Coordinator::new( - node, - synevi_types::types::TransactionPayload::External(b"foo".to_vec()), - 0, - ) - .await; - - assert_eq!(coordinator.transaction.state, State::PreAccepted); - assert_eq!(*coordinator.transaction.t_zero, *coordinator.transaction.t); - assert_eq!(coordinator.transaction.t_zero.0.get_node(), 0); - assert_eq!(coordinator.transaction.t_zero.0.get_seq(), 1); - assert!(coordinator.transaction.dependencies.is_empty()); - } } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 342a16f..7328419 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -17,7 +17,7 @@ use synevi_types::types::{ }; use synevi_types::{Executor, State, SyneviError, Transaction, T}; use tokio::task::JoinSet; -use tracing::instrument; +use tracing::{error, instrument}; use ulid::Ulid; #[derive(Debug, Default)] @@ -216,9 +216,9 @@ where return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; - let mut coordinator = - Coordinator::new(self.clone(), TransactionPayload::External(transaction), id).await; - let tx_result = coordinator.run().await?; + let tx_result = Coordinator::new(self.clone()) + .pre_accept(id, TransactionPayload::External(transaction)) + .await?; match tx_result { ExecutorResult::External(e) => Ok(e), @@ -237,8 +237,9 @@ where return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; - let mut coordinator = Coordinator::new(self.clone(), transaction, id).await; - let result = coordinator.run().await; + let result = Coordinator::new(self.clone()) + .pre_accept(id, transaction) + .await; result } @@ -258,13 +259,17 @@ where } #[instrument(level = "trace", skip(self))] - pub async fn commit(&self, event: UpsertEvent) -> Result<(), SyneviError> { + pub(crate) async fn commit(&self, event: UpsertEvent) -> Result<(), SyneviError> { let t0_commit = event.t_zero.clone(); let t_commit = event.t.clone(); - - let prev_event = self.event_store.get_event(t0_commit)?; - - self.event_store.upsert_tx(event.clone())?; + let event_store = self.event_store.clone(); + let event_clone = event.clone(); + let prev_event = tokio::task::spawn_blocking(move || { + let prev_event = event_store.get_event(t0_commit)?; + event_store.upsert_tx(event_clone)?; + Ok::<_, SyneviError>(prev_event) + }) + .await??; self.wait_handler.notify_commit(&t0_commit, &t_commit); if !prev_event.is_some_and(|e| e.state > State::Committed || e.dependencies.is_empty()) { if let Some(waiter) = self.wait_handler.get_waiter(&event) { @@ -279,14 +284,17 @@ where } #[instrument(level = "trace", skip(self))] - pub async fn apply( + pub(crate) async fn apply( &self, mut event: UpsertEvent, request_hashes: Option, - ) -> Result<(Option>, Hashes), SyneviError> { + ) -> Result<(InternalSyneviResult, Hashes), SyneviError> { let t0_apply = event.t_zero.clone(); + let event_store = self.event_store.clone(); - let needs_wait = if let Some(prev_event) = self.event_store.get_event(t0_apply)? { + let needs_wait = if let Some(prev_event) = + tokio::task::spawn_blocking(move || event_store.get_event(t0_apply)).await?? + { prev_event.state < State::Applied } else { let mut commit_event = event.clone(); @@ -309,20 +317,16 @@ where } // - Check transaction hash -> SyneviError::MismatchingTransactionHash - let mut node_hashes = self - .event_store - .get_or_update_transaction_hash(event.clone())?; + let event_store = self.event_store.clone(); + let event_clone = event.clone(); + let mut node_hashes = tokio::task::spawn_blocking(move || { + event_store.get_or_update_transaction_hash(event_clone) + }) + .await??; if let Some(hashes) = &request_hashes { if hashes.transaction_hash != node_hashes.transaction_hash { - println!( - "MismatchedTransactionHashes @ Node: {} -request: {:?} -got: {:?}", - self.get_serial(), - hashes.transaction_hash, - node_hashes.transaction_hash - ); + error!(?hashes, ?node_hashes, "Mismatched hashes"); return Err(SyneviError::MismatchedTransactionHashes); } } @@ -336,22 +340,28 @@ got: {:?}", )?; let result = self.execute(transaction).await; - // - Check execution hash -> SyneviError::MismatchingExecutionHash - let mut hasher = Sha3_256::new(); - postcard::to_io(&result, &mut hasher)?; - let execution_hash: [u8; 32] = hasher.finalize().into(); - if let Some(hashes) = request_hashes { - if hashes.execution_hash != execution_hash { - return Err(SyneviError::MismatchedExecutionHashes); - } - } - node_hashes.execution_hash = execution_hash; - event.hashes = Some(node_hashes.clone()); + let event_store = self.event_store.clone(); - // - Upsert - self.event_store.upsert_tx(event)?; + let (result, node_hashes) = tokio::task::spawn_blocking(move || { + // - Check execution hash -> SyneviError::MismatchingExecutionHash + let mut hasher = Sha3_256::new(); + postcard::to_io(&result, &mut hasher)?; + let execution_hash: [u8; 32] = hasher.finalize().into(); + if let Some(hashes) = request_hashes { + if hashes.execution_hash != execution_hash { + return Err(SyneviError::MismatchedExecutionHashes); + } + } + node_hashes.execution_hash = execution_hash; + event.hashes = Some(node_hashes.clone()); + + // - Upsert + event_store.upsert_tx(event)?; + Ok((result, node_hashes)) + }) + .await??; self.wait_handler.notify_apply(&t0_apply); - Ok((Some(result), node_hashes)) + Ok((result, node_hashes)) } async fn execute( @@ -577,13 +587,12 @@ mod tests { transaction: Vec, ) -> Result<(), SyneviError> { let _permit = self.semaphore.acquire().await?; - let mut coordinator = Coordinator::new( - self.clone(), - synevi_types::types::TransactionPayload::External(transaction), - id, - ) - .await; - coordinator.failing_pre_accept().await?; + Coordinator::new(self.clone()) + .failing_pre_accept( + id, + synevi_types::types::TransactionPayload::External(transaction), + ) + .await?; Ok(()) } } From 614066e8db3bdb4590f9e7ce653d424566931dc2 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 26 Oct 2024 15:08:10 +0200 Subject: [PATCH 28/40] chore: replica put potential blocking tasks into spawn_blocking --- synevi_core/src/replica.rs | 149 +++++++++++++++---------------------- 1 file changed, 61 insertions(+), 88 deletions(-) diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index 3fcb3dc..dfbcf13 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -66,42 +66,26 @@ where let request_id = u128::from_be_bytes(request.id.as_slice().try_into()?); trace!(?request_id, "Replica: PreAccept"); - - // TODO(performance): Remove the lock here - // Creates contention on the event store - if let Some(ballot) = self - .node - .event_store - .accept_tx_ballot(&t0, Ballot::default()) - { - if ballot != Ballot::default() { - return Ok(PreAcceptResponse { - nack: true, - ..Default::default() - }); + let event_store = self.node.event_store.clone(); + tokio::task::spawn_blocking(move || { + if let Some(ballot) = event_store.accept_tx_ballot(&t0, Ballot::default()) { + if ballot != Ballot::default() { + return Ok(PreAcceptResponse { + nack: true, + ..Default::default() + }); + } } - } - - // let waiting_time = self.network.get_waiting_time(node_serial).await; - - // let (sx, rx) = oneshot::channel(); - let (t, deps) = self - .node - .event_store - .pre_accept_tx(request_id, t0, request.event)?; - - // self.reorder_buffer - // .send_msg(t0, sx, request.event, waiting_time) - // .await?; - - // let (t, deps) = rx.await?; + let (t, deps) = event_store.pre_accept_tx(request_id, t0, request.event)?; - Ok(PreAcceptResponse { - timestamp: t.into(), - dependencies: into_dependency(&deps), - nack: false, + Ok(PreAcceptResponse { + timestamp: t.into(), + dependencies: into_dependency(&deps), + nack: false, + }) }) + .await? } #[instrument(level = "trace", skip(self, request))] @@ -116,12 +100,9 @@ where trace!(?request_id, "Replica: Accept"); - let dependencies = { - if let Some(ballot) = self - .node - .event_store - .accept_tx_ballot(&t_zero, request_ballot) - { + let event_store = self.node.event_store.clone(); + tokio::task::spawn_blocking(move || { + if let Some(ballot) = event_store.accept_tx_ballot(&t_zero, request_ballot) { if ballot != request_ballot { return Ok(AcceptResponse { dependencies: Vec::new(), @@ -130,7 +111,7 @@ where } } - self.node.event_store.upsert_tx(UpsertEvent { + event_store.upsert_tx(UpsertEvent { id: request_id, t_zero, t, @@ -140,14 +121,12 @@ where ballot: Some(request_ballot), hashes: None, })?; - - self.node.event_store.get_tx_dependencies(&t, &t_zero) - }; - - Ok(AcceptResponse { - dependencies: into_dependency(&dependencies), - nack: false, + Ok(AcceptResponse { + dependencies: into_dependency(&event_store.get_tx_dependencies(&t, &t_zero)), + nack: false, + }) }) + .await? } #[instrument(level = "trace", skip(self, request))] @@ -228,53 +207,46 @@ where // TODO/WARNING: This was initially in one Mutex lock //let mut event_store = self.node.event_store.lock().await; - if let Some(state) = self.node.event_store.get_event_state(&t_zero) { - // If another coordinator has started recovery with a higher ballot - // Return NACK with the higher ballot number - let request_ballot = Ballot::try_from(request.ballot.as_slice())?; - if let Some(ballot) = self - .node - .event_store - .accept_tx_ballot(&t_zero, request_ballot) - { - if request_ballot != ballot { - return Ok(RecoverResponse { - nack: ballot.into(), - ..Default::default() - }); - } - } - - if matches!(state, State::Undefined) { - self.node - .event_store - .pre_accept_tx(request_id, t_zero, request.event)?; - }; - } else { - self.node - .event_store - .pre_accept_tx(request_id, t_zero, request.event)?; - } - let recover_deps = self.node.event_store.get_recover_deps(&t_zero)?; - + let event_store = self.node.event_store.clone(); self.node .stats .total_recovers .fetch_add(1, Ordering::Relaxed); + tokio::task::spawn_blocking(move || { + if let Some(state) = event_store.get_event_state(&t_zero) { + // If another coordinator has started recovery with a higher ballot + // Return NACK with the higher ballot number + let request_ballot = Ballot::try_from(request.ballot.as_slice())?; + if let Some(ballot) = event_store.accept_tx_ballot(&t_zero, request_ballot) { + if request_ballot != ballot { + return Ok(RecoverResponse { + nack: ballot.into(), + ..Default::default() + }); + } + } - let local_state = self - .node - .event_store - .get_event_state(&t_zero) - .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner()))?; - Ok(RecoverResponse { - local_state: local_state.into(), - wait: into_dependency(&recover_deps.wait), - superseding: recover_deps.superseding, - dependencies: into_dependency(&recover_deps.dependencies), - timestamp: recover_deps.timestamp.into(), - nack: Ballot::default().into(), + if matches!(state, State::Undefined) { + event_store.pre_accept_tx(request_id, t_zero, request.event)?; + }; + } else { + event_store.pre_accept_tx(request_id, t_zero, request.event)?; + } + let recover_deps = event_store.get_recover_deps(&t_zero)?; + + let local_state = event_store + .get_event_state(&t_zero) + .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner()))?; + Ok(RecoverResponse { + local_state: local_state.into(), + wait: into_dependency(&recover_deps.wait), + superseding: recover_deps.superseding, + dependencies: into_dependency(&recover_deps.dependencies), + timestamp: recover_deps.timestamp.into(), + nack: Ballot::default().into(), + }) }) + .await? } #[instrument(level = "trace", skip(self))] @@ -296,7 +268,8 @@ where } // This ensures that this t0 will not get a fast path in the future - self.node.event_store.inc_time_with_guard(t0)?; + let event_store = self.node.event_store.clone(); + tokio::task::spawn_blocking(move || event_store.inc_time_with_guard(t0)).await??; Ok(TryRecoveryResponse { accepted: false }) } } From c28354a0f2444b51a04445b3fb943f30e609fc07 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 26 Oct 2024 15:14:14 +0200 Subject: [PATCH 29/40] chore: Remove unnecessary prints --- synevi_core/src/node.rs | 19 ++------ synevi_core/src/reorder_buffer.rs | 4 +- synevi_core/src/replica.rs | 1 - synevi_network/src/network.rs | 10 ++-- synevi_persistence/src/lmdb_store.rs | 72 ---------------------------- synevi_persistence/src/mem_store.rs | 6 --- synevi_persistence/src/redb_store.rs | 16 +------ 7 files changed, 10 insertions(+), 118 deletions(-) diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 7328419..005c1c9 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -17,7 +17,7 @@ use synevi_types::types::{ }; use synevi_types::{Executor, State, SyneviError, Transaction, T}; use tokio::task::JoinSet; -use tracing::{error, instrument}; +use tracing::{error, instrument, trace}; use ulid::Ulid; #[derive(Debug, Default)] @@ -429,7 +429,7 @@ where match self.wait_handler.check_recovery() { CheckResult::NoRecovery => (), CheckResult::RecoverEvent(recover_event) => { - println!( + trace!( "{}, Recovering event: {:?}", self.get_serial(), recover_event @@ -445,7 +445,7 @@ where }); } CheckResult::RecoverUnknown(t0_recover) => { - println!( + trace!( "{}, Recovering unknown: {:?}", self.get_serial(), t0_recover @@ -454,7 +454,7 @@ where match interface.broadcast_recovery(t0_recover).await { Ok(true) => (), Ok(false) => { - println!("Unknown recovery failed"); + error!("Unknown recovery failed"); self.wait_handler.notify_apply(&t0_recover); } Err(err) => { @@ -474,12 +474,8 @@ where replica: ReplicaConfig, member_host: String, ) -> Result<(), SyneviError> { - // 1. Broadcast self_config to other member - println!("{} Before join", self.get_serial()); let expected = self.network.join_electorate(member_host.clone()).await?; // 2. wait for JoinElectorate responses with expected majority and config from others - - println!("{} Waiting for responded", self.get_serial()); while self .network .get_node_status() @@ -489,18 +485,13 @@ where { tokio::time::sleep(tokio::time::Duration::from_millis(50)).await; } - - println!("{} Sync events", self.get_serial()); let (last_applied, _) = self.event_store.last_applied(); self.sync_events(last_applied, &replica).await?; // 3. Send ReadyJoinElectorate && set myself to ready self.set_ready(); - println!("{}, Before electorate", self.get_serial()); self.network.ready_electorate(member_host).await?; - println!("{}, Ready electorate finished", self.get_serial()); - Ok(()) } @@ -529,7 +520,6 @@ where transaction_hash: event.transaction_hash, }) .await?; - //println!("{response}"); Ok::<(), SyneviError>(()) }); } @@ -545,7 +535,6 @@ where dependencies: event.dependencies, }) .await?; - //println!("{response}"); Ok::<(), SyneviError>(()) }); } diff --git a/synevi_core/src/reorder_buffer.rs b/synevi_core/src/reorder_buffer.rs index 519043b..f794b8a 100644 --- a/synevi_core/src/reorder_buffer.rs +++ b/synevi_core/src/reorder_buffer.rs @@ -7,6 +7,7 @@ use std::{ use synevi_network::network::Network; use synevi_types::{traits::Store, Executor, SyneviError, T, T0}; use tokio::{sync::oneshot, time::timeout}; +use tracing::error; use crate::{node::Node, utils::into_dependency}; @@ -85,7 +86,6 @@ where event, latency, })) => { - //println!("Received message: {:?} latency: {}", t0, latency); let now = Instant::now(); buffer.insert(t0, (notify, event, id)); if current_transaction.1 == T0::default() { @@ -112,7 +112,7 @@ where next_latency = latency; } Ok(Err(e)) => { - println!("Error receiving message {e}") + error!("Error receiving message {e}") } Err(_) => { // Elapsed more than 1.2x average (TODO) latency diff --git a/synevi_core/src/replica.rs b/synevi_core/src/replica.rs index dfbcf13..5317dfb 100644 --- a/synevi_core/src/replica.rs +++ b/synevi_core/src/replica.rs @@ -362,7 +362,6 @@ where sdx.send(response).await.unwrap(); } }); - println!("Returning streaming receiver"); // Stream all events to member Ok(rcv) } diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index c0f0c4d..a2824bc 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -28,6 +28,7 @@ use tokio::sync::{Mutex, RwLock}; use tokio::task::JoinSet; use tonic::metadata::{AsciiMetadataKey, AsciiMetadataValue}; use tonic::transport::{Channel, Server}; +use tracing::error; use ulid::Ulid; #[async_trait::async_trait] @@ -559,12 +560,10 @@ impl NetworkInterface for GrpcNetworkSet { } Ok(Err(e)) => { tracing::error!("Error in response: {:?}", e); - println!("Error in response: {:?}", e); continue; } Err(_) => { tracing::error!("Join error"); - println!("Join error"); continue; } }; @@ -581,7 +580,6 @@ impl NetworkInterface for GrpcNetworkSet { result.push(res); } _ => { - println!("Recover: Join error"); tracing::error!("Recover: Join error"); continue; } @@ -592,12 +590,10 @@ impl NetworkInterface for GrpcNetworkSet { while let Some(r) = &responses.join_next().await { match r { Ok(Err(e)) => { - println!("Apply: Error in response: {:?}", e); tracing::error!("Apply: Error in response: {:?}", e); continue; } Err(_) => { - println!("Apply: Join error"); tracing::error!("Apply: Join error"); continue; } @@ -612,8 +608,8 @@ impl NetworkInterface for GrpcNetworkSet { } if result.len() < majority && !self.members.is_empty() { - println!("Majority not reached: {:?}/{}", result, majority); - println!("Members: {:?}", &self.members); + error!("Majority not reached: {:?}/{}", result, majority); + error!("Members: {:?}", &self.members); return Err(SyneviError::MajorityNotReached); } Ok(result) diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 0ab4d73..64bc99a 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -371,13 +371,6 @@ impl InternalData { let last_t = self.last_applied; - // if last_t > event.t { - // println!( - // "Node: {:?}, store: {:?}", - // self.node_serial, - // self.get_event_store() - // ); - // } // Safeguard assert!(last_t < event.t); @@ -435,15 +428,6 @@ impl InternalData { if last_t > event.t { println!("last_t: {:?}, event.t: {:?}", last_t, event.t); } - // println!( - // "Node: {:?}, last_t: {:?}, event_t: {:?}, store: {:?}", - // self.node_serial, - // last_t, - // event.t, - // self.get_event_store() - // ); - // } - // Safeguard assert!(last_t < event.t); self.last_applied = event.t; @@ -645,60 +629,4 @@ impl InternalData { read_txn.commit()?; Ok(event) } - - // fn get_and_update_hash( - // &self, - // t_zero: T0, - // execution_hash: [u8; 32], - // ) -> Result { - // let t_zero = t_zero.get_inner(); - // let mut write_txn = self.db.write_txn()?; - // let db: EventDb = self - // .db - // .open_database(&write_txn, Some(EVENT_DB_NAME))? - // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - // let Some(mut event) = db.get(&write_txn, &t_zero)? else { - // return Err(SyneviError::EventNotFound(t_zero)); - // }; - // let Some(mut hashes) = event.hashes else { - // return Err(SyneviError::MissingTransactionHash); - // }; - // hashes.execution_hash = execution_hash; - // event.hashes = Some(hashes.clone()); - - // db.put(&mut write_txn, &t_zero, &event)?; - // write_txn.commit()?; - // Ok(hashes) - // } - - // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - // let last = self.last_applied; - // let last_t0 = self - // .mappings - // .get(&last) - // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; - // let read_txn = self.db.read_txn()?; - // let db: EventDb = self - // .db - // .open_database(&read_txn, Some(EVENT_DB_NAME))? - // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - // let event = db - // .get(&read_txn, &last_t0.get_inner())? - // .ok_or_else(|| SyneviError::EventNotFound(last_t0.get_inner()))? - // .hashes - // .ok_or_else(|| SyneviError::MissingExecutionHash)?; - // Ok((last, event.execution_hash)) - // } -} - -#[cfg(test)] -mod tests { - - #[test] - fn test_db() { - // TODO - //let db = Database::new("../../tests/database".to_string()).unwrap(); - //db.init(Bytes::from("key"), Bytes::from("value")) - // .unwrap() - } } diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index f672596..351118f 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -321,12 +321,6 @@ impl InternalStore { } if event.state == State::Applied { - if self.last_applied > event.t { - println!( - "Node: {} | {:?} > {:?} | Deps: {:?}", - self.node_serial, self.last_applied, event.t, event.dependencies - ); - } assert!(self.last_applied < event.t); self.last_applied = event.t; let hashes = upsert_event diff --git a/synevi_persistence/src/redb_store.rs b/synevi_persistence/src/redb_store.rs index b65452b..50ff462 100644 --- a/synevi_persistence/src/redb_store.rs +++ b/synevi_persistence/src/redb_store.rs @@ -365,13 +365,6 @@ impl InternalData { let last_t = self.last_applied; - // if last_t > event.t { - // println!( - // "Node: {:?}, store: {:?}", - // self.node_serial, - // self.get_event_store() - // ); - // } // Safeguard assert!(last_t < event.t); @@ -436,14 +429,7 @@ impl InternalData { if last_t > event.t { println!("last_t: {:?}, event.t: {:?}", last_t, event.t); } - // println!( - // "Node: {:?}, last_t: {:?}, event_t: {:?}, store: {:?}", - // self.node_serial, - // last_t, - // event.t, - // self.get_event_store() - // ); - // } + // Safeguard assert!(last_t < event.t); From af96266bd4358484942433a6cfccb0badc46ace6 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 26 Oct 2024 15:26:34 +0200 Subject: [PATCH 30/40] fix: Wrong path in tests --- tests/consensus_e2e.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index 752c816..ae7a2ad 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -23,7 +23,7 @@ mod tests { let mut nodes: Vec>> = vec![]; for (i, m) in node_names.iter().enumerate() { - let test_path = format!("dev/shm/{m}"); + let test_path = format!("/dev/shm/{m}"); fs::create_dir(&test_path).await.unwrap(); dbg!(&test_path); let store = LmdbStore::new(test_path, i as u16).unwrap(); From 2b4151fbc0af46597d1e93119cbff4b04459c4a5 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Sat, 26 Oct 2024 15:52:05 +0200 Subject: [PATCH 31/40] chore: Update README.md add github actions workflows --- .github/actions-rs/grcov.yml | 2 + .github/workflows/codecov.yml | 30 +++++++++++++++ .github/workflows/lint.yml | 36 ++++++++++++++++++ .github/workflows/test.yml | 24 ++++++++++++ README.md | 72 +++++++++++++++++++++++++++++++++-- 5 files changed, 161 insertions(+), 3 deletions(-) create mode 100644 .github/actions-rs/grcov.yml create mode 100644 .github/workflows/codecov.yml create mode 100644 .github/workflows/lint.yml create mode 100644 .github/workflows/test.yml diff --git a/.github/actions-rs/grcov.yml b/.github/actions-rs/grcov.yml new file mode 100644 index 0000000..dbe2786 --- /dev/null +++ b/.github/actions-rs/grcov.yml @@ -0,0 +1,2 @@ +output-type: lcov +output-file: ./lcov.info \ No newline at end of file diff --git a/.github/workflows/codecov.yml b/.github/workflows/codecov.yml new file mode 100644 index 0000000..4b93f4c --- /dev/null +++ b/.github/workflows/codecov.yml @@ -0,0 +1,30 @@ +on: + push: + branches: [main] + pull_request: +name: coverage +env: + CARGO_NET_GIT_FETCH_WITH_CLI: true +jobs: + test: + runs-on: ubuntu-latest + steps: + - uses: arduino/setup-protoc@v1 + - uses: actions-rs/toolchain@v1 + with: + toolchain: nightly + override: true + - uses: actions/checkout@v2 + with: + submodules: recursive + - uses: actions-rs/cargo@v1 + with: + command: test + args: --all-features --no-fail-fast -- --include-ignored --test-threads=1 + env: + CARGO_INCREMENTAL: '0' + RUSTFLAGS: '-Zprofile -Ccodegen-units=1 -Cinline-threshold=0 -Clink-dead-code -Coverflow-checks=off -Cpanic=abort -Zpanic_abort_tests' + RUSTDOCFLAGS: '-Zprofile -Ccodegen-units=1 -Cinline-threshold=0 -Clink-dead-code -Coverflow-checks=off -Cpanic=abort -Zpanic_abort_tests' + - uses: actions-rs/grcov@v0.1.5 + - name: Upload to codecov.io + uses: codecov/codecov-action@v3 diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml new file mode 100644 index 0000000..40620c6 --- /dev/null +++ b/.github/workflows/lint.yml @@ -0,0 +1,36 @@ +on: + push: + branches: [main, dev] + pull_request: +name: lint +env: + CARGO_NET_GIT_FETCH_WITH_CLI: true +jobs: + style: + runs-on: ubuntu-latest + steps: + - uses: arduino/setup-protoc@v1 + - uses: actions-rs/toolchain@v1 + with: + profile: minimal + toolchain: stable + components: rustfmt, clippy + - uses: actions/checkout@v2 + with: + submodules: recursive + - name: cargo fmt --check + uses: actions-rs/cargo@v1 + with: + command: fmt + args: --check + - name: cargo doc + uses: actions-rs/cargo@v1 + if: always() + with: + command: doc + args: --no-deps --all-features + - name: cargo clippy + uses: actions-rs/clippy-check@v1 + if: always() + with: + token: ${{ secrets.GITHUB_TOKEN }} diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml new file mode 100644 index 0000000..16f95b9 --- /dev/null +++ b/.github/workflows/test.yml @@ -0,0 +1,24 @@ +on: + - push + - pull_request +name: test +jobs: + test: + runs-on: ubuntu-latest + name: test + env: + CARGO_NET_GIT_FETCH_WITH_CLI: true + steps: + - uses: arduino/setup-protoc@v1 + - uses: actions-rs/toolchain@v1 + with: + profile: minimal + toolchain: stable + - uses: actions/checkout@v2 + with: + submodules: recursive + - name: cargo test + uses: actions-rs/cargo@v1 + with: + command: test + args: --all-features -- --include-ignored diff --git a/README.md b/README.md index 7878131..7c3dce2 100644 --- a/README.md +++ b/README.md @@ -1,3 +1,9 @@ +[![Rust](https://img.shields.io/badge/built_with-Rust-dca282.svg)](https://www.rust-lang.org/) +[![License](https://img.shields.io/badge/License-Apache_2.0-brightgreen.svg)](https://github.com/ArunaStorage/synevi/blob/main/LICENSE-APACHE) +[![License](https://img.shields.io/badge/License-MIT-brightgreen.svg)](https://github.com/ArunaStorage/synevi/blob/main/LICENSE-MIT) +![CI](https://github.com/ArunaStorage/synevi/actions/workflows/test.yml/badge.svg) +[![Codecov](https://codecov.io/github/ArunaStorage/synevi/coverage.svg?branch=main)](https://codecov.io/gh/ArunaStorage/synevi) + # Synevi Synevi (greek: συνέβει - "it happened") is a leaderless, strict serializable, embeddable event store for event sourcing. @@ -16,11 +22,71 @@ For a more detailed explanation of the underlying architecture please visit our ### Embedded -TBD +## Getting Started with Synevi + +Synevi provides a distributed event storage system that ensures consistent event ordering across your network. You can get started with either gRPC networking (the default) and your choice of persistent LMDB storage or in-memory storage to match your needs. + +At its core, Synevi works by distributing events through coordinator nodes. When an event is sent to any coordinator, Synevi ensures all nodes execute it in the same order, maintaining consistency across your system. To work with Synevi, you'll need to implement two key traits for your events. + +First, define your events by implementing the `Transaction` trait: + +```rust +pub trait Transaction: Debug + Clone + Send { + type TxErr: Send + Serialize; + type TxOk: Send + Serialize; + + fn as_bytes(&self) -> Vec; + fn from_bytes(bytes: Vec) -> Result + where + Self: Sized; +} +``` + +Then, implement the `Executor` trait to handle how your store processes these events: + +```rust +#[async_trait::async_trait] +pub trait Executor: Send + Sync + 'static { + type Tx: Transaction + Serialize; + + async fn execute(&self, transaction: Self::Tx) -> SyneviResult; +} +``` + +Here's a complete example that shows how to set up a Synevi node: + +```rust +// Set up the network address +let socket_addr = SocketAddr::from_str(&format!("0.0.0.0:{}", 13000 + i)).unwrap(); +let network = synevi::network::GrpcNetwork::new( + socket_addr, + format!("http://0.0.0.0:{}", 13000 + i), + Ulid::new(), + 0u16, +); + +// Choose your storage implementation +// For persistent storage using LMDB: +let test_path = format!("/dev/shm/test/"); +fs::create_dir(&test_path).await.unwrap(); +let store = LmdbStore::new(test_path, 0u16).unwrap(); + +// Or for in-memory storage: +// let store = MemStore::new(0u16).unwrap(); + +// Create your node using your executor (or the DummyExecutor that does nothing) +let node = Node::new(m, i as u16, network, DummyExecutor, store) + .await + .unwrap(); +``` + +Synevi will automatically handle event distribution and ensure proper execution order across all functioning nodes in your network. The system is designed to be resilient, continuing to operate even if some nodes become unavailable. Whether you choose LMDB for persistence or in-memory storage for speed, the same consistency guarantees apply - all events are executed in the same order across every node in your network. + +For production deployments, make sure your `Transaction` implementations include proper error handling and that your `Executor` implementation is thread-safe. Consider your network topology when setting up nodes, as it can impact system performance and reliability. -### As standalone Application +Note: Currently each node can handle only one type of executor, this can make it complicated when having many different types of events, for this you can use the excellent [typetag](https://github.com/dtolnay/typetag) crate that enables serialization of many types that implement shared behavior. -TBD +### As standalone Application (Coming soon) ## Feedback & Contributions From 400179010ba3398b038f4a113addf124c9bd8683 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 28 Oct 2024 16:35:32 +0100 Subject: [PATCH 32/40] chore: Made tests smaller --- tests/consensus_e2e.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/consensus_e2e.rs b/tests/consensus_e2e.rs index ae7a2ad..79d7c1c 100644 --- a/tests/consensus_e2e.rs +++ b/tests/consensus_e2e.rs @@ -58,7 +58,7 @@ mod tests { let mut joinset = tokio::task::JoinSet::new(); - for i in 0..5000 { + for i in 0..100 { let coordinator = coordinator.clone(); joinset.spawn(async move { coordinator @@ -178,7 +178,7 @@ mod tests { let start = std::time::Instant::now(); - for _ in 0..1000 { + for _ in 0..100 { let coordinator1 = coordinator1.clone(); let coordinator2 = coordinator2.clone(); let coordinator3 = coordinator3.clone(); @@ -357,7 +357,7 @@ mod tests { .unwrap(); } - for i in 0..1000 { + for i in 0..100 { coordinator .clone() .transaction(i, Vec::from("This is a transaction")) From 6741322ec94e5944b7ae367dfa3a7795f73496ed Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 28 Oct 2024 16:38:22 +0100 Subject: [PATCH 33/40] chore: Clippy fixes, fmt --- synevi_core/src/coordinator.rs | 5 +++-- synevi_core/src/node.rs | 33 +++++++++++++--------------- synevi_core/src/wait_handler.rs | 16 ++++++-------- synevi_persistence/src/lmdb_store.rs | 2 +- synevi_persistence/src/redb_store.rs | 2 +- synevi_types/src/types.rs | 2 +- 6 files changed, 28 insertions(+), 32 deletions(-) diff --git a/synevi_core/src/coordinator.rs b/synevi_core/src/coordinator.rs index 76e7278..eab77ed 100644 --- a/synevi_core/src/coordinator.rs +++ b/synevi_core/src/coordinator.rs @@ -78,7 +78,7 @@ where id, state: State::PreAccepted, transaction, - t_zero: t_zero, + t_zero, t: T(*t_zero), dependencies: HashSet::default(), ballot: Ballot::default(), @@ -304,6 +304,7 @@ where } #[instrument(level = "trace", skip(self))] + #[allow(clippy::type_complexity)] async fn execute_consensus( &self, mut state_machine: TransactionStateMachine, @@ -546,7 +547,7 @@ pub mod tests { id, state: State::PreAccepted, transaction, - t_zero: t_zero, + t_zero, t: T(*t_zero), dependencies: HashSet::default(), ballot: Ballot::default(), diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index 005c1c9..d42436e 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -116,7 +116,7 @@ where Ok(node) } - pub fn set_ready(&self) -> () { + pub fn set_ready(&self) { self.network .get_node_status() .info @@ -237,11 +237,10 @@ where return Err(SyneviError::NotReady); }; let _permit = self.semaphore.acquire().await?; - let result = Coordinator::new(self.clone()) - .pre_accept(id, transaction) - .await; - result + Coordinator::new(self.clone()) + .pre_accept(id, transaction) + .await } pub fn get_stats(&self) -> (u64, u64, u64) { @@ -260,8 +259,8 @@ where #[instrument(level = "trace", skip(self))] pub(crate) async fn commit(&self, event: UpsertEvent) -> Result<(), SyneviError> { - let t0_commit = event.t_zero.clone(); - let t_commit = event.t.clone(); + let t0_commit = event.t_zero; + let t_commit = event.t; let event_store = self.event_store.clone(); let event_clone = event.clone(); let prev_event = tokio::task::spawn_blocking(move || { @@ -275,7 +274,7 @@ where if let Some(waiter) = self.wait_handler.get_waiter(&event) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); - SyneviError::ReceiveError(format!("Error waiting for commit")) + SyneviError::ReceiveError("Error waiting for commit".to_string()) })? }; } @@ -289,7 +288,7 @@ where mut event: UpsertEvent, request_hashes: Option, ) -> Result<(InternalSyneviResult, Hashes), SyneviError> { - let t0_apply = event.t_zero.clone(); + let t0_apply = event.t_zero; let event_store = self.event_store.clone(); let needs_wait = if let Some(prev_event) = @@ -311,7 +310,7 @@ where if let Some(waiter) = self.wait_handler.get_waiter(&event) { waiter.await.map_err(|e| { tracing::error!("Error waiting for commit: {:?}", e); - SyneviError::ReceiveError(format!("Error waiting for commit")) + SyneviError::ReceiveError("Error waiting for commit".to_string()) })?; } } @@ -369,15 +368,14 @@ where transaction: TransactionPayload<::Tx>, ) -> Result::Tx>, SyneviError> { // TODO: Refactor in execute function - let result = match transaction { - TransactionPayload::None => { - return Err(SyneviError::TransactionNotFound); - } + + match transaction { + TransactionPayload::None => Err(SyneviError::TransactionNotFound), TransactionPayload::External(tx) => self .executor .execute(tx) .await - .map(|e| ExecutorResult::<::Tx>::External(e)), + .map(ExecutorResult::<::Tx>::External), TransactionPayload::Internal(request) => { // TODO: Build special execution let result = match &request { @@ -409,8 +407,7 @@ where Err(err) => Ok(ExecutorResult::Internal(Err(err))), } } - }; - result + } } async fn run_check_recovery(&self) { @@ -752,7 +749,7 @@ mod tests { 0, synevi_network::network::GrpcNetwork::new( SocketAddr::from_str("0.0.0.0:1337").unwrap(), - format!("http://localhost:1337"), + "http://localhost:1337".to_string(), id, 0, ), diff --git a/synevi_core/src/wait_handler.rs b/synevi_core/src/wait_handler.rs index 31a30ad..5f96fba 100644 --- a/synevi_core/src/wait_handler.rs +++ b/synevi_core/src/wait_handler.rs @@ -119,16 +119,14 @@ where pub fn notify_commit(&self, t0_commit: &T0, t_commit: &T) { let mut waiter_lock = self.waiters.lock().expect("Locking waiters failed"); waiter_lock.retain(|_, waiter| { - if waiter.dependencies.contains(t0_commit) { - if t_commit > &waiter.t { - waiter.dependencies.remove(t0_commit); - waiter.waited_since = Instant::now(); - if waiter.dependencies.is_empty() { - for sdx in waiter.sender.drain(..) { - let _ = sdx.send(()); - } - return false; + if waiter.dependencies.contains(t0_commit) && t_commit > &waiter.t { + waiter.dependencies.remove(t0_commit); + waiter.waited_since = Instant::now(); + if waiter.dependencies.is_empty() { + for sdx in waiter.sender.drain(..) { + let _ = sdx.send(()); } + return false; } } true diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 64bc99a..176a5a9 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -585,7 +585,7 @@ impl InternalData { } fn last_applied(&self) -> (T, T0) { - let t = self.last_applied.clone(); + let t = self.last_applied; let t0 = self.mappings.get(&t).cloned().unwrap_or(T0::default()); (t, t0) } diff --git a/synevi_persistence/src/redb_store.rs b/synevi_persistence/src/redb_store.rs index 50ff462..bc8d4cb 100644 --- a/synevi_persistence/src/redb_store.rs +++ b/synevi_persistence/src/redb_store.rs @@ -593,7 +593,7 @@ impl InternalData { } fn last_applied(&self) -> (T, T0) { - let t = self.last_applied.clone(); + let t = self.last_applied; let t0 = self.mappings.get(&t).cloned().unwrap_or(T0::default()); (t, t0) } diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index c178b7d..0baf80e 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -105,7 +105,7 @@ impl Transaction for InternalExecution { bytes.push(0); bytes.extend_from_slice(&id.to_bytes()); bytes.extend_from_slice(&serial.to_be_bytes()); - bytes.extend_from_slice(&new_node_host.as_bytes()); + bytes.extend_from_slice(new_node_host.as_bytes()); } InternalExecution::ReadyElectorate { id, serial } => { bytes.push(1); From 6e38a07b649ad16ad60b731fd3e50f9ead6101ce Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 28 Oct 2024 16:48:16 +0100 Subject: [PATCH 34/40] chore: Added retry backoff for add_member --- synevi_network/src/network.rs | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index a2824bc..ce87209 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -255,7 +255,24 @@ impl Network for GrpcNetwork { host: String, ready: bool, ) -> Result<(), SyneviError> { - let channel = Channel::from_shared(host.clone())?.connect().await?; + let endpoint = Channel::from_shared(host.clone())?; + // Retry connecting to member + let mut backoff = 0u64; + let channel = loop { + match endpoint.connect().await { + Ok(channel) => break channel, + Err(e) => { + if backoff < 5 { + backoff += 1; + } else { + tracing::error!("Backoff limit reached, connecting to member"); + return Err(SyneviError::TonicTransportError(e)); + } + tracing::error!("Error connecting to member: {:?}", e); + tokio::time::sleep(tokio::time::Duration::from_secs(backoff)).await; + } + } + }; let mut writer = self.members.write().await; if writer.get(&id).is_none() { writer.insert( From 7fa169b270cc5d6b4598de69c2203b1311248cc2 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Mon, 28 Oct 2024 23:42:20 +0100 Subject: [PATCH 35/40] feat: Added id to execute function, added get_event_by_id function --- synevi_core/src/lib.rs | 2 +- synevi_core/src/node.rs | 12 +++- synevi_kv/src/kv_store.rs | 2 +- synevi_persistence/src/lmdb_store.rs | 88 ++++++++++++++-------------- synevi_persistence/src/mem_store.rs | 25 ++++++-- synevi_persistence/src/redb_store.rs | 87 +++++++++++---------------- synevi_types/src/traits.rs | 11 ++-- 7 files changed, 115 insertions(+), 112 deletions(-) diff --git a/synevi_core/src/lib.rs b/synevi_core/src/lib.rs index b8e22a3..72cc2ee 100644 --- a/synevi_core/src/lib.rs +++ b/synevi_core/src/lib.rs @@ -124,7 +124,7 @@ pub mod tests { impl Executor for DummyExecutor { type Tx = Vec; - async fn execute(&self, data: Vec) -> SyneviResult { + async fn execute(&self, _id: u128, data: Vec) -> SyneviResult { Ok(Ok(data)) } } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index d42436e..f2bcac3 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -12,7 +12,7 @@ use synevi_network::replica::Replica; use synevi_persistence::mem_store::MemStore; use synevi_types::traits::Store; use synevi_types::types::{ - ExecutorResult, Hashes, InternalExecution, InternalSyneviResult, SyneviResult, + Event, ExecutorResult, Hashes, InternalExecution, InternalSyneviResult, SyneviResult, TransactionPayload, UpsertEvent, }; use synevi_types::{Executor, State, SyneviError, Transaction, T}; @@ -208,6 +208,11 @@ where Ok(()) } + #[instrument(level = "trace", skip(self))] + pub fn get_event_by_id(&self, id: u128) -> Option { + self.event_store.get_event_by_id(id).ok().flatten() + } + #[instrument(level = "trace", skip(self, transaction))] pub async fn transaction(self: Arc, id: u128, transaction: E::Tx) -> SyneviResult { if !self.has_members() { @@ -337,7 +342,7 @@ where .clone() .ok_or_else(|| SyneviError::TransactionNotFound)?, )?; - let result = self.execute(transaction).await; + let result = self.execute(event.id, transaction).await; let event_store = self.event_store.clone(); @@ -365,6 +370,7 @@ where async fn execute( &self, + id: u128, transaction: TransactionPayload<::Tx>, ) -> Result::Tx>, SyneviError> { // TODO: Refactor in execute function @@ -373,7 +379,7 @@ where TransactionPayload::None => Err(SyneviError::TransactionNotFound), TransactionPayload::External(tx) => self .executor - .execute(tx) + .execute(id, tx) .await .map(ExecutorResult::<::Tx>::External), TransactionPayload::Internal(request) => { diff --git a/synevi_kv/src/kv_store.rs b/synevi_kv/src/kv_store.rs index 573da55..ca5fbd5 100644 --- a/synevi_kv/src/kv_store.rs +++ b/synevi_kv/src/kv_store.rs @@ -58,7 +58,7 @@ impl synevi_types::Transaction for Transaction { #[async_trait::async_trait] impl Executor for KVExecutor { type Tx = Transaction; - async fn execute(&self, transaction: Self::Tx) -> SyneviResult { + async fn execute(&self, _id: u128, transaction: Self::Tx) -> SyneviResult { Ok(match transaction { Transaction::Read { key } => { let Some(key) = self.store.lock().unwrap().get(&key).cloned() else { diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 176a5a9..5fab42a 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -19,6 +19,7 @@ use tokio::sync::mpsc::Receiver; use tracing::instrument; const EVENT_DB_NAME: &str = "events"; +const ID_MAPPINGS_DB_NAME: &str = "id_mappings"; type EventDb = Database, SerdeBincode>; #[derive(Clone, Debug)] @@ -29,6 +30,8 @@ pub struct LmdbStore { #[derive(Clone, Debug)] struct InternalData { db: Env, + events: EventDb, + id_mappings: Database, U128>, pub(crate) mappings: BTreeMap, // Key: t, value t0 pub last_applied: T, // t of last applied entry pub(crate) latest_time: MonoTime, // last created or recognized t0 @@ -48,6 +51,8 @@ impl LmdbStore { let env_clone = env.clone(); let mut write_txn = env.write_txn()?; let events_db: Option = env.open_database(&write_txn, Some(EVENT_DB_NAME))?; + let id_mappings: Database<_, _> = + env.create_database(&mut write_txn, Some(ID_MAPPINGS_DB_NAME))?; match events_db { Some(db) => { let result = db @@ -84,6 +89,8 @@ impl LmdbStore { //db: env_clone, data: Arc::new(Mutex::new(InternalData { db: env_clone, + events: db, + id_mappings, mappings, last_applied, latest_time, @@ -93,11 +100,13 @@ impl LmdbStore { }) } None => { - let _: EventDb = env.create_database(&mut write_txn, Some(EVENT_DB_NAME))?; + let events: EventDb = env.create_database(&mut write_txn, Some(EVENT_DB_NAME))?; write_txn.commit()?; Ok(LmdbStore { data: Arc::new(Mutex::new(InternalData { db: env_clone, + events, + id_mappings, mappings: BTreeMap::default(), last_applied: T::default(), latest_time: MonoTime::default(), @@ -241,6 +250,15 @@ impl Store for LmdbStore { event.state = State::Applied; Ok(event.hash_event(lock.latest_hash)) } + + fn get_event_by_id(&self, id: u128) -> Result, SyneviError> { + let lock = self.data.lock().expect("poisoned lock, aborting"); + let read_txn = lock.db.read_txn()?; + let Some(mapping) = lock.id_mappings.get(&read_txn, &id)? else { + return Ok(None); + }; + lock.events.get(&read_txn, &mapping).map_err(Into::into) + } } impl InternalData { @@ -317,15 +335,11 @@ impl InternalData { #[instrument(level = "trace")] fn accept_tx_ballot(&self, t_zero: &T0, ballot: Ballot) -> Option { let mut write_txn = self.db.write_txn().ok()?; - let events_db: EventDb = self - .db - .open_database(&write_txn, Some(EVENT_DB_NAME)) - .ok()??; - let mut event = events_db.get(&write_txn, &t_zero.get_inner()).ok()??; + let mut event = self.events.get(&write_txn, &t_zero.get_inner()).ok()??; if event.ballot < ballot { event.ballot = ballot; - let _ = events_db.put(&mut write_txn, &t_zero.get_inner(), &event); + let _ = self.events.put(&mut write_txn, &t_zero.get_inner(), &event); } write_txn.commit().ok()?; //self.db.force_sync().ok()?; @@ -343,15 +357,20 @@ impl InternalData { } let mut write_txn = self.db.write_txn()?; - let events_db: EventDb = self - .db - .open_database(&write_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; + let events_db: EventDb = self.events; + let event = events_db.get(&write_txn, &upsert_event.t_zero.get_inner())?; let Some(mut event) = event else { let mut event = Event::from(upsert_event.clone()); + // Not an update -> Add events id mapping + self.id_mappings.put( + &mut write_txn, + &upsert_event.id, + &upsert_event.t_zero.get_inner(), + )?; + if matches!(event.state, State::Applied) { self.mappings.insert(event.t, event.t_zero); if let Some(deps) = upsert_event.dependencies { @@ -449,11 +468,8 @@ impl InternalData { #[instrument(level = "trace")] fn get_recover_deps(&self, t_zero: &T0) -> Result { let read_txn = self.db.read_txn()?; - let db: EventDb = self - .db - .open_database(&read_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - let timestamp = db + let timestamp = self + .events .get(&read_txn, &t_zero.get_inner())? .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner()))? .t; @@ -463,7 +479,8 @@ impl InternalData { }; for (t_dep, t_zero_dep) in self.mappings.range(self.last_applied..) { - let dep_event = db + let dep_event = self + .events .get(&read_txn, &t_zero_dep.get_inner())? .ok_or_else(|| SyneviError::DependencyNotFound(t_zero_dep.get_inner()))?; match dep_event.state { @@ -508,11 +525,8 @@ impl InternalData { fn get_event_state(&self, t_zero: &T0) -> Option { let read_txn = self.db.read_txn().ok()?; - let db: EventDb = self - .db - .open_database(&read_txn, Some(EVENT_DB_NAME)) - .ok()??; - let state = db + let state = self + .events .get(&read_txn, &t_zero.get_inner()) .ok()? .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner())) @@ -535,15 +549,12 @@ impl InternalData { } let mut write_txn = self.db.write_txn()?; - let db: EventDb = self - .db - .open_database(&write_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - let event = db.get(&write_txn, &t_zero_recover.get_inner())?; + let event = self.events.get(&write_txn, &t_zero_recover.get_inner())?; if let Some(mut event) = event { event.ballot = Ballot(event.ballot.next_with_node(node_serial).into_time()); - db.put(&mut write_txn, &t_zero_recover.get_inner(), &event)?; + self.events + .put(&mut write_txn, &t_zero_recover.get_inner(), &event)?; write_txn.commit()?; Ok(Some(RecoverEvent { @@ -563,13 +574,8 @@ impl InternalData { fn get_event_store(&self) -> BTreeMap { // TODO: Remove unwrap and change trait result let read_txn = self.db.read_txn().unwrap(); - let events_db: Database, SerdeBincode> = self - .db - .open_database(&read_txn, Some(EVENT_DB_NAME)) - .unwrap() - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME)) - .unwrap(); - let result = events_db + let result = self + .events .iter(&read_txn) .unwrap() .filter_map(|e| { @@ -601,13 +607,11 @@ impl InternalData { None if last_applied == T::default() => T0::default(), _ => return Err(SyneviError::EventNotFound(last_applied.get_inner())), }; + + let events_db = self.events; tokio::task::spawn_blocking(move || { let read_txn = db.read_txn()?; let range = last_applied_t0.get_inner()..; - let events_db: EventDb = db - .open_database(&read_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - for result in events_db.range(&read_txn, &range)? { let (_t0, event) = result?; sdx.blocking_send(Ok(event)) @@ -621,11 +625,7 @@ impl InternalData { fn get_event(&self, t_zero: T0) -> Result, SyneviError> { let read_txn = self.db.read_txn()?; - let db: EventDb = self - .db - .open_database(&read_txn, Some(EVENT_DB_NAME))? - .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - let event = db.get(&read_txn, &t_zero.get_inner())?; + let event = self.events.get(&read_txn, &t_zero.get_inner())?; read_txn.commit()?; Ok(event) } diff --git a/synevi_persistence/src/mem_store.rs b/synevi_persistence/src/mem_store.rs index 351118f..619df16 100644 --- a/synevi_persistence/src/mem_store.rs +++ b/synevi_persistence/src/mem_store.rs @@ -1,6 +1,6 @@ use ahash::RandomState; use monotime::MonoTime; -use std::collections::{BTreeMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt::Debug; use std::sync::{Arc, Mutex}; use synevi_types::error::SyneviError; @@ -14,10 +14,11 @@ use tracing::instrument; #[derive(Debug, Clone)] struct InternalStore { - pub events: BTreeMap, // Key: t0, value: Event - pub(crate) mappings: BTreeMap, // Key: t, value t0 - pub last_applied: T, // t of last applied entry - pub(crate) latest_time: MonoTime, // last created or recognized time + pub(crate) id_map: HashMap, // Key: id, value: t0 + pub events: BTreeMap, // Key: t0, value: Event + pub(crate) mappings: BTreeMap, // Key: t, value t0 + pub last_applied: T, // t of last applied entry + pub(crate) latest_time: MonoTime, // last created or recognized time pub node_serial: u16, latest_hash: [u8; 32], } @@ -31,6 +32,7 @@ impl MemStore { #[instrument(level = "trace")] pub fn new(node_serial: u16) -> Result { let store = Arc::new(Mutex::new(InternalStore { + id_map: HashMap::default(), events: BTreeMap::default(), mappings: BTreeMap::default(), last_applied: T::default(), @@ -149,6 +151,15 @@ impl Store for MemStore { .cloned()) } + fn get_event_by_id(&self, id: u128) -> Result, SyneviError> { + let store = self.store.lock().expect("poisoned lock, aborting"); + Ok(store + .id_map + .get(&id) + .and_then(|t0| store.events.get(t0)) + .cloned()) + } + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.store.lock().expect("poisoned lock, aborting"); lock.latest_time = lock @@ -258,6 +269,10 @@ impl InternalStore { let Some(event) = self.events.get_mut(&upsert_event.t_zero) else { let mut event = Event::from(upsert_event.clone()); + + // Not an update -> Add id mapping + self.id_map.insert(event.id, event.t_zero); + if matches!(event.state, State::Applied) { self.mappings.insert(event.t, event.t_zero); if let Some(deps) = upsert_event.dependencies { diff --git a/synevi_persistence/src/redb_store.rs b/synevi_persistence/src/redb_store.rs index bc8d4cb..d4d529e 100644 --- a/synevi_persistence/src/redb_store.rs +++ b/synevi_persistence/src/redb_store.rs @@ -15,6 +15,7 @@ use tokio::sync::mpsc::Receiver; use tracing::instrument; const TABLE: TableDefinition = TableDefinition::new("events"); +const ID_MAPPINGS_DB: TableDefinition = TableDefinition::new("id_mappings"); #[derive(Clone, Debug)] pub struct RedbStore { @@ -37,6 +38,7 @@ impl RedbStore { { let write_txn = db.begin_write().unwrap(); let _ = write_txn.open_table(TABLE).unwrap(); + let _ = write_txn.open_table(ID_MAPPINGS_DB).unwrap(); write_txn.commit().unwrap(); } let read_txn = db.begin_read().unwrap(); @@ -209,6 +211,14 @@ impl Store for RedbStore { .get_event(t_zero) } + #[instrument(level = "trace", skip(self))] + fn get_event_by_id(&self, t_zero: u128) -> Result, SyneviError> { + self.data + .lock() + .expect("poisoned lock, aborting") + .get_event_by_id(t_zero) + } + fn inc_time_with_guard(&self, guard: T0) -> Result<(), SyneviError> { let mut lock = self.data.lock().expect("poisoned lock, aborting"); lock.latest_time = lock @@ -346,6 +356,12 @@ impl InternalData { let Some(mut event) = event else { let mut event = Event::from(upsert_event.clone()); + // Not an update -> Insert mapping + { + let mut table = write_txn.open_table(ID_MAPPINGS_DB).ok().unwrap(); + let _ = table.insert(&upsert_event.id, &upsert_event.t_zero.get_inner()); + } + if matches!(event.state, State::Applied) { self.mappings.insert(event.t, event.t_zero); if let Some(deps) = upsert_event.dependencies { @@ -637,59 +653,24 @@ impl InternalData { Ok(event) } - // fn get_and_update_hash( - // &self, - // t_zero: T0, - // execution_hash: [u8; 32], - // ) -> Result { - // let t_zero = t_zero.get_inner(); - // let mut write_txn = self.db.write_txn()?; - // let db: EventDb = self - // .db - // .open_database(&write_txn, Some(EVENT_DB_NAME))? - // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - // let Some(mut event) = db.get(&write_txn, &t_zero)? else { - // return Err(SyneviError::EventNotFound(t_zero)); - // }; - // let Some(mut hashes) = event.hashes else { - // return Err(SyneviError::MissingTransactionHash); - // }; - // hashes.execution_hash = execution_hash; - // event.hashes = Some(hashes.clone()); - - // db.put(&mut write_txn, &t_zero, &event)?; - // write_txn.commit()?; - // Ok(hashes) - // } - - // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError> { - // let last = self.last_applied; - // let last_t0 = self - // .mappings - // .get(&last) - // .ok_or_else(|| SyneviError::EventNotFound(last.get_inner()))?; - // let read_txn = self.db.read_txn()?; - // let db: EventDb = self - // .db - // .open_database(&read_txn, Some(EVENT_DB_NAME))? - // .ok_or_else(|| SyneviError::DatabaseNotFound(EVENT_DB_NAME))?; - // let event = db - // .get(&read_txn, &last_t0.get_inner())? - // .ok_or_else(|| SyneviError::EventNotFound(last_t0.get_inner()))? - // .hashes - // .ok_or_else(|| SyneviError::MissingExecutionHash)?; - // Ok((last, event.execution_hash)) - // } -} - -#[cfg(test)] -mod tests { + fn get_event_by_id(&self, t_zero: u128) -> Result, SyneviError> { + let read_txn = self.db.begin_read().ok().unwrap(); + let event = { + let mapping_table = read_txn.open_table(ID_MAPPINGS_DB).ok().unwrap(); + let t_zero = mapping_table + .get(&t_zero) + .ok() + .unwrap() + .map(|e| e.value()) + .ok_or_else(|| SyneviError::EventNotFound(t_zero))?; - #[test] - fn test_db() { - // TODO - //let db = Database::new("../../tests/database".to_string()).unwrap(); - //db.init(Bytes::from("key"), Bytes::from("value")) - // .unwrap() + let table = read_txn.open_table(TABLE).ok().unwrap(); + table + .get(&t_zero) + .ok() + .unwrap() + .map(|e| bincode::deserialize::(e.value()).ok().unwrap()) + }; + Ok(event) } } diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index a9bf2c7..51564e0 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -36,7 +36,7 @@ impl Transaction for Vec { pub trait Executor: Send + Sync + 'static { type Tx: Transaction + Serialize; // Executor expects a type with interior mutability - async fn execute(&self, transaction: Self::Tx) -> SyneviResult; + async fn execute(&self, id: u128, transaction: Self::Tx) -> SyneviResult; } #[async_trait::async_trait] @@ -45,8 +45,8 @@ where E: Executor, { type Tx = E::Tx; - async fn execute(&self, transaction: Self::Tx) -> SyneviResult { - self.as_ref().execute(transaction).await + async fn execute(&self, id: u128, transaction: Self::Tx) -> SyneviResult { + self.as_ref().execute(id, transaction).await } } @@ -57,11 +57,11 @@ where { type Tx = E::Tx; - async fn execute(&self, transaction: Self::Tx) -> SyneviResult { + async fn execute(&self, id: u128, transaction: Self::Tx) -> SyneviResult { self.upgrade() .ok_or_else(|| SyneviError::ArcDropped)? .as_ref() - .execute(transaction) + .execute(id, transaction) .await } } @@ -102,6 +102,7 @@ pub trait Store: Send + Sync + Sized + 'static { // fn last_applied_hash(&self) -> Result<(T, [u8; 32]), SyneviError>; fn get_event(&self, t_zero: T0) -> Result, SyneviError>; + fn get_event_by_id(&self, id: u128) -> Result, SyneviError>; fn get_events_after( &self, last_applied: T, From 1a7a09ed529ff6eb6567326bda1f90cffe36adf8 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Wed, 30 Oct 2024 15:05:11 +0100 Subject: [PATCH 36/40] chore: Re-export lmdb store in synevi --- synevi/src/lib.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/synevi/src/lib.rs b/synevi/src/lib.rs index 3d0e280..b233fe4 100644 --- a/synevi/src/lib.rs +++ b/synevi/src/lib.rs @@ -16,3 +16,8 @@ pub mod network { pub use synevi_network::network::BroadcastResponse; } } + +pub mod storage { + pub use synevi_persistence::lmdb_store::LmdbStore; + pub use synevi_persistence::mem_store::MemStore; +} From 1a46b330cdb1c2129e76f0a5659e0c02997181b7 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Mon, 18 Nov 2024 10:37:14 +0100 Subject: [PATCH 37/40] fix: Fixed add_members to ignore self config --- synevi_network/src/network.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/synevi_network/src/network.rs b/synevi_network/src/network.rs index ce87209..dd1fdfd 100644 --- a/synevi_network/src/network.rs +++ b/synevi_network/src/network.rs @@ -255,6 +255,9 @@ impl Network for GrpcNetwork { host: String, ready: bool, ) -> Result<(), SyneviError> { + if self.self_status.info.id == id { + return Ok(()); + } let endpoint = Channel::from_shared(host.clone())?; // Retry connecting to member let mut backoff = 0u64; From 35078fea9b24427feae74c1f9d8758d027aa3701 Mon Sep 17 00:00:00 2001 From: Sebastian Beyvers Date: Wed, 27 Nov 2024 18:00:21 +0100 Subject: [PATCH 38/40] feat: Added decoding function for events --- synevi_core/Cargo.toml | 1 + synevi_core/src/node.rs | 9 +++++++++ synevi_persistence/src/lmdb_store.rs | 6 +----- synevi_types/src/traits.rs | 4 ++-- synevi_types/src/types.rs | 4 ++-- 5 files changed, 15 insertions(+), 9 deletions(-) diff --git a/synevi_core/Cargo.toml b/synevi_core/Cargo.toml index 8367b62..548b2cb 100644 --- a/synevi_core/Cargo.toml +++ b/synevi_core/Cargo.toml @@ -30,6 +30,7 @@ async-channel = "2.3.1" ahash = { workspace = true } sha3 = "0.10.8" postcard = {version = "1.0.10", features = ["use-std"]} +bincode = "1.3.3" [dev-dependencies] monotime = { path = "../monotime", features = ["unsafe_time"] } diff --git a/synevi_core/src/node.rs b/synevi_core/src/node.rs index f2bcac3..fe4a509 100644 --- a/synevi_core/src/node.rs +++ b/synevi_core/src/node.rs @@ -213,6 +213,15 @@ where self.event_store.get_event_by_id(id).ok().flatten() } + #[instrument(level = "trace", skip(self))] + pub fn decode_event(&self, event: Event) -> Result, SyneviError> { + let transaction = TransactionPayload::::from_bytes(event.transaction)?; + match transaction { + TransactionPayload::External(tx) => Ok(Some(tx)), + _ => Ok(None), + } + } + #[instrument(level = "trace", skip(self, transaction))] pub async fn transaction(self: Arc, id: u128, transaction: E::Tx) -> SyneviResult { if !self.has_members() { diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 5fab42a..3033e6d 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -519,7 +519,6 @@ impl InternalData { recover_deps.dependencies.insert(*t_zero_dep); } } - read_txn.commit()?; Ok(recover_deps) } @@ -532,7 +531,6 @@ impl InternalData { .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner())) .ok()? .state; - read_txn.commit().ok()?; Some(state) } @@ -567,6 +565,7 @@ impl InternalData { ballot: event.ballot, })) } else { + write_txn.commit()?; Ok(None) } } @@ -586,7 +585,6 @@ impl InternalData { } }) .collect::>(); - read_txn.commit().unwrap(); result } @@ -617,7 +615,6 @@ impl InternalData { sdx.blocking_send(Ok(event)) .map_err(|e| SyneviError::SendError(e.to_string()))?; } - read_txn.commit()?; Ok::<(), SyneviError>(()) }); Ok(rcv) @@ -626,7 +623,6 @@ impl InternalData { fn get_event(&self, t_zero: T0) -> Result, SyneviError> { let read_txn = self.db.read_txn()?; let event = self.events.get(&read_txn, &t_zero.get_inner())?; - read_txn.commit()?; Ok(event) } } diff --git a/synevi_types/src/traits.rs b/synevi_types/src/traits.rs index 51564e0..53aed86 100644 --- a/synevi_types/src/traits.rs +++ b/synevi_types/src/traits.rs @@ -1,5 +1,5 @@ use ahash::RandomState; -use serde::Serialize; +use serde::{de::DeserializeOwned, Deserialize, Serialize}; use std::{ collections::{BTreeMap, HashSet}, sync::{Arc, Weak}, @@ -34,7 +34,7 @@ impl Transaction for Vec { #[async_trait::async_trait] pub trait Executor: Send + Sync + 'static { - type Tx: Transaction + Serialize; + type Tx: Transaction + Serialize + DeserializeOwned; // Executor expects a type with interior mutability async fn execute(&self, id: u128, transaction: Self::Tx) -> SyneviResult; } diff --git a/synevi_types/src/types.rs b/synevi_types/src/types.rs index 0baf80e..22f88aa 100644 --- a/synevi_types/src/types.rs +++ b/synevi_types/src/types.rs @@ -31,7 +31,7 @@ pub struct Waiter { pub sender: Vec>, } -#[derive(Default, PartialEq, PartialOrd, Ord, Eq, Clone, Debug, Serialize)] +#[derive(Default, PartialEq, PartialOrd, Ord, Eq, Clone, Debug, Serialize, Deserialize)] pub enum TransactionPayload { #[default] None, @@ -39,7 +39,7 @@ pub enum TransactionPayload { Internal(InternalExecution), } -#[derive(Debug, Clone, Serialize, Eq, PartialEq, PartialOrd, Ord)] +#[derive(Debug, Clone, Serialize, Eq, PartialEq, PartialOrd, Ord, Deserialize)] pub enum InternalExecution { JoinElectorate { id: Ulid, From 85e6caf57538d51be7b4b9f970109e4733617f72 Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Thu, 6 Feb 2025 15:35:20 +0100 Subject: [PATCH 39/40] feat: Higher map size for lmdb --- synevi_persistence/src/lmdb_store.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index 3033e6d..fe8644c 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -43,7 +43,7 @@ impl LmdbStore { pub fn new(path: String, node_serial: u16) -> Result { let env = unsafe { EnvOpenOptions::new() - .map_size(1024 * 1024 * 1024) + .map_size(10 * 1024 * 1024 * 1024) .max_dbs(16) .flags(EnvFlags::MAP_ASYNC | EnvFlags::WRITE_MAP) .open(path)? From c22be6eec2ad52b1924a2feae80cee146def11fa Mon Sep 17 00:00:00 2001 From: lfbrehm <97600985+lfbrehm@users.noreply.github.com> Date: Tue, 11 Feb 2025 14:28:16 +0100 Subject: [PATCH 40/40] fix: Added missing read commits --- synevi_persistence/src/lmdb_store.rs | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/synevi_persistence/src/lmdb_store.rs b/synevi_persistence/src/lmdb_store.rs index fe8644c..4e898a1 100644 --- a/synevi_persistence/src/lmdb_store.rs +++ b/synevi_persistence/src/lmdb_store.rs @@ -257,7 +257,9 @@ impl Store for LmdbStore { let Some(mapping) = lock.id_mappings.get(&read_txn, &id)? else { return Ok(None); }; - lock.events.get(&read_txn, &mapping).map_err(Into::into) + let result = lock.events.get(&read_txn, &mapping).map_err(Into::into); + read_txn.commit()?; + result } } @@ -519,6 +521,8 @@ impl InternalData { recover_deps.dependencies.insert(*t_zero_dep); } } + + read_txn.commit()?; Ok(recover_deps) } @@ -531,6 +535,8 @@ impl InternalData { .ok_or_else(|| SyneviError::EventNotFound(t_zero.get_inner())) .ok()? .state; + + read_txn.commit().ok()?; Some(state) } @@ -585,6 +591,8 @@ impl InternalData { } }) .collect::>(); + + read_txn.commit().unwrap(); result } @@ -615,6 +623,8 @@ impl InternalData { sdx.blocking_send(Ok(event)) .map_err(|e| SyneviError::SendError(e.to_string()))?; } + + read_txn.commit()?; Ok::<(), SyneviError>(()) }); Ok(rcv) @@ -623,6 +633,8 @@ impl InternalData { fn get_event(&self, t_zero: T0) -> Result, SyneviError> { let read_txn = self.db.read_txn()?; let event = self.events.get(&read_txn, &t_zero.get_inner())?; + + read_txn.commit()?; Ok(event) } }