From bce5e3e8189b3f0d3b5eb2aed758a483ba032036 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Wed, 31 Jul 2024 12:54:07 +0100 Subject: [PATCH 01/13] feat: Poll the main node for batch to vote on (BFT-496) (#161) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Poll the main node for which batch height to vote on. - [x] Return an `Option` rather than a `Vec` of `BatchQC` from `BatchVotes::find_quorum`: we decided not to implement voting on multiple heights for now, which makes it just confusing that a vector is returned. - [x] Create an `AttestationStatusClient` trait with a method to poll the next `BatchNumber` to vote on, which can be injected as a `dyn` dependency - [x] Create an `AttestationStatusWatch` that contains the next `BatchNumber` to vote on; this is updated by a single background task and listened to by multiple subscribers - [x] Create an `AttestationStatusRunner` which is supposed to be started along with the `BatchStore` in `zksync-era` and polls the client; the point of this is to push out the poll interval configuration to the edge and make the `AttestationStatusWatch` the integration point - [x] Change `AttesterRunner` to wait for the `AttestationStatusWatch` to change, then the payload to appear, then cast the vote; it doesn't need to worry about reverts because the node will restart if that happens - [x] Change `run_batch_qc_finder` to wait for the `AttestationStatusWatch` to change, then wait for the next available QC, and save it; this might produce gaps which is normal on the external nodes, but undersireable on the main node - on the main node it only produces gaps if the attesters vote on higher heights despite the main node telling them not to, which if they are majority honest should not happen, and if they are not then what can we do anyway - [x] Rename `PersistentBatchStore::earliest_batch_number_to_sign` to `next_batch_to_attest` in accordance with the new method on `consensus_dal`. - [ ] ~~Initialise `BatchNumber::min_batch_number` to by asking the main node where to resume from~~ This initialisation isn't necessary because 1) we decided that we'll only allow 1 vote per attester, so there is no attack vector of casting votes from batch 0 and 2) `run_batch_qc_finder` first waits for the API status to appear and then prunes all preceding data, so an older QC will not be saved. It was also undesirable to have to initialise from an API that might return nothing (or errors) for an unknown amount of time. ### Poll interval The `AttesterStatusRunner` polls the API at fixed intervals without any exponential backoff. I thought 5s would be a reasonable default value. With 60s batches this seems fine because the *next* batch to sign will be signalled as soon as the current batch QC is inserted into the database, which is well ahead of time of when the next batch and its commitment will be available. If we think we'll need to catch up with historical batch signatures it might be a bit slow. We generally still have to account for gossiping around the votes though, which in a large enough network could be on the order of several seconds as well. ### Potential deadlock There is a pathological sequence of events that could prevent the feature from starting on a fresh database with no prior batch QCs: 1. Say we start from batch 0, and a QC is gathered, but saving it to the database (which is an async process) takes a long time on the main node. 2. Say it takes so long that batch 1 and batch 2 are created, but none of them have a QC yet. 3. Say we have two attesters A and B, and A polled the API when it showed batch 2, and the main node set its minimum batch number in the vote registry to batch 2 as well. 4. Now the QC for batch 0 is saved to the database and the API indicates the next one to vote on is batch 1. 5. Attester B casts its vote on batch 1 but it goes ignored by the main node vote registry because its looking for a quorum with at least batch number 2. 6. Having missed an attestation the main node will never save a QC and never move on until it's restarted. Note that even if the registry minimum batch number was adjusted _down_ that might happen _after_ the vote for batch 1 has already been discarded, and because new votes are only pushed once through gossip there is no guarantee that it will get it again. The solution is coming in the form of fixing the starting point of gossip to be where the genesis starts, even if that means filling a potentially long history of batches in the beginning. See https://github.com/matter-labs/zksync-era/pull/2539 ## Why ❔ We want to collect batch QCs without gaps, and for now put the main node in charge of what to vote on. The API to serve this information is in https://github.com/matter-labs/zksync-era/pull/2480 and this PR is the follow up to make polling that information part of the signing process. --- node/Cargo.lock | 1 + node/actors/bft/src/testonly/run.rs | 4 +- node/actors/executor/Cargo.toml | 1 + node/actors/executor/src/attestation.rs | 148 ++++++++++++++---- node/actors/executor/src/lib.rs | 12 +- node/actors/executor/src/tests.rs | 9 ++ node/actors/network/src/consensus/tests.rs | 2 + .../network/src/gossip/attestation_status.rs | 57 +++++++ node/actors/network/src/gossip/batch_votes.rs | 22 +-- node/actors/network/src/gossip/mod.rs | 61 +++++--- node/actors/network/src/gossip/tests/mod.rs | 30 ++-- node/actors/network/src/lib.rs | 6 +- node/actors/network/src/testonly.rs | 38 ++++- node/libs/storage/src/batch_store/metrics.rs | 4 +- node/libs/storage/src/batch_store/mod.rs | 32 ++-- node/libs/storage/src/testonly/in_memory.rs | 2 +- node/tools/src/config.rs | 44 +++++- 17 files changed, 349 insertions(+), 124 deletions(-) create mode 100644 node/actors/network/src/gossip/attestation_status.rs diff --git a/node/Cargo.lock b/node/Cargo.lock index 22ab5e9e..3ea3e4a4 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -4127,6 +4127,7 @@ name = "zksync_consensus_executor" version = "0.1.0-rc.4" dependencies = [ "anyhow", + "async-trait", "rand 0.8.5", "test-casing", "tokio", diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index 9f907f14..bc11e121 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -14,7 +14,7 @@ use zksync_concurrency::{ }, oneshot, scope, }; -use zksync_consensus_network as network; +use zksync_consensus_network::{self as network}; use zksync_consensus_roles::validator; use zksync_consensus_storage::{testonly::TestMemoryStorage, BlockStore}; use zksync_consensus_utils::pipe; @@ -135,9 +135,11 @@ impl Test { for (i, net) in nets.into_iter().enumerate() { let store = TestMemoryStorage::new(ctx, genesis).await; s.spawn_bg(async { Ok(store.runner.run(ctx).await?) }); + if self.nodes[i].0 == Behavior::Honest { honest.push(store.blocks.clone()); } + nodes.push(Node { net, behavior: self.nodes[i].0, diff --git a/node/actors/executor/Cargo.toml b/node/actors/executor/Cargo.toml index dc6b5193..56696efc 100644 --- a/node/actors/executor/Cargo.toml +++ b/node/actors/executor/Cargo.toml @@ -20,6 +20,7 @@ zksync_consensus_utils.workspace = true zksync_protobuf.workspace = true anyhow.workspace = true +async-trait.workspace = true rand.workspace = true tracing.workspace = true vise.workspace = true diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index 3a5146fc..621fa1c3 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -3,19 +3,21 @@ use crate::Attester; use anyhow::Context; use std::sync::Arc; -use zksync_concurrency::{ctx, time}; -use zksync_consensus_network::gossip::BatchVotesPublisher; +use zksync_concurrency::{ctx, sync, time}; +use zksync_consensus_network::gossip::{ + AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, +}; use zksync_consensus_roles::attester; use zksync_consensus_storage::{BatchStore, BlockStore}; -const POLL_INTERVAL: time::Duration = time::Duration::seconds(1); - /// Polls the database for new batches to be signed and publishes them to the gossip channel. pub(super) struct AttesterRunner { block_store: Arc, batch_store: Arc, attester: Attester, publisher: BatchVotesPublisher, + status: AttestationStatusReceiver, + poll_interval: time::Duration, } impl AttesterRunner { @@ -25,16 +27,20 @@ impl AttesterRunner { batch_store: Arc, attester: Attester, publisher: BatchVotesPublisher, + status: AttestationStatusReceiver, + poll_interval: time::Duration, ) -> Self { Self { block_store, batch_store, attester, publisher, + status, + poll_interval, } } /// Poll the database for new L1 batches and publish our signature over the batch. - pub(super) async fn run(self, ctx: &ctx::Ctx) -> ctx::Result<()> { + pub(super) async fn run(mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { let public_key = self.attester.key.public(); // TODO: In the future when we have attester rotation these checks will have to be checked inside the loop. let Some(attesters) = self.block_store.genesis().attesters.as_ref() else { @@ -48,28 +54,27 @@ impl AttesterRunner { let genesis = self.block_store.genesis().hash(); - // Find the initial range of batches that we want to (re)sign after a (re)start. - let last_batch_number = self - .batch_store - .wait_until_persisted(ctx, attester::BatchNumber(0)) - .await - .context("wait_until_persisted")? - .last - .unwrap_or_default(); + // Subscribe starts as seen but we don't want to miss the first item. + self.status.mark_changed(); - // Determine the batch to start signing from. - let earliest_batch_number = self - .batch_store - .earliest_batch_number_to_sign(ctx) - .await - .context("earliest_batch_number_to_sign")? - .unwrap_or(last_batch_number); + loop { + let Some(batch_number) = sync::changed(ctx, &mut self.status) + .await? + .next_batch_to_attest + else { + continue; + }; - tracing::info!(%earliest_batch_number, %last_batch_number, "attesting batches"); + tracing::info!(%batch_number, "attestation status"); - let mut batch_number = earliest_batch_number; + // We can avoid actively polling the database in `wait_for_batch_to_sign` by waiting its existence + // to be indicated in memory (which itself relies on polling). This happens once we have the commitment, + // which for nodes that get the blocks through BFT should happen after execution. Nodes which + // rely on batch sync don't participate in attestations as they need the batch on L1 first. + self.batch_store + .wait_until_persisted(ctx, batch_number) + .await?; - loop { // Try to get the next batch to sign; the commitment might not be available just yet. let batch = self.wait_for_batch_to_sign(ctx, batch_number).await?; @@ -84,16 +89,6 @@ impl AttesterRunner { .publish(attesters, &genesis, &self.attester.key, batch) .await .context("publish")?; - - batch_number = batch_number.next(); - - // We can avoid actively polling the database by waiting for the next persisted batch to appear - // in the memory (which itself relies on polling). This happens once we have the commitment, - // which for nodes that get the blocks through BFT should happen after execution. Nodes which - // rely on batch sync don't participate in attestations as they need the batch on L1 first. - self.batch_store - .wait_until_persisted(ctx, batch_number) - .await?; } } @@ -112,8 +107,93 @@ impl AttesterRunner { { return Ok(batch); } else { - ctx.sleep(POLL_INTERVAL).await?; + ctx.sleep(self.poll_interval).await?; + } + } + } +} + +/// An interface which is used by attesters and nodes collecting votes over gossip to determine +/// which is the next batch they are all supposed to be voting on, according to the main node. +/// +/// This is a convenience interface to be used with the [AttestationStatusRunner]. +#[async_trait::async_trait] +pub trait AttestationStatusClient: 'static + Send + Sync { + /// Get the next batch number for which the main node expects a batch QC to be formed. + /// + /// The API might return an error while genesis is being created, which we represent with `None` + /// here and mean that we'll have to try again later. + async fn next_batch_to_attest( + &self, + ctx: &ctx::Ctx, + ) -> ctx::Result>; +} + +/// Use an [AttestationStatusClient] to periodically poll the main node and update the [AttestationStatusWatch]. +/// +/// This is provided for convenience. +pub struct AttestationStatusRunner { + status: Arc, + client: Box, + poll_interval: time::Duration, +} + +impl AttestationStatusRunner { + /// Create a new runner to poll the main node. + pub fn new( + status: Arc, + client: Box, + poll_interval: time::Duration, + ) -> Self { + Self { + status, + client, + poll_interval, + } + } + + /// Runner based on a [BatchStore]. + pub fn new_from_store( + status: Arc, + store: Arc, + poll_interval: time::Duration, + ) -> Self { + Self::new( + status, + Box::new(LocalAttestationStatusClient(store)), + poll_interval, + ) + } + + /// Run the poll loop. + pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + loop { + match self.client.next_batch_to_attest(ctx).await { + Ok(Some(batch_number)) => { + self.status.update(batch_number).await; + } + Ok(None) => tracing::debug!("waiting for attestation status..."), + Err(error) => tracing::error!( + ?error, + "failed to poll attestation status, retrying later..." + ), + } + if let Err(ctx::Canceled) = ctx.sleep(self.poll_interval).await { + return Ok(()); } } } } + +/// Implement the attestation status for the main node by returning the next to vote on from the [BatchStore]. +struct LocalAttestationStatusClient(Arc); + +#[async_trait::async_trait] +impl AttestationStatusClient for LocalAttestationStatusClient { + async fn next_batch_to_attest( + &self, + ctx: &ctx::Ctx, + ) -> ctx::Result> { + self.0.next_batch_to_attest(ctx).await + } +} diff --git a/node/actors/executor/src/lib.rs b/node/actors/executor/src/lib.rs index f0475687..adaf4016 100644 --- a/node/actors/executor/src/lib.rs +++ b/node/actors/executor/src/lib.rs @@ -9,13 +9,13 @@ use std::{ }; use zksync_concurrency::{ctx, limiter, net, scope, time}; use zksync_consensus_bft as bft; -use zksync_consensus_network as network; +use zksync_consensus_network::{self as network, gossip::AttestationStatusWatch}; use zksync_consensus_roles::{attester, node, validator}; use zksync_consensus_storage::{BatchStore, BlockStore, ReplicaStore}; use zksync_consensus_utils::pipe; use zksync_protobuf::kB; -mod attestation; +pub mod attestation; mod io; #[cfg(test)] mod tests; @@ -70,6 +70,9 @@ pub struct Config { /// Http debug page configuration. /// If None, debug page is disabled pub debug_page: Option, + + /// How often to poll the database looking for the batch commitment. + pub batch_poll_interval: time::Duration, } impl Config { @@ -97,6 +100,8 @@ pub struct Executor { pub validator: Option, /// Validator-specific node data. pub attester: Option, + /// Status showing where the main node wants attester to cast their votes. + pub attestation_status: Arc, } impl Executor { @@ -138,6 +143,7 @@ impl Executor { self.block_store.clone(), self.batch_store.clone(), network_actor_pipe, + self.attestation_status.clone(), ); net.register_metrics(); s.spawn(async { runner.run(ctx).await.context("Network stopped") }); @@ -149,6 +155,8 @@ impl Executor { self.batch_store.clone(), attester, net.batch_vote_publisher(), + self.attestation_status.subscribe(), + self.config.batch_poll_interval, ); s.spawn(async { runner.run(ctx).await?; diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 1c8c9f41..6ce5c057 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -23,9 +23,16 @@ fn config(cfg: &network::Config) -> Config { gossip_static_outbound: cfg.gossip.static_outbound.clone(), rpc: cfg.rpc.clone(), debug_page: None, + batch_poll_interval: time::Duration::seconds(1), } } +/// The test executors below are not running with attesters, so we just create an [AttestationStatusWatch] +/// that will never be updated. +fn never_attest() -> Arc { + Arc::new(AttestationStatusWatch::default()) +} + fn validator( cfg: &network::Config, block_store: Arc, @@ -42,6 +49,7 @@ fn validator( payload_manager: Box::new(bft::testonly::RandomPayload(1000)), }), attester: None, + attestation_status: never_attest(), } } @@ -56,6 +64,7 @@ fn fullnode( batch_store, validator: None, attester: None, + attestation_status: never_attest(), } } diff --git a/node/actors/network/src/consensus/tests.rs b/node/actors/network/src/consensus/tests.rs index 60049757..052aea13 100644 --- a/node/actors/network/src/consensus/tests.rs +++ b/node/actors/network/src/consensus/tests.rs @@ -256,9 +256,11 @@ async fn test_address_change() { // should get reconstructed. cfgs[0].server_addr = net::tcp::testonly::reserve_listener(); cfgs[0].public_addr = (*cfgs[0].server_addr).into(); + let (node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone(), store.batches.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node0"))); + nodes[0] = node0; for n in &nodes { n.wait_for_consensus_connections().await; diff --git a/node/actors/network/src/gossip/attestation_status.rs b/node/actors/network/src/gossip/attestation_status.rs new file mode 100644 index 00000000..5509bf81 --- /dev/null +++ b/node/actors/network/src/gossip/attestation_status.rs @@ -0,0 +1,57 @@ +use std::fmt; + +use zksync_concurrency::sync; +use zksync_consensus_roles::attester; + +use crate::watch::Watch; + +/// Coordinate the attestation by showing the status as seen by the main node. +#[derive(Debug, Clone)] +pub struct AttestationStatus { + /// Next batch number where voting is expected. + /// + /// Its value is `None` until the background process polling the main node + /// can establish a value to start from. + pub next_batch_to_attest: Option, +} + +/// The subscription over the attestation status which voters can monitor for change. +pub type AttestationStatusReceiver = sync::watch::Receiver; + +/// A [Watch] over an [AttestationStatus] which we can use to notify components about +/// changes in the batch number the main node expects attesters to vote on. +pub struct AttestationStatusWatch(Watch); + +impl fmt::Debug for AttestationStatusWatch { + fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { + fmt.debug_struct("AttestationStatusWatch") + .finish_non_exhaustive() + } +} + +impl Default for AttestationStatusWatch { + fn default() -> Self { + Self(Watch::new(AttestationStatus { + next_batch_to_attest: None, + })) + } +} + +impl AttestationStatusWatch { + /// Subscribes to AttestationStatus updates. + pub fn subscribe(&self) -> AttestationStatusReceiver { + self.0.subscribe() + } + + /// Set the next batch number to attest on and notify subscribers it changed. + pub async fn update(&self, next_batch_to_attest: attester::BatchNumber) { + let this = self.0.lock().await; + this.send_if_modified(|status| { + if status.next_batch_to_attest == Some(next_batch_to_attest) { + return false; + } + status.next_batch_to_attest = Some(next_batch_to_attest); + true + }); + } +} diff --git a/node/actors/network/src/gossip/batch_votes.rs b/node/actors/network/src/gossip/batch_votes.rs index 1f5ad613..1ea6b8cb 100644 --- a/node/actors/network/src/gossip/batch_votes.rs +++ b/node/actors/network/src/gossip/batch_votes.rs @@ -38,7 +38,7 @@ impl BatchUpdateStats { pub(crate) struct BatchVotes { /// The latest vote received from each attester. We only keep the last one /// for now, hoping that with 1 minute batches there's plenty of time for - /// the quorum to be reached, but eventually we'll have to allow multiple + /// the quorum to be reached, but eventually we might have to allow multiple /// votes across different heights. pub(crate) votes: im::HashMap>>, @@ -51,6 +51,11 @@ pub(crate) struct BatchVotes { im::OrdMap>, /// The minimum batch number for which we are still interested in votes. + /// + /// Because we only store 1 vote per attester the memory is very much bounded, + /// but this extra pruning mechanism can be used to clear votes of attesters + /// who have been removed from the committee, as well as to get rid of the + /// last quorum we found and stored, and look for the a new one in the next round. pub(crate) min_batch_number: attester::BatchNumber, } @@ -131,20 +136,19 @@ impl BatchVotes { /// Check if we have achieved quorum for any of the batch hashes. /// - /// The return value is a vector because eventually we will be potentially waiting for - /// quorums on multiple heights simultaneously. + /// Returns the first quorum it finds, after which we expect that the state of the main node or L1 + /// will indicate that attestation on the next height can happen, which will either naturally move + /// the QC, or we can do so by increasing the `min_batch_number`. /// - /// For repeated queries we can supply a skip list of heights for which we already saved the QC. - pub(super) fn find_quorums( + /// While we only store 1 vote per attester we'll only ever have at most one quorum anyway. + pub(super) fn find_quorum( &self, attesters: &attester::Committee, genesis: &attester::GenesisHash, - skip: impl Fn(attester::BatchNumber) -> bool, - ) -> Vec { + ) -> Option { let threshold = attesters.threshold(); self.support .iter() - .filter(|(number, _)| !skip(**number)) .flat_map(|(number, candidates)| { candidates .iter() @@ -170,7 +174,7 @@ impl BatchVotes { } }) }) - .collect() + .next() } /// Set the minimum batch number for which we admit votes. diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index f5afd92b..5da55cce 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -12,6 +12,7 @@ //! Static connections constitute a rigid "backbone" of the gossip network, which is insensitive to //! eclipse attack. Dynamic connections are supposed to improve the properties of the gossip //! network graph (minimize its diameter, increase connectedness). +pub use self::attestation_status::{AttestationStatusReceiver, AttestationStatusWatch}; pub use self::batch_votes::BatchVotesPublisher; use self::batch_votes::BatchVotesWatch; use crate::{gossip::ValidatorAddrsWatch, io, pool::PoolWatch, Config, MeteredStreamStats}; @@ -22,6 +23,7 @@ use zksync_concurrency::{ctx, ctx::channel, error::Wrap as _, scope, sync}; use zksync_consensus_roles::{node, validator}; use zksync_consensus_storage::{BatchStore, BlockStore}; +mod attestation_status; mod batch_votes; mod fetch; mod handshake; @@ -58,6 +60,8 @@ pub(crate) struct Network { pub(crate) fetch_queue: fetch::Queue, /// TESTONLY: how many time push_validator_addrs rpc was called by the peers. pub(crate) push_validator_addrs_calls: AtomicUsize, + /// Shared watch over the current attestation status as indicated by the main node. + pub(crate) attestation_status: Arc, } impl Network { @@ -67,6 +71,7 @@ impl Network { block_store: Arc, batch_store: Arc, sender: channel::UnboundedSender, + attestation_status: Arc, ) -> Arc { Arc::new(Self { sender, @@ -82,6 +87,7 @@ impl Network { block_store, batch_store, push_validator_addrs_calls: 0.into(), + attestation_status, }) } @@ -148,37 +154,50 @@ impl Network { .await; } - /// Task that keeps hearing about new votes and looks for an L1 batch qc. - /// It will propagate the QC if there's enough votes. + /// Task that reacts to new votes being added and looks for an L1 batch QC. + /// It persists the certificate once the quorum threshold is passed. pub(crate) async fn run_batch_qc_finder(&self, ctx: &ctx::Ctx) -> ctx::Result<()> { let Some(attesters) = self.genesis().attesters.as_ref() else { + tracing::info!("no attesters in genesis, not looking for batch QCs"); return Ok(()); }; let genesis = self.genesis().hash(); - let mut sub = self.batch_votes.subscribe(); + + let mut recv_votes = self.batch_votes.subscribe(); + let mut recv_status = self.attestation_status.subscribe(); + + // Subscribe starts as seen but we don't want to miss the first item. + recv_status.mark_changed(); + loop { - // In the future when we might be gossiping about multiple batches at the same time, - // we can collect the ones we submitted into a skip list until we see them confirmed - // on L1 and we can finally increase the minimum as well. - let quorums = { - let votes = sync::changed(ctx, &mut sub).await?; - votes.find_quorums(attesters, &genesis, |_| false) + // Wait until the status indicates that we're ready to sign the next batch. + let Some(next_batch_number) = sync::changed(ctx, &mut recv_status) + .await? + .next_batch_to_attest + else { + continue; }; - for qc in quorums { - // In the future this should come from confirmations, but for now it's best effort, so we can forget ASAP. - // TODO: An initial value could be looked up in the database even now. - let next_batch_number = qc.message.number.next(); + // Get rid of all previous votes. We don't expect this to go backwards without regenesis, which will involve a restart. + self.batch_votes + .set_min_batch_number(next_batch_number) + .await; - self.batch_store - .persist_batch_qc(ctx, qc) - .await - .wrap("persist_batch_qc")?; + // Now wait until we find the next quorum, whatever it is: + // * on the main node, if attesters are honest, they will vote on the next batch number and the main node will not see gaps + // * on external nodes the votes might be affected by changes in the value returned by the API, and there might be gaps + // What is important, though, is that the batch number does not move backwards while we look for a quorum, because attesters + // (re)casting earlier votes will go ignored by those fixed on a higher min_batch_number, and gossip will only be attempted once. + // The possibility of this will be fixed by deterministally picking a start batch number based on fork indicated by genesis. + let qc = sync::wait_for_some(ctx, &mut recv_votes, |votes| { + votes.find_quorum(attesters, &genesis) + }) + .await?; - self.batch_votes - .set_min_batch_number(next_batch_number) - .await; - } + self.batch_store + .persist_batch_qc(ctx, qc) + .await + .wrap("persist_batch_qc")?; } } } diff --git a/node/actors/network/src/gossip/tests/mod.rs b/node/actors/network/src/gossip/tests/mod.rs index 6811071f..e0f5f3c2 100644 --- a/node/actors/network/src/gossip/tests/mod.rs +++ b/node/actors/network/src/gossip/tests/mod.rs @@ -643,8 +643,8 @@ fn test_batch_votes_quorum() { let rng = &mut ctx::test_root(&ctx::RealClock).rng(); for _ in 0..10 { - let size = rng.gen_range(1..20); - let keys: Vec = (0..size).map(|_| rng.gen()).collect(); + let committee_size = rng.gen_range(1..20); + let keys: Vec = (0..committee_size).map(|_| rng.gen()).collect(); let attesters = attester::Committee::new(keys.iter().map(|k| attester::WeightedAttester { key: k.public(), weight: rng.gen_range(1..=100), @@ -673,27 +673,17 @@ fn test_batch_votes_quorum() { // Check that as soon as we have quorum it's found. if batches[b].1 >= attesters.threshold() { - let qs = votes.find_quorums(&attesters, &genesis, |_| false); - assert!(!qs.is_empty(), "should find quorum"); - assert!(qs[0].message == *batch); - assert!(qs[0].signatures.keys().count() > 0); + let qc = votes + .find_quorum(&attesters, &genesis) + .expect("should find quorum"); + assert!(qc.message == *batch); + assert!(qc.signatures.keys().count() > 0); } } - if let Some(quorum) = batches - .iter() - .find(|b| b.1 >= attesters.threshold()) - .map(|(b, _)| b) - { - // Check that a quorum can be skipped - assert!(votes - .find_quorums(&attesters, &genesis, |b| b == quorum.number) - .is_empty()); - } else { - // Check that if there was no quoroum then we don't find any. - assert!(votes - .find_quorums(&attesters, &genesis, |_| false) - .is_empty()); + // Check that if there was no quoroum then we don't find any. + if !batches.iter().any(|b| b.1 >= attesters.threshold()) { + assert!(votes.find_quorum(&attesters, &genesis).is_none()); } // Check that the minimum batch number prunes data. diff --git a/node/actors/network/src/lib.rs b/node/actors/network/src/lib.rs index 1409b087..1d5f3cbd 100644 --- a/node/actors/network/src/lib.rs +++ b/node/actors/network/src/lib.rs @@ -1,6 +1,6 @@ //! Network actor maintaining a pool of outbound and inbound connections to other nodes. use anyhow::Context as _; -use gossip::BatchVotesPublisher; +use gossip::{AttestationStatusWatch, BatchVotesPublisher}; use std::sync::Arc; use tracing::Instrument as _; use zksync_concurrency::{ @@ -57,8 +57,10 @@ impl Network { block_store: Arc, batch_store: Arc, pipe: ActorPipe, + attestation_status: Arc, ) -> (Arc, Runner) { - let gossip = gossip::Network::new(cfg, block_store, batch_store, pipe.send); + let gossip = + gossip::Network::new(cfg, block_store, batch_store, pipe.send, attestation_status); let consensus = consensus::Network::new(gossip.clone()); let net = Arc::new(Self { gossip, consensus }); ( diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index dfc28dd5..d8d8c8ed 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -1,6 +1,7 @@ //! Testonly utilities. #![allow(dead_code)] use crate::{ + gossip::AttestationStatusWatch, io::{ConsensusInputMessage, Target}, Config, GossipConfig, Network, RpcConfig, Runner, }; @@ -12,7 +13,10 @@ use std::{ collections::{HashMap, HashSet}, sync::Arc, }; -use zksync_concurrency::{ctx, ctx::channel, io, limiter, net, scope, sync}; +use zksync_concurrency::{ + ctx::{self, channel}, + io, limiter, net, scope, sync, time, +}; use zksync_consensus_roles::{node, validator}; use zksync_consensus_storage::{BatchStore, BlockStore}; use zksync_consensus_utils::pipe; @@ -156,7 +160,9 @@ pub fn new_fullnode(rng: &mut impl Rng, peer: &Config) -> Config { /// Runner for Instance. pub struct InstanceRunner { - runner: Runner, + net_runner: Runner, + attestation_status: Arc, + batch_store: Arc, terminate: channel::Receiver<()>, } @@ -164,7 +170,17 @@ impl InstanceRunner { /// Runs the instance background processes. pub async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { scope::run!(ctx, |ctx, s| async { - s.spawn_bg(self.runner.run(ctx)); + s.spawn_bg(self.net_runner.run(ctx)); + s.spawn_bg(async { + loop { + if let Ok(Some(n)) = self.batch_store.next_batch_to_attest(ctx).await { + self.attestation_status.update(n).await; + } + if ctx.sleep(time::Duration::seconds(1)).await.is_err() { + return Ok(()); + } + } + }); let _ = self.terminate.recv(ctx).await; Ok(()) }) @@ -181,8 +197,18 @@ impl Instance { block_store: Arc, batch_store: Arc, ) -> (Self, InstanceRunner) { + // Semantically we'd want this to be created at the same level as the stores, + // but doing so would introduce a lot of extra cruft in setting up tests. + let attestation_status = Arc::new(AttestationStatusWatch::default()); + let (actor_pipe, dispatcher_pipe) = pipe::new(); - let (net, runner) = Network::new(cfg, block_store, batch_store, actor_pipe); + let (net, net_runner) = Network::new( + cfg, + block_store, + batch_store.clone(), + actor_pipe, + attestation_status.clone(), + ); let (terminate_send, terminate_recv) = channel::bounded(1); ( Self { @@ -191,7 +217,9 @@ impl Instance { terminate: terminate_send, }, InstanceRunner { - runner, + net_runner, + attestation_status, + batch_store, terminate: terminate_recv, }, ) diff --git a/node/libs/storage/src/batch_store/metrics.rs b/node/libs/storage/src/batch_store/metrics.rs index cabe7e34..deddc351 100644 --- a/node/libs/storage/src/batch_store/metrics.rs +++ b/node/libs/storage/src/batch_store/metrics.rs @@ -7,9 +7,9 @@ pub(super) struct PersistentBatchStore { /// Latency of a successful `get_batch()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] pub(super) batch_latency: vise::Histogram, - /// Latency of a successful `earliest_batch_number_to_sign()` call. + /// Latency of a successful `next_batch_to_attest_latency()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) earliest_batch_latency: vise::Histogram, + pub(super) next_batch_to_attest_latency: vise::Histogram, /// Latency of a successful `get_batch_to_sign()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] pub(super) batch_to_sign_latency: vise::Histogram, diff --git a/node/libs/storage/src/batch_store/mod.rs b/node/libs/storage/src/batch_store/mod.rs index 3b6b6fe9..889014f1 100644 --- a/node/libs/storage/src/batch_store/mod.rs +++ b/node/libs/storage/src/batch_store/mod.rs @@ -53,16 +53,12 @@ pub trait PersistentBatchStore: 'static + fmt::Debug + Send + Sync { /// Range of batches persisted in storage. fn persisted(&self) -> sync::watch::Receiver; - /// Get the earliest of L1 batches which are missing the corresponding L1 batch quorum certificates - /// and potentially need to be signed by attesters. + /// Get the next L1 batch for which attesters are expected to produce a quorum certificate. /// - /// A replica might never have a complete history of L1 batch QCs; once the L1 batch is included on L1, - /// the replicas might use the [attester::SyncBatch] route to obtain them, in which case they will not - /// have a QC and no reason to get them either. The store will have sufficient information to decide - /// where it's still necessary to gossip votes; for example the main node will want to have a QC on - /// every batch while it's the one submitting them to L1, while replicas can ask the L1 what is considered - /// final. - async fn earliest_batch_number_to_sign( + /// An external node might never have a complete history of L1 batch QCs. Once the L1 batch is included on L1, + /// the external nodes might use the [attester::SyncBatch] route to obtain them, in which case they will not + /// have a QC and no reason to get them either. The main node, however, will want to have a QC for all batches. + async fn next_batch_to_attest( &self, ctx: &ctx::Ctx, ) -> ctx::Result>; @@ -279,28 +275,20 @@ impl BatchStore { Ok(batch) } - /// Retrieve the minimum batch number that doesn't have a QC yet and potentially need to be signed. - /// - /// There might be unsigned batches before this one in the database, however we don't consider it - /// necessary to sign them any more, because for example they have already been submitted to L1. - /// - /// There might also be signed batches *after* this one, due to the way gossiping works, but we - /// might still have to fill the gaps by (re)submitting our signature to allow them to be submitted. - /// - /// Returns `None` if all existing batches are signed, or there are not batches yet to be signed at all. - pub async fn earliest_batch_number_to_sign( + /// Retrieve the next batch number that doesn't have a QC yet and will need to be signed. + pub async fn next_batch_to_attest( &self, ctx: &ctx::Ctx, ) -> ctx::Result> { let t = metrics::PERSISTENT_BATCH_STORE - .earliest_batch_latency + .next_batch_to_attest_latency .start(); let batch = self .persistent - .earliest_batch_number_to_sign(ctx) + .next_batch_to_attest(ctx) .await - .wrap("persistent.get_batch_to_sign()")?; + .wrap("persistent.next_batch_to_attest()")?; t.observe(); Ok(batch) diff --git a/node/libs/storage/src/testonly/in_memory.rs b/node/libs/storage/src/testonly/in_memory.rs index 084da660..79a21207 100644 --- a/node/libs/storage/src/testonly/in_memory.rs +++ b/node/libs/storage/src/testonly/in_memory.rs @@ -139,7 +139,7 @@ impl PersistentBatchStore for BatchStore { Ok(certs.get(last_batch_number).cloned()) } - async fn earliest_batch_number_to_sign( + async fn next_batch_to_attest( &self, _ctx: &ctx::Ctx, ) -> ctx::Result> { diff --git a/node/tools/src/config.rs b/node/tools/src/config.rs index 221baf04..f659bdb3 100644 --- a/node/tools/src/config.rs +++ b/node/tools/src/config.rs @@ -7,13 +7,14 @@ use std::{ fs, io, net::SocketAddr, path::PathBuf, + sync::Arc, }; use tokio_rustls::rustls::pki_types::{CertificateDer, PrivateKeyDer}; -use zksync_concurrency::{ctx, net}; +use zksync_concurrency::{ctx, net, scope, time}; use zksync_consensus_bft as bft; use zksync_consensus_crypto::{read_optional_text, read_required_text, Text, TextFmt}; -use zksync_consensus_executor as executor; -use zksync_consensus_network::http; +use zksync_consensus_executor::{self as executor, attestation::AttestationStatusRunner}; +use zksync_consensus_network::{gossip::AttestationStatusWatch, http}; use zksync_consensus_roles::{attester, node, validator}; use zksync_consensus_storage::testonly::{TestMemoryStorage, TestMemoryStorageRunner}; use zksync_consensus_utils::debug_page; @@ -259,9 +260,23 @@ impl Configs { pub async fn make_executor( &self, ctx: &ctx::Ctx, - ) -> ctx::Result<(executor::Executor, TestMemoryStorageRunner)> { + ) -> ctx::Result<(executor::Executor, TestExecutorRunner)> { let replica_store = store::RocksDB::open(self.app.genesis.clone(), &self.database).await?; let store = TestMemoryStorage::new(ctx, &self.app.genesis).await; + + // We don't have an API to poll in this setup, we can only create a local store based attestation client. + let attestation_status = Arc::new(AttestationStatusWatch::default()); + let attestation_status_runner = AttestationStatusRunner::new_from_store( + attestation_status.clone(), + store.batches.clone(), + time::Duration::seconds(1), + ); + + let runner = TestExecutorRunner { + storage_runner: store.runner, + attestation_status_runner, + }; + let e = executor::Executor { config: executor::Config { server_addr: self.app.server_addr, @@ -283,6 +298,7 @@ impl Configs { .expect("Could not obtain private key for debug page"), } }), + batch_poll_interval: time::Duration::seconds(1), }, block_store: store.blocks, batch_store: store.batches, @@ -302,8 +318,9 @@ impl Configs { .attester_key .as_ref() .map(|key| executor::Attester { key: key.clone() }), + attestation_status, }; - Ok((e, store.runner)) + Ok((e, runner)) } } @@ -328,3 +345,20 @@ fn load_private_key(path: &PathBuf) -> anyhow::Result> { // Load and return a single private key. Ok(rustls_pemfile::private_key(&mut reader).map(|key| key.expect("Private key not found"))?) } + +pub struct TestExecutorRunner { + storage_runner: TestMemoryStorageRunner, + attestation_status_runner: AttestationStatusRunner, +} + +impl TestExecutorRunner { + /// Runs the storage and the attestation status. + pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + scope::run!(ctx, |ctx, s| async { + s.spawn(self.storage_runner.run(ctx)); + s.spawn(self.attestation_status_runner.run(ctx)); + Ok(()) + }) + .await + } +} From 638b23e291a35fe9109c68d42f277903aaae0c42 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Wed, 31 Jul 2024 15:44:43 +0100 Subject: [PATCH 02/13] improve: AttestationStatusWatch::next_batch_to_attest non-optional (BFT-496) (#165) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ `AttestationStatusWatch` must be initialised with a `BatchNumber`, it cannot have `None` any more. `AttestationStatusRunner::new` was replaced with the `AttestationStatusRunner::init` method which asynchronously polls the API until the first value is returned, and then returns itself along with the `AttestationStatusWatch` it created. This can then be passed to the `Executor`, while the `AttestationStatusRunner::run` will keep the status up to date in the background. ## Why ❔ In the review of https://github.com/matter-labs/era-consensus/pull/161 it was observed that the `Executor` can wait until this data is available. In theory it is only unavailable if the main node API is down, in which case an external node couldn't pull Genesis either and would probably fail during startup, or if the Genesis itself is still under construction in the database, which is a transient state under which an external node as mentioned would not start, and apparently the main node doesn't need the `Executor` to get over it. By removing `None` as an option for `next_batch_to_attest` the state is easier to reason about. --- node/actors/executor/src/attestation.rs | 77 ++++++++++++------- node/actors/executor/src/tests.rs | 2 +- .../network/src/gossip/attestation_status.rs | 19 +++-- node/actors/network/src/gossip/mod.rs | 7 +- .../network/src/gossip/tests/fetch_blocks.rs | 1 + node/actors/network/src/testonly.rs | 5 +- node/tools/src/config.rs | 16 ++-- 7 files changed, 72 insertions(+), 55 deletions(-) diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index 621fa1c3..4d301136 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -58,12 +58,9 @@ impl AttesterRunner { self.status.mark_changed(); loop { - let Some(batch_number) = sync::changed(ctx, &mut self.status) + let batch_number = sync::changed(ctx, &mut self.status) .await? - .next_batch_to_attest - else { - continue; - }; + .next_batch_to_attest; tracing::info!(%batch_number, "attestation status"); @@ -139,48 +136,70 @@ pub struct AttestationStatusRunner { } impl AttestationStatusRunner { - /// Create a new runner to poll the main node. - pub fn new( - status: Arc, + /// Create a new [AttestationStatusWatch] and an [AttestationStatusRunner] to poll the main node. + /// + /// It polls the [AttestationStatusClient] until it returns a value to initialize the status with. + pub async fn init( + ctx: &ctx::Ctx, client: Box, poll_interval: time::Duration, - ) -> Self { - Self { - status, + ) -> ctx::OrCanceled<(Arc, Self)> { + let status = Arc::new(AttestationStatusWatch::new(attester::BatchNumber(0))); + let mut runner = Self { + status: status.clone(), client, poll_interval, - } + }; + runner.poll_until_some(ctx).await?; + Ok((status, runner)) } - /// Runner based on a [BatchStore]. - pub fn new_from_store( - status: Arc, + /// Initialize an [AttestationStatusWatch] based on a [BatchStore] and return it along with the [AttestationStatusRunner]. + pub async fn init_from_store( + ctx: &ctx::Ctx, store: Arc, poll_interval: time::Duration, - ) -> Self { - Self::new( - status, + ) -> ctx::OrCanceled<(Arc, Self)> { + Self::init( + ctx, Box::new(LocalAttestationStatusClient(store)), poll_interval, ) + .await } /// Run the poll loop. - pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + pub async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { + let _ = self.poll_forever(ctx).await; + Ok(()) + } + + /// Poll the client forever in a loop or until canceled. + async fn poll_forever(&mut self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { + loop { + self.poll_until_some(ctx).await?; + ctx.sleep(self.poll_interval).await?; + } + } + + /// Poll the client until some data is returned and write it into the status. + async fn poll_until_some(&mut self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { loop { match self.client.next_batch_to_attest(ctx).await { - Ok(Some(batch_number)) => { - self.status.update(batch_number).await; + Ok(Some(next_batch_to_attest)) => { + self.status.update(next_batch_to_attest).await; + } + Ok(None) => { + tracing::debug!("waiting for attestation status...") + } + Err(error) => { + tracing::error!( + ?error, + "failed to poll attestation status, retrying later..." + ) } - Ok(None) => tracing::debug!("waiting for attestation status..."), - Err(error) => tracing::error!( - ?error, - "failed to poll attestation status, retrying later..." - ), - } - if let Err(ctx::Canceled) = ctx.sleep(self.poll_interval).await { - return Ok(()); } + ctx.sleep(self.poll_interval).await?; } } } diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 6ce5c057..5ef9b12e 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -30,7 +30,7 @@ fn config(cfg: &network::Config) -> Config { /// The test executors below are not running with attesters, so we just create an [AttestationStatusWatch] /// that will never be updated. fn never_attest() -> Arc { - Arc::new(AttestationStatusWatch::default()) + Arc::new(AttestationStatusWatch::new(attester::BatchNumber(0))) } fn validator( diff --git a/node/actors/network/src/gossip/attestation_status.rs b/node/actors/network/src/gossip/attestation_status.rs index 5509bf81..b4e19f6d 100644 --- a/node/actors/network/src/gossip/attestation_status.rs +++ b/node/actors/network/src/gossip/attestation_status.rs @@ -10,9 +10,9 @@ use crate::watch::Watch; pub struct AttestationStatus { /// Next batch number where voting is expected. /// - /// Its value is `None` until the background process polling the main node - /// can establish a value to start from. - pub next_batch_to_attest: Option, + /// The node is expected to poll the main node during initialization until + /// the batch to start from is established. + pub next_batch_to_attest: attester::BatchNumber, } /// The subscription over the attestation status which voters can monitor for change. @@ -29,15 +29,14 @@ impl fmt::Debug for AttestationStatusWatch { } } -impl Default for AttestationStatusWatch { - fn default() -> Self { +impl AttestationStatusWatch { + /// Create a new watch going from a specific batch number. + pub fn new(next_batch_to_attest: attester::BatchNumber) -> Self { Self(Watch::new(AttestationStatus { - next_batch_to_attest: None, + next_batch_to_attest, })) } -} -impl AttestationStatusWatch { /// Subscribes to AttestationStatus updates. pub fn subscribe(&self) -> AttestationStatusReceiver { self.0.subscribe() @@ -47,10 +46,10 @@ impl AttestationStatusWatch { pub async fn update(&self, next_batch_to_attest: attester::BatchNumber) { let this = self.0.lock().await; this.send_if_modified(|status| { - if status.next_batch_to_attest == Some(next_batch_to_attest) { + if status.next_batch_to_attest == next_batch_to_attest { return false; } - status.next_batch_to_attest = Some(next_batch_to_attest); + status.next_batch_to_attest = next_batch_to_attest; true }); } diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 5da55cce..a6a4e4e1 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -171,12 +171,9 @@ impl Network { loop { // Wait until the status indicates that we're ready to sign the next batch. - let Some(next_batch_number) = sync::changed(ctx, &mut recv_status) + let next_batch_number = sync::changed(ctx, &mut recv_status) .await? - .next_batch_to_attest - else { - continue; - }; + .next_batch_to_attest; // Get rid of all previous votes. We don't expect this to go backwards without regenesis, which will involve a restart. self.batch_votes diff --git a/node/actors/network/src/gossip/tests/fetch_blocks.rs b/node/actors/network/src/gossip/tests/fetch_blocks.rs index c42e50a5..79a310f9 100644 --- a/node/actors/network/src/gossip/tests/fetch_blocks.rs +++ b/node/actors/network/src/gossip/tests/fetch_blocks.rs @@ -26,6 +26,7 @@ async fn test_simple() { scope::run!(ctx, |ctx, s| async { let store = TestMemoryStorage::new(ctx, &setup.genesis).await; s.spawn_bg(store.runner.run(ctx)); + let (_node, runner) = crate::testonly::Instance::new( cfg.clone(), store.blocks.clone(), diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index d8d8c8ed..c61b7658 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -17,7 +17,7 @@ use zksync_concurrency::{ ctx::{self, channel}, io, limiter, net, scope, sync, time, }; -use zksync_consensus_roles::{node, validator}; +use zksync_consensus_roles::{attester, node, validator}; use zksync_consensus_storage::{BatchStore, BlockStore}; use zksync_consensus_utils::pipe; @@ -199,7 +199,8 @@ impl Instance { ) -> (Self, InstanceRunner) { // Semantically we'd want this to be created at the same level as the stores, // but doing so would introduce a lot of extra cruft in setting up tests. - let attestation_status = Arc::new(AttestationStatusWatch::default()); + let attestation_status = + Arc::new(AttestationStatusWatch::new(attester::BatchNumber::default())); let (actor_pipe, dispatcher_pipe) = pipe::new(); let (net, net_runner) = Network::new( diff --git a/node/tools/src/config.rs b/node/tools/src/config.rs index f659bdb3..30eed712 100644 --- a/node/tools/src/config.rs +++ b/node/tools/src/config.rs @@ -7,14 +7,13 @@ use std::{ fs, io, net::SocketAddr, path::PathBuf, - sync::Arc, }; use tokio_rustls::rustls::pki_types::{CertificateDer, PrivateKeyDer}; use zksync_concurrency::{ctx, net, scope, time}; use zksync_consensus_bft as bft; use zksync_consensus_crypto::{read_optional_text, read_required_text, Text, TextFmt}; use zksync_consensus_executor::{self as executor, attestation::AttestationStatusRunner}; -use zksync_consensus_network::{gossip::AttestationStatusWatch, http}; +use zksync_consensus_network::http; use zksync_consensus_roles::{attester, node, validator}; use zksync_consensus_storage::testonly::{TestMemoryStorage, TestMemoryStorageRunner}; use zksync_consensus_utils::debug_page; @@ -265,12 +264,13 @@ impl Configs { let store = TestMemoryStorage::new(ctx, &self.app.genesis).await; // We don't have an API to poll in this setup, we can only create a local store based attestation client. - let attestation_status = Arc::new(AttestationStatusWatch::default()); - let attestation_status_runner = AttestationStatusRunner::new_from_store( - attestation_status.clone(), - store.batches.clone(), - time::Duration::seconds(1), - ); + let (attestation_status, attestation_status_runner) = + AttestationStatusRunner::init_from_store( + ctx, + store.batches.clone(), + time::Duration::seconds(1), + ) + .await?; let runner = TestExecutorRunner { storage_runner: store.runner, From aa29e87de0f87a55d2917d4e1e4de99d04364934 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Thu, 1 Aug 2024 01:46:05 +0100 Subject: [PATCH 03/13] fix: AttestationStatusRunner::poll_until_some return after success (BFT-496) (#166) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Fixes `AttestationStatusRunner::poll_until_some` to return after a successful poll. Originally it returned a value, then decided to write it into the field and forgot to add a return statement. ## Why ❔ Tests in https://github.com/matter-labs/zksync-era/pull/2544 time out because the initialisation never completes and the `Executor` is never started. --- node/actors/executor/src/attestation.rs | 3 +- node/actors/executor/src/tests.rs | 66 ++++++++++++++++++++++++- 2 files changed, 67 insertions(+), 2 deletions(-) diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index 4d301136..db55131f 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -188,9 +188,10 @@ impl AttestationStatusRunner { match self.client.next_batch_to_attest(ctx).await { Ok(Some(next_batch_to_attest)) => { self.status.update(next_batch_to_attest).await; + return Ok(()); } Ok(None) => { - tracing::debug!("waiting for attestation status...") + tracing::info!("waiting for attestation status...") } Err(error) => { tracing::error!( diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 5ef9b12e..92a09462 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -1,8 +1,11 @@ //! High-level tests for `Executor`. +use std::sync::atomic::AtomicU64; + use super::*; +use attestation::{AttestationStatusClient, AttestationStatusRunner}; use rand::Rng as _; use tracing::Instrument as _; -use zksync_concurrency::testonly::abort_on_panic; +use zksync_concurrency::{sync, testonly::abort_on_panic}; use zksync_consensus_bft as bft; use zksync_consensus_network::testonly::{new_configs, new_fullnode}; use zksync_consensus_roles::validator::{testonly::Setup, BlockNumber}; @@ -312,3 +315,64 @@ async fn test_validator_syncing_from_fullnode() { .await .unwrap(); } + +/// Test that the AttestationStatusRunner initialises and then polls the status. +#[tokio::test] +async fn test_attestation_status_runner() { + abort_on_panic(); + let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(5)); + let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + + #[derive(Default)] + struct MockAttestationStatus { + batch_number: AtomicU64, + } + + #[async_trait::async_trait] + impl AttestationStatusClient for MockAttestationStatus { + async fn next_batch_to_attest( + &self, + _ctx: &ctx::Ctx, + ) -> ctx::Result> { + let curr = self + .batch_number + .fetch_add(1u64, std::sync::atomic::Ordering::Relaxed); + if curr == 0 { + // Return None initially to see that the runner will deal with it. + Ok(None) + } else { + // The first actual result will be 1 on the 2nd poll. + Ok(Some(attester::BatchNumber(curr))) + } + } + } + + scope::run!(ctx, |ctx, s| async { + let (status, runner) = AttestationStatusRunner::init( + ctx, + Box::new(MockAttestationStatus::default()), + time::Duration::milliseconds(100), + ) + .await + .unwrap(); + + let mut recv_status = status.subscribe(); + recv_status.mark_changed(); + + // Check that the value has been initialised to a non-default value. + { + let status = sync::changed(ctx, &mut recv_status).await?; + assert_eq!(status.next_batch_to_attest.0, 1); + } + // Now start polling for new values. + s.spawn_bg(runner.run(ctx)); + // Check that polling sets the value. + { + let status = sync::changed(ctx, &mut recv_status).await?; + assert_eq!(status.next_batch_to_attest.0, 2); + } + Ok(()) + }) + .await + .unwrap(); +} From ecbabff2d0de26470da639622f7563cf84c79a1b Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Thu, 1 Aug 2024 14:24:35 +0100 Subject: [PATCH 04/13] feat: Add GenesisHash to AttestationStatusClient and reject unexpected updates (BFT-496) (#167) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - [x] Add the initial `GenesisHash` to `AttestationStatus` and change `AttestationStatusWatch::update` to take `GenesisHash` and return an error if the latest from the client indicates a change in genesis. This stops the runner and thus should stop the node itself when the error bubbles up. On external nodes this is redundant with the routine that monitors the main node API for changes. On the main node I shouldn't happen. - [x] Change `next_batch_to_attest() -> Option` into `attestation_status() -> Option` on `AttestationStatusClient`, so it now includes the `GenesisHash` as well. - [x] `LocalAttestationStatusClient` returns the `GenesisHash` it was started with ## Why ❔ This came out of a discussion here: https://github.com/matter-labs/zksync-era/pull/2544#discussion_r1699598459 Notably the `ConsensusDal::attestation_status()` now returns the `GenesisHash`, which I thought was only to signal through the API from the main node to the external nodes that a reorg has happened (which they would eventually learn anyway through polling the genesis endpoint, causing them to restart), and they should not attest for batches if they are on a different fork. The comment raised the issue that on the main node I discarded the `genesis` field from the response because I assumed it can only be the same as the one we started the `Executor` with, and second guessing it in the `BatchStore` would be awkward: the original genesis wasn't available to check against (it's cached in the `BlockStore`) and running a query to fetch it (even if the `PersistentBatchStore` supported it) would just compare it to what it already is. The way I handled this mismatch for external nodes was to just stop updating the status by returning `None` and wait for the restart, treating it like any other transient RPC error, it was just logged. It does make sense though to elevate it higher and be able to stop the node if it's in an inconsistent state. Now it's part of the `AttestationStatusWatch` because that is what we consider to be our interface with the node, (and not the `AttestationStatusRunner` which is a convenience construct). ### Reorgs without restart? If an inconsistency happened on the main node it wouldn't necessarily have to be fatal: if the genesis was allowed to change, and the components such as the `BlockStore` were able to handle it at all, then ostensibly the `AttestationStatus*` stuff could recover as well, for example by resetting the `BatchVotes` if they see a change in the `genesis`. The problem currently is that they might have already received and discarded votes for the new fork, which would not be gossiped again until clients are reconnected. Apparently we don't plan to handle regenesis on the fly, so `AttestationStatus::genesis` is only present to prevent any attempt at changing it by _causing_ a restart instead. ### Atomicity In the first version of this PR the `BatchStore` and `PersistentBatchStore` were also changed to have an `attestation_status()` method that returned a `GenesisHash` along with the `BatchNumber`. The only way I could make sense of this was if 1) changes in genesis while running the main node were allowed and 2) they could happen between calls to `BlockStore::genesis()` and `BatchStore::next_batch_to_attest()`, since those are not atomic methods. We discussed that this will not be supported, the `Genesis` cached in `BlockStore` will not change. Since we only start the `Executor` and the `AttestationStatusRunner` [after regenesis](https://github.com/matter-labs/zksync-era/blob/1d206c0af8f28eb00eb1498d6f2cdbb45ffef72a/core/node/consensus/src/mn.rs#L39) in `zksync-era`, we don't have to worry about this changing. In that light it would be counter intuitive to return the genesis hash from `BatchStore`. (In the future we could consider using Software Transactional Memory to have an in-memory representation of the state where we can do consistent read and writes between multiple Watch-like references. The DAL is capable of transactional reads, and it would be nice if we could combine reading for example here the genesis and the last/next batch and not have to worry about having to compare hashes, when all of these are local reads.) --- node/actors/executor/src/attestation.rs | 65 +++++++++++------- node/actors/executor/src/tests.rs | 67 ++++++++++++++----- .../network/src/gossip/attestation_status.rs | 46 ++++++++++++- node/actors/network/src/gossip/mod.rs | 4 +- node/actors/network/src/testonly.rs | 18 +++-- node/tools/src/config.rs | 1 + 6 files changed, 152 insertions(+), 49 deletions(-) diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index db55131f..c949b9bd 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -5,7 +5,7 @@ use anyhow::Context; use std::sync::Arc; use zksync_concurrency::{ctx, sync, time}; use zksync_consensus_network::gossip::{ - AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, + AttestationStatus, AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, }; use zksync_consensus_roles::attester; use zksync_consensus_storage::{BatchStore, BlockStore}; @@ -120,10 +120,10 @@ pub trait AttestationStatusClient: 'static + Send + Sync { /// /// The API might return an error while genesis is being created, which we represent with `None` /// here and mean that we'll have to try again later. - async fn next_batch_to_attest( - &self, - ctx: &ctx::Ctx, - ) -> ctx::Result>; + /// + /// The genesis hash is returned along with the new batch number to facilitate detecting reorgs + /// on the main node as soon as possible and prevent inconsistent state from entering the system. + async fn attestation_status(&self, ctx: &ctx::Ctx) -> ctx::Result>; } /// Use an [AttestationStatusClient] to periodically poll the main node and update the [AttestationStatusWatch]. @@ -143,8 +143,12 @@ impl AttestationStatusRunner { ctx: &ctx::Ctx, client: Box, poll_interval: time::Duration, - ) -> ctx::OrCanceled<(Arc, Self)> { - let status = Arc::new(AttestationStatusWatch::new(attester::BatchNumber(0))); + genesis: attester::GenesisHash, + ) -> ctx::Result<(Arc, Self)> { + let status = Arc::new(AttestationStatusWatch::new( + genesis, + attester::BatchNumber::default(), + )); let mut runner = Self { status: status.clone(), client, @@ -157,25 +161,32 @@ impl AttestationStatusRunner { /// Initialize an [AttestationStatusWatch] based on a [BatchStore] and return it along with the [AttestationStatusRunner]. pub async fn init_from_store( ctx: &ctx::Ctx, - store: Arc, + batch_store: Arc, poll_interval: time::Duration, - ) -> ctx::OrCanceled<(Arc, Self)> { + genesis: attester::GenesisHash, + ) -> ctx::Result<(Arc, Self)> { Self::init( ctx, - Box::new(LocalAttestationStatusClient(store)), + Box::new(LocalAttestationStatusClient { + genesis, + batch_store, + }), poll_interval, + genesis, ) .await } /// Run the poll loop. pub async fn run(mut self, ctx: &ctx::Ctx) -> anyhow::Result<()> { - let _ = self.poll_forever(ctx).await; - Ok(()) + match self.poll_forever(ctx).await { + Ok(()) | Err(ctx::Error::Canceled(_)) => Ok(()), + Err(ctx::Error::Internal(err)) => Err(err), + } } /// Poll the client forever in a loop or until canceled. - async fn poll_forever(&mut self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { + async fn poll_forever(&mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { loop { self.poll_until_some(ctx).await?; ctx.sleep(self.poll_interval).await?; @@ -183,11 +194,13 @@ impl AttestationStatusRunner { } /// Poll the client until some data is returned and write it into the status. - async fn poll_until_some(&mut self, ctx: &ctx::Ctx) -> ctx::OrCanceled<()> { + async fn poll_until_some(&mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { loop { - match self.client.next_batch_to_attest(ctx).await { - Ok(Some(next_batch_to_attest)) => { - self.status.update(next_batch_to_attest).await; + match self.client.attestation_status(ctx).await { + Ok(Some(status)) => { + self.status + .update(status.genesis, status.next_batch_to_attest) + .await?; return Ok(()); } Ok(None) => { @@ -206,14 +219,20 @@ impl AttestationStatusRunner { } /// Implement the attestation status for the main node by returning the next to vote on from the [BatchStore]. -struct LocalAttestationStatusClient(Arc); +struct LocalAttestationStatusClient { + /// We don't expect the genesis to change while the main node is running, + /// so we can just cache the genesis hash and return it for every request. + genesis: attester::GenesisHash, + batch_store: Arc, +} #[async_trait::async_trait] impl AttestationStatusClient for LocalAttestationStatusClient { - async fn next_batch_to_attest( - &self, - ctx: &ctx::Ctx, - ) -> ctx::Result> { - self.0.next_batch_to_attest(ctx).await + async fn attestation_status(&self, ctx: &ctx::Ctx) -> ctx::Result> { + let batch_number = self.batch_store.next_batch_to_attest(ctx).await?; + Ok(batch_number.map(|n| AttestationStatus { + genesis: self.genesis, + next_batch_to_attest: n, + })) } } diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 92a09462..287a2717 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -1,5 +1,5 @@ //! High-level tests for `Executor`. -use std::sync::atomic::AtomicU64; +use std::sync::{atomic::AtomicU64, Mutex}; use super::*; use attestation::{AttestationStatusClient, AttestationStatusRunner}; @@ -7,7 +7,10 @@ use rand::Rng as _; use tracing::Instrument as _; use zksync_concurrency::{sync, testonly::abort_on_panic}; use zksync_consensus_bft as bft; -use zksync_consensus_network::testonly::{new_configs, new_fullnode}; +use zksync_consensus_network::{ + gossip::AttestationStatus, + testonly::{new_configs, new_fullnode}, +}; use zksync_consensus_roles::validator::{testonly::Setup, BlockNumber}; use zksync_consensus_storage::{ testonly::{in_memory, TestMemoryStorage}, @@ -32,8 +35,11 @@ fn config(cfg: &network::Config) -> Config { /// The test executors below are not running with attesters, so we just create an [AttestationStatusWatch] /// that will never be updated. -fn never_attest() -> Arc { - Arc::new(AttestationStatusWatch::new(attester::BatchNumber(0))) +fn never_attest(genesis: &validator::Genesis) -> Arc { + Arc::new(AttestationStatusWatch::new( + genesis.hash(), + attester::BatchNumber::default(), + )) } fn validator( @@ -42,6 +48,7 @@ fn validator( batch_store: Arc, replica_store: impl ReplicaStore, ) -> Executor { + let attestation_status = never_attest(block_store.genesis()); Executor { config: config(cfg), block_store, @@ -52,7 +59,7 @@ fn validator( payload_manager: Box::new(bft::testonly::RandomPayload(1000)), }), attester: None, - attestation_status: never_attest(), + attestation_status, } } @@ -61,13 +68,14 @@ fn fullnode( block_store: Arc, batch_store: Arc, ) -> Executor { + let attestation_status = never_attest(block_store.genesis()); Executor { config: config(cfg), block_store, batch_store, validator: None, attester: None, - attestation_status: never_attest(), + attestation_status, } } @@ -322,18 +330,22 @@ async fn test_attestation_status_runner() { abort_on_panic(); let _guard = zksync_concurrency::testonly::set_timeout(time::Duration::seconds(5)); let ctx = &ctx::test_root(&ctx::AffineClock::new(10.0)); + let rng = &mut ctx.rng(); + + let genesis: attester::GenesisHash = rng.gen(); - #[derive(Default)] + #[derive(Clone)] struct MockAttestationStatus { - batch_number: AtomicU64, + genesis: Arc>, + batch_number: Arc, } #[async_trait::async_trait] impl AttestationStatusClient for MockAttestationStatus { - async fn next_batch_to_attest( + async fn attestation_status( &self, _ctx: &ctx::Ctx, - ) -> ctx::Result> { + ) -> ctx::Result> { let curr = self .batch_number .fetch_add(1u64, std::sync::atomic::Ordering::Relaxed); @@ -342,16 +354,25 @@ async fn test_attestation_status_runner() { Ok(None) } else { // The first actual result will be 1 on the 2nd poll. - Ok(Some(attester::BatchNumber(curr))) + let status = AttestationStatus { + genesis: *self.genesis.lock().unwrap(), + next_batch_to_attest: attester::BatchNumber(curr), + }; + Ok(Some(status)) } } } - scope::run!(ctx, |ctx, s| async { + let res = scope::run!(ctx, |ctx, s| async { + let client = MockAttestationStatus { + genesis: Arc::new(Mutex::new(genesis)), + batch_number: Arc::new(AtomicU64::default()), + }; let (status, runner) = AttestationStatusRunner::init( ctx, - Box::new(MockAttestationStatus::default()), + Box::new(client.clone()), time::Duration::milliseconds(100), + genesis, ) .await .unwrap(); @@ -364,15 +385,27 @@ async fn test_attestation_status_runner() { let status = sync::changed(ctx, &mut recv_status).await?; assert_eq!(status.next_batch_to_attest.0, 1); } - // Now start polling for new values. - s.spawn_bg(runner.run(ctx)); + // Now start polling for new values. Starting in the foreground because we want it to fail in the end. + s.spawn(runner.run(ctx)); // Check that polling sets the value. { let status = sync::changed(ctx, &mut recv_status).await?; assert_eq!(status.next_batch_to_attest.0, 2); } + // Change the genesis returned by the client. It should cause the scope to fail. + { + let mut genesis = client.genesis.lock().unwrap(); + *genesis = rng.gen(); + } Ok(()) }) - .await - .unwrap(); + .await; + + match res { + Ok(()) => panic!("expected to fail when the genesis changed"), + Err(e) => assert!( + e.to_string().contains("genesis changed"), + "only expect failures due to genesis change" + ), + } } diff --git a/node/actors/network/src/gossip/attestation_status.rs b/node/actors/network/src/gossip/attestation_status.rs index b4e19f6d..514bb84b 100644 --- a/node/actors/network/src/gossip/attestation_status.rs +++ b/node/actors/network/src/gossip/attestation_status.rs @@ -6,13 +6,20 @@ use zksync_consensus_roles::attester; use crate::watch::Watch; /// Coordinate the attestation by showing the status as seen by the main node. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct AttestationStatus { /// Next batch number where voting is expected. /// /// The node is expected to poll the main node during initialization until /// the batch to start from is established. pub next_batch_to_attest: attester::BatchNumber, + /// The hash of the genesis of the chain to which the L1 batches belong. + /// + /// We don't expect to handle a regenesis on the fly without restarting the + /// node, so this value is not expected to change; it's here only to stop + /// any attempt at updating the status with a batch number that refers + /// to a different fork. + pub genesis: attester::GenesisHash, } /// The subscription over the attestation status which voters can monitor for change. @@ -31,8 +38,12 @@ impl fmt::Debug for AttestationStatusWatch { impl AttestationStatusWatch { /// Create a new watch going from a specific batch number. - pub fn new(next_batch_to_attest: attester::BatchNumber) -> Self { + pub fn new( + genesis: attester::GenesisHash, + next_batch_to_attest: attester::BatchNumber, + ) -> Self { Self(Watch::new(AttestationStatus { + genesis, next_batch_to_attest, })) } @@ -43,8 +54,36 @@ impl AttestationStatusWatch { } /// Set the next batch number to attest on and notify subscribers it changed. - pub async fn update(&self, next_batch_to_attest: attester::BatchNumber) { + /// + /// Fails if the genesis we want to update to is not the same as the watch was started with, + /// because the rest of the system is not expected to be able to handle reorgs without a + /// restart of the node. + pub async fn update( + &self, + genesis: attester::GenesisHash, + next_batch_to_attest: attester::BatchNumber, + ) -> anyhow::Result<()> { let this = self.0.lock().await; + { + let status = this.borrow(); + anyhow::ensure!( + status.genesis == genesis, + "the attestation status genesis changed: {:?} -> {:?}", + status.genesis, + genesis + ); + // The next batch to attest moving backwards could cause the voting process + // to get stuck due to the way gossiping works and the BatchVotes discards + // votes below the expected minimum: even if we clear the votes, we might + // not get them again from any peer. By returning an error we can cause + // the node to be restarted and connections re-established for fresh gossip. + anyhow::ensure!( + status.next_batch_to_attest <= next_batch_to_attest, + "next batch to attest moved backwards: {} -> {}", + status.next_batch_to_attest, + next_batch_to_attest + ); + } this.send_if_modified(|status| { if status.next_batch_to_attest == next_batch_to_attest { return false; @@ -52,5 +91,6 @@ impl AttestationStatusWatch { status.next_batch_to_attest = next_batch_to_attest; true }); + Ok(()) } } diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index a6a4e4e1..90e5a1d5 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -12,7 +12,9 @@ //! Static connections constitute a rigid "backbone" of the gossip network, which is insensitive to //! eclipse attack. Dynamic connections are supposed to improve the properties of the gossip //! network graph (minimize its diameter, increase connectedness). -pub use self::attestation_status::{AttestationStatusReceiver, AttestationStatusWatch}; +pub use self::attestation_status::{ + AttestationStatus, AttestationStatusReceiver, AttestationStatusWatch, +}; pub use self::batch_votes::BatchVotesPublisher; use self::batch_votes::BatchVotesWatch; use crate::{gossip::ValidatorAddrsWatch, io, pool::PoolWatch, Config, MeteredStreamStats}; diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index c61b7658..00de2120 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -162,6 +162,7 @@ pub fn new_fullnode(rng: &mut impl Rng, peer: &Config) -> Config { pub struct InstanceRunner { net_runner: Runner, attestation_status: Arc, + block_store: Arc, batch_store: Arc, terminate: channel::Receiver<()>, } @@ -172,9 +173,13 @@ impl InstanceRunner { scope::run!(ctx, |ctx, s| async { s.spawn_bg(self.net_runner.run(ctx)); s.spawn_bg(async { + let genesis = self.block_store.genesis().hash(); loop { - if let Ok(Some(n)) = self.batch_store.next_batch_to_attest(ctx).await { - self.attestation_status.update(n).await; + if let Ok(Some(batch_number)) = self.batch_store.next_batch_to_attest(ctx).await + { + self.attestation_status + .update(genesis, batch_number) + .await?; } if ctx.sleep(time::Duration::seconds(1)).await.is_err() { return Ok(()); @@ -199,13 +204,15 @@ impl Instance { ) -> (Self, InstanceRunner) { // Semantically we'd want this to be created at the same level as the stores, // but doing so would introduce a lot of extra cruft in setting up tests. - let attestation_status = - Arc::new(AttestationStatusWatch::new(attester::BatchNumber::default())); + let attestation_status = Arc::new(AttestationStatusWatch::new( + block_store.genesis().hash(), + attester::BatchNumber::default(), + )); let (actor_pipe, dispatcher_pipe) = pipe::new(); let (net, net_runner) = Network::new( cfg, - block_store, + block_store.clone(), batch_store.clone(), actor_pipe, attestation_status.clone(), @@ -220,6 +227,7 @@ impl Instance { InstanceRunner { net_runner, attestation_status, + block_store, batch_store, terminate: terminate_recv, }, diff --git a/node/tools/src/config.rs b/node/tools/src/config.rs index 30eed712..97d49fbf 100644 --- a/node/tools/src/config.rs +++ b/node/tools/src/config.rs @@ -269,6 +269,7 @@ impl Configs { ctx, store.batches.clone(), time::Duration::seconds(1), + self.app.genesis.hash(), ) .await?; From 0b424a5548dc04f82738ed30774d95d46bd9d7f2 Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Thu, 1 Aug 2024 17:28:51 +0200 Subject: [PATCH 05/13] 0.1.0-rc.5 (#168) --- node/Cargo.lock | 24 ++++++++++++------------ node/Cargo.toml | 24 ++++++++++++------------ 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index 3ea3e4a4..677ef3a3 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3305,7 +3305,7 @@ dependencies = [ [[package]] name = "tester" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "clap", @@ -4059,7 +4059,7 @@ dependencies = [ [[package]] name = "zksync_concurrency" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "assert_matches", @@ -4077,7 +4077,7 @@ dependencies = [ [[package]] name = "zksync_consensus_bft" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "assert_matches", @@ -4101,7 +4101,7 @@ dependencies = [ [[package]] name = "zksync_consensus_crypto" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "blst", @@ -4124,7 +4124,7 @@ dependencies = [ [[package]] name = "zksync_consensus_executor" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "async-trait", @@ -4145,7 +4145,7 @@ dependencies = [ [[package]] name = "zksync_consensus_network" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "assert_matches", @@ -4181,7 +4181,7 @@ dependencies = [ [[package]] name = "zksync_consensus_roles" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "assert_matches", @@ -4202,7 +4202,7 @@ dependencies = [ [[package]] name = "zksync_consensus_storage" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "assert_matches", @@ -4224,7 +4224,7 @@ dependencies = [ [[package]] name = "zksync_consensus_tools" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "async-trait", @@ -4259,7 +4259,7 @@ dependencies = [ [[package]] name = "zksync_consensus_utils" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "rand 0.8.5", @@ -4269,7 +4269,7 @@ dependencies = [ [[package]] name = "zksync_protobuf" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "bit-vec", @@ -4291,7 +4291,7 @@ dependencies = [ [[package]] name = "zksync_protobuf_build" -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" dependencies = [ "anyhow", "heck", diff --git a/node/Cargo.toml b/node/Cargo.toml index 721ac98b..c5904c27 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -22,23 +22,23 @@ homepage = "https://matter-labs.io/" repository = "https://github.com/matter-labs/era-consensus" license = "MIT OR Apache-2.0" keywords = ["blockchain", "zksync"] -version = "0.1.0-rc.4" +version = "0.1.0-rc.5" [workspace.dependencies] # Crates from this repo. -zksync_consensus_bft = { version = "=0.1.0-rc.4", path = "actors/bft" } -zksync_consensus_crypto = { version = "=0.1.0-rc.4", path = "libs/crypto" } -zksync_consensus_executor = { version = "=0.1.0-rc.4", path = "actors/executor" } -zksync_consensus_network = { version = "=0.1.0-rc.4", path = "actors/network" } -zksync_consensus_roles = { version = "=0.1.0-rc.4", path = "libs/roles" } -zksync_consensus_storage = { version = "=0.1.0-rc.4", path = "libs/storage" } -zksync_consensus_tools = { version = "=0.1.0-rc.4", path = "tools" } -zksync_consensus_utils = { version = "=0.1.0-rc.4", path = "libs/utils" } +zksync_consensus_bft = { version = "=0.1.0-rc.5", path = "actors/bft" } +zksync_consensus_crypto = { version = "=0.1.0-rc.5", path = "libs/crypto" } +zksync_consensus_executor = { version = "=0.1.0-rc.5", path = "actors/executor" } +zksync_consensus_network = { version = "=0.1.0-rc.5", path = "actors/network" } +zksync_consensus_roles = { version = "=0.1.0-rc.5", path = "libs/roles" } +zksync_consensus_storage = { version = "=0.1.0-rc.5", path = "libs/storage" } +zksync_consensus_tools = { version = "=0.1.0-rc.5", path = "tools" } +zksync_consensus_utils = { version = "=0.1.0-rc.5", path = "libs/utils" } # Crates from this repo that might become independent in the future. -zksync_concurrency = { version = "=0.1.0-rc.4", path = "libs/concurrency" } -zksync_protobuf = { version = "=0.1.0-rc.4", path = "libs/protobuf" } -zksync_protobuf_build = { version = "=0.1.0-rc.4", path = "libs/protobuf_build" } +zksync_concurrency = { version = "=0.1.0-rc.5", path = "libs/concurrency" } +zksync_protobuf = { version = "=0.1.0-rc.5", path = "libs/protobuf" } +zksync_protobuf_build = { version = "=0.1.0-rc.5", path = "libs/protobuf_build" } # Crates from Matter Labs. pairing = { package = "pairing_ce", version = "=0.28.6" } From a004a379ae603752f946638d7c702677be9416bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bruno=20Fran=C3=A7a?= Date: Fri, 2 Aug 2024 14:54:38 +0100 Subject: [PATCH 06/13] Fixed cargo deny config (#173) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Changes the config of cargo deny because a breaking version of deny was published. --- node/Cargo.lock | 572 ++++++++++++++++++++++-------------------------- node/deny.toml | 28 +-- 2 files changed, 271 insertions(+), 329 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index 677ef3a3..cd86fd1b 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -62,7 +62,7 @@ dependencies = [ "getrandom", "once_cell", "version_check", - "zerocopy", + "zerocopy 0.7.35", ] [[package]] @@ -88,9 +88,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstream" -version = "0.6.14" +version = "0.6.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "418c75fa768af9c03be99d17643f93f79bbba589895012a80e3452a19ddda15b" +checksum = "64e15c1ab1f89faffbf04a634d5e1962e9074f2741eef6d97f3c4e322426d526" dependencies = [ "anstyle", "anstyle-parse", @@ -103,36 +103,36 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.7" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" +checksum = "1bec1de6f59aedf83baf9ff929c98f2ad654b97c9510f4e70cf6f661d49fd5b1" [[package]] name = "anstyle-parse" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c03a11a9034d92058ceb6ee011ce58af4a9bf61491aa7e1e59ecd24bd40d22d4" +checksum = "eb47de1e80c2b463c735db5b217a0ddc39d612e7ac9e2e96a5aed1f57616c1cb" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad186efb764318d35165f1758e7dcef3b10628e26d41a44bc5550652e6804391" +checksum = "6d36fc52c7f6c869915e99412912f22093507da8d9e942ceaf66fe4b7c14422a" dependencies = [ - "windows-sys 0.52.0", + "windows-sys", ] [[package]] name = "anstyle-wincon" -version = "3.0.3" +version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61a38449feb7068f52bb06c12759005cf459ee52bb4adc1d5a7c4322d716fb19" +checksum = "5bf74e1b6e971609db8ca7a9ce79fd5768ab6ae46441c572e46cf596f59e57f8" dependencies = [ "anstyle", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -178,18 +178,18 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -206,9 +206,9 @@ checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" [[package]] name = "aws-lc-rs" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8a47f2fb521b70c11ce7369a6c5fa4bd6af7e5d62ec06303875bafe7c6ba245" +checksum = "4ae74d9bd0a7530e8afd1770739ad34b36838829d6ad61818f9230f683f5ad77" dependencies = [ "aws-lc-sys", "mirai-annotations", @@ -218,9 +218,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.19.0" +version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2927c7af777b460b7ccd95f8b67acd7b4c04ec8896bf0c8e80ba30523cffc057" +checksum = "0f0e249228c6ad2d240c2dc94b714d711629d52bad946075d8e9b2f5391f0703" dependencies = [ "bindgen 0.69.4", "cc", @@ -308,7 +308,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -317,7 +317,7 @@ version = "0.69.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a00dc851838a2120612785d195287475a3ac45514741da670b735818822129a0" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cexpr", "clang-sys", "itertools 0.12.1", @@ -330,7 +330,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.66", + "syn 2.0.72", "which", ] @@ -348,9 +348,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" [[package]] name = "bitmaps" @@ -381,9 +381,9 @@ dependencies = [ [[package]] name = "blst" -version = "0.3.12" +version = "0.3.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62dc83a094a71d43eeadd254b1ec2d24cb6a0bb6cadce00df51f0db594711a32" +checksum = "4378725facc195f1a538864863f6de233b500a8862747e7f165078a419d5e874" dependencies = [ "cc", "glob", @@ -393,9 +393,9 @@ dependencies = [ [[package]] name = "build_html" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3108fe6fe7ac796fb7625bdde8fa2b67b5a7731496251ca57c7b8cadd78a16a1" +checksum = "225eb82ce9e70dcc0cfa6e404d0f353326b6e163bf500ec4711cec317d11935c" [[package]] name = "bumpalo" @@ -411,9 +411,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.1" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12916984aab3fa6e39d655a33e09c0071eb36d6ab3aea5c2d78551f1df6d952" +checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" [[package]] name = "bytesize" @@ -440,13 +440,12 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.99" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" +checksum = "26a5c3fd7bfa1ce3897a3a3501d362b2d87b7f2583ebcb4a949ec25911025cbc" dependencies = [ "jobserver", "libc", - "once_cell", ] [[package]] @@ -555,9 +554,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.7" +version = "4.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5db83dced34638ad474f39f250d7fea9598bdd239eaced1bdf45d597da0f433f" +checksum = "0fbb260a053428790f3de475e304ff84cdbc4face759ea7a3e64c1edd938a7fc" dependencies = [ "clap_builder", "clap_derive", @@ -565,9 +564,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.7" +version = "4.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7e204572485eb3fbf28f871612191521df159bc3e15a9f5064c66dba3a8c05f" +checksum = "64b17d7ea74e9f833c7dbf2cbe4fb12ff26783eda4782a8975b72f895c9b4d99" dependencies = [ "anstream", "anstyle", @@ -577,21 +576,21 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.5" +version = "4.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c780290ccf4fb26629baa7a1081e68ced113f1d3ec302fa5948f1c381ebf06c6" +checksum = "501d359d5f3dcaf6ecdeee48833ae73ec6e42723a1e52419c79abf9507eec0a0" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "clap_lex" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" +checksum = "1462739cb27611015575c0c11df5df7601141071f07518d56fcc1be504cbec97" [[package]] name = "cmake" @@ -604,9 +603,9 @@ dependencies = [ [[package]] name = "colorchoice" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b6a852b24ab71dffc585bcb46eaf7959d175cb865a7152e35b348d1b2960422" +checksum = "d3fd119d74b830634cea2a0f58bbd0d54540518a14397557951e79340abc28c0" [[package]] name = "combine" @@ -787,14 +786,14 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "darling" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83b2eb4d90d12bdda5ed17de686c2acb4c57914f8f921b8da7e112b5a36f3fe1" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" dependencies = [ "darling_core", "darling_macro", @@ -802,27 +801,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "622687fe0bac72a04e5599029151f5796111b90f1baaa9b544d807a5e31cd120" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "darling_macro" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "733cabb43482b1a1b53eee8583c2b9e8684d592215ea83efd305dd31bc2f0178" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -932,9 +931,9 @@ dependencies = [ [[package]] name = "either" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dca9240753cf90908d7e4aac30f630662b02aebaa1b58a3cadabdb23385b58b" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" [[package]] name = "elliptic-curve" @@ -977,7 +976,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -1128,7 +1127,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -1288,7 +1287,7 @@ version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" dependencies = [ - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -1326,9 +1325,9 @@ dependencies = [ [[package]] name = "http-body" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cac85db508abc24a2e48553ba12a996e87244a0395ce011e62b37158745d643" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" dependencies = [ "bytes", "http 1.1.0", @@ -1343,7 +1342,7 @@ dependencies = [ "bytes", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "pin-project-lite", ] @@ -1361,9 +1360,9 @@ checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "hyper" -version = "0.14.29" +version = "0.14.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f361cde2f109281a220d4307746cdfd5ee3f410da58a70377762396775634b33" +checksum = "a152ddd61dfaec7273fe8419ab357f33aee0d914c5f4efbf0d96fa749eea5ec9" dependencies = [ "bytes", "futures-channel", @@ -1384,16 +1383,16 @@ dependencies = [ [[package]] name = "hyper" -version = "1.3.1" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe575dd17d0862a9a33781c8c4696a55c320909004a67a00fb286ba8b1bc496d" +checksum = "50dfd22e0e76d0f662d429a5f80fcaf3855009297eab6a0a9f8543834744ba05" dependencies = [ "bytes", "futures-channel", "futures-util", "h2", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "httparse", "httpdate", "itoa", @@ -1411,7 +1410,7 @@ checksum = "5ee4be2c948921a1a5320b629c4193916ed787a7f7f293fd3f7f5a6c9de74155" dependencies = [ "futures-util", "http 1.1.0", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "log", "rustls", @@ -1428,7 +1427,7 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3203a961e5c83b6f5498933e78b6b263e208c197b63e9c6c53cc82ffd3f63793" dependencies = [ - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "pin-project-lite", "tokio", @@ -1437,16 +1436,16 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b875924a60b96e5d7b9ae7b066540b1dd1cbd90d1828f54c92e02a283351c56" +checksum = "3ab92f4f49ee4fb4f997c784b7a2e0fa70050211e0b6a287f898c3c9785ca956" dependencies = [ "bytes", "futures-channel", "futures-util", "http 1.1.0", - "http-body 1.0.0", - "hyper 1.3.1", + "http-body 1.0.1", + "hyper 1.4.1", "pin-project-lite", "socket2", "tokio", @@ -1487,9 +1486,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.6" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +checksum = "de3fc2e30ba82dd1b3911c8de1ffc143c74a914a14e99514d7637e3099df5ea0" dependencies = [ "equivalent", "hashbrown", @@ -1521,14 +1520,14 @@ checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" dependencies = [ "hermit-abi", "libc", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] name = "is_terminal_polyfill" -version = "1.70.0" +version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8478577c03552c21db0e2724ffb8986a5ce7af88107e6be5d2ee6e158c12800" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" [[package]] name = "itertools" @@ -1576,9 +1575,9 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" dependencies = [ "libc", ] @@ -1620,9 +1619,9 @@ dependencies = [ [[package]] name = "jsonrpsee" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95a130d27083a4001b7b2d72a19f08786299550f76c9bd5307498dce2c2b20fa" +checksum = "62b089779ad7f80768693755a031cc14a7766aba707cbe886674e3f79e9b7e47" dependencies = [ "jsonrpsee-core", "jsonrpsee-http-client", @@ -1633,9 +1632,9 @@ dependencies = [ [[package]] name = "jsonrpsee-core" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21545a9445fbd582840ff5160a9a3e12b8e6da582151cdb07bde9a1970ba3a24" +checksum = "79712302e737d23ca0daa178e752c9334846b08321d439fd89af9a384f8c830b" dependencies = [ "anyhow", "async-trait", @@ -1643,7 +1642,7 @@ dependencies = [ "bytes", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", "jsonrpsee-types", "parking_lot", @@ -1658,14 +1657,14 @@ dependencies = [ [[package]] name = "jsonrpsee-http-client" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb25cab482c8512c4f3323a5c90b95a3b8f7c90681a87bf7a68b942d52f08933" +checksum = "2d90064e04fb9d7282b1c71044ea94d0bbc6eff5621c66f1a0bce9e9de7cf3ac" dependencies = [ "async-trait", "base64 0.22.1", - "http-body 1.0.0", - "hyper 1.3.1", + "http-body 1.0.1", + "hyper 1.4.1", "hyper-rustls", "hyper-util", "jsonrpsee-core", @@ -1683,16 +1682,16 @@ dependencies = [ [[package]] name = "jsonrpsee-server" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "810f63eff0f78fa8d413d678c0e55b702e2ea61d4587774c0db4ea2fc554ef92" +checksum = "654afab2e92e5d88ebd8a39d6074483f3f2bfdf91c5ac57fe285e7127cdd4f51" dependencies = [ "anyhow", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "jsonrpsee-core", "jsonrpsee-types", @@ -1711,9 +1710,9 @@ dependencies = [ [[package]] name = "jsonrpsee-types" -version = "0.23.1" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f511b714bca46f9a3e97c0e0eb21d2c112e83e444d2db535b5ec7093f5836d73" +checksum = "d9c465fbe385238e861fdc4d1c85e04ada6c1fd246161d26385c1b311724d2af" dependencies = [ "beef", "http 1.1.0", @@ -1784,9 +1783,9 @@ dependencies = [ "futures", "home", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-rustls", "hyper-timeout", "hyper-util", @@ -1835,7 +1834,7 @@ dependencies = [ "proc-macro2", "quote", "serde_json", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -1868,9 +1867,9 @@ dependencies = [ [[package]] name = "lazy_static" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "lazycell" @@ -1886,12 +1885,12 @@ checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" [[package]] name = "libloading" -version = "0.8.3" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2a198fb6b0eada2a8df47933734e6d35d350665a33a3593d7164fa52c75c19" +checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4" dependencies = [ "cfg-if", - "windows-targets 0.52.5", + "windows-targets", ] [[package]] @@ -1938,7 +1937,7 @@ checksum = "f8dccda732e04fa3baf2e17cf835bfe2601c7c2edafd64417c627dabae3a8cda" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -1959,9 +1958,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" [[package]] name = "logos" @@ -1983,7 +1982,7 @@ dependencies = [ "proc-macro2", "quote", "regex-syntax 0.6.29", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -1997,9 +1996,9 @@ dependencies = [ [[package]] name = "lz4-sys" -version = "1.9.5" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9764018d143cc854c9f17f0b907de70f14393b1f502da6375dce70f00514eb3" +checksum = "109de74d5d2353660401699a4174a4ff23fcc649caf553df71933c7fb45ad868" dependencies = [ "cc", "libc", @@ -2040,7 +2039,7 @@ checksum = "49e7bc1560b95a3c4a25d03de42fe76ca718ab92d1a22a55b9b4cf67b3ae635c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -2066,13 +2065,14 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.11" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" +checksum = "4569e456d394deccd22ce1c1913e6ea0e54519f577285001215d33557431afe4" dependencies = [ + "hermit-abi", "libc", "wasi", - "windows-sys 0.48.0", + "windows-sys", ] [[package]] @@ -2109,9 +2109,9 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", @@ -2153,9 +2153,9 @@ dependencies = [ [[package]] name = "object" -version = "0.36.0" +version = "0.36.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "576dfe1fc8f9df304abb159d767a29d0476f7750fbf8aa7ad07816004a207434" +checksum = "3f203fa8daa7bb185f760ae12bd8e097f63d17041dcdcaf675ac54cdf863170e" dependencies = [ "memchr", ] @@ -2168,9 +2168,9 @@ checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "oorandom" -version = "11.1.3" +version = "11.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" [[package]] name = "opaque-debug" @@ -2238,7 +2238,7 @@ dependencies = [ "libc", "redox_syscall", "smallvec", - "windows-targets 0.52.5", + "windows-targets", ] [[package]] @@ -2271,9 +2271,9 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.10" +version = "2.7.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "560131c633294438da9f7c4b08189194b20946c8274c6b9e38881a7874dc8ee8" +checksum = "cd53dff83f26735fdc1ca837098ccf133605d794cdae66acfc2bfac3ec809d95" dependencies = [ "memchr", "thiserror", @@ -2282,9 +2282,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.7.10" +version = "2.7.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26293c9193fbca7b1a3bf9b79dc1e388e927e6cacaa78b4a3ab705a1d3d41459" +checksum = "2a548d2beca6773b1c244554d36fcf8548a8a58e74156968211567250e48e49a" dependencies = [ "pest", "pest_generator", @@ -2292,22 +2292,22 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.10" +version = "2.7.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ec22af7d3fb470a85dd2ca96b7c577a1eb4ef6f1683a9fe9a8c16e136c04687" +checksum = "3c93a82e8d145725dcbaf44e5ea887c8a869efdcc28706df2d08c69e17077183" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "pest_meta" -version = "2.7.10" +version = "2.7.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7a240022f37c361ec1878d646fc5b7d7c4d28d5946e1a80ad5a7a4f4ca0bdcd" +checksum = "a941429fea7e08bedec25e4f6785b6ffaacc6b755da98df5ef3e7dcf4a124c4f" dependencies = [ "once_cell", "pest", @@ -2341,7 +2341,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -2431,9 +2431,12 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "dee4364d9f3b902ef14fab8a1ddffb783a1cb6b4bba3bfc1fa3922732c7de97f" +dependencies = [ + "zerocopy 0.6.6", +] [[package]] name = "pretty_assertions" @@ -2452,23 +2455,23 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f12335488a2f3b0a83b14edad48dca9879ce89b2edd10e80237e4e852dd645e" dependencies = [ "proc-macro2", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "proc-macro2" -version = "1.0.85" +version = "1.0.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" +checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" dependencies = [ "unicode-ident", ] [[package]] name = "prometheus-client" -version = "0.22.2" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1ca959da22a332509f2a73ae9e5f23f9dcfc31fd3a54d71f159495bd5909baa" +checksum = "504ee9ff529add891127c4827eb481bd69dc0ebc72e9a682e187db4caa60c3ca" dependencies = [ "dtoa", "itoa", @@ -2484,7 +2487,7 @@ checksum = "440f724eba9f6996b75d63681b0a92b06947f1457076d503a4d2e2c8f56442b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -2514,7 +2517,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.66", + "syn 2.0.72", "tempfile", ] @@ -2528,7 +2531,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -2699,11 +2702,11 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.2" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd" +checksum = "2a908a6e00f1fdd0dfd9c0eb08ce85126f6d8bbda50017e74bc4a4b7d4a926a4" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", ] [[package]] @@ -2772,7 +2775,7 @@ dependencies = [ "libc", "spin", "untrusted", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -2818,18 +2821,18 @@ version = "0.38.34" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] name = "rustls" -version = "0.23.10" +version = "0.23.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" +checksum = "c58f8c84392efc0a126acce10fa59ff7b3d2ac06ab451a33f2741989b806b044" dependencies = [ "aws-lc-rs", "log", @@ -2843,9 +2846,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f1fb85efa936c42c6d5fc28d2629bb51e4b2f4b8a5211e297d599cc5a093792" +checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" dependencies = [ "openssl-probe", "rustls-pemfile", @@ -2872,9 +2875,9 @@ checksum = "976295e77ce332211c0d24d92c0e83e50f5c5f046d11082cea19f3df13a3562d" [[package]] name = "rustls-platform-verifier" -version = "0.3.1" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5f0d26fa1ce3c790f9590868f0109289a044acb954525f933e2aa3b871c157d" +checksum = "93bda3f493b9abe5b93b3e7e3ecde0df292f2bd28c0296b90586ee0055ff5123" dependencies = [ "core-foundation", "core-foundation-sys", @@ -2893,15 +2896,15 @@ dependencies = [ [[package]] name = "rustls-platform-verifier-android" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84e217e7fdc8466b5b35d30f8c0a30febd29173df4a3a0c2115d306b9c4117ad" +checksum = "f87165f0995f63a9fbeea62b64d10b4d9d8e78ec6d7d51fb2125fda7bb36788f" [[package]] name = "rustls-webpki" -version = "0.102.4" +version = "0.102.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff448f7e92e913c4b7d4c6d8e4540a1724b319b4152b8aef6d4cf8339712b33e" +checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" dependencies = [ "aws-lc-rs", "ring", @@ -2930,7 +2933,7 @@ version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fbc91545643bcf3a0bbb6569265615222618bdf33ce4ffbbd13c4bbd4c093534" dependencies = [ - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -2954,7 +2957,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -2989,11 +2992,11 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.11.0" +version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c627723fd09706bacdb5cf41499e95098555af3c3c29d014dc3c458ef6be11c0" +checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "core-foundation", "core-foundation-sys", "libc", @@ -3003,9 +3006,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.11.0" +version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "317936bbbd05227752583946b9e66d7ce3b489f84e11a94a510b4437fef407d7" +checksum = "75da29fe9b9b08fe9d6b22b5b4bcbc75d8db3aa31e639aa56bb62e9d46bfceaf" dependencies = [ "core-foundation-sys", "libc", @@ -3019,9 +3022,9 @@ checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12" dependencies = [ "serde_derive", ] @@ -3038,13 +3041,13 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3055,16 +3058,17 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "serde_json" -version = "1.0.117" +version = "1.0.122" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3" +checksum = "784b6203951c57ff748476b126ccb5e8e2959a5c19e5c617ab1956be3dbc68da" dependencies = [ "itoa", + "memchr", "ryu", "serde", ] @@ -3196,7 +3200,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ce305eb0b4296696835b71df73eb912e0f1ffd2556a501fcede6e0c50349191c" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -3245,9 +3249,9 @@ checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" [[package]] name = "subtle" -version = "2.5.0" +version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "syn" @@ -3262,9 +3266,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.66" +version = "2.0.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" +checksum = "dc4b9b9bf2add8093d3f2c0204471e951b2285580335de42f9d2534f3ae7a8af" dependencies = [ "proc-macro2", "quote", @@ -3280,7 +3284,7 @@ dependencies = [ "cfg-if", "fastrand", "rustix", - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -3300,7 +3304,7 @@ checksum = "f9b53c7124dd88026d5d98a1eb1fd062a578b7d783017c9298825526c7fb6427" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3319,22 +3323,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" +checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" +checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3387,9 +3391,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -3415,32 +3419,31 @@ dependencies = [ [[package]] name = "tokio" -version = "1.38.0" +version = "1.39.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" +checksum = "daa4fb1bc778bd6f04cbfc4bb2d06a7396a8f299dc33ea1900cedaa316f467b1" dependencies = [ "backtrace", "bytes", "libc", "mio", - "num_cpus", "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", "tokio-macros", - "windows-sys 0.48.0", + "windows-sys", ] [[package]] name = "tokio-macros" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" +checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3505,10 +3508,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e9cd434a998747dd2c4276bc96ee2e0c7a2eadf3cae88e52be55a05fa9053f5" dependencies = [ "base64 0.21.7", - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", "mime", "pin-project-lite", @@ -3549,7 +3552,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3689,9 +3692,9 @@ checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" [[package]] name = "version_check" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" [[package]] name = "vise" @@ -3713,7 +3716,7 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "23981b18d697026f5430249ab01ba739ef2edc463e400042394331cb2bb63494" dependencies = [ - "hyper 0.14.29", + "hyper 0.14.30", "once_cell", "tokio", "tracing", @@ -3728,7 +3731,7 @@ checksum = "8bb19c33cd5f04dcf4e767635e058a998edbc2b7fca32ade0a4a1cea0f8e9b34" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -3777,7 +3780,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", "wasm-bindgen-shared", ] @@ -3799,7 +3802,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3822,9 +3825,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c452ad30530b54a4d8e71952716a212b08efd0f3562baa66c29a618b07da7c3" +checksum = "bd7c23921eeb1713a4e851530e9b9756e4fb0e89978582942612524cf09f01cd" dependencies = [ "rustls-pki-types", ] @@ -3863,7 +3866,7 @@ version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" dependencies = [ - "windows-sys 0.52.0", + "windows-sys", ] [[package]] @@ -3872,169 +3875,124 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" -[[package]] -name = "windows-sys" -version = "0.48.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" -dependencies = [ - "windows-targets 0.48.5", -] - [[package]] name = "windows-sys" version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.5", + "windows-targets", ] [[package]] name = "windows-targets" -version = "0.48.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" dependencies = [ - "windows_aarch64_gnullvm 0.48.5", - "windows_aarch64_msvc 0.48.5", - "windows_i686_gnu 0.48.5", - "windows_i686_msvc 0.48.5", - "windows_x86_64_gnu 0.48.5", - "windows_x86_64_gnullvm 0.48.5", - "windows_x86_64_msvc 0.48.5", -] - -[[package]] -name = "windows-targets" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" -dependencies = [ - "windows_aarch64_gnullvm 0.52.5", - "windows_aarch64_msvc 0.52.5", - "windows_i686_gnu 0.52.5", + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", "windows_i686_gnullvm", - "windows_i686_msvc 0.52.5", - "windows_x86_64_gnu 0.52.5", - "windows_x86_64_gnullvm 0.52.5", - "windows_x86_64_msvc 0.52.5", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", ] [[package]] name = "windows_aarch64_gnullvm" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" - -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_msvc" -version = "0.48.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" - -[[package]] -name = "windows_aarch64_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" - -[[package]] -name = "windows_i686_gnu" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_i686_gnu" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" - -[[package]] -name = "windows_i686_msvc" -version = "0.48.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_x86_64_gnu" -version = "0.48.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnullvm" -version = "0.48.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] -name = "windows_x86_64_gnullvm" -version = "0.52.5" +name = "windows_x86_64_msvc" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] -name = "windows_x86_64_msvc" -version = "0.48.5" +name = "yansi" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" [[package]] -name = "windows_x86_64_msvc" -version = "0.52.5" +name = "zerocopy" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" +checksum = "854e949ac82d619ee9a14c66a1b674ac730422372ccb759ce0c39cabcf2bf8e6" +dependencies = [ + "byteorder", + "zerocopy-derive 0.6.6", +] [[package]] -name = "yansi" -version = "0.5.1" +name = "zerocopy" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +dependencies = [ + "zerocopy-derive 0.7.35", +] [[package]] -name = "zerocopy" -version = "0.7.34" +name = "zerocopy-derive" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae87e3fcd617500e5d106f0380cf7b77f3c6092aae37191433159dda23cfb087" +checksum = "125139de3f6b9d625c39e2efdd73d41bdac468ccd556556440e322be0e1bbd91" dependencies = [ - "zerocopy-derive", + "proc-macro2", + "quote", + "syn 2.0.72", ] [[package]] name = "zerocopy-derive" -version = "0.7.34" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -4054,7 +4012,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] @@ -4154,7 +4112,7 @@ dependencies = [ "build_html", "bytesize", "http-body-util", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "im", "once_cell", @@ -4301,14 +4259,14 @@ dependencies = [ "prost-reflect", "protox", "quote", - "syn 2.0.66", + "syn 2.0.72", ] [[package]] name = "zstd-sys" -version = "2.0.11+zstd.1.5.6" +version = "2.0.12+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75652c55c0b6f3e6f12eb786fe1bc960396bf05a1eb3bf1f3691c3610ac2e6d4" +checksum = "0a4e40c320c3cb459d9a9ff6de98cff88f4751ee9275d140e2be94a2b74e4c13" dependencies = [ "cc", "pkg-config", diff --git a/node/deny.toml b/node/deny.toml index d135b269..2d14d01b 100644 --- a/node/deny.toml +++ b/node/deny.toml @@ -1,37 +1,18 @@ +[graph] +# We only check dependencies against these platforms. targets = [ { triple = "x86_64-unknown-linux-musl" }, { triple = "x86_64-apple-darwin" }, ] -[advisories] -# The lint level for unmaintained crates -unmaintained = "deny" -# The lint level for crates that have been yanked from their source registry -yanked = "deny" -# The lint level for crates with security notices. -notice = "deny" -# Threshold for security vulnerabilities, any vulnerability with a CVSS score -# lower than the range specified will be ignored. Note that ignored advisories -# will still output a note when they are encountered. -# * None - CVSS Score 0.0 -# * Low - CVSS Score 0.1 - 3.9 -# * Medium - CVSS Score 4.0 - 6.9 -# * High - CVSS Score 7.0 - 8.9 -# * Critical - CVSS Score 9.0 - 10.0 -severity-threshold = "medium" - [licenses] -# We want to deny every license that isn't explicitly added to the allow list. -unlicensed = "deny" -copyleft = "deny" -allow-osi-fsf-free = "neither" -default = "deny" # We want to set a high confidence threshold for license detection. confidence-threshold = 1.0 # Licenses that are allowed to be used in crates. allow = [ # Permissive licenses "Apache-2.0", + "BSD-2-Clause", "BSD-3-Clause", "OpenSSL", "ISC", @@ -76,6 +57,9 @@ skip = [ { name = "http", version = "0.2.12"}, { name = "http-body", version = "0.4.6"}, { name = "hyper", version = "0.14.28"}, + + # Old version required by rand. + { name = "zerocopy", version = "0.6.6" }, ] [sources] From d1d3b51e77067855b87bd51d5d52206563d88982 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Fri, 2 Aug 2024 15:06:30 +0100 Subject: [PATCH 07/13] improve: Do not initialise the AttestationStatus (BFT-496) (#169) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ `AttestationRunner::init` should not try to initialise the `AttestationStatus` from the API. Let it be `None` and do the init in the background. This is partially undoing what https://github.com/matter-labs/era-consensus/pull/165 did. ## Why ❔ The rollout procedure to mainnet was that the EN goes first, then the main node. The EN cannot start without the main node API. --- node/actors/executor/src/attestation.rs | 45 ++++++++++--------- node/actors/executor/src/tests.rs | 29 +++++------- .../network/src/gossip/attestation_status.rs | 33 +++++++------- node/actors/network/src/gossip/mod.rs | 11 ++--- node/actors/network/src/testonly.rs | 8 ++-- 5 files changed, 60 insertions(+), 66 deletions(-) diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index c949b9bd..8ccee89c 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -5,7 +5,7 @@ use anyhow::Context; use std::sync::Arc; use zksync_concurrency::{ctx, sync, time}; use zksync_consensus_network::gossip::{ - AttestationStatus, AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, + AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, }; use zksync_consensus_roles::attester; use zksync_consensus_storage::{BatchStore, BlockStore}; @@ -58,9 +58,12 @@ impl AttesterRunner { self.status.mark_changed(); loop { - let batch_number = sync::changed(ctx, &mut self.status) + let Some(batch_number) = sync::changed(ctx, &mut self.status) .await? - .next_batch_to_attest; + .next_batch_to_attest + else { + continue; + }; tracing::info!(%batch_number, "attestation status"); @@ -123,7 +126,10 @@ pub trait AttestationStatusClient: 'static + Send + Sync { /// /// The genesis hash is returned along with the new batch number to facilitate detecting reorgs /// on the main node as soon as possible and prevent inconsistent state from entering the system. - async fn attestation_status(&self, ctx: &ctx::Ctx) -> ctx::Result>; + async fn attestation_status( + &self, + ctx: &ctx::Ctx, + ) -> ctx::Result>; } /// Use an [AttestationStatusClient] to periodically poll the main node and update the [AttestationStatusWatch]. @@ -140,21 +146,19 @@ impl AttestationStatusRunner { /// /// It polls the [AttestationStatusClient] until it returns a value to initialize the status with. pub async fn init( - ctx: &ctx::Ctx, + _ctx: &ctx::Ctx, client: Box, poll_interval: time::Duration, genesis: attester::GenesisHash, ) -> ctx::Result<(Arc, Self)> { - let status = Arc::new(AttestationStatusWatch::new( - genesis, - attester::BatchNumber::default(), - )); - let mut runner = Self { + let status = Arc::new(AttestationStatusWatch::new(genesis)); + let runner = Self { status: status.clone(), client, poll_interval, }; - runner.poll_until_some(ctx).await?; + // This would initialise the status to some value, however the EN was rolled out first without the main node API. + // runner.poll_until_some(ctx).await?; Ok((status, runner)) } @@ -197,10 +201,8 @@ impl AttestationStatusRunner { async fn poll_until_some(&mut self, ctx: &ctx::Ctx) -> ctx::Result<()> { loop { match self.client.attestation_status(ctx).await { - Ok(Some(status)) => { - self.status - .update(status.genesis, status.next_batch_to_attest) - .await?; + Ok(Some((genesis, next_batch_to_attest))) => { + self.status.update(genesis, next_batch_to_attest).await?; return Ok(()); } Ok(None) => { @@ -228,11 +230,12 @@ struct LocalAttestationStatusClient { #[async_trait::async_trait] impl AttestationStatusClient for LocalAttestationStatusClient { - async fn attestation_status(&self, ctx: &ctx::Ctx) -> ctx::Result> { - let batch_number = self.batch_store.next_batch_to_attest(ctx).await?; - Ok(batch_number.map(|n| AttestationStatus { - genesis: self.genesis, - next_batch_to_attest: n, - })) + async fn attestation_status( + &self, + ctx: &ctx::Ctx, + ) -> ctx::Result> { + let next_batch_to_attest = self.batch_store.next_batch_to_attest(ctx).await?; + + Ok(next_batch_to_attest.map(|n| (self.genesis, n))) } } diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 287a2717..fbaaea93 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -7,10 +7,7 @@ use rand::Rng as _; use tracing::Instrument as _; use zksync_concurrency::{sync, testonly::abort_on_panic}; use zksync_consensus_bft as bft; -use zksync_consensus_network::{ - gossip::AttestationStatus, - testonly::{new_configs, new_fullnode}, -}; +use zksync_consensus_network::testonly::{new_configs, new_fullnode}; use zksync_consensus_roles::validator::{testonly::Setup, BlockNumber}; use zksync_consensus_storage::{ testonly::{in_memory, TestMemoryStorage}, @@ -36,10 +33,7 @@ fn config(cfg: &network::Config) -> Config { /// The test executors below are not running with attesters, so we just create an [AttestationStatusWatch] /// that will never be updated. fn never_attest(genesis: &validator::Genesis) -> Arc { - Arc::new(AttestationStatusWatch::new( - genesis.hash(), - attester::BatchNumber::default(), - )) + Arc::new(AttestationStatusWatch::new(genesis.hash())) } fn validator( @@ -345,7 +339,7 @@ async fn test_attestation_status_runner() { async fn attestation_status( &self, _ctx: &ctx::Ctx, - ) -> ctx::Result> { + ) -> ctx::Result> { let curr = self .batch_number .fetch_add(1u64, std::sync::atomic::Ordering::Relaxed); @@ -354,11 +348,9 @@ async fn test_attestation_status_runner() { Ok(None) } else { // The first actual result will be 1 on the 2nd poll. - let status = AttestationStatus { - genesis: *self.genesis.lock().unwrap(), - next_batch_to_attest: attester::BatchNumber(curr), - }; - Ok(Some(status)) + let genesis = *self.genesis.lock().unwrap(); + let next_batch_to_attest = attester::BatchNumber(curr); + Ok(Some((genesis, next_batch_to_attest))) } } } @@ -380,17 +372,18 @@ async fn test_attestation_status_runner() { let mut recv_status = status.subscribe(); recv_status.mark_changed(); - // Check that the value has been initialised to a non-default value. + // Check that the value has *not* been initialised to a non-default value. { let status = sync::changed(ctx, &mut recv_status).await?; - assert_eq!(status.next_batch_to_attest.0, 1); + assert!(status.next_batch_to_attest.is_none()); } // Now start polling for new values. Starting in the foreground because we want it to fail in the end. s.spawn(runner.run(ctx)); // Check that polling sets the value. { let status = sync::changed(ctx, &mut recv_status).await?; - assert_eq!(status.next_batch_to_attest.0, 2); + assert!(status.next_batch_to_attest.is_some()); + assert_eq!(status.next_batch_to_attest.unwrap().0, 1); } // Change the genesis returned by the client. It should cause the scope to fail. { @@ -405,7 +398,7 @@ async fn test_attestation_status_runner() { Ok(()) => panic!("expected to fail when the genesis changed"), Err(e) => assert!( e.to_string().contains("genesis changed"), - "only expect failures due to genesis change" + "only expect failures due to genesis change; got: {e}" ), } } diff --git a/node/actors/network/src/gossip/attestation_status.rs b/node/actors/network/src/gossip/attestation_status.rs index 514bb84b..629845b8 100644 --- a/node/actors/network/src/gossip/attestation_status.rs +++ b/node/actors/network/src/gossip/attestation_status.rs @@ -10,9 +10,9 @@ use crate::watch::Watch; pub struct AttestationStatus { /// Next batch number where voting is expected. /// - /// The node is expected to poll the main node during initialization until - /// the batch to start from is established. - pub next_batch_to_attest: attester::BatchNumber, + /// The field is optional so that we can start an external node without the main node API + /// already deployed, which is how the initial rollout is. + pub next_batch_to_attest: Option, /// The hash of the genesis of the chain to which the L1 batches belong. /// /// We don't expect to handle a regenesis on the fly without restarting the @@ -37,14 +37,11 @@ impl fmt::Debug for AttestationStatusWatch { } impl AttestationStatusWatch { - /// Create a new watch going from a specific batch number. - pub fn new( - genesis: attester::GenesisHash, - next_batch_to_attest: attester::BatchNumber, - ) -> Self { + /// Create a new watch with the current genesis, and a yet-to-be-determined batch number. + pub fn new(genesis: attester::GenesisHash) -> Self { Self(Watch::new(AttestationStatus { genesis, - next_batch_to_attest, + next_batch_to_attest: None, })) } @@ -77,18 +74,20 @@ impl AttestationStatusWatch { // votes below the expected minimum: even if we clear the votes, we might // not get them again from any peer. By returning an error we can cause // the node to be restarted and connections re-established for fresh gossip. - anyhow::ensure!( - status.next_batch_to_attest <= next_batch_to_attest, - "next batch to attest moved backwards: {} -> {}", - status.next_batch_to_attest, - next_batch_to_attest - ); + if let Some(old_batch_to_attest) = status.next_batch_to_attest { + anyhow::ensure!( + old_batch_to_attest <= next_batch_to_attest, + "next batch to attest moved backwards: {} -> {}", + old_batch_to_attest, + next_batch_to_attest + ); + } } this.send_if_modified(|status| { - if status.next_batch_to_attest == next_batch_to_attest { + if status.next_batch_to_attest == Some(next_batch_to_attest) { return false; } - status.next_batch_to_attest = next_batch_to_attest; + status.next_batch_to_attest = Some(next_batch_to_attest); true }); Ok(()) diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 90e5a1d5..88663383 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -173,14 +173,15 @@ impl Network { loop { // Wait until the status indicates that we're ready to sign the next batch. - let next_batch_number = sync::changed(ctx, &mut recv_status) + let Some(batch_number) = sync::changed(ctx, &mut recv_status) .await? - .next_batch_to_attest; + .next_batch_to_attest + else { + continue; + }; // Get rid of all previous votes. We don't expect this to go backwards without regenesis, which will involve a restart. - self.batch_votes - .set_min_batch_number(next_batch_number) - .await; + self.batch_votes.set_min_batch_number(batch_number).await; // Now wait until we find the next quorum, whatever it is: // * on the main node, if attesters are honest, they will vote on the next batch number and the main node will not see gaps diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index 00de2120..7728bc45 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -17,7 +17,7 @@ use zksync_concurrency::{ ctx::{self, channel}, io, limiter, net, scope, sync, time, }; -use zksync_consensus_roles::{attester, node, validator}; +use zksync_consensus_roles::{node, validator}; use zksync_consensus_storage::{BatchStore, BlockStore}; use zksync_consensus_utils::pipe; @@ -204,10 +204,8 @@ impl Instance { ) -> (Self, InstanceRunner) { // Semantically we'd want this to be created at the same level as the stores, // but doing so would introduce a lot of extra cruft in setting up tests. - let attestation_status = Arc::new(AttestationStatusWatch::new( - block_store.genesis().hash(), - attester::BatchNumber::default(), - )); + let attestation_status = + Arc::new(AttestationStatusWatch::new(block_store.genesis().hash())); let (actor_pipe, dispatcher_pipe) = pipe::new(); let (net, net_runner) = Network::new( From 920c95fe584da4ca1794a85f8097ce1a76cad15a Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Fri, 2 Aug 2024 15:07:17 +0100 Subject: [PATCH 08/13] improve: Relax the signature check in BatchVote (#170) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ `BatchVote::update` ignores invalid signatures instead of returning an error. ## Why ❔ Because on `stage-v2` the `moby` variant of the external node runs an earlier version of the code without the `Batch::genesis` field, which causes it to reject all votes coming from the main and the other external nodes. The error causes it to disconnect its gossip peers, which is a bit heavy handed given that this is an unstable feature. --- node/actors/network/src/gossip/batch_votes.rs | 15 +++++++++++---- node/actors/network/src/gossip/tests/mod.rs | 5 +++-- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/node/actors/network/src/gossip/batch_votes.rs b/node/actors/network/src/gossip/batch_votes.rs index 1ea6b8cb..abe8d969 100644 --- a/node/actors/network/src/gossip/batch_votes.rs +++ b/node/actors/network/src/gossip/batch_votes.rs @@ -80,6 +80,11 @@ impl BatchVotes { /// (all entries verified so far are added). /// /// Returns statistics about new entries added. + /// + /// For now it doesn't return an error if a vote with an invalid signature + /// is encountered, so that the node doesn't disconnect from peer if it + /// happens to have a new field in `Batch`. This is only until the feature + /// is stabilized. pub(super) fn update( &mut self, attesters: &attester::Committee, @@ -125,10 +130,12 @@ impl BatchVotes { } // Check the signature before insertion. - d.verify()?; - - self.add(d.clone(), weight); - stats.added(d.msg.number, weight); + if let Err(e) = d.verify() { + tracing::error!(error =? e, "failed to verify batch vote: {e:#}"); + } else { + self.add(d.clone(), weight); + stats.added(d.msg.number, weight); + } } Ok(stats) diff --git a/node/actors/network/src/gossip/tests/mod.rs b/node/actors/network/src/gossip/tests/mod.rs index e0f5f3c2..942a7ec3 100644 --- a/node/actors/network/src/gossip/tests/mod.rs +++ b/node/actors/network/src/gossip/tests/mod.rs @@ -601,7 +601,7 @@ async fn test_batch_votes() { votes.update(&attesters, &genesis, &update).await.unwrap(); assert_eq!(want.0, sub.borrow_and_update().votes); - // Invalid signature, should be rejected. + // Invalid signature, should be ignored. let mut k0v3 = mk_batch( rng, &keys[1], @@ -612,7 +612,8 @@ async fn test_batch_votes() { assert!(votes .update(&attesters, &genesis, &[Arc::new(k0v3)]) .await - .is_err()); + .is_ok()); + assert_eq!(want.0, sub.borrow_and_update().votes); // Invalid genesis, should be rejected. let other_genesis = rng.gen(); From 179b2512b62689efedf831d4a6a705c9fa88466b Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Fri, 2 Aug 2024 15:28:21 +0100 Subject: [PATCH 09/13] Bump version to 0.1.0-rc.6 (#174) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Bumps the version to `0.1.0-rc.6` ## Why ❔ So that we can release https://github.com/matter-labs/era-consensus/pull/170 and https://github.com/matter-labs/era-consensus/pull/169 to make zksync-era backwards compatible. --- node/Cargo.lock | 24 ++++++++++++------------ node/Cargo.toml | 24 ++++++++++++------------ 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index cd86fd1b..b4b0c225 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3309,7 +3309,7 @@ dependencies = [ [[package]] name = "tester" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "clap", @@ -4017,7 +4017,7 @@ dependencies = [ [[package]] name = "zksync_concurrency" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "assert_matches", @@ -4035,7 +4035,7 @@ dependencies = [ [[package]] name = "zksync_consensus_bft" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "assert_matches", @@ -4059,7 +4059,7 @@ dependencies = [ [[package]] name = "zksync_consensus_crypto" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "blst", @@ -4082,7 +4082,7 @@ dependencies = [ [[package]] name = "zksync_consensus_executor" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "async-trait", @@ -4103,7 +4103,7 @@ dependencies = [ [[package]] name = "zksync_consensus_network" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "assert_matches", @@ -4139,7 +4139,7 @@ dependencies = [ [[package]] name = "zksync_consensus_roles" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "assert_matches", @@ -4160,7 +4160,7 @@ dependencies = [ [[package]] name = "zksync_consensus_storage" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "assert_matches", @@ -4182,7 +4182,7 @@ dependencies = [ [[package]] name = "zksync_consensus_tools" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "async-trait", @@ -4217,7 +4217,7 @@ dependencies = [ [[package]] name = "zksync_consensus_utils" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "rand 0.8.5", @@ -4227,7 +4227,7 @@ dependencies = [ [[package]] name = "zksync_protobuf" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "bit-vec", @@ -4249,7 +4249,7 @@ dependencies = [ [[package]] name = "zksync_protobuf_build" -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" dependencies = [ "anyhow", "heck", diff --git a/node/Cargo.toml b/node/Cargo.toml index c5904c27..dfa6b36d 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -22,23 +22,23 @@ homepage = "https://matter-labs.io/" repository = "https://github.com/matter-labs/era-consensus" license = "MIT OR Apache-2.0" keywords = ["blockchain", "zksync"] -version = "0.1.0-rc.5" +version = "0.1.0-rc.6" [workspace.dependencies] # Crates from this repo. -zksync_consensus_bft = { version = "=0.1.0-rc.5", path = "actors/bft" } -zksync_consensus_crypto = { version = "=0.1.0-rc.5", path = "libs/crypto" } -zksync_consensus_executor = { version = "=0.1.0-rc.5", path = "actors/executor" } -zksync_consensus_network = { version = "=0.1.0-rc.5", path = "actors/network" } -zksync_consensus_roles = { version = "=0.1.0-rc.5", path = "libs/roles" } -zksync_consensus_storage = { version = "=0.1.0-rc.5", path = "libs/storage" } -zksync_consensus_tools = { version = "=0.1.0-rc.5", path = "tools" } -zksync_consensus_utils = { version = "=0.1.0-rc.5", path = "libs/utils" } +zksync_consensus_bft = { version = "=0.1.0-rc.6", path = "actors/bft" } +zksync_consensus_crypto = { version = "=0.1.0-rc.6", path = "libs/crypto" } +zksync_consensus_executor = { version = "=0.1.0-rc.6", path = "actors/executor" } +zksync_consensus_network = { version = "=0.1.0-rc.6", path = "actors/network" } +zksync_consensus_roles = { version = "=0.1.0-rc.6", path = "libs/roles" } +zksync_consensus_storage = { version = "=0.1.0-rc.6", path = "libs/storage" } +zksync_consensus_tools = { version = "=0.1.0-rc.6", path = "tools" } +zksync_consensus_utils = { version = "=0.1.0-rc.6", path = "libs/utils" } # Crates from this repo that might become independent in the future. -zksync_concurrency = { version = "=0.1.0-rc.5", path = "libs/concurrency" } -zksync_protobuf = { version = "=0.1.0-rc.5", path = "libs/protobuf" } -zksync_protobuf_build = { version = "=0.1.0-rc.5", path = "libs/protobuf_build" } +zksync_concurrency = { version = "=0.1.0-rc.6", path = "libs/concurrency" } +zksync_protobuf = { version = "=0.1.0-rc.6", path = "libs/protobuf" } +zksync_protobuf_build = { version = "=0.1.0-rc.6", path = "libs/protobuf_build" } # Crates from Matter Labs. pairing = { package = "pairing_ce", version = "=0.28.6" } From dd411587788f7ea5a64ee3bcbb270e093e1182b1 Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Mon, 5 Aug 2024 12:46:25 +0200 Subject: [PATCH 10/13] bumped capability ID of the push_batch_votes rpc (#176) We added an extra field to the Batch message, without considering the fact that an extra field won't be recognized by old binaries and they will fail to verify signatures on Batch messages. They will consider sending such invalid signatures malicious and will disconnect from the peer. To avoid this we create a new RPC instead (by just bumping the capability id on push_batch_votes RPC) which will not be interpreted by the old binaries at all. --- node/actors/bft/src/testonly/twins/partition.rs | 8 ++++---- node/actors/executor/src/tests.rs | 3 +-- node/actors/network/src/gossip/attestation_status.rs | 4 +--- node/actors/network/src/gossip/mod.rs | 9 +++++---- node/actors/network/src/mux/mod.rs | 4 +++- node/actors/network/src/noise/stream.rs | 4 +++- node/actors/network/src/rpc/push_batch_votes.rs | 12 +++++++++++- node/actors/network/src/rpc/tests.rs | 3 ++- node/libs/concurrency/src/ctx/channel.rs | 1 + node/libs/concurrency/src/sync/prunable_mpsc/mod.rs | 2 +- node/libs/protobuf_build/src/lib.rs | 1 + 11 files changed, 33 insertions(+), 18 deletions(-) diff --git a/node/actors/bft/src/testonly/twins/partition.rs b/node/actors/bft/src/testonly/twins/partition.rs index 28c5728c..94f99ff6 100644 --- a/node/actors/bft/src/testonly/twins/partition.rs +++ b/node/actors/bft/src/testonly/twins/partition.rs @@ -75,12 +75,12 @@ impl<'a, T> Partitioner<'a, T> { /// /// Take the table in [partitions] as an example with 4 items and 3 partitions. /// * `A` is the first item, so it can only go into `P1`; `P2` can't be used while `P1` is empty, - /// otherwise we'd be generating redundant combinations. + /// otherwise we'd be generating redundant combinations. /// * `B` can go minimally into `P1` because there are 2 more items after it, which is enough - /// to fill all remaining partitions; or it can go into `P2`, because `P1` is filled. + /// to fill all remaining partitions; or it can go into `P2`, because `P1` is filled. /// * `C` depends on what we did with `B`: if `B` is in `P1` then `C` has to minimally go into - /// `P2` to make sure no partition will be left empty at the end; if `B` is in `P2` then `C` - /// can go either in `P1`, `P2` or `P3`. + /// `P2` to make sure no partition will be left empty at the end; if `B` is in `P2` then `C` + /// can go either in `P1`, `P2` or `P3`. /// /// The algorithm has to traverse all possible placements with backtracking. fn go(&mut self, idx: usize, first_empty: usize) { diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index fbaaea93..50ba7297 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -1,9 +1,8 @@ //! High-level tests for `Executor`. -use std::sync::{atomic::AtomicU64, Mutex}; - use super::*; use attestation::{AttestationStatusClient, AttestationStatusRunner}; use rand::Rng as _; +use std::sync::{atomic::AtomicU64, Mutex}; use tracing::Instrument as _; use zksync_concurrency::{sync, testonly::abort_on_panic}; use zksync_consensus_bft as bft; diff --git a/node/actors/network/src/gossip/attestation_status.rs b/node/actors/network/src/gossip/attestation_status.rs index 629845b8..b10b0208 100644 --- a/node/actors/network/src/gossip/attestation_status.rs +++ b/node/actors/network/src/gossip/attestation_status.rs @@ -1,10 +1,8 @@ +use crate::watch::Watch; use std::fmt; - use zksync_concurrency::sync; use zksync_consensus_roles::attester; -use crate::watch::Watch; - /// Coordinate the attestation by showing the status as seen by the main node. #[derive(Debug, Clone, PartialEq, Eq)] pub struct AttestationStatus { diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 88663383..4e72acd0 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -9,14 +9,15 @@ //! * static connections (explicitly declared in configs of both ends of the connection). //! * dynamic connections (additional randomized connections which are established to improve //! the throughput of the network). +//! //! Static connections constitute a rigid "backbone" of the gossip network, which is insensitive to //! eclipse attack. Dynamic connections are supposed to improve the properties of the gossip //! network graph (minimize its diameter, increase connectedness). -pub use self::attestation_status::{ - AttestationStatus, AttestationStatusReceiver, AttestationStatusWatch, -}; -pub use self::batch_votes::BatchVotesPublisher; use self::batch_votes::BatchVotesWatch; +pub use self::{ + attestation_status::{AttestationStatus, AttestationStatusReceiver, AttestationStatusWatch}, + batch_votes::BatchVotesPublisher, +}; use crate::{gossip::ValidatorAddrsWatch, io, pool::PoolWatch, Config, MeteredStreamStats}; use fetch::RequestItem; use std::sync::{atomic::AtomicUsize, Arc}; diff --git a/node/actors/network/src/mux/mod.rs b/node/actors/network/src/mux/mod.rs index a797a38a..1144f27f 100644 --- a/node/actors/network/src/mux/mod.rs +++ b/node/actors/network/src/mux/mod.rs @@ -40,10 +40,12 @@ //! * capability 0 has 4 reusable streams //! * capability 1 has 3 reusable streams //! * capability 2 has 5 reusable streams +//! //! then //! * streams 0,1,2,3 belong to capability 0. //! * streams 4,5,6 belong to capability 1. //! * streams 7,8,9,10,11 belong to capability 2. +//! //! NOTE that the 2^13 limit applies to the TOTAL number of reusable streams across all //! capabilities. //! @@ -59,7 +61,7 @@ //! Multipexer protocol: //! 1. peer A and B exchange their multiplexer configs. //! 2. based on both configs, A and B determine how many stream IDs will be used and to which -//! capabilities they will belong. +//! capabilities they will belong. //! 3. For every stream IDs in use execute Reusable Stream protocol concurrently. //! //! Reusable stream protocol (for stream ID = x, with capability = c, where A is outbound peer and diff --git a/node/actors/network/src/noise/stream.rs b/node/actors/network/src/noise/stream.rs index dc39f07d..7e903a2a 100644 --- a/node/actors/network/src/noise/stream.rs +++ b/node/actors/network/src/noise/stream.rs @@ -48,7 +48,7 @@ const MAX_PAYLOAD_LEN: usize = MAX_TRANSPORT_MSG_LEN - AUTHDATA_LEN; /// ++ . /// /// Length of the frame len field. -const LENGTH_FIELD_LEN: usize = std::mem::size_of::(); +const LENGTH_FIELD_LEN: usize = size_of::(); /// Max size of the whole frame (length field + data). const MAX_FRAME_LEN: usize = MAX_TRANSPORT_MSG_LEN + LENGTH_FIELD_LEN; @@ -236,6 +236,7 @@ where /// * The amount of data read can be determined by the increase /// in the length of the slice returned by ReadBuf::filled. /// * If the difference is 0, EOF has been reached. + /// /// From std::io::Read: /// * If error was returned, no bytes were read. /// @@ -309,6 +310,7 @@ where { /// from futures::io::AsyncWrite: /// * poll_write must try to make progress by flushing if needed to become writable + /// /// from std::io::Write: /// * call to write represents at most one attempt to write to any wrapped object. /// * 0 TYPICALLY means that that the underlying object is no longer able to accept bytes OR diff --git a/node/actors/network/src/rpc/push_batch_votes.rs b/node/actors/network/src/rpc/push_batch_votes.rs index 51ab7642..e0e67126 100644 --- a/node/actors/network/src/rpc/push_batch_votes.rs +++ b/node/actors/network/src/rpc/push_batch_votes.rs @@ -8,8 +8,18 @@ use zksync_protobuf::ProtoFmt; /// PushBatchVotes RPC. pub(crate) struct Rpc; +/// Deprecated, because adding `genesis_hash` to `validator::Batch` +/// was not backward compatible - old binaries couldn't verify +/// signatures on messages with `genesis_hash` and were treating it +/// as malicious behavior. +#[allow(dead_code)] +pub(super) const V1: mux::CapabilityId = 5; + +/// Current version. +pub(super) const V2: mux::CapabilityId = 8; + impl super::Rpc for Rpc { - const CAPABILITY_ID: mux::CapabilityId = 5; + const CAPABILITY_ID: mux::CapabilityId = V2; const INFLIGHT: u32 = 1; const METHOD: &'static str = "push_batch_votes"; type Req = Req; diff --git a/node/actors/network/src/rpc/tests.rs b/node/actors/network/src/rpc/tests.rs index d00969e2..0f444bf5 100644 --- a/node/actors/network/src/rpc/tests.rs +++ b/node/actors/network/src/rpc/tests.rs @@ -17,7 +17,8 @@ fn test_capability_rpc_correspondence() { push_block_store_state::Rpc::CAPABILITY_ID, get_block::Rpc::CAPABILITY_ID, ping::Rpc::CAPABILITY_ID, - push_batch_votes::Rpc::CAPABILITY_ID, + push_batch_votes::V1, + push_batch_votes::V2, push_batch_store_state::Rpc::CAPABILITY_ID, get_batch::Rpc::CAPABILITY_ID, ]; diff --git a/node/libs/concurrency/src/ctx/channel.rs b/node/libs/concurrency/src/ctx/channel.rs index fac02028..7d7736c9 100644 --- a/node/libs/concurrency/src/ctx/channel.rs +++ b/node/libs/concurrency/src/ctx/channel.rs @@ -3,6 +3,7 @@ //! Note that channel disconnection is not observable by default: //! * send() always succeeds, unless canceled //! * recv() always succeeds, unless canceled +//! //! This //! * simplifies the channel interface //! * prevents users from relying on send/recv failure diff --git a/node/libs/concurrency/src/sync/prunable_mpsc/mod.rs b/node/libs/concurrency/src/sync/prunable_mpsc/mod.rs index f8358d33..ba8f0bd6 100644 --- a/node/libs/concurrency/src/sync/prunable_mpsc/mod.rs +++ b/node/libs/concurrency/src/sync/prunable_mpsc/mod.rs @@ -32,7 +32,7 @@ pub enum SelectionFunctionResult { /// * [`T`]: The type of data that will be sent through the channel. /// * [`filter_predicate`]: A predicate that checks the newly sent value and avoids adding to the queue if it returns false /// * [`selection_function`]: A function that determines whether an unreceived, pending value in the buffer (represented by the first `T`) should be pruned -/// based on a newly sent value (represented by the second `T`) or the new value should be filtered out, or both should be kept. +/// based on a newly sent value (represented by the second `T`) or the new value should be filtered out, or both should be kept. pub fn channel( filter_predicate: impl 'static + Sync + Send + Fn(&T) -> bool, selection_function: impl 'static + Sync + Send + Fn(&T, &T) -> SelectionFunctionResult, diff --git a/node/libs/protobuf_build/src/lib.rs b/node/libs/protobuf_build/src/lib.rs index ed1bcc26..16589f33 100644 --- a/node/libs/protobuf_build/src/lib.rs +++ b/node/libs/protobuf_build/src/lib.rs @@ -14,6 +14,7 @@ //! cargo cache. //! 2. grep through all target/debug/build/*/output files to find all "PROTOBUF_DESCRIPTOR=..." //! lines and merge the descriptor files by simply concatenating them. +//! //! Note that you can run this procedure for 2 revisions of the repo and look for breaking //! changes by running "buf breaking --against " where before.binpb //! and after.binpb are the concatenated descriptors from those 2 revisions. From d74da1887e5a6e55295a96e3c990f17263daf1b8 Mon Sep 17 00:00:00 2001 From: Akosh Farkash Date: Mon, 5 Aug 2024 18:07:16 +0100 Subject: [PATCH 11/13] Pin Rust version to 1.79 (#177) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Created a `rust-toolchain` file pinning it to the stable version when 1.79 was released. Reverted the change to `stream.rs` brought about by 1.80 in https://github.com/matter-labs/era-consensus/pull/176 Bumps the version to `0.1.0-rc.8` ## Why ❔ Because the `zksync-era` build [doesn't work](https://github.com/matter-labs/zksync-era/actions/runs/10249428261/job/28352782628?pr=2583) with `size_of` without qualification like `core::mem::` or `std::mem::`. `zksync-era` is pinned at `nightly-2024-05-07` at the moment. I tried to update it to the last nightly where it was still Rust 1.80, which was `nightly-2024-06-08` but it cannot compile `boojum`, nor does it compile with a later nightly I tried where it's already Rust 1.82. --- node/Cargo.lock | 24 ++++++++++++------------ node/Cargo.toml | 24 ++++++++++++------------ node/actors/network/src/noise/stream.rs | 2 +- rust-toolchain | 1 + 4 files changed, 26 insertions(+), 25 deletions(-) create mode 100644 rust-toolchain diff --git a/node/Cargo.lock b/node/Cargo.lock index b4b0c225..aade5307 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3309,7 +3309,7 @@ dependencies = [ [[package]] name = "tester" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "clap", @@ -4017,7 +4017,7 @@ dependencies = [ [[package]] name = "zksync_concurrency" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "assert_matches", @@ -4035,7 +4035,7 @@ dependencies = [ [[package]] name = "zksync_consensus_bft" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "assert_matches", @@ -4059,7 +4059,7 @@ dependencies = [ [[package]] name = "zksync_consensus_crypto" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "blst", @@ -4082,7 +4082,7 @@ dependencies = [ [[package]] name = "zksync_consensus_executor" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "async-trait", @@ -4103,7 +4103,7 @@ dependencies = [ [[package]] name = "zksync_consensus_network" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "assert_matches", @@ -4139,7 +4139,7 @@ dependencies = [ [[package]] name = "zksync_consensus_roles" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "assert_matches", @@ -4160,7 +4160,7 @@ dependencies = [ [[package]] name = "zksync_consensus_storage" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "assert_matches", @@ -4182,7 +4182,7 @@ dependencies = [ [[package]] name = "zksync_consensus_tools" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "async-trait", @@ -4217,7 +4217,7 @@ dependencies = [ [[package]] name = "zksync_consensus_utils" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "rand 0.8.5", @@ -4227,7 +4227,7 @@ dependencies = [ [[package]] name = "zksync_protobuf" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "bit-vec", @@ -4249,7 +4249,7 @@ dependencies = [ [[package]] name = "zksync_protobuf_build" -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" dependencies = [ "anyhow", "heck", diff --git a/node/Cargo.toml b/node/Cargo.toml index dfa6b36d..61284261 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -22,23 +22,23 @@ homepage = "https://matter-labs.io/" repository = "https://github.com/matter-labs/era-consensus" license = "MIT OR Apache-2.0" keywords = ["blockchain", "zksync"] -version = "0.1.0-rc.6" +version = "0.1.0-rc.8" [workspace.dependencies] # Crates from this repo. -zksync_consensus_bft = { version = "=0.1.0-rc.6", path = "actors/bft" } -zksync_consensus_crypto = { version = "=0.1.0-rc.6", path = "libs/crypto" } -zksync_consensus_executor = { version = "=0.1.0-rc.6", path = "actors/executor" } -zksync_consensus_network = { version = "=0.1.0-rc.6", path = "actors/network" } -zksync_consensus_roles = { version = "=0.1.0-rc.6", path = "libs/roles" } -zksync_consensus_storage = { version = "=0.1.0-rc.6", path = "libs/storage" } -zksync_consensus_tools = { version = "=0.1.0-rc.6", path = "tools" } -zksync_consensus_utils = { version = "=0.1.0-rc.6", path = "libs/utils" } +zksync_consensus_bft = { version = "=0.1.0-rc.8", path = "actors/bft" } +zksync_consensus_crypto = { version = "=0.1.0-rc.8", path = "libs/crypto" } +zksync_consensus_executor = { version = "=0.1.0-rc.8", path = "actors/executor" } +zksync_consensus_network = { version = "=0.1.0-rc.8", path = "actors/network" } +zksync_consensus_roles = { version = "=0.1.0-rc.8", path = "libs/roles" } +zksync_consensus_storage = { version = "=0.1.0-rc.8", path = "libs/storage" } +zksync_consensus_tools = { version = "=0.1.0-rc.8", path = "tools" } +zksync_consensus_utils = { version = "=0.1.0-rc.8", path = "libs/utils" } # Crates from this repo that might become independent in the future. -zksync_concurrency = { version = "=0.1.0-rc.6", path = "libs/concurrency" } -zksync_protobuf = { version = "=0.1.0-rc.6", path = "libs/protobuf" } -zksync_protobuf_build = { version = "=0.1.0-rc.6", path = "libs/protobuf_build" } +zksync_concurrency = { version = "=0.1.0-rc.8", path = "libs/concurrency" } +zksync_protobuf = { version = "=0.1.0-rc.8", path = "libs/protobuf" } +zksync_protobuf_build = { version = "=0.1.0-rc.8", path = "libs/protobuf_build" } # Crates from Matter Labs. pairing = { package = "pairing_ce", version = "=0.28.6" } diff --git a/node/actors/network/src/noise/stream.rs b/node/actors/network/src/noise/stream.rs index 7e903a2a..2e9462f9 100644 --- a/node/actors/network/src/noise/stream.rs +++ b/node/actors/network/src/noise/stream.rs @@ -48,7 +48,7 @@ const MAX_PAYLOAD_LEN: usize = MAX_TRANSPORT_MSG_LEN - AUTHDATA_LEN; /// ++ . /// /// Length of the frame len field. -const LENGTH_FIELD_LEN: usize = size_of::(); +const LENGTH_FIELD_LEN: usize = std::mem::size_of::(); /// Max size of the whole frame (length field + data). const MAX_FRAME_LEN: usize = MAX_TRANSPORT_MSG_LEN + LENGTH_FIELD_LEN; diff --git a/rust-toolchain b/rust-toolchain new file mode 100644 index 00000000..32a89c96 --- /dev/null +++ b/rust-toolchain @@ -0,0 +1 @@ +stable-2024-06-13 \ No newline at end of file From 635d3b79c67fe2f01a91c70883dea864e844f928 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 7 Aug 2024 16:09:44 +0400 Subject: [PATCH 12/13] chore: Bump vise & release rc.9 (#179) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ `vise` 0.1.0 has compilation issues on modern nightly because of `linkme` crates, so the new version was migrated to use `ctor` instead. ## Why ❔ Unblock the path to using modern nightly Rust. --- node/Cargo.lock | 68 +++++++++++++++++++++---------------------------- node/Cargo.toml | 28 ++++++++++---------- 2 files changed, 43 insertions(+), 53 deletions(-) diff --git a/node/Cargo.lock b/node/Cargo.lock index aade5307..46f89e46 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -753,6 +753,16 @@ dependencies = [ "typenum", ] +[[package]] +name = "ctor" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" +dependencies = [ + "quote", + "syn 2.0.72", +] + [[package]] name = "ctr" version = "0.9.2" @@ -1920,26 +1930,6 @@ dependencies = [ "vcpkg", ] -[[package]] -name = "linkme" -version = "0.3.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccb76662d78edc9f9bf56360d6919bdacc8b7761227727e5082f128eeb90bbf5" -dependencies = [ - "linkme-impl", -] - -[[package]] -name = "linkme-impl" -version = "0.3.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8dccda732e04fa3baf2e17cf835bfe2601c7c2edafd64417c627dabae3a8cda" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.72", -] - [[package]] name = "linux-raw-sys" version = "0.4.14" @@ -3309,7 +3299,7 @@ dependencies = [ [[package]] name = "tester" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "clap", @@ -3698,13 +3688,13 @@ checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" [[package]] name = "vise" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229baafe01d5177b63c6ee1def80d8e39a2365e64caf69ddb05a57594b15647c" +checksum = "90ade36f3548b1524396f4de7b36f4f210c8a01dfab568eb2bff466af64eb6e5" dependencies = [ "compile-fmt", + "ctor", "elsa", - "linkme", "once_cell", "prometheus-client", "vise-macros", @@ -3712,9 +3702,9 @@ dependencies = [ [[package]] name = "vise-exporter" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23981b18d697026f5430249ab01ba739ef2edc463e400042394331cb2bb63494" +checksum = "671d3b894d5d0849f0a597f56bf071f42d4f2a1cbcf2f78ca21f870ab7c0cc2b" dependencies = [ "hyper 0.14.30", "once_cell", @@ -3725,9 +3715,9 @@ dependencies = [ [[package]] name = "vise-macros" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb19c33cd5f04dcf4e767635e058a998edbc2b7fca32ade0a4a1cea0f8e9b34" +checksum = "6a511871dc5de990a3b2a0e715facfbc5da848c0c0395597a1415029fb7c250a" dependencies = [ "proc-macro2", "quote", @@ -4017,7 +4007,7 @@ dependencies = [ [[package]] name = "zksync_concurrency" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "assert_matches", @@ -4035,7 +4025,7 @@ dependencies = [ [[package]] name = "zksync_consensus_bft" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "assert_matches", @@ -4059,7 +4049,7 @@ dependencies = [ [[package]] name = "zksync_consensus_crypto" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "blst", @@ -4082,7 +4072,7 @@ dependencies = [ [[package]] name = "zksync_consensus_executor" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "async-trait", @@ -4103,7 +4093,7 @@ dependencies = [ [[package]] name = "zksync_consensus_network" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "assert_matches", @@ -4139,7 +4129,7 @@ dependencies = [ [[package]] name = "zksync_consensus_roles" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "assert_matches", @@ -4160,7 +4150,7 @@ dependencies = [ [[package]] name = "zksync_consensus_storage" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "assert_matches", @@ -4182,7 +4172,7 @@ dependencies = [ [[package]] name = "zksync_consensus_tools" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "async-trait", @@ -4217,7 +4207,7 @@ dependencies = [ [[package]] name = "zksync_consensus_utils" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "rand 0.8.5", @@ -4227,7 +4217,7 @@ dependencies = [ [[package]] name = "zksync_protobuf" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "bit-vec", @@ -4249,7 +4239,7 @@ dependencies = [ [[package]] name = "zksync_protobuf_build" -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" dependencies = [ "anyhow", "heck", diff --git a/node/Cargo.toml b/node/Cargo.toml index 61284261..5fc25193 100644 --- a/node/Cargo.toml +++ b/node/Cargo.toml @@ -22,28 +22,28 @@ homepage = "https://matter-labs.io/" repository = "https://github.com/matter-labs/era-consensus" license = "MIT OR Apache-2.0" keywords = ["blockchain", "zksync"] -version = "0.1.0-rc.8" +version = "0.1.0-rc.9" [workspace.dependencies] # Crates from this repo. -zksync_consensus_bft = { version = "=0.1.0-rc.8", path = "actors/bft" } -zksync_consensus_crypto = { version = "=0.1.0-rc.8", path = "libs/crypto" } -zksync_consensus_executor = { version = "=0.1.0-rc.8", path = "actors/executor" } -zksync_consensus_network = { version = "=0.1.0-rc.8", path = "actors/network" } -zksync_consensus_roles = { version = "=0.1.0-rc.8", path = "libs/roles" } -zksync_consensus_storage = { version = "=0.1.0-rc.8", path = "libs/storage" } -zksync_consensus_tools = { version = "=0.1.0-rc.8", path = "tools" } -zksync_consensus_utils = { version = "=0.1.0-rc.8", path = "libs/utils" } +zksync_consensus_bft = { version = "=0.1.0-rc.9", path = "actors/bft" } +zksync_consensus_crypto = { version = "=0.1.0-rc.9", path = "libs/crypto" } +zksync_consensus_executor = { version = "=0.1.0-rc.9", path = "actors/executor" } +zksync_consensus_network = { version = "=0.1.0-rc.9", path = "actors/network" } +zksync_consensus_roles = { version = "=0.1.0-rc.9", path = "libs/roles" } +zksync_consensus_storage = { version = "=0.1.0-rc.9", path = "libs/storage" } +zksync_consensus_tools = { version = "=0.1.0-rc.9", path = "tools" } +zksync_consensus_utils = { version = "=0.1.0-rc.9", path = "libs/utils" } # Crates from this repo that might become independent in the future. -zksync_concurrency = { version = "=0.1.0-rc.8", path = "libs/concurrency" } -zksync_protobuf = { version = "=0.1.0-rc.8", path = "libs/protobuf" } -zksync_protobuf_build = { version = "=0.1.0-rc.8", path = "libs/protobuf_build" } +zksync_concurrency = { version = "=0.1.0-rc.9", path = "libs/concurrency" } +zksync_protobuf = { version = "=0.1.0-rc.9", path = "libs/protobuf" } +zksync_protobuf_build = { version = "=0.1.0-rc.9", path = "libs/protobuf_build" } # Crates from Matter Labs. pairing = { package = "pairing_ce", version = "=0.28.6" } -vise = "0.1.0" -vise-exporter = "0.1.0" +vise = "0.2.0" +vise-exporter = "0.2.0" # Crates from third-parties. anyhow = "1" From e9f8dfeb2073d4a16e78c788dbb02ca6f313f687 Mon Sep 17 00:00:00 2001 From: Daniyar Itegulov Date: Thu, 8 Aug 2024 18:25:41 +1000 Subject: [PATCH 13/13] feat: instrument code with tracing spans (#178) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ This PR instruments all interesting* places within the system. **\*** "interesting" here is very subjective and I tried to use my understanding of the code to judge what should be covered and what shouldn't. Feel free to give feedback ## Why ❔ Better observability --- node/actors/bft/src/lib.rs | 41 +++--- node/actors/executor/src/attestation.rs | 75 ++++++----- .../network/src/consensus/handshake/mod.rs | 2 + node/actors/network/src/consensus/mod.rs | 40 +++--- node/actors/network/src/gossip/batch_votes.rs | 2 + node/actors/network/src/gossip/mod.rs | 127 +++++++++++------- node/actors/network/src/gossip/runner.rs | 4 +- node/actors/network/src/lib.rs | 3 +- node/libs/concurrency/src/ctx/mod.rs | 14 +- node/libs/concurrency/src/ctx/tests.rs | 6 +- node/libs/storage/src/batch_store/mod.rs | 46 +++++-- node/libs/storage/src/block_store/mod.rs | 50 +++++-- node/libs/storage/src/testonly/in_memory.rs | 1 + node/tools/src/store.rs | 2 +- 14 files changed, 264 insertions(+), 149 deletions(-) diff --git a/node/actors/bft/src/lib.rs b/node/actors/bft/src/lib.rs index c677735a..efa74e1d 100644 --- a/node/actors/bft/src/lib.rs +++ b/node/actors/bft/src/lib.rs @@ -19,6 +19,7 @@ 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_consensus_roles::validator; @@ -90,24 +91,34 @@ impl Config { // 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. loop { - let InputMessage::Network(req) = pipe.recv.recv(ctx).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, - }; + async { + let InputMessage::Network(req) = 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); + replica_send.send(new_req); + leader_send.send(req); + } + M::ReplicaCommit(_) => leader_send.send(req), + M::LeaderPrepare(_) | M::LeaderCommit(_) => replica_send.send(req), } - M::ReplicaCommit(_) => leader_send.send(req), - M::LeaderPrepare(_) | M::LeaderCommit(_) => replica_send.send(req), + + ctx::Ok(()) } + .instrument(tracing::info_span!("bft_iter")) + .await?; } }) .await; diff --git a/node/actors/executor/src/attestation.rs b/node/actors/executor/src/attestation.rs index 8ccee89c..e0bd63d5 100644 --- a/node/actors/executor/src/attestation.rs +++ b/node/actors/executor/src/attestation.rs @@ -3,6 +3,7 @@ use crate::Attester; use anyhow::Context; use std::sync::Arc; +use tracing::Instrument; use zksync_concurrency::{ctx, sync, time}; use zksync_consensus_network::gossip::{ AttestationStatusReceiver, AttestationStatusWatch, BatchVotesPublisher, @@ -58,56 +59,70 @@ impl AttesterRunner { self.status.mark_changed(); loop { - let Some(batch_number) = sync::changed(ctx, &mut self.status) - .await? - .next_batch_to_attest - else { - continue; - }; - - tracing::info!(%batch_number, "attestation status"); - - // We can avoid actively polling the database in `wait_for_batch_to_sign` by waiting its existence - // to be indicated in memory (which itself relies on polling). This happens once we have the commitment, - // which for nodes that get the blocks through BFT should happen after execution. Nodes which - // rely on batch sync don't participate in attestations as they need the batch on L1 first. - self.batch_store - .wait_until_persisted(ctx, batch_number) + async { + let Some(batch_number) = sync::changed(ctx, &mut self.status) + .instrument(tracing::info_span!("wait_for_attestation_status")) + .await? + .next_batch_to_attest + else { + return Ok(()); + }; + + tracing::info!(%batch_number, "attestation status"); + + // We can avoid actively polling the database in `wait_for_batch_to_sign` by waiting its existence + // to be indicated in memory (which itself relies on polling). This happens once we have the commitment, + // which for nodes that get the blocks through BFT should happen after execution. Nodes which + // rely on batch sync don't participate in attestations as they need the batch on L1 first. + self.batch_store + .wait_until_persisted(ctx, batch_number) + .await?; + + // Try to get the next batch to sign; the commitment might not be available just yet. + let batch = AttesterRunner::wait_for_batch_to_sign( + ctx, + batch_number, + &self.batch_store, + self.poll_interval, + ) .await?; - // Try to get the next batch to sign; the commitment might not be available just yet. - let batch = self.wait_for_batch_to_sign(ctx, batch_number).await?; + // The certificates might be collected out of order because of how gossip works; + // we could query the DB to see if we already have a QC, or we can just go ahead + // and publish our vote, and let others ignore it. - // The certificates might be collected out of order because of how gossip works; - // we could query the DB to see if we already have a QC, or we can just go ahead - // and publish our vote, and let others ignore it. + tracing::info!(%batch_number, "publishing attestation"); - tracing::info!(%batch_number, "publishing attestation"); + // We only have to publish a vote once; future peers can pull it from the register. + self.publisher + .publish(attesters, &genesis, &self.attester.key, batch) + .await + .context("publish")?; - // We only have to publish a vote once; future peers can pull it from the register. - self.publisher - .publish(attesters, &genesis, &self.attester.key, batch) - .await - .context("publish")?; + ctx::Ok(()) + } + .instrument(tracing::info_span!("attestation_iter")) + .await?; } } /// Wait for the batch commitment to become available. + #[tracing::instrument(skip_all, fields(l1_batch = %number))] async fn wait_for_batch_to_sign( - &self, ctx: &ctx::Ctx, number: attester::BatchNumber, + batch_store: &BatchStore, + poll_interval: time::Duration, ) -> ctx::Result { loop { - if let Some(batch) = self - .batch_store + if let Some(batch) = batch_store .batch_to_sign(ctx, number) .await .context("batch_to_sign")? { return Ok(batch); } else { - ctx.sleep(self.poll_interval).await?; + ctx.sleep(poll_interval).await?; } } } diff --git a/node/actors/network/src/consensus/handshake/mod.rs b/node/actors/network/src/consensus/handshake/mod.rs index 9dc6bdb6..5b0e7561 100644 --- a/node/actors/network/src/consensus/handshake/mod.rs +++ b/node/actors/network/src/consensus/handshake/mod.rs @@ -58,6 +58,7 @@ pub(super) enum Error { Stream(#[from] ctx::Error), } +#[tracing::instrument(name = "handshake::outbound", skip_all)] pub(super) async fn outbound( ctx: &ctx::Ctx, me: &validator::SecretKey, @@ -93,6 +94,7 @@ pub(super) async fn outbound( Ok(()) } +#[tracing::instrument(name = "handshake::inbound", skip_all)] pub(super) async fn inbound( ctx: &ctx::Ctx, me: &validator::SecretKey, diff --git a/node/actors/network/src/consensus/mod.rs b/node/actors/network/src/consensus/mod.rs index c7acea92..3d4f5ad9 100644 --- a/node/actors/network/src/consensus/mod.rs +++ b/node/actors/network/src/consensus/mod.rs @@ -163,7 +163,7 @@ impl Network { /// Performs handshake of an inbound stream. /// Closes the stream if there is another inbound stream opened from the same validator. - #[tracing::instrument(level = "info", name = "consensus", skip_all)] + #[tracing::instrument(name = "consensus::run_inbound_stream", skip_all)] pub(crate) async fn run_inbound_stream( &self, ctx: &ctx::Ctx, @@ -195,7 +195,7 @@ impl Network { res } - #[tracing::instrument(level = "info", name = "consensus", skip_all)] + #[tracing::instrument(name = "consensus::run_outbound_stream", skip_all, fields(?peer, %addr))] async fn run_outbound_stream( &self, ctx: &ctx::Ctx, @@ -282,6 +282,7 @@ impl Network { res } + #[tracing::instrument(skip_all)] async fn run_loopback_stream(&self, ctx: &ctx::Ctx) -> anyhow::Result<()> { let addr = *self .gossip @@ -295,7 +296,6 @@ impl Network { format!("{:?} resolved to no addresses", self.gossip.cfg.public_addr) })?; self.run_outbound_stream(ctx, &self.key.public(), addr) - .instrument(tracing::info_span!("loopback", ?addr)) .await } @@ -314,24 +314,26 @@ impl Network { } let addrs = &mut self.gossip.validator_addrs.subscribe(); let mut addr = None; + while ctx.is_active() { - // Wait for a new address, or retry with the old one after timeout. - if let Ok(new) = - sync::wait_for(&ctx.with_timeout(config::CONNECT_RETRY), addrs, |addrs| { - addrs.get(peer).map(|x| x.msg.addr) != addr - }) - .await - { - addr = new.get(peer).map(|x| x.msg.addr); - } - let Some(addr) = addr else { continue }; - if let Err(err) = self - .run_outbound_stream(ctx, peer, addr) - .instrument(tracing::info_span!("out", ?addr)) - .await - { - tracing::info!("run_outbound_stream({peer:?},{addr}): {err:#}"); + async { + // Wait for a new address, or retry with the old one after timeout. + if let Ok(new) = + sync::wait_for(&ctx.with_timeout(config::CONNECT_RETRY), addrs, |addrs| { + addrs.get(peer).map(|x| x.msg.addr) != addr + }) + .instrument(tracing::info_span!("wait_for_address")) + .await + { + addr = new.get(peer).map(|x| x.msg.addr); + } + let Some(addr) = addr else { return }; + if let Err(err) = self.run_outbound_stream(ctx, peer, addr).await { + tracing::info!("run_outbound_stream({peer:?},{addr}): {err:#}"); + } } + .instrument(tracing::info_span!("maintain_connection_iter")) + .await; } } } diff --git a/node/actors/network/src/gossip/batch_votes.rs b/node/actors/network/src/gossip/batch_votes.rs index abe8d969..b48fb205 100644 --- a/node/actors/network/src/gossip/batch_votes.rs +++ b/node/actors/network/src/gossip/batch_votes.rs @@ -286,6 +286,7 @@ impl BatchVotesWatch { } /// Set the minimum batch number on the votes and discard old data. + #[tracing::instrument(skip_all, fields(%min_batch_number))] pub(crate) async fn set_min_batch_number(&self, min_batch_number: attester::BatchNumber) { let this = self.0.lock().await; this.send_modify(|votes| votes.set_min_batch_number(min_batch_number)); @@ -308,6 +309,7 @@ impl fmt::Debug for BatchVotesPublisher { impl BatchVotesPublisher { /// Sign an L1 batch and push it into the batch, which should cause it to be gossiped by the network. + #[tracing::instrument(skip_all, fields(l1_batch = %batch.number))] pub async fn publish( &self, attesters: &attester::Committee, diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 4e72acd0..7d22062d 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -21,6 +21,7 @@ pub use self::{ use crate::{gossip::ValidatorAddrsWatch, io, pool::PoolWatch, Config, MeteredStreamStats}; use fetch::RequestItem; use std::sync::{atomic::AtomicUsize, Arc}; +use tracing::Instrument; pub(crate) use validator_addrs::*; use zksync_concurrency::{ctx, ctx::channel, error::Wrap as _, scope, sync}; use zksync_consensus_roles::{node, validator}; @@ -109,20 +110,28 @@ impl Network { let number = ctx::NoCopy(next); next = next + 1; // Fetch a block asynchronously. - s.spawn(async { - let _permit = permit; - let number = number.into(); - let _: ctx::OrCanceled<()> = scope::run!(ctx, |ctx, s| async { - s.spawn_bg(self.fetch_queue.request(ctx, RequestItem::Block(number))); - // Cancel fetching as soon as block is queued for storage. - self.block_store.wait_until_queued(ctx, number).await?; - Err(ctx::Canceled) - }) - .await; - // Wait until the block is actually persisted, so that the amount of blocks - // stored in memory is bounded. - self.block_store.wait_until_persisted(ctx, number).await - }); + s.spawn( + async { + let _permit = permit; + let number = number.into(); + let _: ctx::OrCanceled<()> = scope::run!(ctx, |ctx, s| async { + s.spawn_bg( + self.fetch_queue + .request(ctx, RequestItem::Block(number)) + .instrument(tracing::info_span!("fetch_block_request")), + ); + // Cancel fetching as soon as block is queued for storage. + self.block_store.wait_until_queued(ctx, number).await?; + Err(ctx::Canceled) + }) + .instrument(tracing::info_span!("wait_for_block_to_queue")) + .await; + // Wait until the block is actually persisted, so that the amount of blocks + // stored in memory is bounded. + self.block_store.wait_until_persisted(ctx, number).await + } + .instrument(tracing::info_span!("fetch_block_from_peer", l2_block = %next)), + ); } }) .await; @@ -138,20 +147,28 @@ impl Network { let number = ctx::NoCopy(next); next = next + 1; // Fetch a batch asynchronously. - s.spawn(async { - let _permit = permit; - let number = number.into(); - let _: ctx::OrCanceled<()> = scope::run!(ctx, |ctx, s| async { - s.spawn_bg(self.fetch_queue.request(ctx, RequestItem::Batch(number))); - // Cancel fetching as soon as batch is queued for storage. - self.batch_store.wait_until_queued(ctx, number).await?; - Err(ctx::Canceled) - }) - .await; - // Wait until the batch is actually persisted, so that the amount of batches - // stored in memory is bounded. - self.batch_store.wait_until_persisted(ctx, number).await - }); + s.spawn( + async { + let _permit = permit; + let number = number.into(); + let _: ctx::OrCanceled<()> = scope::run!(ctx, |ctx, s| async { + s.spawn_bg( + self.fetch_queue + .request(ctx, RequestItem::Batch(number)) + .instrument(tracing::info_span!("fetch_block_request")), + ); + // Cancel fetching as soon as batch is queued for storage. + self.batch_store.wait_until_queued(ctx, number).await?; + Err(ctx::Canceled) + }) + .instrument(tracing::info_span!("wait_for_batch_to_queue")) + .await; + // Wait until the batch is actually persisted, so that the amount of batches + // stored in memory is bounded. + self.batch_store.wait_until_persisted(ctx, number).await + } + .instrument(tracing::info_span!("fetch_batch_from_peer", l1_batch = %next)), + ); } }) .await; @@ -173,32 +190,40 @@ impl Network { recv_status.mark_changed(); loop { - // Wait until the status indicates that we're ready to sign the next batch. - let Some(batch_number) = sync::changed(ctx, &mut recv_status) - .await? - .next_batch_to_attest - else { - continue; - }; + async { + // Wait until the status indicates that we're ready to sign the next batch. + let Some(batch_number) = sync::changed(ctx, &mut recv_status) + .instrument(tracing::info_span!("wait_for_attestation_status")) + .await? + .next_batch_to_attest + else { + return Ok(()); + }; - // Get rid of all previous votes. We don't expect this to go backwards without regenesis, which will involve a restart. - self.batch_votes.set_min_batch_number(batch_number).await; + // Get rid of all previous votes. We don't expect this to go backwards without regenesis, which will involve a restart. + self.batch_votes.set_min_batch_number(batch_number).await; - // Now wait until we find the next quorum, whatever it is: - // * on the main node, if attesters are honest, they will vote on the next batch number and the main node will not see gaps - // * on external nodes the votes might be affected by changes in the value returned by the API, and there might be gaps - // What is important, though, is that the batch number does not move backwards while we look for a quorum, because attesters - // (re)casting earlier votes will go ignored by those fixed on a higher min_batch_number, and gossip will only be attempted once. - // The possibility of this will be fixed by deterministally picking a start batch number based on fork indicated by genesis. - let qc = sync::wait_for_some(ctx, &mut recv_votes, |votes| { - votes.find_quorum(attesters, &genesis) - }) - .await?; + // Now wait until we find the next quorum, whatever it is: + // * on the main node, if attesters are honest, they will vote on the next batch number and the main node will not see gaps + // * on external nodes the votes might be affected by changes in the value returned by the API, and there might be gaps + // What is important, though, is that the batch number does not move backwards while we look for a quorum, because attesters + // (re)casting earlier votes will go ignored by those fixed on a higher min_batch_number, and gossip will only be attempted once. + // The possibility of this will be fixed by deterministally picking a start batch number based on fork indicated by genesis. + let qc = sync::wait_for_some(ctx, &mut recv_votes, |votes| { + votes.find_quorum(attesters, &genesis) + }) + .instrument(tracing::info_span!("wait_for_quorum")) + .await?; + + self.batch_store + .persist_batch_qc(ctx, qc) + .await + .wrap("persist_batch_qc")?; - self.batch_store - .persist_batch_qc(ctx, qc) - .await - .wrap("persist_batch_qc")?; + ctx::Ok(()) + } + .instrument(tracing::info_span!("new_votes_iter")) + .await?; } } } diff --git a/node/actors/network/src/gossip/runner.rs b/node/actors/network/src/gossip/runner.rs index 2596ccd7..effd0466 100644 --- a/node/actors/network/src/gossip/runner.rs +++ b/node/actors/network/src/gossip/runner.rs @@ -366,7 +366,7 @@ impl Network { /// Handles an inbound stream. /// Closes the stream if there is another inbound stream opened from the same peer. - #[tracing::instrument(level = "info", name = "gossip", skip_all)] + #[tracing::instrument(name = "gossip::run_inbound_stream", skip_all)] pub(crate) async fn run_inbound_stream( &self, ctx: &ctx::Ctx, @@ -384,7 +384,7 @@ impl Network { } /// Connects to a peer and handles the resulting stream. - #[tracing::instrument(level = "info", name = "gossip", skip_all)] + #[tracing::instrument(name = "gossip::run_outbound_stream", skip_all)] pub(crate) async fn run_outbound_stream( &self, ctx: &ctx::Ctx, diff --git a/node/actors/network/src/lib.rs b/node/actors/network/src/lib.rs index 1d5f3cbd..94b692c6 100644 --- a/node/actors/network/src/lib.rs +++ b/node/actors/network/src/lib.rs @@ -151,7 +151,6 @@ impl Runner { .net .gossip .run_outbound_stream(ctx, peer, addr.clone()) - .instrument(tracing::info_span!("out", ?addr)) .await; if let Err(err) = res { tracing::info!("gossip.run_outbound_stream({addr:?}): {err:#}"); @@ -209,7 +208,7 @@ impl Runner { } anyhow::Ok(()) } - .instrument(tracing::info_span!("in", ?addr)) + .instrument(tracing::info_span!("accept_connection", ?addr)) .await; if let Err(err) = res { tracing::info!("{addr}: {err:#}"); diff --git a/node/libs/concurrency/src/ctx/mod.rs b/node/libs/concurrency/src/ctx/mod.rs index bf4f01d5..58ac2d54 100644 --- a/node/libs/concurrency/src/ctx/mod.rs +++ b/node/libs/concurrency/src/ctx/mod.rs @@ -185,8 +185,8 @@ impl Ctx { ) -> CtxAware>> { CtxAware(async { tokio::select! { - output = fut => Ok(output), - () = self.0.canceled.cancel_safe_recv() => Err(Canceled), + output = fut => OrCanceled::Ok(output), + () = self.0.canceled.cancel_safe_recv() => OrCanceled::Err(Canceled), } }) } @@ -271,6 +271,16 @@ pub enum Error { /// Alias for Result with `ctx::Error`. pub type Result = std::result::Result; +/// Equivalent to Ok::<_, ctx::Error>(value). +/// +/// This simplifies creation of an ctx::Result in places where type inference +/// cannot deduce the `E` type of the result — without needing to write +/// `Ok::<_, ctx::Error>(value)`. +#[allow(non_snake_case)] +pub fn Ok(t: T) -> Result { + Result::Ok(t) +} + impl crate::error::Wrap for Error { fn with_wrap C>(self, f: F) -> Self { match self { diff --git a/node/libs/concurrency/src/ctx/tests.rs b/node/libs/concurrency/src/ctx/tests.rs index cf24ac50..41f33cd3 100644 --- a/node/libs/concurrency/src/ctx/tests.rs +++ b/node/libs/concurrency/src/ctx/tests.rs @@ -9,7 +9,7 @@ async fn test_run_canceled() { scope::run!(ctx, |ctx, s| async { s.cancel(); assert!(!ctx.is_active()); - Ok::<(), ()>(()) + std::result::Result::Ok::<(), ()>(()) }) .await .unwrap(); @@ -46,7 +46,7 @@ async fn test_sleep_until() { }); clock.advance(1001 * sec); tracing::info!("root task terminating"); - Ok(()) + std::result::Result::Ok(()) }) .await; assert_eq!(Err(9), res); @@ -55,7 +55,7 @@ async fn test_sleep_until() { let res = scope::run!(ctx, |ctx, s| async { s.spawn(async { assert!(ctx.sleep_until(t).await.is_err()); - Ok(()) + std::result::Result::Ok(()) }); clock.advance_until(t - sec); R::Err(1) diff --git a/node/libs/storage/src/batch_store/mod.rs b/node/libs/storage/src/batch_store/mod.rs index 889014f1..43d060a0 100644 --- a/node/libs/storage/src/batch_store/mod.rs +++ b/node/libs/storage/src/batch_store/mod.rs @@ -1,6 +1,7 @@ //! Defines storage layer for batches of blocks. use anyhow::Context as _; use std::{collections::VecDeque, fmt, sync::Arc}; +use tracing::Instrument; use zksync_concurrency::{ctx, error::Wrap as _, scope, sync}; use zksync_consensus_roles::{attester, validator}; @@ -148,6 +149,7 @@ impl Inner { true } + #[tracing::instrument(skip_all)] fn truncate_cache(&mut self) { while self.cache.len() > Self::CACHE_CAPACITY && self.persisted.contains(self.cache[0].number) @@ -192,24 +194,41 @@ impl BatchStoreRunner { s.spawn::<()>(async { let mut persisted = persisted; loop { - let persisted = sync::changed(ctx, &mut persisted).await?.clone(); - self.0.inner.send_modify(|inner| { - // XXX: In `BlockStoreRunner` update both the `queued` and the `persisted` here. - inner.persisted = persisted; - inner.truncate_cache(); - }); + async { + let persisted = sync::changed(ctx, &mut persisted) + .instrument(tracing::info_span!("wait_for_batch_store_change")) + .await? + .clone(); + self.0.inner.send_modify(|inner| { + // XXX: In `BlockStoreRunner` update both the `queued` and the `persisted` here. + inner.persisted = persisted; + inner.truncate_cache(); + }); + + ctx::Ok(()) + } + .instrument(tracing::info_span!("truncate_batch_cache_iter")) + .await?; } }); let inner = &mut self.0.inner.subscribe(); loop { - let batch = sync::wait_for(ctx, inner, |inner| inner.queued.contains(queue_next)) - .await? - .batch(queue_next) - .unwrap(); + async { + let batch = + sync::wait_for(ctx, inner, |inner| inner.queued.contains(queue_next)) + .instrument(tracing::info_span!("wait_for_next_batch")) + .await? + .batch(queue_next) + .unwrap(); - queue_next = queue_next.next(); + queue_next = queue_next.next(); - self.0.persistent.queue_next_batch(ctx, batch).await?; + self.0.persistent.queue_next_batch(ctx, batch).await?; + + ctx::Ok(()) + } + .instrument(tracing::info_span!("queue_persist_batch_iter")) + .await?; } }) .await; @@ -347,6 +366,7 @@ impl BatchStore { } /// Wait until the database has a batch, then attach the corresponding QC. + #[tracing::instrument(skip_all, fields(l1_batch = %qc.message.number))] pub async fn persist_batch_qc(&self, ctx: &ctx::Ctx, qc: attester::BatchQC) -> ctx::Result<()> { let t = metrics::BATCH_STORE.persist_batch_qc.start(); // The `store_qc` implementation in `zksync-era` retries the insertion of the QC if the payload @@ -368,6 +388,7 @@ impl BatchStore { /// Waits until the given batch is queued (in memory, or persisted). /// Note that it doesn't mean that the batch is actually available, as old batches might get pruned. + #[tracing::instrument(skip_all, fields(l1_batch = %number))] pub async fn wait_until_queued( &self, ctx: &ctx::Ctx, @@ -388,6 +409,7 @@ impl BatchStore { /// Waits until the given batch is stored persistently. /// Note that it doesn't mean that the batch is actually available, as old batches might get pruned. + #[tracing::instrument(skip_all, fields(l1_batch = %number))] pub async fn wait_until_persisted( &self, ctx: &ctx::Ctx, diff --git a/node/libs/storage/src/block_store/mod.rs b/node/libs/storage/src/block_store/mod.rs index a901c0fe..14397c19 100644 --- a/node/libs/storage/src/block_store/mod.rs +++ b/node/libs/storage/src/block_store/mod.rs @@ -1,6 +1,7 @@ //! Defines storage layer for finalized blocks. use anyhow::Context as _; use std::{collections::VecDeque, fmt, sync::Arc}; +use tracing::Instrument; use zksync_concurrency::{ctx, error::Wrap as _, scope, sync}; use zksync_consensus_roles::validator; @@ -117,6 +118,13 @@ impl Inner { } /// Updates `persisted` field. + #[tracing::instrument( + skip_all, + fields( + first_l2_block = %persisted.first, + last_l2_block = ?persisted.last.as_ref().map(|l| l.message.proposal.number) + ) + )] fn update_persisted(&mut self, persisted: BlockStoreState) -> anyhow::Result<()> { if persisted.next() < self.persisted.next() { anyhow::bail!("head block has been removed from storage, this is not supported"); @@ -181,25 +189,41 @@ impl BlockStoreRunner { let mut persisted = self.0.persistent.persisted(); persisted.mark_changed(); loop { - let new = sync::changed(ctx, &mut persisted).await?.clone(); - sync::try_send_modify(&self.0.inner, |inner| inner.update_persisted(new))?; + async { + let new = sync::changed(ctx, &mut persisted) + .instrument(tracing::info_span!("wait_for_block_store_change")) + .await? + .clone(); + sync::try_send_modify(&self.0.inner, |inner| inner.update_persisted(new))?; + + ctx::Ok(()) + } + .instrument(tracing::info_span!("watch_persistent_state_iteration")) + .await?; } }); // Task queueing blocks to be persisted. let inner = &mut self.0.inner.subscribe(); let mut queue_next = validator::BlockNumber(0); loop { - let block = sync::wait_for_some(ctx, inner, |inner| { - inner.block(queue_next.max(inner.persisted.next())) - }) + async { + let block = sync::wait_for_some(ctx, inner, |inner| { + inner.block(queue_next.max(inner.persisted.next())) + }) + .instrument(tracing::info_span!("wait_for_next_block")) + .await?; + queue_next = block.number().next(); + // TODO: monitor errors as well. + let t = metrics::PERSISTENT_BLOCK_STORE + .queue_next_block_latency + .start(); + self.0.persistent.queue_next_block(ctx, block).await?; + t.observe(); + + ctx::Ok(()) + } + .instrument(tracing::info_span!("queue_persist_block_iteration")) .await?; - queue_next = block.number().next(); - // TODO: monitor errors as well. - let t = metrics::PERSISTENT_BLOCK_STORE - .queue_next_block_latency - .start(); - self.0.persistent.queue_next_block(ctx, block).await?; - t.observe(); } }) .await; @@ -309,6 +333,7 @@ impl BlockStore { /// Waits until the given block is queued (in memory, or persisted). /// Note that it doesn't mean that the block is actually available, as old blocks might get pruned. + #[tracing::instrument(skip_all, fields(l2_block = %number))] pub async fn wait_until_queued( &self, ctx: &ctx::Ctx, @@ -324,6 +349,7 @@ impl BlockStore { /// Waits until the given block is stored persistently. /// Note that it doesn't mean that the block is actually available, as old blocks might get pruned. + #[tracing::instrument(skip_all, fields(l2_block = %number))] pub async fn wait_until_persisted( &self, ctx: &ctx::Ctx, diff --git a/node/libs/storage/src/testonly/in_memory.rs b/node/libs/storage/src/testonly/in_memory.rs index 79a21207..2ca8858e 100644 --- a/node/libs/storage/src/testonly/in_memory.rs +++ b/node/libs/storage/src/testonly/in_memory.rs @@ -202,6 +202,7 @@ impl PersistentBatchStore for BatchStore { Ok(batches.get(idx as usize).cloned()) } + #[tracing::instrument(skip_all, fields(l1_batch = %batch.number))] async fn queue_next_batch( &self, _ctx: &ctx::Ctx, diff --git a/node/tools/src/store.rs b/node/tools/src/store.rs index 8847a27c..c6622b75 100644 --- a/node/tools/src/store.rs +++ b/node/tools/src/store.rs @@ -137,7 +137,7 @@ impl PersistentBlockStore for RocksDB { .wrap(number) } - #[tracing::instrument(level = "debug", skip(self))] + #[tracing::instrument(skip_all, fields(l2_block = %block.justification.message.proposal.number))] async fn queue_next_block( &self, _ctx: &ctx::Ctx,