From a1636b89ef8c2b95815217090a55a729cd3cdab0 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Fri, 14 Jun 2024 18:02:14 +0100 Subject: [PATCH 1/5] BFT-466: LeaderPrepare wait for previous block deadlock (#128) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - [x] Create a test to artificially create a situation in which some nodes have the proposal payload but not the CommitQC that would finalize it and allow it to be gossiped, while others have the HighQC received as part of a new proposal, but they don't have the previous payload and wait for it to be gossiped to them, creating a deadlock and stalling consensus. - [x] The solution is to timeout waiting for the block to appear in the store and move on to the new view, broadcast the HighQC as part of the ReplicaPrepare, which should allow the others to finalise the block. ## Why ❔ To deal with the loss of liveness caused by multiple leaders who owned both the payload and the CommitQC going down, and further replicas stop participating in consensus until they have the previous payload which never comes. --- node/actors/bft/src/leader/state_machine.rs | 7 +- node/actors/bft/src/replica/leader_prepare.rs | 24 ++- node/actors/bft/src/replica/state_machine.rs | 9 +- node/actors/bft/src/testonly/run.rs | 127 ++++++------ node/actors/bft/src/tests.rs | 183 +++++++++++++++++- 5 files changed, 275 insertions(+), 75 deletions(-) diff --git a/node/actors/bft/src/leader/state_machine.rs b/node/actors/bft/src/leader/state_machine.rs index 0be48689..5d9be667 100644 --- a/node/actors/bft/src/leader/state_machine.rs +++ b/node/actors/bft/src/leader/state_machine.rs @@ -94,11 +94,16 @@ impl StateMachine { 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 => { - tracing::info!("process_replica_prepare: {err:#}"); + // It's broadcasted now, so everyone gets it. + tracing::debug!("process_replica_prepare: {err:#}"); } _ => { tracing::warn!("process_replica_prepare: {err:#}"); diff --git a/node/actors/bft/src/replica/leader_prepare.rs b/node/actors/bft/src/replica/leader_prepare.rs index 14d86c7f..55e43cf0 100644 --- a/node/actors/bft/src/replica/leader_prepare.rs +++ b/node/actors/bft/src/replica/leader_prepare.rs @@ -2,7 +2,7 @@ use super::StateMachine; use zksync_concurrency::{ctx, error::Wrap}; use zksync_consensus_network::io::{ConsensusInputMessage, Target}; -use zksync_consensus_roles::validator; +use zksync_consensus_roles::validator::{self, BlockNumber}; /// Errors that can occur when processing a "leader prepare" message. #[derive(Debug, thiserror::Error)] @@ -45,6 +45,12 @@ pub(crate) enum Error { /// 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), @@ -107,8 +113,15 @@ impl StateMachine { 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 { @@ -121,9 +134,9 @@ impl StateMachine { // Defensively assume that PayloadManager cannot verify proposal until the previous block is stored. self.config .block_store - .wait_until_persisted(ctx, prev) + .wait_until_persisted(&ctx.with_deadline(self.timeout_deadline), prev) .await - .map_err(ctx::Error::Canceled)?; + .map_err(|_| Error::MissingPreviousPayload { prev_number: prev })?; } if let Err(err) = self .config @@ -150,11 +163,6 @@ impl StateMachine { self.view = message.view().number; self.phase = validator::Phase::Commit; self.high_vote = Some(commit_vote.clone()); - if let Some(high_qc) = high_qc { - // Try to create a finalized block with this CommitQC and our block proposal cache. - // This gives us another chance to finalize a block that we may have missed before. - self.save_block(ctx, high_qc).await.wrap("save_block()")?; - } // If we received a new block proposal, store it in our cache. if let Some(payload) = &message.proposal_payload { self.block_proposal_cache diff --git a/node/actors/bft/src/replica/state_machine.rs b/node/actors/bft/src/replica/state_machine.rs index b00aedd0..238cc0e3 100644 --- a/node/actors/bft/src/replica/state_machine.rs +++ b/node/actors/bft/src/replica/state_machine.rs @@ -116,10 +116,13 @@ impl StateMachine { 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::info!("process_replica_prepare: {err:#}"); + tracing::debug!("process_replica_prepare: {err:#}"); } _ => { tracing::warn!("process_replica_prepare: {err:#}"); @@ -140,6 +143,9 @@ impl StateMachine { 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 { .. } => { @@ -164,6 +170,7 @@ impl StateMachine { 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 { .. } => { diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 45afd0b9..20ddb72e 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -19,11 +19,10 @@ use zksync_consensus_roles::validator; use zksync_consensus_storage::{testonly::new_store, BlockStore}; use zksync_consensus_utils::pipe; -#[derive(Clone)] pub(crate) enum Network { Real, Mock, - Twins(PortSplitSchedule), + Twins(PortRouter), } /// Number of phases within a view for which we consider different partitions. @@ -46,9 +45,46 @@ pub(crate) type PortPartition = HashSet; pub(crate) type PortSplit = Vec; /// A schedule contains a list of splits (one for each phase) for every view. 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. +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. + Custom(Box), +} + +impl PortRouter { + /// Decide whether a message can be sent from a sender to a target in the given view/phase. + /// + /// Returning `None` means the there was no more routing data and the test can decide to + /// allow all communication or to abort a runaway test. + fn can_send(&self, msg: &validator::ConsensusMsg, from: Port, to: Port) -> Option { + match self { + PortRouter::Splits(splits) => { + // Here we assume that all instances start from view 0 in the tests. + // If the view is higher than what we have planned for, assume no partitions. + // Every node is guaranteed to be present in only one partition. + let view_number = msg.view().number.0 as usize; + let phase_number = msg_phase_number(msg); + splits + .get(view_number) + .and_then(|ps| ps.get(phase_number)) + .map(|partitions| { + partitions + .iter() + .any(|p| p.contains(&from) && p.contains(&to)) + }) + } + PortRouter::Custom(f) => f(msg, from, to), + } + } +} /// Config for the test. Determines the parameters to run the test with. -#[derive(Clone)] pub(crate) struct Test { pub(crate) network: Network, pub(crate) nodes: Vec<(Behavior, u64)>, @@ -124,7 +160,7 @@ async fn run_nodes(ctx: &ctx::Ctx, network: &Network, specs: &[Node]) -> anyhow: match network { Network::Real => run_nodes_real(ctx, specs).await, Network::Mock => run_nodes_mock(ctx, specs).await, - Network::Twins(splits) => run_nodes_twins(ctx, specs, splits).await, + Network::Twins(router) => run_nodes_twins(ctx, specs, router).await, } } @@ -210,7 +246,7 @@ async fn run_nodes_mock(ctx: &ctx::Ctx, specs: &[Node]) -> anyhow::Result<()> { async fn run_nodes_twins( ctx: &ctx::Ctx, specs: &[Node], - splits: &PortSplitSchedule, + router: &PortRouter, ) -> anyhow::Result<()> { scope::run!(ctx, |ctx, s| async { // All known network ports of a validator, so that we can tell if any of @@ -282,7 +318,7 @@ async fn run_nodes_twins( s.spawn(async move { twins_receive_loop( ctx, - splits, + router, validator_ports, sends, TwinsGossipConfig { @@ -312,7 +348,7 @@ async fn run_nodes_twins( /// 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, - splits: &PortSplitSchedule, + router: &PortRouter, validator_ports: &HashMap>, sends: &HashMap>, gossip: TwinsGossipConfig<'_>, @@ -396,19 +432,8 @@ async fn twins_receive_loop( }; while let Ok(io::InputMessage::Consensus(message)) = recv.recv(ctx).await { - let view_number = message.message.msg.view().number.0 as usize; - let phase_number = input_msg_phase_number(&message); + let view = message.message.msg.view().number.0 as usize; let kind = message.message.msg.label(); - // Here we assume that all instances start from view 0 in the tests. - // If the view is higher than what we have planned for, assume no partitions. - // Every node is guaranteed to be present in only one partition. - let partitions_opt = splits.get(view_number).and_then(|ps| ps.get(phase_number)); - - if partitions_opt.is_none() { - bail!( - "ran out of scheduled rounds; most likely cannot finalize blocks even if we go on" - ); - } let msg = || { io::OutputMessage::Consensus(io::ConsensusReq { @@ -417,49 +442,27 @@ async fn twins_receive_loop( }) }; + let can_send = |to| { + match router.can_send(&message.message.msg, port, to) { + Some(can_send) => Ok(can_send), + None => bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") + } + }; + match message.recipient { - io::Target::Broadcast => match partitions_opt { - None => { - tracing::info!( - "broadcasting view={view_number} from={port} target=all kind={kind}" - ); - for target_port in sends.keys() { - send_or_stash(true, *target_port, msg()); - } - } - Some(ps) => { - for p in ps { - let can_send = p.contains(&port); - tracing::info!("broadcasting view={view_number} from={port} target={p:?} kind={kind} can_send={can_send}"); - for target_port in p { - send_or_stash(can_send, *target_port, msg()); - } - } + 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(v) => { - let target_ports = &validator_ports[&v]; - - match partitions_opt { - None => { - for target_port in target_ports { - tracing::info!( - "unicasting view={view_number} from={port} target={target_port} kind={kind}" - ); - send_or_stash(true, *target_port, msg()); - } - } - Some(ps) => { - for p in ps { - let can_send = p.contains(&port); - for target_port in target_ports { - if p.contains(target_port) { - tracing::info!("unicasting view={view_number} from={port} target={target_port } kind={kind} can_send={can_send}"); - send_or_stash(can_send, *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()); } } } @@ -554,9 +557,9 @@ fn output_msg_commit_qc(msg: &io::OutputMessage) -> Option<&validator::CommitQC> } /// Index of the phase in which the message appears, to decide which partitioning to apply. -fn input_msg_phase_number(msg: &io::ConsensusInputMessage) -> usize { +fn msg_phase_number(msg: &validator::ConsensusMsg) -> usize { use validator::ConsensusMsg; - let phase = match msg.message.msg { + let phase = match msg { ConsensusMsg::ReplicaPrepare(_) => 0, ConsensusMsg::LeaderPrepare(_) => 0, ConsensusMsg::ReplicaCommit(_) => 0, diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index 47920147..e014312e 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -3,14 +3,14 @@ use std::collections::HashMap; use crate::testonly::{ twins::{Cluster, HasKey, ScenarioGenerator, Twin}, ut_harness::UTHarness, - Behavior, Network, PortSplitSchedule, Test, NUM_PHASES, + Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, NUM_PHASES, }; 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, + LeaderSelectionMode, PublicKey, SecretKey, ViewNumber, }; async fn run_test(behavior: Behavior, network: Network) { @@ -420,7 +420,7 @@ async fn run_twins( } Test { - network: Network::Twins(splits), + network: Network::Twins(PortRouter::Splits(splits)), nodes: nodes.clone(), blocks_to_finalize, } @@ -430,3 +430,180 @@ async fn run_twins( 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(flavor = "multi_thread")] +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, +) -> anyhow::Result<()> { + zksync_concurrency::testonly::abort_on_panic(); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); + let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + + 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 = spec.into(); + + 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.genesis) + .await +} From 6577dcf2a7d4b3a57eb577873dd3220c9cd100ed Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Sat, 15 Jun 2024 22:57:46 +0200 Subject: [PATCH 2/5] made interleaving more deterministic --- node/Cargo.lock | 1 + node/actors/bft/Cargo.toml | 3 +- node/actors/bft/src/testonly/run.rs | 45 ++++++++++----- node/actors/bft/src/tests.rs | 89 +++++++++++++---------------- 4 files changed, 75 insertions(+), 63 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index 63c3e3e1..e2c6f1df 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3825,6 +3825,7 @@ dependencies = [ "once_cell", "pretty_assertions", "rand 0.8.5", + "test-casing", "thiserror", "tokio", "tracing", diff --git a/node/actors/bft/Cargo.toml b/node/actors/bft/Cargo.toml index 06634d72..99932bfa 100644 --- a/node/actors/bft/Cargo.toml +++ b/node/actors/bft/Cargo.toml @@ -24,9 +24,10 @@ tracing.workspace = true vise.workspace = true [dev-dependencies] -tokio.workspace = true +tokio = { workspace = true, features = ["full","test-util"]} assert_matches.workspace = true pretty_assertions.workspace = true +test-casing.workspace = true [lints] workspace = true diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 20ddb72e..0d17d696 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -1,5 +1,5 @@ use super::{Behavior, Node}; -use anyhow::bail; +use anyhow::Context as _; use network::{io, Config}; use rand::seq::SliceRandom; use std::{ @@ -91,9 +91,29 @@ pub(crate) struct Test { pub(crate) blocks_to_finalize: usize, } +#[derive(thiserror::Error, Debug)] +pub(crate) enum TestError { + #[error("finalized conflicting blocks")] + BlockConflict, + #[error(transparent)] + Other(#[from] ctx::Error), +} + +impl From for TestError { + fn from(err: anyhow::Error) -> Self { + Self::Other(err.into()) + } +} + +impl From for TestError { + fn from(err: ctx::Canceled) -> Self { + Self::Other(err.into()) + } +} + impl Test { /// Run a test with the given parameters and a random network setup. - pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> Result<(), TestError> { let rng = &mut ctx.rng(); let setup = validator::testonly::Setup::new_with_weights( rng, @@ -109,13 +129,13 @@ impl Test { ctx: &ctx::Ctx, nets: Vec, genesis: &validator::Genesis, - ) -> anyhow::Result<()> { + ) -> Result<(), TestError> { let mut nodes = vec![]; let mut honest = vec![]; scope::run!(ctx, |ctx, s| async { for (i, net) in nets.into_iter().enumerate() { let (store, runner) = new_store(ctx, genesis).await; - s.spawn_bg(runner.run(ctx)); + s.spawn_bg(async { Ok(runner.run(ctx).await?) }); if self.nodes[i].0 == Behavior::Honest { honest.push(store.clone()); } @@ -126,7 +146,7 @@ impl Test { }); } assert!(!honest.is_empty()); - s.spawn_bg(run_nodes(ctx, &self.network, &nodes)); + s.spawn_bg(async { Ok(run_nodes(ctx, &self.network, &nodes).await?) }); // Run the nodes until all honest nodes store enough finalized blocks. assert!(self.blocks_to_finalize > 0); @@ -140,16 +160,15 @@ impl Test { for i in 0..self.blocks_to_finalize as u64 { let i = first + i; // Only comparing the payload; the justification might be different. - let want = honest[0] - .block(ctx, i) - .await? - .expect("checked its existence") - .payload; + let want = honest[0].block(ctx, i).await?.context("missing block")?; for store in &honest[1..] { - assert_eq!(want, store.block(ctx, i).await?.unwrap().payload); + let got = store.block(ctx, i).await?.context("missing block")?; + if want.payload != got.payload { + return Err(TestError::BlockConflict); + } } } - Ok(()) + Ok::<_, TestError>(()) }) .await } @@ -445,7 +464,7 @@ async fn twins_receive_loop( let can_send = |to| { match router.can_send(&message.message.msg, port, to) { Some(can_send) => Ok(can_send), - None => bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") + None => anyhow::bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") } }; diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index e014312e..eda7f594 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -1,10 +1,11 @@ -use std::collections::HashMap; - use crate::testonly::{ twins::{Cluster, HasKey, ScenarioGenerator, Twin}, ut_harness::UTHarness, - Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, NUM_PHASES, + Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, TestError, NUM_PHASES, }; +use assert_matches::assert_matches; +use std::collections::HashMap; +use test_casing::test_casing; use zksync_concurrency::{ctx, scope, time}; use zksync_consensus_network::testonly::new_configs_for_validators; use zksync_consensus_roles::validator::{ @@ -14,6 +15,7 @@ use zksync_consensus_roles::validator::{ }; 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); @@ -36,42 +38,42 @@ async fn run_test(behavior: Behavior, network: Network) { .unwrap() } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn honest_mock_network() { run_test(Behavior::Honest, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn honest_real_network() { run_test(Behavior::Honest, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn offline_mock_network() { run_test(Behavior::Offline, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn offline_real_network() { run_test(Behavior::Offline, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn random_mock_network() { run_test(Behavior::Random, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn random_real_network() { run_test(Behavior::Random, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn byzantine_mock_network() { run_test(Behavior::Byzantine, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn byzantine_real_network() { run_test(Behavior::Byzantine, Network::Real).await } @@ -213,8 +215,9 @@ async fn non_proposing_leader() { /// /// This should be a simple sanity check that the network works and consensus /// is achieved under the most favourable conditions. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_wo_twins_wo_partitions() { + tokio::time::pause(); // n<6 implies f=0 and q=n run_twins(5, 0, 10).await.unwrap(); } @@ -225,37 +228,39 @@ async fn twins_network_wo_twins_wo_partitions() { /// /// This should be a sanity check that without Byzantine behaviour the consensus /// is resilient to temporary network partitions. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_wo_twins_w_partitions() { + tokio::time::pause(); // n=6 implies f=1 and q=5; 6 is the minimum where partitions are possible. run_twins(6, 0, 5).await.unwrap(); } /// Run Twins scenarios with random number of nodes and 1 twin. -#[tokio::test(flavor = "multi_thread")] -async fn twins_network_w1_twins_w_partitions() { +#[test_casing(5, 6..=10)] +#[tokio::test] +async fn twins_network_w1_twins_w_partitions(num_replicas: usize) { + tokio::time::pause(); // n>=6 implies f>=1 and q=n-f - for num_replicas in 6..=10 { - // 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, 10).await.unwrap(); - } + // 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, 10).await.unwrap(); } /// Run Twins scenarios with higher number of nodes and 2 twins. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_w2_twins_w_partitions() { + tokio::time::pause(); // n>=11 implies f>=2 and q=n-f - run_twins(11, 2, 10).await.unwrap(); + run_twins(11, 2, 8).await.unwrap(); } /// Run Twins scenario with more twins than tolerable and expect it to fail. -#[tokio::test(flavor = "multi_thread")] -#[should_panic] +#[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. - run_twins(5, 1, 100).await.unwrap(); + assert_matches!(run_twins(5, 1, 100).await, Err(TestError::BlockConflict)); } /// Create network configuration for a given number of replicas and twins and run [Test]. @@ -263,29 +268,14 @@ async fn run_twins( num_replicas: usize, num_twins: usize, num_scenarios: usize, -) -> anyhow::Result<()> { - let num_honest = validator::threshold(num_replicas as u64) as usize; - let max_faulty = num_replicas - num_honest; - - // If we pass more twins than tolerable faulty replicas then it should fail with an assertion error, - // but if we abort the process on panic then the #[should_panic] attribute doesn't work with `cargo nextest`. - if num_twins <= max_faulty { - zksync_concurrency::testonly::abort_on_panic(); - } - zksync_concurrency::testonly::init_tracing(); +) -> Result<(), TestError> { + zksync_concurrency::testonly::abort_on_panic(); // Use a single timeout for all scenarios to finish. // A single scenario with 11 replicas took 3-5 seconds. // Panic on timeout; works with `cargo nextest` and the `abort_on_panic` above. - // If we are in the mode where we are looking for faults and `abort_on_panic` is disabled, - // then this will not have any effect and the simulation will run for as long as it takes to - // go through all the configured scenarios, and then fail because it didn't panic if no fault was found. - // If it panics for another reason it might be misleading though, so ideally it should finish early. - // It would be nicer to actually inspect the panic and make sure it's the right kind of assertion. - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); - // Using `ctc.with_timeout` would stop a runaway execution even without `abort_on_panic` but - // it would make the test pass for a different reason, not because it found an error but because it ran long. - let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); + let ctx = &ctx::test_root(&ctx::RealClock); #[derive(PartialEq, Debug)] struct Replica { @@ -435,7 +425,7 @@ async fn run_twins( /// 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(flavor = "multi_thread")] +#[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. @@ -551,10 +541,11 @@ async fn run_with_custom_router( gossip_peers: usize, blocks_to_finalize: usize, make_router: impl FnOnce(HashMap) -> PortRouter, -) -> anyhow::Result<()> { +) -> Result<(), TestError> { + tokio::time::pause(); zksync_concurrency::testonly::abort_on_panic(); - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); - let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); + let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); From 23e4e71374bf4394de30ba24efd37cbf7b7bb286 Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Sat, 15 Jun 2024 23:06:09 +0200 Subject: [PATCH 3/5] reverting commit pushed by accident --- node/Cargo.lock | 1 - node/actors/bft/Cargo.toml | 3 +- node/actors/bft/src/testonly/run.rs | 45 +++++---------- node/actors/bft/src/tests.rs | 89 ++++++++++++++++------------- 4 files changed, 63 insertions(+), 75 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index e2c6f1df..63c3e3e1 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3825,7 +3825,6 @@ dependencies = [ "once_cell", "pretty_assertions", "rand 0.8.5", - "test-casing", "thiserror", "tokio", "tracing", diff --git a/node/actors/bft/Cargo.toml b/node/actors/bft/Cargo.toml index 99932bfa..06634d72 100644 --- a/node/actors/bft/Cargo.toml +++ b/node/actors/bft/Cargo.toml @@ -24,10 +24,9 @@ tracing.workspace = true vise.workspace = true [dev-dependencies] -tokio = { workspace = true, features = ["full","test-util"]} +tokio.workspace = true assert_matches.workspace = true pretty_assertions.workspace = true -test-casing.workspace = true [lints] workspace = true diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 0d17d696..20ddb72e 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -1,5 +1,5 @@ use super::{Behavior, Node}; -use anyhow::Context as _; +use anyhow::bail; use network::{io, Config}; use rand::seq::SliceRandom; use std::{ @@ -91,29 +91,9 @@ pub(crate) struct Test { pub(crate) blocks_to_finalize: usize, } -#[derive(thiserror::Error, Debug)] -pub(crate) enum TestError { - #[error("finalized conflicting blocks")] - BlockConflict, - #[error(transparent)] - Other(#[from] ctx::Error), -} - -impl From for TestError { - fn from(err: anyhow::Error) -> Self { - Self::Other(err.into()) - } -} - -impl From for TestError { - fn from(err: ctx::Canceled) -> Self { - Self::Other(err.into()) - } -} - impl Test { /// Run a test with the given parameters and a random network setup. - pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> Result<(), TestError> { + pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> anyhow::Result<()> { let rng = &mut ctx.rng(); let setup = validator::testonly::Setup::new_with_weights( rng, @@ -129,13 +109,13 @@ impl Test { ctx: &ctx::Ctx, nets: Vec, genesis: &validator::Genesis, - ) -> Result<(), TestError> { + ) -> anyhow::Result<()> { let mut nodes = vec![]; let mut honest = vec![]; scope::run!(ctx, |ctx, s| async { for (i, net) in nets.into_iter().enumerate() { let (store, runner) = new_store(ctx, genesis).await; - s.spawn_bg(async { Ok(runner.run(ctx).await?) }); + s.spawn_bg(runner.run(ctx)); if self.nodes[i].0 == Behavior::Honest { honest.push(store.clone()); } @@ -146,7 +126,7 @@ impl Test { }); } assert!(!honest.is_empty()); - s.spawn_bg(async { Ok(run_nodes(ctx, &self.network, &nodes).await?) }); + s.spawn_bg(run_nodes(ctx, &self.network, &nodes)); // Run the nodes until all honest nodes store enough finalized blocks. assert!(self.blocks_to_finalize > 0); @@ -160,15 +140,16 @@ impl Test { for i in 0..self.blocks_to_finalize as u64 { let i = first + i; // Only comparing the payload; the justification might be different. - let want = honest[0].block(ctx, i).await?.context("missing block")?; + let want = honest[0] + .block(ctx, i) + .await? + .expect("checked its existence") + .payload; for store in &honest[1..] { - let got = store.block(ctx, i).await?.context("missing block")?; - if want.payload != got.payload { - return Err(TestError::BlockConflict); - } + assert_eq!(want, store.block(ctx, i).await?.unwrap().payload); } } - Ok::<_, TestError>(()) + Ok(()) }) .await } @@ -464,7 +445,7 @@ async fn twins_receive_loop( let can_send = |to| { match router.can_send(&message.message.msg, port, to) { Some(can_send) => Ok(can_send), - None => anyhow::bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") + None => bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") } }; diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index eda7f594..e014312e 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -1,11 +1,10 @@ +use std::collections::HashMap; + use crate::testonly::{ twins::{Cluster, HasKey, ScenarioGenerator, Twin}, ut_harness::UTHarness, - Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, TestError, NUM_PHASES, + Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, NUM_PHASES, }; -use assert_matches::assert_matches; -use std::collections::HashMap; -use test_casing::test_casing; use zksync_concurrency::{ctx, scope, time}; use zksync_consensus_network::testonly::new_configs_for_validators; use zksync_consensus_roles::validator::{ @@ -15,7 +14,6 @@ use zksync_consensus_roles::validator::{ }; 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); @@ -38,42 +36,42 @@ async fn run_test(behavior: Behavior, network: Network) { .unwrap() } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn honest_mock_network() { run_test(Behavior::Honest, Network::Mock).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn honest_real_network() { run_test(Behavior::Honest, Network::Real).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn offline_mock_network() { run_test(Behavior::Offline, Network::Mock).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn offline_real_network() { run_test(Behavior::Offline, Network::Real).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn random_mock_network() { run_test(Behavior::Random, Network::Mock).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn random_real_network() { run_test(Behavior::Random, Network::Real).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn byzantine_mock_network() { run_test(Behavior::Byzantine, Network::Mock).await } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn byzantine_real_network() { run_test(Behavior::Byzantine, Network::Real).await } @@ -215,9 +213,8 @@ async fn non_proposing_leader() { /// /// This should be a simple sanity check that the network works and consensus /// is achieved under the most favourable conditions. -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn twins_network_wo_twins_wo_partitions() { - tokio::time::pause(); // n<6 implies f=0 and q=n run_twins(5, 0, 10).await.unwrap(); } @@ -228,39 +225,37 @@ async fn twins_network_wo_twins_wo_partitions() { /// /// This should be a sanity check that without Byzantine behaviour the consensus /// is resilient to temporary network partitions. -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn twins_network_wo_twins_w_partitions() { - tokio::time::pause(); // n=6 implies f=1 and q=5; 6 is the minimum where partitions are possible. run_twins(6, 0, 5).await.unwrap(); } /// Run Twins scenarios with random number of nodes and 1 twin. -#[test_casing(5, 6..=10)] -#[tokio::test] -async fn twins_network_w1_twins_w_partitions(num_replicas: usize) { - tokio::time::pause(); +#[tokio::test(flavor = "multi_thread")] +async fn twins_network_w1_twins_w_partitions() { // 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, 10).await.unwrap(); + for num_replicas in 6..=10 { + // 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, 10).await.unwrap(); + } } /// Run Twins scenarios with higher number of nodes and 2 twins. -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn twins_network_w2_twins_w_partitions() { - tokio::time::pause(); // n>=11 implies f>=2 and q=n-f - run_twins(11, 2, 8).await.unwrap(); + run_twins(11, 2, 10).await.unwrap(); } /// Run Twins scenario with more twins than tolerable and expect it to fail. -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] +#[should_panic] 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, 100).await, Err(TestError::BlockConflict)); + run_twins(5, 1, 100).await.unwrap(); } /// Create network configuration for a given number of replicas and twins and run [Test]. @@ -268,14 +263,29 @@ async fn run_twins( num_replicas: usize, num_twins: usize, num_scenarios: usize, -) -> Result<(), TestError> { - zksync_concurrency::testonly::abort_on_panic(); +) -> anyhow::Result<()> { + let num_honest = validator::threshold(num_replicas as u64) as usize; + let max_faulty = num_replicas - num_honest; + + // If we pass more twins than tolerable faulty replicas then it should fail with an assertion error, + // but if we abort the process on panic then the #[should_panic] attribute doesn't work with `cargo nextest`. + if num_twins <= max_faulty { + zksync_concurrency::testonly::abort_on_panic(); + } + zksync_concurrency::testonly::init_tracing(); // Use a single timeout for all scenarios to finish. // 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); + // If we are in the mode where we are looking for faults and `abort_on_panic` is disabled, + // then this will not have any effect and the simulation will run for as long as it takes to + // go through all the configured scenarios, and then fail because it didn't panic if no fault was found. + // If it panics for another reason it might be misleading though, so ideally it should finish early. + // It would be nicer to actually inspect the panic and make sure it's the right kind of assertion. + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); + // Using `ctc.with_timeout` would stop a runaway execution even without `abort_on_panic` but + // it would make the test pass for a different reason, not because it found an error but because it ran long. + let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); #[derive(PartialEq, Debug)] struct Replica { @@ -425,7 +435,7 @@ async fn run_twins( /// 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] +#[tokio::test(flavor = "multi_thread")] 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. @@ -541,11 +551,10 @@ async fn run_with_custom_router( gossip_peers: usize, blocks_to_finalize: usize, make_router: impl FnOnce(HashMap) -> PortRouter, -) -> Result<(), TestError> { - tokio::time::pause(); +) -> anyhow::Result<()> { 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 _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); + let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); let rng = &mut ctx.rng(); From 03409a53b806ee281254c5173844a73a15c0c03c Mon Sep 17 00:00:00 2001 From: pompon0 Date: Sat, 15 Jun 2024 23:39:56 +0200 Subject: [PATCH 4/5] made tests more deterministic (#129) Some of the tests were flaky because of cpu contention affecting interleavings. This fix makes the passing time (almost) deterministic, by the use of `tokio::time::pause()`. The other consequence is that runtime used in multiple tests has been changed from multi-threaded to single-threaded. This also helps the determinism because the nextest parallelism is bounded by the number of cores - if we make all tests single-threaded, behavior of the test will be the same when executed separately and in a test suite. Also converted a should_panic test to regular test for consistency. Tuned size of some of the tests to reasonable execution times. --- node/Cargo.lock | 1 + node/actors/bft/Cargo.toml | 3 +- node/actors/bft/src/testonly/run.rs | 45 ++++++++++----- node/actors/bft/src/tests.rs | 89 +++++++++++++---------------- 4 files changed, 75 insertions(+), 63 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index 63c3e3e1..e2c6f1df 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3825,6 +3825,7 @@ dependencies = [ "once_cell", "pretty_assertions", "rand 0.8.5", + "test-casing", "thiserror", "tokio", "tracing", diff --git a/node/actors/bft/Cargo.toml b/node/actors/bft/Cargo.toml index 06634d72..99932bfa 100644 --- a/node/actors/bft/Cargo.toml +++ b/node/actors/bft/Cargo.toml @@ -24,9 +24,10 @@ tracing.workspace = true vise.workspace = true [dev-dependencies] -tokio.workspace = true +tokio = { workspace = true, features = ["full","test-util"]} assert_matches.workspace = true pretty_assertions.workspace = true +test-casing.workspace = true [lints] workspace = true diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 20ddb72e..0d17d696 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -1,5 +1,5 @@ use super::{Behavior, Node}; -use anyhow::bail; +use anyhow::Context as _; use network::{io, Config}; use rand::seq::SliceRandom; use std::{ @@ -91,9 +91,29 @@ pub(crate) struct Test { pub(crate) blocks_to_finalize: usize, } +#[derive(thiserror::Error, Debug)] +pub(crate) enum TestError { + #[error("finalized conflicting blocks")] + BlockConflict, + #[error(transparent)] + Other(#[from] ctx::Error), +} + +impl From for TestError { + fn from(err: anyhow::Error) -> Self { + Self::Other(err.into()) + } +} + +impl From for TestError { + fn from(err: ctx::Canceled) -> Self { + Self::Other(err.into()) + } +} + impl Test { /// Run a test with the given parameters and a random network setup. - pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> Result<(), TestError> { let rng = &mut ctx.rng(); let setup = validator::testonly::Setup::new_with_weights( rng, @@ -109,13 +129,13 @@ impl Test { ctx: &ctx::Ctx, nets: Vec, genesis: &validator::Genesis, - ) -> anyhow::Result<()> { + ) -> Result<(), TestError> { let mut nodes = vec![]; let mut honest = vec![]; scope::run!(ctx, |ctx, s| async { for (i, net) in nets.into_iter().enumerate() { let (store, runner) = new_store(ctx, genesis).await; - s.spawn_bg(runner.run(ctx)); + s.spawn_bg(async { Ok(runner.run(ctx).await?) }); if self.nodes[i].0 == Behavior::Honest { honest.push(store.clone()); } @@ -126,7 +146,7 @@ impl Test { }); } assert!(!honest.is_empty()); - s.spawn_bg(run_nodes(ctx, &self.network, &nodes)); + s.spawn_bg(async { Ok(run_nodes(ctx, &self.network, &nodes).await?) }); // Run the nodes until all honest nodes store enough finalized blocks. assert!(self.blocks_to_finalize > 0); @@ -140,16 +160,15 @@ impl Test { for i in 0..self.blocks_to_finalize as u64 { let i = first + i; // Only comparing the payload; the justification might be different. - let want = honest[0] - .block(ctx, i) - .await? - .expect("checked its existence") - .payload; + let want = honest[0].block(ctx, i).await?.context("missing block")?; for store in &honest[1..] { - assert_eq!(want, store.block(ctx, i).await?.unwrap().payload); + let got = store.block(ctx, i).await?.context("missing block")?; + if want.payload != got.payload { + return Err(TestError::BlockConflict); + } } } - Ok(()) + Ok::<_, TestError>(()) }) .await } @@ -445,7 +464,7 @@ async fn twins_receive_loop( let can_send = |to| { match router.can_send(&message.message.msg, port, to) { Some(can_send) => Ok(can_send), - None => bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") + None => anyhow::bail!("ran out of port schedule; we probably wouldn't finalize blocks even if we continued") } }; diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index e014312e..eda7f594 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -1,10 +1,11 @@ -use std::collections::HashMap; - use crate::testonly::{ twins::{Cluster, HasKey, ScenarioGenerator, Twin}, ut_harness::UTHarness, - Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, NUM_PHASES, + Behavior, Network, Port, PortRouter, PortSplitSchedule, Test, TestError, NUM_PHASES, }; +use assert_matches::assert_matches; +use std::collections::HashMap; +use test_casing::test_casing; use zksync_concurrency::{ctx, scope, time}; use zksync_consensus_network::testonly::new_configs_for_validators; use zksync_consensus_roles::validator::{ @@ -14,6 +15,7 @@ use zksync_consensus_roles::validator::{ }; 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); @@ -36,42 +38,42 @@ async fn run_test(behavior: Behavior, network: Network) { .unwrap() } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn honest_mock_network() { run_test(Behavior::Honest, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn honest_real_network() { run_test(Behavior::Honest, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn offline_mock_network() { run_test(Behavior::Offline, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn offline_real_network() { run_test(Behavior::Offline, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn random_mock_network() { run_test(Behavior::Random, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn random_real_network() { run_test(Behavior::Random, Network::Real).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn byzantine_mock_network() { run_test(Behavior::Byzantine, Network::Mock).await } -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn byzantine_real_network() { run_test(Behavior::Byzantine, Network::Real).await } @@ -213,8 +215,9 @@ async fn non_proposing_leader() { /// /// This should be a simple sanity check that the network works and consensus /// is achieved under the most favourable conditions. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_wo_twins_wo_partitions() { + tokio::time::pause(); // n<6 implies f=0 and q=n run_twins(5, 0, 10).await.unwrap(); } @@ -225,37 +228,39 @@ async fn twins_network_wo_twins_wo_partitions() { /// /// This should be a sanity check that without Byzantine behaviour the consensus /// is resilient to temporary network partitions. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_wo_twins_w_partitions() { + tokio::time::pause(); // n=6 implies f=1 and q=5; 6 is the minimum where partitions are possible. run_twins(6, 0, 5).await.unwrap(); } /// Run Twins scenarios with random number of nodes and 1 twin. -#[tokio::test(flavor = "multi_thread")] -async fn twins_network_w1_twins_w_partitions() { +#[test_casing(5, 6..=10)] +#[tokio::test] +async fn twins_network_w1_twins_w_partitions(num_replicas: usize) { + tokio::time::pause(); // n>=6 implies f>=1 and q=n-f - for num_replicas in 6..=10 { - // 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, 10).await.unwrap(); - } + // 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, 10).await.unwrap(); } /// Run Twins scenarios with higher number of nodes and 2 twins. -#[tokio::test(flavor = "multi_thread")] +#[tokio::test] async fn twins_network_w2_twins_w_partitions() { + tokio::time::pause(); // n>=11 implies f>=2 and q=n-f - run_twins(11, 2, 10).await.unwrap(); + run_twins(11, 2, 8).await.unwrap(); } /// Run Twins scenario with more twins than tolerable and expect it to fail. -#[tokio::test(flavor = "multi_thread")] -#[should_panic] +#[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. - run_twins(5, 1, 100).await.unwrap(); + assert_matches!(run_twins(5, 1, 100).await, Err(TestError::BlockConflict)); } /// Create network configuration for a given number of replicas and twins and run [Test]. @@ -263,29 +268,14 @@ async fn run_twins( num_replicas: usize, num_twins: usize, num_scenarios: usize, -) -> anyhow::Result<()> { - let num_honest = validator::threshold(num_replicas as u64) as usize; - let max_faulty = num_replicas - num_honest; - - // If we pass more twins than tolerable faulty replicas then it should fail with an assertion error, - // but if we abort the process on panic then the #[should_panic] attribute doesn't work with `cargo nextest`. - if num_twins <= max_faulty { - zksync_concurrency::testonly::abort_on_panic(); - } - zksync_concurrency::testonly::init_tracing(); +) -> Result<(), TestError> { + zksync_concurrency::testonly::abort_on_panic(); // Use a single timeout for all scenarios to finish. // A single scenario with 11 replicas took 3-5 seconds. // Panic on timeout; works with `cargo nextest` and the `abort_on_panic` above. - // If we are in the mode where we are looking for faults and `abort_on_panic` is disabled, - // then this will not have any effect and the simulation will run for as long as it takes to - // go through all the configured scenarios, and then fail because it didn't panic if no fault was found. - // If it panics for another reason it might be misleading though, so ideally it should finish early. - // It would be nicer to actually inspect the panic and make sure it's the right kind of assertion. - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); - // Using `ctc.with_timeout` would stop a runaway execution even without `abort_on_panic` but - // it would make the test pass for a different reason, not because it found an error but because it ran long. - let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); + let ctx = &ctx::test_root(&ctx::RealClock); #[derive(PartialEq, Debug)] struct Replica { @@ -435,7 +425,7 @@ async fn run_twins( /// 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(flavor = "multi_thread")] +#[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. @@ -551,10 +541,11 @@ async fn run_with_custom_router( gossip_peers: usize, blocks_to_finalize: usize, make_router: impl FnOnce(HashMap) -> PortRouter, -) -> anyhow::Result<()> { +) -> Result<(), TestError> { + tokio::time::pause(); zksync_concurrency::testonly::abort_on_panic(); - let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(30)); - let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(60)); + let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); From 23118bd0acbfd9718c794d82ebfb2716e8d95229 Mon Sep 17 00:00:00 2001 From: pompon0 Date: Wed, 19 Jun 2024 10:38:43 +0200 Subject: [PATCH 5/5] bumped deps (#130) https://rustsec.org/advisories/RUSTSEC-2024-0344 --- node/Cargo.lock | 241 +++++++++++++++++++++++------------------------- 1 file changed, 117 insertions(+), 124 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index e2c6f1df..0bb6f64a 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.21.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" dependencies = [ "gimli", ] @@ -133,9 +133,9 @@ dependencies = [ [[package]] name = "anstyle-query" -version = "1.0.3" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a64c907d4e79225ac72e2a354c9ce84d50ebb4586dee56c82b3ee73004f537f5" +checksum = "ad186efb764318d35165f1758e7dcef3b10628e26d41a44bc5550652e6804391" dependencies = [ "windows-sys 0.52.0", ] @@ -170,7 +170,7 @@ checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -192,9 +192,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.71" +version = "0.3.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b05800d2e817c8b3b4b54abd461726265fa9789ae34330622f2db9ee696f9d" +checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a" dependencies = [ "addr2line", "cc", @@ -262,7 +262,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -321,9 +321,9 @@ dependencies = [ [[package]] name = "blst" -version = "0.3.11" +version = "0.3.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c94087b935a822949d3291a9989ad2b2051ea141eda0fd4e478a75f6aa3e604b" +checksum = "62dc83a094a71d43eeadd254b1ec2d24cb6a0bb6cadce00df51f0db594711a32" dependencies = [ "cc", "glob", @@ -368,9 +368,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.98" +version = "1.0.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41c270e7540d725e65ac7f1b212ac8ce349719624d7bcff99f8e2e488e8cf03f" +checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" dependencies = [ "jobserver", "libc", @@ -469,9 +469,9 @@ dependencies = [ [[package]] name = "clang-sys" -version = "1.7.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67523a3b4be3ce1989d607a828d036249522dd9c1c8de7f4dd2dae43a37369d1" +checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", @@ -480,9 +480,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.4" +version = "4.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bc066a67923782aa8515dbaea16946c5bcc5addbd668bb80af688e53e548a0" +checksum = "5db83dced34638ad474f39f250d7fea9598bdd239eaced1bdf45d597da0f433f" dependencies = [ "clap_builder", "clap_derive", @@ -490,9 +490,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.2" +version = "4.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae129e2e766ae0ec03484e609954119f123cc1fe650337e155d03b022f24f7b4" +checksum = "f7e204572485eb3fbf28f871612191521df159bc3e15a9f5064c66dba3a8c05f" dependencies = [ "anstream", "anstyle", @@ -502,21 +502,21 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.4" +version = "4.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "528131438037fd55894f62d6e9f068b8f45ac57ffa77517819645d10aed04f64" +checksum = "c780290ccf4fb26629baa7a1081e68ced113f1d3ec302fa5948f1c381ebf06c6" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] name = "clap_lex" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" +checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" [[package]] name = "colorchoice" @@ -662,16 +662,15 @@ dependencies = [ [[package]] name = "curve25519-dalek" -version = "4.1.2" +version = "4.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a677b8922c94e01bdbb12126b0bc852f00447528dee1782229af9c720c3f348" +checksum = "97fb8b7c4503de7d6ae7b42ab72a5a59857b4c937ec27a3d4539dba95b5ab2be" dependencies = [ "cfg-if", "cpufeatures", "curve25519-dalek-derive", "digest 0.10.7", "fiat-crypto", - "platforms", "rustc_version", "subtle", "zeroize", @@ -685,7 +684,7 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -709,7 +708,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -720,7 +719,7 @@ checksum = "733cabb43482b1a1b53eee8583c2b9e8684d592215ea83efd305dd31bc2f0178" dependencies = [ "darling_core", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1014,7 +1013,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1081,9 +1080,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.1" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" +checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" [[package]] name = "glob" @@ -1207,9 +1206,9 @@ checksum = "add0ab9360ddbd88cfeb3bd9574a1d85cfdfa14db10b3e21d3700dbc4328758f" [[package]] name = "httparse" -version = "1.8.0" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" +checksum = "0fcc0b4a115bf80b728eb8ea024ad5bd707b615bfed49e0665b6e0f86fd082d9" [[package]] name = "httpdate" @@ -1219,9 +1218,9 @@ checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "hyper" -version = "0.14.28" +version = "0.14.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf96e135eb83a2a8ddf766e426a841d8ddd7449d5f00d34ea02b41d2f19eef80" +checksum = "f361cde2f109281a220d4307746cdfd5ee3f410da58a70377762396775634b33" dependencies = [ "bytes", "futures-channel", @@ -1638,7 +1637,7 @@ dependencies = [ "proc-macro2", "quote", "serde_json", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1713,9 +1712,9 @@ dependencies = [ [[package]] name = "libz-sys" -version = "1.1.16" +version = "1.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e143b5e666b2695d28f6bca6497720813f699c9602dd7f5cac91008b8ada7f9" +checksum = "c15da26e5af7e25c90b37a2d75cdbf940cf4a55316de9d84c679c9b8bfabf82e" dependencies = [ "cc", "pkg-config", @@ -1724,22 +1723,22 @@ dependencies = [ [[package]] name = "linkme" -version = "0.3.26" +version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "833222afbfe72868ac8f9770c91a33673f0d5fefc37c9dbe94aa3548b571623f" +checksum = "ccb76662d78edc9f9bf56360d6919bdacc8b7761227727e5082f128eeb90bbf5" dependencies = [ "linkme-impl", ] [[package]] name = "linkme-impl" -version = "0.3.26" +version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39f0dea92dbea3271557cc2e1848723967bba81f722f95026860974ec9283f08" +checksum = "f8dccda732e04fa3baf2e17cf835bfe2601c7c2edafd64417c627dabae3a8cda" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1784,7 +1783,7 @@ dependencies = [ "proc-macro2", "quote", "regex-syntax 0.6.29", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1798,9 +1797,9 @@ dependencies = [ [[package]] name = "lz4-sys" -version = "1.9.4" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +checksum = "e9764018d143cc854c9f17f0b907de70f14393b1f502da6375dce70f00514eb3" dependencies = [ "cc", "libc", @@ -1817,9 +1816,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.2" +version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" [[package]] name = "miette" @@ -1841,7 +1840,7 @@ checksum = "49e7bc1560b95a3c4a25d03de42fe76ca718ab92d1a22a55b9b4cf67b3ae635c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -1858,9 +1857,9 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae" +checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08" dependencies = [ "adler", ] @@ -1948,9 +1947,9 @@ dependencies = [ [[package]] name = "object" -version = "0.32.2" +version = "0.36.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +checksum = "576dfe1fc8f9df304abb159d767a29d0476f7750fbf8aa7ad07816004a207434" dependencies = [ "memchr", ] @@ -2008,9 +2007,9 @@ dependencies = [ [[package]] name = "parking_lot" -version = "0.12.2" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e4af0ca4f6caed20e900d564c242b8e5d4903fdacf31d3daf527b66fe6f42fb" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" dependencies = [ "lock_api", "parking_lot_core", @@ -2082,7 +2081,7 @@ dependencies = [ "pest_meta", "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2123,7 +2122,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2154,17 +2153,11 @@ version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" -[[package]] -name = "platforms" -version = "3.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db23d408679286588f4d4644f965003d056e3dd5abcaaa938116871d7ce2fee7" - [[package]] name = "plotters" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2c224ba00d7cadd4d5c660deaf2098e5e80e07846537c51f9cfa4be50c1fd45" +checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" dependencies = [ "num-traits", "plotters-backend", @@ -2175,15 +2168,15 @@ dependencies = [ [[package]] name = "plotters-backend" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e76628b4d3a7581389a35d5b6e2139607ad7c75b17aed325f210aa91f4a9609" +checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" [[package]] name = "plotters-svg" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38f6d39893cca0701371e3c27294f09797214b86f1fb951b89ade8ec04e2abab" +checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" dependencies = [ "plotters-backend", ] @@ -2240,14 +2233,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f12335488a2f3b0a83b14edad48dca9879ce89b2edd10e80237e4e852dd645e" dependencies = [ "proc-macro2", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] name = "proc-macro2" -version = "1.0.83" +version = "1.0.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b33eb56c327dec362a9e55b3ad14f9d2f0904fb5a5b03b513ab5465399e9f43" +checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" dependencies = [ "unicode-ident", ] @@ -2272,7 +2265,7 @@ checksum = "440f724eba9f6996b75d63681b0a92b06947f1457076d503a4d2e2c8f56442b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2302,7 +2295,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.65", + "syn 2.0.66", "tempfile", ] @@ -2316,7 +2309,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2487,23 +2480,23 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e" +checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd" dependencies = [ "bitflags 2.5.0", ] [[package]] name = "regex" -version = "1.10.4" +version = "1.10.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c" +checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.6", - "regex-syntax 0.8.3", + "regex-automata 0.4.7", + "regex-syntax 0.8.4", ] [[package]] @@ -2517,13 +2510,13 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" +checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.3", + "regex-syntax 0.8.4", ] [[package]] @@ -2534,9 +2527,9 @@ checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" [[package]] name = "regex-syntax" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56" +checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "rfc6979" @@ -2682,9 +2675,9 @@ dependencies = [ [[package]] name = "schemars" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0218ceea14babe24a4a5836f86ade86c1effbc198164e619194cb5069187e29" +checksum = "09c024468a378b7e36765cd36702b7a90cc3cba11654f6685c8f233408e89e92" dependencies = [ "dyn-clone", "schemars_derive", @@ -2694,14 +2687,14 @@ dependencies = [ [[package]] name = "schemars_derive" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ed5a1ccce8ff962e31a165d41f6e2a2dd1245099dc4d594f5574a86cd90f4d3" +checksum = "b1eee588578aff73f856ab961cd2f79e36bc45d7ded33a7562adba4667aecc0e" dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2775,9 +2768,9 @@ checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" [[package]] name = "serde" -version = "1.0.202" +version = "1.0.203" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "226b61a0d411b2ba5ff6d7f73a476ac4f8bb900373459cd00fab8512828ba395" +checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" dependencies = [ "serde_derive", ] @@ -2794,13 +2787,13 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.202" +version = "1.0.203" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6048858004bcff69094cd972ed40a32500f153bd3be9f716b2eed2e8217c4838" +checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -2811,7 +2804,7 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3020,9 +3013,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.65" +version = "2.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2863d96a84c6439701d7a38f9de935ec562c8832cc55d1dde0f513b52fad106" +checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" dependencies = [ "proc-macro2", "quote", @@ -3058,7 +3051,7 @@ checksum = "f9b53c7124dd88026d5d98a1eb1fd062a578b7d783017c9298825526c7fb6427" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3092,7 +3085,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3160,9 +3153,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.37.0" +version = "1.38.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" +checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" dependencies = [ "backtrace", "bytes", @@ -3189,13 +3182,13 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3304,7 +3297,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3387,9 +3380,9 @@ dependencies = [ [[package]] name = "unicode-width" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68f5e5f3158ecfd4b8ff6fe086db7c8467a2dfdac97fe420f2b7c4aa97af66d6" +checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" [[package]] name = "universal-hash" @@ -3415,9 +3408,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.5.0" +version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" +checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" dependencies = [ "form_urlencoded", "idna", @@ -3426,9 +3419,9 @@ dependencies = [ [[package]] name = "utf8parse" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "valuable" @@ -3480,7 +3473,7 @@ source = "git+https://github.com/matter-labs/vise.git?rev=a5bb80c9ce7168663114ee dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -3529,7 +3522,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", "wasm-bindgen-shared", ] @@ -3551,7 +3544,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3774,14 +3767,14 @@ checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] name = "zeroize" -version = "1.7.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" +checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" dependencies = [ "zeroize_derive", ] @@ -3794,7 +3787,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] @@ -4027,14 +4020,14 @@ dependencies = [ "prost-reflect", "protox", "quote", - "syn 2.0.65", + "syn 2.0.66", ] [[package]] name = "zstd-sys" -version = "2.0.10+zstd.1.5.6" +version = "2.0.11+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" +checksum = "75652c55c0b6f3e6f12eb786fe1bc960396bf05a1eb3bf1f3691c3610ac2e6d4" dependencies = [ "cc", "pkg-config",