From 00f621498a5dd6b862030afe5ba465a1ab8083e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bruno=20Fran=C3=A7a?= Date: Fri, 1 Nov 2024 19:29:40 +0000 Subject: [PATCH] feat: ChonkyBFT logic (#210) - Everything in chonky_bft folder is basically new. - UTHarness was moved to chonky_bft/testonly.rs. Several changes to it though. - All tests that use UTHarness were moved to chonky_bft/tests. Some of the old unit tests were repurposed, but a fair amount of the tests in chonky_bft/tests are new. - The tests in tests.rs were split between tests/mod.rs and tests/twins.rs (except for a few that used UTHarness and were moved as said before). There were no changes to them though. Part of BFT-452 --- .../bft/src/{replica => chonky_bft}/block.rs | 50 +- .../commit.rs} | 137 +-- node/actors/bft/src/chonky_bft/mod.rs | 314 ++++++ node/actors/bft/src/chonky_bft/new_view.rs | 175 ++++ node/actors/bft/src/chonky_bft/proposal.rs | 257 +++++ node/actors/bft/src/chonky_bft/proposer.rs | 104 ++ node/actors/bft/src/chonky_bft/testonly.rs | 306 ++++++ .../actors/bft/src/chonky_bft/tests/commit.rs | 426 ++++++++ node/actors/bft/src/chonky_bft/tests/mod.rs | 127 +++ .../bft/src/chonky_bft/tests/new_view.rs | 204 ++++ .../bft/src/chonky_bft/tests/proposal.rs | 364 +++++++ .../bft/src/chonky_bft/tests/timeout.rs | 439 +++++++++ node/actors/bft/src/chonky_bft/timeout.rs | 186 ++++ node/actors/bft/src/leader/mod.rs | 11 - node/actors/bft/src/leader/replica_prepare.rs | 146 --- node/actors/bft/src/leader/state_machine.rs | 270 ------ node/actors/bft/src/leader/tests.rs | 914 ------------------ node/actors/bft/src/lib.rs | 63 +- node/actors/bft/src/metrics.rs | 45 +- node/actors/bft/src/replica/leader_commit.rs | 100 -- node/actors/bft/src/replica/leader_prepare.rs | 189 ---- node/actors/bft/src/replica/mod.rs | 15 - node/actors/bft/src/replica/new_view.rs | 48 - .../actors/bft/src/replica/replica_prepare.rs | 104 -- node/actors/bft/src/replica/state_machine.rs | 255 ----- node/actors/bft/src/replica/tests.rs | 625 ------------ node/actors/bft/src/replica/timer.rs | 35 - node/actors/bft/src/testonly/make.rs | 16 +- node/actors/bft/src/testonly/mod.rs | 20 +- node/actors/bft/src/testonly/run.rs | 83 +- node/actors/bft/src/testonly/ut_harness.rs | 334 ------- node/actors/bft/src/tests.rs | 610 ------------ node/actors/bft/src/tests/mod.rs | 53 + node/actors/bft/src/tests/twins.rs | 265 +++++ node/actors/network/src/consensus/mod.rs | 17 +- node/actors/network/src/consensus/tests.rs | 11 +- node/actors/network/src/io.rs | 7 - node/actors/network/src/testonly.rs | 19 +- .../libs/roles/src/proto/validator/keys.proto | 6 +- .../roles/src/proto/validator/messages.proto | 5 +- node/libs/roles/src/validator/conv.rs | 2 - .../roles/src/validator/messages/block.rs | 10 + .../roles/src/validator/messages/committee.rs | 2 +- .../roles/src/validator/messages/consensus.rs | 3 +- .../src/validator/messages/leader_proposal.rs | 82 +- .../src/validator/messages/replica_commit.rs | 14 +- .../src/validator/messages/replica_timeout.rs | 12 + .../messages/tests/leader_proposal.rs | 68 -- .../roles/src/validator/messages/tests/mod.rs | 1 - .../messages/tests/replica_timeout.rs | 16 +- .../src/validator/messages/tests/versions.rs | 4 +- node/libs/roles/src/validator/testonly.rs | 5 +- node/libs/storage/src/proto/mod.proto | 2 + node/libs/storage/src/replica_store.rs | 13 +- node/libs/storage/src/testonly/mod.rs | 3 +- spec/README.md | 35 +- spec/informal-spec/README.md | 8 +- spec/informal-spec/replica.rs | 44 +- spec/informal-spec/types.rs | 2 +- spec/protocol-spec/README.md | 2 +- 60 files changed, 3560 insertions(+), 4123 deletions(-) rename node/actors/bft/src/{replica => chonky_bft}/block.rs (54%) rename node/actors/bft/src/{leader/replica_commit.rs => chonky_bft/commit.rs} (50%) create mode 100644 node/actors/bft/src/chonky_bft/mod.rs create mode 100644 node/actors/bft/src/chonky_bft/new_view.rs create mode 100644 node/actors/bft/src/chonky_bft/proposal.rs create mode 100644 node/actors/bft/src/chonky_bft/proposer.rs create mode 100644 node/actors/bft/src/chonky_bft/testonly.rs create mode 100644 node/actors/bft/src/chonky_bft/tests/commit.rs create mode 100644 node/actors/bft/src/chonky_bft/tests/mod.rs create mode 100644 node/actors/bft/src/chonky_bft/tests/new_view.rs create mode 100644 node/actors/bft/src/chonky_bft/tests/proposal.rs create mode 100644 node/actors/bft/src/chonky_bft/tests/timeout.rs create mode 100644 node/actors/bft/src/chonky_bft/timeout.rs delete mode 100644 node/actors/bft/src/leader/mod.rs delete mode 100644 node/actors/bft/src/leader/replica_prepare.rs delete mode 100644 node/actors/bft/src/leader/state_machine.rs delete mode 100644 node/actors/bft/src/leader/tests.rs delete mode 100644 node/actors/bft/src/replica/leader_commit.rs delete mode 100644 node/actors/bft/src/replica/leader_prepare.rs delete mode 100644 node/actors/bft/src/replica/mod.rs delete mode 100644 node/actors/bft/src/replica/new_view.rs delete mode 100644 node/actors/bft/src/replica/replica_prepare.rs delete mode 100644 node/actors/bft/src/replica/state_machine.rs delete mode 100644 node/actors/bft/src/replica/tests.rs delete mode 100644 node/actors/bft/src/replica/timer.rs delete mode 100644 node/actors/bft/src/testonly/ut_harness.rs delete mode 100644 node/actors/bft/src/tests.rs create mode 100644 node/actors/bft/src/tests/mod.rs create mode 100644 node/actors/bft/src/tests/twins.rs diff --git a/node/actors/bft/src/replica/block.rs b/node/actors/bft/src/chonky_bft/block.rs similarity index 54% rename from node/actors/bft/src/replica/block.rs rename to node/actors/bft/src/chonky_bft/block.rs index 2d6dc0a8..3ec22860 100644 --- a/node/actors/bft/src/replica/block.rs +++ b/node/actors/bft/src/chonky_bft/block.rs @@ -1,31 +1,17 @@ use super::StateMachine; -use zksync_concurrency::ctx; +use zksync_concurrency::{ctx, error::Wrap as _}; use zksync_consensus_roles::validator; +use zksync_consensus_storage as storage; impl StateMachine { /// Tries to build a finalized block from the given CommitQC. We simply search our /// block proposal cache for the matching block, and if we find it we build the block. - /// If this method succeeds, it sends the finalized block to the executor. - /// It also updates the High QC in the replica state machine, if the received QC is - /// higher. - #[tracing::instrument(level = "debug", skip_all)] + /// If this method succeeds, it saves the finalized block to storage. pub(crate) async fn save_block( &mut self, 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.header().number) else { return Ok(()); }; @@ -46,7 +32,11 @@ impl StateMachine { .block_store .queue_block(ctx, block.clone().into()) .await?; + // For availability, replica should not proceed until it stores the block persistently. + // Rationale is that after save_block, there is start_new_view which prunes the + // cache. Without persisting this block, if all replicas crash just after + // start_new_view, the payload becomes unavailable. self.config .block_store .wait_until_persisted(ctx, block.header().number) @@ -55,6 +45,32 @@ impl StateMachine { let number_metric = &crate::metrics::METRICS.finalized_block_number; let current_number = number_metric.get(); number_metric.set(current_number.max(block.header().number.0)); + + Ok(()) + } + + /// Backups the replica state to DB. + pub(crate) async fn backup_state(&self, ctx: &ctx::Ctx) -> ctx::Result<()> { + let mut proposals = vec![]; + for (number, payloads) in &self.block_proposal_cache { + proposals.extend(payloads.values().map(|p| storage::Proposal { + number: *number, + payload: p.clone(), + })); + } + let backup = storage::ReplicaState { + view: self.view_number, + phase: self.phase, + high_vote: self.high_vote.clone(), + high_commit_qc: self.high_commit_qc.clone(), + high_timeout_qc: self.high_timeout_qc.clone(), + proposals, + }; + self.config + .replica_store + .set_state(ctx, &backup) + .await + .wrap("set_state()")?; Ok(()) } } diff --git a/node/actors/bft/src/leader/replica_commit.rs b/node/actors/bft/src/chonky_bft/commit.rs similarity index 50% rename from node/actors/bft/src/leader/replica_commit.rs rename to node/actors/bft/src/chonky_bft/commit.rs index 08b57e16..f824f267 100644 --- a/node/actors/bft/src/leader/replica_commit.rs +++ b/node/actors/bft/src/chonky_bft/commit.rs @@ -1,43 +1,59 @@ -//! Handler of a ReplicaCommit message. - use super::StateMachine; use crate::metrics; use std::collections::HashSet; -use zksync_concurrency::{ctx, metrics::LatencyHistogramExt as _}; -use zksync_consensus_network::io::{ConsensusInputMessage, Target}; +use zksync_concurrency::{ctx, error::Wrap, metrics::LatencyHistogramExt as _}; use zksync_consensus_roles::validator; -/// Errors that can occur when processing a "replica commit" message. +/// Errors that can occur when processing a ReplicaCommit message. #[derive(Debug, thiserror::Error)] pub(crate) enum Error { /// Message signer isn't part of the validator set. - #[error("Message signer isn't part of the validator set (signer: {signer:?})")] + #[error("message signer isn't part of the validator set (signer: {signer:?})")] NonValidatorSigner { /// Signer of the message. signer: Box, }, - /// Past view or phase. - #[error("past view/phase (current view: {current_view:?}, current phase: {current_phase:?})")] + /// Past view. + #[error("past view (current view: {current_view:?})")] Old { /// Current view. current_view: validator::ViewNumber, - /// Current phase. - current_phase: validator::Phase, }, - /// 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(#[source] validator::ReplicaCommitVerifyError), + /// Duplicate signer. We already have a commit message from the same validator + /// for the same or past view. + #[error("duplicate signer (message view: {message_view:?}, signer: {signer:?})")] + DuplicateSigner { + /// View number of the message. + message_view: validator::ViewNumber, + /// Signer of the message. + signer: Box, + }, /// Invalid message signature. #[error("invalid signature: {0:#}")] InvalidSignature(#[source] anyhow::Error), + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] validator::ReplicaCommitVerifyError), + /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. + #[error(transparent)] + Internal(#[from] ctx::Error), +} + +impl Wrap for Error { + fn with_wrap C>( + self, + f: F, + ) -> Self { + match self { + Error::Internal(err) => Error::Internal(err.with_wrap(f)), + err => err, + } + } } impl StateMachine { - /// Processes `ReplicaCommit` message. - pub(crate) fn process_replica_commit( + /// Processes a ReplicaCommit message. + pub(crate) async fn on_commit( &mut self, ctx: &ctx::Ctx, signed_message: validator::Signed, @@ -55,25 +71,21 @@ impl StateMachine { }); } - // If the message is from the "past", we discard it. - // That is, it's from a previous view or phase, or if we already received a message - // from the same validator and for the same view. - if (message.view.number, validator::Phase::Commit) < (self.view, self.phase) - || self - .replica_commit_views - .get(author) - .is_some_and(|view_number| *view_number >= message.view.number) - { + // If the message is from a past view, ignore it. + if message.view.number < self.view_number { return Err(Error::Old { - current_view: self.view, - current_phase: self.phase, + current_view: self.view_number, }); } - // If the message is for a view when we are not a leader, we discard it. - if self.config.genesis().view_leader(message.view.number) != self.config.secret_key.public() - { - return Err(Error::NotLeaderInView); + // If we already have a message from the same validator for the same or past view, ignore it. + if let Some(&view) = self.commit_views_cache.get(author) { + if view >= message.view.number { + return Err(Error::DuplicateSigner { + message_view: message.view.number, + signer: author.clone().into(), + }); + } } // ----------- Checking the signed part of the message -------------- @@ -89,7 +101,7 @@ impl StateMachine { // We add the message to the incrementally-constructed QC. let commit_qc = self - .commit_qcs + .commit_qcs_cache .entry(message.view.number) .or_default() .entry(message.clone()) @@ -98,58 +110,51 @@ impl StateMachine { // Should always succeed as all checks have been already performed commit_qc .add(&signed_message, self.config.genesis()) - .expect("Could not add message to CommitQC"); + .expect("could not add message to CommitQC"); // Calculate the CommitQC signers weight. let weight = self.config.genesis().validators.weight(&commit_qc.signers); - // Update commit message current view number for author - self.replica_commit_views + // Update view number of last commit message for author + self.commit_views_cache .insert(author.clone(), message.view.number); // Clean up commit_qcs for the case that no replica is at the view - // of a given CommitQC + // of a given CommitQC. // This prevents commit_qcs map from growing indefinitely in case some - // malicious replica starts spamming messages for future views - let active_views: HashSet<_> = self.replica_commit_views.values().collect(); - self.commit_qcs + // malicious replica starts spamming messages for future views. + let active_views: HashSet<_> = self.commit_views_cache.values().collect(); + self.commit_qcs_cache .retain(|view_number, _| active_views.contains(view_number)); - // Now we check if we have enough weight to continue. + // Now we check if we have enough weight to continue. If not, we wait for more messages. if weight < self.config.genesis().validators.quorum_threshold() { return Ok(()); }; - // ----------- Update the state machine -------------- - let now = ctx.now(); - metrics::METRICS - .leader_commit_phase_latency - .observe_latency(now - self.phase_start); - self.view = message.view.number.next(); - self.phase = validator::Phase::Prepare; - self.phase_start = now; - - // ----------- Prepare our message and send it. -------------- + // ----------- We have a QC. Now we process it. -------------- - // Consume the incrementally-constructed QC for this view. - let justification = self - .commit_qcs + // Consume the created commit QC for this view. + let commit_qc = self + .commit_qcs_cache .remove(&message.view.number) .unwrap() .remove(message) .unwrap(); - // Broadcast the leader commit message to all replicas (ourselves included). - let output_message = ConsensusInputMessage { - message: self - .config - .secret_key - .sign_msg(validator::ConsensusMsg::LeaderCommit( - validator::LeaderCommit { justification }, - )), - recipient: Target::Broadcast, - }; - self.outbound_pipe.send(output_message.into()); + // We update our state with the new commit QC. + self.process_commit_qc(ctx, &commit_qc) + .await + .wrap("process_commit_qc()")?; + + // Metrics. We observe the latency of committing to a block measured + // from the start of this view. + metrics::METRICS + .commit_latency + .observe_latency(ctx.now() - self.view_start); + + // Start a new view. + self.start_new_view(ctx, message.view.number.next()).await?; Ok(()) } diff --git a/node/actors/bft/src/chonky_bft/mod.rs b/node/actors/bft/src/chonky_bft/mod.rs new file mode 100644 index 00000000..b5cf9c8e --- /dev/null +++ b/node/actors/bft/src/chonky_bft/mod.rs @@ -0,0 +1,314 @@ +use crate::{io::OutputMessage, metrics, Config}; +use std::{ + cmp::max, + collections::{BTreeMap, HashMap}, + sync::Arc, +}; +use zksync_concurrency::{ + ctx, + error::Wrap as _, + metrics::LatencyHistogramExt as _, + sync::{self, prunable_mpsc::SelectionFunctionResult}, + time, +}; +use zksync_consensus_network::io::ConsensusReq; +use zksync_consensus_roles::validator::{self, ConsensusMsg}; + +mod block; +mod commit; +mod new_view; +mod proposal; +/// The proposer module contains the logic for the proposer role in ChonkyBFT. +pub(crate) mod proposer; +#[cfg(test)] +pub(crate) mod testonly; +#[cfg(test)] +mod tests; +mod timeout; + +/// The duration of the view timeout. +pub(crate) const VIEW_TIMEOUT_DURATION: time::Duration = time::Duration::milliseconds(2000); + +/// The StateMachine struct contains the state of the replica and implements all the +/// logic of ChonkyBFT. +#[derive(Debug)] +pub(crate) struct StateMachine { + /// Consensus configuration. + pub(crate) config: Arc, + /// Pipe through which replica sends network messages. + pub(super) outbound_pipe: ctx::channel::UnboundedSender, + /// Pipe through which replica receives network requests. + pub(crate) inbound_pipe: sync::prunable_mpsc::Receiver, + /// The sender part of the proposer watch channel. This is used to notify the proposer loop + /// and send the needed justification. + pub(crate) proposer_pipe: sync::watch::Sender>, + + /// The current view number. + pub(crate) view_number: validator::ViewNumber, + /// The current phase. + pub(crate) phase: validator::Phase, + /// The highest block proposal that the replica has committed to. + pub(crate) high_vote: Option, + /// The highest commit quorum certificate known to the replica. + pub(crate) high_commit_qc: Option, + /// The highest timeout quorum certificate known to the replica. + pub(crate) high_timeout_qc: Option, + + /// A cache of the received block proposals. + pub(crate) block_proposal_cache: + BTreeMap>, + /// Latest view each validator has signed a ReplicaCommit message for. + pub(crate) commit_views_cache: BTreeMap, + /// Commit QCs indexed by view number and then by message. + pub(crate) commit_qcs_cache: + BTreeMap>, + /// Latest view each validator has signed a ReplicaTimeout message for. + pub(crate) timeout_views_cache: BTreeMap, + /// Timeout QCs indexed by view number. + pub(crate) timeout_qcs_cache: BTreeMap, + + /// The deadline to receive a proposal for this view before timing out. + pub(crate) view_timeout: time::Deadline, + /// Time when the current view phase has started. Used for metrics. + pub(crate) view_start: time::Instant, +} + +impl StateMachine { + /// Creates a new [`StateMachine`] instance, attempting to recover a past state from the storage module, + /// otherwise initializes the state machine with the current head block. + /// + /// Returns a tuple containing: + /// * The newly created [`StateMachine`] instance. + /// * A sender handle that should be used to send values to be processed by the instance, asynchronously. + pub(crate) async fn start( + ctx: &ctx::Ctx, + config: Arc, + outbound_pipe: ctx::channel::UnboundedSender, + proposer_pipe: sync::watch::Sender>, + ) -> ctx::Result<(Self, sync::prunable_mpsc::Sender)> { + 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 + .entry(proposal.number) + .or_default() + .insert(proposal.payload.hash(), proposal.payload); + } + + let (send, recv) = sync::prunable_mpsc::channel( + StateMachine::inbound_filter_predicate, + StateMachine::inbound_selection_function, + ); + + let this = Self { + config, + outbound_pipe, + inbound_pipe: recv, + proposer_pipe, + view_number: backup.view, + phase: backup.phase, + high_vote: backup.high_vote, + high_commit_qc: backup.high_commit_qc, + high_timeout_qc: backup.high_timeout_qc, + block_proposal_cache, + commit_views_cache: BTreeMap::new(), + commit_qcs_cache: BTreeMap::new(), + timeout_views_cache: BTreeMap::new(), + timeout_qcs_cache: BTreeMap::new(), + view_timeout: time::Deadline::Finite(ctx.now() + VIEW_TIMEOUT_DURATION), + view_start: ctx.now(), + }; + + Ok((this, send)) + } + + /// Runs a loop to process incoming messages (may be `None` if the channel times out while waiting for a message). + /// This is the main entry point for the state machine, + /// potentially triggering state modifications and message sending to the executor. + pub(crate) async fn run(mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { + self.view_start = ctx.now(); + + // If this is the first view, we immediately timeout. This will force the replicas + // to synchronize right at the beginning and will provide a justification for the + // next view. This is necessary because the first view is not justified by any + // previous view. + if self.view_number == validator::ViewNumber(0) { + self.start_timeout(ctx).await?; + } + + // Main loop. + loop { + let recv = self + .inbound_pipe + .recv(&ctx.with_deadline(self.view_timeout)) + .await; + + // Check for non-timeout cancellation. + if !ctx.is_active() { + return Ok(()); + } + + // Check for timeout. If we are already in a timeout phase, we don't + // timeout again. Note though that the underlying network implementation + // needs to keep retrying messages until they are delivered. Otherwise + // the consensus can halt! + if recv.is_err() && self.phase != validator::Phase::Timeout { + self.start_timeout(ctx).await?; + continue; + } + + // Process the message. + let req = recv.unwrap(); + let now = ctx.now(); + let label = match &req.msg.msg { + ConsensusMsg::LeaderProposal(_) => { + let res = match self + .on_proposal(ctx, req.msg.cast().unwrap()) + .await + .wrap("on_proposal()") + { + Ok(()) => Ok(()), + Err(err) => { + match err { + // If the error is internal, we stop here. + proposal::Error::Internal(err) => { + tracing::error!("on_proposal: internal error: {err:#}"); + return Err(err); + } + // If the error is due to an old message, we log it at a lower level. + proposal::Error::Old { .. } => { + tracing::debug!("on_proposal: {err:#}"); + } + _ => { + tracing::warn!("on_proposal: {err:#}"); + } + } + Err(()) + } + }; + metrics::ConsensusMsgLabel::LeaderProposal.with_result(&res) + } + ConsensusMsg::ReplicaCommit(_) => { + let res = match self + .on_commit(ctx, req.msg.cast().unwrap()) + .await + .wrap("on_commit()") + { + Ok(()) => Ok(()), + Err(err) => { + match err { + // If the error is internal, we stop here. + commit::Error::Internal(err) => { + tracing::error!("on_commit: internal error: {err:#}"); + return Err(err); + } + // If the error is due to an old message, we log it at a lower level. + commit::Error::Old { .. } => { + tracing::debug!("on_commit: {err:#}"); + } + _ => { + tracing::warn!("on_commit: {err:#}"); + } + } + Err(()) + } + }; + metrics::ConsensusMsgLabel::ReplicaCommit.with_result(&res) + } + ConsensusMsg::ReplicaTimeout(_) => { + let res = match self + .on_timeout(ctx, req.msg.cast().unwrap()) + .await + .wrap("on_timeout()") + { + Ok(()) => Ok(()), + Err(err) => { + match err { + // If the error is internal, we stop here. + timeout::Error::Internal(err) => { + tracing::error!("on_timeout: internal error: {err:#}"); + return Err(err); + } + // If the error is due to an old message, we log it at a lower level. + timeout::Error::Old { .. } => { + tracing::debug!("on_timeout: {err:#}"); + } + _ => { + tracing::warn!("on_timeout: {err:#}"); + } + } + Err(()) + } + }; + metrics::ConsensusMsgLabel::ReplicaTimeout.with_result(&res) + } + ConsensusMsg::ReplicaNewView(_) => { + let res = match self + .on_new_view(ctx, req.msg.cast().unwrap()) + .await + .wrap("on_new_view()") + { + Ok(()) => Ok(()), + Err(err) => { + match err { + // If the error is internal, we stop here. + new_view::Error::Internal(err) => { + tracing::error!("on_new_view: internal error: {err:#}"); + return Err(err); + } + // If the error is due to an old message, we log it at a lower level. + new_view::Error::Old { .. } => { + tracing::debug!("on_new_view: {err:#}"); + } + _ => { + tracing::warn!("on_new_view: {err:#}"); + } + } + Err(()) + } + }; + metrics::ConsensusMsgLabel::ReplicaNewView.with_result(&res) + } + }; + metrics::METRICS.message_processing_latency[&label].observe_latency(ctx.now() - now); + + // Notify network actor that the message has been processed. + // Ignore sending error. + let _ = req.ack.send(()); + } + } + + fn inbound_filter_predicate(new_req: &ConsensusReq) -> bool { + // Verify message signature + new_req.msg.verify().is_ok() + } + + fn inbound_selection_function( + old_req: &ConsensusReq, + new_req: &ConsensusReq, + ) -> SelectionFunctionResult { + if old_req.msg.key != new_req.msg.key || old_req.msg.msg.label() != new_req.msg.msg.label() + { + SelectionFunctionResult::Keep + } else { + // Discard older message + if old_req.msg.msg.view().number < new_req.msg.msg.view().number { + SelectionFunctionResult::DiscardOld + } else { + SelectionFunctionResult::DiscardNew + } + } + } + + /// Processes a (already verified) CommitQC. It bumps the local high_commit_qc and if + /// we have the proposal corresponding to this qc, we save the corresponding block to DB. + pub(crate) async fn process_commit_qc( + &mut self, + ctx: &ctx::Ctx, + qc: &validator::CommitQC, + ) -> ctx::Result<()> { + self.high_commit_qc = max(Some(qc.clone()), self.high_commit_qc.clone()); + self.save_block(ctx, qc).await.wrap("save_block()") + } +} diff --git a/node/actors/bft/src/chonky_bft/new_view.rs b/node/actors/bft/src/chonky_bft/new_view.rs new file mode 100644 index 00000000..ff401ebb --- /dev/null +++ b/node/actors/bft/src/chonky_bft/new_view.rs @@ -0,0 +1,175 @@ +use std::cmp::max; + +use super::StateMachine; +use crate::{chonky_bft::VIEW_TIMEOUT_DURATION, metrics}; +use zksync_concurrency::{ctx, error::Wrap, metrics::LatencyHistogramExt as _, time}; +use zksync_consensus_network::io::ConsensusInputMessage; +use zksync_consensus_roles::validator; + +/// Errors that can occur when processing a ReplicaNewView message. +#[derive(Debug, thiserror::Error)] +pub(crate) enum Error { + /// Message signer isn't part of the validator set. + #[error("message signer isn't part of the validator set (signer: {signer:?})")] + NonValidatorSigner { + /// Signer of the message. + signer: Box, + }, + /// Past view or phase. + #[error("past view (current view: {current_view:?})")] + Old { + /// Current view. + current_view: validator::ViewNumber, + }, + /// Invalid message signature. + #[error("invalid signature: {0:#}")] + InvalidSignature(#[source] anyhow::Error), + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] validator::ReplicaNewViewVerifyError), + /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. + #[error(transparent)] + Internal(#[from] ctx::Error), +} + +impl Wrap for Error { + fn with_wrap C>( + self, + f: F, + ) -> Self { + match self { + Error::Internal(err) => Error::Internal(err.with_wrap(f)), + err => err, + } + } +} + +impl StateMachine { + /// Processes a ReplicaNewView message. + pub(crate) async fn on_new_view( + &mut self, + ctx: &ctx::Ctx, + signed_message: validator::Signed, + ) -> Result<(), Error> { + // ----------- Checking origin of the message -------------- + + // Unwrap message. + let message = &signed_message.msg; + let author = &signed_message.key; + + // Check that the message signer is in the validator committee. + if !self.config.genesis().validators.contains(author) { + return Err(Error::NonValidatorSigner { + signer: author.clone().into(), + }); + } + + // If the message is from a past view, ignore it. + if message.view().number < self.view_number { + return Err(Error::Old { + current_view: self.view_number, + }); + } + + // ----------- Checking the signed part of the message -------------- + + // 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. -------------- + + // Update the state machine. + match &message.justification { + validator::ProposalJustification::Commit(qc) => self + .process_commit_qc(ctx, qc) + .await + .wrap("process_commit_qc()")?, + validator::ProposalJustification::Timeout(qc) => { + if let Some(high_qc) = qc.high_qc() { + self.process_commit_qc(ctx, high_qc) + .await + .wrap("process_commit_qc()")?; + } + self.high_timeout_qc = max(Some(qc.clone()), self.high_timeout_qc.clone()); + } + }; + + // If the message is for a future view, we need to start a new view. + if message.view().number > self.view_number { + self.start_new_view(ctx, message.view().number).await?; + } + + Ok(()) + } + + /// This blocking method is used whenever we start a new view. + pub(crate) async fn start_new_view( + &mut self, + ctx: &ctx::Ctx, + view: validator::ViewNumber, + ) -> ctx::Result<()> { + // Update the state machine. + self.view_number = view; + self.phase = validator::Phase::Prepare; + self.proposer_pipe + .send(Some(self.get_justification())) + .expect("justification_watch.send() failed"); + + // Clear the block proposal cache. + if let Some(qc) = self.high_commit_qc.as_ref() { + self.block_proposal_cache + .retain(|k, _| k > &qc.header().number); + } + + // Backup our state. + self.backup_state(ctx).await.wrap("backup_state()")?; + + // Broadcast our new view message. + let output_message = ConsensusInputMessage { + message: self + .config + .secret_key + .sign_msg(validator::ConsensusMsg::ReplicaNewView( + validator::ReplicaNewView { + justification: self.get_justification(), + }, + )), + }; + self.outbound_pipe.send(output_message.into()); + + // Log the event and update the metrics. + tracing::info!("Starting view {}", self.view_number); + metrics::METRICS.replica_view_number.set(self.view_number.0); + let now = ctx.now(); + metrics::METRICS + .view_latency + .observe_latency(now - self.view_start); + self.view_start = now; + + // Reset the timeout. + self.view_timeout = time::Deadline::Finite(ctx.now() + VIEW_TIMEOUT_DURATION); + + Ok(()) + } + + /// Makes a justification (for a ReplicaNewView or a LeaderProposal) based on the current state. + pub(crate) fn get_justification(&self) -> validator::ProposalJustification { + // We need some QC in order to be able to create a justification. + // In fact, it should be impossible to get here without a QC. Because + // we only get here after starting a new view, which requires a QC. + assert!(self.high_commit_qc.is_some() || self.high_timeout_qc.is_some()); + + // We use the highest QC as the justification. If both have the same view, we use the CommitQC. + if self.high_commit_qc.as_ref().map(|x| x.view()) + >= self.high_timeout_qc.as_ref().map(|x| &x.view) + { + validator::ProposalJustification::Commit(self.high_commit_qc.clone().unwrap()) + } else { + validator::ProposalJustification::Timeout(self.high_timeout_qc.clone().unwrap()) + } + } +} diff --git a/node/actors/bft/src/chonky_bft/proposal.rs b/node/actors/bft/src/chonky_bft/proposal.rs new file mode 100644 index 00000000..b89f09d8 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/proposal.rs @@ -0,0 +1,257 @@ +use crate::metrics; + +use super::StateMachine; +use std::cmp::max; +use zksync_concurrency::{ctx, error::Wrap, metrics::LatencyHistogramExt as _}; +use zksync_consensus_network::io::ConsensusInputMessage; +use zksync_consensus_roles::validator::{self, BlockHeader, BlockNumber}; + +/// Errors that can occur when processing a LeaderProposal message. +#[derive(Debug, thiserror::Error)] +pub(crate) enum Error { + /// Message for a past view or phase. + #[error( + "message for a past view / phase (current view: {current_view:?}, current phase: {current_phase:?})" + )] + Old { + /// Current view. + current_view: validator::ViewNumber, + /// Current phase. + current_phase: validator::Phase, + }, + /// Invalid leader. + #[error( + "invalid leader (correct leader: {correct_leader:?}, received leader: {received_leader:?})" + )] + InvalidLeader { + /// Correct leader. + correct_leader: validator::PublicKey, + /// Received leader. + received_leader: validator::PublicKey, + }, + /// Invalid message signature. + #[error("invalid signature: {0:#}")] + InvalidSignature(#[source] anyhow::Error), + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] validator::LeaderProposalVerifyError), + /// Leader proposed a block that was already pruned from replica's storage. + #[error("leader proposed a block that was already pruned from replica's storage")] + ProposalAlreadyPruned, + /// Reproposal with an unnecessary payload. + #[error("reproposal with an unnecessary payload")] + ReproposalWithPayload, + /// Block proposal payload missing. + #[error("block proposal payload missing")] + MissingPayload, + /// Oversized payload. + #[error("block proposal with an oversized payload (payload size: {payload_size})")] + ProposalOversizedPayload { + /// Size of the payload. + payload_size: usize, + }, + /// Previous payload missing. + #[error("previous block proposal payload missing from store (block number: {prev_number})")] + MissingPreviousPayload { + /// The number of the missing block + prev_number: BlockNumber, + }, + /// Invalid payload. + #[error("invalid payload: {0:#}")] + InvalidPayload(#[source] anyhow::Error), + /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. + #[error(transparent)] + Internal(#[from] ctx::Error), +} + +impl Wrap for Error { + fn with_wrap C>( + self, + f: F, + ) -> Self { + match self { + Error::Internal(err) => Error::Internal(err.with_wrap(f)), + err => err, + } + } +} + +impl StateMachine { + /// Processes a LeaderProposal message. + pub(crate) async fn on_proposal( + &mut self, + ctx: &ctx::Ctx, + signed_message: validator::Signed, + ) -> Result<(), Error> { + // ----------- Checking origin of the message -------------- + + // Unwrap message. + let message = &signed_message.msg; + let author = &signed_message.key; + let view = message.view().number; + + // Check that the message is for the current view or a future view. We only allow proposals for + // the current view if we have not voted or timed out yet. + if view < self.view_number + || (view == self.view_number && self.phase != validator::Phase::Prepare) + { + return Err(Error::Old { + current_view: self.view_number, + current_phase: self.phase, + }); + } + + // Check that it comes from the correct leader. + let leader = self.config.genesis().view_leader(view); + if author != &leader { + return Err(Error::InvalidLeader { + correct_leader: leader, + received_leader: author.clone(), + }); + } + + // ----------- Checking the message -------------- + + signed_message.verify().map_err(Error::InvalidSignature)?; + + message + .verify(self.config.genesis()) + .map_err(Error::InvalidMessage)?; + + let (implied_block_number, implied_block_hash) = message + .justification + .get_implied_block(self.config.genesis()); + + // Replica MUSTN'T vote for blocks which have been already pruned for storage. + // (because it won't be able to persist and broadcast them once finalized). + // TODO(gprusak): it should never happen, we should add safety checks to prevent + // pruning blocks not known to be finalized. + if implied_block_number < self.config.block_store.queued().first { + return Err(Error::ProposalAlreadyPruned); + } + + let block_hash = match implied_block_hash { + // This is a reproposal. + // We let the leader repropose blocks without sending them in the proposal + // (it sends only the block number + block hash). That allows a leader to + // repropose a block without having it stored. Sending reproposals without + // a payload is an optimization that allows us to not wait for a leader that + // has the previous proposal stored (which can take 4f views), and to somewhat + // speed up reproposals by skipping block broadcast. + // This only saves time because we have a gossip network running in parallel, + // and any time a replica is able to create a finalized block (by possessing + // both the block and the commit QC) it broadcasts the finalized block (this + // was meant to propagate the block to full nodes, but of course validators + // will end up receiving it as well). + Some(hash) => { + // We check that the leader didn't send a payload with the reproposal. + // This isn't technically needed for the consensus to work (it will remain + // safe and live), but it's a good practice to avoid unnecessary data in + // blockchain. + // This unnecessary payload would also effectively be a source of free + // data availability, which the leaders would be incentivized to abuse. + if message.proposal_payload.is_some() { + return Err(Error::ReproposalWithPayload); + }; + + hash + } + // This is a new proposal, so we need to verify it (i.e. execute it). + None => { + // Check that the payload is present. + let Some(ref payload) = message.proposal_payload else { + return Err(Error::MissingPayload); + }; + + if payload.len() > self.config.max_payload_size { + return Err(Error::ProposalOversizedPayload { + payload_size: payload.len(), + }); + } + + // Defensively assume that PayloadManager cannot verify proposal until the previous block is stored. + // Note that it doesn't mean that the block is actually available, as old blocks might get pruned or + // we might just have started from a snapshot state. It just means that we have the state of the chain + // up to the previous block. + if let Some(prev) = implied_block_number.prev() { + self.config + .block_store + .wait_until_persisted(&ctx.with_deadline(self.view_timeout), prev) + .await + .map_err(|_| Error::MissingPreviousPayload { prev_number: prev })?; + } + + // Execute the payload. + if let Err(err) = self + .config + .payload_manager + .verify(ctx, implied_block_number, payload) + .await + { + return Err(match err { + ctx::Error::Internal(err) => Error::InvalidPayload(err), + err @ ctx::Error::Canceled(_) => Error::Internal(err), + }); + } + + // The proposal is valid. We cache it, waiting for it to be committed. + self.block_proposal_cache + .entry(implied_block_number) + .or_default() + .insert(payload.hash(), payload.clone()); + + payload.hash() + } + }; + + // ----------- All checks finished. Now we process the message. -------------- + + // Metrics. We observe the latency of receiving a proposal measured + // from the start of this view. + metrics::METRICS + .proposal_latency + .observe_latency(ctx.now() - self.view_start); + + // Create our commit vote. + let commit_vote = validator::ReplicaCommit { + view: message.view(), + proposal: BlockHeader { + number: implied_block_number, + payload: block_hash, + }, + }; + + // Update the state machine. + self.view_number = message.view().number; + self.phase = validator::Phase::Commit; + self.high_vote = Some(commit_vote.clone()); + match &message.justification { + validator::ProposalJustification::Commit(qc) => self + .process_commit_qc(ctx, qc) + .await + .wrap("process_commit_qc()")?, + validator::ProposalJustification::Timeout(qc) => { + if let Some(high_qc) = qc.high_qc() { + self.process_commit_qc(ctx, high_qc) + .await + .wrap("process_commit_qc()")?; + } + self.high_timeout_qc = max(Some(qc.clone()), self.high_timeout_qc.clone()); + } + }; + + // Backup our state. + self.backup_state(ctx).await.wrap("backup_state()")?; + + // Broadcast our commit message. + let output_message = ConsensusInputMessage { + message: self + .config + .secret_key + .sign_msg(validator::ConsensusMsg::ReplicaCommit(commit_vote)), + }; + self.outbound_pipe.send(output_message.into()); + + Ok(()) + } +} diff --git a/node/actors/bft/src/chonky_bft/proposer.rs b/node/actors/bft/src/chonky_bft/proposer.rs new file mode 100644 index 00000000..4a6dd843 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/proposer.rs @@ -0,0 +1,104 @@ +use crate::{io::OutputMessage, metrics, Config}; +use std::sync::Arc; +use zksync_concurrency::{ctx, error::Wrap as _, sync}; +use zksync_consensus_network::io::ConsensusInputMessage; +use zksync_consensus_roles::validator; + +use super::VIEW_TIMEOUT_DURATION; + +/// The proposer loop is responsible for proposing new blocks to the network. It watches for new +/// justifications from the replica and if it is the leader for the view, it proposes a new block. +pub(crate) async fn run_proposer( + ctx: &ctx::Ctx, + cfg: Arc, + outbound_pipe: ctx::channel::UnboundedSender, + mut justification_watch: sync::watch::Receiver>, +) -> ctx::Result<()> { + loop { + // Wait for a new justification to be available. + let Some(justification) = sync::changed(ctx, &mut justification_watch).await?.clone() + else { + continue; + }; + + // If we are not the leader for this view, skip it. + if cfg.genesis().view_leader(justification.view().number) != cfg.secret_key.public() { + continue; + } + + // Create a proposal for the given justification, within the timeout. + let proposal = match create_proposal( + &ctx.with_timeout(VIEW_TIMEOUT_DURATION), + cfg.clone(), + justification, + ) + .await + { + Ok(proposal) => proposal, + Err(ctx::Error::Canceled(_)) => { + tracing::error!("run_proposer(): timed out while creating a proposal"); + continue; + } + Err(ctx::Error::Internal(err)) => { + tracing::error!("run_proposer(): internal error: {err:#}"); + return Err(ctx::Error::Internal(err)); + } + }; + + // Broadcast our proposal to all replicas (ourselves included). + let msg = cfg + .secret_key + .sign_msg(validator::ConsensusMsg::LeaderProposal(proposal)); + + outbound_pipe.send(ConsensusInputMessage { message: msg }.into()); + } +} + +/// Creates a proposal for the given justification. +pub(crate) async fn create_proposal( + ctx: &ctx::Ctx, + cfg: Arc, + justification: validator::ProposalJustification, +) -> ctx::Result { + // Get the block number and check if this must be a reproposal. + let (block_number, opt_block_hash) = justification.get_implied_block(cfg.genesis()); + + let proposal_payload = match opt_block_hash { + // There was some proposal last view that a subquorum of replicas + // voted for and could have been finalized. We need to repropose it. + Some(_) => None, + // The previous proposal was finalized, so we can propose a new block. + None => { + // Defensively assume that PayloadManager cannot propose until the previous block is stored. + if let Some(prev) = block_number.prev() { + cfg.block_store.wait_until_persisted(ctx, prev).await?; + } + + let payload = cfg + .payload_manager + .propose(ctx, block_number) + .await + .wrap("payload_manager.propose()")?; + + if payload.0.len() > cfg.max_payload_size { + return Err(anyhow::format_err!( + "proposed payload too large: got {}B, max {}B", + payload.0.len(), + cfg.max_payload_size + ) + .into()); + } + + metrics::METRICS + .proposal_payload_size + .observe(payload.0.len()); + + Some(payload) + } + }; + + Ok(validator::LeaderProposal { + proposal_payload, + justification, + }) +} diff --git a/node/actors/bft/src/chonky_bft/testonly.rs b/node/actors/bft/src/chonky_bft/testonly.rs new file mode 100644 index 00000000..2367d1e7 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/testonly.rs @@ -0,0 +1,306 @@ +use crate::testonly::RandomPayload; +use crate::{ + chonky_bft::{self, commit, new_view, proposal, timeout, StateMachine}, + io::OutputMessage, + Config, PayloadManager, +}; +use assert_matches::assert_matches; +use std::sync::Arc; +use zksync_concurrency::sync::prunable_mpsc; +use zksync_concurrency::{ctx, sync}; +use zksync_consensus_network as network; +use zksync_consensus_network::io::ConsensusReq; +use zksync_consensus_roles::validator; +use zksync_consensus_storage::{ + testonly::{in_memory, TestMemoryStorage}, + BlockStoreRunner, +}; +use zksync_consensus_utils::enum_util::Variant; + +pub(crate) const MAX_PAYLOAD_SIZE: usize = 1000; + +/// `UTHarness` provides various utilities for unit tests. +/// It is designed to simplify the setup and execution of test cases by encapsulating +/// common testing functionality. +/// +/// It should be instantiated once for every test case. +#[cfg(test)] +pub(crate) struct UTHarness { + pub(crate) replica: StateMachine, + pub(crate) keys: Vec, + pub(crate) outbound_pipe: ctx::channel::UnboundedReceiver, + pub(crate) inbound_pipe: prunable_mpsc::Sender, + pub(crate) _proposer_pipe: sync::watch::Receiver>, +} + +impl UTHarness { + /// Creates a new `UTHarness` with the specified validator set size. + pub(crate) async fn new( + ctx: &ctx::Ctx, + num_validators: usize, + ) -> (UTHarness, BlockStoreRunner) { + Self::new_with_payload_manager( + ctx, + num_validators, + Box::new(RandomPayload(MAX_PAYLOAD_SIZE)), + ) + .await + } + + /// 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; + let (util, runner) = UTHarness::new(ctx, num_validators).await; + assert!(util.genesis().validators.max_faulty_weight() > 0); + (util, runner) + } + + pub(crate) async fn new_with_payload_manager( + ctx: &ctx::Ctx, + num_validators: usize, + payload_manager: Box, + ) -> (UTHarness, BlockStoreRunner) { + let rng = &mut ctx.rng(); + let setup = validator::testonly::Setup::new(rng, num_validators); + let store = TestMemoryStorage::new(ctx, &setup).await; + let (send, recv) = ctx::channel::unbounded(); + let (proposer_sender, proposer_receiver) = sync::watch::channel(None); + + let cfg = Arc::new(Config { + secret_key: setup.validator_keys[0].clone(), + block_store: store.blocks.clone(), + replica_store: Box::new(in_memory::ReplicaStore::default()), + payload_manager, + max_payload_size: MAX_PAYLOAD_SIZE, + }); + let (replica, input_pipe) = + StateMachine::start(ctx, cfg.clone(), send.clone(), proposer_sender) + .await + .unwrap(); + let mut this = UTHarness { + replica, + keys: setup.validator_keys.clone(), + outbound_pipe: recv, + inbound_pipe: input_pipe, + _proposer_pipe: proposer_receiver, + }; + + let timeout = this.new_replica_timeout(ctx).await; + this.process_replica_timeout_all(ctx, timeout).await; + + (this, store.runner) + } + + pub(crate) fn owner_key(&self) -> &validator::SecretKey { + &self.replica.config.secret_key + } + + pub(crate) fn leader_key(&self) -> validator::SecretKey { + let leader = self.view_leader(self.replica.view_number); + self.keys + .iter() + .find(|key| key.public() == leader) + .unwrap() + .clone() + } + + pub(crate) fn view(&self) -> validator::View { + validator::View { + genesis: self.genesis().hash(), + number: self.replica.view_number, + } + } + + pub(crate) fn view_leader(&self, view: validator::ViewNumber) -> validator::PublicKey { + self.genesis().view_leader(view) + } + + pub(crate) fn genesis(&self) -> &validator::Genesis { + self.replica.config.genesis() + } + + pub(crate) async fn new_leader_proposal(&self, ctx: &ctx::Ctx) -> validator::LeaderProposal { + let justification = self.replica.get_justification(); + chonky_bft::proposer::create_proposal(ctx, self.replica.config.clone(), justification) + .await + .unwrap() + } + + pub(crate) async fn new_replica_commit(&mut self, ctx: &ctx::Ctx) -> validator::ReplicaCommit { + let proposal = self.new_leader_proposal(ctx).await; + self.process_leader_proposal(ctx, self.leader_key().sign_msg(proposal)) + .await + .unwrap() + .msg + } + + pub(crate) async fn new_replica_timeout( + &mut self, + ctx: &ctx::Ctx, + ) -> validator::ReplicaTimeout { + self.replica.start_timeout(ctx).await.unwrap(); + self.try_recv().unwrap().msg + } + + pub(crate) async fn new_replica_new_view(&self) -> validator::ReplicaNewView { + let justification = self.replica.get_justification(); + validator::ReplicaNewView { justification } + } + + pub(crate) async fn new_commit_qc( + &mut self, + ctx: &ctx::Ctx, + mutate_fn: impl FnOnce(&mut validator::ReplicaCommit), + ) -> validator::CommitQC { + let mut msg = self.new_replica_commit(ctx).await; + mutate_fn(&mut msg); + let mut qc = validator::CommitQC::new(msg.clone(), self.genesis()); + for key in &self.keys { + qc.add(&key.sign_msg(msg.clone()), self.genesis()).unwrap(); + } + qc + } + + // #[allow(dead_code)] + // pub(crate) fn new_timeout_qc( + // &mut self, + // mutate_fn: impl FnOnce(&mut validator::ReplicaTimeout), + // ) -> validator::TimeoutQC { + // let mut msg = self.new_replica_timeout(); + // mutate_fn(&mut msg); + // let mut qc = validator::TimeoutQC::new(msg.view.clone()); + // for key in &self.keys { + // qc.add(&key.sign_msg(msg.clone()), self.genesis()).unwrap(); + // } + // qc + // } + + pub(crate) async fn process_leader_proposal( + &mut self, + ctx: &ctx::Ctx, + msg: validator::Signed, + ) -> Result, proposal::Error> { + self.replica.on_proposal(ctx, msg).await?; + Ok(self.try_recv().unwrap()) + } + + pub(crate) async fn process_replica_commit( + &mut self, + ctx: &ctx::Ctx, + msg: validator::Signed, + ) -> Result>, commit::Error> { + self.replica.on_commit(ctx, msg).await?; + Ok(self.try_recv()) + } + + pub(crate) async fn process_replica_timeout( + &mut self, + ctx: &ctx::Ctx, + msg: validator::Signed, + ) -> Result>, timeout::Error> { + self.replica.on_timeout(ctx, msg).await?; + Ok(self.try_recv()) + } + + pub(crate) async fn process_replica_new_view( + &mut self, + ctx: &ctx::Ctx, + msg: validator::Signed, + ) -> Result>, new_view::Error> { + self.replica.on_new_view(ctx, msg).await?; + Ok(self.try_recv()) + } + + pub(crate) async fn process_replica_commit_all( + &mut self, + ctx: &ctx::Ctx, + msg: validator::ReplicaCommit, + ) -> validator::Signed { + let mut threshold_reached = false; + let mut cur_weight = 0; + + for key in self.keys.iter() { + let res = self.replica.on_commit(ctx, key.sign_msg(msg.clone())).await; + let val_index = self.genesis().validators.index(&key.public()).unwrap(); + + cur_weight += self.genesis().validators.get(val_index).unwrap().weight; + + if threshold_reached { + assert_matches!(res, Err(commit::Error::Old { .. })); + } else { + res.unwrap(); + if cur_weight >= self.genesis().validators.quorum_threshold() { + threshold_reached = true; + } + } + } + + self.try_recv().unwrap() + } + + pub(crate) async fn process_replica_timeout_all( + &mut self, + ctx: &ctx::Ctx, + msg: validator::ReplicaTimeout, + ) -> validator::Signed { + let mut threshold_reached = false; + let mut cur_weight = 0; + + for key in self.keys.iter() { + let res = self + .replica + .on_timeout(ctx, key.sign_msg(msg.clone())) + .await; + let val_index = self.genesis().validators.index(&key.public()).unwrap(); + + cur_weight += self.genesis().validators.get(val_index).unwrap().weight; + + if threshold_reached { + assert_matches!(res, Err(timeout::Error::Old { .. })); + } else { + res.unwrap(); + if cur_weight >= self.genesis().validators.quorum_threshold() { + threshold_reached = true; + } + } + } + + self.try_recv().unwrap() + } + + /// Produces a block, by executing the full view. + pub(crate) async fn produce_block(&mut self, ctx: &ctx::Ctx) { + let replica_commit = self.new_replica_commit(ctx).await; + self.process_replica_commit_all(ctx, replica_commit).await; + } + + /// Triggers replica timeout, processes the new validator::ReplicaTimeout + /// to start a new view, then executes the whole new view to make sure + /// that the consensus recovers after a timeout. + pub(crate) async fn produce_block_after_timeout(&mut self, ctx: &ctx::Ctx) { + let cur_view = self.replica.view_number; + + self.replica.start_timeout(ctx).await.unwrap(); + let replica_timeout = self.try_recv().unwrap().msg; + self.process_replica_timeout_all(ctx, replica_timeout).await; + + assert_eq!(self.replica.view_number, cur_view.next()); + + self.produce_block(ctx).await; + } + + pub(crate) fn send(&self, msg: validator::Signed) { + self.inbound_pipe.send(ConsensusReq { + msg, + ack: zksync_concurrency::oneshot::channel().0, + }); + } + + fn try_recv>(&mut self) -> Option> { + self.outbound_pipe.try_recv().map(|message| match message { + OutputMessage::Network(network::io::ConsensusInputMessage { message, .. }) => { + message.cast().unwrap() + } + }) + } +} diff --git a/node/actors/bft/src/chonky_bft/tests/commit.rs b/node/actors/bft/src/chonky_bft/tests/commit.rs new file mode 100644 index 00000000..0d65d110 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/tests/commit.rs @@ -0,0 +1,426 @@ +use crate::chonky_bft::{commit, testonly::UTHarness}; +use assert_matches::assert_matches; +use pretty_assertions::assert_eq; +use rand::Rng; +use zksync_concurrency::{ctx, scope}; +use zksync_consensus_roles::validator; + +#[tokio::test] +async fn commit_yield_new_view_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 cur_view = util.replica.view_number; + let replica_commit = util.new_replica_commit(ctx).await; + assert_eq!(util.replica.phase, validator::Phase::Commit); + + let new_view = util + .process_replica_commit_all(ctx, replica_commit.clone()) + .await + .msg; + assert_eq!(util.replica.view_number, cur_view.next()); + assert_eq!(util.replica.phase, validator::Phase::Prepare); + assert_eq!(new_view.view().number, cur_view.next()); + assert_matches!(new_view.justification, validator::ProposalJustification::Commit(qc) => { + assert_eq!(qc.message.proposal, replica_commit.proposal); + }); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn commit_non_validator_signer() { + 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_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(commit::Error::NonValidatorSigner { signer }) => { + assert_eq!(*signer, non_validator_key.public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_commit_old() { + 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 replica_commit = util.new_replica_commit(ctx).await; + replica_commit.view.number = validator::ViewNumber(util.replica.view_number.0 - 1); + let res = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit)) + .await; + + assert_matches!( + res, + Err(commit::Error::Old { current_view }) => { + assert_eq!(current_view, util.replica.view_number); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn commit_duplicate_signer() { + 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 mut replica_commit = util.new_replica_commit(ctx).await; + assert!(util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit.clone())) + .await + .unwrap() + .is_none()); + + // Processing twice same ReplicaCommit for same view gets DuplicateSigner error + let res = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit.clone())) + .await; + assert_matches!( + res, + Err(commit::Error::DuplicateSigner { + message_view, + signer + })=> { + assert_eq!(message_view, util.replica.view_number); + assert_eq!(*signer, util.owner_key().public()); + } + ); + + // Processing twice different ReplicaCommit for same view gets DuplicateSigner error too + replica_commit.proposal.number = replica_commit.proposal.number.next(); + let res = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit.clone())) + .await; + assert_matches!( + res, + Err(commit::Error::DuplicateSigner { + message_view, + signer + })=> { + assert_eq!(message_view, util.replica.view_number); + assert_eq!(*signer, util.owner_key().public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn commit_invalid_sig() { + 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 msg = util.new_replica_commit(ctx).await; + let mut replica_commit = util.owner_key().sign_msg(msg); + replica_commit.sig = ctx.rng().gen(); + + let res = util.process_replica_commit(ctx, replica_commit).await; + assert_matches!(res, Err(commit::Error::InvalidSignature(..))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn commit_invalid_message() { + 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 replica_commit = util.new_replica_commit(ctx).await; + replica_commit.view.genesis = ctx.rng().gen(); + + let res = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit)) + .await; + assert_matches!(res, Err(commit::Error::InvalidMessage(_))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_commit_num_received_below_threshold() { + 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 replica_commit = util.new_replica_commit(ctx).await; + for i in 0..util.genesis().validators.quorum_threshold() as usize - 1 { + assert!(util + .process_replica_commit(ctx, util.keys[i].sign_msg(replica_commit.clone())) + .await + .unwrap() + .is_none()); + } + let res = util + .process_replica_commit( + ctx, + util.keys[util.genesis().validators.quorum_threshold() as usize - 1] + .sign_msg(replica_commit.clone()), + ) + .await + .unwrap() + .unwrap() + .msg; + assert_matches!(res.justification, validator::ProposalJustification::Commit(qc) => { + assert_eq!(qc.message.proposal, replica_commit.proposal); + }); + for i in util.genesis().validators.quorum_threshold() as usize..util.keys.len() { + let res = util + .process_replica_commit(ctx, util.keys[i].sign_msg(replica_commit.clone())) + .await; + assert_matches!(res, Err(commit::Error::Old { .. })); + } + + Ok(()) + }) + .await + .unwrap(); +} + +/// ReplicaCommit received before receiving LeaderProposal. +/// Whether replica accepts or rejects the message it doesn't matter. +/// It just shouldn't crash. +#[tokio::test] +async fn replica_commit_unexpected_proposal() { + 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)); + + util.produce_block(ctx).await; + let replica_commit = validator::ReplicaCommit { + view: util.view(), + proposal: validator::BlockHeader { + number: util + .replica + .high_commit_qc + .as_ref() + .unwrap() + .message + .proposal + .number + .next(), + payload: ctx.rng().gen(), + }, + }; + + let _ = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit)) + .await; + + Ok(()) + }) + .await + .unwrap(); +} + +/// Proposal should be the same for every ReplicaCommit +/// Check it doesn't fail if one validator sends a different proposal in +/// the ReplicaCommit +#[tokio::test] +async fn replica_commit_different_proposals() { + 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 replica_commit = util.new_replica_commit(ctx).await; + + // Process a modified replica_commit (ie. from a malicious or wrong node) + let mut bad_replica_commit = replica_commit.clone(); + bad_replica_commit.proposal.number = replica_commit.proposal.number.next(); + util.process_replica_commit(ctx, util.owner_key().sign_msg(bad_replica_commit)) + .await + .unwrap(); + + // The rest of the validators sign the correct one + let mut replica_commit_result = None; + for i in 1..util.keys.len() { + replica_commit_result = util + .process_replica_commit(ctx, util.keys[i].sign_msg(replica_commit.clone())) + .await + .unwrap(); + } + + // Check correct proposal has been committed + assert_matches!(replica_commit_result.unwrap().msg.justification, validator::ProposalJustification::Commit(qc) => { + assert_eq!(qc.message.proposal, replica_commit.proposal); + }); + + Ok(()) + }) + .await + .unwrap(); +} + +/// Check that leader won't accumulate undefined amount of messages if +/// it's spammed with ReplicaCommit messages for future views +#[tokio::test] +async fn replica_commit_limit_messages_in_memory() { + 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 mut replica_commit = util.new_replica_commit(ctx).await; + let mut view = util.view(); + // Spam it with 200 messages for different views + for _ in 0..200 { + replica_commit.view = view; + let res = util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit.clone())) + .await; + assert_matches!(res, Ok(_)); + view.number = view.number.next(); + } + + // Ensure only 1 commit_qc is in memory, as the previous 199 were discarded each time + // a new message was processed + assert_eq!(util.replica.commit_qcs_cache.len(), 1); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_commit_filter_functions_test() { + 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 replica_commit = util.new_replica_commit(ctx).await; + let msg = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit.clone(), + )); + + // Send a msg with invalid signature + let mut invalid_msg = msg.clone(); + invalid_msg.sig = ctx.rng().gen(); + util.send(invalid_msg); + + // Send a correct message + util.send(msg.clone()); + + // Validate only correct message is received + assert_eq!(util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, msg); + + // Send a msg with view number = 2 + let mut replica_commit_from_view_2 = replica_commit.clone(); + replica_commit_from_view_2.view.number = validator::ViewNumber(2); + let msg_from_view_2 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit_from_view_2, + )); + util.send(msg_from_view_2); + + // Send a msg with view number = 4, will prune message from view 2 + let mut replica_commit_from_view_4 = replica_commit.clone(); + replica_commit_from_view_4.view.number = validator::ViewNumber(4); + let msg_from_view_4 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit_from_view_4, + )); + util.send(msg_from_view_4.clone()); + + // Send a msg with view number = 3, will be discarded, as it is older than message from view 4 + let mut replica_commit_from_view_3 = replica_commit.clone(); + replica_commit_from_view_3.view.number = validator::ViewNumber(3); + let msg_from_view_3 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit_from_view_3, + )); + util.send(msg_from_view_3); + + // Validate only message from view 4 is received + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_view_4 + ); + + // Send a msg from validator 0 + let msg_from_validator_0 = util.keys[0].sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit.clone(), + )); + util.send(msg_from_validator_0.clone()); + + // Send a msg from validator 1 + let msg_from_validator_1 = util.keys[1].sign_msg(validator::ConsensusMsg::ReplicaCommit( + replica_commit.clone(), + )); + util.send(msg_from_validator_1.clone()); + + //Validate both are present in the inbound_pipe + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_validator_0 + ); + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_validator_1 + ); + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/bft/src/chonky_bft/tests/mod.rs b/node/actors/bft/src/chonky_bft/tests/mod.rs new file mode 100644 index 00000000..92a30580 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/tests/mod.rs @@ -0,0 +1,127 @@ +use crate::chonky_bft::testonly::UTHarness; +use zksync_concurrency::{ctx, scope}; +use zksync_consensus_roles::validator; + +mod commit; +mod new_view; +mod proposal; +//mod proposer; +mod timeout; + +/// Sanity check of the happy path. +#[tokio::test] +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)); + + util.produce_block(ctx).await; + + Ok(()) + }) + .await + .unwrap(); +} + +/// Sanity check of block production after timeout +#[tokio::test] +async fn block_production_timeout() { + 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.produce_block_after_timeout(ctx).await; + + Ok(()) + }) + .await + .unwrap(); +} + +/// Sanity check of block production with reproposal. +#[tokio::test] +async fn block_production_timeout_reproposal() { + 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 replica_commit = util.new_replica_commit(ctx).await; + let mut timeout = util.new_replica_timeout(ctx).await; + + for i in 0..util.genesis().validators.subquorum_threshold() as usize { + util.process_replica_timeout(ctx, util.keys[i].sign_msg(timeout.clone())) + .await + .unwrap(); + } + timeout.high_vote = None; + for i in util.genesis().validators.subquorum_threshold() as usize..util.keys.len() { + let _ = util + .process_replica_timeout(ctx, util.keys[i].sign_msg(timeout.clone())) + .await; + } + + assert!(util.replica.high_commit_qc.is_none()); + util.produce_block(ctx).await; + assert_eq!( + util.replica.high_commit_qc.unwrap().message.proposal, + replica_commit.proposal + ); + + Ok(()) + }) + .await + .unwrap(); +} + +/// Testing liveness after the network becomes idle with replica in commit phase. +#[tokio::test] +async fn block_production_timeout_in_commit() { + 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; + + // Replica is in `Phase::Commit`, but should still accept messages from newer views. + assert_eq!(util.replica.phase, validator::Phase::Commit); + util.produce_block_after_timeout(ctx).await; + + Ok(()) + }) + .await + .unwrap(); +} + +/// Testing liveness after the network becomes idle with replica having some cached commit messages for the current view. +#[tokio::test] +async fn block_production_timeout_some_commits() { + 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 replica_commit = util.new_replica_commit(ctx).await; + assert!(util + .process_replica_commit(ctx, util.owner_key().sign_msg(replica_commit)) + .await + .unwrap() + .is_none()); + + // Replica is in `Phase::Commit`, but should still accept prepares from newer views. + assert_eq!(util.replica.phase, validator::Phase::Commit); + util.produce_block_after_timeout(ctx).await; + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/bft/src/chonky_bft/tests/new_view.rs b/node/actors/bft/src/chonky_bft/tests/new_view.rs new file mode 100644 index 00000000..7f1f6550 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/tests/new_view.rs @@ -0,0 +1,204 @@ +use crate::chonky_bft::{new_view, testonly::UTHarness}; +use assert_matches::assert_matches; +use rand::Rng; +use zksync_concurrency::{ctx, scope}; +use zksync_consensus_roles::validator; + +#[tokio::test] +async fn new_view_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 commit_1 = validator::ReplicaCommit { + view: util.view().next(), + proposal: validator::BlockHeader { + number: validator::BlockNumber(1), + payload: ctx.rng().gen(), + }, + }; + let mut commit_qc_1 = validator::CommitQC::new(commit_1.clone(), util.genesis()); + for key in &util.keys { + commit_qc_1 + .add(&key.sign_msg(commit_1.clone()), util.genesis()) + .unwrap(); + } + let new_view_1 = validator::ReplicaNewView { + justification: validator::ProposalJustification::Commit(commit_qc_1.clone()), + }; + + let commit_2 = validator::ReplicaCommit { + view: commit_1.view.next(), + proposal: validator::BlockHeader { + number: commit_1.proposal.number.next(), + payload: ctx.rng().gen(), + }, + }; + let mut commit_qc_2 = validator::CommitQC::new(commit_2.clone(), util.genesis()); + for key in &util.keys { + commit_qc_2 + .add(&key.sign_msg(commit_2.clone()), util.genesis()) + .unwrap(); + } + let new_view_2 = validator::ReplicaNewView { + justification: validator::ProposalJustification::Commit(commit_qc_2.clone()), + }; + + let timeout = validator::ReplicaTimeout { + view: commit_2.view.next(), + high_vote: None, + high_qc: Some(commit_qc_2.clone()), + }; + let mut timeout_qc = validator::TimeoutQC::new(timeout.view); + for key in &util.keys { + timeout_qc + .add(&key.sign_msg(timeout.clone()), util.genesis()) + .unwrap(); + } + let new_view_3 = validator::ReplicaNewView { + justification: validator::ProposalJustification::Timeout(timeout_qc.clone()), + }; + + // Check that first new view with commit QC updates the view and high commit QC. + let res = util + .process_replica_new_view(ctx, util.owner_key().sign_msg(new_view_1.clone())) + .await + .unwrap() + .unwrap() + .msg; + assert_eq!(util.view(), new_view_1.view()); + assert_matches!(res.justification, validator::ProposalJustification::Commit(qc) => { + assert_eq!(util.replica.high_commit_qc.clone().unwrap(), qc); + }); + + // Check that the third new view with timeout QC updates the view, high timeout QC and high commit QC. + let res = util + .process_replica_new_view(ctx, util.owner_key().sign_msg(new_view_3.clone())) + .await + .unwrap() + .unwrap() + .msg; + assert_eq!(util.view(), new_view_3.view()); + assert_matches!(res.justification, validator::ProposalJustification::Timeout(qc) => { + assert_eq!(util.replica.high_timeout_qc.clone().unwrap(), qc); + assert_eq!(util.replica.high_commit_qc.clone().unwrap(), qc.high_qc().unwrap().clone()); + }); + + // Check that the second new view with commit QC is ignored and doesn't affect the state. + let res = util + .process_replica_new_view(ctx, util.owner_key().sign_msg(new_view_2.clone())) + .await; + assert_eq!(util.view(), new_view_3.view()); + assert_eq!(util.replica.high_timeout_qc.clone().unwrap(), timeout_qc); + assert_eq!( + util.replica.high_commit_qc.clone().unwrap(), + timeout_qc.high_qc().unwrap().clone() + ); + assert_matches!( + res, + Err(new_view::Error::Old { current_view }) => { + assert_eq!(current_view, util.replica.view_number); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn new_view_non_validator_signer() { + 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_new_view = util.new_replica_new_view().await; + let non_validator_key: validator::SecretKey = ctx.rng().gen(); + let res = util + .process_replica_new_view(ctx, non_validator_key.sign_msg(replica_new_view)) + .await; + + assert_matches!( + res, + Err(new_view::Error::NonValidatorSigner { signer }) => { + assert_eq!(*signer, non_validator_key.public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_new_view_old() { + 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_new_view = util.new_replica_new_view().await; + util.produce_block(ctx).await; + let res = util + .process_replica_new_view(ctx, util.owner_key().sign_msg(replica_new_view)) + .await; + + assert_matches!( + res, + Err(new_view::Error::Old { current_view }) => { + assert_eq!(current_view, util.replica.view_number); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn new_view_invalid_sig() { + 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 msg = util.new_replica_new_view().await; + let mut replica_new_view = util.owner_key().sign_msg(msg); + replica_new_view.sig = ctx.rng().gen(); + + let res = util.process_replica_new_view(ctx, replica_new_view).await; + assert_matches!(res, Err(new_view::Error::InvalidSignature(..))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn new_view_invalid_message() { + 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 res = util + .process_replica_new_view(ctx, util.owner_key().sign_msg(ctx.rng().gen())) + .await; + assert_matches!(res, Err(new_view::Error::InvalidMessage(_))); + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/bft/src/chonky_bft/tests/proposal.rs b/node/actors/bft/src/chonky_bft/tests/proposal.rs new file mode 100644 index 00000000..2c5d10dc --- /dev/null +++ b/node/actors/bft/src/chonky_bft/tests/proposal.rs @@ -0,0 +1,364 @@ +use crate::{ + chonky_bft::{ + proposal, + testonly::{UTHarness, MAX_PAYLOAD_SIZE}, + }, + testonly::RejectPayload, +}; +use assert_matches::assert_matches; +use rand::Rng; +use zksync_concurrency::{ctx, scope}; +use zksync_consensus_roles::validator; + +#[tokio::test] +async fn proposal_yield_replica_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(ctx, 1).await; + s.spawn_bg(runner.run(ctx)); + + let proposal = util.new_leader_proposal(ctx).await; + let replica_commit = util + .process_leader_proposal(ctx, util.owner_key().sign_msg(proposal.clone())) + .await + .unwrap(); + + assert_eq!( + replica_commit.msg, + validator::ReplicaCommit { + view: proposal.view(), + proposal: validator::BlockHeader { + number: proposal.justification.get_implied_block(util.genesis()).0, + payload: proposal.proposal_payload.unwrap().hash() + }, + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_old_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 proposal = util.new_leader_proposal(ctx).await; + + util.replica.phase = validator::Phase::Commit; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal.clone())) + .await; + + assert_matches!( + res, + Err(proposal::Error::Old { current_view, current_phase }) => { + assert_eq!(current_view, util.replica.view_number); + assert_eq!(current_phase, util.replica.phase); + } + ); + + util.replica.phase = validator::Phase::Timeout; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal.clone())) + .await; + + assert_matches!( + res, + Err(proposal::Error::Old { current_view, current_phase }) => { + assert_eq!(current_view, util.replica.view_number); + assert_eq!(current_phase, util.replica.phase); + } + ); + + util.replica.phase = validator::Phase::Prepare; + util.replica.view_number = util.replica.view_number.next(); + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!( + res, + Err(proposal::Error::Old { current_view, current_phase }) => { + assert_eq!(current_view, util.replica.view_number); + assert_eq!(current_phase, util.replica.phase); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_invalid_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 proposal = util.new_leader_proposal(ctx).await; + + assert_ne!( + util.view_leader(proposal.view().number), + util.owner_key().public() + ); + + let res = util + .process_leader_proposal(ctx, util.owner_key().sign_msg(proposal)) + .await; + + assert_matches!( + res, + Err(proposal::Error::InvalidLeader { correct_leader, received_leader }) => { + assert_eq!(correct_leader, util.keys[1].public()); + assert_eq!(received_leader, util.keys[0].public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_invalid_signature() { + 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 proposal = util.new_leader_proposal(ctx).await; + let mut signed_proposal = util.leader_key().sign_msg(proposal); + signed_proposal.sig = ctx.rng().gen(); + + let res = util.process_leader_proposal(ctx, signed_proposal).await; + + assert_matches!(res, Err(proposal::Error::InvalidSignature(_))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_invalid_message() { + 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 proposal = util.new_leader_proposal(ctx).await; + proposal.justification = ctx.rng().gen(); + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!(res, Err(proposal::Error::InvalidMessage(_))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_pruned_block() { + 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 fake_commit = validator::ReplicaCommit { + view: util.view(), + proposal: validator::BlockHeader { + number: util + .replica + .config + .block_store + .queued() + .first + .prev() + .unwrap() + .prev() + .unwrap(), + payload: ctx.rng().gen(), + }, + }; + + util.process_replica_commit_all(ctx, fake_commit).await; + + // The replica should now produce a proposal for an already pruned block number. + let proposal = util.new_leader_proposal(ctx).await; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!(res, Err(proposal::Error::ProposalAlreadyPruned)); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_reproposal_with_payload() { + 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)); + + util.new_replica_commit(ctx).await; + let replica_timeout = util.new_replica_timeout(ctx).await; + util.process_replica_timeout_all(ctx, replica_timeout).await; + + let mut proposal = util.new_leader_proposal(ctx).await; + assert!(proposal.proposal_payload.is_none()); + proposal.proposal_payload = Some(ctx.rng().gen()); + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!(res, Err(proposal::Error::ReproposalWithPayload)); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_missing_payload() { + 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 proposal = util.new_leader_proposal(ctx).await; + proposal.proposal_payload = None; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!(res, Err(proposal::Error::MissingPayload)); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_proposal_oversized_payload() { + 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 payload = validator::Payload(vec![0; MAX_PAYLOAD_SIZE + 1]); + let mut proposal = util.new_leader_proposal(ctx).await; + proposal.proposal_payload = Some(payload); + + let res = util + .process_leader_proposal(ctx, util.owner_key().sign_msg(proposal)) + .await; + assert_matches!( + res, + Err(proposal::Error::ProposalOversizedPayload{ payload_size }) => { + assert_eq!(payload_size, MAX_PAYLOAD_SIZE + 1); + } + ); + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_missing_previous_payload() { + 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 missing_payload_number = util.replica.config.block_store.queued().first.next(); + let fake_commit = validator::ReplicaCommit { + view: util.view(), + proposal: validator::BlockHeader { + number: missing_payload_number, + payload: ctx.rng().gen(), + }, + }; + + util.process_replica_commit_all(ctx, fake_commit).await; + + let proposal = validator::LeaderProposal { + proposal_payload: Some(ctx.rng().gen()), + justification: validator::ProposalJustification::Commit( + util.replica.high_commit_qc.clone().unwrap(), + ), + }; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!( + res, + Err(proposal::Error::MissingPreviousPayload { prev_number } ) => { + assert_eq!(prev_number, missing_payload_number); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn proposal_invalid_payload() { + 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_with_payload_manager(ctx, 1, Box::new(RejectPayload)).await; + s.spawn_bg(runner.run(ctx)); + + let proposal = util.new_leader_proposal(ctx).await; + + let res = util + .process_leader_proposal(ctx, util.leader_key().sign_msg(proposal)) + .await; + + assert_matches!(res, Err(proposal::Error::InvalidPayload(_))); + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/bft/src/chonky_bft/tests/timeout.rs b/node/actors/bft/src/chonky_bft/tests/timeout.rs new file mode 100644 index 00000000..0ee2dc9f --- /dev/null +++ b/node/actors/bft/src/chonky_bft/tests/timeout.rs @@ -0,0 +1,439 @@ +use crate::chonky_bft::{testonly::UTHarness, timeout}; +use assert_matches::assert_matches; +use rand::Rng; +use zksync_concurrency::{ctx, scope}; +use zksync_consensus_roles::validator; + +#[tokio::test] +async fn timeout_yield_new_view_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 cur_view = util.replica.view_number; + let replica_timeout = util.new_replica_timeout(ctx).await; + assert_eq!(util.replica.phase, validator::Phase::Timeout); + + let new_view = util + .process_replica_timeout_all(ctx, replica_timeout.clone()) + .await + .msg; + assert_eq!(util.replica.view_number, cur_view.next()); + assert_eq!(util.replica.phase, validator::Phase::Prepare); + assert_eq!(new_view.view().number, cur_view.next()); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn timeout_non_validator_signer() { + 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_timeout = util.new_replica_timeout(ctx).await; + let non_validator_key: validator::SecretKey = ctx.rng().gen(); + let res = util + .process_replica_timeout(ctx, non_validator_key.sign_msg(replica_timeout)) + .await; + + assert_matches!( + res, + Err(timeout::Error::NonValidatorSigner { signer }) => { + assert_eq!(*signer, non_validator_key.public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_timeout_old() { + 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 replica_timeout = util.new_replica_timeout(ctx).await; + replica_timeout.view.number = validator::ViewNumber(util.replica.view_number.0 - 1); + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(replica_timeout)) + .await; + + assert_matches!( + res, + Err(timeout::Error::Old { current_view }) => { + assert_eq!(current_view, util.replica.view_number); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn timeout_duplicate_signer() { + 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)); + + util.produce_block(ctx).await; + + let replica_timeout = util.new_replica_timeout(ctx).await; + assert!(util + .process_replica_timeout(ctx, util.owner_key().sign_msg(replica_timeout.clone())) + .await + .unwrap() + .is_none()); + + // Processing twice same ReplicaTimeout for same view gets DuplicateSigner error + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(replica_timeout.clone())) + .await; + assert_matches!( + res, + Err(timeout::Error::DuplicateSigner { + message_view, + signer + })=> { + assert_eq!(message_view, util.replica.view_number); + assert_eq!(*signer, util.owner_key().public()); + } + ); + + // Processing twice different ReplicaTimeout for same view gets DuplicateSigner error too + // replica_timeout.high_vote = None; + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(replica_timeout.clone())) + .await; + assert_matches!( + res, + Err(timeout::Error::DuplicateSigner { + message_view, + signer + })=> { + assert_eq!(message_view, util.replica.view_number); + assert_eq!(*signer, util.owner_key().public()); + } + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn timeout_invalid_sig() { + 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 msg = util.new_replica_timeout(ctx).await; + let mut replica_timeout = util.owner_key().sign_msg(msg); + replica_timeout.sig = ctx.rng().gen(); + + let res = util.process_replica_timeout(ctx, replica_timeout).await; + assert_matches!(res, Err(timeout::Error::InvalidSignature(..))); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn timeout_invalid_message() { + 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_timeout = util.new_replica_timeout(ctx).await; + + let mut bad_replica_timeout = replica_timeout.clone(); + bad_replica_timeout.view.genesis = ctx.rng().gen(); + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(bad_replica_timeout)) + .await; + assert_matches!( + res, + Err(timeout::Error::InvalidMessage( + validator::ReplicaTimeoutVerifyError::BadView(_) + )) + ); + + let mut bad_replica_timeout = replica_timeout.clone(); + bad_replica_timeout.high_vote = Some(ctx.rng().gen()); + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(bad_replica_timeout)) + .await; + assert_matches!( + res, + Err(timeout::Error::InvalidMessage( + validator::ReplicaTimeoutVerifyError::InvalidHighVote(_) + )) + ); + + let mut bad_replica_timeout = replica_timeout.clone(); + bad_replica_timeout.high_qc = Some(ctx.rng().gen()); + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(bad_replica_timeout)) + .await; + assert_matches!( + res, + Err(timeout::Error::InvalidMessage( + validator::ReplicaTimeoutVerifyError::InvalidHighQC(_) + )) + ); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn timeout_num_received_below_threshold() { + 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 replica_timeout = util.new_replica_timeout(ctx).await; + for i in 0..util.genesis().validators.quorum_threshold() as usize - 1 { + assert!(util + .process_replica_timeout(ctx, util.keys[i].sign_msg(replica_timeout.clone())) + .await + .unwrap() + .is_none()); + } + let res = util + .process_replica_timeout( + ctx, + util.keys[util.genesis().validators.quorum_threshold() as usize - 1] + .sign_msg(replica_timeout.clone()), + ) + .await + .unwrap() + .unwrap() + .msg; + assert_matches!(res.justification, validator::ProposalJustification::Timeout(qc) => { + assert_eq!(qc.view, replica_timeout.view); + }); + for i in util.genesis().validators.quorum_threshold() as usize..util.keys.len() { + let res = util + .process_replica_timeout(ctx, util.keys[i].sign_msg(replica_timeout.clone())) + .await; + assert_matches!(res, Err(timeout::Error::Old { .. })); + } + + Ok(()) + }) + .await + .unwrap(); +} + +/// Check all ReplicaTimeout are included for weight calculation +/// even on different messages for the same view. +#[tokio::test] +async fn timeout_weight_different_messages() { + 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 view = util.view(); + util.produce_block(ctx).await; + + let replica_timeout = util.new_replica_timeout(ctx).await; + util.replica.phase = validator::Phase::Prepare; // To allow processing of proposal later. + let proposal = replica_timeout.clone().high_vote.unwrap().proposal; + + // Create a different proposal for the same view + let mut different_proposal = proposal; + different_proposal.number = different_proposal.number.next(); + + // Create a new ReplicaTimeout with the different proposal + let mut other_replica_timeout = replica_timeout.clone(); + let mut high_vote = other_replica_timeout.high_vote.clone().unwrap(); + high_vote.proposal = different_proposal; + let high_qc = util + .new_commit_qc(ctx, |msg: &mut validator::ReplicaCommit| { + msg.proposal = different_proposal; + msg.view = view; + }) + .await; + other_replica_timeout.high_vote = Some(high_vote); + other_replica_timeout.high_qc = Some(high_qc); + + let validators = util.keys.len(); + + // half of the validators sign replica_timeout + for i in 0..validators / 2 { + util.process_replica_timeout(ctx, util.keys[i].sign_msg(replica_timeout.clone())) + .await + .unwrap(); + } + + let mut res = None; + // The rest of the validators until threshold sign other_replica_timeout + for i in validators / 2..util.genesis().validators.quorum_threshold() as usize { + res = util + .process_replica_timeout(ctx, util.keys[i].sign_msg(other_replica_timeout.clone())) + .await + .unwrap(); + } + + assert_matches!(res.unwrap().msg.justification, validator::ProposalJustification::Timeout(qc) => { + assert_eq!(qc.view, replica_timeout.view); + assert_eq!(qc.high_vote(util.genesis()).unwrap(), proposal); + }); + + Ok(()) + }) + .await + .unwrap(); +} + +/// Check that leader won't accumulate undefined amount of messages if +/// it's spammed with ReplicaTimeout messages for future views +#[tokio::test] +async fn replica_timeout_limit_messages_in_memory() { + 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 mut replica_timeout = util.new_replica_timeout(ctx).await; + let mut view = util.view(); + // Spam it with 200 messages for different views + for _ in 0..200 { + replica_timeout.view = view; + let res = util + .process_replica_timeout(ctx, util.owner_key().sign_msg(replica_timeout.clone())) + .await; + assert_matches!(res, Ok(_)); + view.number = view.number.next(); + } + + // Ensure only 1 timeout_qc is in memory, as the previous 199 were discarded each time + // a new message was processed + assert_eq!(util.replica.timeout_qcs_cache.len(), 1); + + Ok(()) + }) + .await + .unwrap(); +} + +#[tokio::test] +async fn replica_timeout_filter_functions_test() { + 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 replica_timeout = util.new_replica_timeout(ctx).await; + let msg = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout.clone(), + )); + + // Send a msg with invalid signature + let mut invalid_msg = msg.clone(); + invalid_msg.sig = ctx.rng().gen(); + util.send(invalid_msg); + + // Send a correct message + util.send(msg.clone()); + + // Validate only correct message is received + assert_eq!(util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, msg); + + // Send a msg with view number = 2 + let mut replica_timeout_from_view_2 = replica_timeout.clone(); + replica_timeout_from_view_2.view.number = validator::ViewNumber(2); + let msg_from_view_2 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout_from_view_2, + )); + util.send(msg_from_view_2); + + // Send a msg with view number = 4, will prune message from view 2 + let mut replica_timeout_from_view_4 = replica_timeout.clone(); + replica_timeout_from_view_4.view.number = validator::ViewNumber(4); + let msg_from_view_4 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout_from_view_4, + )); + util.send(msg_from_view_4.clone()); + + // Send a msg with view number = 3, will be discarded, as it is older than message from view 4 + let mut replica_timeout_from_view_3 = replica_timeout.clone(); + replica_timeout_from_view_3.view.number = validator::ViewNumber(3); + let msg_from_view_3 = util + .owner_key() + .sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout_from_view_3, + )); + util.send(msg_from_view_3); + + // Validate only message from view 4 is received + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_view_4 + ); + + // Send a msg from validator 0 + let msg_from_validator_0 = util.keys[0].sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout.clone(), + )); + util.send(msg_from_validator_0.clone()); + + // Send a msg from validator 1 + let msg_from_validator_1 = util.keys[1].sign_msg(validator::ConsensusMsg::ReplicaTimeout( + replica_timeout.clone(), + )); + util.send(msg_from_validator_1.clone()); + + // Validate both are present in the inbound_pipe + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_validator_0 + ); + assert_eq!( + util.replica.inbound_pipe.recv(ctx).await.unwrap().msg, + msg_from_validator_1 + ); + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/bft/src/chonky_bft/timeout.rs b/node/actors/bft/src/chonky_bft/timeout.rs new file mode 100644 index 00000000..84b02ff3 --- /dev/null +++ b/node/actors/bft/src/chonky_bft/timeout.rs @@ -0,0 +1,186 @@ +use super::StateMachine; +use crate::metrics; +use std::{cmp::max, collections::HashSet}; +use zksync_concurrency::{ctx, error::Wrap}; +use zksync_consensus_network::io::ConsensusInputMessage; +use zksync_consensus_roles::validator; + +/// Errors that can occur when processing a ReplicaTimeout message. +#[derive(Debug, thiserror::Error)] +pub(crate) enum Error { + /// Message signer isn't part of the validator set. + #[error("message signer isn't part of the validator set (signer: {signer:?})")] + NonValidatorSigner { + /// Signer of the message. + signer: Box, + }, + /// Past view. + #[error("past view (current view: {current_view:?})")] + Old { + /// Current view. + current_view: validator::ViewNumber, + }, + /// Duplicate signer. We already have a timeout message from the same validator + /// for the same or past view. + #[error("duplicate signer (message view: {message_view:?}, signer: {signer:?})")] + DuplicateSigner { + /// View number of the message. + message_view: validator::ViewNumber, + /// Signer of the message. + signer: Box, + }, + /// Invalid message signature. + #[error("invalid signature: {0:#}")] + InvalidSignature(#[source] anyhow::Error), + /// Invalid message. + #[error("invalid message: {0:#}")] + InvalidMessage(#[source] validator::ReplicaTimeoutVerifyError), + /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. + #[error(transparent)] + Internal(#[from] ctx::Error), +} + +impl Wrap for Error { + fn with_wrap C>( + self, + f: F, + ) -> Self { + match self { + Error::Internal(err) => Error::Internal(err.with_wrap(f)), + err => err, + } + } +} + +impl StateMachine { + /// Processes a ReplicaTimeout message. + pub(crate) async fn on_timeout( + &mut self, + ctx: &ctx::Ctx, + signed_message: validator::Signed, + ) -> Result<(), Error> { + // ----------- Checking origin of the message -------------- + + // Unwrap message. + let message = &signed_message.msg; + let author = &signed_message.key; + + // Check that the message signer is in the validator committee. + if !self.config.genesis().validators.contains(author) { + return Err(Error::NonValidatorSigner { + signer: author.clone().into(), + }); + } + + // If the message is from a past view, ignore it. + if message.view.number < self.view_number { + return Err(Error::Old { + current_view: self.view_number, + }); + } + + // If we already have a message from the same validator for the same or past view, ignore it. + if let Some(&view) = self.timeout_views_cache.get(author) { + if view >= message.view.number { + return Err(Error::DuplicateSigner { + message_view: message.view.number, + signer: author.clone().into(), + }); + } + } + + // ----------- Checking the signed part of the message -------------- + + // 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. -------------- + + // We add the message to the incrementally-constructed QC. + let timeout_qc = self + .timeout_qcs_cache + .entry(message.view.number) + .or_insert_with(|| validator::TimeoutQC::new(message.view)); + + // Should always succeed as all checks have been already performed + timeout_qc + .add(&signed_message, self.config.genesis()) + .expect("could not add message to TimeoutQC"); + + // Calculate the TimeoutQC signers weight. + let weight = timeout_qc.weight(&self.config.genesis().validators); + + // Update view number of last timeout message for author + self.timeout_views_cache + .insert(author.clone(), message.view.number); + + // Clean up timeout_qcs for the case that no replica is at the view + // of a given TimeoutQC + // This prevents timeout_qcs map from growing indefinitely in case some + // malicious replica starts spamming messages for future views + let active_views: HashSet<_> = self.timeout_views_cache.values().collect(); + self.timeout_qcs_cache + .retain(|view_number, _| active_views.contains(view_number)); + + // Now we check if we have enough weight to continue. If not, we wait for more messages. + if weight < self.config.genesis().validators.quorum_threshold() { + return Ok(()); + }; + + // ----------- We have a QC. Now we process it. -------------- + + // Consume the created timeout QC for this view. + let timeout_qc = self.timeout_qcs_cache.remove(&message.view.number).unwrap(); + + // We update our state with the new timeout QC. + if let Some(commit_qc) = timeout_qc.high_qc() { + self.process_commit_qc(ctx, commit_qc) + .await + .wrap("process_commit_qc()")?; + } + self.high_timeout_qc = max(Some(timeout_qc.clone()), self.high_timeout_qc.clone()); + + // Start a new view. + self.start_new_view(ctx, message.view.number.next()).await?; + + Ok(()) + } + + /// This blocking method is used whenever we timeout in a view. + pub(crate) async fn start_timeout(&mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { + // Update the state machine. + self.phase = validator::Phase::Timeout; + + // Backup our state. + self.backup_state(ctx).await.wrap("backup_state()")?; + + // Broadcast our timeout message. + let output_message = ConsensusInputMessage { + message: self + .config + .secret_key + .sign_msg(validator::ConsensusMsg::ReplicaTimeout( + validator::ReplicaTimeout { + view: validator::View { + genesis: self.config.genesis().hash(), + number: self.view_number, + }, + high_vote: self.high_vote.clone(), + high_qc: self.high_commit_qc.clone(), + }, + )), + }; + + self.outbound_pipe.send(output_message.into()); + + // Log the event. + tracing::info!("Timed out at view {}", self.view_number); + metrics::METRICS.replica_view_number.set(self.view_number.0); + + Ok(()) + } +} diff --git a/node/actors/bft/src/leader/mod.rs b/node/actors/bft/src/leader/mod.rs deleted file mode 100644 index f4615904..00000000 --- a/node/actors/bft/src/leader/mod.rs +++ /dev/null @@ -1,11 +0,0 @@ -//! Implements the leader role in the Fastest-HotStuff consensus algorithm. The leader is the role that proposes blocks -//! 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. - -pub(crate) mod replica_commit; -pub(crate) mod replica_prepare; -mod state_machine; -#[cfg(test)] -mod tests; - -pub(crate) use self::state_machine::StateMachine; diff --git a/node/actors/bft/src/leader/replica_prepare.rs b/node/actors/bft/src/leader/replica_prepare.rs deleted file mode 100644 index 57186e82..00000000 --- a/node/actors/bft/src/leader/replica_prepare.rs +++ /dev/null @@ -1,146 +0,0 @@ -//! Handler of a ReplicaPrepare message. -use super::StateMachine; -use std::collections::HashSet; -use zksync_concurrency::{ctx, error::Wrap}; -use zksync_consensus_roles::validator; - -/// Errors that can occur when processing a "replica prepare" message. -#[derive(Debug, thiserror::Error)] -pub(crate) enum Error { - /// Message signer isn't part of the validator set. - #[error("Message signer isn't part of the validator set (signer: {signer:?})")] - NonValidatorSigner { - /// Signer of the message. - signer: validator::PublicKey, - }, - /// Past view or phase. - #[error("past view/phase (current view: {current_view:?}, current phase: {current_phase:?})")] - Old { - /// Current view. - current_view: validator::ViewNumber, - /// Current phase. - current_phase: validator::Phase, - }, - /// The node is not a leader for this message's view. - #[error("we are not a leader for this message's view")] - NotLeaderInView, - /// Invalid message signature. - #[error("invalid signature: {0:#}")] - InvalidSignature(#[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), -} - -impl Wrap for Error { - fn with_wrap C>( - self, - f: F, - ) -> Self { - match self { - Error::Internal(err) => Error::Internal(err.with_wrap(f)), - err => err, - } - } -} - -impl StateMachine { - /// Processes `ReplicaPrepare` message. - pub(crate) async fn process_replica_prepare( - &mut self, - ctx: &ctx::Ctx, - signed_message: validator::Signed, - ) -> Result<(), Error> { - // ----------- Checking origin of the message -------------- - - // Unwrap message. - let message = signed_message.msg.clone(); - let author = &signed_message.key; - - // Check that the message signer is in the validator set. - if !self.config.genesis().validators.contains(author) { - return Err(Error::NonValidatorSigner { - signer: author.clone(), - }); - } - - // If the message is from the "past", we discard it. - // That is, it's from a previous view or phase, or if we already received a message - // from the same validator and for the same view. - if (message.view.number, validator::Phase::Prepare) < (self.view, self.phase) - || self - .replica_prepare_views - .get(author) - .is_some_and(|view_number| *view_number >= message.view.number) - { - return Err(Error::Old { - current_view: self.view, - current_phase: self.phase, - }); - } - - // If the message is for a view when we are not a leader, we discard it. - if self.config.genesis().view_leader(message.view.number) != self.config.secret_key.public() - { - return Err(Error::NotLeaderInView); - } - - // ----------- Checking the signed part of the message -------------- - - // Check the signature on the message. - signed_message.verify().map_err(Error::InvalidSignature)?; - - // Verify the message. - message - .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. - let prepare_qc = self - .prepare_qcs - .entry(message.view.number) - .or_insert_with(|| validator::PrepareQC::new(message.view.clone())); - - // Should always succeed as all checks have been already performed - prepare_qc - .add(&signed_message, self.config.genesis()) - .expect("Could not add message to PrepareQC"); - - // Calculate the PrepareQC signers weight. - let weight = prepare_qc.weight(&self.config.genesis().validators); - - // Update prepare message current view number for author - self.replica_prepare_views - .insert(author.clone(), message.view.number); - - // Clean up prepare_qcs for the case that no replica is at the view - // of a given PrepareQC - // This prevents prepare_qcs map from growing indefinitely in case some - // malicious replica starts spamming messages for future views - let active_views: HashSet<_> = self.replica_prepare_views.values().collect(); - self.prepare_qcs - .retain(|view_number, _| active_views.contains(view_number)); - - // Now we check if we have enough weight to continue. - if weight < self.config.genesis().validators.quorum_threshold() { - return Ok(()); - } - - // ----------- Update the state machine -------------- - - 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.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 deleted file mode 100644 index 9e668751..00000000 --- a/node/actors/bft/src/leader/state_machine.rs +++ /dev/null @@ -1,270 +0,0 @@ -use crate::{metrics, Config, OutputSender}; -use std::{collections::BTreeMap, sync::Arc, unreachable}; -use zksync_concurrency::{ - ctx, - error::Wrap as _, - metrics::LatencyHistogramExt as _, - sync::{self, prunable_mpsc::SelectionFunctionResult}, - time, -}; -use zksync_consensus_network::io::{ConsensusInputMessage, ConsensusReq, Target}; -use zksync_consensus_roles::validator; - -/// The StateMachine struct contains the state of the leader. This is a simple state machine. We just store -/// replica messages and produce leader messages (including proposing blocks) when we reach the threshold for -/// those messages. When participating in consensus we are not the leader most of the time. -pub(crate) struct StateMachine { - /// Consensus configuration and output channel. - pub(crate) config: Arc, - /// Pipe through which leader sends network messages. - pub(crate) outbound_pipe: OutputSender, - /// Pipe through which leader receives network requests. - pub(crate) inbound_pipe: sync::prunable_mpsc::Receiver, - /// The current view number. This might not match the replica's view number, we only have this here - /// to make the leader advance monotonically in time and stop it from accepting messages from the past. - pub(crate) view: validator::ViewNumber, - /// The current phase. This might not match the replica's phase, we only have this here - /// to make the leader advance monotonically in time and stop it from accepting messages from the past. - pub(crate) phase: validator::Phase, - /// Time when the current phase has started. - pub(crate) phase_start: time::Instant, - /// Latest view each validator has signed a ReplicaPrepare message for. - pub(crate) replica_prepare_views: BTreeMap, - /// Prepare QCs indexed by view number. - pub(crate) prepare_qcs: BTreeMap, - /// Newest prepare QC composed from the `ReplicaPrepare` messages. - pub(crate) prepare_qc: sync::watch::Sender>, - /// Commit QCs indexed by view number and then by message. - pub(crate) commit_qcs: - BTreeMap>, - /// Latest view each validator has signed a ReplicaCommit message for. - pub(crate) replica_commit_views: BTreeMap, -} - -impl StateMachine { - /// Creates a new [`StateMachine`] instance. - /// - /// Returns a tuple containing: - /// * The newly created [`StateMachine`] instance. - /// * A sender handle that should be used to send values to be processed by the instance, asynchronously. - pub(crate) fn new( - ctx: &ctx::Ctx, - config: Arc, - outbound_pipe: OutputSender, - ) -> (Self, sync::prunable_mpsc::Sender) { - let (send, recv) = sync::prunable_mpsc::channel( - StateMachine::inbound_filter_predicate, - StateMachine::inbound_selection_function, - ); - - let this = StateMachine { - config, - outbound_pipe, - view: validator::ViewNumber(0), - phase: validator::Phase::Prepare, - phase_start: ctx.now(), - replica_prepare_views: BTreeMap::new(), - prepare_qcs: BTreeMap::new(), - prepare_qc: sync::watch::channel(None).0, - commit_qcs: BTreeMap::new(), - inbound_pipe: recv, - replica_commit_views: BTreeMap::new(), - }; - - (this, send) - } - - /// Runs a loop to process incoming messages. - /// This is the main entry point for the state machine, - /// potentially triggering state modifications and message sending to the executor. - pub(crate) async fn run(mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { - loop { - let req = self.inbound_pipe.recv(ctx).await?; - - let now = ctx.now(); - use validator::ConsensusMsg as M; - let label = match &req.msg.msg { - M::ReplicaPrepare(_) => { - let res = match self - .process_replica_prepare(ctx, req.msg.cast().unwrap()) - .await - .wrap("process_replica_prepare()") - { - Ok(()) => Ok(()), - Err(err) => { - match err { - super::replica_prepare::Error::Internal(e) => { - tracing::error!( - "process_replica_prepare: internal error: {e:#}" - ); - - return Err(e); - } - super::replica_prepare::Error::Old { .. } - | super::replica_prepare::Error::NotLeaderInView => { - // It's broadcasted now, so everyone gets it. - tracing::debug!("process_replica_prepare: {err:#}"); - } - _ => { - tracing::warn!("process_replica_prepare: {err:#}"); - } - } - Err(()) - } - }; - metrics::ConsensusMsgLabel::ReplicaPrepare.with_result(&res) - } - M::ReplicaCommit(_) => { - let res = self - .process_replica_commit(ctx, req.msg.cast().unwrap()) - .map_err(|err| { - tracing::warn!("process_replica_commit: {err:#}"); - }); - metrics::ConsensusMsgLabel::ReplicaCommit.with_result(&res) - } - _ => unreachable!(), - }; - metrics::METRICS.leader_processing_latency[&label].observe_latency(ctx.now() - now); - - // Notify network actor that the message has been processed. - // Ignore sending error. - let _ = req.ack.send(()); - } - } - - /// In a loop, receives a PrepareQC and sends a LeaderPrepare containing it. - /// Every subsequent PrepareQC has to be for a higher view than the previous one (otherwise it - /// is skipped). In case payload generation takes too long, some PrepareQC may be elided, so - /// that the validator doesn't spend time on generating payloads for already expired views. - pub(crate) async fn run_proposer( - ctx: &ctx::Ctx, - config: &Config, - mut prepare_qc: sync::watch::Receiver>, - pipe: &OutputSender, - ) -> ctx::Result<()> { - let mut next_view = validator::ViewNumber(0); - loop { - let Some(prepare_qc) = sync::changed(ctx, &mut prepare_qc).await?.clone() else { - continue; - }; - if prepare_qc.view.number < next_view { - continue; - }; - next_view = prepare_qc.view.number.next(); - Self::propose(ctx, config, prepare_qc, pipe) - .await - .wrap("propose()")?; - } - } - - /// Sends a LeaderPrepare for the given PrepareQC. - /// Uses `payload_source` to generate a payload if needed. - pub(crate) async fn propose( - ctx: &ctx::Ctx, - cfg: &Config, - justification: validator::PrepareQC, - pipe: &OutputSender, - ) -> ctx::Result<()> { - 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 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 Some(&proposal) != high_qc.map(|qc| &qc.message.proposal) => { - (proposal, None) - } - // The previous block was finalized, so we can propose a new block. - _ => { - let number = match high_qc { - Some(qc) => qc.header().number.next(), - None => cfg.genesis().first_block, - }; - // Defensively assume that PayloadManager cannot propose until the previous block is stored. - if let Some(prev) = number.prev() { - cfg.block_store.wait_until_persisted(ctx, prev).await?; - } - let payload = cfg - .payload_manager - .propose(ctx, number) - .await - .wrap("payload_manager.propose()")?; - if payload.0.len() > cfg.max_payload_size { - return Err(anyhow::format_err!( - "proposed payload too large: got {}B, max {}B", - payload.0.len(), - cfg.max_payload_size - ) - .into()); - } - metrics::METRICS - .leader_proposal_payload_size - .observe(payload.0.len()); - let proposal = validator::BlockHeader { - number, - payload: payload.hash(), - }; - (proposal, Some(payload)) - } - }; - - // ----------- Prepare our message and send it -------------- - - // Broadcast the leader prepare message to all replicas (ourselves included). - let msg = cfg - .secret_key - .sign_msg(validator::ConsensusMsg::LeaderPrepare( - validator::LeaderPrepare { - proposal, - proposal_payload: payload, - justification, - }, - )); - pipe.send( - ConsensusInputMessage { - message: msg, - recipient: Target::Broadcast, - } - .into(), - ); - Ok(()) - } - - fn inbound_filter_predicate(new_req: &ConsensusReq) -> bool { - // Verify message signature - new_req.msg.verify().is_ok() - } - - fn inbound_selection_function( - old_req: &ConsensusReq, - new_req: &ConsensusReq, - ) -> SelectionFunctionResult { - if old_req.msg.key != new_req.msg.key { - return SelectionFunctionResult::Keep; - } - use validator::ConsensusMsg as M; - match (&old_req.msg.msg, &new_req.msg.msg) { - (M::ReplicaPrepare(old), M::ReplicaPrepare(new)) => { - // Discard older message - if old.view.number < new.view.number { - SelectionFunctionResult::DiscardOld - } else { - SelectionFunctionResult::DiscardNew - } - } - (M::ReplicaCommit(old), M::ReplicaCommit(new)) => { - // Discard older message - if old.view.number < new.view.number { - SelectionFunctionResult::DiscardOld - } else { - SelectionFunctionResult::DiscardNew - } - } - _ => SelectionFunctionResult::Keep, - } - } -} diff --git a/node/actors/bft/src/leader/tests.rs b/node/actors/bft/src/leader/tests.rs deleted file mode 100644 index e6453384..00000000 --- a/node/actors/bft/src/leader/tests.rs +++ /dev/null @@ -1,914 +0,0 @@ -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, Phase, ViewNumber}; - -#[tokio::test] -async fn replica_prepare_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)); - tracing::info!("started"); - util.new_leader_prepare(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_sanity_yield_leader_prepare() { - 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)); - - util.produce_block(ctx).await; - let replica_prepare = util.new_replica_prepare(); - let leader_prepare = util - .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) - .await - .unwrap() - .unwrap(); - assert_eq!(leader_prepare.msg.view(), &replica_prepare.view); - assert_eq!( - leader_prepare.msg.justification, - util.new_prepare_qc(|msg| *msg = replica_prepare) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_sanity_yield_leader_prepare_reproposal() { - 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.process_replica_timeout(ctx).await; - 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!( - 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; - assert_eq!(map.len(), 1); - assert_eq!(*map.first_key_value().unwrap().0, replica_prepare); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_bad_chain() { - 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 replica_prepare = util.new_replica_prepare(); - replica_prepare.view.genesis = rng.gen(); - let res = util - .process_replica_prepare(ctx, util.sign(replica_prepare)) - .await; - assert_matches!( - res, - Err(replica_prepare::Error::InvalidMessage( - validator::ReplicaPrepareVerifyError::View(_) - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_non_validator_signer() { - 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 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(replica_prepare::Error::NonValidatorSigner { signer }) => { - assert_eq!(signer, non_validator_key.public()); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_old_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 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, util.sign(replica_prepare)) - .await; - assert_matches!( - res, - Err(replica_prepare::Error::Old { - current_view: ViewNumber(2), - current_phase: Phase::Prepare, - }) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_during_commit() { - 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(); - util.leader.view = util.replica.view; - util.leader.phase = Phase::Commit; - let res = util - .process_replica_prepare(ctx, util.sign(replica_prepare)) - .await; - assert_matches!( - res, - Err(replica_prepare::Error::Old { - current_view, - current_phase: Phase::Commit, - }) => { - assert_eq!(current_view, util.replica.view); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_not_leader_in_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, 2).await; - s.spawn_bg(runner.run(ctx)); - - 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 - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_already_exists() { - 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)); - - util.set_owner_as_view_leader(); - 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 - .unwrap() - .is_none()); - let res = util - .process_replica_prepare(ctx, replica_prepare.clone()) - .await; - assert_matches!(res, Err(replica_prepare::Error::Old { .. })); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_num_received_below_threshold() { - 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)); - - util.set_owner_as_view_leader(); - let replica_prepare = util.new_replica_prepare(); - assert!(util - .process_replica_prepare(ctx, util.sign(replica_prepare)) - .await - .unwrap() - .is_none()); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_invalid_sig() { - 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 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(replica_prepare::Error::InvalidSignature(_))); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -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)); - - 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(replica_prepare::Error::InvalidMessage( - validator::ReplicaPrepareVerifyError::HighQC(_) - )) - ); - Ok(()) - }) - .await - .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(); - 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)); - - 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 - .unwrap(); -} - -/// Check all ReplicaPrepare are included for weight calculation -/// even on different messages for the same view. -#[tokio::test] -async fn replica_prepare_different_messages() { - 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.produce_block(ctx).await; - - let view = util.replica_view(); - let replica_prepare = util.new_replica_prepare(); - - // Create a different proposal for the same view - let proposal = replica_prepare.clone().high_vote.unwrap().proposal; - let mut different_proposal = proposal; - different_proposal.number = different_proposal.number.next(); - - // Create a new ReplicaPrepare with the different proposal - let mut other_replica_prepare = replica_prepare.clone(); - let mut high_vote = other_replica_prepare.high_vote.clone().unwrap(); - high_vote.proposal = different_proposal; - let high_qc = util.new_commit_qc(|msg| { - msg.proposal = different_proposal; - msg.view = view.clone() - }); - - other_replica_prepare.high_vote = Some(high_vote); - other_replica_prepare.high_qc = Some(high_qc); - - let validators = util.keys.len(); - - // half of the validators sign replica_prepare - for i in 0..validators / 2 { - util.process_replica_prepare(ctx, util.keys[i].sign_msg(replica_prepare.clone())) - .await - .unwrap(); - } - - let mut replica_commit_result = None; - // The rest of the validators until threshold sign other_replica_prepare - for i in validators / 2..util.genesis().validators.quorum_threshold() as usize { - replica_commit_result = util - .process_replica_prepare(ctx, util.keys[i].sign_msg(other_replica_prepare.clone())) - .await - .unwrap(); - } - - // That should be enough for a proposal to be committed (even with different proposals) - assert_matches!(replica_commit_result, Some(_)); - - // Check the first proposal has been committed (as it has more votes) - let message = replica_commit_result.unwrap().msg; - assert_eq!(message.proposal, proposal); - Ok(()) - }) - .await - .unwrap(); -} - -/// Check that leader won't accumulate undefined amount of messages if -/// it's spammed with ReplicaPrepare messages for future views -#[tokio::test] -async fn replica_prepare_limit_messages_in_memory() { - 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 mut replica_prepare = util.new_replica_prepare(); - let mut view = util.replica_view(); - // Spam it with 200 messages for different views - for _ in 0..200 { - replica_prepare.view = view.clone(); - let res = util - .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) - .await; - assert_matches!(res, Ok(_)); - // Since we have 2 replicas, we have to send only even numbered views - // to hit the same leader (the other replica will be leader on odd numbered views) - view.number = view.number.next().next(); - } - // Ensure only 1 prepare_qc is in memory, as the previous 199 were discarded each time - // new message is processed - assert_eq!(util.leader.prepare_qcs.len(), 1); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_prepare_filter_functions_test() { - 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 replica_prepare = util.new_replica_prepare(); - let msg = util.sign(validator::ConsensusMsg::ReplicaPrepare( - replica_prepare.clone(), - )); - - // Send a msg with invalid signature - let mut invalid_msg = msg.clone(); - invalid_msg.sig = ctx.rng().gen(); - util.leader_send(invalid_msg); - - // Send a correct message - util.leader_send(msg.clone()); - - // Validate only correct message is received - assert_eq!(util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, msg); - - // Send a msg with view number = 2 - let mut replica_commit_from_view_2 = replica_prepare.clone(); - replica_commit_from_view_2.view.number = ViewNumber(2); - let msg_from_view_2 = util.sign(validator::ConsensusMsg::ReplicaPrepare( - replica_commit_from_view_2, - )); - util.leader_send(msg_from_view_2); - - // Send a msg with view number = 4, will prune message from view 2 - let mut replica_commit_from_view_4 = replica_prepare.clone(); - replica_commit_from_view_4.view.number = ViewNumber(4); - let msg_from_view_4 = util.sign(validator::ConsensusMsg::ReplicaPrepare( - replica_commit_from_view_4, - )); - util.leader_send(msg_from_view_4.clone()); - - // Send a msg with view number = 3, will be discarded, as it is older than message from view 4 - let mut replica_commit_from_view_3 = replica_prepare.clone(); - replica_commit_from_view_3.view.number = ViewNumber(3); - let msg_from_view_3 = util.sign(validator::ConsensusMsg::ReplicaPrepare( - replica_commit_from_view_3, - )); - util.leader_send(msg_from_view_3); - - // Validate only message from view 4 is received - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_view_4 - ); - - // Send a msg from validator 0 - let msg_from_validator_0 = util.keys[0].sign_msg(validator::ConsensusMsg::ReplicaPrepare( - replica_prepare.clone(), - )); - util.leader_send(msg_from_validator_0.clone()); - - // Send a msg from validator 1 - let msg_from_validator_1 = util.keys[1].sign_msg(validator::ConsensusMsg::ReplicaPrepare( - replica_prepare.clone(), - )); - util.leader_send(msg_from_validator_1.clone()); - - //Validate both are present in the inbound_pipe - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_validator_0 - ); - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_validator_1 - ); - - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_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)); - - util.new_leader_commit(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_sanity_yield_leader_commit() { - 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)); - - util.produce_block(ctx).await; - let replica_commit = util.new_replica_commit(ctx).await; - let leader_commit = util - .process_replica_commit(ctx, util.sign(replica_commit.clone())) - .await - .unwrap() - .unwrap(); - assert_eq!( - leader_commit.msg.justification, - util.new_commit_qc(|msg| *msg = replica_commit) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_bad_chain() { - 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 replica_commit = util.new_replica_commit(ctx).await; - replica_commit.view.genesis = rng.gen(); - let res = util - .process_replica_commit(ctx, util.sign(replica_commit)) - .await; - assert_matches!( - res, - Err(replica_commit::Error::InvalidMessage( - validator::ReplicaCommitVerifyError::BadView(_) - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_non_validator_signer() { - 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_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(replica_commit::Error::NonValidatorSigner { signer }) => { - assert_eq!(*signer, non_validator_key.public()); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_old() { - 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 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(replica_commit::Error::Old { current_view, current_phase }) => { - assert_eq!(current_view, util.replica.view); - assert_eq!(current_phase, util.replica.phase); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_not_leader_in_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, 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, util.sign(replica_commit)) - .await; - assert_matches!(res, Err(replica_commit::Error::NotLeaderInView)); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_already_exists() { - 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 replica_commit = util.new_replica_commit(ctx).await; - assert!(util - .process_replica_commit(ctx, util.sign(replica_commit.clone())) - .await - .unwrap() - .is_none()); - - // Processing twice same ReplicaCommit for same view gets DuplicateSignature error - let res = util - .process_replica_commit(ctx, util.sign(replica_commit.clone())) - .await; - assert_matches!(res, Err(replica_commit::Error::Old { .. })); - - // Processing twice different ReplicaCommit for same view gets DuplicateSignature error too - let mut different_replica_commit = replica_commit.clone(); - different_replica_commit.proposal.number = replica_commit.proposal.number.next(); - let res = util - .process_replica_commit(ctx, util.sign(different_replica_commit.clone())) - .await; - assert_matches!(res, Err(replica_commit::Error::Old { .. })); - - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_num_received_below_threshold() { - 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 replica_prepare = util.new_replica_prepare(); - assert!(util - .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) - .await - .unwrap() - .is_none()); - let replica_prepare = util.keys[1].sign_msg(replica_prepare); - let leader_prepare = util - .process_replica_prepare(ctx, replica_prepare) - .await - .unwrap() - .unwrap(); - let replica_commit = util - .process_leader_prepare(ctx, leader_prepare) - .await - .unwrap(); - util.process_replica_commit(ctx, replica_commit.clone()) - .await - .unwrap(); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_invalid_sig() { - 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 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(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(); - 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)); - - 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 - .unwrap(); -} - -/// Proposal should be the same for every ReplicaCommit -/// Check it doesn't fail if one validator sends a different proposal in -/// the ReplicaCommit -#[tokio::test] -async fn replica_commit_different_proposals() { - 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 replica_commit = util.new_replica_commit(ctx).await; - - // Process a modified replica_commit (ie. from a malicious or wrong node) - let mut bad_replica_commit = replica_commit.clone(); - bad_replica_commit.proposal.number = replica_commit.proposal.number.next(); - util.process_replica_commit(ctx, util.sign(bad_replica_commit)) - .await - .unwrap(); - - // The rest of the validators sign the correct one - let mut replica_commit_result = None; - for i in 1..util.keys.len() { - replica_commit_result = util - .process_replica_commit(ctx, util.keys[i].sign_msg(replica_commit.clone())) - .await - .unwrap(); - } - - // Check correct proposal has been committed - assert_matches!( - replica_commit_result, - Some(leader_commit) => { - assert_eq!( - leader_commit.msg.justification.message.proposal, - replica_commit.proposal - ); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -/// Check that leader won't accumulate undefined amount of messages if -/// it's spammed with ReplicaCommit messages for future views -#[tokio::test] -async fn replica_commit_limit_messages_in_memory() { - 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 mut replica_commit = util.new_replica_commit(ctx).await; - let mut view = util.replica_view(); - // Spam it with 200 messages for different views - for _ in 0..200 { - replica_commit.view = view.clone(); - let res = util - .process_replica_commit(ctx, util.sign(replica_commit.clone())) - .await; - assert_matches!(res, Ok(_)); - // Since we have 2 replicas, we have to send only even numbered views - // to hit the same leader (the other replica will be leader on odd numbered views) - view.number = view.number.next().next(); - } - // Ensure only 1 commit_qc is in memory, as the previous 199 were discarded each time - // new message is processed - assert_eq!(util.leader.commit_qcs.len(), 1); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn replica_commit_filter_functions_test() { - 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 replica_commit = util.new_replica_commit(ctx).await; - let msg = util.sign(validator::ConsensusMsg::ReplicaCommit( - replica_commit.clone(), - )); - - // Send a msg with invalid signature - let mut invalid_msg = msg.clone(); - invalid_msg.sig = ctx.rng().gen(); - util.leader_send(invalid_msg); - - // Send a correct message - util.leader_send(msg.clone()); - - // Validate only correct message is received - assert_eq!(util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, msg); - - // Send a msg with view number = 2 - let mut replica_commit_from_view_2 = replica_commit.clone(); - replica_commit_from_view_2.view.number = ViewNumber(2); - let msg_from_view_2 = util.sign(validator::ConsensusMsg::ReplicaCommit( - replica_commit_from_view_2, - )); - util.leader_send(msg_from_view_2); - - // Send a msg with view number = 4, will prune message from view 2 - let mut replica_commit_from_view_4 = replica_commit.clone(); - replica_commit_from_view_4.view.number = ViewNumber(4); - let msg_from_view_4 = util.sign(validator::ConsensusMsg::ReplicaCommit( - replica_commit_from_view_4, - )); - util.leader_send(msg_from_view_4.clone()); - - // Send a msg with view number = 3, will be discarded, as it is older than message from view 4 - let mut replica_commit_from_view_3 = replica_commit.clone(); - replica_commit_from_view_3.view.number = ViewNumber(3); - let msg_from_view_3 = util.sign(validator::ConsensusMsg::ReplicaCommit( - replica_commit_from_view_3, - )); - util.leader_send(msg_from_view_3); - - // Validate only message from view 4 is received - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_view_4 - ); - - // Send a msg from validator 0 - let msg_from_validator_0 = util.keys[0].sign_msg(validator::ConsensusMsg::ReplicaCommit( - replica_commit.clone(), - )); - util.leader_send(msg_from_validator_0.clone()); - - // Send a msg from validator 1 - let msg_from_validator_1 = util.keys[1].sign_msg(validator::ConsensusMsg::ReplicaCommit( - replica_commit.clone(), - )); - util.leader_send(msg_from_validator_1.clone()); - - //Validate both are present in the inbound_pipe - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_validator_0 - ); - assert_eq!( - util.leader.inbound_pipe.recv(ctx).await.unwrap().msg, - msg_from_validator_1 - ); - - Ok(()) - }) - .await - .unwrap(); -} diff --git a/node/actors/bft/src/lib.rs b/node/actors/bft/src/lib.rs index 89061564..0075b0df 100644 --- a/node/actors/bft/src/lib.rs +++ b/node/actors/bft/src/lib.rs @@ -1,35 +1,20 @@ -//! # Consensus -//! This crate implements the Fastest-HotStuff algorithm that is described in an upcoming paper -//! It is a two-phase unchained consensus with quadratic view change (in number of authenticators, in number of -//! messages it is linear) and optimistic responsiveness. -//! -//! ## Node set -//! Right now, we assume that we have a static node set. In other words, we are running in proof-of-authority. When this repo is updated -//! to proof-of-stake, we will have a dynamic node set. -//! -//! ## Resources -//! - [Fast-HotStuff paper](https://arxiv.org/pdf/2010.11454.pdf) -//! - [HotStuff paper](https://arxiv.org/pdf/1803.05069.pdf) -//! - [HotStuff-2 paper](https://eprint.iacr.org/2023/397.pdf) -//! - [Notes on modern consensus algorithms](https://timroughgarden.github.io/fob21/andy.pdf) -//! - [Blog post comparing several consensus algorithms](https://decentralizedthoughts.github.io/2023-04-01-hotstuff-2/) -//! - Blog posts explaining [safety](https://seafooler.com/2022/01/24/understanding-safety-hotstuff/) and [responsiveness](https://seafooler.com/2022/04/02/understanding-responsiveness-hotstuff/) +//! This crate contains the consensus actor, which is responsible for handling the logic that allows us to reach agreement on blocks. +//! It uses a new cosnensus algorithm developed at Matter Labs, called ChonkyBFT. You can find the specification of the algorithm [here](../../../../spec). use crate::io::{InputMessage, OutputMessage}; use anyhow::Context; pub use config::Config; use std::sync::Arc; use tracing::Instrument; -use zksync_concurrency::{ctx, error::Wrap as _, oneshot, scope}; -use zksync_consensus_network::io::ConsensusReq; +use zksync_concurrency::{ctx, error::Wrap as _, scope, sync}; use zksync_consensus_roles::validator; use zksync_consensus_utils::pipe::ActorPipe; +/// This module contains the implementation of the ChonkyBFT algorithm. +mod chonky_bft; mod config; pub mod io; -mod leader; mod metrics; -mod replica; pub mod testonly; #[cfg(test)] mod tests; @@ -55,9 +40,6 @@ pub trait PayloadManager: std::fmt::Debug + Send + Sync { ) -> ctx::Result<()>; } -/// Channel through which bft actor sends network messages. -pub(crate) type OutputSender = ctx::channel::UnboundedSender; - impl Config { /// Starts the bft actor. It will start running, processing incoming messages and /// sending output messages. @@ -71,56 +53,39 @@ impl Config { genesis.verify().context("genesis().verify()")?; if let Some(prev) = genesis.first_block.prev() { - tracing::info!("Waiting for the pre-genesis blocks to be persisted"); + tracing::info!("Waiting for the pre-fork blocks to be persisted"); if let Err(ctx::Canceled) = self.block_store.wait_until_persisted(ctx, prev).await { return Ok(()); } } let cfg = Arc::new(self); - let (leader, leader_send) = leader::StateMachine::new(ctx, cfg.clone(), pipe.send.clone()); + let (proposer_sender, proposer_receiver) = sync::watch::channel(None); let (replica, replica_send) = - replica::StateMachine::start(ctx, cfg.clone(), pipe.send.clone()).await?; + chonky_bft::StateMachine::start(ctx, cfg.clone(), pipe.send.clone(), proposer_sender) + .await?; let res = scope::run!(ctx, |ctx, s| async { - let prepare_qc_recv = leader.prepare_qc.subscribe(); - s.spawn_bg(async { replica.run(ctx).await.wrap("replica.run()") }); - s.spawn_bg(async { leader.run(ctx).await.wrap("leader.run()") }); s.spawn_bg(async { - leader::StateMachine::run_proposer(ctx, &cfg, prepare_qc_recv, &pipe.send) + chonky_bft::proposer::run_proposer(ctx, cfg.clone(), pipe.send, proposer_receiver) .await .wrap("run_proposer()") }); tracing::info!("Starting consensus actor {:?}", cfg.secret_key.public()); - // This is the infinite loop where the consensus actually runs. The validator waits for either - // a message from the network or for a timeout, and processes each accordingly. + // This is the infinite loop where the consensus actually runs. The validator waits for + // a message from the network and processes it accordingly. loop { async { - let InputMessage::Network(req) = pipe + let InputMessage::Network(msg) = pipe .recv .recv(ctx) .instrument(tracing::info_span!("wait_for_message")) .await?; - use validator::ConsensusMsg as M; - match &req.msg.msg { - M::ReplicaPrepare(_) => { - // This is a hacky way to do a clone. This is necessary since we don't want to derive - // Clone for ConsensusReq. When we change to ChonkyBFT this will be removed anyway. - let (ack, _) = oneshot::channel(); - let new_req = ConsensusReq { - msg: req.msg.clone(), - ack, - }; - replica_send.send(new_req); - leader_send.send(req); - } - M::ReplicaCommit(_) => leader_send.send(req), - M::LeaderPrepare(_) | M::LeaderCommit(_) => replica_send.send(req), - } + replica_send.send(msg); ctx::Ok(()) } diff --git a/node/actors/bft/src/metrics.rs b/node/actors/bft/src/metrics.rs index ea23c711..248b57c9 100644 --- a/node/actors/bft/src/metrics.rs +++ b/node/actors/bft/src/metrics.rs @@ -12,14 +12,14 @@ const PAYLOAD_SIZE_BUCKETS: Buckets = Buckets::exponential( #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue)] #[metrics(rename_all = "snake_case")] pub(crate) enum ConsensusMsgLabel { - /// Label for a `LeaderPrepare` message. - LeaderPrepare, - /// Label for a `LeaderCommit` message. - LeaderCommit, - /// Label for a `ReplicaPrepare` message. - ReplicaPrepare, + /// Label for a `LeaderProposal` message. + LeaderProposal, /// Label for a `ReplicaCommit` message. ReplicaCommit, + /// Label for a `ReplicaTimeout` message. + ReplicaTimeout, + /// Label for a `ReplicaNewView` message. + ReplicaNewView, } impl ConsensusMsgLabel { @@ -53,26 +53,29 @@ pub(crate) struct ProcessingLatencyLabels { #[derive(Debug, Metrics)] #[metrics(prefix = "consensus")] pub(crate) struct ConsensusMetrics { + /// Number of the current view of the replica. + #[metrics(unit = Unit::Seconds)] + pub(crate) replica_view_number: Gauge, + /// Number of the last finalized block observed by the node. + pub(crate) finalized_block_number: Gauge, /// Size of the proposed payload in bytes. #[metrics(buckets = PAYLOAD_SIZE_BUCKETS, unit = Unit::Bytes)] - pub(crate) leader_proposal_payload_size: Histogram, - /// Latency of the commit phase observed by the leader. + pub(crate) proposal_payload_size: Histogram, + /// Latency of receiving a proposal as observed by the replica. Measures from + /// the start of the view until we have a verified proposal. #[metrics(buckets = Buckets::exponential(0.01..=20.0, 1.5), unit = Unit::Seconds)] - pub(crate) leader_commit_phase_latency: Histogram, - /// Currently set timeout after which replica will proceed to the next view. - #[metrics(unit = Unit::Seconds)] - pub(crate) replica_view_timeout: Gauge, + pub(crate) proposal_latency: Histogram, + /// Latency of committing to a block as observed by the replica. Measures from + /// the start of the view until we send a commit vote. + #[metrics(buckets = Buckets::exponential(0.01..=20.0, 1.5), unit = Unit::Seconds)] + pub(crate) commit_latency: Histogram, + /// Latency of a single view as observed by the replica. Measures from + /// the start of the view until the start of the next. + #[metrics(buckets = Buckets::exponential(0.01..=20.0, 1.5), unit = Unit::Seconds)] + pub(crate) view_latency: Histogram, /// Latency of processing messages by the replicas. #[metrics(buckets = Buckets::LATENCIES, unit = Unit::Seconds)] - pub(crate) replica_processing_latency: Family>, - /// Latency of processing messages by the leader. - #[metrics(buckets = Buckets::LATENCIES, unit = Unit::Seconds)] - pub(crate) leader_processing_latency: Family>, - /// Number of the last finalized block observed by the node. - pub(crate) finalized_block_number: Gauge, - /// Number of the current view of the replica. - #[metrics(unit = Unit::Seconds)] - pub(crate) replica_view_number: Gauge, + pub(crate) message_processing_latency: Family>, } /// Global instance of [`ConsensusMetrics`]. diff --git a/node/actors/bft/src/replica/leader_commit.rs b/node/actors/bft/src/replica/leader_commit.rs deleted file mode 100644 index d60b99b3..00000000 --- a/node/actors/bft/src/replica/leader_commit.rs +++ /dev/null @@ -1,100 +0,0 @@ -//! Handler of a LeaderCommit message. -use super::StateMachine; -use zksync_concurrency::{ctx, error::Wrap}; -use zksync_consensus_roles::validator; - -/// Errors that can occur when processing a "leader commit" message. -#[derive(Debug, thiserror::Error)] -pub(crate) enum Error { - /// Invalid leader. - #[error("bad leader: got {got:?}, want {want:?}")] - BadLeader { - /// Received leader. - got: validator::PublicKey, - /// Correct leader. - want: validator::PublicKey, - }, - /// Past view of phase. - #[error("past view/phase (current view: {current_view:?}, current phase: {current_phase:?})")] - Old { - /// Current view. - current_view: validator::ViewNumber, - /// Current phase. - current_phase: validator::Phase, - }, - /// Invalid message signature. - #[error("invalid signature: {0:#}")] - InvalidSignature(#[source] anyhow::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), -} - -impl Wrap for Error { - fn with_wrap C>( - self, - f: F, - ) -> Self { - match self { - Error::Internal(err) => Error::Internal(err.with_wrap(f)), - err => err, - } - } -} - -impl StateMachine { - /// Processes a leader commit message. We can approve this leader message even if we - /// don't have the block proposal stored. It is enough to see the justification. - pub(crate) async fn process_leader_commit( - &mut self, - ctx: &ctx::Ctx, - signed_message: validator::Signed, - ) -> Result<(), Error> { - // ----------- Checking origin of the message -------------- - - // Unwrap message. - let message = &signed_message.msg; - let author = &signed_message.key; - - // Check that it comes from the correct leader. - let leader = self.config.genesis().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 (message.view().number, validator::Phase::Commit) < (self.view, self.phase) { - return Err(Error::Old { - current_view: self.view, - current_phase: self.phase, - }); - } - - // ----------- Checking the signed part of the message -------------- - - // 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. -------------- - - // Try to create a finalized block with this CommitQC and our block proposal cache. - self.save_block(ctx, &message.justification) - .await - .wrap("save_block()")?; - - // Start a new view. But first we skip to the view of this message. - 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 deleted file mode 100644 index 55e43cf0..00000000 --- a/node/actors/bft/src/replica/leader_prepare.rs +++ /dev/null @@ -1,189 +0,0 @@ -//! Handler of a LeaderPrepare message. -use super::StateMachine; -use zksync_concurrency::{ctx, error::Wrap}; -use zksync_consensus_network::io::{ConsensusInputMessage, Target}; -use zksync_consensus_roles::validator::{self, BlockNumber}; - -/// Errors that can occur when processing a "leader prepare" message. -#[derive(Debug, thiserror::Error)] -pub(crate) enum Error { - /// Invalid leader. - #[error( - "invalid leader (correct leader: {correct_leader:?}, received leader: {received_leader:?})" - )] - InvalidLeader { - /// Correct leader. - correct_leader: validator::PublicKey, - /// Received leader. - received_leader: validator::PublicKey, - }, - /// Message for a past view or phase. - #[error( - "message for a past view / phase (current view: {current_view:?}, current phase: {current_phase:?})" - )] - Old { - /// Current view. - current_view: validator::ViewNumber, - /// Current phase. - current_phase: validator::Phase, - }, - /// Invalid message signature. - #[error("invalid signature: {0:#}")] - InvalidSignature(#[source] anyhow::Error), - /// Invalid message. - #[error("invalid message: {0:#}")] - InvalidMessage(#[source] validator::LeaderPrepareVerifyError), - /// Leader proposed a block that was already pruned from replica's storage. - #[error("leader proposed a block that was already pruned from replica's storage")] - ProposalAlreadyPruned, - /// Oversized payload. - #[error("block proposal with an oversized payload (payload size: {payload_size})")] - ProposalOversizedPayload { - /// Size of the payload. - payload_size: usize, - }, - /// Invalid payload. - #[error("invalid payload: {0:#}")] - ProposalInvalidPayload(#[source] anyhow::Error), - /// Previous payload missing. - #[error("previous block proposal payload missing from store (block number: {prev_number})")] - MissingPreviousPayload { - /// The number of the missing block - prev_number: BlockNumber, - }, - /// Internal error. Unlike other error types, this one isn't supposed to be easily recoverable. - #[error(transparent)] - Internal(#[from] ctx::Error), -} - -impl Wrap for Error { - fn with_wrap C>( - self, - f: F, - ) -> Self { - match self { - Error::Internal(err) => Error::Internal(err.with_wrap(f)), - err => err, - } - } -} - -impl StateMachine { - /// Processes a leader prepare message. - pub(crate) async fn process_leader_prepare( - &mut self, - ctx: &ctx::Ctx, - signed_message: validator::Signed, - ) -> Result<(), Error> { - // ----------- Checking origin of the message -------------- - - // Unwrap message. - let message = &signed_message.msg; - let author = &signed_message.key; - let view = message.view().number; - - // Check that it comes from the correct leader. - let leader = self.config.genesis().view_leader(view); - if author != &leader { - return Err(Error::InvalidLeader { - correct_leader: leader, - received_leader: author.clone(), - }); - } - - // If the message is from the "past", we discard it. - if (view, validator::Phase::Prepare) < (self.view, self.phase) { - return Err(Error::Old { - current_view: self.view, - current_phase: self.phase, - }); - } - - // Replica MUSTN'T vote for blocks which have been already pruned for storage. - // (because it won't be able to persist and broadcast them once finalized). - // TODO(gprusak): it should never happen, we should add safety checks to prevent - // pruning blocks not known to be finalized. - if message.proposal.number < self.config.block_store.queued().first { - return Err(Error::ProposalAlreadyPruned); - } - - // ----------- Checking the message -------------- - - signed_message.verify().map_err(Error::InvalidSignature)?; - message - .verify(self.config.genesis()) - .map_err(Error::InvalidMessage)?; - - let high_qc = message.justification.high_qc(); - - 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()")?; - } - - // 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(), - }); - } - - 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.with_deadline(self.timeout_deadline), prev) - .await - .map_err(|_| Error::MissingPreviousPayload { prev_number: prev })?; - } - 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), - }); - } - } - - // ----------- All checks finished. Now we process the message. -------------- - - // Create our commit vote. - let commit_vote = validator::ReplicaCommit { - view: message.view().clone(), - proposal: message.proposal, - }; - - // Update the state machine. - self.view = message.view().number; - self.phase = validator::Phase::Commit; - self.high_vote = Some(commit_vote.clone()); - // If we received a new block proposal, store it in our cache. - if let Some(payload) = &message.proposal_payload { - self.block_proposal_cache - .entry(message.proposal.number) - .or_default() - .insert(payload.hash(), payload.clone()); - } - - // Backup our state. - self.backup_state(ctx).await.wrap("backup_state()")?; - - // Send the replica message to the leader. - let output_message = ConsensusInputMessage { - message: self - .config - .secret_key - .sign_msg(validator::ConsensusMsg::ReplicaCommit(commit_vote)), - recipient: Target::Validator(author.clone()), - }; - self.outbound_pipe.send(output_message.into()); - - Ok(()) - } -} diff --git a/node/actors/bft/src/replica/mod.rs b/node/actors/bft/src/replica/mod.rs deleted file mode 100644 index 640f044b..00000000 --- a/node/actors/bft/src/replica/mod.rs +++ /dev/null @@ -1,15 +0,0 @@ -//! Implements the replica role in the Fastest-HotStuff consensus algorithm. The replica is the role that validates -//! proposals, votes for them and finalizes them. It basically drives the consensus forward. Note that our consensus -//! node will perform both the replica and leader roles simultaneously. - -mod block; -pub(crate) mod leader_commit; -pub(crate) mod leader_prepare; -mod new_view; -pub(crate) mod replica_prepare; -mod state_machine; -#[cfg(test)] -mod tests; -mod timer; - -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 deleted file mode 100644 index 16403136..00000000 --- a/node/actors/bft/src/replica/new_view.rs +++ /dev/null @@ -1,48 +0,0 @@ -use super::StateMachine; -use crate::metrics; -use zksync_concurrency::{ctx, error::Wrap as _}; -use zksync_consensus_network::io::{ConsensusInputMessage, Target}; -use zksync_consensus_roles::validator; - -impl StateMachine { - /// This blocking method is used whenever we start a new view. - pub(crate) async fn start_new_view(&mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { - // Update the state machine. - self.view = self.view.next(); - tracing::info!("Starting view {}", self.view); - metrics::METRICS.replica_view_number.set(self.view.0); - - self.phase = validator::Phase::Prepare; - 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()")?; - - // Send the replica message. - let output_message = ConsensusInputMessage { - message: self - .config - .secret_key - .sign_msg(validator::ConsensusMsg::ReplicaPrepare( - validator::ReplicaPrepare { - view: validator::View { - genesis: self.config.genesis().hash(), - number: self.view, - }, - high_vote: self.high_vote.clone(), - high_qc: self.high_qc.clone(), - }, - )), - recipient: Target::Broadcast, - }; - self.outbound_pipe.send(output_message.into()); - - // Reset the timer. - self.reset_timer(ctx); - Ok(()) - } -} diff --git a/node/actors/bft/src/replica/replica_prepare.rs b/node/actors/bft/src/replica/replica_prepare.rs deleted file mode 100644 index 74b09ad9..00000000 --- a/node/actors/bft/src/replica/replica_prepare.rs +++ /dev/null @@ -1,104 +0,0 @@ -//! Handler of a ReplicaPrepare message. -use super::StateMachine; -use zksync_concurrency::{ctx, error::Wrap}; -use zksync_consensus_roles::validator; - -/// Errors that can occur when processing a "replica prepare" message. -#[derive(Debug, thiserror::Error)] -pub(crate) enum Error { - /// Message signer isn't part of the validator set. - #[error("Message signer isn't part of the validator set (signer: {signer:?})")] - NonValidatorSigner { - /// Signer of the message. - signer: validator::PublicKey, - }, - /// Past view or phase. - #[error("past view/phase (current view: {current_view:?}, current phase: {current_phase:?})")] - Old { - /// Current view. - current_view: validator::ViewNumber, - /// Current phase. - current_phase: validator::Phase, - }, - /// Invalid message signature. - #[error("invalid signature: {0:#}")] - InvalidSignature(#[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), -} - -impl Wrap for Error { - fn with_wrap C>( - self, - f: F, - ) -> Self { - match self { - Error::Internal(err) => Error::Internal(err.with_wrap(f)), - err => err, - } - } -} - -impl StateMachine { - /// Processes `ReplicaPrepare` message. - pub(crate) async fn process_replica_prepare( - &mut self, - ctx: &ctx::Ctx, - signed_message: validator::Signed, - ) -> Result<(), Error> { - // ----------- Checking origin of the message -------------- - - // Unwrap message. - let message = signed_message.msg.clone(); - let author = &signed_message.key; - - // Check that the message signer is in the validator set. - if !self.config.genesis().validators.contains(author) { - return Err(Error::NonValidatorSigner { - signer: author.clone(), - }); - } - - // We only accept this type of message from the future. - if message.view.number <= self.view { - return Err(Error::Old { - current_view: self.view, - current_phase: self.phase, - }); - } - - // ----------- Checking the signed part of the message -------------- - - // Check the signature on the message. - signed_message.verify().map_err(Error::InvalidSignature)?; - - // Extract the QC and verify it. - let Some(high_qc) = message.high_qc else { - return Ok(()); - }; - - high_qc.verify(self.config.genesis()).map_err(|err| { - Error::InvalidMessage(validator::ReplicaPrepareVerifyError::HighQC(err)) - })?; - - // ----------- All checks finished. Now we process the message. -------------- - - let qc_view = high_qc.view().number; - - // Try to create a finalized block with this CommitQC and our block proposal cache. - // It will also update our high QC, if necessary. - self.save_block(ctx, &high_qc).await.wrap("save_block()")?; - - // Skip to a new view, if necessary. - if qc_view >= self.view { - self.view = qc_view; - self.start_new_view(ctx).await.wrap("start_new_view()")?; - } - - Ok(()) - } -} diff --git a/node/actors/bft/src/replica/state_machine.rs b/node/actors/bft/src/replica/state_machine.rs deleted file mode 100644 index 238cc0e3..00000000 --- a/node/actors/bft/src/replica/state_machine.rs +++ /dev/null @@ -1,255 +0,0 @@ -use crate::{metrics, Config, OutputSender}; -use std::{ - collections::{BTreeMap, HashMap}, - sync::Arc, -}; -use zksync_concurrency::{ - ctx, - error::Wrap as _, - metrics::LatencyHistogramExt as _, - sync::{self, prunable_mpsc::SelectionFunctionResult}, - time, -}; -use zksync_consensus_network::io::ConsensusReq; -use zksync_consensus_roles::{validator, validator::ConsensusMsg}; -use zksync_consensus_storage as storage; - -/// The StateMachine struct contains the state of the replica. This is the most complex state machine and is responsible -/// for validating and voting on blocks. When participating in consensus we are always a replica. -#[derive(Debug)] -pub(crate) struct StateMachine { - /// Consensus configuration and output channel. - pub(crate) config: Arc, - /// Pipe through which replica sends network messages. - pub(super) outbound_pipe: OutputSender, - /// Pipe through which replica receives network requests. - inbound_pipe: sync::prunable_mpsc::Receiver, - /// The current view number. - pub(crate) view: validator::ViewNumber, - /// The current phase. - pub(crate) phase: validator::Phase, - /// The highest block proposal that the replica has committed to. - pub(crate) high_vote: Option, - /// The highest commit quorum certificate known to the replica. - pub(crate) high_qc: Option, - /// A cache of the received block proposals. - pub(crate) block_proposal_cache: - BTreeMap>, - /// The deadline to receive an input message. - pub(crate) timeout_deadline: time::Deadline, -} - -impl StateMachine { - /// Creates a new [`StateMachine`] instance, attempting to recover a past state from the storage module, - /// otherwise initializes the state machine with the current head block. - /// - /// Returns a tuple containing: - /// * The newly created [`StateMachine`] instance. - /// * A sender handle that should be used to send values to be processed by the instance, asynchronously. - pub(crate) async fn start( - ctx: &ctx::Ctx, - config: Arc, - outbound_pipe: OutputSender, - ) -> ctx::Result<(Self, sync::prunable_mpsc::Sender)> { - 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 - .entry(proposal.number) - .or_default() - .insert(proposal.payload.hash(), proposal.payload); - } - - let (send, recv) = sync::prunable_mpsc::channel( - StateMachine::inbound_filter_predicate, - StateMachine::inbound_selection_function, - ); - - let mut this = Self { - config, - outbound_pipe, - inbound_pipe: recv, - view: backup.view, - phase: backup.phase, - high_vote: backup.high_vote, - high_qc: backup.high_qc, - block_proposal_cache, - timeout_deadline: time::Deadline::Infinite, - }; - - // We need to start the replica before processing inputs. - this.start_new_view(ctx).await.wrap("start_new_view()")?; - - Ok((this, send)) - } - - /// Runs a loop to process incoming messages (may be `None` if the channel times out while waiting for a message). - /// This is the main entry point for the state machine, - /// potentially triggering state modifications and message sending to the executor. - pub(crate) async fn run(mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { - loop { - let recv = self - .inbound_pipe - .recv(&ctx.with_deadline(self.timeout_deadline)) - .await; - - // Check for non-timeout cancellation. - if !ctx.is_active() { - return Ok(()); - } - - // Check for timeout. - let Some(req) = recv.ok() else { - self.start_new_view(ctx).await?; - continue; - }; - - let now = ctx.now(); - let label = match &req.msg.msg { - ConsensusMsg::ReplicaPrepare(_) => { - let res = match self - .process_replica_prepare(ctx, req.msg.cast().unwrap()) - .await - .wrap("process_replica_prepare()") - { - Ok(()) => Ok(()), - Err(err) => { - match err { - super::replica_prepare::Error::Internal(e) => { - tracing::error!( - "process_replica_prepare: internal error: {e:#}" - ); - return Err(e); - } - super::replica_prepare::Error::Old { .. } => { - tracing::debug!("process_replica_prepare: {err:#}"); - } - _ => { - tracing::warn!("process_replica_prepare: {err:#}"); - } - } - Err(()) - } - }; - metrics::ConsensusMsgLabel::ReplicaPrepare.with_result(&res) - } - ConsensusMsg::LeaderPrepare(_) => { - let res = match self - .process_leader_prepare(ctx, req.msg.cast().unwrap()) - .await - .wrap("process_leader_prepare()") - { - Ok(()) => Ok(()), - Err(err) => { - match err { - super::leader_prepare::Error::Internal(e) => { - tracing::error!( - "process_leader_prepare: internal error: {e:#}" - ); - return Err(e); - } - super::leader_prepare::Error::Old { .. } => { - tracing::info!("process_leader_prepare: {err:#}"); - } - _ => { - tracing::warn!("process_leader_prepare: {err:#}"); - } - } - Err(()) - } - }; - metrics::ConsensusMsgLabel::LeaderPrepare.with_result(&res) - } - ConsensusMsg::LeaderCommit(_) => { - let res = match self - .process_leader_commit(ctx, req.msg.cast().unwrap()) - .await - .wrap("process_leader_commit()") - { - Ok(()) => Ok(()), - Err(err) => { - match err { - super::leader_commit::Error::Internal(e) => { - tracing::error!("process_leader_commit: internal error: {e:#}"); - return Err(e); - } - super::leader_commit::Error::Old { .. } => { - tracing::info!("process_leader_commit: {err:#}"); - } - _ => { - tracing::warn!("process_leader_commit: {err:#}"); - } - } - Err(()) - } - }; - metrics::ConsensusMsgLabel::LeaderCommit.with_result(&res) - } - _ => unreachable!(), - }; - metrics::METRICS.replica_processing_latency[&label].observe_latency(ctx.now() - now); - - // Notify network actor that the message has been processed. - // Ignore sending error. - let _ = req.ack.send(()); - } - } - - /// Backups the replica state to disk. - pub(crate) async fn backup_state(&self, ctx: &ctx::Ctx) -> ctx::Result<()> { - let mut proposals = vec![]; - for (number, payloads) in &self.block_proposal_cache { - proposals.extend(payloads.values().map(|p| storage::Proposal { - number: *number, - payload: p.clone(), - })); - } - let backup = storage::ReplicaState { - view: self.view, - phase: self.phase, - high_vote: self.high_vote.clone(), - high_qc: self.high_qc.clone(), - proposals, - }; - self.config - .replica_store - .set_state(ctx, &backup) - .await - .wrap("put_replica_state")?; - Ok(()) - } - - fn inbound_filter_predicate(new_req: &ConsensusReq) -> bool { - // Verify message signature - new_req.msg.verify().is_ok() - } - - fn inbound_selection_function( - old_req: &ConsensusReq, - new_req: &ConsensusReq, - ) -> SelectionFunctionResult { - if old_req.msg.key != new_req.msg.key { - return SelectionFunctionResult::Keep; - } - - match (&old_req.msg.msg, &new_req.msg.msg) { - (ConsensusMsg::LeaderPrepare(old), ConsensusMsg::LeaderPrepare(new)) => { - // Discard older message - if old.view().number < new.view().number { - SelectionFunctionResult::DiscardOld - } else { - SelectionFunctionResult::DiscardNew - } - } - (ConsensusMsg::LeaderCommit(old), ConsensusMsg::LeaderCommit(new)) => { - // Discard older message - if old.view().number < new.view().number { - SelectionFunctionResult::DiscardOld - } else { - SelectionFunctionResult::DiscardNew - } - } - _ => SelectionFunctionResult::Keep, - } - } -} diff --git a/node/actors/bft/src/replica/tests.rs b/node/actors/bft/src/replica/tests.rs deleted file mode 100644 index afb2a8aa..00000000 --- a/node/actors/bft/src/replica/tests.rs +++ /dev/null @@ -1,625 +0,0 @@ -use super::{leader_commit, leader_prepare}; -use crate::{ - testonly, - testonly::ut_harness::{UTHarness, MAX_PAYLOAD_SIZE}, -}; -use assert_matches::assert_matches; -use rand::Rng; -use zksync_concurrency::{ctx, scope}; -use zksync_consensus_roles::validator::{ - self, CommitQC, Payload, PrepareQC, ReplicaCommit, ReplicaPrepare, -}; - -/// Sanity check of the happy path. -#[tokio::test] -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)); - util.produce_block(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Sanity check of block production with reproposal. -#[tokio::test] -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_leader_commit(ctx).await; - util.process_replica_timeout(ctx).await; - util.produce_block(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_bad_chain() { - 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; - leader_prepare.justification.view.genesis = 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( - validator::PrepareQCVerifyError::View(_) - ) - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_sanity_yield_replica_commit() { - 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 leader_prepare = util.new_leader_prepare(ctx).await; - let replica_commit = util - .process_leader_prepare(ctx, util.sign(leader_prepare.clone())) - .await - .unwrap(); - assert_eq!( - replica_commit.msg, - ReplicaCommit { - view: leader_prepare.view().clone(), - proposal: leader_prepare.proposal, - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_invalid_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 replica_prepare = util.new_replica_prepare(); - assert!(util - .process_replica_prepare(ctx, util.sign(replica_prepare.clone())) - .await - .unwrap() - .is_none()); - - 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.justification.view.number = leader_prepare.justification.view.number.next(); - assert_ne!( - util.view_leader(leader_prepare.view().number), - util.keys[0].public() - ); - - let res = util - .process_leader_prepare(ctx, util.sign(leader_prepare)) - .await; - assert_matches!( - res, - Err(leader_prepare::Error::InvalidLeader { correct_leader, received_leader }) => { - assert_eq!(correct_leader, util.keys[1].public()); - assert_eq!(received_leader, util.keys[0].public()); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_old_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 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 }) => { - assert_eq!(current_view, util.replica.view); - assert_eq!(current_phase, util.replica.phase); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_pruned_block() { - 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; - // We assume default replica state and nontrivial `genesis.fork.first_block` here. - leader_prepare.proposal.number = util - .replica - .config - .block_store - .queued() - .first - .prev() - .unwrap(); - let res = util - .process_leader_prepare(ctx, util.sign(leader_prepare)) - .await; - assert_matches!(res, Err(leader_prepare::Error::ProposalAlreadyPruned)); - Ok(()) - }) - .await - .unwrap(); -} - -/// Tests that `WriteBlockStore::verify_payload` is applied before signing a vote. -#[tokio::test] -async fn leader_prepare_invalid_payload() { - 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_with_payload(ctx, 1, Box::new(testonly::RejectPayload)).await; - s.spawn_bg(runner.run(ctx)); - - 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()) - .unwrap(); - let block = validator::FinalBlock { - payload: leader_prepare.proposal_payload.clone().unwrap(), - justification, - }; - util.replica - .config - .block_store - .queue_block(ctx, block.into()) - .await - .unwrap(); - - let res = util - .process_leader_prepare(ctx, util.sign(leader_prepare)) - .await; - assert_matches!(res, Err(leader_prepare::Error::ProposalInvalidPayload(..))); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_invalid_sig() { - 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 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(..))); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_invalid_prepare_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; - 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 - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_proposal_oversized_payload() { - 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 payload_oversize = MAX_PAYLOAD_SIZE + 1; - 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, util.sign(leader_prepare)) - .await; - assert_matches!( - res, - Err(leader_prepare::Error::ProposalOversizedPayload{ payload_size }) => { - assert_eq!(payload_size, payload_oversize); - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_proposal_mismatched_payload() { - 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; - leader_prepare.proposal_payload = Some(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::ProposalMismatchedPayload - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -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)); - - 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::InvalidMessage( - validator::LeaderPrepareVerifyError::ProposalWhenPreviousNotFinalized - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -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)); - - 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(); -} - -#[tokio::test] -async fn leader_prepare_reproposal_without_quorum() { - 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_many(ctx).await; - s.spawn_bg(runner.run(ctx)); - - 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 - .unwrap(); -} - -#[tokio::test] -async fn leader_prepare_reproposal_when_finalized() { - 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)); - - 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 res = util - .process_leader_prepare(ctx, util.sign(leader_prepare)) - .await; - assert_matches!( - res, - Err(leader_prepare::Error::InvalidMessage( - validator::LeaderPrepareVerifyError::ReproposalWhenFinalized - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -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)); - - 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 expected 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(); - 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 leader_commit = util.new_leader_commit(ctx).await; - let replica_prepare = util - .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 { - view, - high_vote: Some(leader_commit.justification.message.clone()), - high_qc: Some(leader_commit.justification), - } - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_commit_bad_chain() { - 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_commit = util.new_leader_commit(ctx).await; - leader_commit.justification.message.view.genesis = rng.gen(); - let res = util - .process_leader_commit(ctx, util.sign(leader_commit)) - .await; - assert_matches!( - res, - Err(leader_commit::Error::InvalidMessage( - validator::CommitQCVerifyError::InvalidMessage( - validator::ReplicaCommitVerifyError::BadView(_) - ) - )) - ); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -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 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::BadLeader { .. })); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_commit_invalid_sig() { - 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 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 { .. })); - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn leader_commit_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 mut leader_commit = util.new_leader_commit(ctx).await; - leader_commit.justification.signature = rng.gen(); - let res = util - .process_leader_commit(ctx, util.sign(leader_commit)) - .await; - assert_matches!( - res, - Err(leader_commit::Error::InvalidMessage( - validator::CommitQCVerifyError::BadSignature(..) - )) - ); - Ok(()) - }) - .await - .unwrap(); -} diff --git a/node/actors/bft/src/replica/timer.rs b/node/actors/bft/src/replica/timer.rs deleted file mode 100644 index 75570d2d..00000000 --- a/node/actors/bft/src/replica/timer.rs +++ /dev/null @@ -1,35 +0,0 @@ -use super::StateMachine; -use crate::metrics; -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(2000); - /// Max duration of the timeout. - /// Consensus is unusable with this range of timeout anyway, - /// however to make debugging easier we bound it to a specific value. - pub(crate) const MAX_DURATION: time::Duration = time::Duration::seconds(1000000); - - /// Resets the timer. On every timeout we double the duration, starting from a given base duration. - /// This is a simple exponential backoff. - pub(crate) fn reset_timer(&mut self, ctx: &ctx::Ctx) { - let final_view = match self.high_qc.as_ref() { - Some(qc) => qc.view().number.next(), - None => validator::ViewNumber(0), - }; - let f = self - .view - .0 - .saturating_sub(final_view.0) - .try_into() - .unwrap_or(u32::MAX); - let f = 2u64.saturating_pow(f).try_into().unwrap_or(i32::MAX); - let timeout = Self::BASE_DURATION - .saturating_mul(f) - .min(Self::MAX_DURATION); - - 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/make.rs b/node/actors/bft/src/testonly/make.rs index d2b49113..13382860 100644 --- a/node/actors/bft/src/testonly/make.rs +++ b/node/actors/bft/src/testonly/make.rs @@ -1,9 +1,23 @@ //! This module contains utilities that are only meant for testing purposes. +use crate::io::InputMessage; use crate::PayloadManager; -use rand::Rng as _; +use rand::{distributions::Standard, prelude::Distribution, Rng}; use zksync_concurrency::ctx; +use zksync_concurrency::oneshot; +use zksync_consensus_network::io::ConsensusReq; use zksync_consensus_roles::validator; +// Generates a random InputMessage. +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> InputMessage { + let (send, _) = oneshot::channel(); + InputMessage::Network(ConsensusReq { + msg: rng.gen(), + ack: send, + }) + } +} + /// Produces random payload of a given size. #[derive(Debug)] pub struct RandomPayload(pub usize); diff --git a/node/actors/bft/src/testonly/mod.rs b/node/actors/bft/src/testonly/mod.rs index 504bb149..03aed7c0 100644 --- a/node/actors/bft/src/testonly/mod.rs +++ b/node/actors/bft/src/testonly/mod.rs @@ -1,33 +1,15 @@ //! This module contains utilities that are only meant for testing purposes. -use crate::io::InputMessage; -use rand::{distributions::Standard, prelude::Distribution, Rng}; -use zksync_concurrency::oneshot; -use zksync_consensus_network::io::ConsensusReq; - mod make; #[cfg(test)] mod node; #[cfg(test)] mod run; #[cfg(test)] -pub(crate) mod ut_harness; +pub mod twins; pub use make::*; #[cfg(test)] pub(crate) use node::*; #[cfg(test)] pub(crate) use run::*; -#[cfg(test)] -pub mod twins; - -// Generates a random InputMessage. -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> InputMessage { - let (send, _) = oneshot::channel(); - InputMessage::Network(ConsensusReq { - msg: rng.gen(), - ack: send, - }) - } -} diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 8b06968b..f303a0d4 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -29,11 +29,24 @@ pub(crate) enum Network { /// Technically there are 4 phases but that results in tests timing out as /// the chance of a reaching consensus in any round goes down rapidly. /// -/// Instead we can just use two phase-partitions: one for the LeaderCommit, +/// Instead we can just use two phase-partitions: one for the LeaderProposal, /// and another for everything else. This models the typical adversarial -/// scenario of not everyone getting the QC. +/// scenario of not everyone getting the proposal. pub(crate) const NUM_PHASES: usize = 2; +/// Index of the phase in which the message appears, to decide which partitioning to apply. +fn msg_phase_number(msg: &validator::ConsensusMsg) -> usize { + use validator::ConsensusMsg; + let phase = match msg { + ConsensusMsg::LeaderProposal(_) => 0, + ConsensusMsg::ReplicaCommit(_) => 1, + ConsensusMsg::ReplicaTimeout(_) => 1, + ConsensusMsg::ReplicaNewView(_) => 1, + }; + assert!(phase < NUM_PHASES); + phase +} + /// Identify different network identities of twins by their listener port. /// They are all expected to be on localhost, but `ListenerAddr` can't be /// directly used as a map key. @@ -47,12 +60,13 @@ pub(crate) type PortSplitSchedule = Vec<[PortSplit; NUM_PHASES]>; /// Function to decide whether a message can go from a source to a target port. pub(crate) type PortRouterFn = dyn Fn(&validator::ConsensusMsg, Port, Port) -> Option + Sync; -/// A predicate to gover who can communicate to whom a given message. +/// A predicate to govern who can communicate to whom a given message. pub(crate) enum PortRouter { /// List of port splits for each view/phase, where ports in the same partition can send any message to each other. Splits(PortSplitSchedule), /// Custom routing function which can take closer control of which message can be sent in which direction, /// in order to reenact particular edge cases. + #[allow(dead_code)] Custom(Box), } @@ -264,7 +278,6 @@ async fn run_nodes_twins( // Taking these references is necessary for the `scope::run!` environment lifetime rules to compile // with `async move`, which in turn is necessary otherwise it the spawned process could not borrow `port`. // Potentially `ctx::NoCopy` could be used with `port`. - let validator_ports = &validator_ports; let sends = &sends; let stores = &stores; let gossip_targets = &gossip_targets; @@ -283,7 +296,6 @@ async fn run_nodes_twins( twins_receive_loop( ctx, router, - validator_ports, sends, TwinsGossipConfig { targets: &gossip_targets[&port], @@ -308,12 +320,11 @@ async fn run_nodes_twins( /// according to the partition schedule of the port associated with this instance. /// /// We have to simulate the gossip layer which isn't instantiated by these tests. -/// If we don't, then if a replica misses a LeaderPrepare message it won't ever get the payload +/// If we don't, then if a replica misses a LeaderProposal message it won't ever get the payload /// and won't be able to finalize the block, and won't participate further in the consensus. async fn twins_receive_loop( ctx: &ctx::Ctx, router: &PortRouter, - validator_ports: &HashMap>, sends: &HashMap>, gossip: TwinsGossipConfig<'_>, port: Port, @@ -331,7 +342,7 @@ async fn twins_receive_loop( // We need to buffer messages that cannot be delivered due to partitioning, and deliver them later. // The spec says that the network is expected to deliver messages eventually, potentially out of order, - // caveated by the fact that the actual implementation only keeps retrying the last message.. + // caveated by the fact that the actual implementation only keeps retrying the last message. // A separate issue is the definition of "later", without actually adding timing assumptions: // * If we want to allow partitions which don't have enough replicas for a quorum, and the replicas // don't move on from a view until they reach quorum, then "later" could be defined by so many @@ -341,12 +352,7 @@ async fn twins_receive_loop( // can move on to the next view, in which a new partition configuration will allow them to broadcast // to previously isolated peers. // * One idea is to wait until replica A wants to send to replica B in a view when they are no longer - // partitioned, and then unstash all previous A-to-B messages. This would _not_ work with HotStuff - // out of the box, because replicas only communicate with their leader, so if for example B missed - // a LeaderCommit from A in an earlier view, B will not respond to the LeaderPrepare from C because - // they can't commit the earlier block until they get a new message from A. However since - // https://github.com/matter-labs/era-consensus/pull/119 the ReplicaPrepare messages are broadcasted, - // so we shouldn't have to wait long for A to unstash its messages to B. + // partitioned, and then unstash all previous A-to-B messages. // * If that wouldn't be acceptable then we could have some kind of global view of stashed messages // and unstash them as soon as someone moves on to a new view. let mut stashes: HashMap> = HashMap::new(); @@ -413,24 +419,12 @@ async fn twins_receive_loop( } }; - match message.recipient { - io::Target::Broadcast => { - tracing::info!("broadcasting view={view} from={port} kind={kind}"); - for target_port in sends.keys() { - send_or_stash(can_send(*target_port)?, *target_port, msg()); - } - } - io::Target::Validator(ref v) => { - let target_ports = &validator_ports[v]; - tracing::info!( - "unicasting view={view} from={port} target={target_ports:?} kind={kind}" - ); - for target_port in target_ports { - send_or_stash(can_send(*target_port)?, *target_port, msg()); - } - } + tracing::info!("broadcasting view={view} from={port} kind={kind}"); + for target_port in sends.keys() { + send_or_stash(can_send(*target_port)?, *target_port, msg()); } } + Ok(()) } @@ -510,27 +504,20 @@ fn output_msg_label(msg: &io::OutputMessage) -> &str { fn output_msg_commit_qc(msg: &io::OutputMessage) -> Option<&validator::CommitQC> { use validator::ConsensusMsg; - match msg { + + let justification = match msg { io::OutputMessage::Consensus(cr) => match &cr.msg.msg { - ConsensusMsg::ReplicaPrepare(rp) => rp.high_qc.as_ref(), - ConsensusMsg::LeaderPrepare(lp) => lp.justification.high_qc(), - ConsensusMsg::ReplicaCommit(_) => None, - ConsensusMsg::LeaderCommit(lc) => Some(&lc.justification), + ConsensusMsg::ReplicaTimeout(msg) => return msg.high_qc.as_ref(), + ConsensusMsg::ReplicaCommit(_) => return None, + ConsensusMsg::ReplicaNewView(msg) => &msg.justification, + ConsensusMsg::LeaderProposal(msg) => &msg.justification, }, - } -} - -/// Index of the phase in which the message appears, to decide which partitioning to apply. -fn msg_phase_number(msg: &validator::ConsensusMsg) -> usize { - use validator::ConsensusMsg; - let phase = match msg { - ConsensusMsg::ReplicaPrepare(_) => 0, - ConsensusMsg::LeaderPrepare(_) => 0, - ConsensusMsg::ReplicaCommit(_) => 0, - ConsensusMsg::LeaderCommit(_) => 1, }; - assert!(phase < NUM_PHASES); - phase + + match justification { + validator::ProposalJustification::Commit(commit_qc) => Some(commit_qc), + validator::ProposalJustification::Timeout(timeout_qc) => timeout_qc.high_qc(), + } } struct TwinsGossipMessage { diff --git a/node/actors/bft/src/testonly/ut_harness.rs b/node/actors/bft/src/testonly/ut_harness.rs deleted file mode 100644 index 406ea4df..00000000 --- a/node/actors/bft/src/testonly/ut_harness.rs +++ /dev/null @@ -1,334 +0,0 @@ -use crate::{ - io::OutputMessage, - leader, - leader::{replica_commit, replica_prepare}, - replica, - replica::{leader_commit, leader_prepare}, - testonly, Config, PayloadManager, -}; -use assert_matches::assert_matches; -use std::sync::Arc; -use zksync_concurrency::{ctx, sync::prunable_mpsc}; -use zksync_consensus_network as network; -use zksync_consensus_roles::validator; -use zksync_consensus_storage::{ - testonly::{in_memory, TestMemoryStorage}, - BlockStoreRunner, -}; -use zksync_consensus_utils::enum_util::Variant; - -pub(crate) const MAX_PAYLOAD_SIZE: usize = 1000; - -/// `UTHarness` provides various utilities for unit tests. -/// It is designed to simplify the setup and execution of test cases by encapsulating -/// common testing functionality. -/// -/// It should be instantiated once for every test case. -#[cfg(test)] -pub(crate) struct UTHarness { - pub(crate) leader: leader::StateMachine, - pub(crate) replica: replica::StateMachine, - pub(crate) keys: Vec, - pub(crate) leader_send: prunable_mpsc::Sender, - pipe: ctx::channel::UnboundedReceiver, -} - -impl UTHarness { - /// Creates a new `UTHarness` with the specified validator set size. - pub(crate) async fn new( - ctx: &ctx::Ctx, - num_validators: usize, - ) -> (UTHarness, BlockStoreRunner) { - Self::new_with_payload( - ctx, - num_validators, - Box::new(testonly::RandomPayload(MAX_PAYLOAD_SIZE)), - ) - .await - } - - pub(crate) async fn new_with_payload( - ctx: &ctx::Ctx, - num_validators: usize, - payload_manager: Box, - ) -> (UTHarness, BlockStoreRunner) { - let rng = &mut ctx.rng(); - let setup = validator::testonly::Setup::new(rng, num_validators); - let store = TestMemoryStorage::new(ctx, &setup).await; - let (send, recv) = ctx::channel::unbounded(); - - let cfg = Arc::new(Config { - secret_key: setup.validator_keys[0].clone(), - block_store: store.blocks.clone(), - replica_store: Box::new(in_memory::ReplicaStore::default()), - payload_manager, - max_payload_size: MAX_PAYLOAD_SIZE, - }); - let (leader, leader_send) = leader::StateMachine::new(ctx, cfg.clone(), send.clone()); - let (replica, _) = replica::StateMachine::start(ctx, cfg.clone(), send.clone()) - .await - .unwrap(); - let mut this = UTHarness { - leader, - replica, - pipe: recv, - keys: setup.validator_keys.clone(), - leader_send, - }; - let _: validator::Signed = this.try_recv().unwrap(); - (this, store.runner) - } - - /// 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; - let (util, runner) = UTHarness::new(ctx, num_validators).await; - assert!(util.genesis().validators.max_faulty_weight() > 0); - (util, runner) - } - - /// Triggers replica timeout, validates the new validator::ReplicaPrepare - /// then executes the whole new view to make sure that the consensus - /// recovers after a timeout. - pub(crate) async fn produce_block_after_timeout(&mut self, ctx: &ctx::Ctx) { - let want = validator::ReplicaPrepare { - view: validator::View { - genesis: self.genesis().hash(), - number: self.replica.view.next(), - }, - high_qc: self.replica.high_qc.clone(), - 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; - } - - /// 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(); - } - - pub(crate) fn owner_key(&self) -> &validator::SecretKey { - &self.replica.config.secret_key - } - - pub(crate) fn sign>(&self, msg: V) -> validator::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() { - view = view.next(); - } - self.replica.view = view; - } - - pub(crate) fn replica_view(&self) -> validator::View { - validator::View { - genesis: self.genesis().hash(), - number: self.replica.view, - } - } - - pub(crate) fn new_replica_prepare(&mut self) -> validator::ReplicaPrepare { - self.set_owner_as_view_leader(); - validator::ReplicaPrepare { - view: self.replica_view(), - high_vote: self.replica.high_vote.clone(), - high_qc: self.replica.high_qc.clone(), - } - } - - pub(crate) fn new_current_replica_commit(&self) -> validator::ReplicaCommit { - validator::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) -> validator::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) -> validator::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) -> validator::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: validator::Signed, - ) -> Result, leader_prepare::Error> { - self.replica.process_leader_prepare(ctx, msg).await?; - Ok(self.try_recv().unwrap()) - } - - pub(crate) async fn process_leader_commit( - &mut self, - ctx: &ctx::Ctx, - msg: validator::Signed, - ) -> Result, leader_commit::Error> { - self.replica.process_leader_commit(ctx, msg).await?; - Ok(self.try_recv().unwrap()) - } - - #[allow(clippy::result_large_err)] - pub(crate) async fn process_replica_prepare( - &mut self, - ctx: &ctx::Ctx, - msg: validator::Signed, - ) -> 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() { - let prepare_qc = prepare_qc.borrow().clone().unwrap(); - leader::StateMachine::propose( - ctx, - &self.leader.config, - prepare_qc, - &self.leader.outbound_pipe, - ) - .await - .unwrap(); - } - Ok(self.try_recv()) - } - - pub(crate) async fn process_replica_prepare_all( - &mut self, - ctx: &ctx::Ctx, - msg: validator::ReplicaPrepare, - ) -> validator::Signed { - let mut leader_prepare = None; - let msgs: Vec<_> = self.keys.iter().map(|k| k.sign_msg(msg.clone())).collect(); - let mut first_match = true; - for (i, msg) in msgs.into_iter().enumerate() { - let res = self.process_replica_prepare(ctx, msg).await; - match ( - (i + 1) as u64 * self.genesis().validators.iter().next().unwrap().weight - < self.genesis().validators.quorum_threshold(), - first_match, - ) { - (true, _) => assert!(res.unwrap().is_none()), - (false, true) => { - first_match = false; - leader_prepare = res.unwrap() - } - (false, false) => assert_matches!(res, Err(replica_prepare::Error::Old { .. })), - } - } - leader_prepare.unwrap() - } - - pub(crate) async fn process_replica_commit( - &mut self, - ctx: &ctx::Ctx, - msg: validator::Signed, - ) -> Result>, replica_commit::Error> { - self.leader.process_replica_commit(ctx, msg)?; - Ok(self.try_recv()) - } - - async fn process_replica_commit_all( - &mut self, - ctx: &ctx::Ctx, - msg: validator::ReplicaCommit, - ) -> validator::Signed { - let mut first_match = true; - for (i, key) in self.keys.iter().enumerate() { - let res = self - .leader - .process_replica_commit(ctx, key.sign_msg(msg.clone())); - match ( - (i + 1) as u64 * self.genesis().validators.iter().next().unwrap().weight - < self.genesis().validators.quorum_threshold(), - first_match, - ) { - (true, _) => res.unwrap(), - (false, true) => { - first_match = false; - res.unwrap() - } - (false, false) => assert_matches!(res, Err(replica_commit::Error::Old { .. })), - } - } - self.try_recv().unwrap() - } - - fn try_recv>(&mut self) -> Option> { - self.pipe.try_recv().map(|message| match message { - OutputMessage::Network(network::io::ConsensusInputMessage { message, .. }) => { - message.cast().unwrap() - } - }) - } - - pub(crate) async fn process_replica_timeout( - &mut self, - ctx: &ctx::Ctx, - ) -> validator::Signed { - self.replica.start_new_view(ctx).await.unwrap(); - self.try_recv().unwrap() - } - - pub(crate) fn leader_phase(&self) -> validator::Phase { - self.leader.phase - } - - pub(crate) fn view_leader(&self, view: validator::ViewNumber) -> validator::PublicKey { - self.genesis().view_leader(view) - } - - pub(crate) fn genesis(&self) -> &validator::Genesis { - self.replica.config.genesis() - } - - pub(crate) fn new_commit_qc( - &self, - mutate_fn: impl FnOnce(&mut validator::ReplicaCommit), - ) -> validator::CommitQC { - let mut msg = self.new_current_replica_commit(); - mutate_fn(&mut msg); - let mut qc = validator::CommitQC::new(msg, self.genesis()); - for key in &self.keys { - qc.add(&key.sign_msg(qc.message.clone()), self.genesis()) - .unwrap(); - } - qc - } - - pub(crate) fn new_prepare_qc( - &mut self, - mutate_fn: impl FnOnce(&mut validator::ReplicaPrepare), - ) -> validator::PrepareQC { - let mut msg = self.new_replica_prepare(); - mutate_fn(&mut msg); - let mut qc = validator::PrepareQC::new(msg.view.clone()); - for key in &self.keys { - qc.add(&key.sign_msg(msg.clone()), self.genesis()).unwrap(); - } - qc - } - - pub(crate) fn leader_send(&self, msg: validator::Signed) { - self.leader_send.send(network::io::ConsensusReq { - msg, - ack: zksync_concurrency::oneshot::channel().0, - }); - } -} diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs deleted file mode 100644 index ea071661..00000000 --- a/node/actors/bft/src/tests.rs +++ /dev/null @@ -1,610 +0,0 @@ -use crate::testonly::{ - twins::{Cluster, HasKey, ScenarioGenerator, Twin}, - ut_harness::UTHarness, - Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, TestError, NUM_PHASES, -}; -use assert_matches::assert_matches; -use std::collections::HashMap; -use test_casing::{cases, test_casing, TestCases}; -use zksync_concurrency::{ctx, scope, time}; -use zksync_consensus_network::testonly::new_configs_for_validators; -use zksync_consensus_roles::validator::{ - self, - testonly::{Setup, SetupSpec}, - LeaderSelectionMode, PublicKey, SecretKey, ViewNumber, -}; - -async fn run_test(behavior: Behavior, network: Network) { - tokio::time::pause(); - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - - const NODES: usize = 11; - let mut nodes = vec![(behavior, 1u64); NODES]; - // validator::threshold(NODES) will calculate required nodes to validate a message - // given each node weight is 1 - let honest_nodes_amount = validator::quorum_threshold(NODES as u64) as usize; - for n in &mut nodes[0..honest_nodes_amount] { - n.0 = Behavior::Honest; - } - Test { - network, - nodes, - blocks_to_finalize: 10, - } - .run(ctx) - .await - .unwrap() -} - -#[tokio::test] -async fn honest_real_network() { - run_test(Behavior::Honest, Network::Real).await -} - -#[tokio::test] -async fn offline_real_network() { - run_test(Behavior::Offline, Network::Real).await -} - -/// 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(); - 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_prepare(); - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Testing liveness after the network becomes idle with leader having some cached prepare messages for the current view. -#[tokio::test] -async fn timeout_leader_some_prepares() { - 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 replica_prepare = util.new_replica_prepare(); - assert!(util - .process_replica_prepare(ctx, util.sign(replica_prepare)) - .await - .unwrap() - .is_none()); - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Testing liveness after the network becomes idle with leader in commit phase. -#[tokio::test] -async fn timeout_leader_in_commit() { - 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_leader_prepare(ctx).await; - // Leader is in `Phase::Commit`, but should still accept prepares from newer views. - assert_eq!(util.leader.phase, validator::Phase::Commit); - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Testing liveness after the network becomes idle with replica in commit phase. -#[tokio::test] -async fn timeout_replica_in_commit() { - 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; - // Leader is in `Phase::Commit`, but should still accept prepares from newer views. - assert_eq!(util.leader.phase, validator::Phase::Commit); - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Testing liveness after the network becomes idle with leader having some cached commit messages for the current view. -#[tokio::test] -async fn timeout_leader_some_commits() { - 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 replica_commit = util.new_replica_commit(ctx).await; - assert!(util - .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(), validator::Phase::Commit); - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Testing liveness after the network becomes idle with leader in a consecutive prepare phase. -#[tokio::test] -async fn timeout_leader_in_consecutive_prepare() { - 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_leader_commit(ctx).await; - util.produce_block_after_timeout(ctx).await; - Ok(()) - }) - .await - .unwrap(); -} - -/// Not being able to propose a block shouldn't cause a deadlock. -#[tokio::test] -async fn non_proposing_leader() { - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::AffineClock::new(5.)); - Test { - network: Network::Real, - nodes: vec![(Behavior::Honest, 1), (Behavior::HonestNotProposing, 1)], - blocks_to_finalize: 10, - } - .run(ctx) - .await - .unwrap() -} - -/// Run Twins scenarios without actual twins, and with so few nodes that all -/// of them are required for a quorum, which means (currently) there won't be -/// any partitions. -/// -/// This should be a simple sanity check that the network works and consensus -/// is achieved under the most favourable conditions. -#[test_casing(10,0..10)] -#[tokio::test] -async fn twins_network_wo_twins_wo_partitions(num_reseeds: usize) { - tokio::time::pause(); - // n<6 implies f=0 and q=n - run_twins(5, 0, TwinsScenarios::Reseeds(num_reseeds)) - .await - .unwrap(); -} - -/// Run Twins scenarios without actual twins, but enough replicas that partitions -/// can play a role, isolating certain nodes (potentially the leader) in some -/// rounds. -/// -/// This should be a sanity check that without Byzantine behaviour the consensus -/// is resilient to temporary network partitions. -#[test_casing(5,0..5)] -#[tokio::test] -async fn twins_network_wo_twins_w_partitions(num_reseeds: usize) { - tokio::time::pause(); - // n=6 implies f=1 and q=5; 6 is the minimum where partitions are possible. - run_twins(6, 0, TwinsScenarios::Reseeds(num_reseeds)) - .await - .unwrap(); -} - -/// Test cases with 1 twin, with 6-10 replicas, 10 scenarios each. -const CASES_TWINS_1: TestCases<(usize, usize)> = cases! { - (6..=10).flat_map(|num_replicas| (0..10).map(move |num_reseeds| (num_replicas, num_reseeds))) -}; - -/// Run Twins scenarios with random number of nodes and 1 twin. -#[test_casing(50, CASES_TWINS_1)] -#[tokio::test] -async fn twins_network_w1_twins_w_partitions(num_replicas: usize, num_reseeds: usize) { - tokio::time::pause(); - // n>=6 implies f>=1 and q=n-f - // let num_honest = validator::threshold(num_replicas as u64) as usize; - // let max_faulty = num_replicas - num_honest; - // let num_twins = rng.gen_range(1..=max_faulty); - run_twins(num_replicas, 1, TwinsScenarios::Reseeds(num_reseeds)) - .await - .unwrap(); -} - -/// Run Twins scenarios with higher number of nodes and 2 twins. -#[test_casing(5,0..5)] -#[tokio::test] -async fn twins_network_w2_twins_w_partitions(num_reseeds: usize) { - tokio::time::pause(); - // n>=11 implies f>=2 and q=n-f - run_twins(11, 2, TwinsScenarios::Reseeds(num_reseeds)) - .await - .unwrap(); -} - -/// Run Twins scenario with more twins than tolerable and expect it to fail. -#[tokio::test] -async fn twins_network_to_fail() { - tokio::time::pause(); - // With n=5 f=0, so 1 twin means more faulty nodes than expected. - assert_matches!( - run_twins(5, 1, TwinsScenarios::Multiple(100)).await, - Err(TestError::BlockConflict) - ); -} - -/// Govern how many scenarios to execute in the test. -enum TwinsScenarios { - /// Execute N scenarios in a loop. - /// - /// Use this when looking for a counter example, ie. a scenario where consensus fails. - Multiple(usize), - /// Execute 1 scenario after doing N reseeds of the RNG. - /// - /// Use this with the `#[test_casing]` macro to turn scenarios into separate test cases. - Reseeds(usize), -} - -/// Create network configuration for a given number of replicas and twins and run [Test], -async fn run_twins( - num_replicas: usize, - num_twins: usize, - scenarios: TwinsScenarios, -) -> Result<(), TestError> { - zksync_concurrency::testonly::abort_on_panic(); - - // A single scenario with 11 replicas took 3-5 seconds. - // Panic on timeout; works with `cargo nextest` and the `abort_on_panic` above. - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); - let ctx = &ctx::test_root(&ctx::RealClock); - - #[derive(PartialEq, Debug)] - struct Replica { - id: i64, // non-zero ID - public_key: PublicKey, - secret_key: SecretKey, - } - - impl HasKey for Replica { - type Key = PublicKey; - - fn key(&self) -> &Self::Key { - &self.public_key - } - } - - impl Twin for Replica { - fn to_twin(&self) -> Self { - Self { - id: -self.id, - public_key: self.public_key.clone(), - secret_key: self.secret_key.clone(), - } - } - } - - let (num_scenarios, num_reseeds) = match scenarios { - TwinsScenarios::Multiple(n) => (n, 0), - TwinsScenarios::Reseeds(n) => (1, n), - }; - - // Keep scenarios separate by generating a different RNG many times. - let mut rng = ctx.rng(); - for _ in 0..num_reseeds { - rng = ctx.rng(); - } - let rng = &mut rng; - - // The existing test machinery uses the number of finalized blocks as an exit criteria. - let blocks_to_finalize = 3; - // The test is going to disrupt the communication by partitioning nodes, - // where the leader might not be in a partition with enough replicas to - // form a quorum, therefore to allow N blocks to be finalized we need to - // go longer. - let num_rounds = blocks_to_finalize * 10; - // The paper considers 2 or 3 partitions enough. - let max_partitions = 3; - - // Every validator has equal power of 1. - const WEIGHT: u64 = 1; - let mut spec = SetupSpec::new_with_weights(rng, vec![WEIGHT; num_replicas]); - - let replicas = spec - .validator_weights - .iter() - .enumerate() - .map(|(i, (sk, _))| Replica { - id: i as i64 + 1, - public_key: sk.public(), - secret_key: sk.clone(), - }) - .collect::>(); - - let cluster = Cluster::new(replicas, num_twins); - let scenarios = ScenarioGenerator::<_, NUM_PHASES>::new(&cluster, num_rounds, max_partitions); - - // Gossip with more nodes than what can be faulty. - let gossip_peers = num_twins + 1; - - // Create network config for all nodes in the cluster (assigns unique network addresses). - let nets = new_configs_for_validators( - rng, - cluster.nodes().iter().map(|r| &r.secret_key), - gossip_peers, - ); - - let node_to_port = cluster - .nodes() - .iter() - .zip(nets.iter()) - .map(|(node, net)| (node.id, net.server_addr.port())) - .collect::>(); - - assert_eq!(node_to_port.len(), cluster.num_nodes()); - - // Every network needs a behaviour. They are all honest, just some might be duplicated. - let nodes = vec![(Behavior::Honest, WEIGHT); cluster.num_nodes()]; - - // Reuse the same cluster and network setup to run a few scenarios. - for i in 0..num_scenarios { - // Generate a permutation of partitions and leaders for the given number of rounds. - let scenario = scenarios.generate_one(rng); - - // Assign the leadership schedule to the consensus. - spec.leader_selection = - LeaderSelectionMode::Rota(scenario.rounds.iter().map(|rc| rc.leader.clone()).collect()); - - // Generate a new setup with this leadership schedule. - let setup = Setup::from_spec(rng, spec.clone()); - - // Create a network with the partition schedule of the scenario. - let splits: PortSplitSchedule = scenario - .rounds - .iter() - .map(|rc| { - std::array::from_fn(|i| { - rc.phase_partitions[i] - .iter() - .map(|p| p.iter().map(|r| node_to_port[&r.id]).collect()) - .collect() - }) - }) - .collect(); - - tracing::info!( - "num_replicas={num_replicas} num_twins={num_twins} num_nodes={} scenario={i}", - cluster.num_nodes() - ); - - // Debug output of round schedule. - for (r, rc) in scenario.rounds.iter().enumerate() { - // Let's just consider the partition of the LeaderCommit phase, for brevity's sake. - let partitions = &splits[r].last().unwrap(); - - let leader_ports = cluster - .nodes() - .iter() - .filter(|n| n.public_key == *rc.leader) - .map(|n| node_to_port[&n.id]) - .collect::>(); - - let leader_partition_sizes = leader_ports - .iter() - .map(|lp| partitions.iter().find(|p| p.contains(lp)).unwrap().len()) - .collect::>(); - - let leader_isolated = leader_partition_sizes - .iter() - .all(|s| *s < cluster.quorum_size()); - - tracing::info!("round={r} partitions={partitions:?} leaders={leader_ports:?} leader_partition_sizes={leader_partition_sizes:?} leader_isolated={leader_isolated}"); - } - - Test { - network: Network::Twins(PortRouter::Splits(splits)), - nodes: nodes.clone(), - blocks_to_finalize, - } - .run_with_config(ctx, nets.clone(), &setup) - .await? - } - - Ok(()) -} - -/// Test a liveness issue where some validators have the HighQC but don't have the block payload and have to wait for it, -/// while some other validators have the payload but don't have the HighQC and cannot finalize the block, and therefore -/// don't gossip it, which causes a deadlock unless the one with the HighQC moves on and broadcasts what they have, which -/// should cause the others to finalize the block and gossip the payload to them in turn. -#[tokio::test] -async fn test_wait_for_finalized_deadlock() { - // These are the conditions for the deadlock to occur: - // * The problem happens in the handling of LeaderPrepare where the replica waits for the previous block in the justification. - // * For that the replica needs to receive a proposal from a leader that knows the previous block is finalized. - // * For that the leader needs to receive a finalized proposal from an earlier leader, but this proposal did not make it to the replica. - // * Both leaders need to die and never communicate the HighQC they know about to anybody else. - // * The replica has the HighQC but not the payload, and all other replicas might have the payload, but not the HighQC. - // * With two leaders down, and the replica deadlocked, we must lose quorum, so the other nodes cannot repropose the missing block either. - // * In order for 2 leaders to be dow and quorum still be possible, we need at least 11 nodes. - - // Here are a series of steps to reproduce the issue: - // 1. Say we have 11 nodes: [0,1,2,3,4,5,6,7,8,9,10], taking turns leading the views in that order; we need 9 nodes for quorum. The first view is view 1 lead by node 1. - // 2. Node 1 sends LeaderPropose with block 1 to nodes [1-9] and puts together a HighQC. - // 3. Node 1 sends the LeaderCommit to node 2, then dies. - // 4. Node 2 sends LeaderPropose with block 2 to nodes [0, 10], then dies. - // 5. Nodes [0, 10] get stuck processing LeaderPropose because they are waiting for block 1 to appear in their stores. - // 6. Node 3 cannot gather 9 ReplicaPrepare messages for a quorum because nodes [1,2] are down and [0,10] are blocking. Consensus stalls. - - // To simulate this with the Twins network we need to use a custom routing function, because the 2nd leader mustn't broadcast the HighQC - // to its peers, but it must receive their ReplicaPrepare's to be able to construct the PrepareQC; because of this the simple split schedule - // would not be enough as it allows sending messages in both directions. - - // We need 11 nodes so we can turn 2 leaders off. - let num_replicas = 11; - // Let's wait for the first two blocks to be finalised. - // Although theoretically node 1 will be dead after view 1, it will still receive messages and gossip. - let blocks_to_finalize = 2; - // We need more than 1 gossip peer, otherwise the chain of gossip triggers in the Twins network won't kick in, - // and while node 0 will gossip to node 1, node 1 will not send it to node 2, and the test will fail. - let gossip_peers = 2; - - run_with_custom_router( - num_replicas, - gossip_peers, - blocks_to_finalize, - |port_to_id| { - PortRouter::Custom(Box::new(move |msg, from, to| { - use validator::ConsensusMsg::*; - // Map ports back to logical node ID - let from = port_to_id[&from]; - let to = port_to_id[&to]; - let view_number = msg.view().number; - - // If we haven't finalised the blocks in the first few rounds, we failed. - if view_number.0 > 7 { - return None; - } - - // Sending to self is ok. - // If this wasn't here the test would pass even without adding a timeout in process_leader_prepare. - // The reason is that node 2 would move to view 2 as soon as it finalises block 1, but then timeout - // and move to view 3 before they receive any of the ReplicaPrepare from the others, who are still - // waiting to timeout in view 1. By sending ReplicaPrepare to itself it seems to wait or propose. - // Maybe the HighQC doesn't make it from its replica::StateMachine into its leader::StateMachine otherwise. - if from == to { - return Some(true); - } - - let can_send = match view_number { - ViewNumber(1) => { - match from { - // Current leader - 1 => match msg { - // Send the proposal to a subset of nodes - LeaderPrepare(_) => to != 0 && to != 10, - // Send the commit to the next leader only - LeaderCommit(_) => to == 2, - _ => true, - }, - // Replicas - _ => true, - } - } - ViewNumber(2) => match from { - // Previous leader is dead - 1 => false, - // Current leader - 2 => match msg { - // Don't send out the HighQC to the others - ReplicaPrepare(_) => false, - // Send the proposal to the ones which didn't get the previous one - LeaderPrepare(_) => to == 0 || to == 10, - _ => true, - }, - // Replicas - _ => true, - }, - // Previous leaders dead - _ => from != 1 && from != 2, - }; - - // eprintln!( - // "view={view_number} from={from} to={to} kind={} can_send={can_send}", - // msg.label() - // ); - - Some(can_send) - })) - }, - ) - .await - .unwrap(); -} - -/// Run a test with the Twins network controlling exactly who can send to whom in each round. -/// -/// The input for the router is a mapping from port to the index of nodes starting from 0. -/// The first view to be executed is view 1 and will have the node 1 as its leader, and so on, -/// so a routing function can expect view `i` to be lead by node `i`, and express routing -/// rules with the logic IDs. -async fn run_with_custom_router( - num_replicas: usize, - gossip_peers: usize, - blocks_to_finalize: usize, - make_router: impl FnOnce(HashMap) -> PortRouter, -) -> Result<(), TestError> { - tokio::time::pause(); - zksync_concurrency::testonly::abort_on_panic(); - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); - let ctx = &ctx::test_root(&ctx::RealClock); - - let rng = &mut ctx.rng(); - - let mut spec = SetupSpec::new(rng, num_replicas); - - let nodes = spec - .validator_weights - .iter() - .map(|(_, w)| (Behavior::Honest, *w)) - .collect(); - - let nets = new_configs_for_validators( - rng, - spec.validator_weights.iter().map(|(sk, _)| sk), - gossip_peers, - ); - - // Assign the validator rota to be in the order of appearance, not ordered by public key. - spec.leader_selection = LeaderSelectionMode::Rota( - spec.validator_weights - .iter() - .map(|(sk, _)| sk.public()) - .collect(), - ); - - let setup = Setup::from_spec(rng, spec); - - let port_to_id = nets - .iter() - .enumerate() - .map(|(i, net)| (net.server_addr.port(), i)) - .collect::>(); - - // Sanity check the leader schedule - { - let pk = setup.genesis.view_leader(ViewNumber(1)); - let cfg = nets - .iter() - .find(|net| net.validator_key.as_ref().unwrap().public() == pk) - .unwrap(); - let port = cfg.server_addr.port(); - assert_eq!(port_to_id[&port], 1); - } - - Test { - network: Network::Twins(make_router(port_to_id)), - nodes, - blocks_to_finalize, - } - .run_with_config(ctx, nets, &setup) - .await -} diff --git a/node/actors/bft/src/tests/mod.rs b/node/actors/bft/src/tests/mod.rs new file mode 100644 index 00000000..70c005c2 --- /dev/null +++ b/node/actors/bft/src/tests/mod.rs @@ -0,0 +1,53 @@ +use crate::testonly::{Behavior, Network, Test}; +use zksync_concurrency::{ctx, time}; +use zksync_consensus_roles::validator; + +mod twins; + +async fn run_test(behavior: Behavior, network: Network) { + tokio::time::pause(); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); + zksync_concurrency::testonly::abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + + const NODES: usize = 11; + let mut nodes = vec![(behavior, 1u64); NODES]; + // validator::threshold(NODES) will calculate required nodes to validate a message + // given each node weight is 1 + let honest_nodes_amount = validator::quorum_threshold(NODES as u64) as usize; + for n in &mut nodes[0..honest_nodes_amount] { + n.0 = Behavior::Honest; + } + Test { + network, + nodes, + blocks_to_finalize: 10, + } + .run(ctx) + .await + .unwrap() +} + +#[tokio::test] +async fn honest_real_network() { + run_test(Behavior::Honest, Network::Real).await +} + +#[tokio::test] +async fn offline_real_network() { + run_test(Behavior::Offline, Network::Real).await +} + +#[tokio::test] +async fn honest_not_proposing_real_network() { + zksync_concurrency::testonly::abort_on_panic(); + let ctx = &ctx::test_root(&ctx::AffineClock::new(5.)); + Test { + network: Network::Real, + nodes: vec![(Behavior::Honest, 1), (Behavior::HonestNotProposing, 1)], + blocks_to_finalize: 10, + } + .run(ctx) + .await + .unwrap() +} diff --git a/node/actors/bft/src/tests/twins.rs b/node/actors/bft/src/tests/twins.rs new file mode 100644 index 00000000..b6c9b68a --- /dev/null +++ b/node/actors/bft/src/tests/twins.rs @@ -0,0 +1,265 @@ +use crate::testonly::{ + twins::{Cluster, HasKey, ScenarioGenerator, Twin}, + Behavior, Network, PortRouter, PortSplitSchedule, Test, TestError, NUM_PHASES, +}; +use assert_matches::assert_matches; +use std::collections::HashMap; +use test_casing::{cases, test_casing, TestCases}; +use zksync_concurrency::{ctx, time}; +use zksync_consensus_network::testonly::new_configs_for_validators; +use zksync_consensus_roles::validator::{ + testonly::{Setup, SetupSpec}, + LeaderSelectionMode, PublicKey, SecretKey, +}; + +/// Govern how many scenarios to execute in the test. +enum TwinsScenarios { + /// Execute N scenarios in a loop. + /// + /// Use this when looking for a counter example, ie. a scenario where consensus fails. + Multiple(usize), + /// Execute 1 scenario after doing N reseeds of the RNG. + /// + /// Use this with the `#[test_casing]` macro to turn scenarios into separate test cases. + Reseeds(usize), +} + +/// Create network configuration for a given number of replicas and twins and run [Test], +async fn run_twins( + num_replicas: usize, + num_twins: usize, + scenarios: TwinsScenarios, +) -> Result<(), TestError> { + zksync_concurrency::testonly::abort_on_panic(); + + // A single scenario with 11 replicas took 3-5 seconds. + // Panic on timeout; works with `cargo nextest` and the `abort_on_panic` above. + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); + let ctx = &ctx::test_root(&ctx::RealClock); + + #[derive(PartialEq, Debug)] + struct Replica { + id: i64, // non-zero ID + public_key: PublicKey, + secret_key: SecretKey, + } + + impl HasKey for Replica { + type Key = PublicKey; + + fn key(&self) -> &Self::Key { + &self.public_key + } + } + + impl Twin for Replica { + fn to_twin(&self) -> Self { + Self { + id: -self.id, + public_key: self.public_key.clone(), + secret_key: self.secret_key.clone(), + } + } + } + + let (num_scenarios, num_reseeds) = match scenarios { + TwinsScenarios::Multiple(n) => (n, 0), + TwinsScenarios::Reseeds(n) => (1, n), + }; + + // Keep scenarios separate by generating a different RNG many times. + let mut rng = ctx.rng(); + for _ in 0..num_reseeds { + rng = ctx.rng(); + } + let rng = &mut rng; + + // The existing test machinery uses the number of finalized blocks as an exit criteria. + let blocks_to_finalize = 3; + // The test is going to disrupt the communication by partitioning nodes, + // where the leader might not be in a partition with enough replicas to + // form a quorum, therefore to allow N blocks to be finalized we need to + // go longer. + let num_rounds = blocks_to_finalize * 10; + // The paper considers 2 or 3 partitions enough. + let max_partitions = 3; + + // Every validator has equal power of 1. + const WEIGHT: u64 = 1; + let mut spec = SetupSpec::new_with_weights(rng, vec![WEIGHT; num_replicas]); + + let replicas = spec + .validator_weights + .iter() + .enumerate() + .map(|(i, (sk, _))| Replica { + id: i as i64 + 1, + public_key: sk.public(), + secret_key: sk.clone(), + }) + .collect::>(); + + let cluster = Cluster::new(replicas, num_twins); + let scenarios = ScenarioGenerator::<_, NUM_PHASES>::new(&cluster, num_rounds, max_partitions); + + // Gossip with more nodes than what can be faulty. + let gossip_peers = num_twins + 1; + + // Create network config for all nodes in the cluster (assigns unique network addresses). + let nets = new_configs_for_validators( + rng, + cluster.nodes().iter().map(|r| &r.secret_key), + gossip_peers, + ); + + let node_to_port = cluster + .nodes() + .iter() + .zip(nets.iter()) + .map(|(node, net)| (node.id, net.server_addr.port())) + .collect::>(); + + assert_eq!(node_to_port.len(), cluster.num_nodes()); + + // Every network needs a behaviour. They are all honest, just some might be duplicated. + let nodes = vec![(Behavior::Honest, WEIGHT); cluster.num_nodes()]; + + // Reuse the same cluster and network setup to run a few scenarios. + for i in 0..num_scenarios { + // Generate a permutation of partitions and leaders for the given number of rounds. + let scenario = scenarios.generate_one(rng); + + // Assign the leadership schedule to the consensus. + spec.leader_selection = + LeaderSelectionMode::Rota(scenario.rounds.iter().map(|rc| rc.leader.clone()).collect()); + + // Generate a new setup with this leadership schedule. + let setup = Setup::from_spec(rng, spec.clone()); + + // Create a network with the partition schedule of the scenario. + let splits: PortSplitSchedule = scenario + .rounds + .iter() + .map(|rc| { + std::array::from_fn(|i| { + rc.phase_partitions[i] + .iter() + .map(|p| p.iter().map(|r| node_to_port[&r.id]).collect()) + .collect() + }) + }) + .collect(); + + tracing::info!( + "num_replicas={num_replicas} num_twins={num_twins} num_nodes={} scenario={i}", + cluster.num_nodes() + ); + + // Debug output of round schedule. + for (r, rc) in scenario.rounds.iter().enumerate() { + // Let's just consider the partition of the LeaderCommit phase, for brevity's sake. + let partitions = &splits[r].last().unwrap(); + + let leader_ports = cluster + .nodes() + .iter() + .filter(|n| n.public_key == *rc.leader) + .map(|n| node_to_port[&n.id]) + .collect::>(); + + let leader_partition_sizes = leader_ports + .iter() + .map(|lp| partitions.iter().find(|p| p.contains(lp)).unwrap().len()) + .collect::>(); + + let leader_isolated = leader_partition_sizes + .iter() + .all(|s| *s < cluster.quorum_size()); + + tracing::info!("round={r} partitions={partitions:?} leaders={leader_ports:?} leader_partition_sizes={leader_partition_sizes:?} leader_isolated={leader_isolated}"); + } + + Test { + network: Network::Twins(PortRouter::Splits(splits)), + nodes: nodes.clone(), + blocks_to_finalize, + } + .run_with_config(ctx, nets.clone(), &setup) + .await? + } + + Ok(()) +} + +/// Run Twins scenarios without actual twins, and with so few nodes that all +/// of them are required for a quorum, which means (currently) there won't be +/// any partitions. +/// +/// This should be a simple sanity check that the network works and consensus +/// is achieved under the most favourable conditions. +#[test_casing(10,0..10)] +#[tokio::test] +async fn twins_network_wo_twins_wo_partitions(num_reseeds: usize) { + tokio::time::pause(); + // n<6 implies f=0 and q=n + run_twins(5, 0, TwinsScenarios::Reseeds(num_reseeds)) + .await + .unwrap(); +} + +/// Run Twins scenarios without actual twins, but enough replicas that partitions +/// can play a role, isolating certain nodes (potentially the leader) in some +/// rounds. +/// +/// This should be a sanity check that without Byzantine behaviour the consensus +/// is resilient to temporary network partitions. +#[test_casing(5,0..5)] +#[tokio::test] +async fn twins_network_wo_twins_w_partitions(num_reseeds: usize) { + tokio::time::pause(); + // n=6 implies f=1 and q=5; 6 is the minimum where partitions are possible. + run_twins(6, 0, TwinsScenarios::Reseeds(num_reseeds)) + .await + .unwrap(); +} + +/// Test cases with 1 twin, with 6-10 replicas, 10 scenarios each. +const CASES_TWINS_1: TestCases<(usize, usize)> = cases! { + (6..=10).flat_map(|num_replicas| (0..10).map(move |num_reseeds| (num_replicas, num_reseeds))) +}; + +/// Run Twins scenarios with random number of nodes and 1 twin. +#[test_casing(50, CASES_TWINS_1)] +#[tokio::test] +async fn twins_network_w1_twins_w_partitions(num_replicas: usize, num_reseeds: usize) { + tokio::time::pause(); + // n>=6 implies f>=1 and q=n-f + // let num_honest = validator::threshold(num_replicas as u64) as usize; + // let max_faulty = num_replicas - num_honest; + // let num_twins = rng.gen_range(1..=max_faulty); + run_twins(num_replicas, 1, TwinsScenarios::Reseeds(num_reseeds)) + .await + .unwrap(); +} + +/// Run Twins scenarios with higher number of nodes and 2 twins. +#[test_casing(5,0..5)] +#[tokio::test] +async fn twins_network_w2_twins_w_partitions(num_reseeds: usize) { + tokio::time::pause(); + // n>=11 implies f>=2 and q=n-f + run_twins(11, 2, TwinsScenarios::Reseeds(num_reseeds)) + .await + .unwrap(); +} + +/// Run Twins scenario with more twins than tolerable and expect it to fail. +#[tokio::test] +async fn twins_network_to_fail() { + tokio::time::pause(); + assert_matches!( + // All twins! To find a conflict quicker. + run_twins(6, 6, TwinsScenarios::Multiple(150)).await, + Err(TestError::BlockConflict) + ); +} diff --git a/node/actors/network/src/consensus/mod.rs b/node/actors/network/src/consensus/mod.rs index b63a3248..7b6e3f9a 100644 --- a/node/actors/network/src/consensus/mod.rs +++ b/node/actors/network/src/consensus/mod.rs @@ -59,10 +59,10 @@ impl MsgPool { // an implementation detail of the bft crate. Consider moving // this logic there. match (&v.message.msg, &msg.message.msg) { - (M::ReplicaPrepare(_), M::ReplicaPrepare(_)) => {} + (M::LeaderProposal(_), M::LeaderProposal(_)) => {} (M::ReplicaCommit(_), M::ReplicaCommit(_)) => {} - (M::LeaderPrepare(_), M::LeaderPrepare(_)) => {} - (M::LeaderCommit(_), M::LeaderCommit(_)) => {} + (M::ReplicaNewView(_), M::ReplicaNewView(_)) => {} + (M::ReplicaTimeout(_), M::ReplicaTimeout(_)) => {} _ => return true, } // If pool contains a message of the same type which is newer, @@ -229,15 +229,8 @@ impl Network { let mut sub = self.msg_pool.subscribe(); loop { let call = consensus_cli.reserve(ctx).await?; - let msg = loop { - let msg = sub.recv(ctx).await?; - match &msg.recipient { - io::Target::Broadcast => {} - io::Target::Validator(recipient) if recipient == peer => {} - _ => continue, - } - break msg.message.clone(); - }; + let msg = sub.recv(ctx).await?.message.clone(); + s.spawn(async { let req = rpc::consensus::Req(msg); let res = call.call(ctx, &req, RESP_MAX_SIZE).await; diff --git a/node/actors/network/src/consensus/tests.rs b/node/actors/network/src/consensus/tests.rs index dc63d4ee..a34062f9 100644 --- a/node/actors/network/src/consensus/tests.rs +++ b/node/actors/network/src/consensus/tests.rs @@ -26,10 +26,10 @@ async fn test_msg_pool() { // We keep them sorted by type and view, so that it is easy to // compute the expected state of the pool after insertions. let msgs = [ - gen(&mut || M::ReplicaPrepare(rng.gen())), + gen(&mut || M::LeaderProposal(rng.gen())), gen(&mut || M::ReplicaCommit(rng.gen())), - gen(&mut || M::LeaderPrepare(rng.gen())), - gen(&mut || M::LeaderCommit(rng.gen())), + gen(&mut || M::ReplicaNewView(rng.gen())), + gen(&mut || M::ReplicaTimeout(rng.gen())), ]; // Insert messages at random. @@ -42,7 +42,6 @@ async fn test_msg_pool() { want[i] = Some(want[i].unwrap_or(0).max(j)); pool.send(Arc::new(io::ConsensusInputMessage { message: msgs[i][j].clone(), - recipient: io::Target::Broadcast, })); // Here we compare the internal state of the pool to the expected state. // Note that we compare sets of crypto hashes of messages, because the messages themselves do not @@ -310,9 +309,6 @@ async fn test_transmission() { let want: validator::Signed = want.cast().unwrap(); let in_message = io::ConsensusInputMessage { message: want.clone(), - recipient: io::Target::Validator( - nodes[1].cfg().validator_key.as_ref().unwrap().public(), - ), }; nodes[0].pipe.send(in_message.into()); @@ -355,7 +351,6 @@ async fn test_retransmission() { node0.pipe.send( io::ConsensusInputMessage { message: want.clone(), - recipient: io::Target::Broadcast, } .into(), ); diff --git a/node/actors/network/src/io.rs b/node/actors/network/src/io.rs index 9a7412f9..6166deef 100644 --- a/node/actors/network/src/io.rs +++ b/node/actors/network/src/io.rs @@ -13,7 +13,6 @@ pub enum InputMessage { #[derive(Debug, PartialEq)] pub struct ConsensusInputMessage { pub message: validator::Signed, - pub recipient: Target, } impl From for InputMessage { @@ -39,9 +38,3 @@ pub enum OutputMessage { /// Message to the Consensus actor. Consensus(ConsensusReq), } - -#[derive(Clone, Debug, PartialEq, Eq)] -pub enum Target { - Validator(validator::PublicKey), - Broadcast, -} diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index 64f870d9..e6d5ca3d 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -1,9 +1,8 @@ //! Testonly utilities. #![allow(dead_code)] use crate::{ - gossip::attestation, - io::{ConsensusInputMessage, Target}, - Config, GossipConfig, Network, RpcConfig, Runner, + gossip::attestation, io::ConsensusInputMessage, Config, GossipConfig, Network, RpcConfig, + Runner, }; use rand::{ distributions::{Distribution, Standard}, @@ -21,21 +20,9 @@ use zksync_consensus_roles::{node, validator}; use zksync_consensus_storage::BlockStore; use zksync_consensus_utils::pipe; -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> Target { - match rng.gen_range(0..2) { - 0 => Target::Broadcast, - _ => Target::Validator(rng.gen()), - } - } -} - impl Distribution for Standard { fn sample(&self, rng: &mut R) -> ConsensusInputMessage { - ConsensusInputMessage { - message: rng.gen(), - recipient: rng.gen(), - } + ConsensusInputMessage { message: rng.gen() } } } diff --git a/node/libs/roles/src/proto/validator/keys.proto b/node/libs/roles/src/proto/validator/keys.proto index 245e0c57..3a3ffeeb 100644 --- a/node/libs/roles/src/proto/validator/keys.proto +++ b/node/libs/roles/src/proto/validator/keys.proto @@ -3,17 +3,17 @@ syntax = "proto3"; package zksync.roles.validator; message PublicKey { - // The name is wrong, it should be bls12_381. + // TODO: The name is wrong, it should be bls12_381. optional bytes bn254 = 1; // required } message Signature { - // The name is wrong, it should be bls12_381. + // TODO: The name is wrong, it should be bls12_381. optional bytes bn254 = 1; // required } message AggregateSignature { - // The name is wrong, it should be bls12_381. + // TODO: The name is wrong, it should be bls12_381. optional bytes bn254 = 1; // required } diff --git a/node/libs/roles/src/proto/validator/messages.proto b/node/libs/roles/src/proto/validator/messages.proto index 2b7038c5..209c4889 100644 --- a/node/libs/roles/src/proto/validator/messages.proto +++ b/node/libs/roles/src/proto/validator/messages.proto @@ -72,9 +72,8 @@ message ReplicaNewView { } message LeaderProposal { - optional BlockHeader proposal = 1; // required - optional bytes proposal_payload = 2; // optional (depending on justification) - optional ProposalJustification justification = 3; // required + optional bytes proposal_payload = 1; // optional (depending on justification) + optional ProposalJustification justification = 2; // required } message CommitQC { diff --git a/node/libs/roles/src/validator/conv.rs b/node/libs/roles/src/validator/conv.rs index 3dade788..5addcab2 100644 --- a/node/libs/roles/src/validator/conv.rs +++ b/node/libs/roles/src/validator/conv.rs @@ -290,7 +290,6 @@ impl ProtoFmt for LeaderProposal { fn read(r: &Self::Proto) -> anyhow::Result { Ok(Self { - 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")?, }) @@ -298,7 +297,6 @@ impl ProtoFmt for LeaderProposal { fn build(&self) -> Self::Proto { Self::Proto { - proposal: Some(self.proposal.build()), proposal_payload: self.proposal_payload.as_ref().map(|p| p.0.clone()), justification: Some(self.justification.build()), } diff --git a/node/libs/roles/src/validator/messages/block.rs b/node/libs/roles/src/validator/messages/block.rs index 7db2628b..2c0f74af 100644 --- a/node/libs/roles/src/validator/messages/block.rs +++ b/node/libs/roles/src/validator/messages/block.rs @@ -26,6 +26,16 @@ impl Payload { pub fn hash(&self) -> PayloadHash { PayloadHash(Keccak256::new(&self.0)) } + + /// Returns the length of the payload. + pub fn len(&self) -> usize { + self.0.len() + } + + /// Returns `true` if the payload is empty. + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } } /// Hash of the Payload. diff --git a/node/libs/roles/src/validator/messages/committee.rs b/node/libs/roles/src/validator/messages/committee.rs index 018dbb4f..1241c540 100644 --- a/node/libs/roles/src/validator/messages/committee.rs +++ b/node/libs/roles/src/validator/messages/committee.rs @@ -161,7 +161,7 @@ pub struct WeightedValidator { pub weight: Weight, } -/// Voting weight; +/// Voting weight. pub type Weight = u64; /// The mode used for selecting leader for a given view. diff --git a/node/libs/roles/src/validator/messages/consensus.rs b/node/libs/roles/src/validator/messages/consensus.rs index 1b723714..1ff67b4d 100644 --- a/node/libs/roles/src/validator/messages/consensus.rs +++ b/node/libs/roles/src/validator/messages/consensus.rs @@ -152,7 +152,8 @@ impl fmt::Display for ViewNumber { pub struct Signers(pub BitVec); impl Signers { - /// Constructs an empty signers set. + /// Constructs a new Signers bitmap with the given number of validators. All + /// bits are set to false. pub fn new(n: usize) -> Self { Self(BitVec::from_elem(n, false)) } diff --git a/node/libs/roles/src/validator/messages/leader_proposal.rs b/node/libs/roles/src/validator/messages/leader_proposal.rs index c421e054..e05c6668 100644 --- a/node/libs/roles/src/validator/messages/leader_proposal.rs +++ b/node/libs/roles/src/validator/messages/leader_proposal.rs @@ -1,13 +1,11 @@ use super::{ - BlockHeader, BlockNumber, CommitQC, CommitQCVerifyError, Genesis, Payload, PayloadHash, - TimeoutQC, TimeoutQCVerifyError, View, + BlockNumber, CommitQC, CommitQCVerifyError, Genesis, Payload, PayloadHash, TimeoutQC, + TimeoutQCVerifyError, View, }; /// A proposal message from the leader. #[derive(Clone, Debug, PartialEq, Eq)] pub struct LeaderProposal { - /// 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, @@ -26,47 +24,7 @@ impl LeaderProposal { // Check that the justification is valid. self.justification .verify(genesis) - .map_err(LeaderProposalVerifyError::Justification)?; - - // Get the implied block number and payload hash and check it against the proposal. - let (implied_block_number, implied_payload) = self.justification.get_implied_block(genesis); - - if self.proposal.number != implied_block_number { - return Err(LeaderProposalVerifyError::BadBlockNumber { - got: self.proposal.number, - want: implied_block_number, - }); - } - - if let Some(payload_hash) = implied_payload { - if self.proposal.payload != payload_hash { - return Err(LeaderProposalVerifyError::BadPayloadHash { - got: self.proposal.payload, - want: payload_hash, - }); - } - } - - // Check if we are correctly proposing a new block or re-proposing an old one. - if implied_payload.is_none() && self.proposal_payload.is_none() { - return Err(LeaderProposalVerifyError::ReproposalWhenPreviousFinalized); - } - - if implied_payload.is_some() && self.proposal_payload.is_some() { - return Err(LeaderProposalVerifyError::NewProposalWhenPreviousNotFinalized); - } - - // Check that the payload matches the header, if it exists. - if let Some(payload) = &self.proposal_payload { - if payload.hash() != self.proposal.payload { - return Err(LeaderProposalVerifyError::MismatchedPayload { - header: self.proposal.payload, - payload: payload.hash(), - }); - } - } - - Ok(()) + .map_err(LeaderProposalVerifyError::Justification) } } @@ -76,36 +34,6 @@ pub enum LeaderProposalVerifyError { /// Invalid Justification. #[error("Invalid justification: {0:#}")] Justification(ProposalJustificationVerifyError), - /// Bad block number. - #[error("Bad block number: got {got:?}, want {want:?}")] - BadBlockNumber { - /// Received proposal number. - got: BlockNumber, - /// Correct proposal number. - want: BlockNumber, - }, - /// Bad payload hash on reproposal. - #[error("Bad payload hash on reproposal: got {got:?}, want {want:?}")] - BadPayloadHash { - /// Received payload hash. - got: PayloadHash, - /// Correct payload hash. - want: PayloadHash, - }, - /// New block proposal when the previous proposal was not finalized. - #[error("New block proposal when the previous proposal was not finalized")] - NewProposalWhenPreviousNotFinalized, - /// Re-proposal when the previous proposal was finalized. - #[error("Block re-proposal when the previous proposal was finalized")] - ReproposalWhenPreviousFinalized, - /// Mismatched payload. - #[error("Block proposal with mismatched payload: header {header:?}, payload {payload:?}")] - MismatchedPayload { - /// Payload hash on block header. - header: PayloadHash, - /// Correct payload hash. - payload: PayloadHash, - }, } /// Justification for a proposal. This is either a Commit QC or a Timeout QC. @@ -184,9 +112,11 @@ impl ProposalJustification { // Either the previous proposal was finalized or we know for certain // that it couldn't have been finalized (because there is no high vote). // Either way, we can propose a new block. + + // If there is no high QC, then we must be at the start of the chain. let block_number = match high_qc { Some(qc) => qc.header().number.next(), - None => BlockNumber(0), + None => genesis.first_block, }; (block_number, None) diff --git a/node/libs/roles/src/validator/messages/replica_commit.rs b/node/libs/roles/src/validator/messages/replica_commit.rs index 13563a84..939df5cc 100644 --- a/node/libs/roles/src/validator/messages/replica_commit.rs +++ b/node/libs/roles/src/validator/messages/replica_commit.rs @@ -31,7 +31,7 @@ pub enum ReplicaCommitVerifyError { /// A Commit Quorum Certificate. It is an aggregate of signed ReplicaCommit messages. /// The Commit Quorum Certificate is over identical messages, so we only need one message. -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] +#[derive(Clone, Debug, PartialEq, Eq)] pub struct CommitQC { /// The ReplicaCommit message that the QC is for. pub message: ReplicaCommit, @@ -137,6 +137,18 @@ impl CommitQC { } } +impl Ord for CommitQC { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.message.view.number.cmp(&other.message.view.number) + } +} + +impl PartialOrd for CommitQC { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + /// Error returned by `CommitQC::add()`. #[derive(thiserror::Error, Debug)] pub enum CommitQCAddError { diff --git a/node/libs/roles/src/validator/messages/replica_timeout.rs b/node/libs/roles/src/validator/messages/replica_timeout.rs index 5851a0c0..0a9edc3f 100644 --- a/node/libs/roles/src/validator/messages/replica_timeout.rs +++ b/node/libs/roles/src/validator/messages/replica_timeout.rs @@ -212,6 +212,18 @@ impl TimeoutQC { } } +impl Ord for TimeoutQC { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.view.number.cmp(&other.view.number) + } +} + +impl PartialOrd for TimeoutQC { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + /// Error returned by `TimeoutQC::add()`. #[derive(thiserror::Error, Debug)] pub enum TimeoutQCAddError { diff --git a/node/libs/roles/src/validator/messages/tests/leader_proposal.rs b/node/libs/roles/src/validator/messages/tests/leader_proposal.rs index 11b66e63..54b99946 100644 --- a/node/libs/roles/src/validator/messages/tests/leader_proposal.rs +++ b/node/libs/roles/src/validator/messages/tests/leader_proposal.rs @@ -13,17 +13,12 @@ fn test_leader_proposal_verify() { // Valid proposal let payload: Payload = rng.gen(); - let block_header = BlockHeader { - number: setup.next(), - payload: payload.hash(), - }; let commit_qc = match setup.blocks.last().unwrap() { Block::Final(block) => block.justification.clone(), _ => unreachable!(), }; let justification = ProposalJustification::Commit(commit_qc); let proposal = LeaderProposal { - proposal: block_header, proposal_payload: Some(payload.clone()), justification, }; @@ -38,62 +33,6 @@ fn test_leader_proposal_verify() { wrong_proposal.verify(&setup.genesis), Err(LeaderProposalVerifyError::Justification(_)) ); - - // Invalid block number - let mut wrong_proposal = proposal.clone(); - wrong_proposal.proposal.number = BlockNumber(1); - - assert_matches!( - wrong_proposal.verify(&setup.genesis), - Err(LeaderProposalVerifyError::BadBlockNumber { .. }) - ); - - // Wrong reproposal - let mut wrong_proposal = proposal.clone(); - wrong_proposal.proposal_payload = None; - - assert_matches!( - wrong_proposal.verify(&setup.genesis), - Err(LeaderProposalVerifyError::ReproposalWhenPreviousFinalized) - ); - - // Invalid payload - let mut wrong_proposal = proposal.clone(); - wrong_proposal.proposal.payload = rng.gen(); - - assert_matches!( - wrong_proposal.verify(&setup.genesis), - Err(LeaderProposalVerifyError::MismatchedPayload { .. }) - ); - - // New leader proposal with a reproposal - let timeout_qc = setup.make_timeout_qc(rng, ViewNumber(7), Some(&payload)); - let justification = ProposalJustification::Timeout(timeout_qc); - let proposal = LeaderProposal { - proposal: block_header, - proposal_payload: None, - justification, - }; - - assert!(proposal.verify(&setup.genesis).is_ok()); - - // Invalid payload hash - let mut wrong_proposal = proposal.clone(); - wrong_proposal.proposal.payload = rng.gen(); - - assert_matches!( - wrong_proposal.verify(&setup.genesis), - Err(LeaderProposalVerifyError::BadPayloadHash { .. }) - ); - - // Wrong new proposal - let mut wrong_proposal = proposal.clone(); - wrong_proposal.proposal_payload = Some(rng.gen()); - - assert_matches!( - wrong_proposal.verify(&setup.genesis), - Err(LeaderProposalVerifyError::NewProposalWhenPreviousNotFinalized) - ); } #[test] @@ -102,12 +41,7 @@ fn test_justification_get_implied_block() { let rng = &mut ctx.rng(); let mut setup = Setup::new(rng, 6); setup.push_blocks(rng, 3); - let payload: Payload = rng.gen(); - let block_header = BlockHeader { - number: setup.next(), - payload: payload.hash(), - }; // Justification with a commit QC let commit_qc = match setup.blocks.last().unwrap() { @@ -116,7 +50,6 @@ fn test_justification_get_implied_block() { }; let justification = ProposalJustification::Commit(commit_qc); let proposal = LeaderProposal { - proposal: block_header, proposal_payload: Some(payload.clone()), justification, }; @@ -131,7 +64,6 @@ fn test_justification_get_implied_block() { let timeout_qc = setup.make_timeout_qc(rng, ViewNumber(7), Some(&payload)); let justification = ProposalJustification::Timeout(timeout_qc); let proposal = LeaderProposal { - proposal: block_header, proposal_payload: None, justification, }; diff --git a/node/libs/roles/src/validator/messages/tests/mod.rs b/node/libs/roles/src/validator/messages/tests/mod.rs index b8b703b7..798f74ed 100644 --- a/node/libs/roles/src/validator/messages/tests/mod.rs +++ b/node/libs/roles/src/validator/messages/tests/mod.rs @@ -131,7 +131,6 @@ fn genesis_with_attesters() -> Genesis { /// Hardcoded `LeaderProposal`. fn leader_proposal() -> LeaderProposal { LeaderProposal { - proposal: block_header(), proposal_payload: Some(payload()), justification: ProposalJustification::Timeout(timeout_qc()), } diff --git a/node/libs/roles/src/validator/messages/tests/replica_timeout.rs b/node/libs/roles/src/validator/messages/tests/replica_timeout.rs index e07221cc..09e5c49c 100644 --- a/node/libs/roles/src/validator/messages/tests/replica_timeout.rs +++ b/node/libs/roles/src/validator/messages/tests/replica_timeout.rs @@ -46,7 +46,7 @@ fn test_timeout_qc_high_vote() { let msg_c = setup.make_replica_timeout(rng, view_num); // Case with 1 subquorum. - let mut qc = TimeoutQC::new(msg_a.view.clone()); + let mut qc = TimeoutQC::new(msg_a.view); for key in &setup.validator_keys { qc.add(&key.sign_msg(msg_a.clone()), &setup.genesis) @@ -56,7 +56,7 @@ fn test_timeout_qc_high_vote() { assert!(qc.high_vote(&setup.genesis).is_some()); // Case with 2 subquorums. - let mut qc = TimeoutQC::new(msg_a.view.clone()); + let mut qc = TimeoutQC::new(msg_a.view); for key in &setup.validator_keys[0..3] { qc.add(&key.sign_msg(msg_a.clone()), &setup.genesis) @@ -71,7 +71,7 @@ fn test_timeout_qc_high_vote() { assert!(qc.high_vote(&setup.genesis).is_none()); // Case with no subquorums. - let mut qc = TimeoutQC::new(msg_a.view.clone()); + let mut qc = TimeoutQC::new(msg_a.view); for key in &setup.validator_keys[0..2] { qc.add(&key.sign_msg(msg_a.clone()), &setup.genesis) @@ -131,7 +131,7 @@ fn test_timeout_qc_add() { let setup = Setup::new(rng, 3); let view = rng.gen(); let msg = setup.make_replica_timeout(rng, view); - let mut qc = TimeoutQC::new(msg.view.clone()); + let mut qc = TimeoutQC::new(msg.view); // Add the first signature assert!(qc.map.is_empty()); @@ -241,7 +241,7 @@ fn test_timeout_qc_verify() { let mut qc2 = qc.clone(); qc2.map.insert( ReplicaTimeout { - view: qc2.view.clone().next(), + view: qc2.view.next(), high_vote: None, high_qc: None, }, @@ -256,7 +256,7 @@ fn test_timeout_qc_verify() { let mut qc3 = qc.clone(); qc3.map.insert( ReplicaTimeout { - view: qc3.view.clone(), + view: qc3.view, high_vote: None, high_qc: None, }, @@ -271,7 +271,7 @@ fn test_timeout_qc_verify() { let mut qc4 = qc.clone(); qc4.map.insert( ReplicaTimeout { - view: qc4.view.clone(), + view: qc4.view, high_vote: None, high_qc: None, }, @@ -290,7 +290,7 @@ fn test_timeout_qc_verify() { .set(rng.gen_range(0..setup.genesis.validators.len()), true); qc5.map.insert( ReplicaTimeout { - view: qc5.view.clone(), + view: qc5.view, high_vote: None, high_qc: None, }, diff --git a/node/libs/roles/src/validator/messages/tests/versions.rs b/node/libs/roles/src/validator/messages/tests/versions.rs index 3832c523..a19f152a 100644 --- a/node/libs/roles/src/validator/messages/tests/versions.rs +++ b/node/libs/roles/src/validator/messages/tests/versions.rs @@ -95,8 +95,8 @@ mod version1 { fn leader_proposal_change_detector() { msg_change_detector( leader_proposal().insert(), - "validator_msg:keccak256:7b079e4ca3021834fa35745cb042fea6dd5bb89a91ca5ba31ed6ba1765a1e113", - "validator:signature:bls12_381:98ca0f24d87f938b22ac9c2a2720466cd157a502b31ae5627ce5fdbda6de0ad6d2e9b159cf816cd1583644f2f69ecb84", + "validator_msg:keccak256:4c1b2cf1e8fbb00cde86caee200491df15c45d5c88402e227c1f3e1b416c4255", + "validator:signature:bls12_381:81f865807067c6f70f17f9716e6d41c0103c2366abb6721408fb7d27ead6332798bd7b34d5f4a63e324082586b2c69a3", ); } } diff --git a/node/libs/roles/src/validator/testonly.rs b/node/libs/roles/src/validator/testonly.rs index eff17eda..5ceff2ac 100644 --- a/node/libs/roles/src/validator/testonly.rs +++ b/node/libs/roles/src/validator/testonly.rs @@ -198,7 +198,7 @@ impl Setup { } } - /// Creates a ReplicaCommt with a random payload. + /// Creates a ReplicaCommit with a random payload. pub fn make_replica_commit(&self, rng: &mut impl Rng, view: ViewNumber) -> ReplicaCommit { ReplicaCommit { view: self.make_view(view), @@ -209,7 +209,7 @@ impl Setup { } } - /// Creates a ReplicaCommt with the given payload. + /// Creates a ReplicaCommit with the given payload. pub fn make_replica_commit_with_payload( &self, payload: &Payload, @@ -523,7 +523,6 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> LeaderProposal { LeaderProposal { - proposal: rng.gen(), proposal_payload: rng.gen(), justification: rng.gen(), } diff --git a/node/libs/storage/src/proto/mod.proto b/node/libs/storage/src/proto/mod.proto index e06e84da..50744fd3 100644 --- a/node/libs/storage/src/proto/mod.proto +++ b/node/libs/storage/src/proto/mod.proto @@ -13,6 +13,8 @@ message ReplicaState { optional uint64 view = 1; // required; ViewNumber optional roles.validator.Phase phase = 2; // required optional roles.validator.ReplicaCommit high_vote = 3; // optional + // TODO: name should be high_commit_qc optional roles.validator.CommitQC high_qc = 4; // optional repeated Proposal proposals = 5; + optional roles.validator.TimeoutQC high_timeout_qc = 6; // optional } diff --git a/node/libs/storage/src/replica_store.rs b/node/libs/storage/src/replica_store.rs index 465d26d6..da7698ff 100644 --- a/node/libs/storage/src/replica_store.rs +++ b/node/libs/storage/src/replica_store.rs @@ -41,7 +41,9 @@ pub struct ReplicaState { /// The highest block proposal that the replica has committed to. pub high_vote: Option, /// The highest commit quorum certificate known to the replica. - pub high_qc: Option, + pub high_commit_qc: Option, + /// The highest timeout quorum certificate known to the replica. + pub high_timeout_qc: Option, /// A cache of the received block proposals. pub proposals: Vec, } @@ -52,7 +54,8 @@ impl Default for ReplicaState { view: validator::ViewNumber(0), phase: validator::Phase::Prepare, high_vote: None, - high_qc: None, + high_commit_qc: None, + high_timeout_qc: None, proposals: vec![], } } @@ -84,7 +87,8 @@ impl ProtoFmt for ReplicaState { view: validator::ViewNumber(r.view.context("view_number")?), phase: read_required(&r.phase).context("phase")?, high_vote: read_optional(&r.high_vote).context("high_vote")?, - high_qc: read_optional(&r.high_qc).context("high_qc")?, + high_commit_qc: read_optional(&r.high_qc).context("high_commit_qc")?, + high_timeout_qc: read_optional(&r.high_timeout_qc).context("high_timeout_qc")?, proposals: r .proposals .iter() @@ -99,7 +103,8 @@ impl ProtoFmt for ReplicaState { view: Some(self.view.0), phase: Some(self.phase.build()), high_vote: self.high_vote.as_ref().map(|x| x.build()), - high_qc: self.high_qc.as_ref().map(|x| x.build()), + high_qc: self.high_commit_qc.as_ref().map(|x| x.build()), + high_timeout_qc: self.high_timeout_qc.as_ref().map(|x| x.build()), proposals: self.proposals.iter().map(|p| p.build()).collect(), } } diff --git a/node/libs/storage/src/testonly/mod.rs b/node/libs/storage/src/testonly/mod.rs index c36d3b74..e52293ea 100644 --- a/node/libs/storage/src/testonly/mod.rs +++ b/node/libs/storage/src/testonly/mod.rs @@ -26,7 +26,8 @@ impl Distribution for Standard { view: rng.gen(), phase: rng.gen(), high_vote: rng.gen(), - high_qc: rng.gen(), + high_commit_qc: rng.gen(), + high_timeout_qc: rng.gen(), proposals: (0..rng.gen_range(1..11)).map(|_| rng.gen()).collect(), } } diff --git a/spec/README.md b/spec/README.md index 2ef8e82f..4380820a 100644 --- a/spec/README.md +++ b/spec/README.md @@ -1,3 +1,34 @@ -# ChonkyBFT's Specification +# ChonkyBFT -This is a formal specification of ChonkyBFT consensus protocol in Quint. +This folder contains the specification of the ChonkyBFT, a new consensus protocol created by Bruno França and Grzegorz Prusak at Matter Labs. It has both the pseudo-code specification that was used as the basis for the Rust implementation in the rest of this repo and the Quint specification that was used to formally verify the protocol. +Chonky BFT is a consensus protocol inspired by [FaB Paxos](https://www.cs.cornell.edu/lorenzo/papers/Martin06Fast.pdf), [Fast-HotStuff](https://arxiv.org/abs/2010.11454) and [HotStuff-2](https://eprint.iacr.org/2023/397). +It is committee-based and has only one round of voting, single slot finality, quadratic communication and _n=5f+1_ fault tolerance. Let's discuss what were our objectives when designing ChonkyBFT. + +## Design goals in practice vs. theory + +We find that most recent research on consensus algorithms unfortunately has become somewhat detached from the realities of running those same consensus algorithms in practice. This has led to researchers optimizing algorithms along the wrong dimensions. Many times we see tables in papers comparing different algorithms along metrics that genuinely don’t matter when those algorithms are implemented. + +### What doesn’t matter + +- Authenticator complexity: This is probably the worst one. Optimizing to have fewer signatures made sense decades ago when crypto operations were expensive. Today, digital signatures are fast and small. However, many papers (for example HotStuff) still report this measure and even go as far as suggesting threshold signatures over multisignatures, which introduces a much more complex step of distributed key generation instead of spending some more milliseconds on verifying the signatures. +- Message complexity: This also tends to be a red herring. In theory, the fewer messages are passed around the network, the faster the algorithm will be. In practice, it depends on where the bottleneck is. If your algorithm has linear communication, but the leader still has to send and receive N messages, then you are not gaining any meaningful performance. This also has the unfortunate effect of treating every message the same, while in practice a block proposal can be megabytes long and a block commit is a few kilobytes at most. +- Block latency: This is the wrong latency to consider. It doesn’t matter if our block time is 0.1s, if then we have to wait 100 blocks to finalize. All it matters is how long it takes for an user to see their transaction finalized. This has led to algorithms like Narwhal and Tusk, which claim to have just one round of voting but another round “hidden” in the block broadcast mechanism. This actually leads to a worse latency for the user, even though the block times are shorter. + +### What does matter + +- Systemic complexity: This relates to the [systemic vs. encapsulated complexity](https://vitalik.eth.limo/general/2022/02/28/complexity.html) topic. Our consensus algorithms are not run in isolation, they are meant to support other applications. An example of this problem is probabilistic vs provable finality. Algorithms that finalize probabilistically impose complexity on the applications. Exchanges must determine how many confirmations to wait for each different chain they accept, the same for multi-chain dapps, hybrid dapps, block explorers, wallets, etc. Algorithms that finalize provably give a clear signal to every application that they can use. This is important enough that even Ethereum is planning to move to [single-slot finality](https://ethereum.org/en/roadmap/single-slot-finality/#why-aim-for-quicker-finality), because not finalizing every block is not enough. +- Simplicity: To model and implement the algorithm. Your algorithm might be able to save one round-trip in an optimistic scenario, but is it worth it if it’s too complex to create a formal model out of it? And if then the implementation will take 4 engineers and 3 audits? Simple algorithms that can be formally proven and are straight-forward to implement are more secure algorithms. A bug that causes downtime (or even worse, safety violations) is much worse for the UX than slightly slower block times. +- Transaction latency: What was discussed before. The only latency that matters is the one experienced by the user. + +## Lessons learned + +For our particular use case, there are a few lessons that we learned from researching and implementing previous consensus algorithms: + +- Chained consensus is not worth it. It doesn’t improve the throughput or the latency while increasing systemic complexity. We always finalize every block. +- Lower fault tolerance to reduce voting rounds. This we learned from FaB Paxos. Decreasing our fault tolerance from *3f+1* to *5f+1* allows us to finalize in just one voting round. +- Linear communication is not worth it. Quadratic communication for replicas simplifies security (there are fewer cases where we need to consider the effect of a malicious leader), implementation (you can fully separate the leader component) and view changes (constant timeouts are enough, [Jolteon/Ditto](https://arxiv.org/abs/2106.10362) ended up going in that direction after trying to implement HotStuff). Further, the performance drop is likely not significant (see [ParBFT](https://eprint.iacr.org/2023/679.pdf)). +- Re-proposals as a way of guaranteeing that there are no “rogue” blocks. This is a problem that didn’t get any attention so far (as far as we know), and is probably somewhat unique to public blockchains. The issue is that in all committee-based consensus algorithms it is possible that a commit QC (to use HotStuff’s terminology) is formed but that not enough replicas receive it. This will cause a timeout and another block to be proposed. Most algorithms just solve this by saying that the old block is no longer valid. All honest replicas will be in agreement about which block is canonical, but someone who just receives that single block and is not aware of the timeout will think that that particular block was finalized. This breaks the very desirable property of being able to verify that a given block is part of the chain just from seeing the block, without being required to have the entire chain. The way we solve this is to require that block proposals after a timeout (where a commit QC might have been formed) re-propose the previous block. This guarantees that if we see a block with a valid commit QC, then that block is part of the chain (maybe it wasn’t finalized in that particular view, but it was certainly finalized). +- Always justify messages to remove time dependencies. That’s something we got from Fast-HotStuff. Messages should have enough information by themselves that any replica is capable of verifying their validity without any other information (with the exception of having previous blocks, but that’s external to the consensus algorithm anyway). If we don’t, then we introduce subtle timing dependencies. For example, Tendermint had a bug that was only discovered years later, where the solution was that the leader had to wait for the maximum network delay at the end of every round. If that wait doesn’t happen, a lock can occur. Funnily enough, Hotstuff-2 reintroduces this timing dependency in order to get rid of one round-trip, which significantly worsens the difficulty of modelling and implementing such a system. +- Make garbage collection and reconfiguration part of the algorithm. These are parts of the algorithm that will certainly be implemented. If we don’t specify and model them before, we will be left with awkwardly implementing them later on. + +FaB Paxos satisfies the first 4 points and Fast-HotStuff satisfies the 5th. ChonkyBFT is basically FaB Paxos with some ideas from Fast-HotStuff/HotStuff-2. \ No newline at end of file diff --git a/spec/informal-spec/README.md b/spec/informal-spec/README.md index b095cb38..94681c82 100644 --- a/spec/informal-spec/README.md +++ b/spec/informal-spec/README.md @@ -1,11 +1,13 @@ -# ChonkyBFT Specification +# ChonkyBFT Informal Specification -This is a ChonkyBFT specification in pseudocode. +This is the ChonkyBFT specification in pseudocode. + +We’ll assume there’s a static set of nodes. Each node has 3 components: replica, proposer and fetcher. They are modeled as concurrent tasks or actors. Proposer and fetcher can read the replica state, but can’t write to it. There's a couple of considerations that are not described in the pseudo-code: - **Network model**. Messages might be delivered out of order, but we don’t guarantee eventual delivery for *all* messages. Actually, our network only guarantees eventual delivery of the most recent message for each type. That’s because each replica only stores the last outgoing message of each type in memory, and always tries to deliver those messages whenever it reconnects with another replica. - **Garbage collection**. We can’t store all messages, the goal here is to bound the number of messages that each replica stores, in order to avoid DoS attacks. We handle messages like this: - `NewView` messages are never stored, so no garbage collection is necessary. - - We keep all `Proposal` messages until the proposal (or a proposal with the same block number) is finalized (which means any honest replica having both the `Proposal` and the corresponding `CommitQC`, we assume that any honest replica in that situation will immediately broadcast the block on the gossip network). + - We keep all `Proposal` messages until the proposal (or a proposal with the same block number) is finalized (which means any honest replica having both the `Proposal` and the corresponding `CommitQC`, we assume that any honest replica in that situation will immediately broadcast the block on the p2p network. - We only store the newest `CommitVote` **and** `TimeoutVote` for each replica. Honest replicas only change views on QCs, so if they send a newer message, they must also have sent a `NewView` on the transition, which means we can just get the QC from that replica. Even if the other replicas don’t receive the QC, it will just trigger a reproposal. \ No newline at end of file diff --git a/spec/informal-spec/replica.rs b/spec/informal-spec/replica.rs index 5c6e692b..a17b4dc2 100644 --- a/spec/informal-spec/replica.rs +++ b/spec/informal-spec/replica.rs @@ -1,5 +1,6 @@ -// Replica +//! Replica +// This is the state machine that moves the consensus forward. struct ReplicaState { // The view this replica is currently in. view: ViewNumber, @@ -65,15 +66,15 @@ impl ReplicaState { self.high_vote, self.high_commit_qc); - // Update our state so that we can no longer vote commit in this view. - self.phase = Phase::Timeout; + // Update our state so that we can no longer vote commit in this view. + self.phase = Phase::Timeout; - // Send the vote to all replicas (including ourselves). - self.send(vote); + // Send the vote to all replicas (including ourselves). + self.send(vote); } - // Try to get a message from the message queue and process it. We don't - // detail the message queue structure since it's boilerplate. + // Try to get a message from the message queue and process it. We don't + // detail the message queue structure since it's boilerplate. if let Some(message) = message_queue.pop() { match message { Proposal(msg) => { @@ -114,26 +115,25 @@ impl ReplicaState { // As a side result, get the correct block hash. let block_hash = match opt_block_hash { Some(hash) => { - // This is a reproposal. We let the leader repropose blocks without sending - // them in the proposal (it sends only the number + hash). That allows a - // leader to repropose a block without having it stored. - // It is an optimization that allows us to not wait for a leader that has - // the previous proposal stored (which can take 4f views), and to somewhat + // This is a reproposal. + // We let the leader repropose blocks without sending them in the proposal + // (it sends only the block number + block hash). That allows a leader to + // repropose a block without having it stored. Sending reproposals without + // a payload is an optimization that allows us to not wait for a leader that + // has the previous proposal stored (which can take 4f views), and to somewhat // speed up reproposals by skipping block broadcast. // This only saves time because we have a gossip network running in parallel, // and any time a replica is able to create a finalized block (by possessing // both the block and the commit QC) it broadcasts the finalized block (this // was meant to propagate the block to full nodes, but of course validators // will end up receiving it as well). - // However, this can be difficult to model and we might want to just - // ignore the gossip network in the formal model. We will still have liveness - // but in the model we'll end up waiting 4f views to get a leader that has the - // previous block before proposing a new one. This is not that bad, since - // then we can be sure that the consensus will continue even if the gossip - // network is failing for some reason. - // For sanity reasons, we'll check that there's no block in the proposal. - // But this check is completely unnecessary (in theory at least). + // We check that the leader didn't send a payload with the reproposal. + // This isn't technically needed for the consensus to work (it will remain + // safe and live), but it's a good practice to avoid unnecessary data in + // blockchain. + // This unnecessary payload would also effectively be a source of free + // data availability, which the leaders would be incentivized to abuse. assert!(proposal.block.is_none()); hash @@ -169,7 +169,7 @@ impl ReplicaState { self.send(vote); } - // Processed an (already verified) commit_qc received from the network + // Processes a (already verified) commit_qc received from the network // as part of some message. It bumps the local high_commit_qc and if // we have the proposal corresponding to this qc, we append it to the committed_blocks. fn process_commit_qc(&mut self, qc_opt: Option) { @@ -223,7 +223,7 @@ impl ReplicaState { // If the message isn't current, just ignore it. assert!(new_view.view() >= self.view) - // Check that the new view is valid. + // Check that the new view message is valid. assert!(new_view.verify()); // Update our state. diff --git a/spec/informal-spec/types.rs b/spec/informal-spec/types.rs index a2a517c6..65989df8 100644 --- a/spec/informal-spec/types.rs +++ b/spec/informal-spec/types.rs @@ -183,7 +183,7 @@ impl SignedTimeoutVote { } fn verify(self) -> bool { - // If we wish, there are three invariants that are easy to check but don't need to be stricly enforced for correctness: + // If we wish, there are three invariants that are easy to check but don't need to be strictly enforced for correctness: // 1. self.view() >= self.high_vote.view() // 2. self.high_vote.view() >= self.high_commit_qc_view // 3. self.view() > self.high_commit_qc_view diff --git a/spec/protocol-spec/README.md b/spec/protocol-spec/README.md index a051d819..2ee8de05 100644 --- a/spec/protocol-spec/README.md +++ b/spec/protocol-spec/README.md @@ -1,4 +1,4 @@ -# ChonkyBFT +# ChonkyBFT Formal Specification This page summarizes the scope of the Quint specification and the experiments we have done with it. This Quint specification was prepared by Igor Konnov and