diff --git a/node/Cargo.lock b/node/Cargo.lock index e0ca823a..2484f909 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -2971,6 +2971,7 @@ name = "zksync_consensus_network" version = "0.1.0" dependencies = [ "anyhow", + "assert_matches", "async-trait", "im", "once_cell", @@ -2998,6 +2999,7 @@ name = "zksync_consensus_roles" version = "0.1.0" dependencies = [ "anyhow", + "assert_matches", "bit-vec", "hex", "prost", diff --git a/node/Cargo.toml b/node/Cargo.toml index df0a7b33..9c147dc8 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -155,3 +155,5 @@ needless_pass_by_ref_mut = "allow" box_default = "allow" # remove once fix to https://github.com/rust-lang/rust-clippy/issues/11764 is available on CI. map_identity = "allow" +# &*x is not equivalent to x, because it affects borrowing in closures. +borrow_deref_ref = "allow" diff --git a/node/actors/bft/src/config.rs b/node/actors/bft/src/config.rs index c1a94b74..77938549 100644 --- a/node/actors/bft/src/config.rs +++ b/node/actors/bft/src/config.rs @@ -1,7 +1,6 @@ //! The inner data of the consensus state machine. This is shared between the different roles. -use crate::{misc, PayloadManager}; +use crate::PayloadManager; use std::sync::Arc; -use tracing::instrument; use zksync_consensus_roles::validator; use zksync_consensus_storage as storage; @@ -10,8 +9,6 @@ use zksync_consensus_storage as storage; pub struct Config { /// The validator's secret key. pub secret_key: validator::SecretKey, - /// A vector of public keys for all the validators in the network. - pub validator_set: validator::ValidatorSet, /// The maximum size of the payload of a block, in bytes. We will /// reject blocks with payloads larger than this. pub max_payload_size: usize, @@ -24,23 +21,8 @@ pub struct Config { } impl Config { - /// Computes the validator for the given view. - #[instrument(level = "trace", ret)] - pub fn view_leader(&self, view_number: validator::ViewNumber) -> validator::PublicKey { - let index = view_number.0 as usize % self.validator_set.len(); - self.validator_set.get(index).unwrap().clone() - } - - /// Calculate the consensus threshold, the minimum number of votes for any consensus action to be valid, - /// for a given number of replicas. - #[instrument(level = "trace", ret)] - pub fn threshold(&self) -> usize { - misc::consensus_threshold(self.validator_set.len()) - } - - /// Calculate the maximum number of faulty replicas, for a given number of replicas. - #[instrument(level = "trace", ret)] - pub fn faulty_replicas(&self) -> usize { - misc::faulty_replicas(self.validator_set.len()) + /// Genesis. + pub fn genesis(&self) -> &validator::Genesis { + self.block_store.genesis() } } diff --git a/node/actors/bft/src/leader/mod.rs b/node/actors/bft/src/leader/mod.rs index 6cff3091..f4615904 100644 --- a/node/actors/bft/src/leader/mod.rs +++ b/node/actors/bft/src/leader/mod.rs @@ -2,14 +2,10 @@ //! and aggregates replica messages. It mainly acts as a central point of communication for the replicas. Note that //! our consensus node will perform both the replica and leader roles simultaneously. -mod replica_commit; -mod replica_prepare; +pub(crate) mod replica_commit; +pub(crate) mod replica_prepare; mod state_machine; #[cfg(test)] mod tests; -#[cfg(test)] -pub(crate) use self::replica_commit::Error as ReplicaCommitError; -#[cfg(test)] -pub(crate) use self::replica_prepare::Error as ReplicaPrepareError; pub(crate) use self::state_machine::StateMachine; diff --git a/node/actors/bft/src/leader/replica_commit.rs b/node/actors/bft/src/leader/replica_commit.rs index f0d31d4a..2a794bd8 100644 --- a/node/actors/bft/src/leader/replica_commit.rs +++ b/node/actors/bft/src/leader/replica_commit.rs @@ -1,3 +1,4 @@ +//! Handler of a ReplicaCommit message. use super::StateMachine; use crate::metrics; use std::collections::HashMap; @@ -34,6 +35,9 @@ pub(crate) enum Error { /// The processing node is not a lead for this message's view. #[error("we are not a leader for this message's view")] NotLeaderInView, + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(anyhow::Error), /// Duplicate message from a replica. #[error("duplicate message from a replica (existing message: {existing_message:?}")] DuplicateMessage { @@ -55,28 +59,26 @@ impl StateMachine { // ----------- Checking origin of the message -------------- // Unwrap message. - let message = signed_message.msg; + let message = &signed_message.msg; let author = &signed_message.key; // Check protocol version compatibility. - if !crate::PROTOCOL_VERSION.compatible(&message.protocol_version) { + if !crate::PROTOCOL_VERSION.compatible(&message.view.protocol_version) { return Err(Error::IncompatibleProtocolVersion { - message_version: message.protocol_version, + message_version: message.view.protocol_version, local_version: crate::PROTOCOL_VERSION, }); } // Check that the message signer is in the validator set. - let validator_index = - self.config - .validator_set - .index(author) - .ok_or(Error::NonValidatorSigner { - signer: author.clone(), - })?; + if !self.config.genesis().validators.contains(author) { + return Err(Error::NonValidatorSigner { + signer: author.clone(), + }); + } // If the message is from the "past", we discard it. - if (message.view, validator::Phase::Commit) < (self.view, self.phase) { + if (message.view.number, validator::Phase::Commit) < (self.view, self.phase) { return Err(Error::Old { current_view: self.view, current_phase: self.phase, @@ -84,18 +86,24 @@ impl StateMachine { } // If the message is for a view when we are not a leader, we discard it. - if self.config.view_leader(message.view) != self.config.secret_key.public() { + if self + .config + .genesis() + .validators + .view_leader(message.view.number) + != self.config.secret_key.public() + { return Err(Error::NotLeaderInView); } // If we already have a message from the same validator and for the same view, we discard it. if let Some(existing_message) = self .commit_message_cache - .get(&message.view) + .get(&message.view.number) .and_then(|x| x.get(author)) { return Err(Error::DuplicateMessage { - existing_message: existing_message.msg, + existing_message: existing_message.msg.clone(), }); } @@ -104,30 +112,40 @@ impl StateMachine { // Check the signature on the message. signed_message.verify().map_err(Error::InvalidSignature)?; + message + .verify(self.config.genesis()) + .map_err(Error::InvalidMessage)?; + // ----------- All checks finished. Now we process the message. -------------- + // TODO: we have a bug here since we don't check whether replicas commit + // to the same proposal. + // We add the message to the incrementally-constructed QC. self.commit_qcs - .entry(message.view) - .or_insert(CommitQC::new(message, &self.config.validator_set)) - .add(&signed_message.sig, validator_index); + .entry(message.view.number) + .or_insert_with(|| CommitQC::new(message.clone(), self.config.genesis())) + .add(&signed_message, self.config.genesis()); // We store the message in our cache. - let cache_entry = self.commit_message_cache.entry(message.view).or_default(); - cache_entry.insert(author.clone(), signed_message); + let cache_entry = self + .commit_message_cache + .entry(message.view.number) + .or_default(); + cache_entry.insert(author.clone(), signed_message.clone()); // Now we check if we have enough messages to continue. let mut by_proposal: HashMap<_, Vec<_>> = HashMap::new(); for msg in cache_entry.values() { by_proposal.entry(msg.msg.proposal).or_default().push(msg); } - let Some((_, replica_messages)) = by_proposal - .into_iter() - .find(|(_, v)| v.len() >= self.config.threshold()) + let threshold = self.config.genesis().validators.threshold(); + let Some((_, replica_messages)) = + by_proposal.into_iter().find(|(_, v)| v.len() >= threshold) else { return Ok(()); }; - debug_assert_eq!(replica_messages.len(), self.config.threshold()); + debug_assert_eq!(replica_messages.len(), threshold); // ----------- Update the state machine -------------- @@ -135,7 +153,7 @@ impl StateMachine { metrics::METRICS .leader_commit_phase_latency .observe_latency(now - self.phase_start); - self.view = message.view.next(); + self.view = message.view.number.next(); self.phase = validator::Phase::Prepare; self.phase_start = now; @@ -143,10 +161,10 @@ impl StateMachine { // Remove replica commit messages for this view, so that we don't create a new leader commit // for this same view if we receive another replica commit message after this. - self.commit_message_cache.remove(&message.view); + self.commit_message_cache.remove(&message.view.number); // Consume the incrementally-constructed QC for this view. - let justification = self.commit_qcs.remove(&message.view).unwrap(); + let justification = self.commit_qcs.remove(&message.view.number).unwrap(); // Broadcast the leader commit message to all replicas (ourselves included). let output_message = ConsensusInputMessage { @@ -154,10 +172,7 @@ impl StateMachine { .config .secret_key .sign_msg(validator::ConsensusMsg::LeaderCommit( - validator::LeaderCommit { - protocol_version: crate::PROTOCOL_VERSION, - justification, - }, + validator::LeaderCommit { justification }, )), recipient: Target::Broadcast, }; diff --git a/node/actors/bft/src/leader/replica_prepare.rs b/node/actors/bft/src/leader/replica_prepare.rs index b83798aa..4694f682 100644 --- a/node/actors/bft/src/leader/replica_prepare.rs +++ b/node/actors/bft/src/leader/replica_prepare.rs @@ -1,3 +1,4 @@ +//! Handler of a ReplicaPrepare message. use super::StateMachine; use tracing::instrument; use zksync_concurrency::{ctx, error::Wrap}; @@ -37,22 +38,12 @@ pub(crate) enum Error { /// Existing message from the same replica. existing_message: validator::ReplicaPrepare, }, - /// High QC of a future view. - #[error( - "high QC of a future view (high QC view: {high_qc_view:?}, current view: {current_view:?}" - )] - HighQCOfFutureView { - /// Received high QC view. - high_qc_view: validator::ViewNumber, - /// Current view. - current_view: validator::ViewNumber, - }, /// Invalid message signature. #[error("invalid signature: {0:#}")] InvalidSignature(#[source] validator::Error), - /// Invalid `HighQC` message. - #[error("invalid high QC: {0:#}")] - InvalidHighQC(#[source] anyhow::Error), + /// Invalid message. + #[error(transparent)] + InvalidMessage(validator::ReplicaPrepareVerifyError), /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. #[error(transparent)] Internal(#[from] ctx::Error), @@ -84,24 +75,22 @@ impl StateMachine { let author = &signed_message.key; // Check protocol version compatibility. - if !crate::PROTOCOL_VERSION.compatible(&message.protocol_version) { + if !crate::PROTOCOL_VERSION.compatible(&message.view.protocol_version) { return Err(Error::IncompatibleProtocolVersion { - message_version: message.protocol_version, + message_version: message.view.protocol_version, local_version: crate::PROTOCOL_VERSION, }); } // Check that the message signer is in the validator set. - let validator_index = - self.config - .validator_set - .index(author) - .ok_or(Error::NonValidatorSigner { - signer: author.clone(), - })?; + if !self.config.genesis().validators.contains(author) { + return Err(Error::NonValidatorSigner { + signer: author.clone(), + }); + } // If the message is from the "past", we discard it. - if (message.view, validator::Phase::Prepare) < (self.view, self.phase) { + if (message.view.number, validator::Phase::Prepare) < (self.view, self.phase) { return Err(Error::Old { current_view: self.view, current_phase: self.phase, @@ -109,14 +98,20 @@ impl StateMachine { } // If the message is for a view when we are not a leader, we discard it. - if self.config.view_leader(message.view) != self.config.secret_key.public() { + if self + .config + .genesis() + .validators + .view_leader(message.view.number) + != self.config.secret_key.public() + { return Err(Error::NotLeaderInView); } // If we already have a message from the same validator and for the same view, we discard it. if let Some(existing_message) = self .prepare_message_cache - .get(&message.view) + .get(&message.view.number) .and_then(|x| x.get(author)) { return Err(Error::Exists { @@ -129,60 +124,50 @@ impl StateMachine { // Check the signature on the message. signed_message.verify().map_err(Error::InvalidSignature)?; - // ----------- Checking the contents of the message -------------- - - // Verify the high QC. + // Verify the message. message - .high_qc - .verify(&self.config.validator_set, self.config.threshold()) - .map_err(Error::InvalidHighQC)?; - - // If the high QC is for a future view, we discard the message. - // This check is not necessary for correctness, but it's useful to - // guarantee that our proposals don't contain QCs from the future. - if message.high_qc.message.view >= message.view { - return Err(Error::HighQCOfFutureView { - high_qc_view: message.high_qc.message.view, - current_view: message.view, - }); - } + .verify(self.config.genesis()) + .map_err(Error::InvalidMessage)?; // ----------- All checks finished. Now we process the message. -------------- // We add the message to the incrementally-constructed QC. - self.prepare_qcs.entry(message.view).or_default().add( - &signed_message, - validator_index, - &self.config.validator_set, - ); + self.prepare_qcs + .entry(message.view.number) + .or_insert_with(|| validator::PrepareQC::new(message.view.clone())) + .add(&signed_message, self.config.genesis()); // We store the message in our cache. self.prepare_message_cache - .entry(message.view) + .entry(message.view.number) .or_default() .insert(author.clone(), signed_message); // Now we check if we have enough messages to continue. - let num_messages = self.prepare_message_cache.get(&message.view).unwrap().len(); + let num_messages = self + .prepare_message_cache + .get(&message.view.number) + .unwrap() + .len(); - if num_messages < self.config.threshold() { + if num_messages < self.config.genesis().validators.threshold() { return Ok(()); } // Remove replica prepare messages for this view, so that we don't create a new block proposal // for this same view if we receive another replica prepare message after this. - self.prepare_message_cache.remove(&message.view); + self.prepare_message_cache.remove(&message.view.number); - debug_assert_eq!(num_messages, self.config.threshold()); + debug_assert_eq!(num_messages, self.config.genesis().validators.threshold()); // ----------- Update the state machine -------------- - self.view = message.view; + self.view = message.view.number; self.phase = validator::Phase::Commit; self.phase_start = ctx.now(); // Consume the incrementally-constructed QC for this view. - let justification = self.prepare_qcs.remove(&message.view).unwrap(); + let justification = self.prepare_qcs.remove(&message.view.number).unwrap(); self.prepare_qc.send_replace(Some(justification)); Ok(()) diff --git a/node/actors/bft/src/leader/state_machine.rs b/node/actors/bft/src/leader/state_machine.rs index ad08d118..459b1e15 100644 --- a/node/actors/bft/src/leader/state_machine.rs +++ b/node/actors/bft/src/leader/state_machine.rs @@ -135,10 +135,10 @@ impl StateMachine { let Some(prepare_qc) = sync::changed(ctx, &mut prepare_qc).await?.clone() else { continue; }; - if prepare_qc.view() < next_view { + if prepare_qc.view.number < next_view { continue; }; - next_view = prepare_qc.view().next(); + next_view = prepare_qc.view.number.next(); Self::propose(ctx, config, prepare_qc, pipe).await?; } } @@ -151,45 +151,31 @@ impl StateMachine { justification: validator::PrepareQC, pipe: &OutputSender, ) -> ctx::Result<()> { - // Get the highest block voted for and check if there's a quorum of votes for it. To have a quorum - // in this situation, we require 2*f+1 votes, where f is the maximum number of faulty replicas. - let mut count: HashMap<_, usize> = HashMap::new(); - for (vote, signers) in &justification.map { - *count.entry(vote.high_vote.proposal).or_default() += signers.len(); - } - - let highest_vote: Option = count - .iter() - // We only take one value from the iterator because there can only be at most one block with a quorum of 2f+1 votes. - .find_map(|(h, v)| (*v > 2 * cfg.faulty_replicas()).then_some(h)) - .cloned(); - - // Get the highest validator::CommitQC. - let highest_qc: &validator::CommitQC = justification - .map - .keys() - .map(|s| &s.high_qc) - .max_by_key(|qc| qc.message.view) - .unwrap(); + let high_vote = justification.high_vote(cfg.genesis()); + let high_qc = justification.high_qc(); // Create the block proposal to send to the replicas, // and the commit vote to store in our block proposal cache. - let (proposal, payload) = match highest_vote { + let (proposal, payload) = match high_vote { // The previous block was not finalized, so we need to propose it again. // For this we only need the header, since we are guaranteed that at least // f+1 honest replicas have the block and can broadcast it when finalized // (2f+1 have stated that they voted for the block, at most f are malicious). - Some(proposal) if proposal != highest_qc.message.proposal => (proposal, None), + Some(proposal) if Some(&proposal) != high_qc.map(|qc| &qc.message.proposal) => { + (proposal, None) + } // The previous block was finalized, so we can propose a new block. _ => { + let fork = &cfg.genesis().fork; + let (parent, number) = match high_qc { + Some(qc) => (Some(qc.header().hash()), qc.header().number.next()), + None => (fork.first_parent, fork.first_block), + }; // Defensively assume that PayloadManager cannot propose until the previous block is stored. - cfg.block_store - .wait_until_persisted(ctx, highest_qc.header().number) - .await?; - let payload = cfg - .payload_manager - .propose(ctx, highest_qc.header().number.next()) - .await?; + if let Some(prev) = number.prev() { + cfg.block_store.wait_until_persisted(ctx, prev).await?; + } + let payload = cfg.payload_manager.propose(ctx, number).await?; if payload.0.len() > cfg.max_payload_size { return Err(anyhow::format_err!( "proposed payload too large: got {}B, max {}B", @@ -201,8 +187,11 @@ impl StateMachine { metrics::METRICS .leader_proposal_payload_size .observe(payload.0.len()); - let proposal = - validator::BlockHeader::new(&highest_qc.message.proposal, payload.hash()); + let proposal = validator::BlockHeader { + number, + parent, + payload: payload.hash(), + }; (proposal, Some(payload)) } }; @@ -214,8 +203,6 @@ impl StateMachine { .secret_key .sign_msg(validator::ConsensusMsg::LeaderPrepare( validator::LeaderPrepare { - protocol_version: crate::PROTOCOL_VERSION, - view: justification.view(), proposal, proposal_payload: payload, justification, diff --git a/node/actors/bft/src/leader/tests.rs b/node/actors/bft/src/leader/tests.rs index 41f1ee6f..b74e4411 100644 --- a/node/actors/bft/src/leader/tests.rs +++ b/node/actors/bft/src/leader/tests.rs @@ -1,12 +1,10 @@ -use super::{ - replica_commit::Error as ReplicaCommitError, replica_prepare::Error as ReplicaPrepareError, -}; +use super::*; use crate::testonly::ut_harness::UTHarness; use assert_matches::assert_matches; use pretty_assertions::assert_eq; use rand::Rng; use zksync_concurrency::{ctx, scope}; -use zksync_consensus_roles::validator::{self, LeaderCommit, Phase, ViewNumber}; +use zksync_consensus_roles::validator::{self, Phase, ViewNumber}; #[tokio::test] async fn replica_prepare_sanity() { @@ -15,7 +13,7 @@ async fn replica_prepare_sanity() { scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - + tracing::info!("started"); util.new_leader_prepare(ctx).await; Ok(()) }) @@ -31,24 +29,24 @@ async fn replica_prepare_sanity_yield_leader_prepare() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); + util.produce_block(ctx).await; + let replica_prepare = util.new_replica_prepare(); let leader_prepare = util - .process_replica_prepare(ctx, replica_prepare.clone()) + .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) .await .unwrap() .unwrap(); - assert_eq!( - leader_prepare.msg.protocol_version, - replica_prepare.msg.protocol_version - ); - assert_eq!(leader_prepare.msg.view, replica_prepare.msg.view); + assert_eq!(leader_prepare.msg.view(), &replica_prepare.view); assert_eq!( leader_prepare.msg.proposal.parent, - replica_prepare.msg.high_vote.proposal.hash() + replica_prepare + .high_vote + .as_ref() + .map(|v| v.proposal.hash()), ); assert_eq!( leader_prepare.msg.justification, - util.new_prepare_qc(|msg| *msg = replica_prepare.msg) + util.new_prepare_qc(|msg| *msg = replica_prepare) ); Ok(()) }) @@ -66,19 +64,15 @@ async fn replica_prepare_sanity_yield_leader_prepare_reproposal() { util.new_replica_commit(ctx).await; util.process_replica_timeout(ctx).await; - let replica_prepare = util.new_replica_prepare(|_| {}).msg; + let replica_prepare = util.new_replica_prepare(); let leader_prepare = util .process_replica_prepare_all(ctx, replica_prepare.clone()) .await; + assert_eq!(leader_prepare.msg.view(), &replica_prepare.view); assert_eq!( - leader_prepare.msg.protocol_version, - replica_prepare.protocol_version - ); - assert_eq!(leader_prepare.msg.view, replica_prepare.view); - assert_eq!( - leader_prepare.msg.proposal, - replica_prepare.high_vote.proposal + Some(leader_prepare.msg.proposal), + replica_prepare.high_vote.as_ref().map(|v| v.proposal), ); assert_eq!(leader_prepare.msg.proposal_payload, None); let map = leader_prepare.msg.justification.map; @@ -99,13 +93,12 @@ async fn replica_prepare_incompatible_protocol_version() { s.spawn_bg(runner.run(ctx)); let incompatible_protocol_version = util.incompatible_protocol_version(); - let replica_prepare = util.new_replica_prepare(|msg| { - msg.protocol_version = incompatible_protocol_version; - }); - let res = util.process_replica_prepare(ctx, replica_prepare).await; + let mut replica_prepare = util.new_replica_prepare(); + replica_prepare.view.protocol_version = incompatible_protocol_version; + let res = util.process_replica_prepare(ctx, util.sign(replica_prepare)).await; assert_matches!( res, - Err(ReplicaPrepareError::IncompatibleProtocolVersion { message_version, local_version }) => { + Err(replica_prepare::Error::IncompatibleProtocolVersion { message_version, local_version }) => { assert_eq!(message_version, incompatible_protocol_version); assert_eq!(local_version, util.protocol_version()); } @@ -122,14 +115,14 @@ async fn replica_prepare_non_validator_signer() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}).msg; + let replica_prepare = util.new_replica_prepare(); let non_validator_key: validator::SecretKey = ctx.rng().gen(); let res = util .process_replica_prepare(ctx, non_validator_key.sign_msg(replica_prepare)) .await; assert_matches!( res, - Err(ReplicaPrepareError::NonValidatorSigner { signer }) => { + Err(replica_prepare::Error::NonValidatorSigner { signer }) => { assert_eq!(signer, non_validator_key.public()); } ); @@ -147,13 +140,15 @@ async fn replica_prepare_old_view() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); util.leader.view = util.replica.view.next(); util.leader.phase = Phase::Prepare; - let res = util.process_replica_prepare(ctx, replica_prepare).await; + let res = util + .process_replica_prepare(ctx, util.sign(replica_prepare)) + .await; assert_matches!( res, - Err(ReplicaPrepareError::Old { + Err(replica_prepare::Error::Old { current_view: ViewNumber(2), current_phase: Phase::Prepare, }) @@ -172,13 +167,15 @@ async fn replica_prepare_during_commit() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); util.leader.view = util.replica.view; util.leader.phase = Phase::Commit; - let res = util.process_replica_prepare(ctx, replica_prepare).await; + let res = util + .process_replica_prepare(ctx, util.sign(replica_prepare)) + .await; assert_matches!( res, - Err(ReplicaPrepareError::Old { + Err(replica_prepare::Error::Old { current_view, current_phase: Phase::Commit, }) => { @@ -199,12 +196,12 @@ async fn replica_prepare_not_leader_in_view() { let (mut util, runner) = UTHarness::new(ctx, 2).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|msg| { - // Moving to the next view changes the leader. - msg.view = msg.view.next(); - }); - let res = util.process_replica_prepare(ctx, replica_prepare).await; - assert_matches!(res, Err(ReplicaPrepareError::NotLeaderInView)); + let mut replica_prepare = util.new_replica_prepare(); + replica_prepare.view.number = replica_prepare.view.number.next(); + let res = util + .process_replica_prepare(ctx, util.sign(replica_prepare)) + .await; + assert_matches!(res, Err(replica_prepare::Error::NotLeaderInView)); Ok(()) }) .await @@ -220,7 +217,8 @@ async fn replica_prepare_already_exists() { s.spawn_bg(runner.run(ctx)); util.set_owner_as_view_leader(); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); + let replica_prepare = util.sign(replica_prepare.clone()); assert!(util .process_replica_prepare(ctx, replica_prepare.clone()) .await @@ -231,7 +229,7 @@ async fn replica_prepare_already_exists() { .await; assert_matches!( res, - Err(ReplicaPrepareError::Exists { existing_message }) => { + Err(replica_prepare::Error::Exists { existing_message }) => { assert_eq!(existing_message, replica_prepare.msg); } ); @@ -250,9 +248,9 @@ async fn replica_prepare_num_received_below_threshold() { s.spawn_bg(runner.run(ctx)); util.set_owner_as_view_leader(); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); assert!(util - .process_replica_prepare(ctx, replica_prepare) + .process_replica_prepare(ctx, util.sign(replica_prepare)) .await .unwrap() .is_none()); @@ -270,10 +268,11 @@ async fn replica_prepare_invalid_sig() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut replica_prepare = util.new_replica_prepare(|_| {}); + let msg = util.new_replica_prepare(); + let mut replica_prepare = util.sign(msg); replica_prepare.sig = ctx.rng().gen(); let res = util.process_replica_prepare(ctx, replica_prepare).await; - assert_matches!(res, Err(ReplicaPrepareError::InvalidSignature(_))); + assert_matches!(res, Err(replica_prepare::Error::InvalidSignature(_))); Ok(()) }) .await @@ -284,39 +283,22 @@ async fn replica_prepare_invalid_sig() { async fn replica_prepare_invalid_commit_qc() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|msg| msg.high_qc = ctx.rng().gen()); - let res = util.process_replica_prepare(ctx, replica_prepare).await; - assert_matches!(res, Err(ReplicaPrepareError::InvalidHighQC(..))); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_high_qc_of_current_view() { - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - scope::run!(ctx, |ctx, s| async { - let (mut util, runner) = UTHarness::new(ctx, 1).await; - s.spawn_bg(runner.run(ctx)); - - let view = ViewNumber(1); - let qc_view = ViewNumber(1); - util.set_view(view); - let qc = util.new_commit_qc(|msg| msg.view = qc_view); - let replica_prepare = util.new_replica_prepare(|msg| msg.high_qc = qc); - let res = util.process_replica_prepare(ctx, replica_prepare).await; + util.produce_block(ctx).await; + let mut replica_prepare = util.new_replica_prepare(); + replica_prepare.high_qc.as_mut().unwrap().signature = rng.gen(); + let res = util + .process_replica_prepare(ctx, util.sign(replica_prepare)) + .await; assert_matches!( res, - Err(ReplicaPrepareError::HighQCOfFutureView { high_qc_view, current_view }) => { - assert_eq!(high_qc_view, qc_view); - assert_eq!(current_view, view); - } + Err(replica_prepare::Error::InvalidMessage( + validator::ReplicaPrepareVerifyError::HighQC(_) + )) ); Ok(()) }) @@ -324,6 +306,8 @@ async fn replica_prepare_high_qc_of_current_view() { .unwrap(); } +/// Check that leader behaves correctly in case receiving ReplicaPrepare +/// with high_qc with future views (which shouldn't be available yet). #[tokio::test] async fn replica_prepare_high_qc_of_future_view() { zksync_concurrency::testonly::abort_on_panic(); @@ -332,19 +316,24 @@ async fn replica_prepare_high_qc_of_future_view() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let view = ViewNumber(1); - let qc_view = ViewNumber(2); - util.set_view(view); - let qc = util.new_commit_qc(|msg| msg.view = qc_view); - let replica_prepare = util.new_replica_prepare(|msg| msg.high_qc = qc); - let res = util.process_replica_prepare(ctx, replica_prepare).await; - assert_matches!( - res, - Err(ReplicaPrepareError::HighQCOfFutureView{ high_qc_view, current_view }) => { - assert_eq!(high_qc_view, qc_view); - assert_eq!(current_view, view); - } - ); + util.produce_block(ctx).await; + let mut view = util.replica_view(); + let mut replica_prepare = util.new_replica_prepare(); + // Check both the current view and next view. + for _ in 0..2 { + let qc = util.new_commit_qc(|msg| msg.view = view.clone()); + replica_prepare.high_qc = Some(qc); + let res = util + .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) + .await; + assert_matches!( + res, + Err(replica_prepare::Error::InvalidMessage( + validator::ReplicaPrepareVerifyError::HighQCFutureView + )) + ); + view.number = view.number.next(); + } Ok(()) }) .await @@ -374,21 +363,16 @@ async fn replica_commit_sanity_yield_leader_commit() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); + util.produce_block(ctx).await; let replica_commit = util.new_replica_commit(ctx).await; let leader_commit = util - .process_replica_commit(ctx, replica_commit.clone()) + .process_replica_commit(ctx, util.sign(replica_commit.clone())) .await .unwrap() .unwrap(); - assert_matches!( - leader_commit.msg, - LeaderCommit { - protocol_version, - justification, - } => { - assert_eq!(protocol_version, replica_commit.msg.protocol_version); - assert_eq!(justification, util.new_commit_qc(|msg| *msg = replica_commit.msg)); - } + assert_eq!( + leader_commit.msg.justification, + util.new_commit_qc(|msg| *msg = replica_commit) ); Ok(()) }) @@ -405,14 +389,14 @@ async fn replica_commit_incompatible_protocol_version() { s.spawn_bg(runner.run(ctx)); let incompatible_protocol_version = util.incompatible_protocol_version(); - let mut replica_commit = util.new_replica_commit(ctx).await.msg; - replica_commit.protocol_version = incompatible_protocol_version; + let mut replica_commit = util.new_replica_commit(ctx).await; + replica_commit.view.protocol_version = incompatible_protocol_version; let res = util - .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit)) + .process_replica_commit(ctx, util.sign(replica_commit)) .await; assert_matches!( res, - Err(ReplicaCommitError::IncompatibleProtocolVersion { message_version, local_version }) => { + Err(replica_commit::Error::IncompatibleProtocolVersion { message_version, local_version }) => { assert_eq!(message_version, incompatible_protocol_version); assert_eq!(local_version, util.protocol_version()); } @@ -429,14 +413,14 @@ async fn replica_commit_non_validator_signer() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_commit = util.new_replica_commit(ctx).await.msg; + let replica_commit = util.new_replica_commit(ctx).await; let non_validator_key: validator::SecretKey = ctx.rng().gen(); let res = util .process_replica_commit(ctx, non_validator_key.sign_msg(replica_commit)) .await; assert_matches!( res, - Err(ReplicaCommitError::NonValidatorSigner { signer }) => { + Err(replica_commit::Error::NonValidatorSigner { signer }) => { assert_eq!(signer, non_validator_key.public()); } ); @@ -454,13 +438,13 @@ async fn replica_commit_old() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut replica_commit = util.new_replica_commit(ctx).await.msg; - replica_commit.view = util.replica.view.prev(); - let replica_commit = util.owner_key().sign_msg(replica_commit); + let mut replica_commit = util.new_replica_commit(ctx).await; + replica_commit.view.number = ViewNumber(util.replica.view.0 - 1); + let replica_commit = util.sign(replica_commit); let res = util.process_replica_commit(ctx, replica_commit).await; assert_matches!( res, - Err(ReplicaCommitError::Old { current_view, current_phase }) => { + Err(replica_commit::Error::Old { current_view, current_phase }) => { assert_eq!(current_view, util.replica.view); assert_eq!(current_phase, util.replica.phase); } @@ -479,12 +463,14 @@ async fn replica_commit_not_leader_in_view() { let (mut util, runner) = UTHarness::new(ctx, 2).await; s.spawn_bg(runner.run(ctx)); + util.produce_block(ctx).await; let current_view_leader = util.view_leader(util.replica.view); assert_ne!(current_view_leader, util.owner_key().public()); - - let replica_commit = util.new_current_replica_commit(|_| {}); - let res = util.process_replica_commit(ctx, replica_commit).await; - assert_matches!(res, Err(ReplicaCommitError::NotLeaderInView)); + let replica_commit = util.new_current_replica_commit(); + let res = util + .process_replica_commit(ctx, util.sign(replica_commit)) + .await; + assert_matches!(res, Err(replica_commit::Error::NotLeaderInView)); Ok(()) }) .await @@ -501,17 +487,17 @@ async fn replica_commit_already_exists() { let replica_commit = util.new_replica_commit(ctx).await; assert!(util - .process_replica_commit(ctx, replica_commit.clone()) + .process_replica_commit(ctx, util.sign(replica_commit.clone())) .await .unwrap() .is_none()); let res = util - .process_replica_commit(ctx, replica_commit.clone()) + .process_replica_commit(ctx, util.sign(replica_commit.clone())) .await; assert_matches!( res, - Err(ReplicaCommitError::DuplicateMessage { existing_message }) => { - assert_eq!(existing_message, replica_commit.msg) + Err(replica_commit::Error::DuplicateMessage { existing_message }) => { + assert_eq!(existing_message, replica_commit) } ); Ok(()) @@ -528,13 +514,13 @@ async fn replica_commit_num_received_below_threshold() { let (mut util, runner) = UTHarness::new(ctx, 2).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); assert!(util - .process_replica_prepare(ctx, replica_prepare.clone()) + .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) .await .unwrap() .is_none()); - let replica_prepare = util.keys[1].sign_msg(replica_prepare.msg); + let replica_prepare = util.keys[1].sign_msg(replica_prepare); let leader_prepare = util .process_replica_prepare(ctx, replica_prepare) .await @@ -561,16 +547,20 @@ async fn replica_commit_invalid_sig() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut replica_commit = util.new_current_replica_commit(|_| {}); + let msg = util.new_replica_commit(ctx).await; + let mut replica_commit = util.sign(msg); replica_commit.sig = ctx.rng().gen(); let res = util.process_replica_commit(ctx, replica_commit).await; - assert_matches!(res, Err(ReplicaCommitError::InvalidSignature(..))); + assert_matches!(res, Err(replica_commit::Error::InvalidSignature(..))); Ok(()) }) .await .unwrap(); } +/// ReplicaCommit received before sending out LeaderPrepare. +/// Whether leader accepts the message or rejects doesn't matter. +/// It just shouldn't crash. #[tokio::test] async fn replica_commit_unexpected_proposal() { zksync_concurrency::testonly::abort_on_panic(); @@ -579,10 +569,11 @@ async fn replica_commit_unexpected_proposal() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_commit = util.new_current_replica_commit(|_| {}); - util.process_replica_commit(ctx, replica_commit) - .await - .unwrap(); + util.produce_block(ctx).await; + let replica_commit = util.new_current_replica_commit(); + let _ = util + .process_replica_commit(ctx, util.sign(replica_commit)) + .await; Ok(()) }) .await diff --git a/node/actors/bft/src/lib.rs b/node/actors/bft/src/lib.rs index 74b750c2..67aa99f9 100644 --- a/node/actors/bft/src/lib.rs +++ b/node/actors/bft/src/lib.rs @@ -26,7 +26,6 @@ mod config; pub mod io; mod leader; mod metrics; -pub mod misc; mod replica; pub mod testonly; #[cfg(test)] diff --git a/node/actors/bft/src/misc.rs b/node/actors/bft/src/misc.rs deleted file mode 100644 index a8639000..00000000 --- a/node/actors/bft/src/misc.rs +++ /dev/null @@ -1,23 +0,0 @@ -//! Miscellaneous functions related to the consensus. - -/// Calculate the consensus threshold, the minimum number of votes for any consensus action to be valid, -/// for a given number of replicas. -pub fn consensus_threshold(num_validators: usize) -> usize { - let faulty_replicas = faulty_replicas(num_validators); - - // Return the consensus threshold, which is simply: - // t = n - f - num_validators - faulty_replicas -} - -/// Calculate the maximum number of faulty replicas, for a given number of replicas. -pub fn faulty_replicas(num_validators: usize) -> usize { - // Calculate the allowed maximum number of faulty replicas. We want the following relationship to hold: - // n = 5*f + 1 - // for n total replicas and f faulty replicas. This results in the following formula for the maximum - // number of faulty replicas: - // f = floor((n - 1) / 5) - // Because of this, it doesn't make sense to have 5*f + 2 or 5*f + 3 replicas. It won't increase the number - // of allowed faulty replicas. - (num_validators - 1) / 5 -} diff --git a/node/actors/bft/src/replica/block.rs b/node/actors/bft/src/replica/block.rs index 75d2404c..eb00cd9f 100644 --- a/node/actors/bft/src/replica/block.rs +++ b/node/actors/bft/src/replica/block.rs @@ -13,16 +13,22 @@ impl StateMachine { ctx: &ctx::Ctx, commit_qc: &validator::CommitQC, ) -> ctx::Result<()> { + // Update high_qc. + if self + .high_qc + .as_ref() + .map(|qc| qc.view().number < commit_qc.view().number) + .unwrap_or(true) + { + self.high_qc = Some(commit_qc.clone()); + } // TODO(gprusak): for availability of finalized blocks, // replicas should be able to broadcast highest quorums without // the corresponding block (same goes for synchronization). - let Some(cache) = self - .block_proposal_cache - .get(&commit_qc.message.proposal.number) - else { + let Some(cache) = self.block_proposal_cache.get(&commit_qc.header().number) else { return Ok(()); }; - let Some(payload) = cache.get(&commit_qc.message.proposal.payload) else { + let Some(payload) = cache.get(&commit_qc.header().payload) else { return Ok(()); }; let block = validator::FinalBlock { diff --git a/node/actors/bft/src/replica/leader_commit.rs b/node/actors/bft/src/replica/leader_commit.rs index a7e99e9e..926ac6e8 100644 --- a/node/actors/bft/src/replica/leader_commit.rs +++ b/node/actors/bft/src/replica/leader_commit.rs @@ -1,3 +1,4 @@ +//! Handler of a LeaderCommit message. use super::StateMachine; use tracing::instrument; use zksync_concurrency::{ctx, error::Wrap}; @@ -15,14 +16,12 @@ pub(crate) enum Error { local_version: ProtocolVersion, }, /// Invalid leader. - #[error( - "invalid leader (correct leader: {correct_leader:?}, received leader: {received_leader:?})" - )] - InvalidLeader { - /// Correct leader. - correct_leader: validator::PublicKey, + #[error("bad leader: got {got:?}, want {want:?}")] + BadLeader { /// Received leader. - received_leader: validator::PublicKey, + got: validator::PublicKey, + /// Correct leader. + want: validator::PublicKey, }, /// Past view of phase. #[error("past view/phase (current view: {current_view:?}, current phase: {current_phase:?})")] @@ -34,10 +33,10 @@ pub(crate) enum Error { }, /// Invalid message signature. #[error("invalid signature: {0:#}")] - InvalidSignature(#[source] validator::Error), - /// Invalid justification for the message. - #[error("invalid justification: {0:#}")] - InvalidJustification(#[source] anyhow::Error), + InvalidSignature(validator::Error), + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(validator::CommitQCVerifyError), /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. #[error(transparent)] Internal(#[from] ctx::Error), @@ -69,26 +68,30 @@ impl StateMachine { // Unwrap message. let message = &signed_message.msg; let author = &signed_message.key; - let view = message.justification.message.view; // Check protocol version compatibility. - if !crate::PROTOCOL_VERSION.compatible(&message.protocol_version) { + if !crate::PROTOCOL_VERSION.compatible(&message.view().protocol_version) { return Err(Error::IncompatibleProtocolVersion { - message_version: message.protocol_version, + message_version: message.view().protocol_version, local_version: crate::PROTOCOL_VERSION, }); } // Check that it comes from the correct leader. - if author != &self.config.view_leader(view) { - return Err(Error::InvalidLeader { - correct_leader: self.config.view_leader(view), - received_leader: author.clone(), + let leader = self + .config + .genesis() + .validators + .view_leader(message.view().number); + if author != &leader { + return Err(Error::BadLeader { + want: leader, + got: author.clone(), }); } // If the message is from the "past", we discard it. - if (view, validator::Phase::Commit) < (self.view, self.phase) { + if (message.view().number, validator::Phase::Commit) < (self.view, self.phase) { return Err(Error::Old { current_view: self.view, current_phase: self.phase, @@ -99,14 +102,9 @@ impl StateMachine { // Check the signature on the message. signed_message.verify().map_err(Error::InvalidSignature)?; - - // ----------- Checking the justification of the message -------------- - - // Verify the QuorumCertificate. message - .justification - .verify(&self.config.validator_set, self.config.threshold()) - .map_err(Error::InvalidJustification)?; + .verify(self.config.genesis()) + .map_err(Error::InvalidMessage)?; // ----------- All checks finished. Now we process the message. -------------- @@ -115,14 +113,8 @@ impl StateMachine { .await .wrap("save_block()")?; - // Update the state machine. We don't update the view and phase (or backup our state) here - // because we will do it when we start the new view. - if message.justification.message.view >= self.high_qc.message.view { - self.high_qc = message.justification.clone(); - } - // Start a new view. But first we skip to the view of this message. - self.view = view; + self.view = message.view().number; self.start_new_view(ctx).await.wrap("start_new_view()")?; Ok(()) diff --git a/node/actors/bft/src/replica/leader_prepare.rs b/node/actors/bft/src/replica/leader_prepare.rs index 04b6a2e9..7e017bcb 100644 --- a/node/actors/bft/src/replica/leader_prepare.rs +++ b/node/actors/bft/src/replica/leader_prepare.rs @@ -1,5 +1,5 @@ +//! Handler of a LeaderPrepare message. use super::StateMachine; -use std::collections::HashMap; use tracing::instrument; use zksync_concurrency::{ctx, error::Wrap}; use zksync_consensus_network::io::{ConsensusInputMessage, Target}; @@ -39,76 +39,20 @@ pub(crate) enum Error { /// Invalid message signature. #[error("invalid signature: {0:#}")] InvalidSignature(#[source] validator::Error), - /// Invalid `PrepareQC` message. - #[error("invalid PrepareQC: {0:#}")] - InvalidPrepareQC(#[source] anyhow::Error), - /// Invalid `HighQC` message. - #[error("invalid high QC: {0:#}")] - InvalidHighQC(#[source] anyhow::Error), - /// High QC of a future view. - #[error( - "high QC of a future view (high QC view: {high_qc_view:?}, current view: {current_view:?}" - )] - HighQCOfFutureView { - /// Received high QC view. - high_qc_view: validator::ViewNumber, - /// Current view. - current_view: validator::ViewNumber, - }, + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] validator::LeaderPrepareVerifyError), /// Previous proposal was not finalized. - #[error("new block proposal when the previous proposal was not finalized")] - ProposalWhenPreviousNotFinalized, - /// Invalid parent hash. - #[error( - "block proposal with invalid parent hash (correct parent hash: {correct_parent_hash:#?}, \ - received parent hash: {received_parent_hash:#?}, block: {header:?})" - )] - ProposalInvalidParentHash { - /// Correct parent hash. - correct_parent_hash: validator::BlockHeaderHash, - /// Received parent hash. - received_parent_hash: validator::BlockHeaderHash, - /// Header including the incorrect parent hash. - header: validator::BlockHeader, - }, - /// Non-sequential proposal number. - #[error( - "block proposal with non-sequential number (correct proposal number: {correct_number}, \ - received proposal number: {received_number}, block: {header:?})" - )] - ProposalNonSequentialNumber { - /// Correct proposal number. - correct_number: validator::BlockNumber, - /// Received proposal number. - received_number: validator::BlockNumber, - /// Header including the incorrect proposal number. - header: validator::BlockHeader, - }, - /// Mismatched payload. - #[error("block proposal with mismatched payload")] - ProposalMismatchedPayload, + /// Oversized payload. - #[error( - "block proposal with an oversized payload (payload size: {payload_size}, block: {header:?}" - )] + #[error("block proposal with an oversized payload (payload size: {payload_size})")] ProposalOversizedPayload { /// Size of the payload. payload_size: usize, - /// Proposal header corresponding to the payload. - header: validator::BlockHeader, }, /// Invalid payload. #[error("invalid payload: {0:#}")] ProposalInvalidPayload(#[source] anyhow::Error), - /// Re-proposal without quorum. - #[error("block re-proposal without quorum for the re-proposal")] - ReproposalWithoutQuorum, - /// Re-proposal when the previous proposal was finalized. - #[error("block re-proposal when the previous proposal was finalized")] - ReproposalWhenFinalized, - /// Re-proposal of invalid block. - #[error("block re-proposal of invalid block")] - ReproposalInvalidBlock, /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. #[error(transparent)] Internal(#[from] ctx::Error), @@ -139,20 +83,21 @@ impl StateMachine { // Unwrap message. let message = &signed_message.msg; let author = &signed_message.key; - let view = message.view; + let view = message.view().number; // Check protocol version compatibility. - if !crate::PROTOCOL_VERSION.compatible(&message.protocol_version) { + if !crate::PROTOCOL_VERSION.compatible(&message.view().protocol_version) { return Err(Error::IncompatibleProtocolVersion { - message_version: message.protocol_version, + message_version: message.view().protocol_version, local_version: crate::PROTOCOL_VERSION, }); } // Check that it comes from the correct leader. - if author != &self.config.view_leader(view) { + let leader = self.config.genesis().validators.view_leader(view); + if author != &leader { return Err(Error::InvalidLeader { - correct_leader: self.config.view_leader(view), + correct_leader: leader, received_leader: author.clone(), }); } @@ -165,136 +110,40 @@ impl StateMachine { }); } - // ----------- Checking the signed part of the message -------------- + // ----------- Checking the message -------------- signed_message.verify().map_err(Error::InvalidSignature)?; - - // ----------- Checking the justification of the message -------------- - - // Verify the PrepareQC. message - .justification - .verify(view, &self.config.validator_set, self.config.threshold()) - .map_err(Error::InvalidPrepareQC)?; - - // Get the highest block voted and check if there's a quorum of votes for it. To have a quorum - // in this situation, we require 2*f+1 votes, where f is the maximum number of faulty replicas. - let mut vote_count: HashMap<_, usize> = HashMap::new(); - - for (msg, signers) in &message.justification.map { - *vote_count.entry(msg.high_vote.proposal).or_default() += signers.len(); - } + .verify(self.config.genesis()) + .map_err(Error::InvalidMessage)?; + let high_qc = message.justification.high_qc(); - let highest_vote: Option = vote_count - .into_iter() - // We only take one value from the iterator because there can only be at most one block with a quorum of 2f+1 votes. - .find(|(_, v)| *v > 2 * self.config.faulty_replicas()) - .map(|(h, _)| h); - - // Get the highest CommitQC and verify it. - let highest_qc: validator::CommitQC = message - .justification - .map - .keys() - .max_by_key(|m| m.high_qc.message.view) - .unwrap() - .high_qc - .clone(); - - highest_qc - .verify(&self.config.validator_set, self.config.threshold()) - .map_err(Error::InvalidHighQC)?; - - // If the high QC is for a future view, we discard the message. - // This check is not necessary for correctness, but it's useful to - // guarantee that our messages don't contain QCs from the future. - if highest_qc.message.view >= view { - return Err(Error::HighQCOfFutureView { - high_qc_view: highest_qc.message.view, - current_view: view, - }); - } - - // Try to create a finalized block with this CommitQC and our block proposal cache. - // This gives us another chance to finalize a block that we may have missed before. - self.save_block(ctx, &highest_qc) - .await - .wrap("save_block()")?; - - // ----------- Checking the block proposal -------------- - - // Check that the proposal is valid. - match &message.proposal_payload { - // The leader proposed a new block. - Some(payload) => { - // Check that the payload doesn't exceed the maximum size. - if payload.0.len() > self.config.max_payload_size { - return Err(Error::ProposalOversizedPayload { - payload_size: payload.0.len(), - header: message.proposal, - }); - } - - // Check that payload matches the header - if message.proposal.payload != payload.hash() { - return Err(Error::ProposalMismatchedPayload); - } - - // Check that we finalized the previous block. - if highest_vote.is_some() - && highest_vote.as_ref() != Some(&highest_qc.message.proposal) - { - return Err(Error::ProposalWhenPreviousNotFinalized); - } - - // Parent hash should match. - if highest_qc.message.proposal.hash() != message.proposal.parent { - return Err(Error::ProposalInvalidParentHash { - correct_parent_hash: highest_qc.message.proposal.hash(), - received_parent_hash: message.proposal.parent, - header: message.proposal, - }); - } - - // Block number should match. - if highest_qc.message.proposal.number.next() != message.proposal.number { - return Err(Error::ProposalNonSequentialNumber { - correct_number: highest_qc.message.proposal.number.next(), - received_number: message.proposal.number, - header: message.proposal, - }); - } + // Check that the payload doesn't exceed the maximum size. + if let Some(payload) = &message.proposal_payload { + if payload.0.len() > self.config.max_payload_size { + return Err(Error::ProposalOversizedPayload { + payload_size: payload.0.len(), + }); + } - // Payload should be valid. + if let Some(prev) = message.proposal.number.prev() { // Defensively assume that PayloadManager cannot verify proposal until the previous block is stored. self.config .block_store - .wait_until_persisted(ctx, highest_qc.header().number) + .wait_until_persisted(ctx, prev) .await .map_err(ctx::Error::Canceled)?; - if let Err(err) = self - .config - .payload_manager - .verify(ctx, message.proposal.number, payload) - .await - { - return Err(match err { - err @ ctx::Error::Canceled(_) => Error::Internal(err), - ctx::Error::Internal(err) => Error::ProposalInvalidPayload(err), - }); - } } - // The leader is re-proposing a past block. - None => { - let Some(highest_vote) = highest_vote else { - return Err(Error::ReproposalWithoutQuorum); - }; - if highest_vote == highest_qc.message.proposal { - return Err(Error::ReproposalWhenFinalized); - } - if highest_vote != message.proposal { - return Err(Error::ReproposalInvalidBlock); - } + if let Err(err) = self + .config + .payload_manager + .verify(ctx, message.proposal.number, payload) + .await + { + return Err(match err { + err @ ctx::Error::Canceled(_) => Error::Internal(err), + ctx::Error::Internal(err) => Error::ProposalInvalidPayload(err), + }); } } @@ -302,20 +151,19 @@ impl StateMachine { // Create our commit vote. let commit_vote = validator::ReplicaCommit { - protocol_version: crate::PROTOCOL_VERSION, - view, + view: message.view().clone(), proposal: message.proposal, }; // Update the state machine. - self.view = view; + self.view = message.view().number; self.phase = validator::Phase::Commit; - self.high_vote = commit_vote; - - if highest_qc.message.view > self.high_qc.message.view { - self.high_qc = highest_qc; + self.high_vote = Some(commit_vote.clone()); + if let Some(high_qc) = high_qc { + // Try to create a finalized block with this CommitQC and our block proposal cache. + // This gives us another chance to finalize a block that we may have missed before. + self.save_block(ctx, high_qc).await.wrap("save_block()")?; } - // If we received a new block proposal, store it in our cache. if let Some(payload) = &message.proposal_payload { self.block_proposal_cache diff --git a/node/actors/bft/src/replica/mod.rs b/node/actors/bft/src/replica/mod.rs index 69ede4b0..05a72481 100644 --- a/node/actors/bft/src/replica/mod.rs +++ b/node/actors/bft/src/replica/mod.rs @@ -3,16 +3,12 @@ //! node will perform both the replica and leader roles simultaneously. mod block; -mod leader_commit; -mod leader_prepare; +pub(crate) mod leader_commit; +pub(crate) mod leader_prepare; mod new_view; mod state_machine; #[cfg(test)] mod tests; mod timer; -#[cfg(test)] -pub(crate) use self::leader_commit::Error as LeaderCommitError; -#[cfg(test)] -pub(crate) use self::leader_prepare::Error as LeaderPrepareError; pub(crate) use self::state_machine::StateMachine; diff --git a/node/actors/bft/src/replica/new_view.rs b/node/actors/bft/src/replica/new_view.rs index b1f73d7a..eaf9c7da 100644 --- a/node/actors/bft/src/replica/new_view.rs +++ b/node/actors/bft/src/replica/new_view.rs @@ -11,14 +11,13 @@ impl StateMachine { tracing::info!("Starting view {}", self.view.next().0); // Update the state machine. - let next_view = self.view.next(); - - self.view = next_view; + self.view = self.view.next(); self.phase = validator::Phase::Prepare; - - // Clear the block cache. - self.block_proposal_cache - .retain(|k, _| k > &self.high_qc.message.proposal.number); + if let Some(qc) = self.high_qc.as_ref() { + // Clear the block cache. + self.block_proposal_cache + .retain(|k, _| k > &qc.header().number); + } // Backup our state. self.backup_state(ctx).await.wrap("backup_state()")?; @@ -30,13 +29,16 @@ impl StateMachine { .secret_key .sign_msg(validator::ConsensusMsg::ReplicaPrepare( validator::ReplicaPrepare { - protocol_version: crate::PROTOCOL_VERSION, - view: next_view, - high_vote: self.high_vote, + view: validator::View { + protocol_version: crate::PROTOCOL_VERSION, + fork: self.config.genesis().fork.number, + number: self.view, + }, + high_vote: self.high_vote.clone(), high_qc: self.high_qc.clone(), }, )), - recipient: Target::Validator(self.config.view_leader(next_view)), + recipient: Target::Validator(self.config.genesis().validators.view_leader(self.view)), }; self.outbound_pipe.send(output_message.into()); diff --git a/node/actors/bft/src/replica/state_machine.rs b/node/actors/bft/src/replica/state_machine.rs index 15e62335..f1dd946e 100644 --- a/node/actors/bft/src/replica/state_machine.rs +++ b/node/actors/bft/src/replica/state_machine.rs @@ -23,9 +23,9 @@ pub(crate) struct StateMachine { /// The current phase. pub(crate) phase: validator::Phase, /// The highest block proposal that the replica has committed to. - pub(crate) high_vote: validator::ReplicaCommit, + pub(crate) high_vote: Option, /// The highest commit quorum certificate known to the replica. - pub(crate) high_qc: validator::CommitQC, + pub(crate) high_qc: Option, /// A cache of the received block proposals. pub(crate) block_proposal_cache: BTreeMap>, @@ -45,10 +45,7 @@ impl StateMachine { config: Arc, outbound_pipe: OutputSender, ) -> ctx::Result<(Self, sync::prunable_mpsc::Sender)> { - let backup = match config.replica_store.state(ctx).await? { - Some(backup) => backup, - None => config.block_store.subscribe().borrow().last.clone().into(), - }; + let backup = config.replica_store.state(ctx).await?; let mut block_proposal_cache: BTreeMap<_, HashMap<_, _>> = BTreeMap::new(); for proposal in backup.proposals { block_proposal_cache @@ -152,7 +149,7 @@ impl StateMachine { let backup = storage::ReplicaState { view: self.view, phase: self.phase, - high_vote: self.high_vote, + high_vote: self.high_vote.clone(), high_qc: self.high_qc.clone(), proposals, }; diff --git a/node/actors/bft/src/replica/tests.rs b/node/actors/bft/src/replica/tests.rs index 0cd2f731..3866e813 100644 --- a/node/actors/bft/src/replica/tests.rs +++ b/node/actors/bft/src/replica/tests.rs @@ -10,39 +10,32 @@ use zksync_consensus_roles::validator::{ self, CommitQC, Payload, PrepareQC, ReplicaCommit, ReplicaPrepare, ViewNumber, }; +/// Sanity check of the happy path. #[tokio::test] -async fn leader_prepare_sanity() { +async fn block_production() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - - let leader_prepare = util.new_leader_prepare(ctx).await; - util.process_leader_prepare(ctx, leader_prepare) - .await - .unwrap(); + util.produce_block(ctx).await; Ok(()) }) .await .unwrap(); } +/// Sanity check of block production with reproposal. #[tokio::test] -async fn leader_prepare_reproposal_sanity() { +async fn reproposal_block_production() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - - util.new_replica_commit(ctx).await; + util.new_leader_commit(ctx).await; util.process_replica_timeout(ctx).await; - let leader_prepare = util.new_leader_prepare(ctx).await; - assert!(leader_prepare.msg.proposal_payload.is_none()); - util.process_leader_prepare(ctx, leader_prepare) - .await - .unwrap(); + util.produce_block(ctx).await; Ok(()) }) .await @@ -58,10 +51,10 @@ async fn leader_prepare_incompatible_protocol_version() { s.spawn_bg(runner.run(ctx)); let incompatible_protocol_version = util.incompatible_protocol_version(); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.protocol_version = incompatible_protocol_version; + let mut leader_prepare = util.new_leader_prepare(ctx).await; + leader_prepare.justification.view.protocol_version = incompatible_protocol_version; let res = util - .process_leader_prepare(ctx, util.owner_key().sign_msg(leader_prepare)) + .process_leader_prepare(ctx, util.sign(leader_prepare)) .await; assert_matches!( res, @@ -84,15 +77,14 @@ async fn leader_prepare_sanity_yield_replica_commit() { let leader_prepare = util.new_leader_prepare(ctx).await; let replica_commit = util - .process_leader_prepare(ctx, leader_prepare.clone()) + .process_leader_prepare(ctx, util.sign(leader_prepare.clone())) .await .unwrap(); assert_eq!( replica_commit.msg, ReplicaCommit { - protocol_version: leader_prepare.msg.protocol_version, - view: leader_prepare.msg.view, - proposal: leader_prepare.msg.proposal, + view: leader_prepare.view().clone(), + proposal: leader_prepare.proposal, } ); Ok(()) @@ -113,25 +105,29 @@ async fn leader_prepare_invalid_leader() { util.set_view(view); assert_eq!(util.view_leader(view), util.keys[0].public()); - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); assert!(util - .process_replica_prepare(ctx, replica_prepare.clone()) + .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) .await .unwrap() .is_none()); - let replica_prepare = util.keys[1].sign_msg(replica_prepare.msg); + let replica_prepare = util.keys[1].sign_msg(replica_prepare); let mut leader_prepare = util .process_replica_prepare(ctx, replica_prepare) .await .unwrap() .unwrap() .msg; - leader_prepare.view = leader_prepare.view.next(); - assert_ne!(util.view_leader(leader_prepare.view), util.keys[0].public()); + leader_prepare.justification.view.number = leader_prepare.justification.view.number.next(); + assert_ne!( + util.view_leader(leader_prepare.view().number), + util.keys[0].public() + ); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; assert_matches!( res, Err(leader_prepare::Error::InvalidLeader { correct_leader, received_leader }) => { @@ -153,10 +149,11 @@ async fn leader_prepare_old_view() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.view = util.replica.view.prev(); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; + let mut leader_prepare = util.new_leader_prepare(ctx).await; + leader_prepare.justification.view.number.0 = util.replica.view.0 - 1; + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; assert_matches!( res, Err(leader_prepare::Error::Old { current_view, current_phase }) => { @@ -183,17 +180,17 @@ async fn leader_prepare_invalid_payload() { let leader_prepare = util.new_leader_prepare(ctx).await; // Insert a finalized block to the storage. + let mut justification = CommitQC::new( + ReplicaCommit { + view: util.replica_view(), + proposal: leader_prepare.proposal, + }, + util.genesis(), + ); + justification.add(&util.sign(justification.message.clone()), util.genesis()); let block = validator::FinalBlock { - payload: leader_prepare.msg.proposal_payload.clone().unwrap(), - justification: CommitQC::from( - &[util.keys[0].sign_msg(ReplicaCommit { - protocol_version: util.protocol_version(), - view: util.replica.view, - proposal: leader_prepare.msg.proposal, - })], - &util.validator_set(), - ) - .unwrap(), + payload: leader_prepare.proposal_payload.clone().unwrap(), + justification, }; util.replica .config @@ -202,7 +199,9 @@ async fn leader_prepare_invalid_payload() { .await .unwrap(); - let res = util.process_leader_prepare(ctx, leader_prepare).await; + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; assert_matches!(res, Err(leader_prepare::Error::ProposalInvalidPayload(..))); Ok(()) }) @@ -217,8 +216,8 @@ async fn leader_prepare_invalid_sig() { scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - - let mut leader_prepare = util.new_leader_prepare(ctx).await; + let leader_prepare = util.new_leader_prepare(ctx).await; + let mut leader_prepare = util.sign(leader_prepare); leader_prepare.sig = ctx.rng().gen(); let res = util.process_leader_prepare(ctx, leader_prepare).await; assert_matches!(res, Err(leader_prepare::Error::InvalidSignature(..))); @@ -236,30 +235,17 @@ async fn leader_prepare_invalid_prepare_qc() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.justification = ctx.rng().gen(); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::InvalidPrepareQC(_))); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_invalid_high_qc() { - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - scope::run!(ctx, |ctx, s| async { - let (mut util, runner) = UTHarness::new(ctx, 1).await; - s.spawn_bg(runner.run(ctx)); - - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.justification = util.new_prepare_qc(|msg| msg.high_qc = ctx.rng().gen()); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::InvalidHighQC(_))); + let mut leader_prepare = util.new_leader_prepare(ctx).await; + leader_prepare.justification.signature = ctx.rng().gen(); + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; + assert_matches!( + res, + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::Justification(_) + )) + ); Ok(()) }) .await @@ -275,18 +261,17 @@ async fn leader_prepare_proposal_oversized_payload() { s.spawn_bg(runner.run(ctx)); let payload_oversize = MAX_PAYLOAD_SIZE + 1; - let payload_vec = vec![0; payload_oversize]; - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.proposal_payload = Some(Payload(payload_vec)); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); + let payload = Payload(vec![0; payload_oversize]); + let mut leader_prepare = util.new_leader_prepare(ctx).await; + leader_prepare.proposal.payload = payload.hash(); + leader_prepare.proposal_payload = Some(payload); let res = util - .process_leader_prepare(ctx, leader_prepare.clone()) + .process_leader_prepare(ctx, util.sign(leader_prepare)) .await; assert_matches!( res, - Err(leader_prepare::Error::ProposalOversizedPayload{ payload_size, header }) => { + Err(leader_prepare::Error::ProposalOversizedPayload{ payload_size }) => { assert_eq!(payload_size, payload_oversize); - assert_eq!(header, leader_prepare.msg.proposal); } ); Ok(()) @@ -303,11 +288,17 @@ async fn leader_prepare_proposal_mismatched_payload() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; + let mut leader_prepare = util.new_leader_prepare(ctx).await; leader_prepare.proposal_payload = Some(ctx.rng().gen()); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::ProposalMismatchedPayload)); + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; + assert_matches!( + res, + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::ProposalMismatchedPayload + )) + ); Ok(()) }) .await @@ -318,23 +309,28 @@ async fn leader_prepare_proposal_mismatched_payload() { async fn leader_prepare_proposal_when_previous_not_finalized() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); - let mut leader_prepare = util - .process_replica_prepare(ctx, replica_prepare) - .await - .unwrap() - .unwrap() - .msg; - leader_prepare.justification = util.new_prepare_qc(|msg| msg.high_vote = ctx.rng().gen()); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; + tracing::info!("Execute view without replicas receiving the LeaderCommit."); + util.new_leader_commit(ctx).await; + util.process_replica_timeout(ctx).await; + tracing::info!("Make leader repropose the block."); + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!("Modify the message to include a new proposal anyway."); + let payload: Payload = rng.gen(); + leader_prepare.proposal.payload = payload.hash(); + leader_prepare.proposal_payload = Some(payload); + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; assert_matches!( res, - Err(leader_prepare::Error::ProposalWhenPreviousNotFinalized) + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::ProposalWhenPreviousNotFinalized + )) ); Ok(()) }) @@ -343,37 +339,26 @@ async fn leader_prepare_proposal_when_previous_not_finalized() { } #[tokio::test] -async fn leader_prepare_proposal_invalid_parent_hash() { +async fn leader_prepare_bad_parent_hash() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); - let mut leader_prepare = util - .process_replica_prepare(ctx, replica_prepare.clone()) - .await - .unwrap() - .unwrap() - .msg; - leader_prepare.proposal.parent = ctx.rng().gen(); - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util - .process_leader_prepare(ctx, leader_prepare.clone()) - .await; - assert_matches!( - res, - Err(leader_prepare::Error::ProposalInvalidParentHash { - correct_parent_hash, - received_parent_hash, - header - }) => { - assert_eq!(correct_parent_hash, replica_prepare.msg.high_vote.proposal.hash()); - assert_eq!(received_parent_hash, leader_prepare.msg.proposal.parent); - assert_eq!(header, leader_prepare.msg.proposal); - } - ); + tracing::info!("Produce initial block."); + util.produce_block(ctx).await; + tracing::info!("Make leader propose the next block."); + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!("Modify the proposal.parent so that it doesn't match the previous block"); + leader_prepare.proposal.parent = Some(ctx.rng().gen()); + let res = util.process_leader_prepare(ctx, util.sign(leader_prepare.clone())).await; + assert_matches!(res, Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::BadParentHash { got, want } + )) => { + assert_eq!(want, Some(leader_prepare.justification.high_qc().unwrap().message.proposal.hash())); + assert_eq!(got, leader_prepare.proposal.parent); + }); Ok(()) }) .await @@ -381,37 +366,27 @@ async fn leader_prepare_proposal_invalid_parent_hash() { } #[tokio::test] -async fn leader_prepare_proposal_non_sequential_number() { +async fn leader_prepare_bad_block_number() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); scope::run!(ctx, |ctx,s| async { let (mut util,runner) = UTHarness::new(ctx,1).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}); - let mut leader_prepare = util - .process_replica_prepare(ctx, replica_prepare.clone()) - .await - .unwrap() - .unwrap() - .msg; - let correct_num = replica_prepare.msg.high_vote.proposal.number.next(); - assert_eq!(correct_num, leader_prepare.proposal.number); - - let non_seq_num = correct_num.next(); - leader_prepare.proposal.number = non_seq_num; - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util - .process_leader_prepare(ctx, leader_prepare.clone()) - .await; - assert_matches!( - res, - Err(leader_prepare::Error::ProposalNonSequentialNumber { correct_number, received_number, header }) => { - assert_eq!(correct_number, correct_num); - assert_eq!(received_number, non_seq_num); - assert_eq!(header, leader_prepare.msg.proposal); - } - ); + tracing::info!("Produce initial block."); + util.produce_block(ctx).await; + tracing::info!("Make leader propose the next block."); + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!("Modify the proposal.number so that it doesn't match the previous block"); + leader_prepare.proposal.number = rng.gen(); + let res = util.process_leader_prepare(ctx, util.sign(leader_prepare.clone())).await; + assert_matches!(res, Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::BadBlockNumber { got, want } + )) => { + assert_eq!(want, leader_prepare.justification.high_qc().unwrap().message.proposal.number.next()); + assert_eq!(got, leader_prepare.proposal.number); + }); Ok(()) }).await.unwrap(); } @@ -425,29 +400,34 @@ async fn leader_prepare_reproposal_without_quorum() { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - let replica_prepare = util.new_replica_prepare(|_| {}).msg; - let mut leader_prepare = util - .process_replica_prepare_all(ctx, replica_prepare.clone()) - .await - .msg; - - // Turn leader_prepare into an unjustified reproposal. - let replica_prepares: Vec<_> = util - .keys - .iter() - .map(|k| { - let mut msg = replica_prepare.clone(); - msg.high_vote = rng.gen(); - k.sign_msg(msg) - }) - .collect(); - leader_prepare.justification = - PrepareQC::from(&replica_prepares, &util.validator_set()).unwrap(); - leader_prepare.proposal_payload = None; - - let leader_prepare = util.keys[0].sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::ReproposalWithoutQuorum)); + tracing::info!("make leader repropose a block"); + util.new_leader_commit(ctx).await; + util.process_replica_timeout(ctx).await; + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!("modify justification, to make reproposal unjustified"); + let mut replica_prepare: ReplicaPrepare = leader_prepare + .justification + .map + .keys() + .next() + .unwrap() + .clone(); + leader_prepare.justification = PrepareQC::new(leader_prepare.justification.view); + for key in &util.keys { + replica_prepare.high_vote.as_mut().unwrap().proposal.payload = rng.gen(); + leader_prepare + .justification + .add(&key.sign_msg(replica_prepare.clone()), util.genesis()); + } + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; + assert_matches!( + res, + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::ReproposalWithoutQuorum + )) + ); Ok(()) }) .await @@ -462,11 +442,28 @@ async fn leader_prepare_reproposal_when_finalized() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; + tracing::info!("Make leader propose a new block"); + util.produce_block(ctx).await; + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!( + "Modify the message so that it is actually a reproposal of the previous block" + ); + leader_prepare.proposal = leader_prepare + .justification + .high_qc() + .unwrap() + .message + .proposal; leader_prepare.proposal_payload = None; - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::ReproposalWhenFinalized)); + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; + assert_matches!( + res, + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::ReproposalWhenFinalized + )) + ); Ok(()) }) .await @@ -477,40 +474,33 @@ async fn leader_prepare_reproposal_when_finalized() { async fn leader_prepare_reproposal_invalid_block() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_prepare = util.new_leader_prepare(ctx).await.msg; - leader_prepare.justification = util.new_prepare_qc(|msg| msg.high_vote = ctx.rng().gen()); - leader_prepare.proposal_payload = None; - let leader_prepare = util.owner_key().sign_msg(leader_prepare); - let res = util.process_leader_prepare(ctx, leader_prepare).await; - assert_matches!(res, Err(leader_prepare::Error::ReproposalInvalidBlock)); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_commit_sanity() { - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - scope::run!(ctx, |ctx, s| async { - let (mut util, runner) = UTHarness::new_many(ctx).await; - s.spawn_bg(runner.run(ctx)); - - let leader_commit = util.new_leader_commit(ctx).await; - util.process_leader_commit(ctx, leader_commit) - .await - .unwrap(); + tracing::info!("Make leader repropose a block."); + util.new_leader_commit(ctx).await; + util.process_replica_timeout(ctx).await; + let mut leader_prepare = util.new_leader_prepare(ctx).await; + tracing::info!("Make the reproposal different than expected"); + leader_prepare.proposal.payload = rng.gen(); + let res = util + .process_leader_prepare(ctx, util.sign(leader_prepare)) + .await; + assert_matches!( + res, + Err(leader_prepare::Error::InvalidMessage( + validator::LeaderPrepareVerifyError::ReproposalBadBlock + )) + ); Ok(()) }) .await .unwrap(); } +/// Check that replica provides expecte high_vote and high_qc after finalizing a block. #[tokio::test] async fn leader_commit_sanity_yield_replica_prepare() { zksync_concurrency::testonly::abort_on_panic(); @@ -521,16 +511,17 @@ async fn leader_commit_sanity_yield_replica_prepare() { let leader_commit = util.new_leader_commit(ctx).await; let replica_prepare = util - .process_leader_commit(ctx, leader_commit.clone()) + .process_leader_commit(ctx, util.sign(leader_commit.clone())) .await .unwrap(); + let mut view = leader_commit.justification.message.view.clone(); + view.number = view.number.next(); assert_eq!( replica_prepare.msg, ReplicaPrepare { - protocol_version: leader_commit.msg.protocol_version, - view: leader_commit.msg.justification.message.view.next(), - high_vote: leader_commit.msg.justification.message, - high_qc: leader_commit.msg.justification, + view, + high_vote: Some(leader_commit.justification.message.clone()), + high_qc: Some(leader_commit.justification), } ); Ok(()) @@ -548,11 +539,9 @@ async fn leader_commit_incompatible_protocol_version() { s.spawn_bg(runner.run(ctx)); let incompatible_protocol_version = util.incompatible_protocol_version(); - let mut leader_commit = util.new_leader_commit(ctx).await.msg; - leader_commit.protocol_version = incompatible_protocol_version; - let res = util - .process_leader_commit(ctx, util.owner_key().sign_msg(leader_commit)) - .await; + let mut leader_commit = util.new_leader_commit(ctx).await; + leader_commit.justification.message.view.protocol_version = incompatible_protocol_version; + let res = util.process_leader_commit(ctx, util.sign(leader_commit)).await; assert_matches!( res, Err(leader_commit::Error::IncompatibleProtocolVersion { message_version, local_version }) => { @@ -565,21 +554,18 @@ async fn leader_commit_incompatible_protocol_version() { } #[tokio::test] -async fn leader_commit_invalid_leader() { +async fn leader_commit_bad_leader() { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 2).await; s.spawn_bg(runner.run(ctx)); - - let current_view_leader = util.view_leader(util.replica.view); - assert_ne!(current_view_leader, util.owner_key().public()); - - let leader_commit = util.new_leader_commit(ctx).await.msg; + let leader_commit = util.new_leader_commit(ctx).await; + // Sign the leader_prepare with a key of different validator. let res = util .process_leader_commit(ctx, util.keys[1].sign_msg(leader_commit)) .await; - assert_matches!(res, Err(leader_commit::Error::InvalidLeader { .. })); + assert_matches!(res, Err(leader_commit::Error::BadLeader { .. })); Ok(()) }) .await @@ -594,8 +580,8 @@ async fn leader_commit_invalid_sig() { scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - - let mut leader_commit = util.new_leader_commit(ctx).await; + let leader_commit = util.new_leader_commit(ctx).await; + let mut leader_commit = util.sign(leader_commit); leader_commit.sig = rng.gen(); let res = util.process_leader_commit(ctx, leader_commit).await; assert_matches!(res, Err(leader_commit::Error::InvalidSignature { .. })); @@ -614,12 +600,17 @@ async fn leader_commit_invalid_commit_qc() { let (mut util, runner) = UTHarness::new(ctx, 1).await; s.spawn_bg(runner.run(ctx)); - let mut leader_commit = util.new_leader_commit(ctx).await.msg; - leader_commit.justification = rng.gen(); + let mut leader_commit = util.new_leader_commit(ctx).await; + leader_commit.justification.signature = rng.gen(); let res = util - .process_leader_commit(ctx, util.owner_key().sign_msg(leader_commit)) + .process_leader_commit(ctx, util.sign(leader_commit)) .await; - assert_matches!(res, Err(leader_commit::Error::InvalidJustification { .. })); + assert_matches!( + res, + Err(leader_commit::Error::InvalidMessage( + validator::CommitQCVerifyError::BadSignature(..) + )) + ); Ok(()) }) .await diff --git a/node/actors/bft/src/replica/timer.rs b/node/actors/bft/src/replica/timer.rs index df5ef3b3..0cbae88c 100644 --- a/node/actors/bft/src/replica/timer.rs +++ b/node/actors/bft/src/replica/timer.rs @@ -2,17 +2,21 @@ use super::StateMachine; use crate::metrics; use tracing::instrument; use zksync_concurrency::{ctx, metrics::LatencyGaugeExt as _, time}; +use zksync_consensus_roles::validator; impl StateMachine { /// The base duration of the timeout. - pub(crate) const BASE_DURATION: time::Duration = time::Duration::milliseconds(1000); + pub(crate) const BASE_DURATION: time::Duration = time::Duration::milliseconds(2000); /// Resets the timer. On every timeout we double the duration, starting from a given base duration. /// This is a simple exponential backoff. #[instrument(level = "trace", ret)] pub(crate) fn reset_timer(&mut self, ctx: &ctx::Ctx) { - let timeout = - Self::BASE_DURATION * 2u32.pow((self.view.0 - self.high_qc.message.view.0) as u32); + let final_view = match self.high_qc.as_ref() { + Some(qc) => qc.view().number.next(), + None => validator::ViewNumber(0), + }; + let timeout = Self::BASE_DURATION * 2u32.pow((self.view.0 - final_view.0) as u32); metrics::METRICS.replica_view_timeout.set_latency(timeout); self.timeout_deadline = time::Deadline::Finite(ctx.now() + timeout); diff --git a/node/actors/bft/src/testonly/fuzz.rs b/node/actors/bft/src/testonly/fuzz.rs index 2a48b72e..077be553 100644 --- a/node/actors/bft/src/testonly/fuzz.rs +++ b/node/actors/bft/src/testonly/fuzz.rs @@ -1,5 +1,9 @@ use crate::testonly::node::MAX_PAYLOAD_SIZE; -use rand::{seq::SliceRandom, Rng}; +use rand::{ + distributions::{Distribution, Standard}, + seq::SliceRandom, + Rng, +}; use zksync_consensus_roles::validator; /// Trait that allows us to mutate types. It's an approach to fuzzing that instead of starting with completely random inputs @@ -11,6 +15,19 @@ pub(crate) trait Fuzz { fn mutate(&mut self, rng: &mut impl Rng); } +impl Fuzz for Option +where + Standard: Distribution, +{ + fn mutate(&mut self, rng: &mut impl Rng) { + if let Some(v) = self.as_mut() { + v.mutate(rng); + } else { + *self = Some(rng.gen()); + } + } +} + impl Fuzz for validator::Signed { fn mutate(&mut self, rng: &mut impl Rng) { // We give them different weights because we want to mutate the message more often. @@ -36,33 +53,29 @@ impl Fuzz for validator::ConsensusMsg { impl Fuzz for validator::ReplicaPrepare { fn mutate(&mut self, rng: &mut impl Rng) { - match rng.gen_range(0..4) { + match rng.gen_range(0..3) { 0 => self.view = rng.gen(), 1 => self.high_vote.mutate(rng), - 2 => self.high_qc.mutate(rng), - 3 => self.protocol_version = rng.gen(), - _ => unreachable!(), + _ => self.high_qc.mutate(rng), } } } impl Fuzz for validator::ReplicaCommit { fn mutate(&mut self, rng: &mut impl Rng) { - match rng.gen_range(0..3) { + match rng.gen_range(0..2) { 0 => self.view = rng.gen(), - 1 => self.proposal.mutate(rng), - 2 => self.protocol_version = rng.gen(), - _ => unreachable!(), + _ => self.proposal.mutate(rng), } } } +// TODO: why payload is not fuzzed? impl Fuzz for validator::LeaderPrepare { fn mutate(&mut self, rng: &mut impl Rng) { - match rng.gen_range(0..3) { + match rng.gen_range(0..2) { 0 => self.proposal.mutate(rng), 1 => self.justification.mutate(rng), - 2 => self.protocol_version = rng.gen(), _ => unreachable!(), } } @@ -70,11 +83,7 @@ impl Fuzz for validator::LeaderPrepare { impl Fuzz for validator::LeaderCommit { fn mutate(&mut self, rng: &mut impl Rng) { - match rng.gen_range(0..2) { - 0 => self.justification.mutate(rng), - 1 => self.protocol_version = rng.gen(), - _ => unreachable!(), - } + self.justification.mutate(rng); } } diff --git a/node/actors/bft/src/testonly/node.rs b/node/actors/bft/src/testonly/node.rs index 6b221818..d6a25774 100644 --- a/node/actors/bft/src/testonly/node.rs +++ b/node/actors/bft/src/testonly/node.rs @@ -63,11 +63,9 @@ impl Node { let con_send = consensus_pipe.send; scope::run!(ctx, |ctx, s| async { s.spawn(async { - let validator_key = self.net.consensus.as_ref().unwrap().key.clone(); - let validator_set = self.net.validators.clone(); + let validator_key = self.net.validator_key.clone().unwrap(); crate::Config { secret_key: validator_key.clone(), - validator_set, block_store: self.block_store.clone(), replica_store: Box::new(in_memory::ReplicaStore::default()), payload_manager: self.behavior.payload_manager(), diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index e8f479a6..693f85ca 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -25,13 +25,13 @@ impl Test { /// Run a test with the given parameters. pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> anyhow::Result<()> { let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, self.nodes.len()); + let setup = validator::testonly::Setup::new(rng, self.nodes.len()); let nets: Vec<_> = network::testonly::new_configs(rng, &setup, 1); let mut nodes = vec![]; let mut honest = vec![]; scope::run!(ctx, |ctx, s| async { for (i, net) in nets.into_iter().enumerate() { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); if self.nodes[i] == Behavior::Honest { honest.push(store.clone()); @@ -46,10 +46,11 @@ impl Test { s.spawn_bg(run_nodes(ctx, self.network, &nodes)); // Run the nodes until all honest nodes store enough finalized blocks. - let want_block = validator::BlockNumber(self.blocks_to_finalize as u64); + let first = setup.genesis.fork.first_block; + let want_next = validator::BlockNumber(first.0 + self.blocks_to_finalize as u64); for store in &honest { sync::wait_for(ctx, &mut store.subscribe(), |state| { - state.next() > want_block + state.next() > want_next }) .await?; } @@ -76,6 +77,7 @@ async fn run_nodes(ctx: &ctx::Ctx, network: Network, specs: &[Node]) -> anyhow:: let mut nodes = vec![]; for (i, spec) in specs.iter().enumerate() { let (node, runner) = network::testonly::Instance::new( + ctx, spec.net.clone(), spec.block_store.clone(), ); @@ -99,7 +101,7 @@ async fn run_nodes(ctx: &ctx::Ctx, network: Network, specs: &[Node]) -> anyhow:: let mut recvs = vec![]; for (i, spec) in specs.iter().enumerate() { let (actor_pipe, pipe) = pipe::new(); - let key = spec.net.consensus.as_ref().unwrap().key.public(); + let key = spec.net.validator_key.as_ref().unwrap().public(); sends.insert(key, actor_pipe.send); recvs.push(actor_pipe.recv); s.spawn( diff --git a/node/actors/bft/src/testonly/ut_harness.rs b/node/actors/bft/src/testonly/ut_harness.rs index 22eda0a1..8880788c 100644 --- a/node/actors/bft/src/testonly/ut_harness.rs +++ b/node/actors/bft/src/testonly/ut_harness.rs @@ -1,9 +1,9 @@ use crate::{ io::OutputMessage, leader, - leader::{ReplicaCommitError, ReplicaPrepareError}, + leader::{replica_commit, replica_prepare}, replica, - replica::{LeaderCommitError, LeaderPrepareError}, + replica::{leader_commit, leader_prepare}, testonly, Config, PayloadManager, }; use assert_matches::assert_matches; @@ -55,13 +55,12 @@ impl UTHarness { payload_manager: Box, ) -> (UTHarness, BlockStoreRunner) { let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, num_validators); - let (block_store, runner) = new_store(ctx, &setup.blocks[0]).await; + let setup = validator::testonly::Setup::new(rng, num_validators); + let (block_store, runner) = new_store(ctx, &setup.genesis).await; let (send, recv) = ctx::channel::unbounded(); let cfg = Arc::new(Config { secret_key: setup.keys[0].clone(), - validator_set: setup.validator_set(), block_store: block_store.clone(), replica_store: Box::new(in_memory::ReplicaStore::default()), payload_manager, @@ -75,7 +74,7 @@ impl UTHarness { leader, replica, pipe: recv, - keys: setup.keys, + keys: setup.keys.clone(), }; let _: Signed = this.try_recv().unwrap(); (this, runner) @@ -84,7 +83,7 @@ impl UTHarness { /// Creates a new `UTHarness` with minimally-significant validator set size. pub(crate) async fn new_many(ctx: &ctx::Ctx) -> (UTHarness, BlockStoreRunner) { let num_validators = 6; - assert!(crate::misc::faulty_replicas(num_validators) > 0); + assert!(validator::faulty_replicas(num_validators) > 0); UTHarness::new(ctx, num_validators).await } @@ -93,16 +92,23 @@ impl UTHarness { /// recovers after a timeout. pub(crate) async fn produce_block_after_timeout(&mut self, ctx: &ctx::Ctx) { let want = ReplicaPrepare { - protocol_version: self.protocol_version(), - view: self.replica.view.next(), + view: validator::View { + protocol_version: self.protocol_version(), + fork: self.genesis().fork.number, + number: self.replica.view.next(), + }, high_qc: self.replica.high_qc.clone(), - high_vote: self.replica.high_vote, + high_vote: self.replica.high_vote.clone(), }; let replica_prepare = self.process_replica_timeout(ctx).await; assert_eq!(want, replica_prepare.msg); + self.produce_block(ctx).await; + } - let leader_commit = self.new_leader_commit(ctx).await; - self.process_leader_commit(ctx, leader_commit) + /// Produces a block, by executing the full view. + pub(crate) async fn produce_block(&mut self, ctx: &ctx::Ctx) { + let msg = self.new_leader_commit(ctx).await; + self.process_leader_commit(ctx, self.sign(msg)) .await .unwrap(); } @@ -119,6 +125,10 @@ impl UTHarness { &self.replica.config.secret_key } + pub(crate) fn sign>(&self, msg: V) -> Signed { + self.replica.config.secret_key.sign_msg(msg) + } + pub(crate) fn set_owner_as_view_leader(&mut self) { let mut view = self.replica.view; while self.view_leader(view) != self.owner_key().public() { @@ -140,57 +150,53 @@ impl UTHarness { self.replica.view = view } - pub(crate) fn new_replica_prepare( - &mut self, - mutate_fn: impl FnOnce(&mut ReplicaPrepare), - ) -> Signed { - self.set_owner_as_view_leader(); - let mut msg = ReplicaPrepare { + pub(crate) fn replica_view(&self) -> validator::View { + validator::View { protocol_version: self.protocol_version(), - view: self.replica.view, - high_vote: self.replica.high_vote, + fork: self.genesis().fork.number, + number: self.replica.view, + } + } + + pub(crate) fn new_replica_prepare(&mut self) -> ReplicaPrepare { + self.set_owner_as_view_leader(); + ReplicaPrepare { + view: self.replica_view(), + high_vote: self.replica.high_vote.clone(), high_qc: self.replica.high_qc.clone(), - }; - mutate_fn(&mut msg); - self.owner_key().sign_msg(msg) + } } - pub(crate) fn new_current_replica_commit( - &self, - mutate_fn: impl FnOnce(&mut ReplicaCommit), - ) -> Signed { - let mut msg = ReplicaCommit { - protocol_version: self.protocol_version(), - view: self.replica.view, - proposal: self.replica.high_qc.message.proposal, - }; - mutate_fn(&mut msg); - self.owner_key().sign_msg(msg) + pub(crate) fn new_current_replica_commit(&self) -> ReplicaCommit { + ReplicaCommit { + view: self.replica_view(), + proposal: self.replica.high_qc.as_ref().unwrap().message.proposal, + } } - pub(crate) async fn new_leader_prepare(&mut self, ctx: &ctx::Ctx) -> Signed { - let replica_prepare = self.new_replica_prepare(|_| {}).msg; - self.process_replica_prepare_all(ctx, replica_prepare).await + pub(crate) async fn new_leader_prepare(&mut self, ctx: &ctx::Ctx) -> LeaderPrepare { + let msg = self.new_replica_prepare(); + self.process_replica_prepare_all(ctx, msg).await.msg } - pub(crate) async fn new_replica_commit(&mut self, ctx: &ctx::Ctx) -> Signed { - let leader_prepare = self.new_leader_prepare(ctx).await; - self.process_leader_prepare(ctx, leader_prepare) + pub(crate) async fn new_replica_commit(&mut self, ctx: &ctx::Ctx) -> ReplicaCommit { + let msg = self.new_leader_prepare(ctx).await; + self.process_leader_prepare(ctx, self.sign(msg)) .await .unwrap() + .msg } - pub(crate) async fn new_leader_commit(&mut self, ctx: &ctx::Ctx) -> Signed { - let replica_commit = self.new_replica_commit(ctx).await; - self.process_replica_commit_all(ctx, replica_commit.msg) - .await + pub(crate) async fn new_leader_commit(&mut self, ctx: &ctx::Ctx) -> LeaderCommit { + let msg = self.new_replica_commit(ctx).await; + self.process_replica_commit_all(ctx, msg).await.msg } pub(crate) async fn process_leader_prepare( &mut self, ctx: &ctx::Ctx, msg: Signed, - ) -> Result, LeaderPrepareError> { + ) -> Result, leader_prepare::Error> { self.replica.process_leader_prepare(ctx, msg).await?; Ok(self.try_recv().unwrap()) } @@ -199,7 +205,7 @@ impl UTHarness { &mut self, ctx: &ctx::Ctx, msg: Signed, - ) -> Result, LeaderCommitError> { + ) -> Result, leader_commit::Error> { self.replica.process_leader_commit(ctx, msg).await?; Ok(self.try_recv().unwrap()) } @@ -209,7 +215,7 @@ impl UTHarness { &mut self, ctx: &ctx::Ctx, msg: Signed, - ) -> Result>, ReplicaPrepareError> { + ) -> Result>, replica_prepare::Error> { let prepare_qc = self.leader.prepare_qc.subscribe(); self.leader.process_replica_prepare(ctx, msg).await?; if prepare_qc.has_changed().unwrap() { @@ -231,7 +237,7 @@ impl UTHarness { ctx: &ctx::Ctx, msg: ReplicaPrepare, ) -> Signed { - let want_threshold = self.replica.config.threshold(); + let want_threshold = self.genesis().validators.threshold(); let mut leader_prepare = None; let msgs: Vec<_> = self.keys.iter().map(|k| k.sign_msg(msg.clone())).collect(); for (i, msg) in msgs.into_iter().enumerate() { @@ -239,7 +245,7 @@ impl UTHarness { match (i + 1).cmp(&want_threshold) { Ordering::Equal => leader_prepare = res.unwrap(), Ordering::Less => assert!(res.unwrap().is_none()), - Ordering::Greater => assert_matches!(res, Err(ReplicaPrepareError::Old { .. })), + Ordering::Greater => assert_matches!(res, Err(replica_prepare::Error::Old { .. })), } } leader_prepare.unwrap() @@ -249,7 +255,7 @@ impl UTHarness { &mut self, ctx: &ctx::Ctx, msg: Signed, - ) -> Result>, ReplicaCommitError> { + ) -> Result>, replica_commit::Error> { self.leader.process_replica_commit(ctx, msg)?; Ok(self.try_recv()) } @@ -260,12 +266,14 @@ impl UTHarness { msg: ReplicaCommit, ) -> Signed { for (i, key) in self.keys.iter().enumerate() { - let res = self.leader.process_replica_commit(ctx, key.sign_msg(msg)); - let want_threshold = self.replica.config.threshold(); + let res = self + .leader + .process_replica_commit(ctx, key.sign_msg(msg.clone())); + let want_threshold = self.genesis().validators.threshold(); match (i + 1).cmp(&want_threshold) { Ordering::Equal => res.unwrap(), Ordering::Less => res.unwrap(), - Ordering::Greater => assert_matches!(res, Err(ReplicaCommitError::Old { .. })), + Ordering::Greater => assert_matches!(res, Err(replica_commit::Error::Old { .. })), } } self.try_recv().unwrap() @@ -292,25 +300,33 @@ impl UTHarness { } pub(crate) fn view_leader(&self, view: ViewNumber) -> validator::PublicKey { - self.replica.config.view_leader(view) + self.genesis().validators.view_leader(view) } - pub(crate) fn validator_set(&self) -> validator::ValidatorSet { - validator::ValidatorSet::new(self.keys.iter().map(|k| k.public())).unwrap() + pub(crate) fn genesis(&self) -> &validator::Genesis { + self.replica.config.genesis() } pub(crate) fn new_commit_qc(&self, mutate_fn: impl FnOnce(&mut ReplicaCommit)) -> CommitQC { - let msg = self.new_current_replica_commit(mutate_fn).msg; - let msgs: Vec<_> = self.keys.iter().map(|k| k.sign_msg(msg)).collect(); - CommitQC::from(&msgs, &self.validator_set()).unwrap() + let mut msg = self.new_current_replica_commit(); + mutate_fn(&mut msg); + let mut qc = CommitQC::new(msg, self.genesis()); + for key in &self.keys { + qc.add(&key.sign_msg(qc.message.clone()), self.genesis()); + } + qc } pub(crate) fn new_prepare_qc( &mut self, mutate_fn: impl FnOnce(&mut ReplicaPrepare), ) -> PrepareQC { - let msg = self.new_replica_prepare(mutate_fn).msg; - let msgs: Vec<_> = self.keys.iter().map(|k| k.sign_msg(msg.clone())).collect(); - PrepareQC::from(&msgs, &self.validator_set()).unwrap() + let mut msg = self.new_replica_prepare(); + mutate_fn(&mut msg); + let mut qc = PrepareQC::new(msg.view.clone()); + for key in &self.keys { + qc.add(&key.sign_msg(msg.clone()), self.genesis()); + } + qc } } diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index 0ca95e7f..e6bb83b5 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -1,17 +1,15 @@ -use crate::{ - misc::consensus_threshold, - testonly::{ut_harness::UTHarness, Behavior, Network, Test}, -}; -use zksync_concurrency::{ctx, scope}; -use zksync_consensus_roles::validator::Phase; +use crate::testonly::{ut_harness::UTHarness, Behavior, Network, Test}; +use zksync_concurrency::{ctx, scope, time}; +use zksync_consensus_roles::validator; async fn run_test(behavior: Behavior, network: Network) { + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(20)); zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::AffineClock::new(1.)); + let ctx = &ctx::test_root(&ctx::RealClock); const NODES: usize = 11; let mut nodes = vec![behavior; NODES]; - for n in &mut nodes[0..consensus_threshold(NODES)] { + for n in &mut nodes[0..validator::threshold(NODES)] { *n = Behavior::Honest; } Test { @@ -64,7 +62,7 @@ async fn byzantine_real_network() { run_test(Behavior::Byzantine, Network::Real).await } -// Testing liveness after the network becomes idle with leader having no cached prepare messages for the current view. +/// Testing liveness after the network becomes idle with leader having no cached prepare messages for the current view. #[tokio::test] async fn timeout_leader_no_prepares() { zksync_concurrency::testonly::abort_on_panic(); @@ -72,8 +70,7 @@ async fn timeout_leader_no_prepares() { scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - - util.new_replica_prepare(|_| {}); + util.new_replica_prepare(); util.produce_block_after_timeout(ctx).await; Ok(()) }) @@ -89,10 +86,9 @@ async fn timeout_leader_some_prepares() { scope::run!(ctx, |ctx, s| async { let (mut util, runner) = UTHarness::new_many(ctx).await; s.spawn_bg(runner.run(ctx)); - - let replica_prepare = util.new_replica_prepare(|_| {}); + let replica_prepare = util.new_replica_prepare(); assert!(util - .process_replica_prepare(ctx, replica_prepare) + .process_replica_prepare(ctx, util.sign(replica_prepare)) .await .unwrap() .is_none()); @@ -114,7 +110,7 @@ async fn timeout_leader_in_commit() { util.new_leader_prepare(ctx).await; // Leader is in `Phase::Commit`, but should still accept prepares from newer views. - assert_eq!(util.leader.phase, Phase::Commit); + assert_eq!(util.leader.phase, validator::Phase::Commit); util.produce_block_after_timeout(ctx).await; Ok(()) }) @@ -133,7 +129,7 @@ async fn timeout_replica_in_commit() { util.new_replica_commit(ctx).await; // Leader is in `Phase::Commit`, but should still accept prepares from newer views. - assert_eq!(util.leader.phase, Phase::Commit); + assert_eq!(util.leader.phase, validator::Phase::Commit); util.produce_block_after_timeout(ctx).await; Ok(()) }) @@ -152,12 +148,12 @@ async fn timeout_leader_some_commits() { let replica_commit = util.new_replica_commit(ctx).await; assert!(util - .process_replica_commit(ctx, replica_commit) + .process_replica_commit(ctx, util.sign(replica_commit)) .await .unwrap() .is_none()); // Leader is in `Phase::Commit`, but should still accept prepares from newer views. - assert_eq!(util.leader_phase(), Phase::Commit); + assert_eq!(util.leader_phase(), validator::Phase::Commit); util.produce_block_after_timeout(ctx).await; Ok(()) }) diff --git a/node/actors/executor/src/lib.rs b/node/actors/executor/src/lib.rs index 08065a2a..2e0fad85 100644 --- a/node/actors/executor/src/lib.rs +++ b/node/actors/executor/src/lib.rs @@ -6,7 +6,7 @@ use std::{ fmt, sync::Arc, }; -use zksync_concurrency::{ctx, net, scope}; +use zksync_concurrency::{ctx, net, scope, time}; use zksync_consensus_bft as bft; use zksync_consensus_network as network; use zksync_consensus_roles::{node, validator}; @@ -16,16 +16,13 @@ use zksync_consensus_utils::pipe; use zksync_protobuf::kB; mod io; -pub mod testonly; #[cfg(test)] mod tests; -pub use network::consensus::Config as ValidatorConfig; - /// Validator-related part of [`Executor`]. pub struct Validator { /// Consensus network configuration. - pub config: ValidatorConfig, + pub key: validator::SecretKey, /// Store for replica state. pub replica_store: Box, /// Payload manager. @@ -35,7 +32,7 @@ pub struct Validator { impl fmt::Debug for Validator { fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { fmt.debug_struct("ValidatorExecutor") - .field("config", &self.config) + .field("key", &self.key) .finish() } } @@ -46,9 +43,9 @@ pub struct Config { /// IP:port to listen on, for incoming TCP connections. /// Use `0.0.0.0:` to listen on all network interfaces (i.e. on all IPs exposed by this VM). pub server_addr: std::net::SocketAddr, - /// Static specification of validators for Proof of Authority. Should be deprecated once we move - /// to Proof of Stake. - pub validators: validator::ValidatorSet, + /// Public TCP address that other nodes are expected to connect to. + /// It is announced over gossip network. + pub public_addr: std::net::SocketAddr, /// Maximal size of the block payload. pub max_payload_size: usize, @@ -67,8 +64,8 @@ pub struct Config { impl Config { /// Returns gossip network configuration. - pub(crate) fn gossip(&self) -> network::gossip::Config { - network::gossip::Config { + pub(crate) fn gossip(&self) -> network::GossipConfig { + network::GossipConfig { key: self.node_key.clone(), dynamic_inbound_limit: self.gossip_dynamic_inbound_limit, static_inbound: self.gossip_static_inbound.clone(), @@ -93,11 +90,12 @@ impl Executor { fn network_config(&self) -> network::Config { network::Config { server_addr: net::tcp::ListenerAddr::new(self.config.server_addr), - validators: self.config.validators.clone(), + public_addr: self.config.public_addr, gossip: self.config.gossip(), - consensus: self.validator.as_ref().map(|v| v.config.clone()), - enable_pings: true, + validator_key: self.validator.as_ref().map(|v| v.key.clone()), + ping_timeout: Some(time::Duration::seconds(10)), max_block_size: self.config.max_payload_size.saturating_add(kB), + rpc: network::RpcConfig::default(), } } @@ -105,10 +103,11 @@ impl Executor { fn verify(&self) -> anyhow::Result<()> { if let Some(validator) = self.validator.as_ref() { if !self - .config + .block_store + .genesis() .validators .iter() - .any(|key| key == &validator.config.key.public()) + .any(|key| key == &validator.key.public()) { anyhow::bail!("this validator doesn't belong to the consensus"); } @@ -132,26 +131,24 @@ impl Executor { network_dispatcher_pipe, ); - // Create each of the actors. - let validator_set = self.config.validators; - tracing::debug!("Starting actors in separate threads."); scope::run!(ctx, |ctx, s| async { s.spawn_blocking(|| dispatcher.run(ctx).context("IO Dispatcher stopped")); s.spawn(async { - let state = network::State::new(network_config, self.block_store.clone(), None) - .context("Invalid network config")?; - state.register_metrics(); - network::run_network(ctx, state, network_actor_pipe) - .await - .context("Network stopped") + let (net, runner) = network::Network::new( + ctx, + network_config, + self.block_store.clone(), + network_actor_pipe, + ); + net.register_metrics(); + runner.run(ctx).await.context("Network stopped") }); if let Some(validator) = self.validator { s.spawn(async { let validator = validator; bft::Config { - secret_key: validator.config.key.clone(), - validator_set: validator_set.clone(), + secret_key: validator.key.clone(), block_store: self.block_store.clone(), replica_store: validator.replica_store, payload_manager: validator.payload_manager, @@ -162,13 +159,10 @@ impl Executor { .context("Consensus stopped") }); } - sync_blocks::Config::new( - validator_set.clone(), - bft::misc::consensus_threshold(validator_set.len()), - )? - .run(ctx, sync_blocks_actor_pipe, self.block_store.clone()) - .await - .context("Syncing blocks stopped") + sync_blocks::Config::new() + .run(ctx, sync_blocks_actor_pipe, self.block_store.clone()) + .await + .context("Syncing blocks stopped") }) .await } diff --git a/node/actors/executor/src/testonly.rs b/node/actors/executor/src/testonly.rs deleted file mode 100644 index b40caa66..00000000 --- a/node/actors/executor/src/testonly.rs +++ /dev/null @@ -1,53 +0,0 @@ -//! Testing extensions for node executor. -use crate::{Config, ValidatorConfig}; -use rand::Rng; -use zksync_concurrency::net; -use zksync_consensus_network as network; -use zksync_consensus_roles::validator::testonly::GenesisSetup; - -/// Full validator configuration. -#[derive(Debug, Clone)] -#[non_exhaustive] -pub struct ValidatorNode { - /// Full node configuration. - pub node: Config, - /// Consensus configuration of the validator. - pub validator: ValidatorConfig, - /// Genesis configuration (validator set & initial blocks). - pub setup: GenesisSetup, -} - -/// Creates a new full node and configures this validator to accept incoming connections from it. -pub fn connect_full_node(rng: &mut impl Rng, node: &mut Config) -> Config { - let mut new = node.clone(); - new.server_addr = *net::tcp::testonly::reserve_listener(); - new.node_key = rng.gen(); - new.gossip_static_outbound = [(node.node_key.public(), node.server_addr)].into(); - node.gossip_static_inbound.insert(new.node_key.public()); - new -} - -impl ValidatorNode { - /// Generates a validator config for a network with a single validator. - pub fn new(rng: &mut impl Rng) -> Self { - let setup = GenesisSetup::new(rng, 1); - let net_config = network::testonly::new_configs(rng, &setup, 0) - .pop() - .unwrap(); - let validator = net_config.consensus.unwrap(); - let gossip = net_config.gossip; - Self { - node: Config { - server_addr: *net_config.server_addr, - validators: setup.validator_set(), - node_key: gossip.key, - gossip_dynamic_inbound_limit: gossip.dynamic_inbound_limit, - gossip_static_inbound: gossip.static_inbound, - gossip_static_outbound: gossip.static_outbound, - max_payload_size: usize::MAX, - }, - validator, - setup, - } - } -} diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index d85ca660..f95b6641 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -1,41 +1,31 @@ //! High-level tests for `Executor`. - use super::*; -use crate::testonly::{connect_full_node, ValidatorNode}; -use test_casing::test_casing; -use zksync_concurrency::{ - sync, - testonly::{abort_on_panic, set_timeout}, - time, -}; +use zksync_concurrency::testonly::abort_on_panic; use zksync_consensus_bft as bft; -use zksync_consensus_roles::validator::BlockNumber; +use zksync_consensus_network::testonly::{new_configs, new_fullnode}; +use zksync_consensus_roles::validator::{testonly::Setup, BlockNumber}; use zksync_consensus_storage::{ testonly::{in_memory, new_store}, BlockStore, }; -impl Config { - fn into_executor(self, block_store: Arc) -> Executor { - Executor { - config: self, - block_store, - validator: None, - } - } -} - -impl ValidatorNode { - fn into_executor(self, block_store: Arc) -> Executor { - Executor { - config: self.node, - block_store, - validator: Some(Validator { - config: self.validator, - replica_store: Box::new(in_memory::ReplicaStore::default()), - payload_manager: Box::new(bft::testonly::RandomPayload(1000)), - }), - } +fn make_executor(cfg: &network::Config, block_store: Arc) -> Executor { + Executor { + config: Config { + server_addr: *cfg.server_addr, + public_addr: cfg.public_addr, + max_payload_size: usize::MAX, + node_key: cfg.gossip.key.clone(), + gossip_dynamic_inbound_limit: cfg.gossip.dynamic_inbound_limit, + gossip_static_inbound: cfg.gossip.static_inbound.clone(), + gossip_static_outbound: cfg.gossip.static_outbound.clone(), + }, + block_store, + validator: cfg.validator_key.as_ref().map(|key| Validator { + key: key.clone(), + replica_store: Box::new(in_memory::ReplicaStore::default()), + payload_manager: Box::new(bft::testonly::RandomPayload(1000)), + }), } } @@ -45,15 +35,13 @@ async fn executing_single_validator() { let ctx = &ctx::root(); let rng = &mut ctx.rng(); - let validator = ValidatorNode::new(rng); - let (storage, runner) = new_store(ctx, &validator.setup.blocks[0]).await; - let executor = validator.into_executor(storage.clone()); - + let setup = Setup::new(rng, 1); + let cfgs = new_configs(rng, &setup, 0); scope::run!(ctx, |ctx, s| async { + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); - s.spawn_bg(executor.run(ctx)); - let want = BlockNumber(5); - sync::wait_for(ctx, &mut storage.subscribe(), |state| state.next() > want).await?; + s.spawn_bg(make_executor(&cfgs[0], store.clone()).run(ctx)); + store.wait_until_persisted(ctx, BlockNumber(5)).await?; Ok(()) }) .await @@ -66,85 +54,22 @@ async fn executing_validator_and_full_node() { let ctx = &ctx::test_root(&ctx::AffineClock::new(20.0)); let rng = &mut ctx.rng(); - let mut validator = ValidatorNode::new(rng); - let full_node = connect_full_node(rng, &mut validator.node); - let (validator_storage, validator_runner) = new_store(ctx, &validator.setup.blocks[0]).await; - let (full_node_storage, full_node_runner) = new_store(ctx, &validator.setup.blocks[0]).await; - - let validator = validator.into_executor(validator_storage.clone()); - let full_node = full_node.into_executor(full_node_storage.clone()); - - scope::run!(ctx, |ctx, s| async { - s.spawn_bg(validator_runner.run(ctx)); - s.spawn_bg(full_node_runner.run(ctx)); - s.spawn_bg(validator.run(ctx)); - s.spawn_bg(full_node.run(ctx)); - full_node_storage - .wait_until_persisted(ctx, BlockNumber(5)) - .await?; - Ok(()) - }) - .await - .unwrap(); -} - -#[test_casing(2, [false, true])] -#[tokio::test] -async fn syncing_full_node_from_snapshot(delay_block_storage: bool) { - abort_on_panic(); - let _guard = set_timeout(time::Duration::seconds(10)); - - let ctx = &ctx::test_root(&ctx::AffineClock::new(20.0)); - let rng = &mut ctx.rng(); - - let mut validator = ValidatorNode::new(rng); - validator.setup.push_blocks(rng, 10); - let node2 = connect_full_node(rng, &mut validator.node); - - let (store1, store1_runner) = new_store(ctx, &validator.setup.blocks[0]).await; - // Node2 will start from a snapshot. - let (store2, store2_runner) = new_store(ctx, &validator.setup.blocks[4]).await; - - // We spawn 2 non-validator nodes. We will simulate blocks appearing in storage of node1, - // and will expect them to be propagated to node2. - let node1 = validator.node.into_executor(store1.clone()); - let node2 = Executor { - config: node2, - block_store: store2.clone(), - validator: None, - }; - + let setup = Setup::new(rng, 1); + let cfgs = new_configs(rng, &setup, 0); scope::run!(ctx, |ctx, s| async { - s.spawn_bg(store1_runner.run(ctx)); - s.spawn_bg(store2_runner.run(ctx)); - if !delay_block_storage { - // Instead of running consensus on the validator, add the generated blocks manually. - for block in &validator.setup.blocks[1..] { - store1.queue_block(ctx, block.clone()).await.unwrap(); - } - } - s.spawn_bg(node1.run(ctx)); - s.spawn_bg(node2.run(ctx)); - - if delay_block_storage { - // Emulate the validator gradually adding new blocks to the storage. - s.spawn_bg(async { - for block in &validator.setup.blocks[1..] { - ctx.sleep(time::Duration::milliseconds(500)).await?; - store1.queue_block(ctx, block.clone()).await?; - } - Ok(()) - }); - } + // Spawn validator. + let (store, runner) = new_store(ctx, &setup.genesis).await; + s.spawn_bg(runner.run(ctx)); + s.spawn_bg(make_executor(&cfgs[0], store).run(ctx)); - store2.wait_until_persisted(ctx, BlockNumber(10)).await?; + // Spawn full node. + let (store, runner) = new_store(ctx, &setup.genesis).await; + s.spawn_bg(runner.run(ctx)); + s.spawn_bg(make_executor(&new_fullnode(rng, &cfgs[0]), store.clone()).run(ctx)); - // Check that the node didn't receive any blocks with number lesser than the initial snapshot block. - for lesser_block_number in 0..3 { - let block = store2.block(ctx, BlockNumber(lesser_block_number)).await?; - assert!(block.is_none()); - } - anyhow::Ok(()) + // Wait for blocks in full node store. + store.wait_until_persisted(ctx, BlockNumber(5)).await?; + Ok(()) }) .await .unwrap(); diff --git a/node/actors/network/Cargo.toml b/node/actors/network/Cargo.toml index 0edd257c..e1e835a9 100644 --- a/node/actors/network/Cargo.toml +++ b/node/actors/network/Cargo.toml @@ -27,6 +27,7 @@ tracing.workspace = true vise.workspace = true [dev-dependencies] +assert_matches.workspace = true pretty_assertions.workspace = true test-casing.workspace = true tokio.workspace = true diff --git a/node/actors/network/src/config.rs b/node/actors/network/src/config.rs new file mode 100644 index 00000000..27d6c498 --- /dev/null +++ b/node/actors/network/src/config.rs @@ -0,0 +1,82 @@ +//! Network actor configs. +use std::collections::{HashMap, HashSet}; +use zksync_concurrency::{limiter, net, time}; +use zksync_consensus_roles::{node, validator}; + +/// How often we should retry to establish a connection to a validator. +/// TODO(gprusak): once it becomes relevant, choose a more appropriate retry strategy. +pub(crate) const CONNECT_RETRY: time::Duration = time::Duration::seconds(20); + +/// Rate limiting config for RPCs. +#[derive(Debug, Clone)] +pub struct RpcConfig { + /// Max rate of sending/receiving push_validator_addrs messages. + pub push_validator_addrs_rate: limiter::Rate, + /// Max rate of sending/receiving push_block_store_state messages. + pub push_block_store_state_rate: limiter::Rate, + /// Max rate of sending/receiving get_block RPCs. + pub get_block_rate: limiter::Rate, + /// Max rate of sending/receiving consensus messages. + pub consensus_rate: limiter::Rate, +} + +impl Default for RpcConfig { + fn default() -> Self { + Self { + push_validator_addrs_rate: limiter::Rate { + burst: 1, + refresh: time::Duration::seconds(5), + }, + push_block_store_state_rate: limiter::Rate { + burst: 2, + refresh: time::Duration::milliseconds(500), + }, + get_block_rate: limiter::Rate { + burst: 10, + refresh: time::Duration::milliseconds(100), + }, + consensus_rate: limiter::Rate { + burst: 10, + refresh: time::Duration::ZERO, + }, + } + } +} + +/// Gossip network configuration. +#[derive(Debug, Clone)] +pub struct GossipConfig { + /// Private key of the node, every node should have one. + pub key: node::SecretKey, + /// Limit on the number of inbound connections outside + /// of the `static_inbound` set. + pub dynamic_inbound_limit: usize, + /// Inbound connections that should be unconditionally accepted. + pub static_inbound: HashSet, + /// Outbound connections that the node should actively try to + /// establish and maintain. + pub static_outbound: HashMap, +} + +/// Network actor config. +#[derive(Debug, Clone)] +pub struct Config { + /// TCP socket address to listen for inbound connections at. + pub server_addr: net::tcp::ListenerAddr, + /// Public TCP address that other nodes are expected to connect to. + /// It is announced over gossip network. + pub public_addr: std::net::SocketAddr, + /// Gossip network config. + pub gossip: GossipConfig, + /// Private key of the validator. + /// None if the node is NOT a validator. + pub validator_key: Option, + /// Maximal size of the proto-encoded `validator::FinalBlock` in bytes. + pub max_block_size: usize, + /// If a peer doesn't respond to a ping message within `ping_timeout`, + /// the connection is dropped. + /// `None` disables sending ping messages (useful for tests). + pub ping_timeout: Option, + /// Rate limiting config for RPCs. + pub rpc: RpcConfig, +} diff --git a/node/actors/network/src/consensus/handshake/mod.rs b/node/actors/network/src/consensus/handshake/mod.rs index 9536c2d3..06de4a4e 100644 --- a/node/actors/network/src/consensus/handshake/mod.rs +++ b/node/actors/network/src/consensus/handshake/mod.rs @@ -19,6 +19,9 @@ pub(crate) struct Handshake { /// Session ID signed with the validator key. /// Authenticates the peer to be the owner of the validator key. pub(crate) session_id: validator::Signed, + /// Hash of the blockchain genesis specification. + /// Only nodes with the same genesis belong to the same network. + pub(crate) genesis: validator::GenesisHash, } impl ProtoFmt for Handshake { @@ -26,11 +29,13 @@ impl ProtoFmt for Handshake { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { session_id: read_required(&r.session_id).context("session_id")?, + genesis: read_required(&r.genesis).context("genesis")?, }) } fn build(&self) -> Self::Proto { Self::Proto { session_id: Some(self.session_id.build()), + genesis: Some(self.genesis.build()), } } } @@ -38,6 +43,8 @@ impl ProtoFmt for Handshake { /// Error returned by handshake logic. #[derive(Debug, thiserror::Error)] pub(super) enum Error { + #[error("genesis mismatch")] + GenesisMismatch, #[error("session id mismatch")] SessionIdMismatch, #[error("unexpected peer")] @@ -51,6 +58,7 @@ pub(super) enum Error { pub(super) async fn outbound( ctx: &ctx::Ctx, me: &validator::SecretKey, + genesis: validator::GenesisHash, stream: &mut noise::Stream, peer: &validator::PublicKey, ) -> Result<(), Error> { @@ -61,6 +69,7 @@ pub(super) async fn outbound( stream, &Handshake { session_id: me.sign_msg(session_id.clone()), + genesis, }, ) .await @@ -68,6 +77,9 @@ pub(super) async fn outbound( let h: Handshake = frame::recv_proto(ctx, stream, Handshake::max_size()) .await .map_err(Error::Stream)?; + if h.genesis != genesis { + return Err(Error::GenesisMismatch); + } if h.session_id.msg != session_id { return Err(Error::SessionIdMismatch); } @@ -81,6 +93,7 @@ pub(super) async fn outbound( pub(super) async fn inbound( ctx: &ctx::Ctx, me: &validator::SecretKey, + genesis: validator::GenesisHash, stream: &mut noise::Stream, ) -> Result { let ctx = &ctx.with_timeout(TIMEOUT); @@ -88,6 +101,9 @@ pub(super) async fn inbound( let h: Handshake = frame::recv_proto(ctx, stream, Handshake::max_size()) .await .map_err(Error::Stream)?; + if h.genesis != genesis { + return Err(Error::GenesisMismatch); + } if h.session_id.msg != session_id.clone() { return Err(Error::SessionIdMismatch); } @@ -97,6 +113,7 @@ pub(super) async fn inbound( stream, &Handshake { session_id: me.sign_msg(session_id.clone()), + genesis, }, ) .await diff --git a/node/actors/network/src/consensus/handshake/testonly.rs b/node/actors/network/src/consensus/handshake/testonly.rs index d05848df..05d5d8de 100644 --- a/node/actors/network/src/consensus/handshake/testonly.rs +++ b/node/actors/network/src/consensus/handshake/testonly.rs @@ -15,6 +15,7 @@ impl Distribution for Standard { let session_id: node::SessionId = rng.gen(); Handshake { session_id: key.sign_msg(session_id), + genesis: rng.gen(), } } } diff --git a/node/actors/network/src/consensus/handshake/tests.rs b/node/actors/network/src/consensus/handshake/tests.rs index 23bd75cd..ce69744b 100644 --- a/node/actors/network/src/consensus/handshake/tests.rs +++ b/node/actors/network/src/consensus/handshake/tests.rs @@ -1,5 +1,6 @@ use super::*; use crate::{frame, noise, testonly}; +use assert_matches::assert_matches; use rand::Rng; use zksync_concurrency::{ctx, io, scope, testonly::abort_on_panic}; use zksync_consensus_roles::validator; @@ -20,6 +21,8 @@ async fn test_session_id_mismatch() { let key0: validator::SecretKey = rng.gen(); let key1: validator::SecretKey = rng.gen(); + let genesis: validator::GenesisHash = rng.gen(); + // MitM attempt detected on the inbound end. scope::run!(ctx, |ctx, s| async { let (s1, s2) = noise::testonly::pipe(ctx).await; @@ -36,14 +39,14 @@ async fn test_session_id_mismatch() { }); s.spawn(async { let mut s4 = s4; - match inbound(ctx, &key0, &mut s4).await { + match inbound(ctx, &key0, genesis, &mut s4).await { Err(Error::SessionIdMismatch) => Ok(()), res => panic!("unexpected res: {res:?}"), } }); s.spawn(async { let mut s1 = s1; - match outbound(ctx, &key1, &mut s1, &key0.public()).await { + match outbound(ctx, &key1, genesis, &mut s1, &key0.public()).await { Err(Error::Stream(..)) => Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -64,12 +67,13 @@ async fn test_session_id_mismatch() { &mut s2, &Handshake { session_id: key1.sign_msg(rng.gen::()), + genesis, }, ) .await?; Ok(()) }); - match outbound(ctx, &key0, &mut s1, &key1.public()).await { + match outbound(ctx, &key0, genesis, &mut s1, &key1.public()).await { Err(Error::SessionIdMismatch) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -88,16 +92,18 @@ async fn test_peer_mismatch() { let key1: validator::SecretKey = rng.gen(); let key2: validator::SecretKey = rng.gen(); + let genesis: validator::GenesisHash = rng.gen(); + scope::run!(ctx, |ctx, s| async { let (s0, s1) = noise::testonly::pipe(ctx).await; s.spawn(async { let mut s0 = s0; - assert_eq!(key1.public(), inbound(ctx, &key0, &mut s0).await?); + assert_eq!(key1.public(), inbound(ctx, &key0, genesis, &mut s0).await?); Ok(()) }); s.spawn(async { let mut s1 = s1; - match outbound(ctx, &key1, &mut s1, &key2.public()).await { + match outbound(ctx, &key1, genesis, &mut s1, &key2.public()).await { Err(Error::PeerMismatch) => Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -108,6 +114,60 @@ async fn test_peer_mismatch() { .unwrap(); } +#[tokio::test] +async fn test_genesis_mismatch() { + abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + + let key0: validator::SecretKey = rng.gen(); + let key1: validator::SecretKey = rng.gen(); + + tracing::info!("test that inbound handshake rejects mismatching genesis"); + scope::run!(ctx, |ctx, s| async { + let (s0, mut s1) = noise::testonly::pipe(ctx).await; + s.spawn(async { + let mut s0 = s0; + let res = outbound(ctx, &key0, ctx.rng().gen(), &mut s0, &key1.public()).await; + assert_matches!(res, Err(Error::Stream(_))); + Ok(()) + }); + let res = inbound(ctx, &key1, rng.gen(), &mut s1).await; + assert_matches!(res, Err(Error::GenesisMismatch)); + anyhow::Ok(()) + }) + .await + .unwrap(); + + tracing::info!("test that outbound handshake rejects mismatching genesis"); + scope::run!(ctx, |ctx, s| async { + let (s0, mut s1) = noise::testonly::pipe(ctx).await; + s.spawn(async { + let mut s0 = s0; + let res = outbound(ctx, &key0, ctx.rng().gen(), &mut s0, &key1.public()).await; + assert_matches!(res, Err(Error::GenesisMismatch)); + Ok(()) + }); + let session_id = node::SessionId(s1.id().encode()); + let _: Handshake = frame::recv_proto(ctx, &mut s1, Handshake::max_size()) + .await + .unwrap(); + frame::send_proto( + ctx, + &mut s1, + &Handshake { + session_id: key1.sign_msg(session_id), + genesis: rng.gen(), + }, + ) + .await + .unwrap(); + anyhow::Ok(()) + }) + .await + .unwrap(); +} + #[tokio::test] async fn test_invalid_signature() { abort_on_panic(); @@ -117,6 +177,8 @@ async fn test_invalid_signature() { let key0: validator::SecretKey = rng.gen(); let key1: validator::SecretKey = rng.gen(); + let genesis: validator::GenesisHash = rng.gen(); + // Bad signature detected on outbound end. scope::run!(ctx, |ctx, s| async { let (mut s0, s1) = noise::testonly::pipe(ctx).await; @@ -127,7 +189,7 @@ async fn test_invalid_signature() { frame::send_proto(ctx, &mut s1, &h).await?; Ok(()) }); - match outbound(ctx, &key0, &mut s0, &key1.public()).await { + match outbound(ctx, &key0, genesis, &mut s0, &key1.public()).await { Err(Error::Signature(..)) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -142,11 +204,12 @@ async fn test_invalid_signature() { let mut s1 = s1; let mut h = Handshake { session_id: key0.sign_msg(node::SessionId(s1.id().encode())), + genesis, }; h.session_id.key = key1.public(); frame::send_proto(ctx, &mut s1, &h).await }); - match inbound(ctx, &key0, &mut s0).await { + match inbound(ctx, &key0, genesis, &mut s0).await { Err(Error::Signature(..)) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } diff --git a/node/actors/network/src/consensus/mod.rs b/node/actors/network/src/consensus/mod.rs index e6689b4f..32a90540 100644 --- a/node/actors/network/src/consensus/mod.rs +++ b/node/actors/network/src/consensus/mod.rs @@ -1,13 +1,238 @@ //! Consensus network is a full graph of connections between all validators. //! BFT consensus messages are exchanged over this network. +use crate::{config, gossip, io, noise, pool::PoolWatch, preface, rpc}; +use anyhow::Context as _; +use std::{ + collections::{HashMap, HashSet}, + sync::Arc, +}; +use zksync_concurrency::{ctx, oneshot, scope, sync, time}; +use zksync_consensus_roles::validator; +use zksync_protobuf::kB; + mod handshake; -mod runner; -mod state; #[cfg(test)] mod tests; -// Clippy doesn't care about visibility required only for tests. -pub(crate) use runner::*; -#[allow(clippy::wildcard_imports)] -pub use state::Config; -pub(crate) use state::*; +const RESP_MAX_SIZE: usize = kB; +/// Frequency at which the validator broadcasts its own IP address. +/// Although the IP is not likely to change while validator is running, +/// we do this periodically, so that the network can observe if validator +/// is down. +const ADDRESS_ANNOUNCER_INTERVAL: time::Duration = time::Duration::minutes(10); + +/// Consensus network state. +pub(crate) struct Network { + /// Gossip network state to bootstrap consensus network from. + pub(crate) gossip: Arc, + /// This validator's secret key. + pub(crate) key: validator::SecretKey, + /// Set of the currently open inbound connections. + pub(crate) inbound: PoolWatch, + /// Set of the currently open outbound connections. + pub(crate) outbound: PoolWatch, + /// RPC clients for all validators. + pub(crate) clients: HashMap>, +} + +#[async_trait::async_trait] +impl rpc::Handler for &Network { + /// Here we bound the buffering of incoming consensus messages. + fn max_req_size(&self) -> usize { + self.gossip.cfg.max_block_size.saturating_add(kB) + } + + async fn handle( + &self, + ctx: &ctx::Ctx, + req: rpc::consensus::Req, + ) -> anyhow::Result { + let (send, recv) = oneshot::channel(); + self.gossip + .sender + .send(io::OutputMessage::Consensus(io::ConsensusReq { + msg: req.0, + ack: send, + })); + recv.recv_or_disconnected(ctx).await??; + Ok(rpc::consensus::Resp) + } +} + +impl Network { + /// Constructs a new consensus network state. + pub(crate) fn new(ctx: &ctx::Ctx, gossip: Arc) -> Option> { + let key = gossip.cfg.validator_key.clone()?; + let validators: HashSet<_> = gossip.genesis().validators.iter().cloned().collect(); + Some(Arc::new(Self { + key, + inbound: PoolWatch::new(validators.clone(), 0), + outbound: PoolWatch::new(validators.clone(), 0), + clients: validators + .iter() + .map(|peer| { + ( + peer.clone(), + rpc::Client::new(ctx, gossip.cfg.rpc.consensus_rate), + ) + }) + .collect(), + gossip, + })) + } + + /// Sends a message to all validators. + pub(crate) async fn broadcast( + &self, + ctx: &ctx::Ctx, + msg: validator::Signed, + ) -> anyhow::Result<()> { + let req = rpc::consensus::Req(msg); + scope::run!(ctx, |ctx, s| async { + for (peer, client) in &self.clients { + s.spawn(async { + if let Err(err) = client.call(ctx, &req, RESP_MAX_SIZE).await { + tracing::info!("send({:?},): {err:#}", &*peer); + } + Ok(()) + }); + } + Ok(()) + }) + .await + } + + /// Sends a message to the given validator. + pub(crate) async fn send( + &self, + ctx: &ctx::Ctx, + key: &validator::PublicKey, + msg: validator::Signed, + ) -> anyhow::Result<()> { + let client = self.clients.get(key).context("not an active validator")?; + client + .call(ctx, &rpc::consensus::Req(msg), RESP_MAX_SIZE) + .await?; + Ok(()) + } + + /// Performs handshake of an inbound stream. + /// Closes the stream if there is another inbound stream opened from the same validator. + pub(crate) async fn run_inbound_stream( + &self, + ctx: &ctx::Ctx, + mut stream: noise::Stream, + ) -> anyhow::Result<()> { + let peer = + handshake::inbound(ctx, &self.key, self.gossip.genesis().hash(), &mut stream).await?; + self.inbound.insert(peer.clone()).await?; + let res = scope::run!(ctx, |ctx, s| async { + let mut service = rpc::Service::new() + .add_server(rpc::ping::Server, rpc::ping::RATE) + .add_server(self, self.gossip.cfg.rpc.consensus_rate); + if let Some(ping_timeout) = &self.gossip.cfg.ping_timeout { + let ping_client = rpc::Client::::new(ctx, rpc::ping::RATE); + service = service.add_client(&ping_client); + s.spawn(async { + let ping_client = ping_client; + ping_client.ping_loop(ctx, *ping_timeout).await + }); + } + service.run(ctx, stream).await?; + Ok(()) + }) + .await; + self.inbound.remove(&peer).await; + res + } + + async fn run_outbound_stream( + &self, + ctx: &ctx::Ctx, + peer: &validator::PublicKey, + addr: std::net::SocketAddr, + ) -> anyhow::Result<()> { + let client = self.clients.get(peer).context("not an active validator")?; + let mut stream = preface::connect(ctx, addr, preface::Endpoint::ConsensusNet).await?; + handshake::outbound( + ctx, + &self.key, + self.gossip.genesis().hash(), + &mut stream, + peer, + ) + .await?; + self.outbound.insert(peer.clone()).await?; + let res = scope::run!(ctx, |ctx, s| async { + let mut service = rpc::Service::new() + .add_server(rpc::ping::Server, rpc::ping::RATE) + .add_client(client); + if let Some(ping_timeout) = &self.gossip.cfg.ping_timeout { + let ping_client = rpc::Client::::new(ctx, rpc::ping::RATE); + service = service.add_client(&ping_client); + s.spawn(async { + let ping_client = ping_client; + ping_client.ping_loop(ctx, *ping_timeout).await + }); + } + service.run(ctx, stream).await?; + Ok(()) + }) + .await; + self.outbound.remove(peer).await; + res + } + + /// Maintains a connection to the given validator. + /// If connection breaks, it tries to reconnect periodically. + pub(crate) async fn maintain_connection(&self, ctx: &ctx::Ctx, peer: &validator::PublicKey) { + let addrs = &mut self.gossip.validator_addrs.subscribe(); + let mut addr = None; + while ctx.is_active() { + // Wait for a new address, or retry with the old one after timeout. + if let Ok(new) = + sync::wait_for(&ctx.with_timeout(config::CONNECT_RETRY), addrs, |addrs| { + addrs.get(peer).map(|x| x.msg.addr) != addr + }) + .await + { + addr = new.get(peer).map(|x| x.msg.addr); + } + let Some(addr) = addr else { continue }; + if let Err(err) = self.run_outbound_stream(ctx, peer, addr).await { + tracing::info!("run_outbound_stream({peer:?},{addr}): {err:#}"); + } + } + } + + /// Periodically announces this validator's public IP over gossip network, + /// so that other validators can discover and connect to this validator. + pub(crate) async fn run_address_announcer(&self, ctx: &ctx::Ctx) { + let my_addr = self.gossip.cfg.public_addr; + let mut sub = self.gossip.validator_addrs.subscribe(); + while ctx.is_active() { + let ctx = &ctx.with_timeout(ADDRESS_ANNOUNCER_INTERVAL); + let _ = sync::wait_for(ctx, &mut sub, |got| { + got.get(&self.key.public()).map(|x| &x.msg.addr) != Some(&my_addr) + }) + .await; + let next_version = sub + .borrow() + .get(&self.key.public()) + .map(|x| x.msg.version + 1) + .unwrap_or(0); + self.gossip + .validator_addrs + .update( + &self.gossip.genesis().validators, + &[Arc::new(self.key.sign_msg(validator::NetAddress { + addr: my_addr, + version: next_version, + timestamp: ctx.now_utc(), + }))], + ) + .await + .unwrap(); + } + } +} diff --git a/node/actors/network/src/consensus/runner.rs b/node/actors/network/src/consensus/runner.rs deleted file mode 100644 index ca988b0c..00000000 --- a/node/actors/network/src/consensus/runner.rs +++ /dev/null @@ -1,190 +0,0 @@ -//! run_client routine maintaining outbound connections to validators. - -use super::handshake; -use crate::{io, noise, preface, rpc, State}; -use anyhow::Context as _; -use std::{collections::HashMap, sync::Arc}; -use zksync_concurrency::{ctx, ctx::channel, oneshot, scope, sync, time}; -use zksync_consensus_roles::validator; -use zksync_protobuf::kB; - -/// How often we should retry to establish a connection to a validator. -/// TODO(gprusak): once it becomes relevant, choose a more appropriate retry strategy. -const CONNECT_RETRY: time::Duration = time::Duration::seconds(20); -/// A ping request is sent periodically. If ping response doesn't arrive -/// within PING_TIMEOUT, we close the connection. -const PING_TIMEOUT: time::Duration = time::Duration::seconds(10); -/// Each consensus message is expected to be delivered within MSG_TIMEOUT. -/// After that time the message is dropped. -/// TODO(gprusak): for liveness we should retry sending messages until the view -/// changes. That requires tighter integration with the consensus. -const MSG_TIMEOUT: time::Duration = time::Duration::seconds(10); - -struct Server { - out: channel::UnboundedSender, - max_block_size: usize, -} - -#[async_trait::async_trait] -impl rpc::Handler for Server { - /// Here we bound the buffering of incoming consensus messages. - fn max_req_size(&self) -> usize { - self.max_block_size.saturating_add(kB) - } - - async fn handle( - &self, - ctx: &ctx::Ctx, - req: rpc::consensus::Req, - ) -> anyhow::Result { - let (send, recv) = oneshot::channel(); - self.out - .send(io::OutputMessage::Consensus(io::ConsensusReq { - msg: req.0, - ack: send, - })); - recv.recv_or_disconnected(ctx).await??; - Ok(rpc::consensus::Resp) - } -} - -/// Performs handshake of an inbound stream. -/// Closes the stream if there is another inbound stream opened from the same validator. -pub(crate) async fn run_inbound_stream( - ctx: &ctx::Ctx, - state: &State, - sender: &channel::UnboundedSender, - mut stream: noise::Stream, -) -> anyhow::Result<()> { - let consensus_state = state - .consensus - .as_ref() - .context("Node does not accept consensus network connections")?; - let peer = handshake::inbound(ctx, &consensus_state.cfg.key, &mut stream).await?; - consensus_state.inbound.insert(peer.clone()).await?; - let res = scope::run!(ctx, |ctx, s| async { - let mut service = rpc::Service::new() - .add_server(rpc::ping::Server) - .add_server(Server { - out: sender.clone(), - max_block_size: state.cfg.max_block_size, - }); - if state.cfg.enable_pings { - let ping_client = rpc::Client::::new(ctx); - service = service.add_client(&ping_client); - s.spawn(async { - let ping_client = ping_client; - ping_client.ping_loop(ctx, PING_TIMEOUT).await - }); - } - service.run(ctx, stream).await?; - Ok(()) - }) - .await; - consensus_state.inbound.remove(&peer).await; - res -} - -async fn run_outbound_stream( - ctx: &ctx::Ctx, - state: &super::State, - client: &rpc::Client, - peer: &validator::PublicKey, - addr: std::net::SocketAddr, -) -> anyhow::Result<()> { - let mut stream = preface::connect(ctx, addr, preface::Endpoint::ConsensusNet).await?; - handshake::outbound(ctx, &state.cfg.key, &mut stream, peer).await?; - state.outbound.insert(peer.clone()).await?; - let ping_client = rpc::Client::::new(ctx); - let res = scope::run!(ctx, |ctx, s| async { - s.spawn(ping_client.ping_loop(ctx, PING_TIMEOUT)); - rpc::Service::new() - .add_client(&ping_client) - .add_server(rpc::ping::Server) - .add_client(client) - .run(ctx, stream) - .await?; - Ok(()) - }) - .await; - state.outbound.remove(peer).await; - res -} - -/// Runs an Rpc client trying to maintain 1 outbound connection per validator. -pub(crate) async fn run_client( - ctx: &ctx::Ctx, - state: &super::State, - shared_state: &State, - mut receiver: channel::UnboundedReceiver, -) -> anyhow::Result<()> { - let clients: HashMap<_, _> = shared_state - .cfg - .validators - .iter() - .map(|peer| (peer.clone(), rpc::Client::::new(ctx))) - .collect(); - - scope::run!(ctx, |ctx, s| async { - // Spawn outbound connections. - for (peer, client) in &clients { - s.spawn::<()>(async { - let client = &*client; - let addrs = &mut shared_state.gossip.validator_addrs.subscribe(); - let mut addr = None; - while ctx.is_active() { - if let Ok(new) = - sync::wait_for(&ctx.with_timeout(CONNECT_RETRY), addrs, |addrs| { - addrs.get(peer).map(|x| x.msg.addr) != addr - }) - .await - { - addr = new.get(peer).map(|x| x.msg.addr); - } - if let Some(addr) = addr { - if let Err(err) = run_outbound_stream(ctx, state, client, peer, addr).await - { - tracing::info!("run_outbound_stream({addr}): {err:#}"); - } - } - } - Ok(()) - }); - } - - // Call RPCs. - while let Ok(msg) = receiver.recv(ctx).await { - match msg.recipient { - io::Target::Validator(val) => { - let client = clients.get(&val).context("unknown validator")?; - s.spawn(async { - let req = rpc::consensus::Req(msg.message); - if let Err(err) = - client.call(&ctx.with_timeout(MSG_TIMEOUT), &req, kB).await - { - tracing::info!("client.consensus(): {err:#}"); - } - Ok(()) - }); - } - io::Target::Broadcast => { - let req = Arc::new(rpc::consensus::Req(msg.message)); - for client in clients.values() { - let req = req.clone(); - s.spawn(async { - let req = req; - if let Err(err) = - client.call(&ctx.with_timeout(MSG_TIMEOUT), &req, kB).await - { - tracing::info!("client.consensus(): {err:#}"); - } - Ok(()) - }); - } - } - } - } - Ok(()) - }) - .await -} diff --git a/node/actors/network/src/consensus/state.rs b/node/actors/network/src/consensus/state.rs deleted file mode 100644 index e8521283..00000000 --- a/node/actors/network/src/consensus/state.rs +++ /dev/null @@ -1,45 +0,0 @@ -use crate::pool::PoolWatch; -use std::collections::HashSet; -use zksync_consensus_roles::{validator, validator::ValidatorSet}; - -/// Configuration of the consensus network. -#[derive(Debug, Clone)] -pub struct Config { - /// Private key of the validator. Currently only validator nodes - /// are supported, but eventually it will become optional. - pub key: validator::SecretKey, - - /// Public TCP address that other validators are expected to connect to. - /// It is announced over gossip network. - pub public_addr: std::net::SocketAddr, -} - -/// Consensus network state. -pub(crate) struct State { - /// Consensus configuration. - pub(crate) cfg: Config, - /// Set of the currently open inbound connections. - pub(crate) inbound: PoolWatch, - /// Set of the currently open outbound connections. - pub(crate) outbound: PoolWatch, -} - -impl State { - /// Constructs a new State. - pub(crate) fn new(cfg: Config, validators: &ValidatorSet) -> anyhow::Result { - let validators: HashSet<_> = validators.iter().cloned().collect(); - let current_validator_key = cfg.key.public(); - anyhow::ensure!( - validators.contains(¤t_validator_key), - "Validators' public keys {validators:?} do not contain the current validator \ - {current_validator_key:?}; this is not yet supported" - ); - // ^ This check will be relaxed once we support dynamic validator membership - - Ok(Self { - cfg, - inbound: PoolWatch::new(validators.clone(), 0), - outbound: PoolWatch::new(validators, 0), - }) - } -} diff --git a/node/actors/network/src/consensus/tests.rs b/node/actors/network/src/consensus/tests.rs index fcf8c574..f2adc113 100644 --- a/node/actors/network/src/consensus/tests.rs +++ b/node/actors/network/src/consensus/tests.rs @@ -1,5 +1,6 @@ use super::*; -use crate::{io, preface, rpc, testonly}; +use crate::{io, metrics, preface, rpc, testonly}; +use assert_matches::assert_matches; use rand::Rng; use tracing::Instrument as _; use zksync_concurrency::{ctx, net, scope, testonly::abort_on_panic}; @@ -11,14 +12,14 @@ async fn test_one_connection_per_validator() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 3); + let setup = validator::testonly::Setup::new(rng, 3); let nodes = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx,s| async { - let (store,runner) = new_store(ctx,&setup.blocks[0]).await; + let (store,runner) = new_store(ctx,&setup.genesis).await; s.spawn_bg(runner.run(ctx)); let nodes : Vec<_> = nodes.into_iter().enumerate().map(|(i,node)| { - let (node,runner) = testonly::Instance::new(node, store.clone()); + let (node,runner) = testonly::Instance::new(ctx, node, store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }).collect(); @@ -36,16 +37,17 @@ async fn test_one_connection_per_validator() { tracing::info!("Impersonate node 1, and try to establish additional connection to node 0. It should close automatically after the handshake."); let mut stream = preface::connect( ctx, - *nodes[0].state.cfg.server_addr, + *nodes[0].cfg().server_addr, preface::Endpoint::ConsensusNet, ) .await?; handshake::outbound( ctx, - &nodes[1].consensus_config().key, + &nodes[1].cfg().validator_key.clone().unwrap(), + setup.genesis.hash(), &mut stream, - &nodes[0].consensus_config().key.public(), + &nodes[0].cfg().validator_key.as_ref().unwrap().public(), ) .await?; // The connection is expected to be closed automatically by node 0. @@ -61,22 +63,89 @@ async fn test_one_connection_per_validator() { .unwrap(); } +#[tokio::test] +async fn test_genesis_mismatch() { + abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + let setup = validator::testonly::Setup::new(rng, 2); + let cfgs = testonly::new_configs(rng, &setup, /*gossip_peers=*/ 0); + + scope::run!(ctx, |ctx, s| async { + let mut listener = cfgs[1].server_addr.bind().context("server_addr.bind()")?; + + tracing::info!("Start one node, we will simulate the other one."); + let (store, runner) = new_store(ctx, &setup.genesis).await; + s.spawn_bg(runner.run(ctx)); + let (node, runner) = testonly::Instance::new(ctx, cfgs[0].clone(), store.clone()); + s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); + + tracing::info!("Populate the validator_addrs of the running node."); + node.net + .gossip + .validator_addrs + .update( + &setup.genesis.validators, + &[Arc::new(setup.keys[1].sign_msg(validator::NetAddress { + addr: cfgs[1].public_addr, + version: 0, + timestamp: ctx.now_utc(), + }))], + ) + .await + .unwrap(); + + tracing::info!("Accept a connection with mismatching genesis."); + let stream = metrics::MeteredStream::listen(ctx, &mut listener) + .await? + .context("listen()")?; + let (mut stream, endpoint) = preface::accept(ctx, stream) + .await + .context("preface::accept()")?; + assert_eq!(endpoint, preface::Endpoint::ConsensusNet); + tracing::info!("Expect the handshake to fail"); + let res = handshake::inbound(ctx, &setup.keys[1], rng.gen(), &mut stream).await; + assert_matches!(res, Err(handshake::Error::GenesisMismatch)); + + tracing::info!("Try to connect to a node with a mismatching genesis."); + let mut stream = + preface::connect(ctx, cfgs[0].public_addr, preface::Endpoint::ConsensusNet) + .await + .context("preface::connect")?; + let res = handshake::outbound( + ctx, + &setup.keys[1], + rng.gen(), + &mut stream, + &setup.keys[0].public(), + ) + .await; + tracing::info!( + "Expect the peer to verify the mismatching Genesis and close the connection." + ); + assert_matches!(res, Err(handshake::Error::Stream(_))); + Ok(()) + }) + .await + .unwrap(); +} + #[tokio::test(flavor = "multi_thread")] async fn test_address_change() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::AffineClock::new(20.)); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 5); + let setup = validator::testonly::Setup::new(rng, 5); let mut cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); let mut nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new(cfg.clone(), store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg.clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) @@ -87,7 +156,7 @@ async fn test_address_change() { nodes[0].terminate(ctx).await?; // All nodes should lose connection to node[0]. - let key0 = nodes[0].consensus_config().key.public(); + let key0 = nodes[0].cfg().validator_key.as_ref().unwrap().public(); for node in &nodes { node.wait_for_consensus_disconnect(ctx, &key0).await?; } @@ -98,8 +167,8 @@ async fn test_address_change() { // Then it should broadcast its new address and the consensus network // should get reconstructed. cfgs[0].server_addr = net::tcp::testonly::reserve_listener(); - cfgs[0].consensus.as_mut().unwrap().public_addr = *cfgs[0].server_addr; - let (node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.clone()); + cfgs[0].public_addr = *cfgs[0].server_addr; + let (node0, runner) = testonly::Instance::new(ctx, cfgs[0].clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node0"))); nodes[0] = node0; for n in &nodes { @@ -119,18 +188,26 @@ async fn test_transmission() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 2); + let setup = validator::testonly::Setup::new(rng, 2); let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); let mut nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new(cfg.clone(), store.clone()); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); + let (node, runner) = testonly::Instance::new(ctx, cfg.clone(), store.clone()); + let i = ctx::NoCopy(i); + s.spawn_bg(async { + let i = i; + runner + .run(ctx) + .instrument(tracing::info_span!("node", i = *i)) + .await + .context(*i) + }); node }) .collect(); @@ -143,7 +220,9 @@ async fn test_transmission() { let want: validator::Signed = rng.gen(); let in_message = io::ConsensusInputMessage { message: want.clone(), - recipient: io::Target::Validator(nodes[1].consensus_config().key.public()), + recipient: io::Target::Validator( + nodes[1].cfg().validator_key.as_ref().unwrap().public(), + ), }; nodes[0].pipe.send(in_message.into()); diff --git a/node/actors/network/src/event.rs b/node/actors/network/src/event.rs deleted file mode 100644 index adf792e2..00000000 --- a/node/actors/network/src/event.rs +++ /dev/null @@ -1,21 +0,0 @@ -//! Mechanism for network State to report internal events. -//! It is used in tests to await a specific state. -use crate::State; - -impl State { - /// Sends an event to the `self.events` channel. - /// Noop if `self.events` is None. - pub(crate) fn event(&self, e: Event) { - if let Some(events) = &self.events { - events.send(e); - } - } -} - -/// Events observable in tests. -/// Feel free to extend this enum if you need to -/// write a test awaiting some specific event/state. -#[derive(Debug)] -pub enum Event { - ValidatorAddrsUpdated, -} diff --git a/node/actors/network/src/gossip/arcmap.rs b/node/actors/network/src/gossip/arcmap.rs new file mode 100644 index 00000000..2fe3307d --- /dev/null +++ b/node/actors/network/src/gossip/arcmap.rs @@ -0,0 +1,42 @@ +//! Multimap of pointers indexed by `node::PublicKey`. +//! Used to maintain a collection GetBlock rpc clients. +//! TODO(gprusak): consider upgrading PoolWatch instead. +use std::{ + collections::HashMap, + sync::{Arc, Mutex}, +}; +use zksync_consensus_roles::node; + +/// ArcMap +pub(crate) struct ArcMap(Mutex>>>); + +impl Default for ArcMap { + fn default() -> Self { + Self(Mutex::default()) + } +} + +impl ArcMap { + /// Fetches any pointer for the given key. + pub(crate) fn get_any(&self, key: &node::PublicKey) -> Option> { + self.0.lock().unwrap().get(key)?.first().cloned() + } + + /// Insert a pointer. + pub(crate) fn insert(&self, key: node::PublicKey, p: Arc) { + self.0.lock().unwrap().entry(key).or_default().push(p); + } + + /// Removes a pointer. + pub(crate) fn remove(&self, key: node::PublicKey, p: Arc) { + let mut this = self.0.lock().unwrap(); + use std::collections::hash_map::Entry; + let Entry::Occupied(mut e) = this.entry(key) else { + return; + }; + e.get_mut().retain(|c| !Arc::ptr_eq(&p, c)); + if e.get_mut().is_empty() { + e.remove(); + } + } +} diff --git a/node/actors/network/src/gossip/handshake/mod.rs b/node/actors/network/src/gossip/handshake/mod.rs index f5d7b22e..de412474 100644 --- a/node/actors/network/src/gossip/handshake/mod.rs +++ b/node/actors/network/src/gossip/handshake/mod.rs @@ -1,9 +1,8 @@ -use super::Config; -use crate::{frame, noise, proto::gossip as proto}; +use crate::{frame, noise, proto::gossip as proto, GossipConfig}; use anyhow::Context as _; use zksync_concurrency::{ctx, time}; use zksync_consensus_crypto::ByteFmt; -use zksync_consensus_roles::node; +use zksync_consensus_roles::{node, validator}; use zksync_protobuf::{read_required, required, ProtoFmt}; #[cfg(test)] @@ -20,6 +19,9 @@ pub(crate) struct Handshake { /// Session ID signed with the node key. /// Authenticates the peer to be the owner of the node key. pub(crate) session_id: node::Signed, + /// Hash of the blockchain genesis specification. + /// Only nodes with the same genesis belong to the same network. + pub(crate) genesis: validator::GenesisHash, /// Information whether the peer treats this connection as static. /// It is informational only, it doesn't affect the logic of the node. pub(crate) is_static: bool, @@ -30,12 +32,14 @@ impl ProtoFmt for Handshake { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { session_id: read_required(&r.session_id).context("session_id")?, + genesis: read_required(&r.genesis).context("genesis")?, is_static: *required(&r.is_static).context("is_static")?, }) } fn build(&self) -> Self::Proto { Self::Proto { session_id: Some(self.session_id.build()), + genesis: Some(self.genesis.build()), is_static: Some(self.is_static), } } @@ -44,19 +48,22 @@ impl ProtoFmt for Handshake { /// Error returned by gossip handshake logic. #[derive(Debug, thiserror::Error)] pub(super) enum Error { + #[error("genesis mismatch")] + GenesisMismatch, #[error("session id mismatch")] SessionIdMismatch, #[error("unexpected peer")] PeerMismatch, - #[error("validator signature")] + #[error(transparent)] Signature(#[from] node::InvalidSignatureError), - #[error("stream")] - Stream(#[source] anyhow::Error), + #[error(transparent)] + Stream(anyhow::Error), } pub(super) async fn outbound( ctx: &ctx::Ctx, - cfg: &Config, + cfg: &GossipConfig, + genesis: validator::GenesisHash, stream: &mut noise::Stream, peer: &node::PublicKey, ) -> Result<(), Error> { @@ -67,6 +74,7 @@ pub(super) async fn outbound( stream, &Handshake { session_id: cfg.key.sign_msg(session_id.clone()), + genesis, is_static: cfg.static_outbound.contains_key(peer), }, ) @@ -75,6 +83,9 @@ pub(super) async fn outbound( let h: Handshake = frame::recv_proto(ctx, stream, Handshake::max_size()) .await .map_err(Error::Stream)?; + if h.genesis != genesis { + return Err(Error::GenesisMismatch); + } if h.session_id.msg != session_id { return Err(Error::SessionIdMismatch); } @@ -87,7 +98,8 @@ pub(super) async fn outbound( pub(super) async fn inbound( ctx: &ctx::Ctx, - cfg: &Config, + cfg: &GossipConfig, + genesis: validator::GenesisHash, stream: &mut noise::Stream, ) -> Result { let ctx = &ctx.with_timeout(TIMEOUT); @@ -98,12 +110,16 @@ pub(super) async fn inbound( if h.session_id.msg != session_id { return Err(Error::SessionIdMismatch); } + if h.genesis != genesis { + return Err(Error::GenesisMismatch); + } h.session_id.verify()?; frame::send_proto( ctx, stream, &Handshake { session_id: cfg.key.sign_msg(session_id.clone()), + genesis, is_static: cfg.static_inbound.contains(&h.session_id.key), }, ) diff --git a/node/actors/network/src/gossip/handshake/testonly.rs b/node/actors/network/src/gossip/handshake/testonly.rs index d45531d2..c0894adf 100644 --- a/node/actors/network/src/gossip/handshake/testonly.rs +++ b/node/actors/network/src/gossip/handshake/testonly.rs @@ -15,6 +15,7 @@ impl Distribution for Standard { let session_id: node::SessionId = rng.gen(); Handshake { session_id: key.sign_msg(session_id), + genesis: rng.gen(), is_static: rng.gen(), } } diff --git a/node/actors/network/src/gossip/handshake/tests.rs b/node/actors/network/src/gossip/handshake/tests.rs index 2e024861..efb3471a 100644 --- a/node/actors/network/src/gossip/handshake/tests.rs +++ b/node/actors/network/src/gossip/handshake/tests.rs @@ -1,5 +1,6 @@ use super::*; -use crate::{frame, noise, testonly}; +use crate::{frame, noise, testonly, GossipConfig}; +use assert_matches::assert_matches; use rand::Rng; use std::collections::{HashMap, HashSet}; use zksync_concurrency::{ctx, io, scope, testonly::abort_on_panic}; @@ -12,8 +13,8 @@ fn test_schema_encode_decode() { test_encode_random::(rng); } -fn make_cfg(rng: &mut R) -> Config { - Config { +fn make_cfg(rng: &mut R) -> GossipConfig { + GossipConfig { key: rng.gen(), dynamic_inbound_limit: 0, static_inbound: HashSet::default(), @@ -29,6 +30,7 @@ async fn test_session_id_mismatch() { let cfg0 = make_cfg(rng); let cfg1 = make_cfg(rng); + let genesis: validator::GenesisHash = rng.gen(); // MitM attempt detected on the inbound end. scope::run!(ctx, |ctx, s| async { @@ -46,14 +48,14 @@ async fn test_session_id_mismatch() { }); s.spawn(async { let mut s4 = s4; - match inbound(ctx, &cfg0, &mut s4).await { + match inbound(ctx, &cfg0, genesis, &mut s4).await { Err(Error::SessionIdMismatch) => Ok(()), res => panic!("unexpected res: {res:?}"), } }); s.spawn(async { let mut s1 = s1; - match outbound(ctx, &cfg1, &mut s1, &cfg0.key.public()).await { + match outbound(ctx, &cfg1, genesis, &mut s1, &cfg0.key.public()).await { Err(Error::Stream(..)) => Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -74,13 +76,14 @@ async fn test_session_id_mismatch() { &mut s2, &Handshake { session_id: cfg1.key.sign_msg(rng.gen::()), + genesis, is_static: false, }, ) .await?; Ok(()) }); - match outbound(ctx, &cfg0, &mut s1, &cfg1.key.public()).await { + match outbound(ctx, &cfg0, genesis, &mut s1, &cfg1.key.public()).await { Err(Error::SessionIdMismatch) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -99,16 +102,21 @@ async fn test_peer_mismatch() { let cfg1 = make_cfg(rng); let cfg2 = make_cfg(rng); + let genesis: validator::GenesisHash = rng.gen(); + scope::run!(ctx, |ctx, s| async { let (s0, s1) = noise::testonly::pipe(ctx).await; s.spawn(async { let mut s0 = s0; - assert_eq!(cfg1.key.public(), inbound(ctx, &cfg0, &mut s0).await?); + assert_eq!( + cfg1.key.public(), + inbound(ctx, &cfg0, genesis, &mut s0).await? + ); Ok(()) }); s.spawn(async { let mut s1 = s1; - match outbound(ctx, &cfg1, &mut s1, &cfg2.key.public()).await { + match outbound(ctx, &cfg1, genesis, &mut s1, &cfg2.key.public()).await { Err(Error::PeerMismatch) => Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -119,6 +127,61 @@ async fn test_peer_mismatch() { .unwrap(); } +#[tokio::test] +async fn test_genesis_mismatch() { + abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + + let cfg0 = make_cfg(rng); + let cfg1 = make_cfg(rng); + + tracing::info!("test that inbound handshake rejects mismatching genesis"); + scope::run!(ctx, |ctx, s| async { + let (s0, mut s1) = noise::testonly::pipe(ctx).await; + s.spawn(async { + let mut s0 = s0; + let res = outbound(ctx, &cfg0, ctx.rng().gen(), &mut s0, &cfg1.key.public()).await; + assert_matches!(res, Err(Error::Stream(_))); + Ok(()) + }); + let res = inbound(ctx, &cfg1, rng.gen(), &mut s1).await; + assert_matches!(res, Err(Error::GenesisMismatch)); + anyhow::Ok(()) + }) + .await + .unwrap(); + + tracing::info!("test that outbound handshake rejects mismatching genesis"); + scope::run!(ctx, |ctx, s| async { + let (s0, mut s1) = noise::testonly::pipe(ctx).await; + s.spawn(async { + let mut s0 = s0; + let res = outbound(ctx, &cfg0, ctx.rng().gen(), &mut s0, &cfg1.key.public()).await; + assert_matches!(res, Err(Error::GenesisMismatch)); + Ok(()) + }); + let session_id = node::SessionId(s1.id().encode()); + let _: Handshake = frame::recv_proto(ctx, &mut s1, Handshake::max_size()) + .await + .unwrap(); + frame::send_proto( + ctx, + &mut s1, + &Handshake { + session_id: cfg1.key.sign_msg(session_id), + genesis: rng.gen(), + is_static: false, + }, + ) + .await + .unwrap(); + anyhow::Ok(()) + }) + .await + .unwrap(); +} + #[tokio::test] async fn test_invalid_signature() { abort_on_panic(); @@ -128,6 +191,8 @@ async fn test_invalid_signature() { let cfg0 = make_cfg(rng); let cfg1 = make_cfg(rng); + let genesis: validator::GenesisHash = rng.gen(); + // Bad signature detected on outbound end. scope::run!(ctx, |ctx, s| async { let (mut s0, s1) = noise::testonly::pipe(ctx).await; @@ -138,7 +203,7 @@ async fn test_invalid_signature() { frame::send_proto(ctx, &mut s1, &h).await?; Ok(()) }); - match outbound(ctx, &cfg0, &mut s0, &cfg1.key.public()).await { + match outbound(ctx, &cfg0, genesis, &mut s0, &cfg1.key.public()).await { Err(Error::Signature(..)) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } @@ -153,12 +218,13 @@ async fn test_invalid_signature() { let mut s1 = s1; let mut h = Handshake { session_id: cfg0.key.sign_msg(node::SessionId(s1.id().encode())), + genesis, is_static: true, }; h.session_id.key = cfg1.key.public(); frame::send_proto(ctx, &mut s1, &h).await }); - match inbound(ctx, &cfg0, &mut s0).await { + match inbound(ctx, &cfg0, genesis, &mut s0).await { Err(Error::Signature(..)) => anyhow::Ok(()), res => panic!("unexpected res: {res:?}"), } diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 7991a147..e20cf7e8 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -12,11 +12,93 @@ //! Static connections constitute a rigid "backbone" of the gossip network, which is insensitive to //! eclipse attack. Dynamic connections are supposed to improve the properties of the gossip //! network graph (minimize its diameter, increase connectedness). +use crate::{ + gossip::{ArcMap, ValidatorAddrsWatch}, + io, + pool::PoolWatch, + rpc, Config, +}; +use anyhow::Context as _; +use std::sync::{atomic::AtomicUsize, Arc}; + +mod arcmap; mod handshake; mod runner; -mod state; #[cfg(test)] mod tests; +mod validator_addrs; + +pub(crate) use arcmap::*; +pub(crate) use validator_addrs::*; +use zksync_concurrency::{ctx, ctx::channel}; +use zksync_consensus_roles::{node, validator}; +use zksync_consensus_storage::BlockStore; +use zksync_protobuf::kB; + +/// Gossip network state. +pub(crate) struct Network { + /// Gossip network configuration. + pub(crate) cfg: Config, + /// Currently open inbound connections. + pub(crate) inbound: PoolWatch, + /// Currently open outbound connections. + pub(crate) outbound: PoolWatch, + /// Current state of knowledge about validators' endpoints. + pub(crate) validator_addrs: ValidatorAddrsWatch, + /// Block store to serve `get_block` requests from. + pub(crate) block_store: Arc, + /// Clients for `get_block` requests for each currently active peer. + pub(crate) get_block_clients: ArcMap>, + /// Output pipe of the network actor. + pub(crate) sender: channel::UnboundedSender, + /// TESTONLY: how many time push_validator_addrs rpc was called by the peers. + pub(crate) push_validator_addrs_calls: AtomicUsize, +} + +impl Network { + /// Constructs a new State. + pub(crate) fn new( + cfg: Config, + block_store: Arc, + sender: channel::UnboundedSender, + ) -> Arc { + Arc::new(Self { + sender, + inbound: PoolWatch::new( + cfg.gossip.static_inbound.clone(), + cfg.gossip.dynamic_inbound_limit, + ), + outbound: PoolWatch::new(cfg.gossip.static_outbound.keys().cloned().collect(), 0), + validator_addrs: ValidatorAddrsWatch::default(), + block_store, + get_block_clients: ArcMap::default(), + cfg, + push_validator_addrs_calls: 0.into(), + }) + } + + /// Genesis. + pub(crate) fn genesis(&self) -> &validator::Genesis { + self.block_store.genesis() + } -pub(crate) use runner::*; -pub use state::*; + /// Sends a GetBlock RPC to the given peer. + pub(crate) async fn get_block( + &self, + ctx: &ctx::Ctx, + recipient: &node::PublicKey, + number: validator::BlockNumber, + ) -> anyhow::Result> { + Ok(self + .get_block_clients + .get_any(recipient) + .context("recipient is unreachable")? + .call( + ctx, + &rpc::get_block::Req(number), + self.cfg.max_block_size.saturating_add(kB), + ) + .await? + .0) + } +} diff --git a/node/actors/network/src/gossip/runner.rs b/node/actors/network/src/gossip/runner.rs index 744b7b5b..b84ac131 100644 --- a/node/actors/network/src/gossip/runner.rs +++ b/node/actors/network/src/gossip/runner.rs @@ -1,28 +1,13 @@ -use super::{handshake, ValidatorAddrs}; -use crate::{consensus, event::Event, io, noise, preface, rpc, State}; +use super::{handshake, Network, ValidatorAddrs}; +use crate::{io, noise, preface, rpc}; use async_trait::async_trait; -use std::sync::Arc; -use zksync_concurrency::{ - ctx::{self, channel}, - oneshot, scope, sync, time, -}; -use zksync_consensus_roles::{node, validator}; +use std::sync::{atomic::Ordering, Arc}; +use zksync_concurrency::{ctx, oneshot, scope, sync}; +use zksync_consensus_roles::node; use zksync_consensus_storage::BlockStore; use zksync_protobuf::kB; -/// How often we should retry to establish a connection to a validator. -/// TODO(gprusak): once it becomes relevant, choose a more appropriate retry strategy. -pub(crate) const CONNECT_RETRY: time::Duration = time::Duration::seconds(20); -/// A ping request is sent periodically. If ping response doesn't arrive -/// within PING_TIMEOUT, we close the connection. -const PING_TIMEOUT: time::Duration = time::Duration::seconds(5); -/// Frequency at which the validator broadcasts its own IP address. -/// Although the IP is not likely to change while validator is running, -/// we do this periodically, so that the network can observe if validator -/// is down. -const ADDRESS_ANNOUNCER_INTERVAL: time::Duration = time::Duration::minutes(10); - -struct PushValidatorAddrsServer<'a>(&'a State); +struct PushValidatorAddrsServer<'a>(&'a Network); #[async_trait] impl rpc::Handler for PushValidatorAddrsServer<'_> { @@ -34,20 +19,21 @@ impl rpc::Handler for PushValidatorAddrsServer<' _ctx: &ctx::Ctx, req: rpc::push_validator_addrs::Req, ) -> anyhow::Result<()> { - self.0.event(Event::ValidatorAddrsUpdated); self.0 - .gossip + .push_validator_addrs_calls + .fetch_add(1, Ordering::SeqCst); + self.0 .validator_addrs - .update(&self.0.cfg.validators, &req.0[..]) + .update(&self.0.genesis().validators, &req.0[..]) .await?; Ok(()) } } -#[derive(Debug, Clone, Copy)] +#[derive(Clone, Copy)] struct PushBlockStoreStateServer<'a> { peer: &'a node::PublicKey, - sender: &'a channel::UnboundedSender, + net: &'a Network, } #[async_trait] @@ -66,7 +52,7 @@ impl rpc::Handler for PushBlockStoreStateServe state: req.0, response, }; - self.sender.send(message.into()); + self.net.sender.send(message.into()); response_receiver.recv_or_disconnected(ctx).await??; Ok(()) } @@ -86,208 +72,131 @@ impl rpc::Handler for &BlockStore { } } -async fn run_stream( - ctx: &ctx::Ctx, - state: &State, - peer: &node::PublicKey, - sender: &channel::UnboundedSender, - stream: noise::Stream, -) -> anyhow::Result<()> { - let push_validator_addrs_client = rpc::Client::::new(ctx); - let push_validator_addrs_server = PushValidatorAddrsServer(state); - let push_block_store_state_client = rpc::Client::::new(ctx); - let push_block_store_state_server = PushBlockStoreStateServer { peer, sender }; - - let get_block_client = Arc::new(rpc::Client::::new(ctx)); - state - .gossip - .get_block_clients - .insert(peer.clone(), get_block_client.clone()); - - let res = scope::run!(ctx, |ctx, s| async { - let mut service = rpc::Service::new() - .add_client(&push_validator_addrs_client) - .add_server(push_validator_addrs_server) - .add_client(&push_block_store_state_client) - .add_server(push_block_store_state_server) - .add_client(&get_block_client) - .add_server(&*state.gossip.block_store) - .add_server(rpc::ping::Server); - - if state.cfg.enable_pings { - let ping_client = rpc::Client::::new(ctx); - service = service.add_client(&ping_client); - s.spawn(async { - let ping_client = ping_client; - ping_client.ping_loop(ctx, PING_TIMEOUT).await - }); - } - - // Push block store state updates to peer. - s.spawn::<()>(async { - let mut sub = state.gossip.block_store.subscribe(); - sub.mark_changed(); - loop { - let state = sync::changed(ctx, &mut sub).await?.clone(); - let req = rpc::push_block_store_state::Req(state); - push_block_store_state_client.call(ctx, &req, kB).await?; +impl Network { + /// Manages lifecycle of a single connection. + async fn run_stream( + &self, + ctx: &ctx::Ctx, + peer: &node::PublicKey, + stream: noise::Stream, + ) -> anyhow::Result<()> { + let push_validator_addrs_client = rpc::Client::::new( + ctx, + self.cfg.rpc.push_validator_addrs_rate, + ); + let push_validator_addrs_server = PushValidatorAddrsServer(self); + let push_block_store_state_client = rpc::Client::::new( + ctx, + self.cfg.rpc.push_block_store_state_rate, + ); + let push_block_store_state_server = PushBlockStoreStateServer { peer, net: self }; + + let get_block_client = Arc::new(rpc::Client::::new( + ctx, + self.cfg.rpc.get_block_rate, + )); + self.get_block_clients + .insert(peer.clone(), get_block_client.clone()); + + let res = scope::run!(ctx, |ctx, s| async { + let mut service = rpc::Service::new() + .add_client(&push_validator_addrs_client) + .add_server( + push_validator_addrs_server, + self.cfg.rpc.push_validator_addrs_rate, + ) + .add_client(&push_block_store_state_client) + .add_server( + push_block_store_state_server, + self.cfg.rpc.push_block_store_state_rate, + ) + .add_client(&get_block_client) + .add_server(&*self.block_store, self.cfg.rpc.get_block_rate) + .add_server(rpc::ping::Server, rpc::ping::RATE); + + if let Some(ping_timeout) = &self.cfg.ping_timeout { + let ping_client = rpc::Client::::new(ctx, rpc::ping::RATE); + service = service.add_client(&ping_client); + s.spawn(async { + let ping_client = ping_client; + ping_client.ping_loop(ctx, *ping_timeout).await + }); } - }); - s.spawn::<()>(async { - // Push validator addrs updates to peer. - let mut old = ValidatorAddrs::default(); - let mut sub = state.gossip.validator_addrs.subscribe(); - sub.mark_changed(); - loop { - let new = sync::changed(ctx, &mut sub).await?.clone(); - let diff = new.get_newer(&old); - if diff.is_empty() { - continue; + // Push block store state updates to peer. + s.spawn::<()>(async { + let mut sub = self.block_store.subscribe(); + sub.mark_changed(); + loop { + let state = sync::changed(ctx, &mut sub).await?.clone(); + let req = rpc::push_block_store_state::Req(state); + push_block_store_state_client.call(ctx, &req, kB).await?; } - old = new; - let req = rpc::push_validator_addrs::Req(diff); - push_validator_addrs_client.call(ctx, &req, kB).await?; - } - }); - - service.run(ctx, stream).await?; - Ok(()) - }) - .await; - - state - .gossip - .get_block_clients - .remove(peer.clone(), get_block_client); - res -} - -/// Handles an inbound stream. -/// Closes the stream if there is another inbound stream opened from the same peer. -pub(crate) async fn run_inbound_stream( - ctx: &ctx::Ctx, - state: &State, - sender: &channel::UnboundedSender, - mut stream: noise::Stream, -) -> anyhow::Result<()> { - let peer = handshake::inbound(ctx, &state.gossip.cfg, &mut stream).await?; - tracing::Span::current().record("peer", tracing::field::debug(&peer)); - state.gossip.inbound.insert(peer.clone()).await?; - let res = run_stream(ctx, state, &peer, sender, stream).await; - state.gossip.inbound.remove(&peer).await; - res -} - -async fn run_outbound_stream( - ctx: &ctx::Ctx, - state: &State, - sender: &channel::UnboundedSender, - peer: &node::PublicKey, - addr: std::net::SocketAddr, -) -> anyhow::Result<()> { - let mut stream = preface::connect(ctx, addr, preface::Endpoint::GossipNet).await?; - handshake::outbound(ctx, &state.gossip.cfg, &mut stream, peer).await?; - - state.gossip.outbound.insert(peer.clone()).await?; - let res = run_stream(ctx, state, peer, sender, stream).await; - state.gossip.outbound.remove(peer).await; - res -} - -async fn run_address_announcer( - ctx: &ctx::Ctx, - state: &State, - consensus_state: &consensus::State, -) -> ctx::OrCanceled<()> { - let key = &consensus_state.cfg.key; - let my_addr = consensus_state.cfg.public_addr; - let mut sub = state.gossip.validator_addrs.subscribe(); - loop { - if !ctx.is_active() { - return Err(ctx::Canceled); - } - let ctx = &ctx.with_timeout(ADDRESS_ANNOUNCER_INTERVAL); - let _ = sync::wait_for(ctx, &mut sub, |got| { - got.get(&key.public()).map(|x| &x.msg.addr) != Some(&my_addr) - }) - .await; - let next_version = sub - .borrow() - .get(&key.public()) - .map(|x| x.msg.version + 1) - .unwrap_or(0); - state - .gossip - .validator_addrs - .update( - &state.cfg.validators, - &[Arc::new(key.sign_msg(validator::NetAddress { - addr: my_addr, - version: next_version, - timestamp: ctx.now_utc(), - }))], - ) - .await - .unwrap(); - } -} + }); -/// Runs an RPC client trying to maintain 1 outbound connection per validator. -pub(crate) async fn run_client( - ctx: &ctx::Ctx, - state: &State, - sender: &channel::UnboundedSender, - mut receiver: channel::UnboundedReceiver, -) -> anyhow::Result<()> { - scope::run!(ctx, |ctx, s| async { - // Spawn a tasks handling static outbound connections. - for (peer, addr) in &state.gossip.cfg.static_outbound { s.spawn::<()>(async { + // Push validator addrs updates to peer. + let mut old = ValidatorAddrs::default(); + let mut sub = self.validator_addrs.subscribe(); + sub.mark_changed(); loop { - let run_result = run_outbound_stream(ctx, state, sender, peer, *addr).await; - if let Err(err) = run_result { - tracing::info!("run_client_stream(): {err:#}"); + let new = sync::changed(ctx, &mut sub).await?.clone(); + let diff = new.get_newer(&old); + if diff.is_empty() { + continue; } - ctx.sleep(CONNECT_RETRY).await?; + old = new; + let req = rpc::push_validator_addrs::Req(diff); + push_validator_addrs_client.call(ctx, &req, kB).await?; } }); - } - s.spawn(async { - while let Ok(message) = receiver.recv(ctx).await { - s.spawn(async { - let message = message; - let io::SyncBlocksInputMessage::GetBlock { - recipient, - number, - response, - } = message; - let _ = response.send( - match state - .gossip - .get_block(ctx, &recipient, number, state.cfg.max_block_size) - .await - { - Ok(Some(block)) => Ok(block), - Ok(None) => Err(io::GetBlockError::NotAvailable), - Err(err) => Err(io::GetBlockError::Internal(err)), - }, - ); - Ok(()) - }); - } + service.run(ctx, stream).await?; Ok(()) - }); + }) + .await; - if let Some(consensus_state) = &state.consensus { - run_address_announcer(ctx, state, consensus_state).await - } else { - Ok(()) - } - }) - .await - .ok(); + self.get_block_clients + .remove(peer.clone(), get_block_client); + res + } - Ok(()) + /// Handles an inbound stream. + /// Closes the stream if there is another inbound stream opened from the same peer. + pub(crate) async fn run_inbound_stream( + &self, + ctx: &ctx::Ctx, + mut stream: noise::Stream, + ) -> anyhow::Result<()> { + let peer = + handshake::inbound(ctx, &self.cfg.gossip, self.genesis().hash(), &mut stream).await?; + tracing::Span::current().record("peer", tracing::field::debug(&peer)); + self.inbound.insert(peer.clone()).await?; + let res = self.run_stream(ctx, &peer, stream).await; + self.inbound.remove(&peer).await; + res + } + + /// Connects to a peer and handles the resulting stream. + pub(crate) async fn run_outbound_stream( + &self, + ctx: &ctx::Ctx, + peer: &node::PublicKey, + addr: std::net::SocketAddr, + ) -> anyhow::Result<()> { + let mut stream = preface::connect(ctx, addr, preface::Endpoint::GossipNet).await?; + handshake::outbound( + ctx, + &self.cfg.gossip, + self.genesis().hash(), + &mut stream, + peer, + ) + .await?; + + self.outbound.insert(peer.clone()).await?; + let res = self.run_stream(ctx, peer, stream).await; + self.outbound.remove(peer).await; + res + } } diff --git a/node/actors/network/src/gossip/state.rs b/node/actors/network/src/gossip/state.rs index 3b33631f..8b137891 100644 --- a/node/actors/network/src/gossip/state.rs +++ b/node/actors/network/src/gossip/state.rs @@ -1,216 +1 @@ -use crate::{pool::PoolWatch, rpc, watch::Watch}; -use anyhow::Context as _; -use std::{ - collections::{HashMap, HashSet}, - sync::{Arc, Mutex}, -}; -use zksync_concurrency::{ctx, sync}; -use zksync_consensus_roles::{node, validator}; -use zksync_consensus_storage::BlockStore; -use zksync_protobuf::kB; -/// Mapping from validator::PublicKey to a signed validator::NetAddress. -/// Represents the currents state of node's knowledge about the validator endpoints. -#[derive(Clone, Default, PartialEq, Eq)] -pub(crate) struct ValidatorAddrs( - pub(super) im::HashMap>>, -); - -impl ValidatorAddrs { - /// Gets a NetAddress for a given key. - pub(crate) fn get( - &self, - key: &validator::PublicKey, - ) -> Option<&Arc>> { - self.0.get(key) - } - - /// Returns a set of entries of `self` which are newer than the entries in `b`. - pub(super) fn get_newer(&self, b: &Self) -> Vec>> { - let mut newer = vec![]; - for (k, v) in &self.0 { - if let Some(bv) = b.0.get(k) { - if !v.msg.is_newer(&bv.msg) { - continue; - } - } - newer.push(v.clone()); - } - newer - } - - /// Updates the discovery map with entries from `data`. - /// It exits as soon as an invalid entry is found. - /// `self` might get modified even if an error is returned - /// (all entries verified so far are added). - /// Returns true iff some new entry was added. - pub(super) fn update( - &mut self, - validators: &validator::ValidatorSet, - data: &[Arc>], - ) -> anyhow::Result { - let mut changed = false; - - let mut done = HashSet::new(); - for d in data { - // Disallow multiple entries for the same key: - // it is important because a malicious validator may spam us with - // new versions and verifying signatures is expensive. - if done.contains(&d.key) { - anyhow::bail!("duplicate entry for {:?}", d.key); - } - done.insert(d.key.clone()); - if !validators.contains(&d.key) { - // We just skip the entries we are not interested in. - // For now the set of validators is static, so we could treat this as an error, - // however we eventually want the validator set to be dynamic. - continue; - } - if let Some(x) = self.0.get(&d.key) { - if !d.msg.is_newer(&x.msg) { - continue; - } - } - d.verify()?; - self.0.insert(d.key.clone(), d.clone()); - changed = true; - } - Ok(changed) - } -} - -/// Watch wrapper of ValidatorAddrs, -/// which supports subscribing to ValidatorAddr updates. -pub(crate) struct ValidatorAddrsWatch(Watch); - -impl Default for ValidatorAddrsWatch { - fn default() -> Self { - Self(Watch::new(ValidatorAddrs::default())) - } -} - -impl ValidatorAddrsWatch { - /// Subscribes to ValidatorAddrs updates. - pub(crate) fn subscribe(&self) -> sync::watch::Receiver { - self.0.subscribe() - } - - /// Inserts data to ValidatorAddrs. - /// Subscribers are notified iff at least 1 new entry has - /// been inserted. Returns an error iff an invalid - /// entry in `data` has been found. The provider of the - /// invalid entry should be banned. - pub(crate) async fn update( - &self, - validators: &validator::ValidatorSet, - data: &[Arc>], - ) -> anyhow::Result<()> { - let this = self.0.lock().await; - let mut validator_addrs = this.borrow().clone(); - if validator_addrs.update(validators, data)? { - this.send(validator_addrs).ok().unwrap(); - } - Ok(()) - } -} - -/// Gossip network configuration. -#[derive(Debug, Clone)] -pub struct Config { - /// Private key of the node, every node should have one. - pub key: node::SecretKey, - /// Limit on the number of inbound connections outside - /// of the `static_inbound` set. - pub dynamic_inbound_limit: usize, - /// Inbound connections that should be unconditionally accepted. - pub static_inbound: HashSet, - /// Outbound connections that the node should actively try to - /// establish and maintain. - pub static_outbound: HashMap, -} - -/// Multimap of pointers indexed by `node::PublicKey`. -/// Used to maintain a collection GetBlock rpc clients. -/// TODO(gprusak): consider upgrading PoolWatch instead. -pub(crate) struct ArcMap(Mutex>>>); - -impl Default for ArcMap { - fn default() -> Self { - Self(Mutex::default()) - } -} - -impl ArcMap { - /// Fetches any pointer for the given key. - pub(crate) fn get_any(&self, key: &node::PublicKey) -> Option> { - self.0.lock().unwrap().get(key)?.first().cloned() - } - - /// Insert a pointer. - pub(crate) fn insert(&self, key: node::PublicKey, p: Arc) { - self.0.lock().unwrap().entry(key).or_default().push(p); - } - - /// Removes a pointer. - pub(crate) fn remove(&self, key: node::PublicKey, p: Arc) { - let mut this = self.0.lock().unwrap(); - use std::collections::hash_map::Entry; - let Entry::Occupied(mut e) = this.entry(key) else { - return; - }; - e.get_mut().retain(|c| !Arc::ptr_eq(&p, c)); - if e.get_mut().is_empty() { - e.remove(); - } - } -} - -/// Gossip network state. -pub(crate) struct State { - /// Gossip network configuration. - pub(crate) cfg: Config, - /// Currently open inbound connections. - pub(crate) inbound: PoolWatch, - /// Currently open outbound connections. - pub(crate) outbound: PoolWatch, - /// Current state of knowledge about validators' endpoints. - pub(crate) validator_addrs: ValidatorAddrsWatch, - /// Block store to serve `get_block` requests from. - pub(crate) block_store: Arc, - /// Clients for `get_block` requests for each currently active peer. - pub(crate) get_block_clients: ArcMap>, -} - -impl State { - /// Constructs a new State. - pub(crate) fn new(cfg: Config, block_store: Arc) -> Self { - Self { - inbound: PoolWatch::new(cfg.static_inbound.clone(), cfg.dynamic_inbound_limit), - outbound: PoolWatch::new(cfg.static_outbound.keys().cloned().collect(), 0), - validator_addrs: ValidatorAddrsWatch::default(), - block_store, - get_block_clients: ArcMap::default(), - cfg, - } - } - - pub(super) async fn get_block( - &self, - ctx: &ctx::Ctx, - recipient: &node::PublicKey, - number: validator::BlockNumber, - max_block_size: usize, - ) -> anyhow::Result> { - Ok(self - .get_block_clients - .get_any(recipient) - .context("recipient is unreachable")? - .call( - ctx, - &rpc::get_block::Req(number), - max_block_size.saturating_add(kB), - ) - .await? - .0) - } -} diff --git a/node/actors/network/src/gossip/tests.rs b/node/actors/network/src/gossip/tests.rs index 3d4ab322..575a8492 100644 --- a/node/actors/network/src/gossip/tests.rs +++ b/node/actors/network/src/gossip/tests.rs @@ -1,11 +1,11 @@ use super::*; -use crate::{event::Event, io, preface, rpc, rpc::Rpc as _, testonly}; -use anyhow::Context as _; +use crate::{io, metrics, preface, rpc, testonly}; +use assert_matches::assert_matches; use pretty_assertions::assert_eq; use rand::Rng; use std::{ collections::{HashMap, HashSet}, - sync::Arc, + sync::{atomic::Ordering, Arc}, }; use test_casing::{test_casing, Product}; use tracing::Instrument as _; @@ -23,14 +23,14 @@ async fn test_one_connection_per_node() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 5); + let setup = validator::testonly::Setup::new(rng, 5); let cfgs = testonly::new_configs(rng, &setup, 2); scope::run!(ctx, |ctx,s| async { - let (store,runner) = new_store(ctx,&setup.blocks[0]).await; + let (store,runner) = new_store(ctx,&setup.genesis).await; s.spawn_bg(runner.run(ctx)); let mut nodes : Vec<_> = cfgs.iter().enumerate().map(|(i,cfg)| { - let (node,runner) = testonly::Instance::new(cfg.clone(), store.clone()); + let (node,runner) = testonly::Instance::new(ctx, cfg.clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }).collect(); @@ -52,7 +52,7 @@ async fn test_one_connection_per_node() { .await .context("preface::connect")?; - handshake::outbound(ctx, &cfgs[0].gossip, &mut stream, peer) + handshake::outbound(ctx, &cfgs[0].gossip, setup.genesis.hash(), &mut stream, peer) .await .context("handshake::outbound")?; tracing::info!("The connection is expected to be closed automatically by peer."); @@ -225,17 +225,17 @@ async fn test_validator_addrs_propagation() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::AffineClock::new(40.)); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 10); + let setup = validator::testonly::Setup::new(rng, 10); let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); let nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new(cfg.clone(), store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg.clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) @@ -243,13 +243,15 @@ async fn test_validator_addrs_propagation() { let want: HashMap<_, _> = cfgs .iter() .map(|cfg| { - let cfg = cfg.consensus.as_ref().unwrap(); - (cfg.key.public(), cfg.public_addr) + ( + cfg.validator_key.as_ref().unwrap().public(), + cfg.public_addr, + ) }) .collect(); for (i, node) in nodes.iter().enumerate() { tracing::info!("awaiting for node[{i}] to learn validator_addrs"); - let sub = &mut node.state.gossip.validator_addrs.subscribe(); + let sub = &mut node.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| want == to_addr_map(got)).await?; } Ok(()) @@ -258,6 +260,57 @@ async fn test_validator_addrs_propagation() { .unwrap(); } +#[tokio::test] +async fn test_genesis_mismatch() { + abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + let setup = validator::testonly::Setup::new(rng, 2); + let cfgs = testonly::new_configs(rng, &setup, 1); + + scope::run!(ctx, |ctx, s| async { + let mut listener = cfgs[1].server_addr.bind().context("server_addr.bind()")?; + + tracing::info!("Start one node, we will simulate the other one."); + let (store, runner) = new_store(ctx, &setup.genesis).await; + s.spawn_bg(runner.run(ctx)); + let (_node, runner) = testonly::Instance::new(ctx, cfgs[0].clone(), store.clone()); + s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); + + tracing::info!("Accept a connection with mismatching genesis."); + let stream = metrics::MeteredStream::listen(ctx, &mut listener) + .await? + .context("listen()")?; + let (mut stream, endpoint) = preface::accept(ctx, stream) + .await + .context("preface::accept()")?; + assert_eq!(endpoint, preface::Endpoint::GossipNet); + tracing::info!("Expect the handshake to fail"); + let res = handshake::inbound(ctx, &cfgs[1].gossip, rng.gen(), &mut stream).await; + assert_matches!(res, Err(handshake::Error::GenesisMismatch)); + + tracing::info!("Try to connect to a node with a mismatching genesis."); + let mut stream = preface::connect(ctx, cfgs[0].public_addr, preface::Endpoint::GossipNet) + .await + .context("preface::connect")?; + let res = handshake::outbound( + ctx, + &cfgs[1].gossip, + rng.gen(), + &mut stream, + &cfgs[0].gossip.key.public(), + ) + .await; + tracing::info!( + "Expect the peer to verify the mismatching Genesis and close the connection." + ); + assert_matches!(res, Err(handshake::Error::Stream(_))); + Ok(()) + }) + .await + .unwrap(); +} + const EXCHANGED_STATE_COUNT: usize = 5; const NETWORK_CONNECTIVITY_CASES: [(usize, usize); 5] = [(2, 1), (3, 2), (5, 3), (10, 4), (10, 7)]; @@ -272,21 +325,21 @@ async fn syncing_blocks(node_count: usize, gossip_peers: usize) { let ctx = &ctx::test_root(&ctx::AffineClock::new(20.0)); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::GenesisSetup::new(rng, node_count); + let mut setup = validator::testonly::Setup::new(rng, node_count); + setup.push_blocks(rng, EXCHANGED_STATE_COUNT); let cfgs = testonly::new_configs(rng, &setup, gossip_peers); scope::run!(ctx, |ctx, s| async { let mut nodes = vec![]; for (i, cfg) in cfgs.into_iter().enumerate() { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store); + let (node, runner) = testonly::Instance::new(ctx, cfg, store); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } - setup.push_blocks(rng, EXCHANGED_STATE_COUNT); for block in &setup.blocks { for node in &nodes { - node.state + node.net .gossip .block_store .queue_block(ctx, block.clone()) @@ -319,7 +372,7 @@ async fn wait_for_updates( else { continue; }; - if state.last == block.justification { + if state.last.as_ref() == Some(&block.justification) { updates.insert(peer); } response.send(()).ok(); @@ -345,7 +398,7 @@ async fn uncoordinated_block_syncing( let ctx = &ctx::test_root(&ctx::AffineClock::new(20.0)); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::GenesisSetup::empty(rng, node_count); + let mut setup = validator::testonly::Setup::new(rng, node_count); setup.push_blocks(rng, EXCHANGED_STATE_COUNT); scope::run!(ctx, |ctx, s| async { for (i, cfg) in testonly::new_configs(rng, &setup, gossip_peers) @@ -353,13 +406,13 @@ async fn uncoordinated_block_syncing( .enumerate() { let i = i; - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg, store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); s.spawn(async { let store = store; - for block in &setup.blocks[1..] { + for block in &setup.blocks { ctx.sleep(state_generation_interval).await?; store.queue_block(ctx, block.clone()).await.unwrap(); } @@ -383,20 +436,26 @@ async fn getting_blocks_from_peers(node_count: usize, gossip_peers: usize) { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, node_count); + let mut setup = validator::testonly::Setup::new(rng, node_count); + setup.push_blocks(rng, 1); let cfgs = testonly::new_configs(rng, &setup, gossip_peers); // All inbound and outbound peers should answer the request. let expected_successful_responses = (2 * gossip_peers).min(node_count - 1); - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; scope::run!(ctx, |ctx, s| async { + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); + store + .queue_block(ctx, setup.blocks[0].clone()) + .await + .unwrap(); + let mut nodes: Vec<_> = cfgs .into_iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new(cfg, store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg, store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) @@ -410,7 +469,7 @@ async fn getting_blocks_from_peers(node_count: usize, gossip_peers: usize) { let (response, response_receiver) = oneshot::channel(); node.pipe.send( io::SyncBlocksInputMessage::GetBlock { - recipient: peer.state.gossip.cfg.key.public(), + recipient: peer.net.gossip.cfg.gossip.key.public(), number: setup.blocks[0].header().number, response, } @@ -429,7 +488,7 @@ async fn getting_blocks_from_peers(node_count: usize, gossip_peers: usize) { let last = nodes.pop().unwrap(); last.terminate(ctx).await?; - let stopped_node_key = last.state.gossip.cfg.key.public(); + let stopped_node_key = last.net.gossip.cfg.gossip.key.public(); for node in &nodes { tracing::info!("wait for disconnection"); node.wait_for_gossip_disconnect(ctx, &stopped_node_key) @@ -463,17 +522,21 @@ async fn validator_node_restart() { abort_on_panic(); let _guard = set_timeout(time::Duration::seconds(5)); - let clock = &ctx::ManualClock::new(); - let ctx = &ctx::test_root(clock); + let clock = ctx::ManualClock::new(); + let ctx = &ctx::test_root(&clock); let rng = &mut ctx.rng(); let zero = time::Duration::ZERO; let sec = time::Duration::seconds(1); - let setup = validator::testonly::GenesisSetup::new(rng, 2); + let setup = validator::testonly::Setup::new(rng, 2); let mut cfgs = testonly::new_configs(rng, &setup, 1); - let (store, store_runner) = new_store(ctx, &setup.blocks[0]).await; - let (mut node1, node1_runner) = testonly::Instance::new(cfgs[1].clone(), store.clone()); + // Set the rpc refresh time to 0, so that any updates are immediately propagated. + for cfg in &mut cfgs { + cfg.rpc.push_validator_addrs_rate.refresh = time::Duration::ZERO; + } + let (store, store_runner) = new_store(ctx, &setup.genesis).await; + let (node1, node1_runner) = testonly::Instance::new(ctx, cfgs[1].clone(), store.clone()); scope::run!(ctx, |ctx, s| async { s.spawn_bg(store_runner.run(ctx)); s.spawn_bg( @@ -481,15 +544,6 @@ async fn validator_node_restart() { .run(ctx) .instrument(tracing::info_span!("node1")), ); - s.spawn_bg(async { - // Progress time whenever node1 receives an update. - // TODO(gprusak): alternatively we could entirely disable time progress - // by setting refresh time to 0 in tests. - while let Ok(Event::ValidatorAddrsUpdated) = node1.events.recv(ctx).await { - clock.advance(rpc::push_validator_addrs::Rpc::RATE.refresh); - } - Ok(()) - }); // We restart the node0 after shifting the UTC clock back and forth. // node0 is expected to learn what was is the currently broadcasted @@ -497,11 +551,9 @@ async fn validator_node_restart() { let mut utc_times = HashSet::new(); let start = ctx.now_utc(); for clock_shift in [zero, sec, -2 * sec, 4 * sec, 10 * sec, -30 * sec] { - tracing::error!("DUPA {clock_shift}"); // Set the new addr to broadcast. - let mutated_config = cfgs[0].consensus.as_mut().unwrap(); let addr0 = mk_addr(rng); - mutated_config.public_addr = addr0; + cfgs[0].public_addr = addr0; // Shift the UTC clock. let now = start + clock_shift; assert!( @@ -511,18 +563,17 @@ async fn validator_node_restart() { clock.set_utc(now); tracing::info!("now = {now:?}"); + // _node0 contains pipe, which has to exist to prevent the connection from dying + // early. + let (_node0, runner) = testonly::Instance::new(ctx, cfgs[0].clone(), store.clone()); scope::run!(ctx, |ctx, s| async { - // _node0 contains pipe, which has to exist to prevent the connection from dying - // early. - let (_node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node0"))); tracing::info!("wait for the update to arrive to node1"); - let sub = &mut node1.state.gossip.validator_addrs.subscribe(); + let sub = &mut node1.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| { let Some(got) = got.get(&setup.keys[0].public()) else { return false; }; - tracing::info!("got.addr = {}", got.msg.addr); got.msg.addr == addr0 }) .await?; @@ -549,32 +600,35 @@ async fn rate_limiting() { // construct star topology. let n = 10; - let setup = validator::testonly::GenesisSetup::new(rng, n); + let setup = validator::testonly::Setup::new(rng, n); let mut cfgs = testonly::new_configs(rng, &setup, 0); let want: HashMap<_, _> = cfgs .iter() .map(|cfg| { - let consensus_cfg = cfg.consensus.as_ref().unwrap(); - (consensus_cfg.key.public(), consensus_cfg.public_addr) + ( + cfg.validator_key.as_ref().unwrap().public(), + cfg.public_addr, + ) }) .collect(); for i in 1..n { let key = cfgs[i].gossip.key.public().clone(); - let public_addr = cfgs[i].consensus.as_ref().unwrap().public_addr; + let public_addr = cfgs[i].public_addr; cfgs[0].gossip.static_outbound.insert(key, public_addr); } let mut nodes = vec![]; scope::run!(ctx, |ctx, s| async { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); // Spawn the satellite nodes and wait until they register // their own address. for (i, cfg) in cfgs[1..].iter().enumerate() { - let (node, runner) = testonly::Instance::new(cfg.clone(), store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg.clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); - let sub = &mut node.state.gossip.validator_addrs.subscribe(); + let sub = &mut node.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| { - got.get(&node.consensus_config().key.public()).is_some() + got.get(&node.cfg().validator_key.as_ref().unwrap().public()) + .is_some() }) .await .unwrap(); @@ -582,15 +636,15 @@ async fn rate_limiting() { } // Spawn the center node. - let (center, runner) = testonly::Instance::new(cfgs[0].clone(), store.clone()); + let (center, runner) = testonly::Instance::new(ctx, cfgs[0].clone(), store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node[0]"))); // Await for the center to receive all validator addrs. - let sub = &mut center.state.gossip.validator_addrs.subscribe(); + let sub = &mut center.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| want == to_addr_map(got)).await?; // Advance time and wait for all other nodes to receive validator addrs. - clock.advance(rpc::push_validator_addrs::Rpc::RATE.refresh); + clock.advance(center.cfg().rpc.push_validator_addrs_rate.refresh); for node in &nodes { - let sub = &mut node.state.gossip.validator_addrs.subscribe(); + let sub = &mut node.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| want == to_addr_map(got)).await?; } Ok(()) @@ -600,10 +654,11 @@ async fn rate_limiting() { // Check that the satellite nodes received either 1 or 2 updates. for n in &mut nodes { - let mut count = 0; - while let Some(Event::ValidatorAddrsUpdated) = n.events.try_recv() { - count += 1; - } - assert!((1..=2).contains(&count)); + let got = n + .net + .gossip + .push_validator_addrs_calls + .load(Ordering::SeqCst); + assert!((1..=2).contains(&got), "got {got} want 1 or 2"); } } diff --git a/node/actors/network/src/gossip/validator_addrs.rs b/node/actors/network/src/gossip/validator_addrs.rs new file mode 100644 index 00000000..ed176bf5 --- /dev/null +++ b/node/actors/network/src/gossip/validator_addrs.rs @@ -0,0 +1,110 @@ +//! Global state distributed by active validators, observed by all the nodes in the network. +use crate::watch::Watch; +use std::{collections::HashSet, sync::Arc}; +use zksync_concurrency::sync; +use zksync_consensus_roles::validator; + +/// Mapping from validator::PublicKey to a signed validator::NetAddress. +/// Represents the currents state of node's knowledge about the validator endpoints. +#[derive(Clone, Default, PartialEq, Eq)] +pub(crate) struct ValidatorAddrs( + pub(super) im::HashMap>>, +); + +impl ValidatorAddrs { + /// Gets a NetAddress for a given key. + pub(crate) fn get( + &self, + key: &validator::PublicKey, + ) -> Option<&Arc>> { + self.0.get(key) + } + + /// Returns a set of entries of `self` which are newer than the entries in `b`. + pub(super) fn get_newer(&self, b: &Self) -> Vec>> { + let mut newer = vec![]; + for (k, v) in &self.0 { + if let Some(bv) = b.0.get(k) { + if !v.msg.is_newer(&bv.msg) { + continue; + } + } + newer.push(v.clone()); + } + newer + } + + /// Updates the discovery map with entries from `data`. + /// It exits as soon as an invalid entry is found. + /// `self` might get modified even if an error is returned + /// (all entries verified so far are added). + /// Returns true iff some new entry was added. + pub(super) fn update( + &mut self, + validators: &validator::ValidatorSet, + data: &[Arc>], + ) -> anyhow::Result { + let mut changed = false; + + let mut done = HashSet::new(); + for d in data { + // Disallow multiple entries for the same key: + // it is important because a malicious validator may spam us with + // new versions and verifying signatures is expensive. + if done.contains(&d.key) { + anyhow::bail!("duplicate entry for {:?}", d.key); + } + done.insert(d.key.clone()); + if !validators.contains(&d.key) { + // We just skip the entries we are not interested in. + // For now the set of validators is static, so we could treat this as an error, + // however we eventually want the validator set to be dynamic. + continue; + } + if let Some(x) = self.0.get(&d.key) { + if !d.msg.is_newer(&x.msg) { + continue; + } + } + d.verify()?; + self.0.insert(d.key.clone(), d.clone()); + changed = true; + } + Ok(changed) + } +} + +/// Watch wrapper of ValidatorAddrs, +/// which supports subscribing to ValidatorAddr updates. +pub(crate) struct ValidatorAddrsWatch(Watch); + +impl Default for ValidatorAddrsWatch { + fn default() -> Self { + Self(Watch::new(ValidatorAddrs::default())) + } +} + +impl ValidatorAddrsWatch { + /// Subscribes to ValidatorAddrs updates. + pub(crate) fn subscribe(&self) -> sync::watch::Receiver { + self.0.subscribe() + } + + /// Inserts data to ValidatorAddrs. + /// Subscribers are notified iff at least 1 new entry has + /// been inserted. Returns an error iff an invalid + /// entry in `data` has been found. The provider of the + /// invalid entry should be banned. + pub(crate) async fn update( + &self, + validators: &validator::ValidatorSet, + data: &[Arc>], + ) -> anyhow::Result<()> { + let this = self.0.lock().await; + let mut validator_addrs = this.borrow().clone(); + if validator_addrs.update(validators, data)? { + this.send(validator_addrs).ok().unwrap(); + } + Ok(()) + } +} diff --git a/node/actors/network/src/lib.rs b/node/actors/network/src/lib.rs index 0f811d39..73c25cb4 100644 --- a/node/actors/network/src/lib.rs +++ b/node/actors/network/src/lib.rs @@ -1,11 +1,12 @@ //! Network actor maintaining a pool of outbound and inbound connections to other nodes. +use anyhow::Context as _; +use std::sync::Arc; +use zksync_concurrency::{ctx, ctx::channel, scope, time}; +use zksync_consensus_storage::BlockStore; +use zksync_consensus_utils::pipe::ActorPipe; -// &*x is not equivalent to x, because it affects borrowing in closures. -#![allow(clippy::borrow_deref_ref)] - -pub use state::*; +mod config; pub mod consensus; -mod event; mod frame; pub mod gossip; pub mod io; @@ -21,3 +22,186 @@ pub mod testonly; #[cfg(test)] mod tests; mod watch; + +pub use config::*; + +/// State of the network actor observable outside of the actor. +pub struct Network { + /// Consensus network state. + pub(crate) consensus: Option>, + /// Gossip network state. + pub(crate) gossip: Arc, +} + +/// Runner of the Network background tasks. +#[must_use] +pub struct Runner { + /// Network state. + net: Arc, + /// Receiver of the messages from the dispatcher. + receiver: channel::UnboundedReceiver, +} + +impl Network { + /// Constructs a new network actor state. + /// Call `run_network` to run the actor. + pub fn new( + ctx: &ctx::Ctx, + cfg: Config, + block_store: Arc, + pipe: ActorPipe, + ) -> (Arc, Runner) { + let gossip = gossip::Network::new(cfg, block_store, pipe.send); + let consensus = consensus::Network::new(ctx, gossip.clone()); + let net = Arc::new(Self { gossip, consensus }); + ( + net.clone(), + Runner { + net, + receiver: pipe.recv, + }, + ) + } + + /// Registers metrics for this state. + pub fn register_metrics(self: &Arc) { + metrics::NetworkGauges::register(Arc::downgrade(self)); + } + + /// Handles a dispatcher message. + async fn handle_message( + &self, + ctx: &ctx::Ctx, + message: io::InputMessage, + ) -> anyhow::Result<()> { + /// Timeout for handling a consensus message. + const CONSENSUS_MSG_TIMEOUT: time::Duration = time::Duration::seconds(10); + /// Timeout for a GetBlock RPC. + const GET_BLOCK_TIMEOUT: time::Duration = time::Duration::seconds(10); + + match message { + io::InputMessage::Consensus(message) => { + let consensus = self.consensus.as_ref().context("not a validator node")?; + let ctx = &ctx.with_timeout(CONSENSUS_MSG_TIMEOUT); + match message.recipient { + io::Target::Validator(key) => { + consensus.send(ctx, &key, message.message).await? + } + io::Target::Broadcast => consensus.broadcast(ctx, message.message).await?, + } + } + io::InputMessage::SyncBlocks(io::SyncBlocksInputMessage::GetBlock { + recipient, + number, + response, + }) => { + let ctx = &ctx.with_timeout(GET_BLOCK_TIMEOUT); + let _ = response.send(match self.gossip.get_block(ctx, &recipient, number).await { + Ok(Some(block)) => Ok(block), + Ok(None) => Err(io::GetBlockError::NotAvailable), + Err(err) => Err(io::GetBlockError::Internal(err)), + }); + } + } + Ok(()) + } +} + +impl Runner { + /// Runs the network actor. + pub async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + let mut listener = self + .net + .gossip + .cfg + .server_addr + .bind() + .context("server_addr.bind()")?; + + scope::run!(ctx, |ctx, s| async { + // Handle incoming messages. + s.spawn(async { + // We don't propagate cancellation errors + while let Ok(message) = self.receiver.recv(ctx).await { + s.spawn(async { + if let Err(err) = self.net.handle_message(ctx, message).await { + tracing::info!("handle_message(): {err:#}"); + } + Ok(()) + }); + } + Ok(()) + }); + + // Maintain static gossip connections. + for (peer, addr) in &self.net.gossip.cfg.gossip.static_outbound { + s.spawn(async { + loop { + let run_result = + self.net.gossip.run_outbound_stream(ctx, peer, *addr).await; + if let Err(err) = run_result { + tracing::info!("gossip.run_outbound_stream(): {err:#}"); + } + if let Err(ctx::Canceled) = ctx.sleep(CONNECT_RETRY).await { + return Ok(()); + } + } + }); + } + + if let Some(c) = &self.net.consensus { + // If we are active validator ... + if c.gossip.genesis().validators.contains(&c.key.public()) { + // Maintain outbound connections. + for peer in c.clients.keys() { + s.spawn(async { + c.maintain_connection(ctx, peer).await; + Ok(()) + }); + } + // Announce IP periodically. + s.spawn(async { + c.run_address_announcer(ctx).await; + Ok(()) + }); + } + } + + // TODO(gprusak): add rate limit and inflight limit for inbound handshakes. + while let Ok(stream) = metrics::MeteredStream::listen(ctx, &mut listener).await { + let stream = stream.context("listener.accept()")?; + s.spawn(async { + let res = async { + let (stream, endpoint) = preface::accept(ctx, stream) + .await + .context("preface::accept()")?; + match endpoint { + preface::Endpoint::ConsensusNet => { + if let Some(c) = &self.net.consensus { + c.run_inbound_stream(ctx, stream) + .await + .context("consensus.run_inbound_stream()")?; + } + } + preface::Endpoint::GossipNet => { + self.net + .gossip + .run_inbound_stream(ctx, stream) + .await + .context("gossip.run_inbound_stream()")?; + } + } + anyhow::Ok(()) + } + .await; + if let Err(err) = res { + tracing::info!("{err:#}"); + } + Ok(()) + }); + } + Ok(()) + }) + .await + } +} diff --git a/node/actors/network/src/metrics.rs b/node/actors/network/src/metrics.rs index 8503b7da..86fa1cf5 100644 --- a/node/actors/network/src/metrics.rs +++ b/node/actors/network/src/metrics.rs @@ -1,6 +1,6 @@ //! General-purpose network metrics. -use crate::state::State; +use crate::Network; use std::{ net::SocketAddr, pin::Pin, @@ -138,7 +138,7 @@ pub(crate) struct NetworkGauges { impl NetworkGauges { /// Registers a metrics collector for the specified state. - pub(crate) fn register(state_ref: Weak) { + pub(crate) fn register(state_ref: Weak) { #[vise::register] static COLLECTOR: Collector> = Collector::new(); diff --git a/node/actors/network/src/mux/tests/mod.rs b/node/actors/network/src/mux/tests/mod.rs index e797a917..0c43d0ea 100644 --- a/node/actors/network/src/mux/tests/mod.rs +++ b/node/actors/network/src/mux/tests/mod.rs @@ -9,7 +9,6 @@ use std::{ }, }; use zksync_concurrency::{ctx, scope, testonly::abort_on_panic}; -use zksync_consensus_utils::no_copy::NoCopy; use zksync_protobuf::ProtoFmt as _; mod proto; @@ -193,10 +192,6 @@ fn expected(res: Result<(), mux::RunError>) -> Result<(), mux::RunError> { // * multiple capabilities are used at the same time. // * ends use totally different configs // * messages are larger than frames -// -// TODO(gprusak): in case the test fails it may be hard to find the actual bug, because -// this test covers a lot of features. In such situation more specific tests -// checking 1 property at a time should be added. #[test] fn mux_with_noise() { abort_on_panic(); @@ -247,35 +242,35 @@ fn mux_with_noise() { scope::run!(ctx, |ctx, s| async { let (s1, s2) = noise::testonly::pipe(ctx).await; for (cap, q) in mux1.connect.clone() { - let cap = NoCopy::from(cap); + let cap = ctx::NoCopy(cap); s.spawn_bg(async { run_server(ctx, q, *cap) .await - .with_context(|| format!("server({})", cap.into_inner())) + .with_context(|| format!("server({})", cap.into())) }); } for (cap, q) in mux1.accept.clone() { - let cap = NoCopy::from(cap); + let cap = ctx::NoCopy(cap); s.spawn(async { run_client(ctx, q, *cap) .await - .with_context(|| format!("client({})", cap.into_inner())) + .with_context(|| format!("client({})", cap.into())) }); } for (cap, q) in mux2.connect.clone() { - let cap = NoCopy::from(cap); + let cap = ctx::NoCopy(cap); s.spawn_bg(async { run_server(ctx, q, *cap) .await - .with_context(|| format!("server({})", cap.into_inner())) + .with_context(|| format!("server({})", cap.into())) }); } for (cap, q) in mux2.accept.clone() { - let cap = NoCopy::from(cap); + let cap = ctx::NoCopy(cap); s.spawn(async { run_client(ctx, q, *cap) .await - .with_context(|| format!("client({})", cap.into_inner())) + .with_context(|| format!("client({})", cap.into())) }); } s.spawn_bg(async { expected(mux1.run(ctx, s1).await).context("mux1.run()") }); diff --git a/node/actors/network/src/proto/consensus.proto b/node/actors/network/src/proto/consensus.proto index e86aaeaf..919b6d57 100644 --- a/node/actors/network/src/proto/consensus.proto +++ b/node/actors/network/src/proto/consensus.proto @@ -7,7 +7,8 @@ import "zksync/std.proto"; // First message exchanged in the encrypted session. message Handshake { - optional roles.validator.Signed session_id = 1; + optional roles.validator.Signed session_id = 1; // required + optional roles.validator.GenesisHash genesis = 2; // required } message ConsensusReq { diff --git a/node/actors/network/src/proto/gossip.proto b/node/actors/network/src/proto/gossip.proto index 976ba1b8..db219151 100644 --- a/node/actors/network/src/proto/gossip.proto +++ b/node/actors/network/src/proto/gossip.proto @@ -7,8 +7,9 @@ import "zksync/roles/validator.proto"; // First message exchanged in the encrypted session. message Handshake { - optional roles.node.Signed session_id = 1; - optional bool is_static = 2; + optional roles.node.Signed session_id = 1; // required + optional roles.validator.GenesisHash genesis = 3; // required + optional bool is_static = 2; // required } message PushValidatorAddrs { @@ -21,9 +22,9 @@ message PushValidatorAddrs { // and actively fetch newest blocks. message PushBlockStoreState { // First L2 block that the node has locally. - optional roles.validator.CommitQC first = 1; + optional uint64 first = 1; // required; BlockNumber // Last L2 block that the node has locally. - optional roles.validator.CommitQC last = 2; + optional roles.validator.CommitQC last = 2; // optional } // Asks the server to send an L2 block (including its transactions). diff --git a/node/actors/network/src/rpc/consensus.rs b/node/actors/network/src/rpc/consensus.rs index 498cd054..40082c2a 100644 --- a/node/actors/network/src/rpc/consensus.rs +++ b/node/actors/network/src/rpc/consensus.rs @@ -1,6 +1,5 @@ //! Defines RPC for passing consensus messages. use crate::{mux, proto::consensus as proto}; -use zksync_concurrency::{limiter, time}; use zksync_consensus_roles::validator; use zksync_protobuf::{read_required, ProtoFmt}; @@ -10,10 +9,6 @@ pub(crate) struct Rpc; impl super::Rpc for Rpc { const CAPABILITY_ID: mux::CapabilityId = 0; const INFLIGHT: u32 = 3; - const RATE: limiter::Rate = limiter::Rate { - burst: 10, - refresh: time::Duration::ZERO, - }; const METHOD: &'static str = "consensus"; type Req = Req; type Resp = Resp; diff --git a/node/actors/network/src/rpc/get_block.rs b/node/actors/network/src/rpc/get_block.rs index c4996168..45b208d0 100644 --- a/node/actors/network/src/rpc/get_block.rs +++ b/node/actors/network/src/rpc/get_block.rs @@ -1,7 +1,6 @@ //! RPC for fetching a block from peer. use crate::{mux, proto::gossip as proto}; use anyhow::Context; -use zksync_concurrency::{limiter, time}; use zksync_consensus_roles::validator::{BlockNumber, FinalBlock}; use zksync_protobuf::{read_optional, ProtoFmt}; @@ -13,10 +12,6 @@ pub(crate) struct Rpc; impl super::Rpc for Rpc { const CAPABILITY_ID: mux::CapabilityId = 4; const INFLIGHT: u32 = 5; - const RATE: limiter::Rate = limiter::Rate { - burst: 10, - refresh: time::Duration::milliseconds(100), - }; const METHOD: &'static str = "get_block"; type Req = Req; diff --git a/node/actors/network/src/rpc/mod.rs b/node/actors/network/src/rpc/mod.rs index d98a08b3..184326cd 100644 --- a/node/actors/network/src/rpc/mod.rs +++ b/node/actors/network/src/rpc/mod.rs @@ -52,9 +52,6 @@ pub(crate) trait Rpc: Sync + Send + 'static { /// Maximal number of calls executed in parallel. /// Both client and server enforce this limit. const INFLIGHT: u32; - /// Maximal rate at which calls can be made. - /// Both client and server enforce this limit. - const RATE: limiter::Rate; /// Name of the RPC, used in prometheus metrics. const METHOD: &'static str; /// Type of the request message. @@ -122,9 +119,9 @@ pub(crate) struct Client { impl Client { /// Constructs a new client. - pub(crate) fn new(ctx: &ctx::Ctx) -> Self { + pub(crate) fn new(ctx: &ctx::Ctx, rate: limiter::Rate) -> Self { Client { - limiter: limiter::Limiter::new(ctx, R::RATE), + limiter: limiter::Limiter::new(ctx, rate), queue: mux::StreamQueue::new(R::INFLIGHT), _rpc: std::marker::PhantomData, } @@ -175,6 +172,7 @@ pub(crate) trait Handler: Sync + Send { struct Server> { handler: H, queue: Arc, + rate: limiter::Rate, _rpc: std::marker::PhantomData, } @@ -188,7 +186,7 @@ impl> ServerTrait for Server { /// Serves the incoming RPCs, respecting the rate limit and /// max inflight limit. async fn serve(&self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { - let limiter = limiter::Limiter::new(ctx, R::RATE); + let limiter = limiter::Limiter::new(ctx, self.rate); scope::run!(ctx, |ctx, s| async { for _ in 0..R::INFLIGHT { s.spawn::<()>(async { @@ -279,7 +277,11 @@ impl<'a> Service<'a> { } /// Adds a server to the RPC service. - pub(crate) fn add_server(mut self, handler: impl Handler + 'a) -> Self { + pub(crate) fn add_server( + mut self, + handler: impl Handler + 'a, + rate: limiter::Rate, + ) -> Self { let queue = mux::StreamQueue::new(R::INFLIGHT); if self .mux @@ -295,6 +297,7 @@ impl<'a> Service<'a> { self.servers.push(Box::new(Server { handler, queue, + rate, _rpc: std::marker::PhantomData, })); self diff --git a/node/actors/network/src/rpc/ping.rs b/node/actors/network/src/rpc/ping.rs index 5307eb1c..a11e07ed 100644 --- a/node/actors/network/src/rpc/ping.rs +++ b/node/actors/network/src/rpc/ping.rs @@ -1,5 +1,5 @@ //! Defines an RPC for sending ping messages. -use crate::{mux, proto::ping as proto, rpc::Rpc as _}; +use crate::{mux, proto::ping as proto}; use anyhow::Context as _; use rand::Rng; use zksync_concurrency::{ctx, limiter, time}; @@ -11,15 +11,19 @@ pub(crate) struct Rpc; impl super::Rpc for Rpc { const CAPABILITY_ID: mux::CapabilityId = 2; const INFLIGHT: u32 = 1; - const RATE: limiter::Rate = limiter::Rate { - burst: 1, - refresh: time::Duration::seconds(10), - }; const METHOD: &'static str = "ping"; type Req = Req; type Resp = Resp; } +/// Hardcoded expected rate supported by the server. +/// This needs to be part of the protocol, so that both parties agree on when +/// connection is alive. +pub(crate) const RATE: limiter::Rate = limiter::Rate { + burst: 2, + refresh: time::Duration::seconds(1), +}; + /// Canonical Ping server implementation, /// which responds with data from the request. pub(crate) struct Server; @@ -35,7 +39,7 @@ impl super::Handler for Server { } impl super::Client { - /// Calls ping RPC every `Rpc.RATE.refresh`. + /// Sends a ping every `timeout`. /// Returns an error if any single ping request fails or /// exceeds `timeout`. pub(crate) async fn ping_loop( @@ -44,13 +48,17 @@ impl super::Client { timeout: time::Duration, ) -> anyhow::Result<()> { loop { - ctx.sleep(Rpc::RATE.refresh).await?; - let ctx = &ctx.with_timeout(timeout); let req = Req(ctx.rng().gen()); - let resp = self.call(ctx, &req, kB).await.context("ping")?; + let resp = self + .call(&ctx.with_timeout(timeout), &req, kB) + .await + .context("ping")?; if req.0 != resp.0 { anyhow::bail!("bad ping response"); } + if let Err(ctx::Canceled) = ctx.sleep(timeout).await { + return Ok(()); + } } } } diff --git a/node/actors/network/src/rpc/push_block_store_state.rs b/node/actors/network/src/rpc/push_block_store_state.rs index ef340c25..52c1161c 100644 --- a/node/actors/network/src/rpc/push_block_store_state.rs +++ b/node/actors/network/src/rpc/push_block_store_state.rs @@ -1,9 +1,9 @@ //! RPC for notifying peer about our BlockStore state. use crate::{mux, proto::gossip as proto}; use anyhow::Context; -use zksync_concurrency::{limiter, time}; +use zksync_consensus_roles::validator; use zksync_consensus_storage::BlockStoreState; -use zksync_protobuf::{read_required, ProtoFmt}; +use zksync_protobuf::{read_optional, required, ProtoFmt}; /// PushBlockStoreState RPC. #[derive(Debug)] @@ -12,10 +12,6 @@ pub(crate) struct Rpc; impl super::Rpc for Rpc { const CAPABILITY_ID: mux::CapabilityId = 3; const INFLIGHT: u32 = 1; - const RATE: limiter::Rate = limiter::Rate { - burst: 2, - refresh: time::Duration::milliseconds(500), - }; const METHOD: &'static str = "push_block_store_state"; type Req = Req; @@ -31,15 +27,15 @@ impl ProtoFmt for Req { fn read(message: &Self::Proto) -> anyhow::Result { Ok(Self(BlockStoreState { - first: read_required(&message.first).context("first")?, - last: read_required(&message.last).context("last")?, + first: validator::BlockNumber(*required(&message.first).context("first")?), + last: read_optional(&message.last).context("last")?, })) } fn build(&self) -> Self::Proto { Self::Proto { - first: Some(self.0.first.build()), - last: Some(self.0.last.build()), + first: Some(self.0.first.0), + last: self.0.last.as_ref().map(|x| x.build()), } } } diff --git a/node/actors/network/src/rpc/push_validator_addrs.rs b/node/actors/network/src/rpc/push_validator_addrs.rs index 020a984f..c58803bf 100644 --- a/node/actors/network/src/rpc/push_validator_addrs.rs +++ b/node/actors/network/src/rpc/push_validator_addrs.rs @@ -2,7 +2,6 @@ use crate::{mux, proto::gossip as proto}; use anyhow::Context as _; use std::sync::Arc; -use zksync_concurrency::{limiter, time}; use zksync_consensus_roles::validator; use zksync_protobuf::ProtoFmt; @@ -12,10 +11,6 @@ pub(crate) struct Rpc; impl super::Rpc for Rpc { const CAPABILITY_ID: mux::CapabilityId = 1; const INFLIGHT: u32 = 1; - const RATE: limiter::Rate = limiter::Rate { - burst: 1, - refresh: time::Duration::seconds(5), - }; const METHOD: &'static str = "push_validator_addrs"; type Req = Req; diff --git a/node/actors/network/src/rpc/tests.rs b/node/actors/network/src/rpc/tests.rs index 96020721..3770320c 100644 --- a/node/actors/network/src/rpc/tests.rs +++ b/node/actors/network/src/rpc/tests.rs @@ -45,15 +45,21 @@ async fn test_ping() { let clock = ctx::ManualClock::new(); let ctx = &ctx::test_root(&clock); let (s1, s2) = noise::testonly::pipe(ctx).await; - let client = Client::::new(ctx); + let client = Client::::new(ctx, ping::RATE); scope::run!(ctx, |ctx, s| async { s.spawn_bg(async { - expected(Service::new().add_server(ping::Server).run(ctx, s1).await).context("server") + expected( + Service::new() + .add_server(ping::Server, ping::RATE) + .run(ctx, s1) + .await, + ) + .context("server") }); s.spawn_bg(async { expected(Service::new().add_client(&client).run(ctx, s2).await).context("client") }); - for _ in 0..ping::Rpc::RATE.burst { + for _ in 0..ping::RATE.burst { let req = ping::Req(ctx.rng().gen()); let resp = client.call(ctx, &req, kB).await?; assert_eq!(req.0, resp.0); @@ -63,7 +69,7 @@ async fn test_ping() { let req = ping::Req(ctx.rng().gen()); let resp = client.call(ctx, &req, kB).await?; assert_eq!(req.0, resp.0); - assert!(ctx.now() >= now + ping::Rpc::RATE.refresh); + assert!(ctx.now() >= now + ping::RATE.refresh); Ok(()) }) .await @@ -76,7 +82,7 @@ struct PingServer { } const PING_COUNT: u64 = 3; -const PING_TIMEOUT: time::Duration = time::Duration::seconds(3); +const PING_TIMEOUT: time::Duration = time::Duration::seconds(6); #[async_trait::async_trait] impl Handler for PingServer { @@ -101,8 +107,7 @@ async fn test_ping_loop() { clock.set_advance_on_sleep(); let ctx = &ctx::test_root(&clock); let (s1, s2) = noise::testonly::pipe(ctx).await; - let client = Client::::new(ctx); - let max = 5; + let client = Client::::new(ctx, ping::RATE); scope::run!(ctx, |ctx, s| async { s.spawn_bg(async { // Clock is passed to the server, so that it can @@ -112,13 +117,20 @@ async fn test_ping_loop() { pings: 0.into(), }; - // Use independent clock for server, because - // otherwise both clocks get autoincremented too aggresively. - let clock = ctx::ManualClock::new(); - clock.set_advance_on_sleep(); - let ctx = &ctx::test_with_clock(ctx, &clock); - - expected(Service::new().add_server(server).run(ctx, s1).await).context("server") + expected( + Service::new() + .add_server( + server, + limiter::Rate { + burst: 1, + // with `refresh = 0`, server will never autoadvance time. + refresh: time::Duration::ZERO, + }, + ) + .run(ctx, s1) + .await, + ) + .context("server") }); s.spawn_bg(async { expected(Service::new().add_client(&client).run(ctx, s2).await).context("client") @@ -126,8 +138,9 @@ async fn test_ping_loop() { let now = ctx.now(); assert!(client.ping_loop(ctx, PING_TIMEOUT).await.is_err()); let got = ctx.now() - now; - let want = (max - ping::Rpc::RATE.burst) as u32 * ping::Rpc::RATE.refresh + PING_TIMEOUT; - assert!(got >= want, "want at least {want} latency, but got {got}"); + // PING_COUNT will succeed and the next with time out. + let want = (PING_COUNT + 1) as u32 * PING_TIMEOUT; + assert_eq!(got, want); Ok(()) }) .await @@ -136,13 +149,14 @@ async fn test_ping_loop() { struct ExampleRpc; +const RATE: limiter::Rate = limiter::Rate { + burst: 10, + refresh: time::Duration::ZERO, +}; + impl Rpc for ExampleRpc { const CAPABILITY_ID: mux::CapabilityId = 0; const INFLIGHT: u32 = 5; - const RATE: limiter::Rate = limiter::Rate { - burst: 10, - refresh: time::Duration::ZERO, - }; const METHOD: &'static str = "example"; type Req = (); type Resp = (); @@ -166,10 +180,16 @@ async fn test_inflight() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); let (s1, s2) = noise::testonly::pipe(ctx).await; - let client = Client::::new(ctx); + let client = Client::::new(ctx, RATE); scope::run!(ctx, |ctx, s| async { s.spawn_bg(async { - expected(Service::new().add_server(ExampleServer).run(ctx, s1).await).context("server") + expected( + Service::new() + .add_server(ExampleServer, RATE) + .run(ctx, s1) + .await, + ) + .context("server") }); s.spawn_bg(async { expected(Service::new().add_client(&client).run(ctx, s2).await).context("client") diff --git a/node/actors/network/src/state.rs b/node/actors/network/src/state.rs index 545281b9..529b999a 100644 --- a/node/actors/network/src/state.rs +++ b/node/actors/network/src/state.rs @@ -1,165 +1 @@ //! Network actor maintaining a pool of outbound and inbound connections to other nodes. -use super::{consensus, event::Event, gossip, metrics, preface}; -use crate::io::{InputMessage, OutputMessage}; -use anyhow::Context as _; -use std::sync::Arc; -use zksync_concurrency::{ctx, ctx::channel, net, scope}; -use zksync_consensus_roles::validator; -use zksync_consensus_storage::BlockStore; -use zksync_consensus_utils::pipe::ActorPipe; - -/// Network actor config. -#[derive(Debug, Clone)] -pub struct Config { - /// TCP socket address to listen for inbound connections at. - pub server_addr: net::tcp::ListenerAddr, - /// Validators which - /// - client should establish outbound connections to. - /// - server should accept inbound connections from (1 per validator). - pub validators: validator::ValidatorSet, - /// Gossip network config. - pub gossip: gossip::Config, - /// Consensus network config. If not present, the node will not participate in the consensus network. - pub consensus: Option, - /// Maximal size of the proto-encoded `validator::FinalBlock` in bytes. - pub max_block_size: usize, - /// Enables pinging the peers to make sure that they are alive. - pub enable_pings: bool, -} - -/// Part of configuration shared among network modules. -#[derive(Debug)] -pub(crate) struct SharedConfig { - /// TCP socket address to listen for inbound connections at. - pub(crate) server_addr: net::tcp::ListenerAddr, - /// Validators which - /// - client should establish outbound connections to. - /// - server should accept inbound connections from (1 per validator). - pub(crate) validators: validator::ValidatorSet, - /// Enables pinging the peers to make sure that they are alive. - pub(crate) enable_pings: bool, - /// Maximal size of the proto-encoded `validator::FinalBlock` in bytes. - pub(crate) max_block_size: usize, -} - -/// State of the network actor observable outside of the actor. -pub struct State { - /// Configuration shared among network modules. - pub(crate) cfg: SharedConfig, - /// Consensus network state. - pub(crate) consensus: Option, - /// Gossip network state. - pub(crate) gossip: gossip::State, - - /// TESTONLY: channel of network events which the tests can observe. - // TODO(gprusak): consider if it would be enough to make it pub(crate). - pub(crate) events: Option>, -} - -impl State { - /// Constructs a new network actor state. - /// Call `run_network` to run the actor. - pub fn new( - cfg: Config, - block_store: Arc, - events: Option>, - ) -> anyhow::Result> { - let consensus = cfg - .consensus - .map(|consensus_cfg| consensus::State::new(consensus_cfg, &cfg.validators)) - .transpose()?; - let this = Self { - gossip: gossip::State::new(cfg.gossip, block_store), - consensus, - events, - cfg: SharedConfig { - server_addr: cfg.server_addr, - validators: cfg.validators, - enable_pings: cfg.enable_pings, - max_block_size: cfg.max_block_size, - }, - }; - Ok(Arc::new(this)) - } - - /// Registers metrics for this state. - pub fn register_metrics(self: &Arc) { - metrics::NetworkGauges::register(Arc::downgrade(self)); - } -} - -/// Runs the network actor. -/// WARNING: it is a bug to call multiple times in parallel -/// run_network with the same `state` argument. -/// TODO(gprusak): consider a "runnable" wrapper of `State` -/// which will be consumed by `run_network`. This way we -/// could prevent the bug above. -pub async fn run_network( - ctx: &ctx::Ctx, - state: Arc, - mut pipe: ActorPipe, -) -> anyhow::Result<()> { - let mut listener = state.cfg.server_addr.bind()?; - let (consensus_send, consensus_recv) = channel::unbounded(); - let (gossip_send, gossip_recv) = channel::unbounded(); - - scope::run!(ctx, |ctx, s| async { - s.spawn(async { - // We don't propagate cancellation errors - while let Ok(message) = pipe.recv.recv(ctx).await { - match message { - InputMessage::Consensus(message) => { - consensus_send.send(message); - } - InputMessage::SyncBlocks(message) => { - gossip_send.send(message); - } - } - } - Ok(()) - }); - - s.spawn(async { - gossip::run_client(ctx, state.as_ref(), &pipe.send, gossip_recv) - .await - .context("gossip::run_client") - }); - - if let Some(consensus_state) = &state.consensus { - s.spawn(async { - consensus::run_client(ctx, consensus_state, state.as_ref(), consensus_recv) - .await - .context("consensus::run_client") - }); - } - - // TODO(gprusak): add rate limit and inflight limit for inbound handshakes. - while let Ok(stream) = metrics::MeteredStream::listen(ctx, &mut listener).await { - let stream = stream.context("listener.accept()")?; - s.spawn(async { - let res = async { - let (stream, endpoint) = preface::accept(ctx, stream).await?; - match endpoint { - preface::Endpoint::ConsensusNet => { - consensus::run_inbound_stream(ctx, &state, &pipe.send, stream) - .await - .context("consensus::run_inbound_stream()") - } - preface::Endpoint::GossipNet => { - gossip::run_inbound_stream(ctx, &state, &pipe.send, stream) - .await - .context("gossip::run_inbound_stream()") - } - } - } - .await; - if let Err(err) = res { - tracing::info!("{err:#}"); - } - Ok(()) - }); - } - Ok(()) - }) - .await -} diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index a1c76721..022bb747 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -1,17 +1,12 @@ //! Testonly utilities. #![allow(dead_code)] -use crate::{consensus, event::Event, gossip, Config, State}; +use crate::{Config, GossipConfig, Network, RpcConfig, Runner}; use rand::Rng; use std::{ collections::{HashMap, HashSet}, sync::Arc, }; -use zksync_concurrency::{ - ctx, - ctx::channel, - io, net, scope, - sync::{self}, -}; +use zksync_concurrency::{ctx, ctx::channel, io, net, scope, sync}; use zksync_consensus_roles::{node, validator}; use zksync_consensus_storage::BlockStore; use zksync_consensus_utils::pipe; @@ -45,9 +40,7 @@ pub(crate) async fn forward( /// events channel. pub struct Instance { /// State of the instance. - pub(crate) state: Arc, - /// Stream of events. - pub(crate) events: channel::UnboundedReceiver, + pub(crate) net: Arc, /// Termination signal that can be sent to the node. pub(crate) terminate: channel::Sender<()>, /// Dispatcher end of the network pipe. @@ -55,30 +48,28 @@ pub struct Instance { } /// Construct configs for `n` validators of the consensus. -pub fn new_configs( - rng: &mut R, - setup: &validator::testonly::GenesisSetup, +pub fn new_configs( + rng: &mut impl Rng, + setup: &validator::testonly::Setup, gossip_peers: usize, ) -> Vec { let configs = setup.keys.iter().map(|key| { let addr = net::tcp::testonly::reserve_listener(); Config { server_addr: addr, - validators: setup.validator_set(), + public_addr: *addr, // Pings are disabled in tests by default to avoid dropping connections // due to timeouts. - enable_pings: false, - consensus: Some(consensus::Config { - key: key.clone(), - public_addr: *addr, - }), - gossip: gossip::Config { + ping_timeout: None, + validator_key: Some(key.clone()), + gossip: GossipConfig { key: rng.gen(), - dynamic_inbound_limit: setup.keys.len(), + dynamic_inbound_limit: usize::MAX, static_inbound: HashSet::default(), static_outbound: HashMap::default(), }, max_block_size: usize::MAX, + rpc: RpcConfig::default(), } }); let mut cfgs: Vec<_> = configs.collect(); @@ -95,18 +86,39 @@ pub fn new_configs( cfgs } +/// Constructs a config for a non-validator node, which will +/// establish a gossip connection to `peer`. +pub fn new_fullnode(rng: &mut impl Rng, peer: &Config) -> Config { + let addr = net::tcp::testonly::reserve_listener(); + Config { + server_addr: addr, + public_addr: *addr, + // Pings are disabled in tests by default to avoid dropping connections + // due to timeouts. + ping_timeout: None, + validator_key: None, + gossip: GossipConfig { + key: rng.gen(), + dynamic_inbound_limit: usize::MAX, + static_inbound: HashSet::default(), + static_outbound: [(peer.gossip.key.public(), peer.public_addr)].into(), + }, + max_block_size: usize::MAX, + rpc: RpcConfig::default(), + } +} + /// Runner for Instance. pub struct InstanceRunner { - state: Arc, + runner: Runner, terminate: channel::Receiver<()>, - pipe: pipe::ActorPipe, } impl InstanceRunner { /// Runs the instance background processes. pub async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { scope::run!(ctx, |ctx, s| async { - s.spawn_bg(crate::run_network(ctx, self.state, self.pipe)); + s.spawn_bg(self.runner.run(ctx)); let _ = self.terminate.recv(ctx).await; Ok(()) }) @@ -118,22 +130,22 @@ impl InstanceRunner { impl Instance { /// Construct an instance for a given config. - pub fn new(cfg: Config, block_store: Arc) -> (Self, InstanceRunner) { - let (events_send, events_recv) = channel::unbounded(); + pub fn new( + ctx: &ctx::Ctx, + cfg: Config, + block_store: Arc, + ) -> (Self, InstanceRunner) { let (actor_pipe, dispatcher_pipe) = pipe::new(); - let state = - State::new(cfg, block_store, Some(events_send)).expect("Invalid network config"); + let (net, runner) = Network::new(ctx, cfg, block_store, actor_pipe); let (terminate_send, terminate_recv) = channel::bounded(1); ( Self { - state: state.clone(), - events: events_recv, + net, pipe: dispatcher_pipe, terminate: terminate_send, }, InstanceRunner { - state: state.clone(), - pipe: actor_pipe, + runner, terminate: terminate_recv, }, ) @@ -154,30 +166,25 @@ impl Instance { } /// State getter. - pub fn state(&self) -> &Arc { - &self.state + pub fn state(&self) -> &Arc { + &self.net } - /// Returns the consensus config for this node, assuming it is a validator. - pub fn consensus_config(&self) -> &consensus::Config { - &self - .state - .consensus - .as_ref() - .expect("Node is not a validator") - .cfg + /// Genesis. + pub fn genesis(&self) -> &validator::Genesis { + self.net.gossip.genesis() } /// Returns the gossip config for this node. - pub fn gossip_config(&self) -> &gossip::Config { - &self.state.gossip.cfg + pub fn cfg(&self) -> &Config { + &self.net.gossip.cfg } /// Wait for static outbound gossip connections to be established. pub async fn wait_for_gossip_connections(&self) { - let gossip_state = &self.state.gossip; - let want: HashSet<_> = gossip_state.cfg.static_outbound.keys().cloned().collect(); - gossip_state + let want: HashSet<_> = self.cfg().gossip.static_outbound.keys().cloned().collect(); + self.net + .gossip .outbound .subscribe() .wait_for(|got| want.is_subset(got.current())) @@ -187,9 +194,9 @@ impl Instance { /// Waits for all the consensus connections to be established. pub async fn wait_for_consensus_connections(&self) { - let consensus_state = self.state.consensus.as_ref().unwrap(); + let consensus_state = self.net.consensus.as_ref().unwrap(); - let want: HashSet<_> = self.state.cfg.validators.iter().cloned().collect(); + let want: HashSet<_> = self.genesis().validators.iter().cloned().collect(); consensus_state .inbound .subscribe() @@ -210,7 +217,7 @@ impl Instance { ctx: &ctx::Ctx, peer: &node::PublicKey, ) -> ctx::OrCanceled<()> { - let state = &self.state.gossip; + let state = &self.net.gossip; sync::wait_for(ctx, &mut state.inbound.subscribe(), |got| { !got.current().contains(peer) }) @@ -228,7 +235,7 @@ impl Instance { ctx: &ctx::Ctx, peer: &validator::PublicKey, ) -> ctx::OrCanceled<()> { - let state = self.state.consensus.as_ref().unwrap(); + let state = self.net.consensus.as_ref().unwrap(); sync::wait_for(ctx, &mut state.inbound.subscribe(), |got| { !got.current().contains(peer) }) @@ -253,8 +260,8 @@ pub async fn instant_network( let mut addrs = vec![]; let nodes: Vec<_> = nodes.collect(); for node in &nodes { - let key = node.consensus_config().key.public(); - let sub = &mut node.state.gossip.validator_addrs.subscribe(); + let key = node.cfg().validator_key.as_ref().unwrap().public(); + let sub = &mut node.net.gossip.validator_addrs.subscribe(); loop { if let Some(addr) = sync::changed(ctx, sub).await?.get(&key) { addrs.push(addr.clone()); @@ -264,10 +271,10 @@ pub async fn instant_network( } // Broadcast validator addrs. for node in &nodes { - node.state + node.net .gossip .validator_addrs - .update(&node.state.cfg.validators, &addrs) + .update(&node.genesis().validators, &addrs) .await .unwrap(); } diff --git a/node/actors/network/src/tests.rs b/node/actors/network/src/tests.rs index 75d10b1f..c1cde500 100644 --- a/node/actors/network/src/tests.rs +++ b/node/actors/network/src/tests.rs @@ -11,16 +11,16 @@ async fn test_metrics() { abort_on_panic(); let ctx = &mut ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let setup = validator::testonly::GenesisSetup::new(rng, 3); + let setup = validator::testonly::Setup::new(rng, 3); let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let (store, runner) = new_store(ctx, &setup.blocks[0]).await; + let (store, runner) = new_store(ctx, &setup.genesis).await; s.spawn_bg(runner.run(ctx)); let nodes: Vec<_> = cfgs .into_iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new(cfg, store.clone()); + let (node, runner) = testonly::Instance::new(ctx, cfg, store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) diff --git a/node/actors/network/src/watch.rs b/node/actors/network/src/watch.rs index 459d46d4..806d7678 100644 --- a/node/actors/network/src/watch.rs +++ b/node/actors/network/src/watch.rs @@ -3,12 +3,10 @@ use zksync_concurrency::sync; /// Wrapper of the tokio::sync::Watch. pub(crate) struct Watch { - /// Mutex-wrapped sender, so that it can be - /// accessed via immutable reference. + /// `sync::watch::Sender` contains synchronous mutex. + /// We wrap it into an async mutex to wait for it asynchronously. send: sync::Mutex>, - /// By keeping a copy of a receiver, sender - /// is never closed, which gets rid of send - /// errors. + /// Receiver outside of the mutex so that `subscribe()` can be nonblocking. recv: sync::watch::Receiver, } diff --git a/node/actors/sync_blocks/src/config.rs b/node/actors/sync_blocks/src/config.rs index d524cd72..a3c9c88a 100644 --- a/node/actors/sync_blocks/src/config.rs +++ b/node/actors/sync_blocks/src/config.rs @@ -1,16 +1,9 @@ //! Configuration for the `SyncBlocks` actor. - use zksync_concurrency::time; -use zksync_consensus_roles::validator::ValidatorSet; /// Configuration for the `SyncBlocks` actor. #[derive(Debug)] pub struct Config { - /// Set of validators authoring blocks. - pub(crate) validator_set: ValidatorSet, - /// Consensus threshold for blocks quorum certificates. - pub(crate) consensus_threshold: usize, - /// Maximum number of blocks to attempt to get concurrently from all peers in total. pub(crate) max_concurrent_blocks: usize, /// Maximum number of blocks to attempt to get concurrently from any single peer. @@ -20,26 +13,20 @@ pub struct Config { pub(crate) sleep_interval_for_get_block: time::Duration, } +impl Default for Config { + fn default() -> Self { + Self::new() + } +} + impl Config { /// Creates a new configuration with the provided mandatory params. - pub fn new(validator_set: ValidatorSet, consensus_threshold: usize) -> anyhow::Result { - anyhow::ensure!( - consensus_threshold > 0, - "`consensus_threshold` must be positive" - ); - anyhow::ensure!(validator_set.len() > 0, "`validator_set` must not be empty"); - anyhow::ensure!( - consensus_threshold <= validator_set.len(), - "`consensus_threshold` must not exceed length of `validator_set`" - ); - - Ok(Self { - validator_set, - consensus_threshold, + pub fn new() -> Self { + Self { max_concurrent_blocks: 20, max_concurrent_blocks_per_peer: 5, sleep_interval_for_get_block: time::Duration::seconds(10), - }) + } } /// Sets the maximum number of blocks to attempt to get concurrently. diff --git a/node/actors/sync_blocks/src/lib.rs b/node/actors/sync_blocks/src/lib.rs index 977ad696..e46e340a 100644 --- a/node/actors/sync_blocks/src/lib.rs +++ b/node/actors/sync_blocks/src/lib.rs @@ -28,7 +28,7 @@ impl Config { ) -> anyhow::Result<()> { let peer_states = PeerStates::new(self, storage.clone(), pipe.send); let result: ctx::Result<()> = scope::run!(ctx, |ctx, s| async { - s.spawn_bg(async { Ok(peer_states.run_block_fetcher(ctx).await?) }); + s.spawn_bg(async { peer_states.run_block_fetcher(ctx).await }); loop { match pipe.recv.recv(ctx).await? { InputMessage::Network(SyncBlocksRequest::UpdatePeerSyncState { diff --git a/node/actors/sync_blocks/src/peers/mod.rs b/node/actors/sync_blocks/src/peers/mod.rs index ae4e3f1c..f57ea8bd 100644 --- a/node/actors/sync_blocks/src/peers/mod.rs +++ b/node/actors/sync_blocks/src/peers/mod.rs @@ -1,5 +1,4 @@ //! Peer states tracked by the `SyncBlocks` actor. - use self::events::PeerStateEvent; use crate::{io, Config}; use anyhow::Context as _; @@ -13,11 +12,10 @@ use zksync_concurrency::{ }; use zksync_consensus_network::io::SyncBlocksInputMessage; use zksync_consensus_roles::{ - node, + node, validator, validator::{BlockNumber, FinalBlock}, }; use zksync_consensus_storage::{BlockStore, BlockStoreState}; -use zksync_consensus_utils::no_copy::NoCopy; mod events; #[cfg(test)] @@ -42,6 +40,10 @@ pub(crate) struct PeerStates { } impl PeerStates { + fn genesis(&self) -> &validator::Genesis { + self.storage.genesis() + } + /// Creates a new instance together with a handle. pub(crate) fn new( config: Config, @@ -68,37 +70,34 @@ impl PeerStates { state: BlockStoreState, ) -> anyhow::Result<()> { use std::collections::hash_map::Entry; - - let last = state.last.header().number; - anyhow::ensure!(state.first.header().number <= state.last.header().number); - state - .last - .verify(&self.config.validator_set, self.config.consensus_threshold) - .context("state.last.verify()")?; + let Some(last) = &state.last else { + return Ok(()); + }; + last.verify(self.genesis()).context("state.last.verify()")?; let mut peers = self.peers.lock().unwrap(); match peers.entry(peer.clone()) { - Entry::Occupied(mut e) => e.get_mut().state = state, + Entry::Occupied(mut e) => e.get_mut().state = state.clone(), Entry::Vacant(e) => { let permits = self.config.max_concurrent_blocks_per_peer; e.insert(PeerState { - state, + state: state.clone(), get_block_semaphore: Arc::new(sync::Semaphore::new(permits)), }); } } self.highest_peer_block .send_if_modified(|highest_peer_block| { - if *highest_peer_block >= last { + if *highest_peer_block >= last.header().number { return false; } - *highest_peer_block = last; + *highest_peer_block = last.header().number; true }); Ok(()) } /// Task fetching blocks from peers which are not present in storage. - pub(crate) async fn run_block_fetcher(&self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { + pub(crate) async fn run_block_fetcher(&self, ctx: &ctx::Ctx) -> ctx::Result<()> { let sem = sync::Semaphore::new(self.config.max_concurrent_blocks); scope::run!(ctx, |ctx, s| async { let mut next = self.storage.subscribe().borrow().next(); @@ -109,11 +108,11 @@ impl PeerStates { }) .await?; let permit = sync::acquire(ctx, &sem).await?; - let block_number = NoCopy::from(next); + let block_number = ctx::NoCopy(next); next = next.next(); s.spawn(async { let _permit = permit; - self.fetch_block(ctx, block_number.into_inner()).await + self.fetch_block(ctx, block_number.into()).await }); } }) @@ -122,17 +121,19 @@ impl PeerStates { /// Fetches the block from peers and puts it to storage. /// Early exits if the block appeared in storage from other source. - async fn fetch_block(&self, ctx: &ctx::Ctx, block_number: BlockNumber) -> ctx::OrCanceled<()> { + async fn fetch_block(&self, ctx: &ctx::Ctx, block_number: BlockNumber) -> ctx::Result<()> { let _ = scope::run!(ctx, |ctx, s| async { s.spawn_bg(async { let block = self.fetch_block_from_peers(ctx, block_number).await?; self.storage.queue_block(ctx, block).await }); // Cancel fetching as soon as block is queued for storage. - self.storage.wait_until_queued(ctx, block_number).await + self.storage.wait_until_queued(ctx, block_number).await?; + Ok(()) }) .await; - self.storage.wait_until_persisted(ctx, block_number).await + self.storage.wait_until_persisted(ctx, block_number).await?; + Ok(()) } /// Fetches the block from peers. @@ -200,9 +201,7 @@ impl PeerStates { ) .into()); } - block - .validate(&self.config.validator_set, self.config.consensus_threshold) - .context("block.validate()")?; + block.verify(self.genesis()).context("block.validate()")?; Ok(block) } diff --git a/node/actors/sync_blocks/src/peers/tests/basics.rs b/node/actors/sync_blocks/src/peers/tests/basics.rs index 4ccda284..18468c7d 100644 --- a/node/actors/sync_blocks/src/peers/tests/basics.rs +++ b/node/actors/sync_blocks/src/peers/tests/basics.rs @@ -3,8 +3,9 @@ use super::*; use crate::{ io, - tests::{send_block, sync_state}, + tests::{make_response, sync_state}, }; +use rand::seq::SliceRandom as _; #[derive(Debug)] struct UpdatingPeerStateWithSingleBlock; @@ -26,7 +27,7 @@ impl Test for UpdatingPeerStateWithSingleBlock { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, 1)) + .update(&peer_key, sync_state(&setup, setup.blocks.first())) .unwrap(); // Check that the actor has sent a `get_block` request to the peer @@ -37,16 +38,18 @@ impl Test for UpdatingPeerStateWithSingleBlock { response, }) = message; assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(1)); + assert_eq!(number, setup.blocks[0].number()); // Emulate the peer sending a correct response. - send_block(&setup, BlockNumber(1), response); + response.send(make_response(setup.blocks.first())).unwrap(); let peer_event = events_receiver.recv(ctx).await?; - assert_matches!(peer_event, PeerStateEvent::GotBlock(BlockNumber(1))); + assert_matches!(peer_event, PeerStateEvent::GotBlock(n) if n == setup.blocks[0].number()); // Check that the block has been saved locally. - storage.wait_until_persisted(ctx, BlockNumber(1)).await?; + storage + .wait_until_persisted(ctx, setup.blocks[0].number()) + .await?; Ok(()) } } @@ -75,7 +78,7 @@ impl Test for CancelingBlockRetrieval { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, 1)) + .update(&peer_key, sync_state(&setup, setup.blocks.first())) .unwrap(); // Check that the actor has sent a `get_block` request to the peer @@ -83,7 +86,7 @@ impl Test for CancelingBlockRetrieval { message_receiver.recv(ctx).await?; // Emulate receiving block using external means. - storage.queue_block(ctx, setup.blocks[1].clone()).await?; + storage.queue_block(ctx, setup.blocks[0].clone()).await?; // Retrieval of the block must be canceled. response.closed().await; @@ -113,12 +116,12 @@ impl Test for FilteringBlockRetrieval { } = handles; // Emulate receiving block using external means. - storage.queue_block(ctx, setup.blocks[1].clone()).await?; + storage.queue_block(ctx, setup.blocks[0].clone()).await?; let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, 2)) + .update(&peer_key, sync_state(&setup, setup.blocks.get(1))) .unwrap(); // Check that the actor has sent `get_block` request to the peer, but only for block #2. @@ -127,8 +130,7 @@ impl Test for FilteringBlockRetrieval { recipient, number, .. }) = message; assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(2)); - + assert_eq!(number, setup.blocks[1].number()); assert!(message_receiver.try_recv().is_none()); Ok(()) } @@ -150,10 +152,12 @@ impl UpdatingPeerStateWithMultipleBlocks { impl Test for UpdatingPeerStateWithMultipleBlocks { const BLOCK_COUNT: usize = 10; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.max_concurrent_blocks_per_peer = Self::MAX_CONCURRENT_BLOCKS; // ^ We want to test rate limiting for peers config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -169,11 +173,11 @@ impl Test for UpdatingPeerStateWithMultipleBlocks { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, Self::BLOCK_COUNT - 1).clone()) + .update(&peer_key, sync_state(&setup, setup.blocks.last()).clone()) .unwrap(); - let mut requested_blocks = HashMap::with_capacity(Self::MAX_CONCURRENT_BLOCKS); - for _ in 1..Self::BLOCK_COUNT { + let mut requested_blocks = HashMap::new(); + for _ in setup.blocks.iter() { let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { recipient, number, @@ -191,7 +195,7 @@ impl Test for UpdatingPeerStateWithMultipleBlocks { // Answer a random request. let number = *requested_blocks.keys().choose(rng).unwrap(); let response = requested_blocks.remove(&number).unwrap(); - send_block(&setup, number, response); + response.send(make_response(setup.block(number))).unwrap(); let peer_event = events_receiver.recv(ctx).await?; assert_matches!(peer_event, PeerStateEvent::GotBlock(got) if got == number); @@ -201,7 +205,7 @@ impl Test for UpdatingPeerStateWithMultipleBlocks { // Answer all remaining requests. for (number, response) in requested_blocks { - send_block(&setup, number, response); + response.send(make_response(setup.block(number))).unwrap(); let peer_event = events_receiver.recv(ctx).await?; assert_matches!(peer_event, PeerStateEvent::GotBlock(got) if got == number); } @@ -226,8 +230,10 @@ struct DisconnectingPeer; impl Test for DisconnectingPeer { const BLOCK_COUNT: usize = 5; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -243,7 +249,7 @@ impl Test for DisconnectingPeer { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, 1)) + .update(&peer_key, sync_state(&setup, setup.blocks.first())) .unwrap(); // Drop the response sender emulating peer disconnect. @@ -255,7 +261,7 @@ impl Test for DisconnectingPeer { .. }) = &msg; assert_eq!(recipient, &peer_key); - assert_eq!(number, &BlockNumber(1)); + assert_eq!(number, &setup.blocks[0].number()); } drop(msg); @@ -273,7 +279,7 @@ impl Test for DisconnectingPeer { // Re-connect the peer with an updated state. peer_states - .update(&peer_key, sync_state(&setup, 2)) + .update(&peer_key, sync_state(&setup, setup.blocks.get(1))) .unwrap(); // Ensure that blocks are re-requested. clock.advance(BLOCK_SLEEP_INTERVAL); @@ -287,18 +293,20 @@ impl Test for DisconnectingPeer { response, }) = message; assert_eq!(recipient, peer_key); - assert!(responses.insert(number.0, response).is_none()); + assert!(responses.insert(number, response).is_none()); } - assert!(responses.contains_key(&1)); - assert!(responses.contains_key(&2)); + assert!(responses.contains_key(&setup.blocks[0].number())); + assert!(responses.contains_key(&setup.blocks[1].number())); // Send one of the responses and drop the other request. - let response = responses.remove(&2).unwrap(); - send_block(&setup, BlockNumber(2), response); + let response = responses.remove(&setup.blocks[1].number()).unwrap(); + response.send(make_response(setup.blocks.get(1))).unwrap(); - wait_for_event(ctx, &mut events_receiver, |ev| { - matches!(ev, PeerStateEvent::GotBlock(BlockNumber(2))) - }) + wait_for_event( + ctx, + &mut events_receiver, + |ev| matches!(ev, PeerStateEvent::GotBlock(n) if n==setup.blocks[1].number()), + ) .await?; drop(responses); wait_for_event( @@ -314,7 +322,7 @@ impl Test for DisconnectingPeer { // Re-connect the peer with the same state. peer_states - .update(&peer_key, sync_state(&setup, 2)) + .update(&peer_key, sync_state(&setup, setup.blocks.get(1))) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); @@ -325,17 +333,17 @@ impl Test for DisconnectingPeer { response, }) = message; assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(1)); - send_block(&setup, number, response); + assert_eq!(number, setup.blocks[0].number()); + response.send(make_response(setup.blocks.first())).unwrap(); let peer_event = events_receiver.recv(ctx).await?; - assert_matches!(peer_event, PeerStateEvent::GotBlock(BlockNumber(1))); + assert_matches!(peer_event, PeerStateEvent::GotBlock(n) if n==setup.blocks[0].number()); // Check that no new requests are sent (all blocks are downloaded). clock.advance(BLOCK_SLEEP_INTERVAL); assert_matches!(message_receiver.try_recv(), None); - storage.wait_until_persisted(ctx, BlockNumber(2)).await?; + storage.wait_until_persisted(ctx, BlockNumber(1)).await?; Ok(()) } } @@ -347,18 +355,14 @@ async fn disconnecting_peer() { #[derive(Debug)] struct DownloadingBlocksInGaps { - local_block_numbers: Vec, + local_blocks: Vec, increase_peer_block_number_during_test: bool, } impl DownloadingBlocksInGaps { - fn new(local_block_numbers: &[usize]) -> Self { + fn new(local_blocks: &[usize]) -> Self { Self { - local_block_numbers: local_block_numbers - .iter() - .copied() - .inspect(|&number| assert!(number > 0 && number < Self::BLOCK_COUNT)) - .collect(), + local_blocks: local_blocks.to_vec(), increase_peer_block_number_during_test: false, } } @@ -368,10 +372,12 @@ impl DownloadingBlocksInGaps { impl Test for DownloadingBlocksInGaps { const BLOCK_COUNT: usize = 10; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.max_concurrent_blocks = 1; // ^ Forces the node to download blocks in a deterministic order config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -385,30 +391,31 @@ impl Test for DownloadingBlocksInGaps { } = handles; scope::run!(ctx, |ctx, s| async { - for &block_number in &self.local_block_numbers { - s.spawn(storage.queue_block(ctx, setup.blocks[block_number].clone())); + for n in &self.local_blocks { + s.spawn(storage.queue_block(ctx, setup.blocks[*n].clone())); } let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); - let mut last_peer_block_number = if self.increase_peer_block_number_during_test { - rng.gen_range(1..Self::BLOCK_COUNT) + let mut last_peer_block = if self.increase_peer_block_number_during_test { + setup.blocks.choose(rng) } else { - Self::BLOCK_COUNT - 1 + setup.blocks.last() }; peer_states - .update(&peer_key, sync_state(&setup, last_peer_block_number)) + .update(&peer_key, sync_state(&setup, last_peer_block)) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); - let expected_block_numbers = - (1..Self::BLOCK_COUNT).filter(|number| !self.local_block_numbers.contains(number)); - // Check that all missing blocks are requested. - for expected_number in expected_block_numbers { - if expected_number > last_peer_block_number { - last_peer_block_number = rng.gen_range(expected_number..Self::BLOCK_COUNT); + for n in 0..setup.blocks.len() { + if self.local_blocks.contains(&n) { + continue; + } + let n = setup.blocks[n].number(); + if n > last_peer_block.unwrap().number() { + last_peer_block = setup.blocks.iter().filter(|b| b.number() >= n).choose(rng); peer_states - .update(&peer_key, sync_state(&setup, last_peer_block_number)) + .update(&peer_key, sync_state(&setup, last_peer_block)) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); } @@ -420,8 +427,8 @@ impl Test for DownloadingBlocksInGaps { }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, peer_key); - assert!(number.0 <= last_peer_block_number as u64); - send_block(&setup, number, response); + assert!(number <= last_peer_block.unwrap().number()); + response.send(make_response(setup.block(number))).unwrap(); storage.wait_until_persisted(ctx, number).await?; clock.advance(BLOCK_SLEEP_INTERVAL); } @@ -437,10 +444,10 @@ const LOCAL_BLOCK_NUMBERS: [&[usize]; 3] = [&[1, 9], &[3, 5, 6, 8], &[4]]; #[test_casing(6, Product((LOCAL_BLOCK_NUMBERS, [false, true])))] #[tokio::test] async fn downloading_blocks_in_gaps( - local_block_numbers: &[usize], + local_blocks: &[usize], increase_peer_block_number_during_test: bool, ) { - let mut test = DownloadingBlocksInGaps::new(local_block_numbers); + let mut test = DownloadingBlocksInGaps::new(local_blocks); test.increase_peer_block_number_during_test = increase_peer_block_number_during_test; test_peer_states(test).await; } @@ -452,8 +459,10 @@ struct LimitingGetBlockConcurrency; impl Test for LimitingGetBlockConcurrency { const BLOCK_COUNT: usize = 5; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.max_concurrent_blocks = 3; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -467,12 +476,12 @@ impl Test for LimitingGetBlockConcurrency { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, Self::BLOCK_COUNT - 1)) + .update(&peer_key, sync_state(&setup, setup.blocks.last())) .unwrap(); // The actor should request 3 new blocks it's now aware of from the only peer it's currently // aware of. Note that blocks may be queried in any order. - let mut message_responses = HashMap::with_capacity(3); + let mut message_responses = HashMap::new(); for _ in 0..3 { let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { recipient, @@ -480,26 +489,28 @@ impl Test for LimitingGetBlockConcurrency { response, }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, peer_key); - assert!(message_responses.insert(number.0, response).is_none()); + assert!(message_responses.insert(number, response).is_none()); } assert_matches!(message_receiver.try_recv(), None); assert_eq!( message_responses.keys().copied().collect::>(), - HashSet::from([1, 2, 3]) + setup.blocks[0..3].iter().map(|b| b.number()).collect(), ); tracing::info!("blocks requrested"); // Send a correct response. - let response = message_responses.remove(&1).unwrap(); - send_block(&setup, BlockNumber(1), response); - storage.wait_until_persisted(ctx, BlockNumber(1)).await?; + let response = message_responses.remove(&setup.blocks[0].number()).unwrap(); + response.send(make_response(setup.blocks.first())).unwrap(); + storage + .wait_until_persisted(ctx, setup.blocks[0].number()) + .await?; // The actor should now request another block. let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { recipient, number, .. }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(4)); + assert_eq!(number, setup.blocks[3].number()); Ok(()) } diff --git a/node/actors/sync_blocks/src/peers/tests/fakes.rs b/node/actors/sync_blocks/src/peers/tests/fakes.rs index bc8e5da1..95c6ddd8 100644 --- a/node/actors/sync_blocks/src/peers/tests/fakes.rs +++ b/node/actors/sync_blocks/src/peers/tests/fakes.rs @@ -2,32 +2,29 @@ use super::*; use crate::tests::sync_state; -use zksync_consensus_roles::{validator, validator::testonly::GenesisSetup}; +use zksync_consensus_roles::{validator, validator::testonly::Setup}; use zksync_consensus_storage::testonly::new_store; #[tokio::test] async fn processing_invalid_sync_states() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut setup = GenesisSetup::empty(rng, 4); + let mut setup = Setup::new(rng, 4); setup.push_blocks(rng, 3); - let (storage, _runner) = new_store(ctx, &setup.blocks[0]).await; + let (storage, _runner) = new_store(ctx, &setup.genesis).await; let (message_sender, _) = channel::unbounded(); - let peer_states = PeerStates::new(test_config(&setup), storage, message_sender); - + let peer_states = PeerStates::new(Config::new(), storage, message_sender); let peer = &rng.gen::().public(); - let mut invalid_sync_state = sync_state(&setup, 1); - invalid_sync_state.first = setup.blocks[2].justification.clone(); - assert!(peer_states.update(peer, invalid_sync_state).is_err()); - let mut invalid_sync_state = sync_state(&setup, 1); - invalid_sync_state.last.message.proposal.number = BlockNumber(5); + let mut invalid_block = setup.blocks[1].clone(); + invalid_block.justification.message.proposal.number = rng.gen(); + let invalid_sync_state = sync_state(&setup, Some(&invalid_block)); assert!(peer_states.update(peer, invalid_sync_state).is_err()); - let mut other_network = GenesisSetup::empty(rng, 4); + let mut other_network = Setup::new(rng, 4); other_network.push_blocks(rng, 2); - let invalid_sync_state = sync_state(&other_network, 1); + let invalid_sync_state = sync_state(&other_network, other_network.blocks.get(1)); assert!(peer_states.update(peer, invalid_sync_state).is_err()); } @@ -49,8 +46,9 @@ impl Test for PeerWithFakeSyncState { let rng = &mut ctx.rng(); let peer_key = rng.gen::().public(); - let mut fake_sync_state = sync_state(&setup, 1); - fake_sync_state.last.message.proposal.number = BlockNumber(42); + let mut invalid_block = setup.blocks[1].clone(); + invalid_block.justification.message.proposal.number = rng.gen(); + let fake_sync_state = sync_state(&setup, Some(&invalid_block)); assert!(peer_states.update(&peer_key, fake_sync_state).is_err()); clock.advance(BLOCK_SLEEP_INTERVAL); @@ -71,8 +69,10 @@ struct PeerWithFakeBlock; impl Test for PeerWithFakeBlock { const BLOCK_COUNT: usize = 10; - fn tweak_config(&self, cfg: &mut Config) { + fn config(&self) -> Config { + let mut cfg = Config::new(); cfg.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; + cfg } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -89,23 +89,23 @@ impl Test for PeerWithFakeBlock { for fake_block in [ // other block than requested - setup.blocks[0].clone(), + setup.blocks[1].clone(), // block with wrong validator set { - let mut setup = GenesisSetup::empty(rng, 4); - setup.push_blocks(rng, 2); - setup.blocks[1].clone() + let mut s = Setup::new(rng, 4); + s.push_blocks(rng, 1); + s.blocks[0].clone() }, // block with mismatching payload, { - let mut block = setup.blocks[1].clone(); + let mut block = setup.blocks[0].clone(); block.payload = validator::Payload(b"invalid".to_vec()); block }, ] { - let peer_key = rng.gen::().public(); + let key = rng.gen::().public(); peer_states - .update(&peer_key, sync_state(&setup, 1)) + .update(&key, sync_state(&setup, setup.blocks.first())) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); @@ -114,16 +114,16 @@ impl Test for PeerWithFakeBlock { number, response, }) = message_receiver.recv(ctx).await?; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(1)); + assert_eq!(recipient, key); + assert_eq!(number, setup.blocks[0].number()); response.send(Ok(fake_block)).unwrap(); wait_for_event(ctx, &mut events_receiver, |ev| { matches!(ev, PeerStateEvent::RpcFailed { - block_number: BlockNumber(1), - peer_key: key, - } if key == peer_key + block_number, + peer_key, + } if peer_key == key && block_number == number ) }) .await?; diff --git a/node/actors/sync_blocks/src/peers/tests/mod.rs b/node/actors/sync_blocks/src/peers/tests/mod.rs index 6349c408..d6c9d66d 100644 --- a/node/actors/sync_blocks/src/peers/tests/mod.rs +++ b/node/actors/sync_blocks/src/peers/tests/mod.rs @@ -1,5 +1,4 @@ use super::*; -use crate::tests::test_config; use assert_matches::assert_matches; use async_trait::async_trait; use rand::{seq::IteratorRandom, Rng}; @@ -16,7 +15,6 @@ use zksync_consensus_storage::testonly::new_store; mod basics; mod fakes; mod multiple_peers; -mod snapshots; const TEST_TIMEOUT: time::Duration = time::Duration::seconds(5); const BLOCK_SLEEP_INTERVAL: time::Duration = time::Duration::milliseconds(5); @@ -33,7 +31,7 @@ async fn wait_for_event( #[derive(Debug)] struct TestHandles { clock: ctx::ManualClock, - setup: validator::testonly::GenesisSetup, + setup: validator::testonly::Setup, peer_states: Arc, storage: Arc, message_receiver: channel::UnboundedReceiver, @@ -43,17 +41,18 @@ struct TestHandles { #[async_trait] trait Test: fmt::Debug + Send + Sync { const BLOCK_COUNT: usize; + // TODO: move this to genesis const GENESIS_BLOCK_NUMBER: usize = 0; - fn tweak_config(&self, _config: &mut Config) { - // Does nothing by default + fn config(&self) -> Config { + Config::new() } async fn initialize_storage( &self, _ctx: &ctx::Ctx, _storage: &BlockStore, - _setup: &validator::testonly::GenesisSetup, + _setup: &validator::testonly::Setup, ) { // Does nothing by default } @@ -69,16 +68,14 @@ async fn test_peer_states(test: T) { let clock = ctx::ManualClock::new(); let ctx = &ctx::test_root(&clock); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::GenesisSetup::new(rng, 4); + let mut setup = validator::testonly::Setup::new(rng, 4); setup.push_blocks(rng, T::BLOCK_COUNT); - let (store, store_run) = new_store(ctx, &setup.blocks[T::GENESIS_BLOCK_NUMBER]).await; + let (store, store_run) = new_store(ctx, &setup.genesis).await; test.initialize_storage(ctx, store.as_ref(), &setup).await; let (message_sender, message_receiver) = channel::unbounded(); let (events_sender, events_receiver) = channel::unbounded(); - let mut config = test_config(&setup); - test.tweak_config(&mut config); - let mut peer_states = PeerStates::new(config, store.clone(), message_sender); + let mut peer_states = PeerStates::new(test.config(), store.clone(), message_sender); peer_states.events_sender = Some(events_sender); let peer_states = Arc::new(peer_states); let test_handles = TestHandles { diff --git a/node/actors/sync_blocks/src/peers/tests/multiple_peers.rs b/node/actors/sync_blocks/src/peers/tests/multiple_peers.rs index dcc21ea5..c6305c36 100644 --- a/node/actors/sync_blocks/src/peers/tests/multiple_peers.rs +++ b/node/actors/sync_blocks/src/peers/tests/multiple_peers.rs @@ -1,7 +1,5 @@ -//! Tests focused on interaction with multiple peers. - use super::*; -use crate::tests::{send_block, sync_state}; +use crate::tests::{make_response, sync_state}; #[derive(Debug)] struct RequestingBlocksFromTwoPeers; @@ -10,10 +8,12 @@ struct RequestingBlocksFromTwoPeers; impl Test for RequestingBlocksFromTwoPeers { const BLOCK_COUNT: usize = 5; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; config.max_concurrent_blocks = 5; config.max_concurrent_blocks_per_peer = 1; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -29,7 +29,7 @@ impl Test for RequestingBlocksFromTwoPeers { let rng = &mut ctx.rng(); let first_peer = rng.gen::().public(); peer_states - .update(&first_peer, sync_state(&setup, 2)) + .update(&first_peer, sync_state(&setup, setup.blocks.get(1))) .unwrap(); let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { @@ -38,14 +38,14 @@ impl Test for RequestingBlocksFromTwoPeers { response: first_peer_response, }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, first_peer); - assert!( - first_peer_block_number == BlockNumber(1) || first_peer_block_number == BlockNumber(2) - ); + assert!(setup.blocks[0..=1] + .iter() + .any(|b| b.number() == first_peer_block_number)); tracing::info!(%first_peer_block_number, "received request"); let second_peer = rng.gen::().public(); peer_states - .update(&second_peer, sync_state(&setup, 4)) + .update(&second_peer, sync_state(&setup, setup.blocks.get(3))) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); @@ -55,13 +55,14 @@ impl Test for RequestingBlocksFromTwoPeers { response: second_peer_response, }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, second_peer); - assert!( - second_peer_block_number == BlockNumber(1) - || second_peer_block_number == BlockNumber(2) - ); + assert!(setup.blocks[0..=1] + .iter() + .any(|b| b.number() == second_peer_block_number)); tracing::info!(%second_peer_block_number, "received requrest"); - send_block(&setup, first_peer_block_number, first_peer_response); + first_peer_response + .send(make_response(setup.block(first_peer_block_number))) + .unwrap(); wait_for_event( ctx, &mut events_receiver, @@ -75,7 +76,7 @@ impl Test for RequestingBlocksFromTwoPeers { assert_matches!(message_receiver.try_recv(), None); peer_states - .update(&first_peer, sync_state(&setup, 4)) + .update(&first_peer, sync_state(&setup, setup.blocks.get(3))) .unwrap(); clock.advance(BLOCK_SLEEP_INTERVAL); // Now the actor can get block #3 from the peer. @@ -86,12 +87,14 @@ impl Test for RequestingBlocksFromTwoPeers { response: first_peer_response, }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, first_peer); - assert!( - first_peer_block_number == BlockNumber(3) || first_peer_block_number == BlockNumber(4) - ); + assert!(setup.blocks[2..=3] + .iter() + .any(|b| b.number() == first_peer_block_number)); tracing::info!(%first_peer_block_number, "received requrest"); - send_block(&setup, first_peer_block_number, first_peer_response); + first_peer_response + .send(make_response(setup.block(first_peer_block_number))) + .unwrap(); wait_for_event( ctx, &mut events_receiver, @@ -107,12 +110,14 @@ impl Test for RequestingBlocksFromTwoPeers { response: first_peer_response, }) = message_receiver.recv(ctx).await?; assert_eq!(recipient, first_peer); - assert!( - first_peer_block_number == BlockNumber(3) || first_peer_block_number == BlockNumber(4) - ); + assert!(setup.blocks[2..=3] + .iter() + .any(|b| b.number() == first_peer_block_number)); tracing::info!(%first_peer_block_number, "received requrest"); - send_block(&setup, second_peer_block_number, second_peer_response); + second_peer_response + .send(make_response(setup.block(second_peer_block_number))) + .unwrap(); wait_for_event( ctx, &mut events_receiver, @@ -120,7 +125,9 @@ impl Test for RequestingBlocksFromTwoPeers { ) .await .unwrap(); - send_block(&setup, first_peer_block_number, first_peer_response); + first_peer_response + .send(make_response(setup.block(first_peer_block_number))) + .unwrap(); wait_for_event( ctx, &mut events_receiver, @@ -132,7 +139,9 @@ impl Test for RequestingBlocksFromTwoPeers { clock.advance(BLOCK_SLEEP_INTERVAL); assert_matches!(message_receiver.try_recv(), None); - storage.wait_until_persisted(ctx, BlockNumber(4)).await?; + storage + .wait_until_persisted(ctx, setup.blocks[3].number()) + .await?; Ok(()) } } @@ -145,17 +154,17 @@ async fn requesting_blocks_from_two_peers() { #[derive(Debug, Clone, Copy)] struct PeerBehavior { /// The peer will go offline after this block. - last_block: BlockNumber, + last_block: usize, /// The peer will stop responding after this block, but will still announce `SyncState` updates. /// Logically, should be `<= last_block`. - last_block_to_return: BlockNumber, + last_block_to_return: usize, } impl Default for PeerBehavior { fn default() -> Self { Self { - last_block: BlockNumber(u64::MAX), - last_block_to_return: BlockNumber(u64::MAX), + last_block: usize::MAX, + last_block_to_return: usize::MAX, } } } @@ -177,7 +186,7 @@ impl RequestingBlocksFromMultiplePeers { } fn create_peers(&self, rng: &mut impl Rng) -> HashMap { - let last_block_number = BlockNumber(Self::BLOCK_COUNT as u64 - 1); + let last_block_number = Self::BLOCK_COUNT - 1; let peers = self.peer_behavior.iter().copied().map(|behavior| { let behavior = PeerBehavior { last_block: behavior.last_block.min(last_block_number), @@ -194,9 +203,11 @@ impl RequestingBlocksFromMultiplePeers { impl Test for RequestingBlocksFromMultiplePeers { const BLOCK_COUNT: usize = 20; - fn tweak_config(&self, config: &mut Config) { + fn config(&self) -> Config { + let mut config = Config::new(); config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; config.max_concurrent_blocks_per_peer = self.max_concurrent_blocks_per_peer; + config } async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { @@ -215,14 +226,13 @@ impl Test for RequestingBlocksFromMultiplePeers { scope::run!(ctx, |ctx, s| async { // Announce peer states. for (peer_key, peer) in peers { - let last_block = peer.last_block.0 as usize; - peer_states.update(peer_key, sync_state(&setup, last_block)).unwrap(); + peer_states.update(peer_key, sync_state(&setup, setup.blocks.get(peer.last_block))).unwrap(); } s.spawn_bg(async { let mut responses_by_peer: HashMap<_, Vec<_>> = HashMap::new(); let mut requested_blocks = HashSet::new(); - while requested_blocks.len() < Self::BLOCK_COUNT - 1 { + while requested_blocks.len() < Self::BLOCK_COUNT { let Ok(message) = message_receiver.recv(ctx).await else { return Ok(()); // Test is finished }; @@ -233,9 +243,9 @@ impl Test for RequestingBlocksFromMultiplePeers { }) = message; tracing::trace!("Block #{number} requested from {recipient:?}"); - assert!(number <= peers[&recipient].last_block); + assert!(number <= setup.blocks[peers[&recipient].last_block].number()); - if peers[&recipient].last_block_to_return < number { + if setup.blocks[peers[&recipient].last_block_to_return].number() < number { tracing::trace!("Dropping request for block #{number} to {recipient:?}"); continue; } @@ -251,7 +261,7 @@ impl Test for RequestingBlocksFromMultiplePeers { // Peer is at capacity, respond to a random request in order to progress let idx = rng.gen_range(0..peer_responses.len()); let (number, response) = peer_responses.remove(idx); - send_block(&setup, number, response); + response.send(make_response(setup.block(number))).unwrap(); } // Respond to some other random requests. @@ -262,22 +272,22 @@ impl Test for RequestingBlocksFromMultiplePeers { continue; } let (number, response) = peer_responses.remove(idx); - send_block(&setup, number, response); + response.send(make_response(setup.block(number))).unwrap(); } } } // Answer to all remaining responses for (number, response) in responses_by_peer.into_values().flatten() { - send_block(&setup, number, response); + response.send(make_response(setup.block(number))).unwrap(); } Ok(()) }); // We advance the clock when a node receives a new block or updates a peer state, // since in both cases some new blocks may become available for download. - let mut block_numbers = HashSet::with_capacity(Self::BLOCK_COUNT - 1); - while block_numbers.len() < Self::BLOCK_COUNT - 1 { + let mut block_numbers = HashSet::with_capacity(Self::BLOCK_COUNT); + while block_numbers.len() < Self::BLOCK_COUNT { let peer_event = events_receiver.recv(ctx).await?; match peer_event { PeerStateEvent::GotBlock(number) => { @@ -291,7 +301,7 @@ impl Test for RequestingBlocksFromMultiplePeers { } } - storage.wait_until_persisted(ctx,BlockNumber(19)).await?; + storage.wait_until_persisted(ctx,setup.blocks.last().unwrap().header().number).await?; Ok(()) }) .await @@ -316,8 +326,8 @@ async fn requesting_blocks_with_failures( ) { let mut test = RequestingBlocksFromMultiplePeers::new(3, max_concurrent_blocks_per_peer); test.respond_probability = respond_probability; - test.peer_behavior[0].last_block = BlockNumber(5); - test.peer_behavior[1].last_block = BlockNumber(15); + test.peer_behavior[0].last_block = 5; + test.peer_behavior[1].last_block = 15; test_peer_states(test).await; } @@ -329,7 +339,7 @@ async fn requesting_blocks_with_unreliable_peers( ) { let mut test = RequestingBlocksFromMultiplePeers::new(3, max_concurrent_blocks_per_peer); test.respond_probability = respond_probability; - test.peer_behavior[0].last_block_to_return = BlockNumber(5); - test.peer_behavior[1].last_block_to_return = BlockNumber(15); + test.peer_behavior[0].last_block_to_return = 5; + test.peer_behavior[1].last_block_to_return = 15; test_peer_states(test).await; } diff --git a/node/actors/sync_blocks/src/peers/tests/snapshots.rs b/node/actors/sync_blocks/src/peers/tests/snapshots.rs index 740e6412..bdb7116d 100644 --- a/node/actors/sync_blocks/src/peers/tests/snapshots.rs +++ b/node/actors/sync_blocks/src/peers/tests/snapshots.rs @@ -1,329 +1,9 @@ //! Tests related to snapshot storage. use super::*; -use crate::tests::{send_block, snapshot_sync_state, sync_state}; +use crate::tests::{send_block, sync_state}; use zksync_consensus_network::io::GetBlockError; -#[derive(Debug)] -struct UpdatingPeerStateWithStorageSnapshot; - -#[async_trait] -impl Test for UpdatingPeerStateWithStorageSnapshot { - const BLOCK_COUNT: usize = 5; - const GENESIS_BLOCK_NUMBER: usize = 2; - - fn tweak_config(&self, config: &mut Config) { - config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; - } - - async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { - let TestHandles { - setup, - peer_states, - storage, - mut message_receiver, - mut events_receiver, - clock, - } = handles; - let rng = &mut ctx.rng(); - let peer_key = rng.gen::().public(); - for stale_block_number in [1, 2] { - peer_states - .update(&peer_key, sync_state(&setup, stale_block_number)) - .unwrap(); - - // No new block requests should be issued. - clock.advance(BLOCK_SLEEP_INTERVAL); - sync::yield_now().await; - assert!(message_receiver.try_recv().is_none()); - } - - peer_states - .update(&peer_key, sync_state(&setup, 3)) - .unwrap(); - - // Check that the actor has sent a `get_block` request to the peer - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response, - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(3)); - - // Emulate the peer sending a correct response. - send_block(&setup, BlockNumber(3), response); - - wait_for_event(ctx, &mut events_receiver, |ev| { - matches!(ev, PeerStateEvent::GotBlock(BlockNumber(3))) - }) - .await - .unwrap(); - - // Check that the block has been saved locally. - storage.wait_until_queued(ctx, BlockNumber(3)).await?; - Ok(()) - } -} - -#[tokio::test] -async fn updating_peer_state_with_storage_snapshot() { - test_peer_states(UpdatingPeerStateWithStorageSnapshot).await; -} - -#[derive(Debug)] -struct FilteringRequestsForSnapshotPeer; - -#[async_trait] -impl Test for FilteringRequestsForSnapshotPeer { - const BLOCK_COUNT: usize = 5; - - fn tweak_config(&self, config: &mut Config) { - config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; - } - - async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { - let TestHandles { - setup, - peer_states, - mut message_receiver, - mut events_receiver, - clock, - .. - } = handles; - - let rng = &mut ctx.rng(); - let peer_key = rng.gen::().public(); - peer_states - .update(&peer_key, snapshot_sync_state(&setup, 2..=2)) - .unwrap(); - - // The peer should only be queried for blocks that it actually has (#2 in this case). - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response, - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(2)); - - // Emulate the peer sending a correct response. - send_block(&setup, BlockNumber(2), response); - wait_for_event(ctx, &mut events_receiver, |ev| { - matches!(ev, PeerStateEvent::GotBlock(BlockNumber(2))) - }) - .await - .unwrap(); - - // No further requests should be made. - clock.advance(BLOCK_SLEEP_INTERVAL); - sync::yield_now().await; - assert!(message_receiver.try_recv().is_none()); - - // Emulate peer receiving / producing a new block. - peer_states - .update(&peer_key, snapshot_sync_state(&setup, 2..=3)) - .unwrap(); - - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response: block3_response, - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(3)); - - // Emulate another peer with full history. - let full_peer_key = rng.gen::().public(); - peer_states - .update(&full_peer_key, sync_state(&setup, 3)) - .unwrap(); - clock.advance(BLOCK_SLEEP_INTERVAL); - - // A node should only request block #1 from the peer; block #3 is already requested, - // and it has #2 locally. - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response, - }) = message; - assert_eq!(recipient, full_peer_key); - assert_eq!(number, BlockNumber(1)); - - send_block(&setup, BlockNumber(1), response); - wait_for_event(ctx, &mut events_receiver, |ev| { - matches!(ev, PeerStateEvent::GotBlock(BlockNumber(1))) - }) - .await - .unwrap(); - - drop(block3_response); // Emulate first peer disconnecting. - wait_for_event( - ctx, - &mut events_receiver, - |ev| matches!(ev,PeerStateEvent::PeerDropped(key) if key == peer_key), - ) - .await - .unwrap(); - clock.advance(BLOCK_SLEEP_INTERVAL); - - // Now, block #3 will be requested from the peer with full history. - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, number, .. - }) = message; - assert_eq!(recipient, full_peer_key); - assert_eq!(number, BlockNumber(3)); - - Ok(()) - } -} - -#[tokio::test] -async fn filtering_requests_for_snapshot_peer() { - test_peer_states(FilteringRequestsForSnapshotPeer).await; -} - -#[derive(Debug)] -struct PruningPeerHistory; - -#[async_trait] -impl Test for PruningPeerHistory { - const BLOCK_COUNT: usize = 5; - - fn tweak_config(&self, config: &mut Config) { - config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; - } - - async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { - let TestHandles { - setup, - peer_states, - mut message_receiver, - mut events_receiver, - clock, - .. - } = handles; - - let rng = &mut ctx.rng(); - let peer_key = rng.gen::().public(); - peer_states - .update(&peer_key, sync_state(&setup, 1)) - .unwrap(); - - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response: block1_response, - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(1)); - - // Emulate peer pruning blocks. - peer_states - .update(&peer_key, snapshot_sync_state(&setup, 3..=3)) - .unwrap(); - - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - response, - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(3)); - - send_block(&setup, BlockNumber(3), response); - wait_for_event(ctx, &mut events_receiver, |ev| { - matches!(ev, PeerStateEvent::GotBlock(BlockNumber(3))) - }) - .await - .unwrap(); - - // No new blocks should be requested (the peer has no block #2). - clock.advance(BLOCK_SLEEP_INTERVAL); - sync::yield_now().await; - assert!(message_receiver.try_recv().is_none()); - - block1_response - .send(Err(GetBlockError::NotAvailable)) - .unwrap(); - // Block #1 should not be requested again (the peer no longer has it). - clock.advance(BLOCK_SLEEP_INTERVAL); - sync::yield_now().await; - assert!(message_receiver.try_recv().is_none()); - - Ok(()) - } -} - -#[tokio::test] -async fn pruning_peer_history() { - test_peer_states(PruningPeerHistory).await; -} - -#[derive(Debug)] -struct BackfillingPeerHistory; - -#[async_trait] -impl Test for BackfillingPeerHistory { - const BLOCK_COUNT: usize = 5; - - fn tweak_config(&self, config: &mut Config) { - config.sleep_interval_for_get_block = BLOCK_SLEEP_INTERVAL; - } - - async fn test(self, ctx: &ctx::Ctx, handles: TestHandles) -> anyhow::Result<()> { - let TestHandles { - setup, - peer_states, - mut message_receiver, - clock, - .. - } = handles; - - let rng = &mut ctx.rng(); - let peer_key = rng.gen::().public(); - peer_states - .update(&peer_key, snapshot_sync_state(&setup, 3..=3)) - .unwrap(); - - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, number, .. - }) = message; - assert_eq!(recipient, peer_key); - assert_eq!(number, BlockNumber(3)); - - peer_states - .update(&peer_key, sync_state(&setup, 3)) - .unwrap(); - clock.advance(BLOCK_SLEEP_INTERVAL); - let mut new_requested_numbers = HashSet::new(); - for _ in 0..2 { - let message = message_receiver.recv(ctx).await?; - let io::OutputMessage::Network(SyncBlocksInputMessage::GetBlock { - recipient, - number, - .. - }) = message; - assert_eq!(recipient, peer_key); - new_requested_numbers.insert(number); - } - assert_eq!( - new_requested_numbers, - HashSet::from([BlockNumber(1), BlockNumber(2)]) - ); - - Ok(()) - } -} - #[tokio::test] async fn backfilling_peer_history() { test_peer_states(BackfillingPeerHistory).await; diff --git a/node/actors/sync_blocks/src/tests/end_to_end.rs b/node/actors/sync_blocks/src/tests/end_to_end.rs index 96f254b1..018b99ee 100644 --- a/node/actors/sync_blocks/src/tests/end_to_end.rs +++ b/node/actors/sync_blocks/src/tests/end_to_end.rs @@ -7,12 +7,13 @@ use std::fmt; use test_casing::test_casing; use tracing::{instrument, Instrument}; use zksync_concurrency::{ - ctx, scope, + ctx, + ctx::channel, + scope, testonly::{abort_on_panic, set_timeout}, }; use zksync_consensus_network as network; use zksync_consensus_storage::testonly::new_store; -use zksync_consensus_utils::no_copy::NoCopy; type NetworkDispatcherPipe = pipe::DispatcherPipe; @@ -20,66 +21,38 @@ type NetworkDispatcherPipe = #[derive(Debug)] struct Node { store: Arc, - setup: Arc, - switch_on_sender: Option>, - _switch_off_sender: oneshot::Sender<()>, + start: channel::Sender<()>, + terminate: channel::Sender<()>, } impl Node { - async fn new_network( - ctx: &ctx::Ctx, - node_count: usize, - gossip_peers: usize, - ) -> (Vec, Vec) { - let rng = &mut ctx.rng(); - // NOTE: originally there were only 4 consensus nodes. - let mut setup = validator::testonly::GenesisSetup::new(rng, node_count); - setup.push_blocks(rng, 20); - let setup = Arc::new(setup); - let mut nodes = vec![]; - let mut runners = vec![]; - for net in network::testonly::new_configs(rng, &setup, gossip_peers) { - let (n, r) = Node::new(ctx, net, setup.clone()).await; - nodes.push(n); - runners.push(r); - } - (nodes, runners) - } - - async fn new( - ctx: &ctx::Ctx, - network: network::Config, - setup: Arc, - ) -> (Self, NodeRunner) { - let (store, store_runner) = new_store(ctx, &setup.blocks[0]).await; - let (switch_on_sender, switch_on_receiver) = oneshot::channel(); - let (switch_off_sender, switch_off_receiver) = oneshot::channel(); + async fn new(ctx: &ctx::Ctx, network: network::Config, setup: &Setup) -> (Self, NodeRunner) { + let (store, store_runner) = new_store(ctx, &setup.genesis).await; + let (start_send, start_recv) = channel::bounded(1); + let (terminate_send, terminate_recv) = channel::bounded(1); let runner = NodeRunner { network, store: store.clone(), store_runner, - setup: setup.clone(), - switch_on_receiver, - switch_off_receiver, + start: start_recv, + terminate: terminate_recv, }; let this = Self { store, - setup, - switch_on_sender: Some(switch_on_sender), - _switch_off_sender: switch_off_sender, + start: start_send, + terminate: terminate_send, }; (this, runner) } - fn switch_on(&mut self) { - self.switch_on_sender.take(); + fn start(&self) { + let _ = self.start.try_send(()); } - async fn put_block(&self, ctx: &ctx::Ctx, block_number: BlockNumber) { - tracing::trace!(%block_number, "Storing new block"); - let block = &self.setup.blocks[block_number.0 as usize]; - self.store.queue_block(ctx, block.clone()).await.unwrap(); + async fn terminate(&self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { + let _ = self.terminate.try_send(()); + self.terminate.closed(ctx).await } } @@ -88,29 +61,25 @@ struct NodeRunner { network: network::Config, store: Arc, store_runner: BlockStoreRunner, - setup: Arc, - switch_on_receiver: oneshot::Receiver<()>, - switch_off_receiver: oneshot::Receiver<()>, + start: channel::Receiver<()>, + terminate: channel::Receiver<()>, } impl NodeRunner { - async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { tracing::info!("NodeRunner::run()"); let key = self.network.gossip.key.public(); let (sync_blocks_actor_pipe, sync_blocks_dispatcher_pipe) = pipe::new(); let (mut network, network_runner) = - network::testonly::Instance::new(self.network.clone(), self.store.clone()); - let sync_blocks_config = test_config(&self.setup); - scope::run!(ctx, |ctx, s| async { + network::testonly::Instance::new(ctx, self.network.clone(), self.store.clone()); + let sync_blocks_config = Config::new(); + let res = scope::run!(ctx, |ctx, s| async { s.spawn_bg(self.store_runner.run(ctx)); s.spawn_bg(network_runner.run(ctx)); network.wait_for_gossip_connections().await; tracing::info!("Node connected to peers"); - self.switch_on_receiver - .recv_or_disconnected(ctx) - .await? - .ok(); + self.start.recv(ctx).await?; tracing::info!("switch_on"); s.spawn_bg( async { @@ -123,11 +92,14 @@ impl NodeRunner { s.spawn_bg(sync_blocks_config.run(ctx, sync_blocks_actor_pipe, self.store.clone())); tracing::info!("Node is fully started"); - let _ = self.switch_off_receiver.recv_or_disconnected(ctx).await; - tracing::info!("Node stopped"); + let _ = self.terminate.recv(ctx).await; + tracing::info!("stopping"); Ok(()) }) - .await + .await; + drop(self.terminate); + tracing::info!("node stopped"); + res } async fn run_executor( @@ -164,7 +136,7 @@ impl NodeRunner { trait GossipNetworkTest: fmt::Debug + Send { /// Returns the number of nodes in the gossip network and number of peers for each node. fn network_params(&self) -> (usize, usize); - async fn test(self, ctx: &ctx::Ctx, network: Vec) -> anyhow::Result<()>; + async fn test(self, ctx: &ctx::Ctx, setup: &Setup, network: Vec) -> anyhow::Result<()>; } #[instrument(level = "trace")] @@ -172,13 +144,22 @@ async fn test_sync_blocks(test: T) { abort_on_panic(); let _guard = set_timeout(TEST_TIMEOUT); let ctx = &ctx::test_root(&ctx::AffineClock::new(25.)); + let rng = &mut ctx.rng(); let (node_count, gossip_peers) = test.network_params(); - let (nodes, runners) = Node::new_network(ctx, node_count, gossip_peers).await; + + let mut setup = validator::testonly::Setup::new(rng, node_count); + setup.push_blocks(rng, 10); scope::run!(ctx, |ctx, s| async { - for (i, runner) in runners.into_iter().enumerate() { + let mut nodes = vec![]; + for (i, net) in network::testonly::new_configs(rng, &setup, gossip_peers) + .into_iter() + .enumerate() + { + let (node, runner) = Node::new(ctx, net, &setup).await; s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); + nodes.push(node); } - test.test(ctx, nodes).await + test.test(ctx, &setup, nodes).await }) .await .unwrap(); @@ -196,52 +177,43 @@ impl GossipNetworkTest for BasicSynchronization { (self.node_count, self.gossip_peers) } - async fn test(self, ctx: &ctx::Ctx, mut node_handles: Vec) -> anyhow::Result<()> { + async fn test(self, ctx: &ctx::Ctx, setup: &Setup, nodes: Vec) -> anyhow::Result<()> { let rng = &mut ctx.rng(); tracing::info!("Check initial node states"); - for node_handle in &mut node_handles { - node_handle.switch_on(); - let state = node_handle.store.subscribe().borrow().clone(); - assert_eq!(state.first.header().number, BlockNumber(0)); - assert_eq!(state.last.header().number, BlockNumber(0)); + for node in &nodes { + node.start(); + let state = node.store.subscribe().borrow().clone(); + assert_eq!(state.first, setup.genesis.fork.first_block); + assert_eq!(state.last, None); } - for block_number in (1..5).map(BlockNumber) { - let sending_node = node_handles.choose(rng).unwrap(); - sending_node.put_block(ctx, block_number).await; - - tracing::info!("Wait until all nodes get block #{block_number}"); - for node_handle in &mut node_handles { - node_handle - .store - .wait_until_persisted(ctx, block_number) - .await?; - tracing::info!("OK"); + for block in &setup.blocks[0..5] { + let node = nodes.choose(rng).unwrap(); + node.store.queue_block(ctx, block.clone()).await.unwrap(); + + tracing::info!("Wait until all nodes get block #{}", block.number()); + for node in &nodes { + node.store.wait_until_persisted(ctx, block.number()).await?; } } - let sending_node = node_handles.choose(rng).unwrap(); + let node = nodes.choose(rng).unwrap(); scope::run!(ctx, |ctx, s| async { // Add a batch of blocks. - for block_number in (5..10).rev().map(BlockNumber) { - let block_number = NoCopy::from(block_number); - s.spawn_bg(async { - sending_node.put_block(ctx, block_number.into_inner()).await; - Ok(()) - }); + for block in setup.blocks[5..].iter().rev() { + s.spawn_bg(node.store.queue_block(ctx, block.clone())); } // Wait until nodes get all new blocks. - for node_handle in &node_handles { - node_handle - .store - .wait_until_persisted(ctx, BlockNumber(9)) - .await?; + let last = setup.blocks.last().unwrap().number(); + for node in &nodes { + node.store.wait_until_persisted(ctx, last).await?; } Ok(()) }) - .await + .await?; + Ok(()) } } @@ -277,37 +249,38 @@ impl GossipNetworkTest for SwitchingOffNodes { (self.node_count, self.node_count / 2) } - async fn test(self, ctx: &ctx::Ctx, mut node_handles: Vec) -> anyhow::Result<()> { + async fn test(self, ctx: &ctx::Ctx, setup: &Setup, mut nodes: Vec) -> anyhow::Result<()> { let rng = &mut ctx.rng(); + nodes.shuffle(rng); - for node_handle in &mut node_handles { - node_handle.switch_on(); + for node in &nodes { + node.start(); } - let mut block_number = BlockNumber(1); - while !node_handles.is_empty() { - tracing::info!("{} nodes left", node_handles.len()); - - let sending_node = node_handles.choose(rng).unwrap(); - sending_node.put_block(ctx, block_number).await; - tracing::info!("block {block_number} inserted"); + for i in 0..nodes.len() { + tracing::info!("{} nodes left", nodes.len() - i); + let block = &setup.blocks[i]; + nodes[i..] + .choose(rng) + .unwrap() + .store + .queue_block(ctx, block.clone()) + .await + .unwrap(); + tracing::info!("block {} inserted", block.number()); // Wait until all remaining nodes get the new block. - for node_handle in &node_handles { - node_handle - .store - .wait_until_persisted(ctx, block_number) - .await?; + for node in &nodes[i..] { + node.store.wait_until_persisted(ctx, block.number()).await?; } - tracing::trace!("All nodes received block #{block_number}"); - block_number = block_number.next(); + tracing::info!("All nodes received block #{}", block.number()); - // Switch off a random node by dropping its handle. + // Terminate a random node. // We start switching off only after the first round, to make sure all nodes are fully // started. - let node_index_to_remove = rng.gen_range(0..node_handles.len()); - node_handles.swap_remove(node_index_to_remove); + nodes[i].terminate(ctx).await.unwrap(); } + tracing::info!("test finished, terminating"); Ok(()) } } @@ -329,30 +302,25 @@ impl GossipNetworkTest for SwitchingOnNodes { (self.node_count, self.node_count / 2) } - async fn test(self, ctx: &ctx::Ctx, mut node_handles: Vec) -> anyhow::Result<()> { + async fn test(self, ctx: &ctx::Ctx, setup: &Setup, mut nodes: Vec) -> anyhow::Result<()> { let rng = &mut ctx.rng(); - - let mut switched_on_nodes = Vec::with_capacity(self.node_count); - let mut block_number = BlockNumber(1); - while switched_on_nodes.len() < self.node_count { - // Switch on a random node. - let node_index_to_switch_on = rng.gen_range(0..node_handles.len()); - let mut node_handle = node_handles.swap_remove(node_index_to_switch_on); - node_handle.switch_on(); - switched_on_nodes.push(node_handle); - - let sending_node = switched_on_nodes.choose(rng).unwrap(); - sending_node.put_block(ctx, block_number).await; + nodes.shuffle(rng); + for i in 0..nodes.len() { + nodes[i].start(); // Switch on a node. + let block = &setup.blocks[i]; + nodes[0..i + 1] + .choose(rng) + .unwrap() + .store + .queue_block(ctx, block.clone()) + .await + .unwrap(); // Wait until all switched on nodes get the new block. - for node_handle in &mut switched_on_nodes { - node_handle - .store - .wait_until_persisted(ctx, block_number) - .await?; + for node in &nodes[0..i + 1] { + node.store.wait_until_persisted(ctx, block.number()).await?; } - tracing::trace!("All nodes received block #{block_number}"); - block_number = block_number.next(); + tracing::trace!("All nodes received block #{}", block.number()); } Ok(()) } diff --git a/node/actors/sync_blocks/src/tests/mod.rs b/node/actors/sync_blocks/src/tests/mod.rs index 54dd2fff..c7273669 100644 --- a/node/actors/sync_blocks/src/tests/mod.rs +++ b/node/actors/sync_blocks/src/tests/mod.rs @@ -1,13 +1,8 @@ //! Tests for the block syncing actor. use super::*; -use rand::{ - distributions::{Distribution, Standard}, - Rng, -}; -use std::ops; -use zksync_concurrency::{oneshot, time}; +use zksync_concurrency::time; use zksync_consensus_network::io::GetBlockError; -use zksync_consensus_roles::validator::{self, testonly::GenesisSetup, BlockNumber, ValidatorSet}; +use zksync_consensus_roles::validator::{self, testonly::Setup}; use zksync_consensus_storage::{BlockStore, BlockStoreRunner, BlockStoreState}; use zksync_consensus_utils::pipe; @@ -15,42 +10,15 @@ mod end_to_end; const TEST_TIMEOUT: time::Duration = time::Duration::seconds(20); -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> Config { - let validator_set: ValidatorSet = rng.gen(); - let consensus_threshold = validator_set.len(); - Config::new(validator_set, consensus_threshold).unwrap() - } -} - -pub(crate) fn test_config(setup: &GenesisSetup) -> Config { - Config::new(setup.validator_set(), setup.keys.len()).unwrap() -} - -pub(crate) fn sync_state(setup: &GenesisSetup, last_block_number: usize) -> BlockStoreState { - snapshot_sync_state(setup, 1..=last_block_number) -} - -pub(crate) fn snapshot_sync_state( - setup: &GenesisSetup, - range: ops::RangeInclusive, -) -> BlockStoreState { - assert!(!range.is_empty()); +pub(crate) fn sync_state(setup: &Setup, last: Option<&validator::FinalBlock>) -> BlockStoreState { BlockStoreState { - first: setup.blocks[*range.start()].justification.clone(), - last: setup.blocks[*range.end()].justification.clone(), + first: setup.genesis.fork.first_block, + last: last.map(|b| b.justification.clone()), } } -pub(crate) fn send_block( - setup: &GenesisSetup, - number: BlockNumber, - response: oneshot::Sender>, -) { - let block = setup - .blocks - .get(number.0 as usize) - .cloned() - .ok_or(GetBlockError::NotAvailable); - response.send(block).ok(); +pub(crate) fn make_response( + block: Option<&validator::FinalBlock>, +) -> Result { + block.cloned().ok_or(GetBlockError::NotAvailable) } diff --git a/node/deny.toml b/node/deny.toml index 37f6ede5..59974629 100644 --- a/node/deny.toml +++ b/node/deny.toml @@ -60,6 +60,9 @@ skip = [ { name = "base64", version = "0.13.1" }, { name = "block-buffer", version = "0.9.0" }, { name = "digest", version = "0.10.7" }, + + # Old versions required by criterion. + { name = "itertools", version = "0.10.5" }, ] [sources] diff --git a/node/libs/concurrency/src/ctx/mod.rs b/node/libs/concurrency/src/ctx/mod.rs index ac67f14d..f86275df 100644 --- a/node/libs/concurrency/src/ctx/mod.rs +++ b/node/libs/concurrency/src/ctx/mod.rs @@ -26,12 +26,14 @@ use std::{fmt, future::Future, pin::Pin, sync::Arc, task}; pub mod channel; mod clock; +mod no_copy; mod rng; mod testonly; #[cfg(test)] mod tests; pub use clock::*; +pub use no_copy::NoCopy; pub use testonly::*; /// Contexts are composed into a tree via `_parent` link. diff --git a/node/libs/utils/src/no_copy.rs b/node/libs/concurrency/src/ctx/no_copy.rs similarity index 50% rename from node/libs/utils/src/no_copy.rs rename to node/libs/concurrency/src/ctx/no_copy.rs index ebef1b33..63744bbe 100644 --- a/node/libs/utils/src/no_copy.rs +++ b/node/libs/concurrency/src/ctx/no_copy.rs @@ -1,25 +1,17 @@ //! No-copy wrapper allowing to carry a `Copy` type into a closure or an `async` block. -use std::ops; - /// No-copy wrapper allowing to carry a `Copy` type into a closure or an `async` block. #[derive(Clone, Debug)] -pub struct NoCopy(T); +pub struct NoCopy(pub T); -impl NoCopy { - /// Converts this wrapper to the contained value. - pub fn into_inner(self) -> T { +impl NoCopy { + /// Extracts the wrapped value. + pub fn into(self) -> T { self.0 } } -impl From for NoCopy { - fn from(value: T) -> Self { - Self(value) - } -} - -impl ops::Deref for NoCopy { +impl std::ops::Deref for NoCopy { type Target = T; fn deref(&self) -> &T { diff --git a/node/libs/concurrency/src/testonly.rs b/node/libs/concurrency/src/testonly.rs index c726eef9..67870101 100644 --- a/node/libs/concurrency/src/testonly.rs +++ b/node/libs/concurrency/src/testonly.rs @@ -38,6 +38,7 @@ pub fn abort_on_panic() { /// Guard which has to be dropped before timeout is reached. /// Otherwise the test will panic. #[allow(unused_tuple_struct_fields)] +#[must_use] pub struct TimeoutGuard(std::sync::mpsc::Sender<()>); /// Panics if (real time) timeout is reached before ctx is canceled. diff --git a/node/libs/roles/Cargo.toml b/node/libs/roles/Cargo.toml index 2af1edf9..89122a76 100644 --- a/node/libs/roles/Cargo.toml +++ b/node/libs/roles/Cargo.toml @@ -22,8 +22,11 @@ serde.workspace = true thiserror.workspace = true tracing.workspace = true +[dev-dependencies] +assert_matches.workspace = true + [build-dependencies] zksync_protobuf_build.workspace = true [lints] -workspace = true \ No newline at end of file +workspace = true diff --git a/node/libs/roles/src/proto/validator.proto b/node/libs/roles/src/proto/validator.proto index 9393369b..f1ec2438 100644 --- a/node/libs/roles/src/proto/validator.proto +++ b/node/libs/roles/src/proto/validator.proto @@ -4,6 +4,21 @@ package zksync.roles.validator; import "zksync/std.proto"; +message Fork { + optional uint64 number = 1; // required; ForkId + optional uint64 first_block = 2; // required; BlockNumber + optional BlockHeaderHash first_parent = 3; // optional +} + +message Genesis { + optional Fork fork = 1; // required + repeated PublicKey validators = 2; +} + +message GenesisHash { + optional bytes keccak256 = 1; // required +} + message PayloadHash { optional bytes keccak256 = 1; // required } @@ -14,7 +29,7 @@ message BlockHeaderHash { message BlockHeader { // Hash of the parent Block. - optional BlockHeaderHash parent = 2; // required + optional BlockHeaderHash parent = 2; // optional // Sequential number of the block = parent.number + 1. optional uint64 number = 3; // required // Hash of the block payload. @@ -26,6 +41,12 @@ message FinalBlock { optional CommitQC justification = 2; // required } +message View { + optional uint32 protocol_version = 1; // required; ProtocolVersion + optional uint64 fork = 2; // required; ForkId + optional uint64 number = 3; // required; ViewNumber +} + message ConsensusMsg { oneof t { // required ReplicaPrepare replica_prepare = 1; @@ -36,32 +57,28 @@ message ConsensusMsg { } message ReplicaPrepare { - optional uint32 protocol_version = 4; // required - optional uint64 view = 1; // required - optional ReplicaCommit high_vote = 2; // required - optional CommitQC high_qc = 3; // required + optional View view = 1; // required + optional ReplicaCommit high_vote = 2; // optional + optional CommitQC high_qc = 3; // optional } message ReplicaCommit { - optional uint32 protocol_version = 3; // required - optional uint64 view = 1; // required + optional View view = 1; // required optional BlockHeader proposal = 2; // required } message LeaderPrepare { - optional uint32 protocol_version = 5; // required - optional uint64 view = 1; // required - optional BlockHeader proposal = 2; // required - optional bytes proposal_payload = 3; // optional (depending on justification) - optional PrepareQC justification = 4; // required + optional BlockHeader proposal = 1; // required + optional bytes proposal_payload = 2; // optional (depending on justification) + optional PrepareQC justification = 3; // required } message LeaderCommit { - optional uint32 protocol_version = 2; // required optional CommitQC justification = 1; // required } message PrepareQC { + optional View view = 4; // required repeated ReplicaPrepare msgs = 1; // required repeated std.BitVector signers = 2; // required optional AggregateSignature sig = 3; // required diff --git a/node/libs/roles/src/validator/conv.rs b/node/libs/roles/src/validator/conv.rs index 33748b43..04886834 100644 --- a/node/libs/roles/src/validator/conv.rs +++ b/node/libs/roles/src/validator/conv.rs @@ -1,15 +1,68 @@ use super::{ AggregateSignature, BlockHeader, BlockHeaderHash, BlockNumber, CommitQC, ConsensusMsg, - FinalBlock, LeaderCommit, LeaderPrepare, Msg, MsgHash, NetAddress, Payload, PayloadHash, Phase, - PrepareQC, ProtocolVersion, PublicKey, ReplicaCommit, ReplicaPrepare, Signature, Signed, - Signers, ViewNumber, + FinalBlock, Fork, ForkNumber, Genesis, GenesisHash, LeaderCommit, LeaderPrepare, Msg, MsgHash, + NetAddress, Payload, PayloadHash, Phase, PrepareQC, ProtocolVersion, PublicKey, ReplicaCommit, + ReplicaPrepare, Signature, Signed, Signers, ValidatorSet, View, ViewNumber, }; use crate::{node::SessionId, proto::validator as proto}; use anyhow::Context as _; use std::collections::BTreeMap; use zksync_consensus_crypto::ByteFmt; use zksync_consensus_utils::enum_util::Variant; -use zksync_protobuf::{read_required, required, ProtoFmt}; +use zksync_protobuf::{read_optional, read_required, required, ProtoFmt}; + +impl ProtoFmt for Fork { + type Proto = proto::Fork; + fn read(r: &Self::Proto) -> anyhow::Result { + Ok(Self { + number: ForkNumber(*required(&r.number).context("number")?), + first_block: BlockNumber(*required(&r.first_block).context("first_block")?), + first_parent: read_optional(&r.first_parent).context("first_parent")?, + }) + } + fn build(&self) -> Self::Proto { + Self::Proto { + number: Some(self.number.0), + first_block: Some(self.first_block.0), + first_parent: self.first_parent.as_ref().map(|x| x.build()), + } + } +} + +impl ProtoFmt for Genesis { + type Proto = proto::Genesis; + fn read(r: &Self::Proto) -> anyhow::Result { + let validators: Vec<_> = r + .validators + .iter() + .enumerate() + .map(|(i, v)| PublicKey::read(v).context(i)) + .collect::>() + .context("validators")?; + Ok(Self { + fork: read_required(&r.fork).context("fork")?, + validators: ValidatorSet::new(validators.into_iter()).context("validators")?, + }) + } + fn build(&self) -> Self::Proto { + Self::Proto { + fork: Some(self.fork.build()), + validators: self.validators.iter().map(|x| x.build()).collect(), + } + } +} + +impl ProtoFmt for GenesisHash { + type Proto = proto::GenesisHash; + fn read(r: &Self::Proto) -> anyhow::Result { + Ok(Self(ByteFmt::decode(required(&r.keccak256)?)?)) + } + fn build(&self) -> Self::Proto { + Self::Proto { + keccak256: Some(self.0.encode()), + } + } +} impl ProtoFmt for BlockHeaderHash { type Proto = proto::BlockHeaderHash; @@ -39,14 +92,14 @@ impl ProtoFmt for BlockHeader { type Proto = proto::BlockHeader; fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { - parent: read_required(&r.parent).context("parent")?, - number: BlockNumber(r.number.context("number")?), + parent: read_optional(&r.parent).context("parent")?, + number: BlockNumber(*required(&r.number).context("number")?), payload: read_required(&r.payload).context("payload")?, }) } fn build(&self) -> Self::Proto { Self::Proto { - parent: Some(self.parent.build()), + parent: self.parent.as_ref().map(ProtoFmt::build), number: Some(self.number.0), payload: Some(self.payload.build()), } @@ -99,24 +152,42 @@ impl ProtoFmt for ConsensusMsg { } } -impl ProtoFmt for ReplicaPrepare { - type Proto = proto::ReplicaPrepare; +impl ProtoFmt for View { + type Proto = proto::View; fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { protocol_version: ProtocolVersion(r.protocol_version.context("protocol_version")?), - view: ViewNumber(*required(&r.view).context("view")?), - high_vote: read_required(&r.high_vote).context("high_vote")?, - high_qc: read_required(&r.high_qc).context("high_qc")?, + fork: ForkNumber(*required(&r.fork).context("fork")?), + number: ViewNumber(*required(&r.number).context("number")?), }) } fn build(&self) -> Self::Proto { Self::Proto { protocol_version: Some(self.protocol_version.0), - view: Some(self.view.0), - high_vote: Some(self.high_vote.build()), - high_qc: Some(self.high_qc.build()), + fork: Some(self.fork.0), + number: Some(self.number.0), + } + } +} + +impl ProtoFmt for ReplicaPrepare { + type Proto = proto::ReplicaPrepare; + + fn read(r: &Self::Proto) -> anyhow::Result { + Ok(Self { + view: read_required(&r.view).context("view")?, + high_vote: read_optional(&r.high_vote).context("high_vote")?, + high_qc: read_optional(&r.high_qc).context("high_qc")?, + }) + } + + fn build(&self) -> Self::Proto { + Self::Proto { + view: Some(self.view.build()), + high_vote: self.high_vote.as_ref().map(ProtoFmt::build), + high_qc: self.high_qc.as_ref().map(ProtoFmt::build), } } } @@ -126,16 +197,14 @@ impl ProtoFmt for ReplicaCommit { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { - protocol_version: ProtocolVersion(r.protocol_version.context("protocol_version")?), - view: ViewNumber(*required(&r.view).context("view")?), + view: read_required(&r.view).context("view")?, proposal: read_required(&r.proposal).context("proposal")?, }) } fn build(&self) -> Self::Proto { Self::Proto { - protocol_version: Some(self.protocol_version.0), - view: Some(self.view.0), + view: Some(self.view.build()), proposal: Some(self.proposal.build()), } } @@ -146,8 +215,6 @@ impl ProtoFmt for LeaderPrepare { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { - protocol_version: ProtocolVersion(r.protocol_version.context("protocol_version")?), - view: ViewNumber(*required(&r.view).context("view")?), proposal: read_required(&r.proposal).context("proposal")?, proposal_payload: r.proposal_payload.as_ref().map(|p| Payload(p.clone())), justification: read_required(&r.justification).context("justification")?, @@ -156,8 +223,6 @@ impl ProtoFmt for LeaderPrepare { fn build(&self) -> Self::Proto { Self::Proto { - protocol_version: Some(self.protocol_version.0), - view: Some(self.view.0), proposal: Some(self.proposal.build()), proposal_payload: self.proposal_payload.as_ref().map(|p| p.0.clone()), justification: Some(self.justification.build()), @@ -170,14 +235,12 @@ impl ProtoFmt for LeaderCommit { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { - protocol_version: ProtocolVersion(r.protocol_version.context("protocol_version")?), justification: read_required(&r.justification).context("justification")?, }) } fn build(&self) -> Self::Proto { Self::Proto { - protocol_version: Some(self.protocol_version.0), justification: Some(self.justification.build()), } } @@ -209,6 +272,7 @@ impl ProtoFmt for PrepareQC { } Ok(Self { + view: read_required(&r.view).context("view")?, map, signature: read_required(&r.sig).context("sig")?, }) @@ -222,6 +286,7 @@ impl ProtoFmt for PrepareQC { .unzip(); Self::Proto { + view: Some(self.view.build()), msgs, signers, sig: Some(self.signature.build()), diff --git a/node/libs/roles/src/validator/messages/block.rs b/node/libs/roles/src/validator/messages/block.rs index 5ea7f486..261035e5 100644 --- a/node/libs/roles/src/validator/messages/block.rs +++ b/node/libs/roles/src/validator/messages/block.rs @@ -1,6 +1,6 @@ //! Messages related to blocks. -use super::CommitQC; +use super::{CommitQC, CommitQCVerifyError}; use std::fmt; use zksync_consensus_crypto::{keccak256::Keccak256, ByteFmt, Text, TextFmt}; @@ -64,8 +64,8 @@ impl BlockNumber { } /// Returns the previous block number. - pub fn prev(self) -> Self { - Self(self.0 - 1) + pub fn prev(self) -> Option { + Some(Self(self.0.checked_sub(1)?)) } } @@ -80,11 +80,6 @@ impl fmt::Display for BlockNumber { pub struct BlockHeaderHash(pub(crate) Keccak256); impl BlockHeaderHash { - /// Constant that the parent of the genesis block should be set to. - pub fn genesis_parent() -> Self { - Self(Keccak256::default()) - } - /// Interprets the specified `bytes` as a block header hash digest (i.e., a reverse operation to [`Self::as_bytes()`]). /// It is caller's responsibility to ensure that `bytes` are actually a block header hash digest. pub fn from_bytes(bytes: [u8; 32]) -> Self { @@ -122,7 +117,7 @@ impl fmt::Debug for BlockHeaderHash { #[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] pub struct BlockHeader { /// Hash of the parent block. - pub parent: BlockHeaderHash, + pub parent: Option, /// Number of the block. pub number: BlockNumber, /// Payload of the block. @@ -135,19 +130,10 @@ impl BlockHeader { BlockHeaderHash(Keccak256::new(&zksync_protobuf::canonical(self))) } - /// Creates a genesis block. - pub fn genesis(payload: PayloadHash, number: BlockNumber) -> Self { - Self { - parent: BlockHeaderHash::genesis_parent(), - number, - payload, - } - } - /// Creates a child block for the given parent. - pub fn new(parent: &BlockHeader, payload: PayloadHash) -> Self { + pub fn next(parent: &BlockHeader, payload: PayloadHash) -> Self { Self { - parent: parent.hash(), + parent: Some(parent.hash()), number: parent.number.next(), payload, } @@ -166,7 +152,7 @@ pub struct FinalBlock { impl FinalBlock { /// Creates a new finalized block. pub fn new(payload: Payload, justification: CommitQC) -> Self { - assert_eq!(justification.message.proposal.payload, payload.hash()); + assert_eq!(justification.header().payload, payload.hash()); Self { payload, justification, @@ -178,12 +164,13 @@ impl FinalBlock { &self.justification.message.proposal } - /// Validates internal consistency of this block. - pub fn validate( - &self, - validators: &super::ValidatorSet, - consensus_threshold: usize, - ) -> Result<(), BlockValidationError> { + /// Number of the block. + pub fn number(&self) -> BlockNumber { + self.header().number + } + + /// Verifies internal consistency of this block. + pub fn verify(&self, genesis: &super::Genesis) -> Result<(), BlockValidationError> { let payload_hash = self.payload.hash(); if payload_hash != self.header().payload { return Err(BlockValidationError::HashMismatch { @@ -192,7 +179,7 @@ impl FinalBlock { }); } self.justification - .verify(validators, consensus_threshold) + .verify(genesis) .map_err(BlockValidationError::Justification) } } @@ -234,5 +221,5 @@ pub enum BlockValidationError { }, /// Failed verifying quorum certificate. #[error("failed verifying quorum certificate: {0:#?}")] - Justification(#[source] anyhow::Error), + Justification(#[source] CommitQCVerifyError), } diff --git a/node/libs/roles/src/validator/messages/consensus.rs b/node/libs/roles/src/validator/messages/consensus.rs index d7a1d9ba..870355b0 100644 --- a/node/libs/roles/src/validator/messages/consensus.rs +++ b/node/libs/roles/src/validator/messages/consensus.rs @@ -1,10 +1,14 @@ //! Messages related to the consensus protocol. - -use super::{BlockHeader, Msg, Payload, Signed}; -use crate::{validator, validator::Signature}; -use anyhow::bail; +use super::{ + BlockHeaderHash, BlockNumber, LeaderCommit, LeaderPrepare, Msg, ReplicaCommit, ReplicaPrepare, +}; +use crate::validator; use bit_vec::BitVec; -use std::collections::{BTreeMap, BTreeSet}; +use std::{ + collections::{BTreeMap, BTreeSet}, + fmt, +}; +use zksync_consensus_crypto::{keccak256::Keccak256, ByteFmt, Text, TextFmt}; use zksync_consensus_utils::enum_util::{BadVariantError, Variant}; /// Version of the consensus algorithm that the validator is using. @@ -40,6 +44,166 @@ impl TryFrom for ProtocolVersion { } } +/// Number of the fork. Newer fork has higher number. +#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct ForkNumber(pub u64); + +impl ForkNumber { + /// Next fork number. + pub fn next(self) -> Self { + Self(self.0 + 1) + } +} + +/// Specification of a fork. +#[derive(Clone, Debug, PartialEq)] +pub struct Fork { + /// Number of the fork. + pub number: ForkNumber, + /// First block of a fork. + pub first_block: BlockNumber, + /// Parent fo the first block of a fork. + pub first_parent: Option, +} + +impl Default for Fork { + fn default() -> Self { + Self { + number: ForkNumber(0), + first_block: BlockNumber(0), + first_parent: None, + } + } +} + +/// A struct that represents a set of validators. It is used to store the current validator set. +/// We represent each validator by its validator public key. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct ValidatorSet { + vec: Vec, + map: BTreeMap, +} + +impl ValidatorSet { + /// Creates a new ValidatorSet from a list of validator public keys. + pub fn new(validators: impl IntoIterator) -> anyhow::Result { + let mut set = BTreeSet::new(); + for validator in validators { + anyhow::ensure!(set.insert(validator), "Duplicate validator in ValidatorSet"); + } + anyhow::ensure!( + !set.is_empty(), + "ValidatorSet must contain at least one validator" + ); + Ok(Self { + vec: set.iter().cloned().collect(), + map: set.into_iter().enumerate().map(|(i, pk)| (pk, i)).collect(), + }) + } + + /// Iterates over validators. + pub fn iter(&self) -> impl Iterator { + self.vec.iter() + } + + /// Returns the number of validators. + #[allow(clippy::len_without_is_empty)] // a valid `ValidatorSet` is always non-empty by construction + pub fn len(&self) -> usize { + self.vec.len() + } + + /// Returns true if the given validator is in the validator set. + pub fn contains(&self, validator: &validator::PublicKey) -> bool { + self.map.contains_key(validator) + } + + /// Get validator by its index in the set. + pub fn get(&self, index: usize) -> Option<&validator::PublicKey> { + self.vec.get(index) + } + + /// Get the index of a validator in the set. + pub fn index(&self, validator: &validator::PublicKey) -> Option { + self.map.get(validator).copied() + } + + /// Computes the validator for the given view. + pub fn view_leader(&self, view_number: ViewNumber) -> validator::PublicKey { + let index = view_number.0 as usize % self.len(); + self.get(index).unwrap().clone() + } + + /// Signature threshold for this validator set. + pub fn threshold(&self) -> usize { + threshold(self.len()) + } + + /// Maximal number of faulty replicas allowed in this validator set. + pub fn faulty_replicas(&self) -> usize { + faulty_replicas(self.len()) + } +} + +/// Calculate the consensus threshold, the minimum number of votes for any consensus action to be valid, +/// for a given number of replicas. +pub fn threshold(n: usize) -> usize { + n - faulty_replicas(n) +} + +/// Calculate the maximum number of faulty replicas, for a given number of replicas. +pub fn faulty_replicas(n: usize) -> usize { + // Calculate the allowed maximum number of faulty replicas. We want the following relationship to hold: + // n = 5*f + 1 + // for n total replicas and f faulty replicas. This results in the following formula for the maximum + // number of faulty replicas: + // f = floor((n - 1) / 5) + // Because of this, it doesn't make sense to have 5*f + 2 or 5*f + 3 replicas. It won't increase the number + // of allowed faulty replicas. + (n - 1) / 5 +} + +/// Genesis of the blockchain, unique for each blockchain instance. +#[derive(Debug, Clone, PartialEq)] +pub struct Genesis { + // TODO(gprusak): add blockchain id here. + /// Set of validators of the chain. + pub validators: ValidatorSet, + /// Fork of the chain to follow. + pub fork: Fork, +} + +/// Hash of the genesis specification. +#[derive(Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct GenesisHash(pub(crate) Keccak256); + +impl Genesis { + /// Hash of the genesis. + pub fn hash(&self) -> GenesisHash { + GenesisHash(Keccak256::new(&zksync_protobuf::canonical(self))) + } +} + +impl TextFmt for GenesisHash { + fn decode(text: Text) -> anyhow::Result { + text.strip("genesis_hash:keccak256:")? + .decode_hex() + .map(Self) + } + + fn encode(&self) -> String { + format!( + "genesis_hash:keccak256:{}", + hex::encode(ByteFmt::encode(&self.0)) + ) + } +} + +impl fmt::Debug for GenesisHash { + fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt.write_str(&TextFmt::encode(self)) + } +} + /// Consensus messages. #[allow(missing_docs)] #[derive(Clone, Debug, PartialEq, Eq)] @@ -61,15 +225,20 @@ impl ConsensusMsg { } } - /// Protocol version of this message. - pub fn protocol_version(&self) -> ProtocolVersion { + /// View of this message. + pub fn view(&self) -> &View { match self { - Self::ReplicaPrepare(m) => m.protocol_version, - Self::ReplicaCommit(m) => m.protocol_version, - Self::LeaderPrepare(m) => m.protocol_version, - Self::LeaderCommit(m) => m.protocol_version, + Self::ReplicaPrepare(m) => &m.view, + Self::ReplicaCommit(m) => &m.view, + Self::LeaderPrepare(m) => m.view(), + Self::LeaderCommit(m) => m.view(), } } + + /// Protocol version of this message. + pub fn protocol_version(&self) -> ProtocolVersion { + self.view().protocol_version + } } impl Variant for ReplicaPrepare { @@ -120,225 +289,21 @@ impl Variant for LeaderCommit { } } -/// A Prepare message from a replica. -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] -pub struct ReplicaPrepare { - /// Protocol version. - pub protocol_version: ProtocolVersion, - /// The number of the current view. - pub view: ViewNumber, - /// The highest block that the replica has committed to. - pub high_vote: ReplicaCommit, - /// The highest CommitQC that the replica has seen. - pub high_qc: CommitQC, -} - -/// A Commit message from a replica. -#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct ReplicaCommit { +/// View specification. +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct View { /// Protocol version. pub protocol_version: ProtocolVersion, + /// Fork this message belongs to. + pub fork: ForkNumber, /// The number of the current view. - pub view: ViewNumber, - /// The header of the block that the replica is committing to. - pub proposal: BlockHeader, + pub number: ViewNumber, } -/// A Prepare message from a leader. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct LeaderPrepare { - /// Protocol version. - pub protocol_version: ProtocolVersion, - /// The number of the current view. - pub view: ViewNumber, - /// The header of the block that the leader is proposing. - pub proposal: BlockHeader, - /// Payload of the block that the leader is proposing. - /// `None` iff this is a reproposal. - pub proposal_payload: Option, - /// The PrepareQC that justifies this proposal from the leader. - pub justification: PrepareQC, -} - -/// A Commit message from a leader. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct LeaderCommit { - /// Protocol version. - pub protocol_version: ProtocolVersion, - /// The CommitQC that justifies the message from the leader. - pub justification: CommitQC, -} - -/// A quorum certificate of replica Prepare messages. Since not all Prepare messages are -/// identical (they have different high blocks and high QCs), we need to keep the high blocks -/// and high QCs in a map. We can still aggregate the signatures though. -#[derive(Clone, Debug, PartialEq, Eq, Default)] -pub struct PrepareQC { - /// Map from replica Prepare messages to the validators that signed them. - pub map: BTreeMap, - /// Aggregate signature of the replica Prepare messages. - pub signature: validator::AggregateSignature, -} - -impl PrepareQC { - /// View of the QC. - pub fn view(&self) -> ViewNumber { - self.map - .keys() - .map(|k| k.view) - .next() - .unwrap_or(ViewNumber(0)) - } - - /// Add a validator's signed message. - /// * `signed_message` - A valid signed `ReplicaPrepare` message. - /// * `validator_index` - The signer index in the validator set. - /// * `validator_set` - The validator set. - pub fn add( - &mut self, - signed_message: &Signed, - validator_index: usize, - validator_set: &ValidatorSet, - ) { - self.map - .entry(signed_message.msg.clone()) - .or_insert_with(|| Signers(BitVec::from_elem(validator_set.len(), false))) - .0 - .set(validator_index, true); - - self.signature.add(&signed_message.sig); - } - - /// Verifies the integrity of the PrepareQC. - pub fn verify( - &self, - view: ViewNumber, - validators: &ValidatorSet, - threshold: usize, - ) -> anyhow::Result<()> { - // First we check that all messages are for the same view number. - for msg in self.map.keys() { - if msg.view != view { - bail!("PrepareQC contains messages for different views!"); - } - } - - // Then we need to do some checks on the signers bit maps. - let mut bit_map = BitVec::from_elem(validators.len(), false); - let mut num_signers = 0; - - for signer_bitmap in self.map.values() { - let signers = signer_bitmap.0.clone(); - - if signers.len() != validators.len() { - bail!("Bit vector in PrepareQC has wrong length!"); - } - - if !signers.any() { - bail!("Empty bit vector in PrepareQC. We require at least one signer for every message!"); - } - - let mut intersection = bit_map.clone(); - intersection.and(&signers); - if intersection.any() { - bail!("Bit vectors in PrepareQC are not disjoint. We require that every validator signs at most one message!"); - } - bit_map.or(&signers); - - num_signers += signers.iter().filter(|b| *b).count(); - } - - // Verify that we have enough signers. - // TODO(gprusak): how about num_signers == threshold to make the certificates more uniform? - if num_signers < threshold { - bail!( - "Insufficient signers in PrepareQC.\nNumber of signers: {}\nThreshold: {}", - num_signers, - threshold - ); - } - - // Now we can verify the signature. - let messages_and_keys = self.map.clone().into_iter().flat_map(|(msg, signers)| { - validators - .iter() - .enumerate() - .filter(|(i, _)| signers.0[*i]) - .map(|(_, pk)| (msg.clone(), pk)) - .collect::>() - }); - - Ok(self.signature.verify_messages(messages_and_keys)?) - } -} - -/// A Commit Quorum Certificate. It is an aggregate of signed replica Commit messages. -/// The Quorum Certificate is supposed to be over identical messages, so we only need one message. -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] -pub struct CommitQC { - /// The replica Commit message that the QC is for. - pub message: ReplicaCommit, - /// The validators that signed this message. - pub signers: Signers, - /// The aggregate signature of the signed replica messages. - pub signature: validator::AggregateSignature, -} - -impl CommitQC { - /// Header of the certified block. - pub fn header(&self) -> &BlockHeader { - &self.message.proposal - } - - /// Create a new empty instance for a given `ReplicaCommit` message and a validator set size. - pub fn new(message: ReplicaCommit, validator_set: &ValidatorSet) -> Self { - Self { - message, - signers: Signers(BitVec::from_elem(validator_set.len(), false)), - signature: validator::AggregateSignature::default(), - } - } - - /// Add a validator's signature. - /// * `sig` - A valid signature. - /// * `validator_index` - The signer index in the validator set. - pub fn add(&mut self, sig: &Signature, validator_index: usize) { - self.signers.0.set(validator_index, true); - self.signature.add(sig); - } - - /// Verifies the signature of the CommitQC. - pub fn verify(&self, validators: &ValidatorSet, threshold: usize) -> anyhow::Result<()> { - let signers = self.signers.0.clone(); - - // First we to do some checks on the signers bit map. - if signers.len() != validators.len() { - bail!("Bit vector in CommitQC has wrong length!"); - } - - if !signers.any() { - bail!("Empty bit vector in CommitQC. We require at least one signer!"); - } - - // Verify that we have enough signers. - let num_signers = signers.iter().filter(|b| *b).count(); - - if num_signers < threshold { - bail!( - "Insufficient signers in CommitQC.\nNumber of signers: {}\nThreshold: {}", - num_signers, - threshold - ); - } - - // Now we can verify the signature. - let messages_and_keys = validators - .iter() - .enumerate() - .filter(|(i, _)| signers[*i]) - .map(|(_, pk)| (self.message, pk)); - - Ok(self.signature.verify_messages(messages_and_keys)?) +impl View { + /// Checks if `self` can occur after `b`. + pub fn after(&self, b: &Self) -> bool { + self.fork == b.fork && self.number > b.number && self.protocol_version >= b.protocol_version } } @@ -348,71 +313,46 @@ impl CommitQC { pub struct Signers(pub BitVec); impl Signers { - /// Returns the number of signers, i.e. the number of validators that signed + /// Constructs an empty signers set. + pub fn new(n: usize) -> Self { + Self(BitVec::from_elem(n, false)) + } + + /// Returns the number of signers, i.e. the number of validators that signed /// the particular message that this signer bitmap refers to. - pub fn len(&self) -> usize { + pub fn count(&self) -> usize { self.0.iter().filter(|b| *b).count() } + /// Size of the corresponding ValidatorSet. + pub fn len(&self) -> usize { + self.0.len() + } + /// Returns true if there are no signers. pub fn is_empty(&self) -> bool { self.0.none() } } -/// A struct that represents a set of validators. It is used to store the current validator set. -/// We represent each validator by its validator public key. -#[derive(Clone, Debug, PartialEq, Eq)] -pub struct ValidatorSet { - vec: Vec, - map: BTreeMap, -} - -impl ValidatorSet { - /// Creates a new ValidatorSet from a list of validator public keys. - pub fn new(validators: impl IntoIterator) -> anyhow::Result { - let mut set = BTreeSet::new(); - - for validator in validators { - if !set.insert(validator) { - bail!("Duplicate validator in ValidatorSet"); - } - } - - if set.is_empty() { - bail!("ValidatorSet must contain at least one validator"); - } - - Ok(Self { - vec: set.iter().cloned().collect(), - map: set.into_iter().enumerate().map(|(i, pk)| (pk, i)).collect(), - }) - } - - /// Iterates over validators. - pub fn iter(&self) -> impl Iterator { - self.vec.iter() - } - - /// Returns the number of validators. - #[allow(clippy::len_without_is_empty)] // a valid `ValidatorSet` is always non-empty by construction - pub fn len(&self) -> usize { - self.vec.len() - } - - /// Returns true if the given validator is in the validator set. - pub fn contains(&self, validator: &validator::PublicKey) -> bool { - self.map.contains_key(validator) +impl std::ops::BitOrAssign<&Self> for Signers { + fn bitor_assign(&mut self, other: &Self) { + self.0.or(&other.0); } +} - /// Get validator by its index in the set. - pub fn get(&self, index: usize) -> Option<&validator::PublicKey> { - self.vec.get(index) +impl std::ops::BitAndAssign<&Self> for Signers { + fn bitand_assign(&mut self, other: &Self) { + self.0.and(&other.0); } +} - /// Get the index of a validator in the set. - pub fn index(&self, validator: &validator::PublicKey) -> Option { - self.map.get(validator).copied() +impl std::ops::BitAnd for &Signers { + type Output = Signers; + fn bitand(self, other: Self) -> Signers { + let mut this = self.clone(); + this &= other; + this } } @@ -425,11 +365,6 @@ impl ViewNumber { pub fn next(self) -> Self { Self(self.0 + 1) } - - /// Get the previous view number. - pub fn prev(self) -> Self { - Self(self.0 - 1) - } } /// An enum that represents the current phase of the consensus. diff --git a/node/libs/roles/src/validator/messages/leader_commit.rs b/node/libs/roles/src/validator/messages/leader_commit.rs new file mode 100644 index 00000000..f9eb7aa9 --- /dev/null +++ b/node/libs/roles/src/validator/messages/leader_commit.rs @@ -0,0 +1,125 @@ +use super::{BlockHeader, Genesis, ReplicaCommit, Signed, Signers, View}; +use crate::validator; + +/// A Commit message from a leader. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct LeaderCommit { + /// The CommitQC that justifies the message from the leader. + pub justification: CommitQC, +} + +impl LeaderCommit { + /// Verifies LeaderCommit. + pub fn verify(&self, genesis: &Genesis) -> Result<(), CommitQCVerifyError> { + self.justification.verify(genesis) + } + + /// View of this message. + pub fn view(&self) -> &View { + self.justification.view() + } +} + +/// A Commit Quorum Certificate. It is an aggregate of signed replica Commit messages. +/// The Quorum Certificate is supposed to be over identical messages, so we only need one message. +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] +pub struct CommitQC { + /// The replica Commit message that the QC is for. + pub message: ReplicaCommit, + /// The validators that signed this message. + pub signers: Signers, + /// The aggregate signature of the signed replica messages. + pub signature: validator::AggregateSignature, +} + +/// Error returned by `CommitQc::verify()`. +#[derive(thiserror::Error, Debug)] +pub enum CommitQCVerifyError { + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] anyhow::Error), + /// Bad signer set. + #[error("signers set doesn't match genesis")] + BadSignersSet, + /// Not enough signers. + #[error("not enough signers: got {got}, want {want}")] + NotEnoughSigners { + /// Got signers. + got: usize, + /// Want signers. + want: usize, + }, + /// Bad signature. + #[error("bad signature: {0:#}")] + BadSignature(#[source] validator::Error), +} + +impl CommitQC { + /// Header of the certified block. + pub fn header(&self) -> &BlockHeader { + &self.message.proposal + } + + /// View of this QC. + pub fn view(&self) -> &View { + &self.message.view + } + + /// Create a new empty instance for a given `ReplicaCommit` message and a validator set size. + pub fn new(message: ReplicaCommit, genesis: &Genesis) -> Self { + Self { + message, + signers: Signers::new(genesis.validators.len()), + signature: validator::AggregateSignature::default(), + } + } + + /// Add a validator's signature. + /// Signature is assumed to be already verified. + pub fn add(&mut self, msg: &Signed, genesis: &Genesis) { + if self.message != msg.msg { + return; + }; + let Some(i) = genesis.validators.index(&msg.key) else { + return; + }; + if self.signers.0[i] { + return; + }; + self.signers.0.set(i, true); + self.signature.add(&msg.sig); + } + + /// Verifies the signature of the CommitQC. + pub fn verify(&self, genesis: &Genesis) -> Result<(), CommitQCVerifyError> { + use CommitQCVerifyError as Error; + self.message + .verify(genesis) + .map_err(Error::InvalidMessage)?; + if self.signers.len() != genesis.validators.len() { + return Err(Error::BadSignersSet); + } + + // Verify that we have enough signers. + let num_signers = self.signers.count(); + let threshold = genesis.validators.threshold(); + if num_signers < threshold { + return Err(Error::NotEnoughSigners { + got: num_signers, + want: threshold, + }); + } + + // Now we can verify the signature. + let messages_and_keys = genesis + .validators + .iter() + .enumerate() + .filter(|(i, _)| self.signers.0[*i]) + .map(|(_, pk)| (self.message.clone(), pk)); + + self.signature + .verify_messages(messages_and_keys) + .map_err(Error::BadSignature) + } +} diff --git a/node/libs/roles/src/validator/messages/leader_prepare.rs b/node/libs/roles/src/validator/messages/leader_prepare.rs new file mode 100644 index 00000000..15580bb3 --- /dev/null +++ b/node/libs/roles/src/validator/messages/leader_prepare.rs @@ -0,0 +1,267 @@ +use super::{ + BlockHeader, BlockHeaderHash, BlockNumber, CommitQC, Genesis, Payload, ReplicaPrepare, + ReplicaPrepareVerifyError, Signed, Signers, View, +}; +use crate::validator; +use std::collections::{BTreeMap, HashMap}; + +/// A quorum certificate of replica Prepare messages. Since not all Prepare messages are +/// identical (they have different high blocks and high QCs), we need to keep the high blocks +/// and high QCs in a map. We can still aggregate the signatures though. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct PrepareQC { + /// View of this QC. + pub view: View, + /// Map from replica Prepare messages to the validators that signed them. + pub map: BTreeMap, + /// Aggregate signature of the replica Prepare messages. + pub signature: validator::AggregateSignature, +} + +/// Error returned by `PrepareQC::verify()`. +#[derive(thiserror::Error, Debug)] +pub enum PrepareQCVerifyError { + /// Inconsistent views. + #[error("inconsistent views of signed messages")] + InconsistentViews, + /// Invalid message. + #[error("msg[{0}]: {1:#}")] + InvalidMessage(usize, ReplicaPrepareVerifyError), + /// Bad message format. + #[error(transparent)] + BadFormat(anyhow::Error), + /// Not enough signers. + #[error("not enough signers: got {got}, want {want}")] + NotEnoughSigners { + /// Got signers. + got: usize, + /// Want signers. + want: usize, + }, + /// Bad signature. + #[error("bad signature: {0:#}")] + BadSignature(validator::Error), +} + +impl PrepareQC { + /// Create a new empty instance for a given `ReplicaCommit` message and a validator set size. + pub fn new(view: View) -> Self { + Self { + view, + map: BTreeMap::new(), + signature: validator::AggregateSignature::default(), + } + } + + /// Get the highest block voted and check if there's a quorum of votes for it. To have a quorum + /// in this situation, we require 2*f+1 votes, where f is the maximum number of faulty replicas. + pub fn high_vote(&self, genesis: &Genesis) -> Option { + let mut count: HashMap<_, usize> = HashMap::new(); + for (msg, signers) in &self.map { + if let Some(v) = &msg.high_vote { + *count.entry(v.proposal).or_default() += signers.count(); + } + } + // We only take one value from the iterator because there can only be at most one block with a quorum of 2f+1 votes. + let min = 2 * genesis.validators.faulty_replicas() + 1; + count.into_iter().find(|x| x.1 >= min).map(|x| x.0) + } + + /// Get the highest CommitQC. + pub fn high_qc(&self) -> Option<&CommitQC> { + self.map + .keys() + .filter_map(|m| m.high_qc.as_ref()) + .max_by_key(|qc| qc.view().number) + } + + /// Add a validator's signed message. + /// Message is assumed to be already verified. + // TODO: check if there is already a message from that validator. + // TODO: verify the message inside instead. + pub fn add(&mut self, msg: &Signed, genesis: &Genesis) { + if msg.msg.view != self.view { + return; + } + let Some(i) = genesis.validators.index(&msg.key) else { + return; + }; + let e = self + .map + .entry(msg.msg.clone()) + .or_insert_with(|| Signers::new(genesis.validators.len())); + if e.0[i] { + return; + }; + e.0.set(i, true); + self.signature.add(&msg.sig); + } + + /// Verifies the integrity of the PrepareQC. + pub fn verify(&self, genesis: &Genesis) -> Result<(), PrepareQCVerifyError> { + use PrepareQCVerifyError as Error; + let mut sum = Signers::new(genesis.validators.len()); + // Check the ReplicaPrepare messages. + for (i, (msg, signers)) in self.map.iter().enumerate() { + if msg.view != self.view { + return Err(Error::InconsistentViews); + } + if signers.len() != sum.len() { + return Err(Error::BadFormat(anyhow::format_err!( + "msg[{i}].signers has wrong length" + ))); + } + if signers.is_empty() { + return Err(Error::BadFormat(anyhow::format_err!( + "msg[{i}] has no signers assigned" + ))); + } + if !(&sum & signers).is_empty() { + return Err(Error::BadFormat(anyhow::format_err!( + "overlapping signature sets for different messages" + ))); + } + msg.verify(genesis) + .map_err(|err| Error::InvalidMessage(i, err))?; + sum |= signers; + } + + // Verify that we have enough signers. + let threshold = genesis.validators.threshold(); + if sum.count() < threshold { + return Err(Error::NotEnoughSigners { + got: sum.count(), + want: threshold, + }); + } + // Now we can verify the signature. + let messages_and_keys = self.map.clone().into_iter().flat_map(|(msg, signers)| { + genesis + .validators + .iter() + .enumerate() + .filter(|(i, _)| signers.0[*i]) + .map(|(_, pk)| (msg.clone(), pk)) + .collect::>() + }); + // TODO(gprusak): This reaggregating is suboptimal. + self.signature + .verify_messages(messages_and_keys) + .map_err(Error::BadSignature) + } +} + +/// A Prepare message from a leader. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct LeaderPrepare { + /// The header of the block that the leader is proposing. + pub proposal: BlockHeader, + /// Payload of the block that the leader is proposing. + /// `None` iff this is a reproposal. + pub proposal_payload: Option, + /// The PrepareQC that justifies this proposal from the leader. + pub justification: PrepareQC, +} + +/// Error returned by `LeaderPrepare::verify()`. +#[derive(thiserror::Error, Debug)] +pub enum LeaderPrepareVerifyError { + /// Justification + #[error("justification: {0:#}")] + Justification(PrepareQCVerifyError), + /// Bad block number. + #[error("bad block number: got {got:?}, want {want:?}")] + BadBlockNumber { + /// Correct proposal number. + want: BlockNumber, + /// Received proposal number. + got: BlockNumber, + }, + /// Bad parent hash. + #[error("bad parent hash: got {got:?}, want {want:?}")] + BadParentHash { + /// Correct parent hash. + want: Option, + /// Received parent hash. + got: Option, + }, + /// New block proposal when the previous proposal was not finalized. + #[error("new block proposal when the previous proposal was not finalized")] + ProposalWhenPreviousNotFinalized, + /// Mismatched payload. + #[error("block proposal with mismatched payload")] + ProposalMismatchedPayload, + /// Re-proposal without quorum. + #[error("block re-proposal without quorum for the re-proposal")] + ReproposalWithoutQuorum, + /// Re-proposal when the previous proposal was finalized. + #[error("block re-proposal when the previous proposal was finalized")] + ReproposalWhenFinalized, + /// Reproposed a bad block. + #[error("Reproposed a bad block")] + ReproposalBadBlock, +} + +impl LeaderPrepare { + /// View of the message. + pub fn view(&self) -> &View { + &self.justification.view + } + + /// Verifies LeaderPrepare. + pub fn verify(&self, genesis: &Genesis) -> Result<(), LeaderPrepareVerifyError> { + use LeaderPrepareVerifyError as Error; + self.justification + .verify(genesis) + .map_err(Error::Justification)?; + let high_vote = self.justification.high_vote(genesis); + let high_qc = self.justification.high_qc(); + + // Check that the proposal is valid. + match &self.proposal_payload { + // The leader proposed a new block. + Some(payload) => { + // Check that payload matches the header + if self.proposal.payload != payload.hash() { + return Err(Error::ProposalMismatchedPayload); + } + // Check that we finalized the previous block. + if high_vote.is_some() + && high_vote.as_ref() != high_qc.map(|qc| &qc.message.proposal) + { + return Err(Error::ProposalWhenPreviousNotFinalized); + } + let (want_parent, want_number) = match high_qc { + Some(qc) => (Some(qc.header().hash()), qc.header().number.next()), + None => (genesis.fork.first_parent, genesis.fork.first_block), + }; + if self.proposal.parent != want_parent { + return Err(Error::BadParentHash { + got: self.proposal.parent, + want: want_parent, + }); + } + if self.proposal.number != want_number { + return Err(Error::BadBlockNumber { + got: self.proposal.number, + want: want_number, + }); + } + } + None => { + let Some(high_vote) = &high_vote else { + return Err(Error::ReproposalWithoutQuorum); + }; + if let Some(high_qc) = &high_qc { + if high_vote.number == high_qc.header().number { + return Err(Error::ReproposalWhenFinalized); + } + } + if high_vote != &self.proposal { + return Err(Error::ReproposalBadBlock); + } + } + } + Ok(()) + } +} diff --git a/node/libs/roles/src/validator/messages/mod.rs b/node/libs/roles/src/validator/messages/mod.rs index 9f328870..bcb34689 100644 --- a/node/libs/roles/src/validator/messages/mod.rs +++ b/node/libs/roles/src/validator/messages/mod.rs @@ -3,9 +3,17 @@ mod block; mod consensus; mod discovery; +mod leader_commit; +mod leader_prepare; mod msg; +mod replica_commit; +mod replica_prepare; pub use block::*; pub use consensus::*; pub use discovery::*; +pub use leader_commit::*; +pub use leader_prepare::*; pub use msg::*; +pub use replica_commit::*; +pub use replica_prepare::*; diff --git a/node/libs/roles/src/validator/messages/replica_commit.rs b/node/libs/roles/src/validator/messages/replica_commit.rs new file mode 100644 index 00000000..963f8f39 --- /dev/null +++ b/node/libs/roles/src/validator/messages/replica_commit.rs @@ -0,0 +1,25 @@ +use super::{BlockHeader, Genesis, View}; + +/// A Commit message from a replica. +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct ReplicaCommit { + /// View of this message. + pub view: View, + /// The header of the block that the replica is committing to. + pub proposal: BlockHeader, +} + +impl ReplicaCommit { + /// Verifies the message. + pub fn verify(&self, genesis: &Genesis) -> anyhow::Result<()> { + anyhow::ensure!(self.view.fork == genesis.fork.number); + anyhow::ensure!(self.proposal.number >= genesis.fork.first_block); + if self.proposal.number == genesis.fork.first_block { + anyhow::ensure!( + self.proposal.parent == genesis.fork.first_parent, + "bad parent of the first block of the fork" + ); + } + Ok(()) + } +} diff --git a/node/libs/roles/src/validator/messages/replica_prepare.rs b/node/libs/roles/src/validator/messages/replica_prepare.rs new file mode 100644 index 00000000..830c1ac0 --- /dev/null +++ b/node/libs/roles/src/validator/messages/replica_prepare.rs @@ -0,0 +1,63 @@ +use super::{CommitQC, CommitQCVerifyError, ForkNumber, Genesis, ReplicaCommit, View}; + +/// A Prepare message from a replica. +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] +pub struct ReplicaPrepare { + /// View of this message. + pub view: View, + /// The highest block that the replica has committed to. + pub high_vote: Option, + /// The highest CommitQC that the replica has seen. + pub high_qc: Option, +} + +/// Error returned by `ReplicaPrepare::verify()`. +#[derive(thiserror::Error, Debug)] +pub enum ReplicaPrepareVerifyError { + /// BadFork. + #[error("bad fork: got {got:?}, want {want:?}")] + BadFork { + /// got + got: ForkNumber, + /// want + want: ForkNumber, + }, + /// FutureHighVoteView. + #[error("high vote from the future")] + HighVoteFutureView, + /// FutureHighQCView. + #[error("high qc from the future")] + HighQCFutureView, + /// HighVote. + #[error("high_vote: {0:#}")] + HighVote(anyhow::Error), + /// HighQC. + #[error("high_qc: {0:#}")] + HighQC(CommitQCVerifyError), +} + +impl ReplicaPrepare { + /// Verifies the message. + pub fn verify(&self, genesis: &Genesis) -> Result<(), ReplicaPrepareVerifyError> { + use ReplicaPrepareVerifyError as Error; + if self.view.fork != genesis.fork.number { + return Err(Error::BadFork { + got: self.view.fork, + want: genesis.fork.number, + }); + } + if let Some(v) = &self.high_vote { + if self.view.number <= v.view.number { + return Err(Error::HighVoteFutureView); + } + v.verify(genesis).map_err(Error::HighVote)?; + } + if let Some(qc) = &self.high_qc { + if self.view.number <= qc.view().number { + return Err(Error::HighQCFutureView); + } + qc.verify(genesis).map_err(Error::HighQC)?; + } + Ok(()) + } +} diff --git a/node/libs/roles/src/validator/mod.rs b/node/libs/roles/src/validator/mod.rs index 66ae9a12..2091a1ad 100644 --- a/node/libs/roles/src/validator/mod.rs +++ b/node/libs/roles/src/validator/mod.rs @@ -9,6 +9,3 @@ mod messages; pub mod testonly; pub use self::{keys::*, messages::*}; -// TODO(gprusak): it should be ok to have an unsigned -// genesis. For now we need a way to bootstrap the chain. -pub use testonly::GenesisSetup; diff --git a/node/libs/roles/src/validator/testonly.rs b/node/libs/roles/src/validator/testonly.rs index 57e24c98..27290f29 100644 --- a/node/libs/roles/src/validator/testonly.rs +++ b/node/libs/roles/src/validator/testonly.rs @@ -1,11 +1,10 @@ //! Test-only utilities. use super::{ AggregateSignature, BlockHeader, BlockHeaderHash, BlockNumber, CommitQC, ConsensusMsg, - FinalBlock, LeaderCommit, LeaderPrepare, Msg, MsgHash, NetAddress, Payload, PayloadHash, Phase, - PrepareQC, ProtocolVersion, PublicKey, ReplicaCommit, ReplicaPrepare, SecretKey, Signature, - Signed, Signers, ValidatorSet, ViewNumber, + FinalBlock, Fork, ForkNumber, Genesis, GenesisHash, LeaderCommit, LeaderPrepare, Msg, MsgHash, + NetAddress, Payload, PayloadHash, Phase, PrepareQC, ProtocolVersion, PublicKey, ReplicaCommit, + ReplicaPrepare, SecretKey, Signature, Signed, Signers, ValidatorSet, View, ViewNumber, }; -use anyhow::{bail, Context}; use bit_vec::BitVec; use rand::{ distributions::{Distribution, Standard}, @@ -15,114 +14,75 @@ use std::sync::Arc; use zksync_concurrency::time; use zksync_consensus_utils::enum_util::Variant; -/// Constructs a CommitQC with `CommitQC.message.proposal` matching header. -/// WARNING: it is not a fully correct CommitQC. -pub fn make_justification( - rng: &mut R, - header: &BlockHeader, - protocol_version: ProtocolVersion, -) -> CommitQC { - CommitQC { - message: ReplicaCommit { - protocol_version, - view: ViewNumber(header.number.0), - proposal: *header, - }, - signers: rng.gen(), - signature: rng.gen(), - } -} - -impl<'a> BlockBuilder<'a> { - /// Builds `GenesisSetup`. - pub fn push(self) { - let msgs: Vec<_> = self - .setup - .keys - .iter() - .map(|sk| sk.sign_msg(self.msg)) - .collect(); - let justification = CommitQC::from(&msgs, &self.setup.validator_set()).unwrap(); - self.setup.blocks.push(FinalBlock { - payload: self.payload, - justification, - }); - } - - /// Sets `protocol_version`. - pub fn protocol_version(mut self, protocol_version: ProtocolVersion) -> Self { - self.msg.protocol_version = protocol_version; - self - } - - /// Sets `block_number`. - pub fn block_number(mut self, block_number: BlockNumber) -> Self { - self.msg.proposal.number = block_number; - self - } - - /// Sets `payload`. - pub fn payload(mut self, payload: Payload) -> Self { - self.msg.proposal.payload = payload.hash(); - self.payload = payload; - self - } -} - -/// GenesisSetup. +/// Test setup. #[derive(Debug, Clone)] -pub struct GenesisSetup { - /// Validators' secret keys. - pub keys: Vec, - /// Initial blocks. - pub blocks: Vec, -} - -/// Builder of GenesisSetup. -pub struct BlockBuilder<'a> { - setup: &'a mut GenesisSetup, - msg: ReplicaCommit, - payload: Payload, -} - -impl GenesisSetup { - /// Constructs GenesisSetup with no blocks. - pub fn empty(rng: &mut impl Rng, validators: usize) -> Self { - Self { - keys: (0..validators).map(|_| rng.gen()).collect(), +pub struct Setup(SetupInner); + +impl Setup { + /// New `Setup` with a given `fork`. + pub fn new_with_fork(rng: &mut impl Rng, validators: usize, fork: Fork) -> Self { + let keys: Vec = (0..validators).map(|_| rng.gen()).collect(); + let genesis = Genesis { + validators: ValidatorSet::new(keys.iter().map(|k| k.public())).unwrap(), + fork, + }; + Self(SetupInner { + keys, + genesis, blocks: vec![], - } + }) } - /// Constructs GenesisSetup with genesis block. + /// New `Setup`. pub fn new(rng: &mut impl Rng, validators: usize) -> Self { - let mut this = Self::empty(rng, validators); - this.push_block(rng.gen()); - this - } - - /// Returns a builder for the next block. - pub fn next_block(&mut self) -> BlockBuilder { - let parent = self.blocks.last().map(|b| b.justification.message); - let payload = Payload(vec![]); - BlockBuilder { - setup: self, - msg: ReplicaCommit { - protocol_version: parent - .map(|m| m.protocol_version) - .unwrap_or(ProtocolVersion::EARLIEST), - view: parent.map(|m| m.view.next()).unwrap_or(ViewNumber(0)), - proposal: parent - .map(|m| BlockHeader::new(&m.proposal, payload.hash())) - .unwrap_or(BlockHeader::genesis(payload.hash(), BlockNumber(0))), - }, - payload, + let fork = Fork { + number: ForkNumber(rng.gen_range(0..100)), + first_block: BlockNumber(rng.gen_range(0..100)), + first_parent: Some(rng.gen()), + }; + Self::new_with_fork(rng, validators, fork) + } + + /// Next block to finalize. + pub fn next(&self) -> BlockNumber { + match self.0.blocks.last() { + Some(b) => b.header().number.next(), + None => self.0.genesis.fork.first_block, } } /// Pushes the next block with the given payload. pub fn push_block(&mut self, payload: Payload) { - self.next_block().payload(payload).push(); + let view = View { + protocol_version: ProtocolVersion::EARLIEST, + fork: self.genesis.fork.number, + number: self + .0 + .blocks + .last() + .map(|b| b.justification.view().number.next()) + .unwrap_or(ViewNumber(0)), + }; + let proposal = match self.0.blocks.last() { + Some(b) => BlockHeader::next(b.header(), payload.hash()), + None => BlockHeader { + parent: self.genesis.fork.first_parent, + number: self.genesis.fork.first_block, + payload: payload.hash(), + }, + }; + let msg = ReplicaCommit { view, proposal }; + let mut justification = CommitQC::new(msg, &self.0.genesis); + for key in &self.0.keys { + justification.add( + &key.sign_msg(justification.message.clone()), + &self.0.genesis, + ); + } + self.0.blocks.push(FinalBlock { + payload, + justification, + }); } /// Pushes `count` blocks with a random payload. @@ -132,36 +92,28 @@ impl GenesisSetup { } } - /// ValidatorSet. - pub fn validator_set(&self) -> ValidatorSet { - ValidatorSet::new(self.keys.iter().map(|k| k.public())).unwrap() + /// Finds the block by the number. + pub fn block(&self, n: BlockNumber) -> Option<&FinalBlock> { + let first = self.0.blocks.first()?.number(); + self.0.blocks.get(n.0.checked_sub(first.0)? as usize) } } -/// Constructs a genesis block with random payload. -pub fn make_genesis_block(rng: &mut impl Rng, protocol_version: ProtocolVersion) -> FinalBlock { - let mut setup = GenesisSetup::new(rng, 3); - setup - .next_block() - .protocol_version(protocol_version) - .payload(rng.gen()) - .push(); - setup.blocks[0].clone() +/// Setup. +#[derive(Debug, Clone)] +pub struct SetupInner { + /// Validators' secret keys. + pub keys: Vec, + /// Past blocks. + pub blocks: Vec, + /// Genesis config. + pub genesis: Genesis, } -/// Constructs a random block with a given parent. -/// WARNING: this is not a fully correct FinalBlock. -pub fn make_block( - rng: &mut R, - parent: &BlockHeader, - protocol_version: ProtocolVersion, -) -> FinalBlock { - let payload: Payload = rng.gen(); - let header = BlockHeader::new(parent, payload.hash()); - let justification = make_justification(rng, &header, protocol_version); - FinalBlock { - payload, - justification, +impl std::ops::Deref for Setup { + type Target = SetupInner; + fn deref(&self) -> &Self::Target { + &self.0 } } @@ -176,69 +128,6 @@ impl AggregateSignature { } } -impl PrepareQC { - /// Creates a new PrepareQC from a list of *signed* replica Prepare messages and the current validator set. - pub fn from( - signed_messages: &[Signed], - validators: &ValidatorSet, - ) -> anyhow::Result { - // Get the view number from the messages, they must all be equal. - let view = signed_messages - .first() - .context("Empty signed messages vector")? - .msg - .view; - - // Create the messages map. - let mut prepare_qc = PrepareQC::default(); - - for signed_message in signed_messages { - if signed_message.msg.view != view { - bail!("Signed messages aren't all for the same view."); - } - - // Get index of the validator in the validator set. - let index = validators - .index(&signed_message.key) - .context("Message signer isn't in the validator set")?; - - prepare_qc.add(signed_message, index, validators); - } - - Ok(prepare_qc) - } -} - -impl CommitQC { - /// Creates a new CommitQC from a list of *signed* replica Commit messages and the current validator set. - /// * `signed_messages` - A list of valid `ReplicaCommit` signed messages. Must contain at least one item. - /// * `validators` - The validator set. - pub fn from( - signed_messages: &[Signed], - validators: &ValidatorSet, - ) -> anyhow::Result { - // Store the signed messages in a Hashmap. - let message = signed_messages[0].msg; - let mut commit_qc = CommitQC::new(message, validators); - - for signed_message in signed_messages { - // Check that the votes are all for the same message. - if signed_message.msg != message { - bail!("CommitQC can only be created from votes for the same message."); - } - - // Get index of the validator in the validator set. - let validator_index = validators - .index(&signed_message.key) - .context("Message signer isn't in the validator set")?; - - commit_qc.add(&signed_message.sig, validator_index); - } - - Ok(commit_qc) - } -} - impl Distribution for Standard { fn sample(&self, rng: &mut R) -> AggregateSignature { AggregateSignature(rng.gen()) @@ -275,6 +164,37 @@ impl Distribution for Standard { } } +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> ForkNumber { + ForkNumber(rng.gen()) + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> GenesisHash { + GenesisHash(rng.gen()) + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> Fork { + Fork { + number: rng.gen(), + first_block: rng.gen(), + first_parent: Some(rng.gen()), + } + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> Genesis { + Genesis { + validators: rng.gen(), + fork: rng.gen(), + } + } +} + impl Distribution for Standard { fn sample(&self, rng: &mut R) -> PayloadHash { PayloadHash(rng.gen()) @@ -316,7 +236,6 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> ReplicaPrepare { ReplicaPrepare { - protocol_version: rng.gen(), view: rng.gen(), high_vote: rng.gen(), high_qc: rng.gen(), @@ -327,7 +246,6 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> ReplicaCommit { ReplicaCommit { - protocol_version: rng.gen(), view: rng.gen(), proposal: rng.gen(), } @@ -337,8 +255,6 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> LeaderPrepare { LeaderPrepare { - protocol_version: rng.gen(), - view: rng.gen(), proposal: rng.gen(), proposal_payload: rng.gen(), justification: rng.gen(), @@ -349,7 +265,6 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> LeaderCommit { LeaderCommit { - protocol_version: rng.gen(), justification: rng.gen(), } } @@ -361,6 +276,7 @@ impl Distribution for Standard { let map = (0..n).map(|_| (rng.gen(), rng.gen())).collect(); PrepareQC { + view: rng.gen(), map, signature: rng.gen(), } @@ -397,6 +313,16 @@ impl Distribution for Standard { } } +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> View { + View { + protocol_version: rng.gen(), + fork: rng.gen(), + number: rng.gen(), + } + } +} + impl Distribution for Standard { fn sample(&self, rng: &mut R) -> Phase { let i = rng.gen_range(0..2); diff --git a/node/libs/roles/src/validator/tests.rs b/node/libs/roles/src/validator/tests.rs index dc26f1cf..5f38b3d8 100644 --- a/node/libs/roles/src/validator/tests.rs +++ b/node/libs/roles/src/validator/tests.rs @@ -1,5 +1,7 @@ use super::*; -use rand::Rng; +use crate::validator::testonly::Setup; +use assert_matches::assert_matches; +use rand::{seq::SliceRandom, Rng}; use std::vec; use zksync_concurrency::ctx; use zksync_consensus_crypto::{ByteFmt, Text, TextFmt}; @@ -84,6 +86,10 @@ fn test_text_encoding() { let msg_hash: MsgHash = rng.gen(); let t = TextFmt::encode(&msg_hash); assert_eq!(msg_hash, Text::new(&t).decode::().unwrap()); + + let genesis_hash: GenesisHash = rng.gen(); + let t = TextFmt::encode(&genesis_hash); + assert_eq!(genesis_hash, Text::new(&t).decode::().unwrap()); } #[test] @@ -103,6 +109,9 @@ fn test_schema_encoding() { test_encode_random::(rng); test_encode_random::(rng); test_encode_random::(rng); + test_encode_random::(rng); + test_encode_random::(rng); + test_encode_random::(rng); } #[test] @@ -119,7 +128,7 @@ fn test_signature_verify() { let sig1 = key1.sign_hash(&msg1); // Matching key and message. - assert!(sig1.verify_hash(&msg1, &key1.public()).is_ok()); + sig1.verify_hash(&msg1, &key1.public()).unwrap(); // Mismatching message. assert!(sig1.verify_hash(&msg2, &key1.public()).is_err()); @@ -145,9 +154,9 @@ fn test_agg_signature_verify() { let agg_sig = AggregateSignature::aggregate(vec![&sig1, &sig2]); // Matching key and message. - assert!(agg_sig + agg_sig .verify_hash([(msg1, &key1.public()), (msg2, &key2.public())].into_iter()) - .is_ok()); + .unwrap(); // Mismatching message. assert!(agg_sig @@ -160,94 +169,114 @@ fn test_agg_signature_verify() { .is_err()); } +fn make_view(number: ViewNumber, setup: &Setup) -> View { + View { + protocol_version: ProtocolVersion::EARLIEST, + fork: setup.genesis.fork.number, + number, + } +} + +fn make_replica_commit(rng: &mut impl Rng, view: ViewNumber, setup: &Setup) -> ReplicaCommit { + ReplicaCommit { + view: make_view(view, setup), + proposal: rng.gen(), + } +} + +fn make_commit_qc(rng: &mut impl Rng, view: ViewNumber, setup: &Setup) -> CommitQC { + let mut qc = CommitQC::new(make_replica_commit(rng, view, setup), &setup.genesis); + for key in &setup.keys { + qc.add(&key.sign_msg(qc.message.clone()), &setup.genesis); + } + qc +} + +fn make_replica_prepare(rng: &mut impl Rng, view: ViewNumber, setup: &Setup) -> ReplicaPrepare { + ReplicaPrepare { + view: make_view(view, setup), + high_vote: { + let view = ViewNumber(rng.gen_range(0..view.0)); + Some(make_replica_commit(rng, view, setup)) + }, + high_qc: { + let view = ViewNumber(rng.gen_range(0..view.0)); + Some(make_commit_qc(rng, view, setup)) + }, + } +} + #[test] fn test_commit_qc() { + use CommitQCVerifyError as Error; let ctx = ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let sk1: SecretKey = rng.gen(); - let sk2: SecretKey = rng.gen(); - let sk3: SecretKey = rng.gen(); - - let msg: ReplicaCommit = rng.gen(); - - let validator_set1 = ValidatorSet::new(vec![ - sk1.public(), - sk2.public(), - sk3.public(), - rng.gen(), - rng.gen(), - ]) - .unwrap(); - let validator_set2 = - ValidatorSet::new(vec![rng.gen(), rng.gen(), rng.gen(), rng.gen(), rng.gen()]).unwrap(); - let validator_set3 = ValidatorSet::new(vec![sk1.public(), sk2.public(), sk3.public()]).unwrap(); - - let qc = CommitQC::from( - &[sk1.sign_msg(msg), sk2.sign_msg(msg), sk3.sign_msg(msg)], - &validator_set1, - ) - .unwrap(); - - // Matching validator set and enough signers. - assert!(qc.verify(&validator_set1, 1).is_ok()); - assert!(qc.verify(&validator_set1, 2).is_ok()); - assert!(qc.verify(&validator_set1, 3).is_ok()); - - // Not enough signers. - assert!(qc.verify(&validator_set1, 4).is_err()); - - // Mismatching validator sets. - assert!(qc.verify(&validator_set2, 3).is_err()); - assert!(qc.verify(&validator_set3, 3).is_err()); + let setup1 = Setup::new(rng, 6); + let setup2 = Setup::new(rng, 6); + let genesis3 = Genesis { + validators: ValidatorSet::new(setup1.genesis.validators.iter().take(3).cloned()).unwrap(), + fork: setup1.genesis.fork.clone(), + }; + + for i in 0..setup1.keys.len() + 1 { + let view = rng.gen(); + let mut qc = CommitQC::new(make_replica_commit(rng, view, &setup1), &setup1.genesis); + for key in &setup1.keys[0..i] { + qc.add(&key.sign_msg(qc.message.clone()), &setup1.genesis); + } + if i >= setup1.genesis.validators.threshold() { + qc.verify(&setup1.genesis).unwrap(); + } else { + assert_matches!( + qc.verify(&setup1.genesis), + Err(Error::NotEnoughSigners { .. }) + ); + } + + // Mismatching validator sets. + assert!(qc.verify(&setup2.genesis).is_err()); + assert!(qc.verify(&genesis3).is_err()); + } } #[test] fn test_prepare_qc() { + use PrepareQCVerifyError as Error; let ctx = ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let sk1: SecretKey = rng.gen(); - let sk2: SecretKey = rng.gen(); - let sk3: SecretKey = rng.gen(); + let setup1 = Setup::new(rng, 6); + let setup2 = Setup::new(rng, 6); + let genesis3 = Genesis { + validators: ValidatorSet::new(setup1.genesis.validators.iter().take(3).cloned()).unwrap(), + fork: setup1.genesis.fork.clone(), + }; let view: ViewNumber = rng.gen(); - let mut msg1: ReplicaPrepare = rng.gen(); - let mut msg2: ReplicaPrepare = rng.gen(); - msg1.view = view; - msg2.view = view; - - let validator_set1 = ValidatorSet::new(vec![ - sk1.public(), - sk2.public(), - sk3.public(), - rng.gen(), - rng.gen(), - ]) - .unwrap(); - let validator_set2 = - ValidatorSet::new(vec![rng.gen(), rng.gen(), rng.gen(), rng.gen(), rng.gen()]).unwrap(); - let validator_set3 = ValidatorSet::new(vec![sk1.public(), sk2.public(), sk3.public()]).unwrap(); - - let agg_qc = PrepareQC::from( - &[ - sk1.sign_msg(msg1.clone()), - sk2.sign_msg(msg2), - sk3.sign_msg(msg1), - ], - &validator_set1, - ) - .unwrap(); - - // Matching validator set and enough signers. - assert!(agg_qc.verify(view, &validator_set1, 1).is_ok()); - assert!(agg_qc.verify(view, &validator_set1, 2).is_ok()); - assert!(agg_qc.verify(view, &validator_set1, 3).is_ok()); - - // Not enough signers. - assert!(agg_qc.verify(view, &validator_set1, 4).is_err()); - - // Mismatching validator sets. - assert!(agg_qc.verify(view, &validator_set2, 3).is_err()); - assert!(agg_qc.verify(view, &validator_set3, 3).is_err()); + let msgs: Vec<_> = (0..3) + .map(|_| make_replica_prepare(rng, view, &setup1)) + .collect(); + + for n in 0..setup1.keys.len() + 1 { + let mut qc = PrepareQC::new(msgs[0].view.clone()); + for key in &setup1.keys[0..n] { + qc.add( + &key.sign_msg(msgs.choose(rng).unwrap().clone()), + &setup1.genesis, + ); + } + if n >= setup1.genesis.validators.threshold() { + qc.verify(&setup1.genesis).unwrap(); + } else { + assert_matches!( + qc.verify(&setup1.genesis), + Err(Error::NotEnoughSigners { .. }) + ); + } + + // Mismatching validator sets. + assert!(qc.verify(&setup2.genesis).is_err()); + assert!(qc.verify(&genesis3).is_err()); + } } diff --git a/node/libs/storage/src/block_store/metrics.rs b/node/libs/storage/src/block_store/metrics.rs index 32dae966..70447ad7 100644 --- a/node/libs/storage/src/block_store/metrics.rs +++ b/node/libs/storage/src/block_store/metrics.rs @@ -4,9 +4,12 @@ use std::time; #[derive(Debug, vise::Metrics)] #[metrics(prefix = "zksync_consensus_storage_persistent_block_store")] pub(super) struct PersistentBlockStore { - /// Latency of a successful `state()` call. + /// Latency of a successful `genesis()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) state_latency: vise::Histogram, + pub(super) genesis_latency: vise::Histogram, + /// Latency of a successful `last()` call. + #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] + pub(super) last_latency: vise::Histogram, /// Latency of a successful `block()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] pub(super) block_latency: vise::Histogram, @@ -21,8 +24,8 @@ pub(super) static PERSISTENT_BLOCK_STORE: vise::Global = v #[derive(Debug, vise::Metrics)] #[metrics(prefix = "zksync_consensus_storage_block_store")] pub(super) struct BlockStore { - /// BlockNumber of the last queued block. - pub(super) last_queued_block: vise::Gauge, - /// BlockNumber of the last persisted block. - pub(super) last_persisted_block: vise::Gauge, + /// BlockNumber of the next block to queue. + pub(super) next_queued_block: vise::Gauge, + /// BlockNumber of the next block to persist. + pub(super) next_persisted_block: vise::Gauge, } diff --git a/node/libs/storage/src/block_store/mod.rs b/node/libs/storage/src/block_store/mod.rs index f58c551d..456bc2e1 100644 --- a/node/libs/storage/src/block_store/mod.rs +++ b/node/libs/storage/src/block_store/mod.rs @@ -1,4 +1,5 @@ //! Defines storage layer for finalized blocks. +use anyhow::Context as _; use std::{collections::VecDeque, fmt, sync::Arc}; use zksync_concurrency::{ctx, error::Wrap as _, sync}; use zksync_consensus_roles::validator; @@ -9,21 +10,27 @@ mod metrics; #[derive(Debug, Clone, PartialEq, Eq)] pub struct BlockStoreState { /// Stored block with the lowest number. - pub first: validator::CommitQC, + /// Currently always same as `genesis.first_block`. + pub first: validator::BlockNumber, /// Stored block with the highest number. - pub last: validator::CommitQC, + /// None iff store is empty. + pub last: Option, } impl BlockStoreState { /// Checks whether block with the given number is stored in the `BlockStore`. pub fn contains(&self, number: validator::BlockNumber) -> bool { - self.first.header().number <= number && number <= self.last.header().number + let Some(last) = &self.last else { return false }; + self.first <= number && number <= last.header().number } /// Number of the next block that can be stored in the `BlockStore`. /// (i.e. `last` + 1). pub fn next(&self) -> validator::BlockNumber { - self.last.header().number.next() + match &self.last { + Some(qc) => qc.header().number.next(), + None => self.first, + } } } @@ -32,12 +39,14 @@ impl BlockStoreState { /// Implementations **must** propagate context cancellation using [`StorageError::Canceled`]. #[async_trait::async_trait] pub trait PersistentBlockStore: fmt::Debug + Send + Sync { - /// Range of blocks available in storage. - /// PersistentBlockStore is expected to always contain at least 1 block, - /// and be append-only storage (never delete blocks). + /// Genesis matching the block store content. + /// Consensus code calls this method only once. + async fn genesis(&self, ctx: &ctx::Ctx) -> ctx::Result; + + /// Last block available in storage. /// Consensus code calls this method only once and then tracks the /// range of available blocks internally. - async fn state(&self, ctx: &ctx::Ctx) -> ctx::Result; + async fn last(&self, ctx: &ctx::Ctx) -> ctx::Result>; /// Gets a block by its number. /// Returns error if block is missing. @@ -74,6 +83,7 @@ struct Inner { pub struct BlockStore { inner: sync::watch::Sender, persistent: Box, + genesis: validator::Genesis, } /// Runner of the BlockStore background tasks. @@ -110,7 +120,7 @@ impl BlockStoreRunner { self.0.inner.send_modify(|inner| { debug_assert_eq!(inner.persisted_state.next(), block.header().number); - inner.persisted_state.last = block.justification.clone(); + inner.persisted_state.last = Some(block.justification.clone()); inner.queue.pop_front(); }); } @@ -132,24 +142,43 @@ impl BlockStore { ctx: &ctx::Ctx, persistent: Box, ) -> ctx::Result<(Arc, BlockStoreRunner)> { - let t = metrics::PERSISTENT_BLOCK_STORE.state_latency.start(); - let state = persistent.state(ctx).await.wrap("persistent.state()")?; + let t = metrics::PERSISTENT_BLOCK_STORE.genesis_latency.start(); + let genesis = persistent.genesis(ctx).await.wrap("persistent.genesis()")?; + t.observe(); + let t = metrics::PERSISTENT_BLOCK_STORE.last_latency.start(); + let last = persistent.last(ctx).await.wrap("persistent.last()")?; t.observe(); - if state.first.header().number > state.last.header().number { - return Err(anyhow::anyhow!("invalid state").into()); + if let Some(last) = &last { + last.verify(&genesis).context("last.verify()")?; } + let state = BlockStoreState { + first: genesis.fork.first_block, + last, + }; let this = Arc::new(Self { - persistent, inner: sync::watch::channel(Inner { queued_state: sync::watch::channel(state.clone()).0, persisted_state: state, queue: VecDeque::new(), }) .0, + genesis, + persistent, }); + // Verify the first block. + if let Some(block) = this.block(ctx, this.genesis.fork.first_block).await? { + block + .verify(&this.genesis) + .with_context(|| format!("verify({:?})", this.genesis.fork.first_block))?; + } Ok((this.clone(), BlockStoreRunner(this))) } + /// Genesis specification for this block store. + pub fn genesis(&self) -> &validator::Genesis { + &self.genesis + } + /// Fetches a block (from queue or persistent storage). pub async fn block( &self, @@ -188,19 +217,35 @@ impl BlockStore { &self, ctx: &ctx::Ctx, block: validator::FinalBlock, - ) -> ctx::OrCanceled<()> { - let number = block.header().number; - sync::wait_for(ctx, &mut self.subscribe(), |queued_state| { - queued_state.next() >= number - }) - .await?; + ) -> ctx::Result<()> { + let number = block.number(); + { + let sub = &mut self.subscribe(); + let queued_state = + sync::wait_for(ctx, sub, |queued_state| queued_state.next() >= number).await?; + if queued_state.next() > number { + return Ok(()); + } + block.verify(&self.genesis).context("block.verify()")?; + // Verify parent hash, if previous block is available. + if let Some(last) = queued_state.last.as_ref() { + if Some(last.header().hash()) != block.header().parent { + return Err(anyhow::format_err!( + "block.parent = {:?}, want {:?}", + block.header().parent, + last.header().hash() + ) + .into()); + } + } + } self.inner.send_if_modified(|inner| { let modified = inner.queued_state.send_if_modified(|queued_state| { // It may happen that the same block is queued_state by 2 calls. if queued_state.next() != number { return false; } - queued_state.last = block.justification.clone(); + queued_state.last = Some(block.justification.clone()); true }); if !modified { @@ -212,14 +257,14 @@ impl BlockStore { Ok(()) } - /// Waits until the given block is queued_state to be stored. + /// Waits until the given block is queued to be stored. pub async fn wait_until_queued( &self, ctx: &ctx::Ctx, number: validator::BlockNumber, ) -> ctx::OrCanceled<()> { sync::wait_for(ctx, &mut self.subscribe(), |queued_state| { - queued_state.contains(number) + number < queued_state.next() }) .await?; Ok(()) @@ -232,7 +277,7 @@ impl BlockStore { number: validator::BlockNumber, ) -> ctx::OrCanceled<()> { sync::wait_for(ctx, &mut self.inner.subscribe(), |inner| { - inner.persisted_state.contains(number) + number < inner.persisted_state.next() }) .await?; Ok(()) @@ -247,10 +292,9 @@ impl BlockStore { fn scrape_metrics(&self) -> metrics::BlockStore { let m = metrics::BlockStore::default(); let inner = self.inner.borrow(); - m.last_queued_block - .set(inner.queued_state.borrow().last.header().number.0); - m.last_persisted_block - .set(inner.persisted_state.last.header().number.0); + m.next_queued_block + .set(inner.queued_state.borrow().next().0); + m.next_persisted_block.set(inner.persisted_state.next().0); m } } diff --git a/node/libs/storage/src/proto/mod.proto b/node/libs/storage/src/proto/mod.proto index 594ef1b0..e06e84da 100644 --- a/node/libs/storage/src/proto/mod.proto +++ b/node/libs/storage/src/proto/mod.proto @@ -5,14 +5,14 @@ package zksync.storage; import "zksync/roles/validator.proto"; message Proposal { - optional uint64 number = 1; - optional bytes payload = 2; + optional uint64 number = 1; // required; BlockNumber + optional bytes payload = 2; // required } message ReplicaState { - optional uint64 view = 1; - optional roles.validator.Phase phase = 2; - optional roles.validator.ReplicaCommit high_vote = 3; - optional roles.validator.CommitQC high_qc = 4; + optional uint64 view = 1; // required; ViewNumber + optional roles.validator.Phase phase = 2; // required + optional roles.validator.ReplicaCommit high_vote = 3; // optional + optional roles.validator.CommitQC high_qc = 4; // optional repeated Proposal proposals = 5; } diff --git a/node/libs/storage/src/replica_store.rs b/node/libs/storage/src/replica_store.rs index 6243cc72..0eaa96fd 100644 --- a/node/libs/storage/src/replica_store.rs +++ b/node/libs/storage/src/replica_store.rs @@ -4,7 +4,7 @@ use anyhow::Context as _; use std::fmt; use zksync_concurrency::ctx; use zksync_consensus_roles::validator; -use zksync_protobuf::{read_required, required, ProtoFmt}; +use zksync_protobuf::{read_optional, read_required, required, ProtoFmt}; /// Storage for [`ReplicaState`]. /// @@ -12,7 +12,7 @@ use zksync_protobuf::{read_required, required, ProtoFmt}; #[async_trait::async_trait] pub trait ReplicaStore: fmt::Debug + Send + Sync { /// Gets the replica state, if it is contained in the database. - async fn state(&self, ctx: &ctx::Ctx) -> ctx::Result>; + async fn state(&self, ctx: &ctx::Ctx) -> ctx::Result; /// Stores the given replica state into the database. async fn set_state(&self, ctx: &ctx::Ctx, state: &ReplicaState) -> ctx::Result<()>; @@ -39,20 +39,20 @@ pub struct ReplicaState { /// The current phase. pub phase: validator::Phase, /// The highest block proposal that the replica has committed to. - pub high_vote: validator::ReplicaCommit, + pub high_vote: Option, /// The highest commit quorum certificate known to the replica. - pub high_qc: validator::CommitQC, + pub high_qc: Option, /// A cache of the received block proposals. pub proposals: Vec, } -impl From for ReplicaState { - fn from(certificate: validator::CommitQC) -> Self { +impl Default for ReplicaState { + fn default() -> Self { Self { - view: certificate.message.view, + view: validator::ViewNumber(0), phase: validator::Phase::Prepare, - high_vote: certificate.message, - high_qc: certificate, + high_vote: None, + high_qc: None, proposals: vec![], } } @@ -83,8 +83,8 @@ impl ProtoFmt for ReplicaState { Ok(Self { view: validator::ViewNumber(r.view.context("view_number")?), phase: read_required(&r.phase).context("phase")?, - high_vote: read_required(&r.high_vote).context("high_vote")?, - high_qc: read_required(&r.high_qc).context("high_qc")?, + high_vote: read_optional(&r.high_vote).context("high_vote")?, + high_qc: read_optional(&r.high_qc).context("high_qc")?, proposals: r .proposals .iter() @@ -98,8 +98,8 @@ impl ProtoFmt for ReplicaState { Self::Proto { view: Some(self.view.0), phase: Some(self.phase.build()), - high_vote: Some(self.high_vote.build()), - high_qc: Some(self.high_qc.build()), + high_vote: self.high_vote.as_ref().map(|x| x.build()), + high_qc: self.high_qc.as_ref().map(|x| x.build()), proposals: self.proposals.iter().map(|p| p.build()).collect(), } } diff --git a/node/libs/storage/src/testonly/in_memory.rs b/node/libs/storage/src/testonly/in_memory.rs index d34f09cd..209d046c 100644 --- a/node/libs/storage/src/testonly/in_memory.rs +++ b/node/libs/storage/src/testonly/in_memory.rs @@ -1,36 +1,51 @@ //! In-memory storage implementation. -use crate::{BlockStoreState, PersistentBlockStore, ReplicaState}; +use crate::{PersistentBlockStore, ReplicaState}; use anyhow::Context as _; -use std::{collections::VecDeque, sync::Mutex}; +use std::{ + collections::VecDeque, + sync::{Arc, Mutex}, +}; use zksync_concurrency::ctx; use zksync_consensus_roles::validator; +#[derive(Debug)] +struct BlockStoreInner { + genesis: validator::Genesis, + blocks: Mutex>, +} + /// In-memory block store. -#[derive(Debug, Default)] -pub struct BlockStore(Mutex>); +#[derive(Clone, Debug)] +pub struct BlockStore(Arc); /// In-memory replica store. -#[derive(Debug, Default)] -pub struct ReplicaStore(Mutex>); +#[derive(Clone, Debug, Default)] +pub struct ReplicaStore(Arc>); impl BlockStore { - /// Creates a new store containing only the specified `genesis_block`. - pub fn new(genesis: validator::FinalBlock) -> Self { - Self(Mutex::new([genesis].into())) + /// New In-memory `BlockStore`. + pub fn new(genesis: validator::Genesis) -> Self { + Self(Arc::new(BlockStoreInner { + genesis, + blocks: Mutex::default(), + })) } } #[async_trait::async_trait] impl PersistentBlockStore for BlockStore { - async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result { - let blocks = self.0.lock().unwrap(); - if blocks.is_empty() { - return Err(anyhow::anyhow!("store is empty").into()); - } - Ok(BlockStoreState { - first: blocks.front().unwrap().justification.clone(), - last: blocks.back().unwrap().justification.clone(), - }) + async fn genesis(&self, _ctx: &ctx::Ctx) -> ctx::Result { + Ok(self.0.genesis.clone()) + } + + async fn last(&self, _ctx: &ctx::Ctx) -> ctx::Result> { + Ok(self + .0 + .blocks + .lock() + .unwrap() + .back() + .map(|b| b.justification.clone())) } async fn block( @@ -38,7 +53,7 @@ impl PersistentBlockStore for BlockStore { _ctx: &ctx::Ctx, number: validator::BlockNumber, ) -> ctx::Result { - let blocks = self.0.lock().unwrap(); + let blocks = self.0.blocks.lock().unwrap(); let front = blocks.front().context("not found")?; let idx = number .0 @@ -52,7 +67,7 @@ impl PersistentBlockStore for BlockStore { _ctx: &ctx::Ctx, block: &validator::FinalBlock, ) -> ctx::Result<()> { - let mut blocks = self.0.lock().unwrap(); + let mut blocks = self.0.blocks.lock().unwrap(); let got = block.header().number; if let Some(last) = blocks.back() { let want = last.header().number.next(); @@ -67,12 +82,12 @@ impl PersistentBlockStore for BlockStore { #[async_trait::async_trait] impl crate::ReplicaStore for ReplicaStore { - async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result> { + async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result { Ok(self.0.lock().unwrap().clone()) } async fn set_state(&self, _ctx: &ctx::Ctx, state: &ReplicaState) -> ctx::Result<()> { - *self.0.lock().unwrap() = Some(state.clone()); + *self.0.lock().unwrap() = state.clone(); Ok(()) } } diff --git a/node/libs/storage/src/testonly/mod.rs b/node/libs/storage/src/testonly/mod.rs index 5977f10e..3a44800b 100644 --- a/node/libs/storage/src/testonly/mod.rs +++ b/node/libs/storage/src/testonly/mod.rs @@ -1,5 +1,6 @@ //! Test-only utilities. use crate::{BlockStore, BlockStoreRunner, PersistentBlockStore, Proposal, ReplicaState}; +use anyhow::Context as _; use rand::{distributions::Standard, prelude::Distribution, Rng}; use std::sync::Arc; use zksync_concurrency::ctx; @@ -31,7 +32,7 @@ impl Distribution for Standard { /// Constructs a new in-memory store with a genesis block. pub async fn new_store( ctx: &ctx::Ctx, - genesis: &validator::FinalBlock, + genesis: &validator::Genesis, ) -> (Arc, BlockStoreRunner) { BlockStore::new(ctx, Box::new(in_memory::BlockStore::new(genesis.clone()))) .await @@ -40,14 +41,40 @@ pub async fn new_store( /// Dumps all the blocks stored in `store`. pub async fn dump(ctx: &ctx::Ctx, store: &dyn PersistentBlockStore) -> Vec { - let range = store.state(ctx).await.unwrap(); + let genesis = store.genesis(ctx).await.unwrap(); + let last = store.last(ctx).await.unwrap(); let mut blocks = vec![]; - for n in range.first.header().number.0..range.next().0 { - let n = validator::BlockNumber(n); + let begin = genesis.fork.first_block; + let end = last + .as_ref() + .map(|qc| qc.header().number.next()) + .unwrap_or(begin); + for n in (begin.0..end.0).map(validator::BlockNumber) { let block = store.block(ctx, n).await.unwrap(); assert_eq!(block.header().number, n); blocks.push(block); } - assert!(store.block(ctx, range.next()).await.is_err()); + assert!(store.block(ctx, end).await.is_err()); blocks } + +/// Verifies storage content. +pub async fn verify(ctx: &ctx::Ctx, store: &BlockStore) -> anyhow::Result<()> { + let range = store.subscribe().borrow().clone(); + let mut parent: Option = None; + for n in (range.first.0..range.next().0).map(validator::BlockNumber) { + async { + let block = store.block(ctx, n).await?.context("missing")?; + block.verify(store.genesis())?; + // Ignore checking the first block parent + if parent.is_some() { + anyhow::ensure!(parent == block.header().parent); + } + parent = Some(block.header().hash()); + Ok(()) + } + .await + .context(n)?; + } + Ok(()) +} diff --git a/node/libs/storage/src/tests.rs b/node/libs/storage/src/tests.rs index 723fd250..4a50c8a0 100644 --- a/node/libs/storage/src/tests.rs +++ b/node/libs/storage/src/tests.rs @@ -1,19 +1,20 @@ use super::*; use crate::{testonly::new_store, ReplicaState}; use zksync_concurrency::{ctx, scope, sync, testonly::abort_on_panic}; -use zksync_consensus_roles::validator; +use zksync_consensus_roles::validator::testonly::Setup; #[tokio::test] async fn test_inmemory_block_store() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let store = &testonly::in_memory::BlockStore::default(); - let mut setup = validator::testonly::GenesisSetup::empty(rng, 3); + let mut setup = Setup::new(rng, 3); setup.push_blocks(rng, 5); + + let store = &testonly::in_memory::BlockStore::new(setup.genesis.clone()); let mut want = vec![]; - for block in setup.blocks { - store.store_next_block(ctx, &block).await.unwrap(); - want.push(block); + for block in &setup.blocks { + store.store_next_block(ctx, block).await.unwrap(); + want.push(block.clone()); assert_eq!(want, testonly::dump(ctx, store).await); } } @@ -30,29 +31,27 @@ async fn test_state_updates() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut genesis = validator::testonly::GenesisSetup::new(rng, 1); - genesis.push_blocks(rng, 1); - - let (store, runner) = new_store(ctx, &genesis.blocks[0]).await; + let mut setup = Setup::new(rng, 1); + setup.push_blocks(rng, 1); + let (store, runner) = new_store(ctx, &setup.genesis).await; scope::run!(ctx, |ctx, s| async { s.spawn_bg(runner.run(ctx)); let sub = &mut store.subscribe(); let state = sub.borrow().clone(); - assert_eq!(state.first, genesis.blocks[0].justification); - assert_eq!(state.last, genesis.blocks[0].justification); + assert_eq!(state.first, setup.genesis.fork.first_block); + assert_eq!(state.last, None); store - .queue_block(ctx, genesis.blocks[1].clone()) + .queue_block(ctx, setup.blocks[0].clone()) .await .unwrap(); let state = sync::wait_for(ctx, sub, |state| { - state.last == genesis.blocks[1].justification + state.last.as_ref() == Some(&setup.blocks[0].justification) }) .await? .clone(); - assert_eq!(state.first, genesis.blocks[0].justification); - assert_eq!(state.last, genesis.blocks[1].justification); + assert_eq!(state.first, setup.blocks[0].header().number); Ok(()) }) .await diff --git a/node/libs/utils/src/lib.rs b/node/libs/utils/src/lib.rs index 26b0011e..31e43ba1 100644 --- a/node/libs/utils/src/lib.rs +++ b/node/libs/utils/src/lib.rs @@ -1,5 +1,4 @@ //! Crate that holds several small utilities and primitives. pub mod enum_util; -pub mod no_copy; pub mod pipe; diff --git a/node/tools/build.rs b/node/tools/build.rs index e4bba2bd..f4cfa5df 100644 --- a/node/tools/build.rs +++ b/node/tools/build.rs @@ -3,7 +3,7 @@ fn main() { zksync_protobuf_build::Config { input_root: "src/proto".into(), proto_root: "zksync/tools".into(), - dependencies: vec![], + dependencies: vec!["::zksync_consensus_roles::proto".parse().unwrap()], protobuf_crate: "::zksync_protobuf".parse().unwrap(), is_public: false, } diff --git a/node/tools/src/bin/localnet_config.rs b/node/tools/src/bin/localnet_config.rs index c50d9d0f..27b95f15 100644 --- a/node/tools/src/bin/localnet_config.rs +++ b/node/tools/src/bin/localnet_config.rs @@ -63,12 +63,8 @@ fn main() -> anyhow::Result<()> { // Generate the keys for all the replicas. let rng = &mut rand::thread_rng(); - let mut genesis = validator::GenesisSetup::empty(rng, addrs.len()); - genesis - .next_block() - .payload(validator::Payload(vec![])) - .push(); - let validator_keys = genesis.keys.clone(); + let setup = validator::testonly::Setup::new(rng, addrs.len()); + let validator_keys = setup.keys.clone(); let node_keys: Vec = (0..addrs.len()).map(|_| rng.gen()).collect(); // Each node will have `gossip_peers` outbound peers. @@ -81,8 +77,7 @@ fn main() -> anyhow::Result<()> { public_addr: addrs[i], metrics_server_addr, - validators: genesis.validator_set(), - genesis_block: genesis.blocks[0].clone(), + genesis: setup.genesis.clone(), max_payload_size: args.payload_size, gossip_dynamic_inbound_limit: 0, diff --git a/node/tools/src/config.rs b/node/tools/src/config.rs index 6673db93..9c9e5aff 100644 --- a/node/tools/src/config.rs +++ b/node/tools/src/config.rs @@ -11,8 +11,8 @@ use zksync_consensus_bft as bft; use zksync_consensus_crypto::{read_optional_text, read_required_text, Text, TextFmt}; use zksync_consensus_executor as executor; use zksync_consensus_roles::{node, validator}; -use zksync_consensus_storage::{BlockStore, BlockStoreRunner, PersistentBlockStore}; -use zksync_protobuf::{required, serde::Serde, ProtoFmt}; +use zksync_consensus_storage::{BlockStore, BlockStoreRunner}; +use zksync_protobuf::{read_required, required, serde::Serde, ProtoFmt}; /// Decodes a proto message from json for arbitrary ProtoFmt. pub fn decode_json(json: &str) -> anyhow::Result { @@ -54,8 +54,7 @@ pub struct AppConfig { pub public_addr: std::net::SocketAddr, pub metrics_server_addr: Option, - pub validators: validator::ValidatorSet, - pub genesis_block: validator::FinalBlock, + pub genesis: validator::Genesis, pub max_payload_size: usize, pub gossip_dynamic_inbound_limit: usize, @@ -67,14 +66,6 @@ impl ProtoFmt for AppConfig { type Proto = proto::AppConfig; fn read(r: &Self::Proto) -> anyhow::Result { - let validators = r.validators.iter().enumerate().map(|(i, v)| { - Text::new(v) - .decode() - .with_context(|| format!("validators[{i}]")) - }); - let validators: anyhow::Result> = validators.collect(); - let validators = validator::ValidatorSet::new(validators?).context("validators")?; - let mut gossip_static_inbound = HashSet::new(); for (i, v) in r.gossip_static_inbound.iter().enumerate() { gossip_static_inbound.insert( @@ -96,8 +87,7 @@ impl ProtoFmt for AppConfig { metrics_server_addr: read_optional_text(&r.metrics_server_addr) .context("metrics_server_addr")?, - validators, - genesis_block: read_required_text(&r.genesis_block).context("genesis_block")?, + genesis: read_required(&r.genesis).context("genesis")?, max_payload_size: required(&r.max_payload_size) .and_then(|x| Ok((*x).try_into()?)) .context("max_payload_size")?, @@ -116,8 +106,7 @@ impl ProtoFmt for AppConfig { public_addr: Some(self.public_addr.encode()), metrics_server_addr: self.metrics_server_addr.as_ref().map(TextFmt::encode), - validators: self.validators.iter().map(TextFmt::encode).collect(), - genesis_block: Some(self.genesis_block.encode()), + genesis: Some(self.genesis.build()), max_payload_size: Some(self.max_payload_size.try_into().unwrap()), gossip_dynamic_inbound_limit: Some( @@ -199,19 +188,12 @@ impl Configs { &self, ctx: &ctx::Ctx, ) -> ctx::Result<(executor::Executor, BlockStoreRunner)> { - let store = store::RocksDB::open(&self.database).await?; - // Store genesis if db is empty. - if store.is_empty().await? { - store - .store_next_block(ctx, &self.app.genesis_block) - .await - .context("store_next_block()")?; - } + let store = store::RocksDB::open(self.app.genesis.clone(), &self.database).await?; let (block_store, runner) = BlockStore::new(ctx, Box::new(store.clone())).await?; let e = executor::Executor { config: executor::Config { server_addr: self.app.server_addr, - validators: self.app.validators.clone(), + public_addr: self.app.public_addr, node_key: self.node_key.clone(), gossip_dynamic_inbound_limit: self.app.gossip_dynamic_inbound_limit, gossip_static_inbound: self.app.gossip_static_inbound.clone(), @@ -220,10 +202,7 @@ impl Configs { }, block_store, validator: self.validator_key.as_ref().map(|key| executor::Validator { - config: executor::ValidatorConfig { - key: key.clone(), - public_addr: self.app.public_addr, - }, + key: key.clone(), replica_store: Box::new(store), payload_manager: Box::new(bft::testonly::RandomPayload(self.app.max_payload_size)), }), diff --git a/node/tools/src/main.rs b/node/tools/src/main.rs index 675932ac..e1f7837f 100644 --- a/node/tools/src/main.rs +++ b/node/tools/src/main.rs @@ -8,7 +8,6 @@ use tracing_subscriber::{prelude::*, Registry}; use vise_exporter::MetricsExporter; use zksync_concurrency::{ctx, scope}; use zksync_consensus_tools::{decode_json, ConfigPaths, NodeAddr, RPCServer}; -use zksync_consensus_utils::no_copy::NoCopy; use zksync_protobuf::serde::Serde; /// Wrapper for Vec. @@ -125,10 +124,8 @@ async fn main() -> anyhow::Result<()> { // Initialize the storage. scope::run!(ctx, |ctx, s| async { - if let Some(addr) = configs.app.metrics_server_addr { - let addr = NoCopy::from(addr); + if let Some(addr) = &configs.app.metrics_server_addr { s.spawn_bg(async { - let addr = addr; MetricsExporter::default() .with_graceful_shutdown(ctx.canceled()) .start(*addr) diff --git a/node/tools/src/proto/mod.proto b/node/tools/src/proto/mod.proto index 3cac756d..5be829df 100644 --- a/node/tools/src/proto/mod.proto +++ b/node/tools/src/proto/mod.proto @@ -27,21 +27,16 @@ // NodePublicKey - public key of the node (gossip network participant) of the form "node:public::" // Currently only ed25519 signature scheme is supported for nodes. // example: "node:public:ed25519:d36607699a0a3fbe3de16947928cf299484219ff62ca20f387795b0859dbe501" -// -// FinalBlock - hex encoded serialized roles.validator.FinalBlock. -// Used to specify the genesis block of the chain. -// This blob of data is not customizable and we don't want to impose -// json-level backward compatibility on anything else but the configs. -// TODO(gprusak): either don't include it at all (derive genesis purely from -// the validator set) or move it to a separate config file. syntax = "proto3"; package zksync.tools; +import "zksync/roles/validator.proto"; + // (public key, ip address) of a gossip network node. message NodeAddr { - optional string key = 1; // [required] NodePublicKey - optional string addr = 2; // [required] IpAddr + optional string key = 1; // required; NodePublicKey + optional string addr = 2; // required; IpAddr } // Application configuration. @@ -50,36 +45,32 @@ message AppConfig { // IP:port to listen on, for incoming TCP connections. // Use `0.0.0.0:` to listen on all network interfaces (i.e. on all IPs exposed by this VM). - optional string server_addr = 1; // [required] IpAddr + optional string server_addr = 1; // required; IpAddr // Public IP:port to advertise, should forward to server_addr. - optional string public_addr = 2; // [required] IpAddr + optional string public_addr = 2; // required; IpAddr // IP:port to serve metrics data for scraping. // Use `0.0.0.0:` to listen on all network interfaces. // If not set, metrics data won't be served. - optional string metrics_server_addr = 3; // [optional] IpAddr + optional string metrics_server_addr = 3; // optional; IpAddr // Consensus - // Public keys of all validators. - repeated string validators = 5; // [required] ValidatorPublicKey - - // Genesis block of the blockchain. - // Will be inserted to storage if not already present. - optional string genesis_block = 6; // [required] FinalBlock + // Specification of the chain. + optional roles.validator.Genesis genesis = 4; // required // Maximal size of the block payload. - optional uint64 max_payload_size = 11; // [required] B + optional uint64 max_payload_size = 5; // required; bytes // Gossip network // Limit on the number of gossip network inbound connections outside // of the `gossip_static_inbound` set. - optional uint64 gossip_dynamic_inbound_limit = 8; // [required] + optional uint64 gossip_dynamic_inbound_limit = 6; // required // Inbound connections that should be unconditionally accepted on the gossip network. - repeated string gossip_static_inbound = 9; // NodePublicKey + repeated string gossip_static_inbound = 7; // NodePublicKey // Outbound gossip network connections that the node should actively try to // establish and maintain. - repeated NodeAddr gossip_static_outbound = 10; + repeated NodeAddr gossip_static_outbound = 8; } diff --git a/node/tools/src/rpc/methods/peers.rs b/node/tools/src/rpc/methods/peers.rs index 35f974d4..fcd50355 100644 --- a/node/tools/src/rpc/methods/peers.rs +++ b/node/tools/src/rpc/methods/peers.rs @@ -1,7 +1,6 @@ //! Peers method for RPC server. -use crate::{decode_json, AppConfig}; - use super::RPCMethod; +use crate::{decode_json, AppConfig}; use jsonrpsee::types::{error::ErrorCode, Params}; use std::fs::{self}; use zksync_consensus_crypto::TextFmt; diff --git a/node/tools/src/store.rs b/node/tools/src/store.rs index 189ee93e..f2832ae3 100644 --- a/node/tools/src/store.rs +++ b/node/tools/src/store.rs @@ -8,7 +8,7 @@ use std::{ }; use zksync_concurrency::{ctx, error::Wrap as _, scope}; use zksync_consensus_roles::validator; -use zksync_consensus_storage::{BlockStoreState, PersistentBlockStore, ReplicaState, ReplicaStore}; +use zksync_consensus_storage::{PersistentBlockStore, ReplicaState, ReplicaStore}; /// Enum used to represent a key in the database. It also acts as a separator between different stores. #[derive(Debug, Clone, PartialEq, Eq)] @@ -47,62 +47,51 @@ impl DatabaseKey { } } +struct Inner { + genesis: validator::Genesis, + db: RwLock, +} + /// Main struct for the Storage module, it just contains the database. Provides a set of high-level /// atomic operations on the database. It "contains" the following data: /// /// - An append-only database of finalized blocks. /// - A backup of the consensus replica state. #[derive(Clone)] -pub(crate) struct RocksDB(Arc>); +pub(crate) struct RocksDB(Arc); impl RocksDB { /// Create a new Storage. It first tries to open an existing database, and if that fails it just creates a /// a new one. We need the genesis block of the chain as input. - pub(crate) async fn open(path: &Path) -> ctx::Result { + pub(crate) async fn open(genesis: validator::Genesis, path: &Path) -> ctx::Result { let mut options = rocksdb::Options::default(); options.create_missing_column_families(true); options.create_if_missing(true); - Ok(Self(Arc::new(RwLock::new( - scope::wait_blocking(|| { - rocksdb::DB::open(&options, path).context("Failed opening RocksDB") - }) - .await?, - )))) + Ok(Self(Arc::new(Inner { + genesis, + db: RwLock::new( + scope::wait_blocking(|| { + rocksdb::DB::open(&options, path).context("Failed opening RocksDB") + }) + .await?, + ), + }))) } - fn state_blocking(&self) -> anyhow::Result> { - let db = self.0.read().unwrap(); - - let mut options = ReadOptions::default(); - options.set_iterate_range(DatabaseKey::BLOCKS_START_KEY..); - let Some(res) = db.iterator_opt(IteratorMode::Start, options).next() else { - return Ok(None); - }; - let (_, first) = res.context("RocksDB error reading first stored block")?; - let first: validator::FinalBlock = - zksync_protobuf::decode(&first).context("Failed decoding first stored block bytes")?; - + fn last_blocking(&self) -> anyhow::Result> { + let db = self.0.db.read().unwrap(); let mut options = ReadOptions::default(); options.set_iterate_range(DatabaseKey::BLOCKS_START_KEY..); - let (_, last) = db + let Some(res) = db .iterator_opt(DatabaseKey::BLOCK_HEAD_ITERATOR, options) .next() - .context("last block not found")? - .context("RocksDB error reading head block")?; + else { + return Ok(None); + }; + let (_, last) = res.context("RocksDB error reading head block")?; let last: validator::FinalBlock = zksync_protobuf::decode(&last).context("Failed decoding head block bytes")?; - - Ok(Some(BlockStoreState { - first: first.justification, - last: last.justification, - })) - } - - /// Checks if BlockStore is empty. - pub(crate) async fn is_empty(&self) -> anyhow::Result { - Ok(scope::wait_blocking(|| self.state_blocking()) - .await? - .is_none()) + Ok(Some(last.justification)) } } @@ -114,10 +103,12 @@ impl fmt::Debug for RocksDB { #[async_trait::async_trait] impl PersistentBlockStore for RocksDB { - async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result { - Ok(scope::wait_blocking(|| self.state_blocking()) - .await? - .context("storage is empty")?) + async fn genesis(&self, _ctx: &ctx::Ctx) -> ctx::Result { + Ok(self.0.genesis.clone()) + } + + async fn last(&self, _ctx: &ctx::Ctx) -> ctx::Result> { + Ok(scope::wait_blocking(|| self.last_blocking()).await?) } async fn block( @@ -126,7 +117,7 @@ impl PersistentBlockStore for RocksDB { number: validator::BlockNumber, ) -> ctx::Result { scope::wait_blocking(|| { - let db = self.0.read().unwrap(); + let db = self.0.db.read().unwrap(); let block = db .get(DatabaseKey::Block(number).encode_key()) .context("RocksDB error")? @@ -144,7 +135,7 @@ impl PersistentBlockStore for RocksDB { block: &validator::FinalBlock, ) -> ctx::Result<()> { scope::wait_blocking(|| { - let db = self.0.write().unwrap(); + let db = self.0.db.write().unwrap(); let block_number = block.header().number; let mut write_batch = rocksdb::WriteBatch::default(); write_batch.put( @@ -163,20 +154,19 @@ impl PersistentBlockStore for RocksDB { #[async_trait::async_trait] impl ReplicaStore for RocksDB { - async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result> { + async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result { Ok(scope::wait_blocking(|| { let Some(raw_state) = self .0 + .db .read() .unwrap() .get(DatabaseKey::ReplicaState.encode_key()) .context("Failed to get ReplicaState from RocksDB")? else { - return Ok(None); + return Ok(ReplicaState::default()); }; - zksync_protobuf::decode(&raw_state) - .map(Some) - .context("Failed to decode replica state!") + zksync_protobuf::decode(&raw_state).context("Failed to decode replica state!") }) .await?) } @@ -184,6 +174,7 @@ impl ReplicaStore for RocksDB { async fn set_state(&self, _ctx: &ctx::Ctx, state: &ReplicaState) -> ctx::Result<()> { Ok(scope::wait_blocking(|| { self.0 + .db .write() .unwrap() .put( diff --git a/node/tools/src/tests.rs b/node/tools/src/tests.rs index 581f1ca2..15d2f840 100644 --- a/node/tools/src/tests.rs +++ b/node/tools/src/tests.rs @@ -5,7 +5,7 @@ use rand::{ }; use tempfile::TempDir; use zksync_concurrency::ctx; -use zksync_consensus_roles::{node, validator::testonly::GenesisSetup}; +use zksync_consensus_roles::{node, validator::testonly::Setup}; use zksync_consensus_storage::{testonly, PersistentBlockStore}; use zksync_protobuf::testonly::test_encode_random; @@ -20,8 +20,7 @@ impl Distribution for Standard { public_addr: make_addr(rng), metrics_server_addr: Some(make_addr(rng)), - validators: rng.gen(), - genesis_block: rng.gen(), + genesis: rng.gen(), gossip_dynamic_inbound_limit: rng.gen(), gossip_static_inbound: (0..5) @@ -47,11 +46,13 @@ async fn test_reopen_rocksdb() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); let dir = TempDir::new().unwrap(); - let mut setup = GenesisSetup::empty(rng, 3); + let mut setup = Setup::new(rng, 3); setup.push_blocks(rng, 5); let mut want = vec![]; for b in &setup.blocks { - let store = store::RocksDB::open(dir.path()).await.unwrap(); + let store = store::RocksDB::open(setup.genesis.clone(), dir.path()) + .await + .unwrap(); store.store_next_block(ctx, b).await.unwrap(); want.push(b.clone()); assert_eq!(want, testonly::dump(ctx, &store).await);