From 6a4a69511b5c0611603eb881e9e3f443e69949bc Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Wed, 9 Oct 2024 18:06:59 +0200 Subject: [PATCH] feat: support for syncing pre-genesis blocks (#203) We will sync blocks with merkle proofs for the whole batches, which obsoletes the unused "batch syncing" logic that we had. I've introduced a concept of PreGenesisBlock which contains a custom Justification, and a verify_pre_genesis_block() method in PersistentBlockStore for verifying those. Some of the RPC requests required tuning to provide backward compatibility. I've removed WIRE from protobuf compatibility check, since it is a subset of WIRE_JSON. --- .github/workflows/protobuf.yaml | 2 +- node/Cargo.lock | 24 +- node/actors/bft/src/lib.rs | 7 + node/actors/bft/src/replica/block.rs | 2 +- node/actors/bft/src/replica/tests.rs | 2 +- node/actors/bft/src/testonly/node.rs | 1 - node/actors/bft/src/testonly/run.rs | 25 +- node/actors/bft/src/testonly/ut_harness.rs | 89 ++--- node/actors/bft/src/tests.rs | 8 +- node/actors/executor/src/lib.rs | 13 +- node/actors/executor/src/tests.rs | 120 ++----- node/actors/network/src/config.rs | 6 +- node/actors/network/src/consensus/tests.rs | 40 +-- node/actors/network/src/gossip/fetch.rs | 66 +--- .../actors/network/src/gossip/loadtest/mod.rs | 6 +- .../network/src/gossip/loadtest/tests.rs | 5 +- node/actors/network/src/gossip/mod.rs | 43 +-- node/actors/network/src/gossip/runner.rs | 148 ++------ node/actors/network/src/gossip/testonly.rs | 4 - .../network/src/gossip/tests/fetch_batches.rs | 330 ----------------- .../network/src/gossip/tests/fetch_blocks.rs | 98 +++-- node/actors/network/src/gossip/tests/mod.rs | 42 +-- .../network/src/gossip/tests/syncing.rs | 220 +++++------- node/actors/network/src/lib.rs | 11 +- node/actors/network/src/proto/gossip.proto | 53 ++- node/actors/network/src/proto/rpc.proto | 6 +- node/actors/network/src/rpc/get_batch.rs | 58 --- node/actors/network/src/rpc/get_block.rs | 49 ++- node/actors/network/src/rpc/mod.rs | 2 - .../network/src/rpc/push_batch_store_state.rs | 42 --- .../network/src/rpc/push_block_store_state.rs | 104 +++++- node/actors/network/src/rpc/testonly.rs | 27 +- node/actors/network/src/rpc/tests.rs | 3 - node/actors/network/src/testonly.rs | 18 +- node/actors/network/src/tests.rs | 5 +- node/libs/roles/src/attester/conv.rs | 32 +- .../roles/src/attester/keys/secret_key.rs | 4 +- .../libs/roles/src/attester/messages/batch.rs | 30 +- node/libs/roles/src/attester/testonly.rs | 15 +- .../roles/src/proto/attester/messages.proto | 6 - .../roles/src/proto/validator/messages.proto | 14 + node/libs/roles/src/validator/conv.rs | 53 ++- .../roles/src/validator/messages/block.rs | 55 +++ node/libs/roles/src/validator/testonly.rs | 118 +++--- node/libs/roles/src/validator/tests.rs | 2 + node/libs/storage/src/batch_store/metrics.rs | 50 --- node/libs/storage/src/batch_store/mod.rs | 340 ------------------ node/libs/storage/src/block_store/metrics.rs | 5 +- node/libs/storage/src/block_store/mod.rs | 159 +++++--- node/libs/storage/src/lib.rs | 4 +- node/libs/storage/src/testonly/in_memory.rs | 119 +++--- node/libs/storage/src/testonly/mod.rs | 131 ++----- node/libs/storage/src/tests.rs | 87 ++++- node/tools/src/bin/deployer.rs | 1 - node/tools/src/bin/localnet_config.rs | 1 - node/tools/src/config.rs | 32 +- node/tools/src/proto/mod.proto | 7 +- .../src/rpc/methods/last_committed_block.rs | 3 +- node/tools/src/rpc/methods/last_view.rs | 11 +- node/tools/src/store.rs | 33 +- node/tools/src/tests.rs | 4 +- 61 files changed, 974 insertions(+), 2021 deletions(-) delete mode 100644 node/actors/network/src/gossip/tests/fetch_batches.rs delete mode 100644 node/actors/network/src/rpc/get_batch.rs delete mode 100644 node/actors/network/src/rpc/push_batch_store_state.rs delete mode 100644 node/libs/storage/src/batch_store/metrics.rs delete mode 100644 node/libs/storage/src/batch_store/mod.rs diff --git a/.github/workflows/protobuf.yaml b/.github/workflows/protobuf.yaml index 128764b4..53e301ed 100644 --- a/.github/workflows/protobuf.yaml +++ b/.github/workflows/protobuf.yaml @@ -72,4 +72,4 @@ jobs: with: github_token: ${{ github.token }} - name: buf breaking - run: buf breaking './after.binpb' --against './before.binpb' --config '{"version":"v1","breaking":{"use":["WIRE_JSON","WIRE"]}}' --error-format 'github-actions' + run: buf breaking './after.binpb' --against './before.binpb' --config '{"version":"v1","breaking":{"use":["WIRE_JSON"]}}' --error-format 'github-actions' diff --git a/node/Cargo.lock b/node/Cargo.lock index e5a547df..346e2af7 100644 --- a/node/Cargo.lock +++ b/node/Cargo.lock @@ -3221,7 +3221,7 @@ dependencies = [ [[package]] name = "tester" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "clap", @@ -3929,7 +3929,7 @@ dependencies = [ [[package]] name = "zksync_concurrency" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "assert_matches", @@ -3947,7 +3947,7 @@ dependencies = [ [[package]] name = "zksync_consensus_bft" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "assert_matches", @@ -3971,7 +3971,7 @@ dependencies = [ [[package]] name = "zksync_consensus_crypto" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "blst", @@ -3991,7 +3991,7 @@ dependencies = [ [[package]] name = "zksync_consensus_executor" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "async-trait", @@ -4013,7 +4013,7 @@ dependencies = [ [[package]] name = "zksync_consensus_network" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "assert_matches", @@ -4051,7 +4051,7 @@ dependencies = [ [[package]] name = "zksync_consensus_roles" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "assert_matches", @@ -4072,7 +4072,7 @@ dependencies = [ [[package]] name = "zksync_consensus_storage" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "assert_matches", @@ -4094,7 +4094,7 @@ dependencies = [ [[package]] name = "zksync_consensus_tools" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "async-trait", @@ -4129,7 +4129,7 @@ dependencies = [ [[package]] name = "zksync_consensus_utils" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "rand", @@ -4139,7 +4139,7 @@ dependencies = [ [[package]] name = "zksync_protobuf" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "bit-vec", @@ -4161,7 +4161,7 @@ dependencies = [ [[package]] name = "zksync_protobuf_build" -version = "0.3.0" +version = "0.4.0" dependencies = [ "anyhow", "heck", diff --git a/node/actors/bft/src/lib.rs b/node/actors/bft/src/lib.rs index efa74e1d..89061564 100644 --- a/node/actors/bft/src/lib.rs +++ b/node/actors/bft/src/lib.rs @@ -70,6 +70,13 @@ impl Config { anyhow::ensure!(genesis.protocol_version == validator::ProtocolVersion::CURRENT); genesis.verify().context("genesis().verify()")?; + if let Some(prev) = genesis.first_block.prev() { + tracing::info!("Waiting for the pre-genesis blocks to be persisted"); + if let Err(ctx::Canceled) = self.block_store.wait_until_persisted(ctx, prev).await { + return Ok(()); + } + } + let cfg = Arc::new(self); let (leader, leader_send) = leader::StateMachine::new(ctx, cfg.clone(), pipe.send.clone()); let (replica, replica_send) = diff --git a/node/actors/bft/src/replica/block.rs b/node/actors/bft/src/replica/block.rs index adea3d78..2d6dc0a8 100644 --- a/node/actors/bft/src/replica/block.rs +++ b/node/actors/bft/src/replica/block.rs @@ -44,7 +44,7 @@ impl StateMachine { ); self.config .block_store - .queue_block(ctx, block.clone()) + .queue_block(ctx, block.clone().into()) .await?; // For availability, replica should not proceed until it stores the block persistently. self.config diff --git a/node/actors/bft/src/replica/tests.rs b/node/actors/bft/src/replica/tests.rs index d05f8eca..10d7407d 100644 --- a/node/actors/bft/src/replica/tests.rs +++ b/node/actors/bft/src/replica/tests.rs @@ -224,7 +224,7 @@ async fn leader_prepare_invalid_payload() { util.replica .config .block_store - .queue_block(ctx, block) + .queue_block(ctx, block.into()) .await .unwrap(); diff --git a/node/actors/bft/src/testonly/node.rs b/node/actors/bft/src/testonly/node.rs index 2ed792eb..49b15314 100644 --- a/node/actors/bft/src/testonly/node.rs +++ b/node/actors/bft/src/testonly/node.rs @@ -34,7 +34,6 @@ pub(super) struct Node { pub(crate) net: network::Config, pub(crate) behavior: Behavior, pub(crate) block_store: Arc, - pub(crate) batch_store: Arc, } impl Node { diff --git a/node/actors/bft/src/testonly/run.rs b/node/actors/bft/src/testonly/run.rs index bc11e121..8b06968b 100644 --- a/node/actors/bft/src/testonly/run.rs +++ b/node/actors/bft/src/testonly/run.rs @@ -15,7 +15,7 @@ use zksync_concurrency::{ oneshot, scope, }; use zksync_consensus_network::{self as network}; -use zksync_consensus_roles::validator; +use zksync_consensus_roles::{validator, validator::testonly::Setup}; use zksync_consensus_storage::{testonly::TestMemoryStorage, BlockStore}; use zksync_consensus_utils::pipe; @@ -114,12 +114,9 @@ impl Test { /// Run a test with the given parameters and a random network setup. pub(crate) async fn run(&self, ctx: &ctx::Ctx) -> Result<(), TestError> { let rng = &mut ctx.rng(); - let setup = validator::testonly::Setup::new_with_weights( - rng, - self.nodes.iter().map(|(_, w)| *w).collect(), - ); + let setup = Setup::new_with_weights(rng, self.nodes.iter().map(|(_, w)| *w).collect()); let nets: Vec<_> = network::testonly::new_configs(rng, &setup, 1); - self.run_with_config(ctx, nets, &setup.genesis).await + self.run_with_config(ctx, nets, &setup).await } /// Run a test with the given parameters and network configuration. @@ -127,13 +124,13 @@ impl Test { &self, ctx: &ctx::Ctx, nets: Vec, - genesis: &validator::Genesis, + setup: &Setup, ) -> Result<(), TestError> { let mut nodes = vec![]; let mut honest = vec![]; scope::run!(ctx, |ctx, s| async { for (i, net) in nets.into_iter().enumerate() { - let store = TestMemoryStorage::new(ctx, genesis).await; + let store = TestMemoryStorage::new(ctx, setup).await; s.spawn_bg(async { Ok(store.runner.run(ctx).await?) }); if self.nodes[i].0 == Behavior::Honest { @@ -144,7 +141,6 @@ impl Test { net, behavior: self.nodes[i].0, block_store: store.blocks, - batch_store: store.batches, }); } assert!(!honest.is_empty()); @@ -152,7 +148,7 @@ impl Test { // Run the nodes until all honest nodes store enough finalized blocks. assert!(self.blocks_to_finalize > 0); - let first = genesis.first_block; + let first = setup.genesis.first_block; let last = first + (self.blocks_to_finalize as u64 - 1); for store in &honest { store.wait_until_queued(ctx, last).await?; @@ -165,7 +161,7 @@ impl Test { let want = honest[0].block(ctx, i).await?.context("missing block")?; for store in &honest[1..] { let got = store.block(ctx, i).await?.context("missing block")?; - if want.payload != got.payload { + if want.payload() != got.payload() { return Err(TestError::BlockConflict); } } @@ -189,11 +185,8 @@ async fn run_nodes_real(ctx: &ctx::Ctx, specs: &[Node]) -> anyhow::Result<()> { scope::run!(ctx, |ctx, s| async { let mut nodes = vec![]; for (i, spec) in specs.iter().enumerate() { - let (node, runner) = network::testonly::Instance::new( - spec.net.clone(), - spec.block_store.clone(), - spec.batch_store.clone(), - ); + let (node, runner) = + network::testonly::Instance::new(spec.net.clone(), spec.block_store.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } diff --git a/node/actors/bft/src/testonly/ut_harness.rs b/node/actors/bft/src/testonly/ut_harness.rs index 0db7740a..53b2acbe 100644 --- a/node/actors/bft/src/testonly/ut_harness.rs +++ b/node/actors/bft/src/testonly/ut_harness.rs @@ -10,11 +10,11 @@ use assert_matches::assert_matches; use std::sync::Arc; use zksync_concurrency::{ctx, sync::prunable_mpsc}; use zksync_consensus_network as network; -use zksync_consensus_roles::validator::{ - self, CommitQC, LeaderCommit, LeaderPrepare, Phase, PrepareQC, ReplicaCommit, ReplicaPrepare, - SecretKey, Signed, ViewNumber, +use zksync_consensus_roles::validator; +use zksync_consensus_storage::{ + testonly::{in_memory, TestMemoryStorage}, + BlockStoreRunner, }; -use zksync_consensus_storage::testonly::{in_memory, TestMemoryStorage, TestMemoryStorageRunner}; use zksync_consensus_utils::enum_util::Variant; pub(crate) const MAX_PAYLOAD_SIZE: usize = 1000; @@ -28,7 +28,7 @@ pub(crate) const MAX_PAYLOAD_SIZE: usize = 1000; pub(crate) struct UTHarness { pub(crate) leader: leader::StateMachine, pub(crate) replica: replica::StateMachine, - pub(crate) keys: Vec, + pub(crate) keys: Vec, pub(crate) leader_send: prunable_mpsc::Sender, pipe: ctx::channel::UnboundedReceiver, } @@ -38,7 +38,7 @@ impl UTHarness { pub(crate) async fn new( ctx: &ctx::Ctx, num_validators: usize, - ) -> (UTHarness, TestMemoryStorageRunner) { + ) -> (UTHarness, BlockStoreRunner) { Self::new_with_payload( ctx, num_validators, @@ -51,10 +51,10 @@ impl UTHarness { ctx: &ctx::Ctx, num_validators: usize, payload_manager: Box, - ) -> (UTHarness, TestMemoryStorageRunner) { + ) -> (UTHarness, BlockStoreRunner) { let rng = &mut ctx.rng(); let setup = validator::testonly::Setup::new(rng, num_validators); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; let (send, recv) = ctx::channel::unbounded(); let cfg = Arc::new(Config { @@ -75,23 +75,23 @@ impl UTHarness { keys: setup.validator_keys.clone(), leader_send, }; - let _: Signed = this.try_recv().unwrap(); + let _: validator::Signed = this.try_recv().unwrap(); (this, store.runner) } /// Creates a new `UTHarness` with minimally-significant validator set size. - pub(crate) async fn new_many(ctx: &ctx::Ctx) -> (UTHarness, TestMemoryStorageRunner) { + pub(crate) async fn new_many(ctx: &ctx::Ctx) -> (UTHarness, BlockStoreRunner) { let num_validators = 6; let (util, runner) = UTHarness::new(ctx, num_validators).await; assert!(util.genesis().validators.max_faulty_weight() > 0); (util, runner) } - /// Triggers replica timeout, validates the new ReplicaPrepare + /// Triggers replica timeout, validates the new validator::ReplicaPrepare /// then executes the whole new view to make sure that the consensus /// recovers after a timeout. pub(crate) async fn produce_block_after_timeout(&mut self, ctx: &ctx::Ctx) { - let want = ReplicaPrepare { + let want = validator::ReplicaPrepare { view: validator::View { genesis: self.genesis().hash(), number: self.replica.view.next(), @@ -112,11 +112,11 @@ impl UTHarness { .unwrap(); } - pub(crate) fn owner_key(&self) -> &SecretKey { + pub(crate) fn owner_key(&self) -> &validator::SecretKey { &self.replica.config.secret_key } - pub(crate) fn sign>(&self, msg: V) -> Signed { + pub(crate) fn sign>(&self, msg: V) -> validator::Signed { self.replica.config.secret_key.sign_msg(msg) } @@ -135,28 +135,28 @@ impl UTHarness { } } - pub(crate) fn new_replica_prepare(&mut self) -> ReplicaPrepare { + pub(crate) fn new_replica_prepare(&mut self) -> validator::ReplicaPrepare { self.set_owner_as_view_leader(); - ReplicaPrepare { + validator::ReplicaPrepare { view: self.replica_view(), high_vote: self.replica.high_vote.clone(), high_qc: self.replica.high_qc.clone(), } } - pub(crate) fn new_current_replica_commit(&self) -> ReplicaCommit { - ReplicaCommit { + pub(crate) fn new_current_replica_commit(&self) -> validator::ReplicaCommit { + validator::ReplicaCommit { view: self.replica_view(), proposal: self.replica.high_qc.as_ref().unwrap().message.proposal, } } - pub(crate) async fn new_leader_prepare(&mut self, ctx: &ctx::Ctx) -> LeaderPrepare { + pub(crate) async fn new_leader_prepare(&mut self, ctx: &ctx::Ctx) -> validator::LeaderPrepare { let msg = self.new_replica_prepare(); self.process_replica_prepare_all(ctx, msg).await.msg } - pub(crate) async fn new_replica_commit(&mut self, ctx: &ctx::Ctx) -> ReplicaCommit { + pub(crate) async fn new_replica_commit(&mut self, ctx: &ctx::Ctx) -> validator::ReplicaCommit { let msg = self.new_leader_prepare(ctx).await; self.process_leader_prepare(ctx, self.sign(msg)) .await @@ -164,7 +164,7 @@ impl UTHarness { .msg } - pub(crate) async fn new_leader_commit(&mut self, ctx: &ctx::Ctx) -> LeaderCommit { + pub(crate) async fn new_leader_commit(&mut self, ctx: &ctx::Ctx) -> validator::LeaderCommit { let msg = self.new_replica_commit(ctx).await; self.process_replica_commit_all(ctx, msg).await.msg } @@ -172,8 +172,8 @@ impl UTHarness { pub(crate) async fn process_leader_prepare( &mut self, ctx: &ctx::Ctx, - msg: Signed, - ) -> Result, leader_prepare::Error> { + msg: validator::Signed, + ) -> Result, leader_prepare::Error> { self.replica.process_leader_prepare(ctx, msg).await?; Ok(self.try_recv().unwrap()) } @@ -181,8 +181,8 @@ impl UTHarness { pub(crate) async fn process_leader_commit( &mut self, ctx: &ctx::Ctx, - msg: Signed, - ) -> Result, leader_commit::Error> { + msg: validator::Signed, + ) -> Result, leader_commit::Error> { self.replica.process_leader_commit(ctx, msg).await?; Ok(self.try_recv().unwrap()) } @@ -191,8 +191,8 @@ impl UTHarness { pub(crate) async fn process_replica_prepare( &mut self, ctx: &ctx::Ctx, - msg: Signed, - ) -> Result>, replica_prepare::Error> { + msg: validator::Signed, + ) -> Result>, replica_prepare::Error> { let prepare_qc = self.leader.prepare_qc.subscribe(); self.leader.process_replica_prepare(ctx, msg).await?; if prepare_qc.has_changed().unwrap() { @@ -212,8 +212,8 @@ impl UTHarness { pub(crate) async fn process_replica_prepare_all( &mut self, ctx: &ctx::Ctx, - msg: ReplicaPrepare, - ) -> Signed { + msg: validator::ReplicaPrepare, + ) -> validator::Signed { let mut leader_prepare = None; let msgs: Vec<_> = self.keys.iter().map(|k| k.sign_msg(msg.clone())).collect(); let mut first_match = true; @@ -238,8 +238,8 @@ impl UTHarness { pub(crate) async fn process_replica_commit( &mut self, ctx: &ctx::Ctx, - msg: Signed, - ) -> Result>, replica_commit::Error> { + msg: validator::Signed, + ) -> Result>, replica_commit::Error> { self.leader.process_replica_commit(ctx, msg)?; Ok(self.try_recv()) } @@ -247,8 +247,8 @@ impl UTHarness { async fn process_replica_commit_all( &mut self, ctx: &ctx::Ctx, - msg: ReplicaCommit, - ) -> Signed { + msg: validator::ReplicaCommit, + ) -> validator::Signed { let mut first_match = true; for (i, key) in self.keys.iter().enumerate() { let res = self @@ -270,7 +270,7 @@ impl UTHarness { self.try_recv().unwrap() } - fn try_recv>(&mut self) -> Option> { + fn try_recv>(&mut self) -> Option> { self.pipe.try_recv().map(|message| match message { OutputMessage::Network(network::io::ConsensusInputMessage { message, .. }) => { message.cast().unwrap() @@ -281,16 +281,16 @@ impl UTHarness { pub(crate) async fn process_replica_timeout( &mut self, ctx: &ctx::Ctx, - ) -> Signed { + ) -> validator::Signed { self.replica.start_new_view(ctx).await.unwrap(); self.try_recv().unwrap() } - pub(crate) fn leader_phase(&self) -> Phase { + pub(crate) fn leader_phase(&self) -> validator::Phase { self.leader.phase } - pub(crate) fn view_leader(&self, view: ViewNumber) -> validator::PublicKey { + pub(crate) fn view_leader(&self, view: validator::ViewNumber) -> validator::PublicKey { self.genesis().view_leader(view) } @@ -298,10 +298,13 @@ impl UTHarness { self.replica.config.genesis() } - pub(crate) fn new_commit_qc(&self, mutate_fn: impl FnOnce(&mut ReplicaCommit)) -> CommitQC { + pub(crate) fn new_commit_qc( + &self, + mutate_fn: impl FnOnce(&mut validator::ReplicaCommit), + ) -> validator::CommitQC { let mut msg = self.new_current_replica_commit(); mutate_fn(&mut msg); - let mut qc = CommitQC::new(msg, self.genesis()); + let mut qc = validator::CommitQC::new(msg, self.genesis()); for key in &self.keys { qc.add(&key.sign_msg(qc.message.clone()), self.genesis()) .unwrap(); @@ -311,18 +314,18 @@ impl UTHarness { pub(crate) fn new_prepare_qc( &mut self, - mutate_fn: impl FnOnce(&mut ReplicaPrepare), - ) -> PrepareQC { + mutate_fn: impl FnOnce(&mut validator::ReplicaPrepare), + ) -> validator::PrepareQC { let mut msg = self.new_replica_prepare(); mutate_fn(&mut msg); - let mut qc = PrepareQC::new(msg.view.clone()); + let mut qc = validator::PrepareQC::new(msg.view.clone()); for key in &self.keys { qc.add(&key.sign_msg(msg.clone()), self.genesis()).unwrap(); } qc } - pub(crate) fn leader_send(&self, msg: Signed) { + pub(crate) fn leader_send(&self, msg: validator::Signed) { self.leader_send.send(network::io::ConsensusReq { msg, ack: zksync_concurrency::oneshot::channel().0, diff --git a/node/actors/bft/src/tests.rs b/node/actors/bft/src/tests.rs index 2579b5ad..13e0dd1a 100644 --- a/node/actors/bft/src/tests.rs +++ b/node/actors/bft/src/tests.rs @@ -374,7 +374,7 @@ async fn run_twins( LeaderSelectionMode::Rota(scenario.rounds.iter().map(|rc| rc.leader.clone()).collect()); // Generate a new setup with this leadership schedule. - let setup = Setup::from(spec.clone()); + let setup = Setup::from_spec(rng, spec.clone()); // Create a network with the partition schedule of the scenario. let splits: PortSplitSchedule = scenario @@ -424,7 +424,7 @@ async fn run_twins( nodes: nodes.clone(), blocks_to_finalize, } - .run_with_config(ctx, nets.clone(), &setup.genesis) + .run_with_config(ctx, nets.clone(), &setup) .await? } @@ -581,7 +581,7 @@ async fn run_with_custom_router( .collect(), ); - let setup: Setup = spec.into(); + let setup = Setup::from_spec(rng, spec); let port_to_id = nets .iter() @@ -605,6 +605,6 @@ async fn run_with_custom_router( nodes, blocks_to_finalize, } - .run_with_config(ctx, nets, &setup.genesis) + .run_with_config(ctx, nets, &setup) .await } diff --git a/node/actors/executor/src/lib.rs b/node/actors/executor/src/lib.rs index 51f3a335..0bf64484 100644 --- a/node/actors/executor/src/lib.rs +++ b/node/actors/executor/src/lib.rs @@ -10,7 +10,7 @@ use zksync_concurrency::{ctx, limiter, net, scope, time}; use zksync_consensus_bft as bft; use zksync_consensus_network as network; use zksync_consensus_roles::{node, validator}; -use zksync_consensus_storage::{BatchStore, BlockStore, ReplicaStore}; +use zksync_consensus_storage::{BlockStore, ReplicaStore}; use zksync_consensus_utils::pipe; use zksync_protobuf::kB; @@ -42,9 +42,6 @@ pub struct Config { pub public_addr: net::Host, /// Maximal size of the block payload. pub max_payload_size: usize, - /// Maximal size of a batch, which includes `max_payload_size` per block in the batch, - /// plus the size of the Merkle proof of the commitment being included on L1. - pub max_batch_size: usize, /// Key of this node. It uniquely identifies the node. /// It should match the secret key provided in the `node_key` file. pub node_key: node::SecretKey, @@ -63,9 +60,6 @@ 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 { @@ -87,8 +81,6 @@ pub struct Executor { pub config: Config, /// Block storage used by the node. pub block_store: Arc, - /// Batch storage used by the node. - pub batch_store: Arc, /// Validator-specific node data. pub validator: Option, /// Attestation controller. Caller should actively configure the batch @@ -107,13 +99,13 @@ impl Executor { validator_key: self.validator.as_ref().map(|v| v.key.clone()), ping_timeout: Some(time::Duration::seconds(10)), max_block_size: self.config.max_payload_size.saturating_add(kB), - max_batch_size: self.config.max_batch_size.saturating_add(kB), max_block_queue_size: 20, tcp_accept_rate: limiter::Rate { burst: 10, refresh: time::Duration::milliseconds(100), }, rpc: self.config.rpc.clone(), + enable_pregenesis: true, } } @@ -136,7 +128,6 @@ impl Executor { let (net, runner) = network::Network::new( network_config, self.block_store.clone(), - self.batch_store.clone(), network_actor_pipe, self.attestation, ); diff --git a/node/actors/executor/src/tests.rs b/node/actors/executor/src/tests.rs index 7fbde0ea..e6ac8861 100644 --- a/node/actors/executor/src/tests.rs +++ b/node/actors/executor/src/tests.rs @@ -18,14 +18,12 @@ fn config(cfg: &network::Config) -> Config { server_addr: *cfg.server_addr, public_addr: cfg.public_addr.clone(), max_payload_size: usize::MAX, - max_batch_size: usize::MAX, node_key: cfg.gossip.key.clone(), gossip_dynamic_inbound_limit: cfg.gossip.dynamic_inbound_limit, gossip_static_inbound: cfg.gossip.static_inbound.clone(), gossip_static_outbound: cfg.gossip.static_outbound.clone(), rpc: cfg.rpc.clone(), debug_page: None, - batch_poll_interval: time::Duration::seconds(1), } } @@ -38,13 +36,11 @@ fn never_attest() -> Arc { fn validator( cfg: &network::Config, block_store: Arc, - batch_store: Arc, replica_store: impl ReplicaStore, ) -> Executor { Executor { config: config(cfg), block_store, - batch_store, validator: Some(Validator { key: cfg.validator_key.clone().unwrap(), replica_store: Box::new(replica_store), @@ -54,15 +50,10 @@ fn validator( } } -fn fullnode( - cfg: &network::Config, - block_store: Arc, - batch_store: Arc, -) -> Executor { +fn fullnode(cfg: &network::Config, block_store: Arc) -> Executor { Executor { config: config(cfg), block_store, - batch_store, validator: None, attestation: never_attest(), } @@ -78,17 +69,9 @@ async fn test_single_validator() { let cfgs = new_configs(rng, &setup, 0); scope::run!(ctx, |ctx, s| async { let replica_store = in_memory::ReplicaStore::default(); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - s.spawn_bg( - validator( - &cfgs[0], - store.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx), - ); + s.spawn_bg(validator(&cfgs[0], store.blocks.clone(), replica_store).run(ctx)); store .blocks .wait_until_persisted(ctx, BlockNumber(5)) @@ -110,17 +93,9 @@ async fn test_many_validators() { scope::run!(ctx, |ctx, s| async { for cfg in cfgs { let replica_store = in_memory::ReplicaStore::default(); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - s.spawn_bg( - validator( - &cfg, - store.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx), - ); + s.spawn_bg(validator(&cfg, store.blocks.clone(), replica_store).run(ctx)); // Spawn a task waiting for blocks to get finalized and delivered to this validator. s.spawn(async { @@ -146,34 +121,18 @@ async fn test_inactive_validator() { scope::run!(ctx, |ctx, s| async { // Spawn validator. let replica_store = in_memory::ReplicaStore::default(); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - s.spawn_bg( - validator( - &cfgs[0], - store.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx), - ); + s.spawn_bg(validator(&cfgs[0], store.blocks.clone(), replica_store).run(ctx)); // Spawn a validator node, which doesn't belong to the consensus. // Therefore it should behave just like a fullnode. - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let mut cfg = new_fullnode(rng, &cfgs[0]); cfg.validator_key = Some(rng.gen()); let replica_store = in_memory::ReplicaStore::default(); - s.spawn_bg( - validator( - &cfg, - store.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx), - ); + s.spawn_bg(validator(&cfg, store.blocks.clone(), replica_store).run(ctx)); // Wait for blocks in inactive validator's store. store @@ -197,29 +156,14 @@ async fn test_fullnode_syncing_from_validator() { scope::run!(ctx, |ctx, s| async { // Spawn validator. let replica_store = in_memory::ReplicaStore::default(); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - s.spawn_bg( - validator( - &cfgs[0], - store.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx), - ); + s.spawn_bg(validator(&cfgs[0], store.blocks.clone(), replica_store).run(ctx)); // Spawn full node. - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - s.spawn_bg( - fullnode( - &new_fullnode(rng, &cfgs[0]), - store.blocks.clone(), - store.batches.clone(), - ) - .run(ctx), - ); + s.spawn_bg(fullnode(&new_fullnode(rng, &cfgs[0]), store.blocks.clone()).run(ctx)); // Wait for blocks in full node store. store @@ -244,18 +188,13 @@ async fn test_validator_syncing_from_fullnode() { scope::run!(ctx, |ctx, s| async { // Run validator and produce some blocks. let replica_store = in_memory::ReplicaStore::default(); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); scope::run!(ctx, |ctx, s| async { s.spawn_bg( - validator( - &cfgs[0], - store.blocks.clone(), - store.batches.clone(), - replica_store.clone(), - ) - .run(ctx) - .instrument(tracing::info_span!("validator")), + validator(&cfgs[0], store.blocks.clone(), replica_store.clone()) + .run(ctx) + .instrument(tracing::info_span!("validator")), ); store .blocks @@ -271,35 +210,26 @@ async fn test_validator_syncing_from_fullnode() { // validator, there would be a race condition between fullnode syncing and validator // terminating. s.spawn_bg( - fullnode( - &new_fullnode(rng, &cfgs[0]), - store.blocks.clone(), - store.batches.clone(), - ) - .run(ctx) - .instrument(tracing::info_span!("fullnode")), + fullnode(&new_fullnode(rng, &cfgs[0]), store.blocks.clone()) + .run(ctx) + .instrument(tracing::info_span!("fullnode")), ); // Restart the validator with empty store (but preserved replica state) and non-trivial // `store.state.first`. // Validator should fetch the past blocks from the full node before producing next blocks. - let last_block = store.blocks.queued().last.as_ref().unwrap().header().number; + let last_block = store.blocks.queued().last.as_ref().unwrap().number(); let store2 = TestMemoryStorage::new_store_with_first_block( ctx, - &setup.genesis, + &setup, setup.genesis.first_block + 2, ) .await; s.spawn_bg(store2.runner.run(ctx)); s.spawn_bg( - validator( - &cfgs[0], - store2.blocks.clone(), - store.batches.clone(), - replica_store, - ) - .run(ctx) - .instrument(tracing::info_span!("validator")), + validator(&cfgs[0], store2.blocks.clone(), replica_store) + .run(ctx) + .instrument(tracing::info_span!("validator")), ); // Wait for the fullnode to fetch the new blocks. diff --git a/node/actors/network/src/config.rs b/node/actors/network/src/config.rs index 1f723b31..3a25be62 100644 --- a/node/actors/network/src/config.rs +++ b/node/actors/network/src/config.rs @@ -103,8 +103,6 @@ pub struct Config { pub validator_key: Option, /// Maximal size of the proto-encoded `validator::FinalBlock` in bytes. pub max_block_size: usize, - /// Maximal size of the proto-encoded `attester::SyncBatch` in bytes. - pub max_batch_size: usize, /// If a peer doesn't respond to a ping message within `ping_timeout`, /// the connection is dropped. /// `None` disables sending ping messages (useful for tests). @@ -113,6 +111,10 @@ pub struct Config { pub tcp_accept_rate: limiter::Rate, /// Rate limiting config for RPCs. pub rpc: RpcConfig, + /// Enables syncing blocks before genesis. + /// In production, it should be set to `true`. + /// Set it to `false` in tests to simulate node behavior before pre-genesis support. + pub enable_pregenesis: bool, /// Maximum number of not-yet-persisted blocks fetched from the network. /// If reached, network actor will wait for more blocks to get persisted /// before fetching the next ones. It is useful for limiting memory consumption diff --git a/node/actors/network/src/consensus/tests.rs b/node/actors/network/src/consensus/tests.rs index 052aea13..dc63d4ee 100644 --- a/node/actors/network/src/consensus/tests.rs +++ b/node/actors/network/src/consensus/tests.rs @@ -97,10 +97,10 @@ async fn test_one_connection_per_validator() { let nodes = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx,s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let nodes : Vec<_> = nodes.into_iter().enumerate().map(|(i,node)| { - let (node,runner) = testonly::Instance::new(node, store.blocks.clone(), store.batches.clone()); + let (node,runner) = testonly::Instance::new(node, store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }).collect(); @@ -156,10 +156,9 @@ async fn test_genesis_mismatch() { let mut listener = cfgs[1].server_addr.bind().context("server_addr.bind()")?; tracing::info!("Start one node, we will simulate the other one."); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = - testonly::Instance::new(cfgs[0].clone(), store.blocks.clone(), store.batches.clone()); + let (node, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); tracing::info!("Populate the validator_addrs of the running node."); @@ -223,17 +222,13 @@ async fn test_address_change() { let setup = validator::testonly::Setup::new(rng, 5); let mut cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let mut nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (node, runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) @@ -257,8 +252,7 @@ async fn test_address_change() { 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()); + let (node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node0"))); nodes[0] = node0; @@ -283,17 +277,13 @@ async fn test_transmission() { let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let mut nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (node, runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone()); let i = ctx::NoCopy(i); s.spawn_bg(async { let i = i; @@ -353,12 +343,11 @@ async fn test_retransmission() { let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); // Spawn the first node. - let (node0, runner) = - testonly::Instance::new(cfgs[0].clone(), store.blocks.clone(), store.batches.clone()); + let (node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx)); // Make first node broadcast a message. @@ -376,11 +365,8 @@ async fn test_retransmission() { for i in 0..2 { tracing::info!("iteration {i}"); scope::run!(ctx, |ctx, s| async { - let (mut node1, runner) = testonly::Instance::new( - cfgs[1].clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (mut node1, runner) = + testonly::Instance::new(cfgs[1].clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx)); loop { if let io::OutputMessage::Consensus(got) = node1.pipe.recv(ctx).await.unwrap() { diff --git a/node/actors/network/src/gossip/fetch.rs b/node/actors/network/src/gossip/fetch.rs index ea4f0d7e..f74e3e5a 100644 --- a/node/actors/network/src/gossip/fetch.rs +++ b/node/actors/network/src/gossip/fetch.rs @@ -3,35 +3,29 @@ use anyhow::Context as _; use std::collections::BTreeMap; use zksync_concurrency::{ctx, oneshot, scope, sync}; use zksync_consensus_roles::{attester, validator}; -use zksync_consensus_storage::{BatchStoreState, BlockStoreState}; +use zksync_consensus_storage::BlockStoreState; /// A block fetching request. type BlockCall = (validator::BlockNumber, oneshot::Sender<()>); -type BatchCall = (attester::BatchNumber, oneshot::Sender<()>); /// Inner state of the `Queue`. type BlockInner = BTreeMap>; -type BatchInner = BTreeMap>; /// A request for a given resource. pub(crate) enum RequestItem { /// Request for a block by number. Block(validator::BlockNumber), - /// Request for a batch by number. - Batch(attester::BatchNumber), } /// Queue of block fetch request. pub(crate) struct Queue { blocks: sync::watch::Sender, - batches: sync::watch::Sender, } impl Default for Queue { fn default() -> Self { Self { blocks: sync::watch::channel(BlockInner::default()).0, - batches: sync::watch::channel(BatchInner::default()).0, } } } @@ -62,11 +56,6 @@ impl Queue { // Send iff the lowest requested block changed. x.first_key_value().unwrap().0 == &n }), - RequestItem::Batch(n) => self.batches.send_if_modified(|x| { - x.insert(n, send); - // Send iff the lowest requested batch changed. - x.first_key_value().unwrap().0 == &n - }), }; match recv.recv_or_disconnected(ctx).await { // Return if completed. @@ -82,12 +71,6 @@ impl Queue { // Send iff the lowest requested block changed. modified }), - RequestItem::Batch(n) => self.batches.send_if_modified(|x| { - let modified = x.first_key_value().map_or(false, |(k, _)| k == &n); - x.remove(&n); - // Send iff the lowest requested batch changed. - modified - }), }; return Err(ctx::Canceled); } @@ -141,51 +124,4 @@ impl Queue { } Err(ctx::Canceled) } - - /// Accepts a batch fetch request, which is contained in the available batches range. - /// Caller is responsible for fetching the batch and adding it to the batch store. - pub(crate) async fn accept_batch( - &self, - ctx: &ctx::Ctx, - available: &mut sync::watch::Receiver, - ) -> ctx::OrCanceled { - let sub = &mut self.batches.subscribe(); - while ctx.is_active() { - // Wait for the lowest requested block to be available. - // This scope is always cancelled, so we ignore the result. - let mut batch_number = None; - let _: Result<(), _> = scope::run!(ctx, |ctx, s| async { - if let Some(n) = sub.borrow_and_update().first_key_value().map(|x| *x.0) { - let n = ctx::NoCopy(n); - s.spawn::<()>(async { - let n = n; - sync::wait_for(ctx, available, |a| a.contains(n.0)).await?; - batch_number = Some(n.0); - Err(ctx::Canceled) - }); - } - // If the lowest requested block changes, we need to restart the wait. - sync::changed(ctx, sub).await?; - Err(ctx::Canceled) - }) - .await; - let Some(batch_number) = batch_number else { - continue; - }; - - // Remove the request from the queue. - let mut res = None; - self.batches.send_if_modified(|x| { - res = x.remove_entry(&batch_number); - // Send iff the lowest requested block changed. - res.is_some() && !x.is_empty() - }); - // It may happen that someone else accepts our request faster. - // In this case we need to wait again. - if let Some(res) = res { - return Ok(res); - } - } - Err(ctx::Canceled) - } } diff --git a/node/actors/network/src/gossip/loadtest/mod.rs b/node/actors/network/src/gossip/loadtest/mod.rs index 4caca21b..081440f0 100644 --- a/node/actors/network/src/gossip/loadtest/mod.rs +++ b/node/actors/network/src/gossip/loadtest/mod.rs @@ -29,7 +29,7 @@ impl<'a> PushBlockStoreStateServer { let state = sync::wait_for(ctx, sub, |s| (|| s.as_ref()?.last.as_ref())().is_some()).await?; let state = state.as_ref().unwrap(); - Ok(state.first..state.last.as_ref().unwrap().header().number + 1) + Ok(state.first..state.last.as_ref().unwrap().number() + 1) } } @@ -43,7 +43,7 @@ impl rpc::Handler for &PushBlockStoreStateServ _ctx: &ctx::Ctx, req: rpc::push_block_store_state::Req, ) -> anyhow::Result<()> { - self.0.send_replace(Some(req.0)); + self.0.send_replace(Some(req.state())); Ok(()) } } @@ -76,7 +76,7 @@ pub struct Loadtest { /// Traffic pattern to generate. pub traffic_pattern: TrafficPattern, /// Channel to send the received responses to. - pub output: Option>>, + pub output: Option>>, } impl Loadtest { diff --git a/node/actors/network/src/gossip/loadtest/tests.rs b/node/actors/network/src/gossip/loadtest/tests.rs index b4c40be9..80a35665 100644 --- a/node/actors/network/src/gossip/loadtest/tests.rs +++ b/node/actors/network/src/gossip/loadtest/tests.rs @@ -17,10 +17,9 @@ async fn test_loadtest() { scope::run!(ctx, |ctx, s| async { // Spawn the node. - let stores = TestMemoryStorage::new(ctx, &setup.genesis).await; + let stores = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(stores.runner.run(ctx)); - let (node, runner) = - testonly::Instance::new(cfg.clone(), stores.blocks.clone(), stores.batches.clone()); + let (node, runner) = testonly::Instance::new(cfg.clone(), stores.blocks.clone()); s.spawn_bg(runner.run(ctx)); // Fill the storage with some blocks. diff --git a/node/actors/network/src/gossip/mod.rs b/node/actors/network/src/gossip/mod.rs index 126d887c..814f6b35 100644 --- a/node/actors/network/src/gossip/mod.rs +++ b/node/actors/network/src/gossip/mod.rs @@ -20,7 +20,7 @@ use tracing::Instrument; pub(crate) use validator_addrs::*; use zksync_concurrency::{ctx, ctx::channel, scope, sync}; use zksync_consensus_roles::{node, validator}; -use zksync_consensus_storage::{BatchStore, BlockStore}; +use zksync_consensus_storage::BlockStore; pub mod attestation; mod fetch; @@ -56,8 +56,6 @@ pub(crate) struct Network { pub(crate) validator_addrs: ValidatorAddrsWatch, /// Block store to serve `get_block` requests from. pub(crate) block_store: Arc, - /// Batch store to serve `get_batch` requests from. - pub(crate) batch_store: Arc, /// Output pipe of the network actor. pub(crate) sender: channel::UnboundedSender, /// Queue of block fetching requests. @@ -77,7 +75,6 @@ impl Network { pub(crate) fn new( cfg: Config, block_store: Arc, - batch_store: Arc, sender: channel::UnboundedSender, attestation: Arc, ) -> Arc { @@ -92,7 +89,6 @@ impl Network { cfg, fetch_queue: fetch::Queue::default(), block_store, - batch_store, push_validator_addrs_calls: 0.into(), attestation, }) @@ -139,41 +135,4 @@ impl Network { }) .await; } - - /// Task fetching batches from peers which are not present in storage. - pub(crate) async fn run_batch_fetcher(&self, ctx: &ctx::Ctx) { - let sem = sync::Semaphore::new(self.cfg.max_block_queue_size); - let _: ctx::OrCanceled<()> = scope::run!(ctx, |ctx, s| async { - let mut next = self.batch_store.queued().next(); - loop { - let permit = sync::acquire(ctx, &sem).await?; - 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)) - .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; - } } diff --git a/node/actors/network/src/gossip/runner.rs b/node/actors/network/src/gossip/runner.rs index 66cf8fd8..bfb05e5a 100644 --- a/node/actors/network/src/gossip/runner.rs +++ b/node/actors/network/src/gossip/runner.rs @@ -5,14 +5,13 @@ use async_trait::async_trait; use rand::seq::SliceRandom; use std::sync::atomic::Ordering; use zksync_concurrency::{ctx, net, scope, sync}; -use zksync_consensus_roles::{attester::BatchNumber, node}; -use zksync_consensus_storage::{BatchStore, BatchStoreState, BlockStore, BlockStoreState}; +use zksync_consensus_roles::node; +use zksync_consensus_storage::BlockStoreState; use zksync_protobuf::kB; /// Receiver of push messages from the peers. struct PushServer<'a> { blocks: sync::watch::Sender, - batches: sync::watch::Sender, net: &'a Network, } @@ -24,11 +23,6 @@ impl<'a> PushServer<'a> { last: None, }) .0, - batches: sync::watch::channel(BatchStoreState { - first: BatchNumber(0), - last: None, - }) - .0, net, } } @@ -96,32 +90,20 @@ impl rpc::Handler for &PushServer<'_> { async fn handle( &self, _ctx: &ctx::Ctx, - req: rpc::push_block_store_state::Req, - ) -> anyhow::Result<()> { - req.0.verify(self.net.genesis())?; - self.blocks.send_replace(req.0); - Ok(()) - } -} - -#[async_trait] -impl rpc::Handler for &PushServer<'_> { - fn max_req_size(&self) -> usize { - 10 * kB - } - async fn handle( - &self, - _ctx: &ctx::Ctx, - req: rpc::push_batch_store_state::Req, + mut req: rpc::push_block_store_state::Req, ) -> anyhow::Result<()> { - req.0.verify()?; - self.batches.send_replace(req.0); + if !self.net.cfg.enable_pregenesis { + req.clear_pregenesis_data(); + } + let state = req.state(); + state.verify(self.net.genesis())?; + self.blocks.send_replace(state); Ok(()) } } #[async_trait] -impl rpc::Handler for &BlockStore { +impl rpc::Handler for &Network { fn max_req_size(&self) -> usize { kB } @@ -130,21 +112,11 @@ impl rpc::Handler for &BlockStore { ctx: &ctx::Ctx, req: rpc::get_block::Req, ) -> anyhow::Result { - Ok(rpc::get_block::Resp(self.block(ctx, req.0).await?)) - } -} - -#[async_trait] -impl rpc::Handler for &BatchStore { - fn max_req_size(&self) -> usize { - kB - } - async fn handle( - &self, - ctx: &ctx::Ctx, - req: rpc::get_batch::Req, - ) -> anyhow::Result { - Ok(rpc::get_batch::Resp(self.batch(ctx, req.0).await?)) + let mut resp = rpc::get_block::Resp(self.block_store.block(ctx, req.0).await?); + if !self.cfg.enable_pregenesis { + resp.clear_pregenesis_data(); + } + Ok(resp) } } @@ -160,14 +132,8 @@ impl Network { ctx, self.cfg.rpc.push_block_store_state_rate, ); - let push_batch_store_state_client = rpc::Client::::new( - ctx, - self.cfg.rpc.push_batch_store_state_rate, - ); let get_block_client = rpc::Client::::new(ctx, self.cfg.rpc.get_block_rate); - let get_batch_client = - rpc::Client::::new(ctx, self.cfg.rpc.get_batch_rate); let push_batch_votes_client = rpc::Client::::new(ctx, self.cfg.rpc.push_batch_votes_rate); @@ -180,21 +146,13 @@ impl Network { self.cfg.rpc.push_validator_addrs_rate, ) .add_client(&push_block_store_state_client) - .add_client(&push_batch_store_state_client) .add_server::( ctx, &push_server, self.cfg.rpc.push_block_store_state_rate, ) - .add_server::( - ctx, - &push_server, - self.cfg.rpc.push_batch_store_state_rate, - ) .add_client(&get_block_client) - .add_server(ctx, &*self.block_store, self.cfg.rpc.get_block_rate) - .add_client(&get_batch_client) - .add_server(ctx, &*self.batch_store, self.cfg.rpc.get_batch_rate) + .add_server::(ctx, self, self.cfg.rpc.get_block_rate) .add_server(ctx, rpc::ping::Server, rpc::ping::RATE) .add_client(&push_batch_votes_client) .add_server::( @@ -254,25 +212,16 @@ impl Network { s.spawn::<()>(async { let mut state = self.block_store.queued(); loop { - let req = rpc::push_block_store_state::Req(state.clone()); + let mut req = + rpc::push_block_store_state::Req::new(state.clone(), self.genesis()); + if !self.cfg.enable_pregenesis { + req.clear_pregenesis_data(); + } push_block_store_state_client.call(ctx, &req, kB).await?; state = self.block_store.wait_for_queued_change(ctx, &state).await?; } }); - // Push batch store state updates to peer. - s.spawn::<()>(async { - let mut state = self.batch_store.queued(); - loop { - let req = rpc::push_batch_store_state::Req(state.clone()); - push_batch_store_state_client.call(ctx, &req, kB).await?; - state = self - .batch_store - .wait_until_queued(ctx, state.next()) - .await?; - } - }); - // Push validator addrs updates to peer. s.spawn::<()>(async { let mut old = ValidatorAddrs::default(); @@ -313,11 +262,13 @@ impl Network { let ctx_with_timeout = self.cfg.rpc.get_block_timeout.map(|t| ctx.with_timeout(t)); let ctx = ctx_with_timeout.as_ref().unwrap_or(ctx); - let block = call + let mut resp = call .call(ctx, &req, self.cfg.max_block_size.saturating_add(kB)) - .await? - .0 - .context("empty response")?; + .await?; + if !self.cfg.enable_pregenesis { + resp.clear_pregenesis_data(); + } + let block = resp.0.context("empty response")?; anyhow::ensure!(block.number() == req.0, "received wrong block"); // Storing the block will fail in case block is invalid. self.block_store @@ -336,51 +287,6 @@ impl Network { } }); - // Perform get_batch calls to peer. - s.spawn::<()>(async { - let state = &mut push_server.batches.subscribe(); - loop { - let call = get_batch_client.reserve(ctx).await?; - let (req, send_resp) = self.fetch_queue.accept_batch(ctx, state).await?; - let req = rpc::get_batch::Req(req); - s.spawn(async { - let req = req; - // Failing to fetch a batch causes a disconnect: - // - peer predeclares which batches are available and race condition - // with batch pruning should be very rare, so we can consider - // an empty response to be offending - // - a stream for the call has been already reserved, - // so the peer is expected to answer immediately. The timeout - // should be high enough to accommodate network hiccups - // - a disconnect is not a ban, so the peer is free to try to - // reconnect. - async { - let ctx_with_timeout = - self.cfg.rpc.get_batch_timeout.map(|t| ctx.with_timeout(t)); - let ctx = ctx_with_timeout.as_ref().unwrap_or(ctx); - let batch = call - .call(ctx, &req, self.cfg.max_batch_size.saturating_add(kB)) - .await? - .0 - .context("empty response")?; - anyhow::ensure!(batch.number == req.0, "received wrong batch"); - // Storing the batch will fail in case batch is invalid. - self.batch_store - .queue_batch(ctx, batch, self.genesis().clone()) - .await - .context("queue_batch()")?; - tracing::info!("fetched batch {}", req.0); - // Send a response that fetching was successful. - // Ignore disconnection error. - let _ = send_resp.send(()); - anyhow::Ok(()) - } - .await - .with_context(|| format!("get_batch({})", req.0)) - }); - } - }); - service.run(ctx, stream).await?; Ok(()) }) diff --git a/node/actors/network/src/gossip/testonly.rs b/node/actors/network/src/gossip/testonly.rs index a2745d25..46b75894 100644 --- a/node/actors/network/src/gossip/testonly.rs +++ b/node/actors/network/src/gossip/testonly.rs @@ -56,15 +56,11 @@ pub(super) async fn connect( accept: [ mux_entry::(ctx), mux_entry::(ctx), - mux_entry::(ctx), - mux_entry::(ctx), ] .into(), connect: [ mux_entry::(ctx), mux_entry::(ctx), - mux_entry::(ctx), - mux_entry::(ctx), ] .into(), }; diff --git a/node/actors/network/src/gossip/tests/fetch_batches.rs b/node/actors/network/src/gossip/tests/fetch_batches.rs deleted file mode 100644 index eb169278..00000000 --- a/node/actors/network/src/gossip/tests/fetch_batches.rs +++ /dev/null @@ -1,330 +0,0 @@ -//! Unit tests of `get_batch` RPC. -use crate::{gossip, mux, rpc}; -use assert_matches::assert_matches; -use tracing::Instrument as _; -use zksync_concurrency::{ctx, limiter, scope, testonly::abort_on_panic}; -use zksync_consensus_roles::validator; -use zksync_consensus_storage::{testonly::TestMemoryStorage, BatchStoreState}; - -#[tokio::test] -async fn test_simple() { - abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, 1); - setup.push_batches(rng, 2); - let mut cfg = crate::testonly::new_configs(rng, &setup, 0)[0].clone(); - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - - 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(), - store.batches.clone(), - ); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - - let (conn, runner) = gossip::testonly::connect(ctx, &cfg, setup.genesis.hash()) - .await - .unwrap(); - s.spawn_bg(async { - assert_matches!(runner.run(ctx).await, Err(mux::RunError::Canceled(_))); - Ok(()) - }); - - tracing::info!("Store is empty so requesting a batch should return an empty response."); - let mut stream = conn.open_client::(ctx).await.unwrap(); - stream - .send(ctx, &rpc::get_batch::Req(setup.batches[1].number)) - .await - .unwrap(); - let resp = stream.recv(ctx).await.unwrap(); - assert_eq!(resp.0, None); - - tracing::info!("Insert a batch."); - store - .batches - .queue_batch(ctx, setup.batches[0].clone(), setup.genesis.clone()) - .await - .unwrap(); - loop { - let mut stream = conn - .open_server::(ctx) - .await - .unwrap(); - let state = stream.recv(ctx).await.unwrap(); - stream.send(ctx, &()).await.unwrap(); - if state.0.contains(setup.batches[0].number) { - tracing::info!("peer reported to have a batch"); - break; - } - } - tracing::info!("fetch that batch."); - - let mut stream = conn.open_client::(ctx).await.unwrap(); - stream - .send(ctx, &rpc::get_batch::Req(setup.batches[0].number)) - .await - .unwrap(); - let resp = stream.recv(ctx).await.unwrap(); - assert_eq!(resp.0, Some(setup.batches[0].clone())); - - tracing::info!("Inform the peer that we have {}", setup.batches[1].number); - let mut stream = conn - .open_client::(ctx) - .await - .unwrap(); - stream - .send( - ctx, - &rpc::push_batch_store_state::Req(BatchStoreState { - first: setup.batches[1].number, - last: Some(setup.batches[1].number), - }), - ) - .await - .unwrap(); - stream.recv(ctx).await.unwrap(); - - tracing::info!("Wait for the client to request that batch"); - let mut stream = conn.open_server::(ctx).await.unwrap(); - let req = stream.recv(ctx).await.unwrap(); - assert_eq!(req.0, setup.batches[1].number); - - tracing::info!("Return the requested batch"); - stream - .send(ctx, &rpc::get_batch::Resp(Some(setup.batches[1].clone()))) - .await - .unwrap(); - - tracing::info!("Wait for the client to store that batch"); - store - .batches - .wait_until_persisted(ctx, setup.batches[1].number) - .await - .unwrap(); - - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn test_concurrent_requests() { - abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, 1); - setup.push_batches(rng, 10); - let mut cfg = crate::testonly::new_configs(rng, &setup, 0)[0].clone(); - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - - 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(), - store.batches.clone(), - ); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - - let mut conns = vec![]; - for _ in 0..4 { - let (conn, runner) = gossip::testonly::connect(ctx, &cfg, setup.genesis.hash()) - .await - .unwrap(); - s.spawn_bg(async { - assert_matches!(runner.run(ctx).await, Err(mux::RunError::Canceled(_))); - Ok(()) - }); - let mut stream = conn - .open_client::(ctx) - .await - .unwrap(); - stream - .send( - ctx, - &rpc::push_batch_store_state::Req(BatchStoreState { - first: setup.batches[0].number, - last: Some(setup.batches.last().unwrap().number), - }), - ) - .await - .unwrap(); - stream.recv(ctx).await.unwrap(); - conns.push(conn); - } - - // Receive a bunch of concurrent requests on various connections. - let mut streams = vec![]; - for (i, batch) in setup.batches.iter().enumerate() { - let mut stream = conns[i % conns.len()] - .open_server::(ctx) - .await - .unwrap(); - let req = stream.recv(ctx).await.unwrap(); - assert_eq!(req.0, batch.number); - streams.push(stream); - } - - // Respond to the requests. - for (i, stream) in streams.into_iter().enumerate() { - stream - .send(ctx, &rpc::get_batch::Resp(Some(setup.batches[i].clone()))) - .await - .unwrap(); - } - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn test_bad_responses() { - abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, 1); - setup.push_batches(rng, 2); - let mut cfg = crate::testonly::new_configs(rng, &setup, 0)[0].clone(); - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - - 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(), - store.batches.clone(), - ); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - - let state = rpc::push_batch_store_state::Req(BatchStoreState { - first: setup.batches[0].number, - last: Some(setup.batches[0].number), - }); - - for resp in [ - // Empty response even though we declared to have the batch. - None, - // Wrong batch. - Some(setup.batches[1].clone()), - // // Malformed batch. - // { - // let mut b = setup.batches[0].clone(); - // b.justification = rng.gen(); - // Some(b) - // }, - ] { - tracing::info!("bad response = {resp:?}"); - - tracing::info!("Connect to peer"); - let (conn, runner) = gossip::testonly::connect(ctx, &cfg, setup.genesis.hash()) - .await - .unwrap(); - let conn_task = s.spawn_bg(async { Ok(runner.run(ctx).await) }); - - tracing::info!("Inform the peer about the batch that we possess"); - let mut stream = conn - .open_client::(ctx) - .await - .unwrap(); - stream.send(ctx, &state).await.unwrap(); - stream.recv(ctx).await.unwrap(); - - tracing::info!("Wait for the client to request that batch"); - let mut stream = conn.open_server::(ctx).await.unwrap(); - let req = stream.recv(ctx).await.unwrap(); - assert_eq!(req.0, setup.batches[0].number); - - tracing::info!("Return a bad response"); - stream.send(ctx, &rpc::get_batch::Resp(resp)).await.unwrap(); - - tracing::info!("Wait for the peer to drop the connection"); - assert_matches!( - conn_task.join(ctx).await.unwrap(), - Err(mux::RunError::Closed) - ); - } - Ok(()) - }) - .await - .unwrap(); -} - -#[tokio::test] -async fn test_retry() { - abort_on_panic(); - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, 1); - setup.push_batches(rng, 1); - let mut cfg = crate::testonly::new_configs(rng, &setup, 0)[0].clone(); - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - - 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(), - store.batches.clone(), - ); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - - let state = rpc::push_batch_store_state::Req(BatchStoreState { - first: setup.batches[0].number, - last: Some(setup.batches[0].number), - }); - - tracing::info!("establish a bunch of connections"); - let mut conns = vec![]; - for _ in 0..4 { - let (conn, runner) = gossip::testonly::connect(ctx, &cfg, setup.genesis.hash()) - .await - .unwrap(); - let task = s.spawn_bg(async { Ok(runner.run(ctx).await) }); - let mut stream = conn - .open_client::(ctx) - .await - .unwrap(); - stream.send(ctx, &state).await.unwrap(); - stream.recv(ctx).await.unwrap(); - conns.push((conn, task)); - } - - for (conn, task) in conns { - tracing::info!("Wait for the client to request a batch"); - let mut stream = conn.open_server::(ctx).await.unwrap(); - let req = stream.recv(ctx).await.unwrap(); - assert_eq!(req.0, setup.batches[0].number); - - tracing::info!("Return a bad response"); - stream.send(ctx, &rpc::get_batch::Resp(None)).await.unwrap(); - - tracing::info!("Wait for the peer to drop the connection"); - assert_matches!(task.join(ctx).await.unwrap(), Err(mux::RunError::Closed)); - } - - Ok(()) - }) - .await - .unwrap(); -} diff --git a/node/actors/network/src/gossip/tests/fetch_blocks.rs b/node/actors/network/src/gossip/tests/fetch_blocks.rs index 79a310f9..19c6512e 100644 --- a/node/actors/network/src/gossip/tests/fetch_blocks.rs +++ b/node/actors/network/src/gossip/tests/fetch_blocks.rs @@ -24,14 +24,10 @@ async fn test_simple() { cfg.validator_key = None; scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (_node, runner) = crate::testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (_node, runner) = crate::testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); let (conn, runner) = gossip::testonly::connect(ctx, &cfg, setup.genesis.hash()) @@ -62,9 +58,9 @@ async fn test_simple() { .open_server::(ctx) .await .unwrap(); - let state = stream.recv(ctx).await.unwrap(); + let resp = stream.recv(ctx).await.unwrap(); stream.send(ctx, &()).await.unwrap(); - if state.0.contains(setup.blocks[0].number()) { + if resp.state().contains(setup.blocks[0].number()) { tracing::info!("peer reported to have a block"); break; } @@ -86,10 +82,13 @@ async fn test_simple() { stream .send( ctx, - &rpc::push_block_store_state::Req(BlockStoreState { - first: setup.blocks[1].number(), - last: Some(setup.blocks[1].justification.clone()), - }), + &rpc::push_block_store_state::Req::new( + BlockStoreState { + first: setup.blocks[1].number(), + last: Some((&setup.blocks[1]).into()), + }, + &setup.genesis, + ), ) .await .unwrap(); @@ -134,13 +133,9 @@ async fn test_concurrent_requests() { cfg.max_block_queue_size = setup.blocks.len(); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (_node, runner) = crate::testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (_node, runner) = crate::testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); let mut conns = vec![]; @@ -159,10 +154,13 @@ async fn test_concurrent_requests() { stream .send( ctx, - &rpc::push_block_store_state::Req(BlockStoreState { - first: setup.blocks[0].number(), - last: Some(setup.blocks.last().unwrap().justification.clone()), - }), + &rpc::push_block_store_state::Req::new( + BlockStoreState { + first: setup.blocks[0].number(), + last: Some(setup.blocks.last().unwrap().into()), + }, + &setup.genesis, + ), ) .await .unwrap(); @@ -209,19 +207,18 @@ async fn test_bad_responses() { cfg.validator_key = None; scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (_node, runner) = crate::testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (_node, runner) = crate::testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - let state = rpc::push_block_store_state::Req(BlockStoreState { - first: setup.blocks[0].number(), - last: Some(setup.blocks[0].justification.clone()), - }); + let state = rpc::push_block_store_state::Req::new( + BlockStoreState { + first: setup.blocks[0].number(), + last: Some((&setup.blocks[0]).into()), + }, + &setup.genesis, + ); for resp in [ // Empty response even though we declared to have the block. @@ -230,9 +227,11 @@ async fn test_bad_responses() { Some(setup.blocks[1].clone()), // Malformed block. { - let mut b = setup.blocks[0].clone(); + let validator::Block::Final(mut b) = setup.blocks[0].clone() else { + panic!(); + }; b.justification = rng.gen(); - Some(b) + Some(b.into()) }, ] { tracing::info!("bad response = {resp:?}"); @@ -285,19 +284,18 @@ async fn test_retry() { cfg.validator_key = None; scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (_node, runner) = crate::testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (_node, runner) = crate::testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); - let state = rpc::push_block_store_state::Req(BlockStoreState { - first: setup.blocks[0].number(), - last: Some(setup.blocks[0].justification.clone()), - }); + let state = rpc::push_block_store_state::Req::new( + BlockStoreState { + first: setup.blocks[0].number(), + last: Some((&setup.blocks[0]).into()), + }, + &setup.genesis, + ); tracing::info!("establish a bunch of connections"); let mut conns = vec![]; @@ -351,14 +349,10 @@ async fn test_announce_truncated_block_range() { scope::run!(ctx, |ctx, s| async { // Build a custom persistent store, so that we can tweak it later. - let mut persistent = - in_memory::BlockStore::new(setup.genesis.clone(), setup.genesis.first_block); + let mut persistent = in_memory::BlockStore::new(&setup, setup.genesis.first_block); let (block_store, runner) = BlockStore::new(ctx, Box::new(persistent.clone())).await?; s.spawn_bg(runner.run(ctx)); - // Use the standard batch store since it doesn't matter. - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; - let (_node, runner) = - crate::testonly::Instance::new(cfg.clone(), block_store, store.batches); + let (_node, runner) = crate::testonly::Instance::new(cfg.clone(), block_store); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); // Fill in all the blocks. for b in &setup.blocks { @@ -386,9 +380,9 @@ async fn test_announce_truncated_block_range() { let mut stream = conn .open_server::(ctx) .await?; - let state = stream.recv(ctx).await.unwrap(); + let resp = stream.recv(ctx).await.unwrap(); stream.send(ctx, &()).await.unwrap(); - if state.0 == *persistent.persisted().borrow() { + if resp.state() == *persistent.persisted().borrow() { break; } } diff --git a/node/actors/network/src/gossip/tests/mod.rs b/node/actors/network/src/gossip/tests/mod.rs index c72343c1..fa96f6cc 100644 --- a/node/actors/network/src/gossip/tests/mod.rs +++ b/node/actors/network/src/gossip/tests/mod.rs @@ -22,7 +22,6 @@ use zksync_concurrency::{ use zksync_consensus_roles::{attester, validator}; use zksync_consensus_storage::testonly::TestMemoryStorage; -mod fetch_batches; mod fetch_blocks; mod syncing; @@ -36,10 +35,10 @@ async fn test_one_connection_per_node() { let cfgs = testonly::new_configs(rng, &setup, 2); scope::run!(ctx, |ctx,s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let mut nodes : Vec<_> = cfgs.iter().enumerate().map(|(i,cfg)| { - let (node,runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone(), store.batches.clone()); + let (node,runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }).collect(); @@ -242,17 +241,13 @@ async fn test_validator_addrs_propagation() { let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let nodes: Vec<_> = cfgs .iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = testonly::Instance::new( - cfg.clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (node, runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) @@ -289,10 +284,9 @@ async fn test_genesis_mismatch() { let mut listener = cfgs[1].server_addr.bind().context("server_addr.bind()")?; tracing::info!("Start one node, we will simulate the other one."); - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (_node, runner) = - testonly::Instance::new(cfgs[0].clone(), store.blocks, store.batches.clone()); + let (_node, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node"))); tracing::info!("Accept a connection with mismatching genesis."); @@ -349,9 +343,8 @@ async fn validator_node_restart() { for cfg in &mut cfgs { cfg.rpc.push_validator_addrs_rate.refresh = time::Duration::ZERO; } - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; - let (node1, node1_runner) = - testonly::Instance::new(cfgs[1].clone(), store.blocks.clone(), store.batches.clone()); + let store = TestMemoryStorage::new(ctx, &setup).await; + let (node1, node1_runner) = testonly::Instance::new(cfgs[1].clone(), store.blocks.clone()); scope::run!(ctx, |ctx, s| async { s.spawn_bg(store.runner.run(ctx)); s.spawn_bg( @@ -380,11 +373,7 @@ async fn validator_node_restart() { // _node0 contains pipe, which has to exist to prevent the connection from dying // early. - let (_node0, runner) = testonly::Instance::new( - cfgs[0].clone(), - store.blocks.clone(), - store.batches.clone(), - ); + let (_node0, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone()); scope::run!(ctx, |ctx, s| async { s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node0"))); tracing::info!("wait for the update to arrive to node1"); @@ -406,7 +395,7 @@ async fn validator_node_restart() { .unwrap(); } -/// Test that SyncValidatorAddrs RPC batches updates +/// Test that PushValidatorAddrs RPC batches updates /// and is rate limited. Test is constructing a gossip /// network with star topology. All nodes are expected /// to receive all updates in 2 rounds of communication. @@ -437,13 +426,12 @@ async fn rate_limiting() { } let mut nodes = vec![]; scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); // Spawn the satellite nodes and wait until they register // their own address. for (i, cfg) in cfgs[1..].iter().enumerate() { - let (node, runner) = - testonly::Instance::new(cfg.clone(), store.blocks.clone(), store.batches.clone()); + let (node, runner) = testonly::Instance::new(cfg.clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); let sub = &mut node.net.gossip.validator_addrs.subscribe(); sync::wait_for(ctx, sub, |got| { @@ -456,8 +444,7 @@ async fn rate_limiting() { } // Spawn the center node. - let (center, runner) = - testonly::Instance::new(cfgs[0].clone(), store.blocks.clone(), store.batches.clone()); + let (center, runner) = testonly::Instance::new(cfgs[0].clone(), store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node[0]"))); // Await for the center to receive all validator addrs. let sub = &mut center.net.gossip.validator_addrs.subscribe(); @@ -524,7 +511,7 @@ async fn test_batch_votes_propagation() { scope::run!(ctx, |ctx, s| async { // All nodes share the same store - store is not used in this test anyway. - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); // Start all nodes. @@ -534,7 +521,6 @@ async fn test_batch_votes_propagation() { let (node, runner) = testonly::Instance::new_from_config(testonly::InstanceConfig { cfg: cfg.clone(), block_store: store.blocks.clone(), - batch_store: store.batches.clone(), attestation: attestation::Controller::new(Some(setup.attester_keys[i].clone())) .into(), }); diff --git a/node/actors/network/src/gossip/tests/syncing.rs b/node/actors/network/src/gossip/tests/syncing.rs index 1bd51446..f38e5f54 100644 --- a/node/actors/network/src/gossip/tests/syncing.rs +++ b/node/actors/network/src/gossip/tests/syncing.rs @@ -28,7 +28,9 @@ async fn coordinated_block_syncing(node_count: usize, gossip_peers: usize) { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, node_count); + let mut spec = validator::testonly::SetupSpec::new(rng, node_count); + spec.first_block = spec.first_pregenesis_block + 2; + let mut setup = validator::testonly::Setup::from_spec(rng, spec); setup.push_blocks(rng, EXCHANGED_STATE_COUNT); let cfgs = testonly::new_configs(rng, &setup, gossip_peers); scope::run!(ctx, |ctx, s| async { @@ -38,9 +40,9 @@ async fn coordinated_block_syncing(node_count: usize, gossip_peers: usize) { cfg.rpc.get_block_rate = limiter::Rate::INF; cfg.rpc.get_block_timeout = None; cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); + let (node, runner) = testonly::Instance::new(cfg, store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } @@ -84,7 +86,9 @@ async fn uncoordinated_block_syncing( let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, node_count); + let mut spec = validator::testonly::SetupSpec::new(rng, node_count); + spec.first_block = spec.first_pregenesis_block + 2; + let mut setup = validator::testonly::Setup::from_spec(rng, spec); setup.push_blocks(rng, EXCHANGED_STATE_COUNT); let cfgs = testonly::new_configs(rng, &setup, gossip_peers); scope::run!(ctx, |ctx, s| async { @@ -94,9 +98,9 @@ async fn uncoordinated_block_syncing( cfg.rpc.get_block_rate = limiter::Rate::INF; cfg.rpc.get_block_timeout = None; cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.clone().run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); + let (node, runner) = testonly::Instance::new(cfg, store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } @@ -149,9 +153,9 @@ async fn test_switching_on_nodes() { cfg.rpc.get_block_rate = limiter::Rate::INF; cfg.rpc.get_block_timeout = None; cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); + let (node, runner) = testonly::Instance::new(cfg, store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); @@ -204,9 +208,9 @@ async fn test_switching_off_nodes() { cfg.rpc.get_block_rate = limiter::Rate::INF; cfg.rpc.get_block_timeout = None; cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); + let (node, runner) = testonly::Instance::new(cfg, store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } @@ -267,10 +271,9 @@ async fn test_different_first_block() { cfg.validator_key = None; // Choose the first block for the node at random. let first = setup.blocks.choose(rng).unwrap().number(); - let store = - TestMemoryStorage::new_store_with_first_block(ctx, &setup.genesis, first).await; + let store = TestMemoryStorage::new_store_with_first_block(ctx, &setup, first).await; s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); + let (node, runner) = testonly::Instance::new(cfg, store.blocks); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } @@ -307,115 +310,6 @@ async fn test_different_first_block() { .unwrap(); } -/// Tests batch syncing with global network synchronization (a next batch becoming available -/// on some node only after nodes have received the previous batch. -#[test_casing(5, NETWORK_CONNECTIVITY_CASES)] -#[tokio::test(flavor = "multi_thread")] -async fn coordinated_batch_syncing(node_count: usize, gossip_peers: usize) { - abort_on_panic(); - let _guard = set_timeout(time::Duration::seconds(20)); - - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, node_count); - setup.push_batches(rng, EXCHANGED_STATE_COUNT); - let cfgs = testonly::new_configs(rng, &setup, gossip_peers); - scope::run!(ctx, |ctx, s| async { - let mut nodes = vec![]; - for (i, mut cfg) in cfgs.into_iter().enumerate() { - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; - s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); - nodes.push(node); - } - for batch in &setup.batches { - nodes - .choose(rng) - .unwrap() - .net - .gossip - .batch_store - .queue_batch(ctx, batch.clone(), setup.genesis.clone()) - .await - .context("queue_batch()")?; - for node in &nodes { - node.net - .gossip - .batch_store - .wait_until_persisted(ctx, batch.number) - .await - .unwrap(); - } - } - Ok(()) - }) - .await - .unwrap(); -} - -/// Tests batch syncing in an uncoordinated network, in which new batches arrive at a schedule. -#[test_casing(10, Product(( - NETWORK_CONNECTIVITY_CASES, - [time::Duration::milliseconds(50), time::Duration::milliseconds(500)], -)))] -#[tokio::test(flavor = "multi_thread")] -async fn uncoordinated_batch_syncing( - (node_count, gossip_peers): (usize, usize), - state_generation_interval: time::Duration, -) { - abort_on_panic(); - let _guard = set_timeout(time::Duration::seconds(20)); - - let ctx = &ctx::test_root(&ctx::RealClock); - let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, node_count); - setup.push_batches(rng, EXCHANGED_STATE_COUNT); - let cfgs = testonly::new_configs(rng, &setup, gossip_peers); - scope::run!(ctx, |ctx, s| async { - let mut nodes = vec![]; - for (i, mut cfg) in cfgs.into_iter().enumerate() { - cfg.rpc.push_batch_store_state_rate = limiter::Rate::INF; - cfg.rpc.get_batch_rate = limiter::Rate::INF; - cfg.rpc.get_batch_timeout = None; - cfg.validator_key = None; - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; - s.spawn_bg(store.runner.run(ctx)); - let (node, runner) = testonly::Instance::new(cfg, store.blocks, store.batches); - s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); - nodes.push(node); - } - for batch in &setup.batches { - nodes - .choose(rng) - .unwrap() - .net - .gossip - .batch_store - .queue_batch(ctx, batch.clone(), setup.genesis.clone()) - .await - .context("queue_batch()")?; - ctx.sleep(state_generation_interval).await?; - } - let last = setup.batches.last().unwrap().number; - for node in &nodes { - node.net - .gossip - .batch_store - .wait_until_persisted(ctx, last) - .await - .unwrap(); - } - Ok(()) - }) - .await - .unwrap(); -} - /// Test checking that if blocks that weren't queued get persisted, /// the syncing can behave accordingly. #[tokio::test(flavor = "multi_thread")] @@ -425,7 +319,9 @@ async fn test_sidechannel_sync() { let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut setup = validator::testonly::Setup::new(rng, 2); + let mut spec = validator::testonly::SetupSpec::new(rng, 2); + spec.first_block = spec.first_pregenesis_block + 2; + let mut setup = validator::testonly::Setup::from_spec(rng, spec); setup.push_blocks(rng, 10); let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { @@ -438,14 +334,11 @@ async fn test_sidechannel_sync() { cfg.validator_key = None; // Build a custom persistent store, so that we can tweak it later. - let persistent = - in_memory::BlockStore::new(setup.genesis.clone(), setup.genesis.first_block); + let persistent = in_memory::BlockStore::new(&setup, setup.genesis.first_block); stores.push(persistent.clone()); let (block_store, runner) = BlockStore::new(ctx, Box::new(persistent)).await?; s.spawn_bg(runner.run(ctx)); - // Use the standard batch store since it doesn't matter. - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; - let (node, runner) = testonly::Instance::new(cfg, block_store, store.batches); + let (node, runner) = testonly::Instance::new(cfg, block_store); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); nodes.push(node); } @@ -480,7 +373,7 @@ async fn test_sidechannel_sync() { stores[1].truncate(setup.blocks[8].number()); // Sync a block suffix. - let suffix = &setup.blocks[5..10]; + let suffix = &setup.blocks[5..]; for b in suffix { nodes[0] .net @@ -497,10 +390,77 @@ async fn test_sidechannel_sync() { .await?; // Check that the expected block range is actually stored. - assert_eq!(setup.blocks[8..10], dump(ctx, &stores[1]).await); + assert_eq!(setup.blocks[8..], dump(ctx, &stores[1]).await); } Ok(()) }) .await .unwrap(); } + +/// Test checking that nodes with/without pregenesis support can sync with each other. +/// This is a backward compatibility test. +#[tokio::test(flavor = "multi_thread")] +async fn test_syncing_without_pregenesis_support() { + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + let mut spec = validator::testonly::SetupSpec::new(rng, 2); + spec.first_block = spec.first_pregenesis_block + 2; + let mut setup = validator::testonly::Setup::from_spec(rng, spec); + setup.push_blocks(rng, 6); + let mut cfgs = testonly::new_configs(rng, &setup, 1); + cfgs[1].enable_pregenesis = false; + for cfg in &mut cfgs { + cfg.rpc.push_block_store_state_rate = limiter::Rate::INF; + cfg.rpc.get_block_rate = limiter::Rate::INF; + cfg.rpc.get_block_timeout = None; + cfg.validator_key = None; + } + + scope::run!(ctx, |ctx, s| async { + let stores = [ + in_memory::BlockStore::new(&setup, setup.blocks[0].number()), + // Node 1 doesn't have pregenesis support. + in_memory::BlockStore::new(&setup, setup.genesis.first_block), + ]; + let mut nodes = vec![]; + for i in 0..cfgs.len() { + let (block_store, runner) = BlockStore::new(ctx, Box::new(stores[i].clone())).await?; + s.spawn_bg(runner.run(ctx)); + let (node, runner) = testonly::Instance::new(cfgs[i].clone(), block_store); + s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); + nodes.push(node); + } + + // Insert blocks in order, alternating between nodes. + // The other node should sync the block. + for i in 0..setup.blocks.len() { + // Select the node which will have block `i`. + // Pregenesis blocks are only for the node that supports them. + let x = if let validator::Block::PreGenesis(_) = &setup.blocks[i] { + 0 + } else { + i % 2 + }; + nodes[x] + .net + .gossip + .block_store + .queue_block(ctx, setup.blocks[i].clone()) + .await?; + } + + // Wait for all nodes to fetch all the blocks. + for n in &nodes { + n.net + .gossip + .block_store + .wait_until_persisted(ctx, setup.blocks.last().unwrap().number()) + .await?; + } + + Ok(()) + }) + .await + .unwrap(); +} diff --git a/node/actors/network/src/lib.rs b/node/actors/network/src/lib.rs index adbb156f..80f57619 100644 --- a/node/actors/network/src/lib.rs +++ b/node/actors/network/src/lib.rs @@ -8,7 +8,7 @@ use zksync_concurrency::{ error::Wrap as _, limiter, scope, }; -use zksync_consensus_storage::{BatchStore, BlockStore}; +use zksync_consensus_storage::BlockStore; use zksync_consensus_utils::pipe::ActorPipe; mod config; @@ -55,11 +55,10 @@ impl Network { pub fn new( cfg: Config, block_store: Arc, - batch_store: Arc, pipe: ActorPipe, attestation: Arc, ) -> (Arc, Runner) { - let gossip = gossip::Network::new(cfg, block_store, batch_store, pipe.send, attestation); + let gossip = gossip::Network::new(cfg, block_store, pipe.send, attestation); let consensus = consensus::Network::new(gossip.clone()); let net = Arc::new(Self { gossip, consensus }); ( @@ -128,12 +127,6 @@ impl Runner { Ok(()) }); - // Fetch missing batches in the background. - s.spawn(async { - self.net.gossip.run_batch_fetcher(ctx).await; - Ok(()) - }); - // Maintain static gossip connections. for (peer, addr) in &self.net.gossip.cfg.gossip.static_outbound { s.spawn::<()>(async { diff --git a/node/actors/network/src/proto/gossip.proto b/node/actors/network/src/proto/gossip.proto index a5ff96b7..44849f59 100644 --- a/node/actors/network/src/proto/gossip.proto +++ b/node/actors/network/src/proto/gossip.proto @@ -34,26 +34,30 @@ message PushBatchVotesResp { repeated roles.attester.Signed votes = 1; } +message Last { + oneof t { + roles.validator.CommitQC final = 1; + uint64 pre_genesis = 2; + } +} + +message BlockStoreState { + // First L2 block that the node has locally. + optional uint64 first = 1; // required + // Last L2 block that the node has locally. + optional Last last = 2; // optional +} + // State of the local block store. // A node is expected to store a continuous range of blocks at all times // and actively fetch newest blocks. message PushBlockStoreState { - // First L2 block that the node has locally. - optional uint64 first = 1; // required; BlockNumber - // Last L2 block that the node has locally. + // [DEPRECATED] First L2 block that the node has locally. + optional uint64 first = 1; // required; has to be >= genesis.first_block. + // [DEPRECATED] Last L2 block that the node has locally. optional roles.validator.CommitQC last = 2; // optional -} - -// State of the local batch store. -// A node is expected to store a continuous range of batches at all times -// and actively fetch newest batch. -message PushBatchStoreState { - reserved 2; - reserved "last"; - // First batch that the node has locally. - optional uint64 first = 1; // required; BatchNumber - // Last batch that the node has locally. - optional uint64 last_v2 = 3; // optional + // Overrides first and last. + optional BlockStoreState state = 3; // optional } // Asks the server to send an L2 block (including its transactions). @@ -64,16 +68,11 @@ message GetBlockRequest { // Response to a `GetBlockRequest`. message GetBlockResponse { - optional roles.validator.FinalBlock block = 1; // optional; missing if block is not available -} - -// Asks the server to send a batch. -message GetBatchRequest { - // Number of the batch to send. - optional uint64 number = 1; -} - -// Response to a `GetBatchRequest`. -message GetBatchResponse { - optional roles.attester.SyncBatch batch = 1; // optional; missing if batch is not available + // NOTE: this should be a oneof, but 'buf breaking' is not smart enough + // to consider putting a preexisting field into a oneof as a compatible change. + // optional; missing if block is not available + // oneof t { + roles.validator.FinalBlock block = 1; + roles.validator.PreGenesisBlock pre_genesis = 2; + // } } diff --git a/node/actors/network/src/proto/rpc.proto b/node/actors/network/src/proto/rpc.proto index 503dbe00..26294479 100644 --- a/node/actors/network/src/proto/rpc.proto +++ b/node/actors/network/src/proto/rpc.proto @@ -3,13 +3,13 @@ syntax = "proto3"; package zksync.network.rpc; enum Capability { - reserved 5; + reserved 5,6,7; + reserved "GET_BATCH","PUSH_BATCH_STORE_STATE"; CONSENSUS = 0; PING = 2; PUSH_VALIDATOR_ADDRS = 1; PUSH_BLOCK_STORE_STATE = 3; GET_BLOCK = 4; PUSH_BATCH_VOTES = 8; - PUSH_BATCH_STORE_STATE = 7; - GET_BATCH = 6; + PUSH_BLOCK_STORE_STATE_V2 = 9; } diff --git a/node/actors/network/src/rpc/get_batch.rs b/node/actors/network/src/rpc/get_batch.rs deleted file mode 100644 index 18c53ca4..00000000 --- a/node/actors/network/src/rpc/get_batch.rs +++ /dev/null @@ -1,58 +0,0 @@ -//! RPC for fetching a batch from peer. -use super::Capability; -use crate::proto::gossip as proto; -use anyhow::Context; -use zksync_consensus_roles::attester; -use zksync_protobuf::{read_optional, ProtoFmt}; - -/// `get_batch` RPC. -#[derive(Debug)] -pub(crate) struct Rpc; - -// TODO: determine more precise `INFLIGHT` / `RATE` values as a result of load testing -impl super::Rpc for Rpc { - const CAPABILITY: Capability = Capability::GetBatch; - const INFLIGHT: u32 = 5; - const METHOD: &'static str = "get_batch"; - - type Req = Req; - type Resp = Resp; -} - -/// Asks the server to send a batch (including its blocks). -#[derive(Debug, PartialEq)] -pub(crate) struct Req(pub(crate) attester::BatchNumber); - -impl ProtoFmt for Req { - type Proto = proto::GetBatchRequest; - - fn read(message: &Self::Proto) -> anyhow::Result { - let number = message.number.context("number")?; - Ok(Self(attester::BatchNumber(number))) - } - - fn build(&self) -> Self::Proto { - let attester::BatchNumber(number) = self.0; - Self::Proto { - number: Some(number), - } - } -} - -/// Response to a [`GetBatchRequest`] containing a batch or a reason it cannot be retrieved. -#[derive(Debug, PartialEq)] -pub(crate) struct Resp(pub(crate) Option); - -impl ProtoFmt for Resp { - type Proto = proto::GetBatchResponse; - - fn read(r: &Self::Proto) -> anyhow::Result { - Ok(Self(read_optional(&r.batch).context("batch")?)) - } - - fn build(&self) -> Self::Proto { - Self::Proto { - batch: self.0.as_ref().map(ProtoFmt::build), - } - } -} diff --git a/node/actors/network/src/rpc/get_block.rs b/node/actors/network/src/rpc/get_block.rs index 84474f7a..7ec44d45 100644 --- a/node/actors/network/src/rpc/get_block.rs +++ b/node/actors/network/src/rpc/get_block.rs @@ -2,8 +2,8 @@ use super::Capability; use crate::proto::gossip as proto; use anyhow::Context; -use zksync_consensus_roles::validator::{BlockNumber, FinalBlock}; -use zksync_protobuf::{read_optional, ProtoFmt}; +use zksync_consensus_roles::validator; +use zksync_protobuf::ProtoFmt; /// `get_block` RPC. #[derive(Debug)] @@ -21,38 +21,67 @@ impl super::Rpc for Rpc { /// Asks the server to send a block (including its transactions). #[derive(Debug, PartialEq)] -pub(crate) struct Req(pub(crate) BlockNumber); +pub(crate) struct Req(pub(crate) validator::BlockNumber); impl ProtoFmt for Req { type Proto = proto::GetBlockRequest; fn read(message: &Self::Proto) -> anyhow::Result { let number = message.number.context("number")?; - Ok(Self(BlockNumber(number))) + Ok(Self(validator::BlockNumber(number))) } fn build(&self) -> Self::Proto { - let BlockNumber(number) = self.0; Self::Proto { - number: Some(number), + number: Some(self.0 .0), } } } /// Response to a [`GetBlockRequest`] containing a block or a reason it cannot be retrieved. #[derive(Debug, PartialEq)] -pub(crate) struct Resp(pub(crate) Option); +pub(crate) struct Resp(pub(crate) Option); + +impl Resp { + /// Clears pregenesis data from the response. + /// Use to simulate node behavior before pre-genesis support. + pub(crate) fn clear_pregenesis_data(&mut self) { + if let Some(validator::Block::PreGenesis(_)) = &self.0 { + self.0 = None; + } + } +} impl ProtoFmt for Resp { type Proto = proto::GetBlockResponse; fn read(r: &Self::Proto) -> anyhow::Result { - Ok(Self(read_optional(&r.block).context("block")?)) + use validator::Block as B; + let block = r + .block + .as_ref() + .map(ProtoFmt::read) + .transpose() + .context("block")? + .map(B::Final); + let pregenesis = r + .pre_genesis + .as_ref() + .map(ProtoFmt::read) + .transpose() + .context("pre_genesis")? + .map(B::PreGenesis); + Ok(Self(block.or(pregenesis))) } fn build(&self) -> Self::Proto { - Self::Proto { - block: self.0.as_ref().map(ProtoFmt::build), + use validator::Block as B; + let mut p = Self::Proto::default(); + match self.0.as_ref() { + Some(B::Final(b)) => p.block = Some(b.build()), + Some(B::PreGenesis(b)) => p.pre_genesis = Some(b.build()), + None => {} } + p } } diff --git a/node/actors/network/src/rpc/mod.rs b/node/actors/network/src/rpc/mod.rs index 7d53b3a5..9b617534 100644 --- a/node/actors/network/src/rpc/mod.rs +++ b/node/actors/network/src/rpc/mod.rs @@ -23,11 +23,9 @@ use std::{collections::BTreeMap, sync::Arc}; use zksync_concurrency::{ctx, io, limiter, metrics::LatencyHistogramExt as _, scope}; pub(crate) mod consensus; -pub(crate) mod get_batch; pub(crate) mod get_block; mod metrics; pub(crate) mod ping; -pub(crate) mod push_batch_store_state; pub(crate) mod push_batch_votes; pub(crate) mod push_block_store_state; pub(crate) mod push_validator_addrs; diff --git a/node/actors/network/src/rpc/push_batch_store_state.rs b/node/actors/network/src/rpc/push_batch_store_state.rs deleted file mode 100644 index 4c561abe..00000000 --- a/node/actors/network/src/rpc/push_batch_store_state.rs +++ /dev/null @@ -1,42 +0,0 @@ -//! RPC for fetching a batch from peer. -use super::Capability; -use crate::proto::gossip as proto; -use anyhow::Context as _; -use zksync_consensus_roles::attester; -use zksync_consensus_storage::BatchStoreState; -use zksync_protobuf::{required, ProtoFmt}; - -/// PushBatchStoreState RPC. -#[derive(Debug)] -pub(crate) struct Rpc; - -impl super::Rpc for Rpc { - const CAPABILITY: Capability = Capability::PushBatchStoreState; - const INFLIGHT: u32 = 1; - const METHOD: &'static str = "push_batch_store_state"; - - type Req = Req; - type Resp = (); -} - -/// Contains the freshest state of the sender's batch store. -#[derive(Debug, Clone, PartialEq, Eq)] -pub(crate) struct Req(pub(crate) BatchStoreState); - -impl ProtoFmt for Req { - type Proto = proto::PushBatchStoreState; - - fn read(message: &Self::Proto) -> anyhow::Result { - Ok(Self(BatchStoreState { - first: attester::BatchNumber(*required(&message.first).context("first")?), - last: message.last_v2.map(attester::BatchNumber), - })) - } - - fn build(&self) -> Self::Proto { - Self::Proto { - first: Some(self.0.first.0), - last_v2: self.0.last.as_ref().map(|n| n.0), - } - } -} diff --git a/node/actors/network/src/rpc/push_block_store_state.rs b/node/actors/network/src/rpc/push_block_store_state.rs index 1134185c..7879c243 100644 --- a/node/actors/network/src/rpc/push_block_store_state.rs +++ b/node/actors/network/src/rpc/push_block_store_state.rs @@ -3,8 +3,8 @@ use super::Capability; use crate::proto::gossip as proto; use anyhow::Context; use zksync_consensus_roles::validator; -use zksync_consensus_storage::BlockStoreState; -use zksync_protobuf::{read_optional, required, ProtoFmt}; +use zksync_consensus_storage::{BlockStoreState, Last}; +use zksync_protobuf::{read_optional, read_optional_repr, required, ProtoFmt, ProtoRepr}; /// PushBlockStoreState RPC. #[derive(Debug)] @@ -20,23 +20,105 @@ impl super::Rpc for Rpc { } /// Contains the freshest state of the sender's block store. -#[derive(Debug, Clone, PartialEq, Eq)] -pub(crate) struct Req(pub(crate) BlockStoreState); +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct Req { + // DEPRECATED: transmitted for backward compatibility. + first: validator::BlockNumber, + // DEPRECATED: transmitted for backward compatibility. + last: Option, + // Block store state. Will be required once we drop + // compatibility for `first` and `last` fields. + state: Option, +} + +impl Req { + /// Constructs a new request. + pub(crate) fn new(state: BlockStoreState, genesis: &validator::Genesis) -> Self { + Req { + first: state.first.max(genesis.first_block), + last: match &state.last { + Some(Last::Final(qc)) => Some(qc.clone()), + _ => None, + }, + state: Some(state), + } + } + + /// Extracts block store state from the request. + pub(crate) fn state(&self) -> BlockStoreState { + match &self.state { + Some(state) => state.clone(), + None => BlockStoreState { + first: self.first, + last: self.last.clone().map(Last::Final), + }, + } + } + + /// Clears pre-genesis info from the request. + /// Use to simulate node behavior before pre-genesis support. + pub(crate) fn clear_pregenesis_data(&mut self) { + self.state = None; + } +} + +impl ProtoRepr for proto::Last { + type Type = Last; + fn read(&self) -> anyhow::Result { + use proto::last::T; + Ok(match self.t.as_ref().context("missing")? { + T::PreGenesis(n) => Last::PreGenesis(validator::BlockNumber(*n)), + T::Final(qc) => Last::Final(ProtoFmt::read(qc).context("final")?), + }) + } + fn build(this: &Self::Type) -> Self { + use proto::last::T; + Self { + t: Some(match this { + Last::PreGenesis(n) => T::PreGenesis(n.0), + Last::Final(qc) => T::Final(qc.build()), + }), + } + } +} + +impl ProtoRepr for proto::BlockStoreState { + type Type = BlockStoreState; + fn read(&self) -> anyhow::Result { + Ok(Self::Type { + first: validator::BlockNumber(*required(&self.first).context("first")?), + last: read_optional_repr(&self.last).context("last")?, + }) + } + fn build(this: &Self::Type) -> Self { + Self { + first: Some(this.first.0), + last: this.last.as_ref().map(ProtoRepr::build), + } + } +} impl ProtoFmt for Req { type Proto = proto::PushBlockStoreState; - fn read(message: &Self::Proto) -> anyhow::Result { - Ok(Self(BlockStoreState { - first: validator::BlockNumber(*required(&message.first).context("first")?), - last: read_optional(&message.last).context("last")?, - })) + fn read(r: &Self::Proto) -> anyhow::Result { + let state: Option = read_optional_repr(&r.state).context("state")?; + Ok(Self { + first: r + .first + .map(validator::BlockNumber) + .or(state.as_ref().map(|s| s.first)) + .context("missing first and state")?, + last: read_optional(&r.last).context("last")?, + state, + }) } fn build(&self) -> Self::Proto { Self::Proto { - first: Some(self.0.first.0), - last: self.0.last.as_ref().map(|x| x.build()), + first: Some(self.first.0), + last: self.last.as_ref().map(|x| x.build()), + state: self.state.as_ref().map(ProtoRepr::build), } } } diff --git a/node/actors/network/src/rpc/testonly.rs b/node/actors/network/src/rpc/testonly.rs index 270e336e..9c3dc0a5 100644 --- a/node/actors/network/src/rpc/testonly.rs +++ b/node/actors/network/src/rpc/testonly.rs @@ -9,7 +9,6 @@ use rand::{ }; use std::sync::Arc; use zksync_consensus_roles::validator; -use zksync_consensus_storage::{BatchStoreState, BlockStoreState}; impl Distribution for Standard { fn sample(&self, rng: &mut R) -> rpc::consensus::Req { @@ -40,10 +39,7 @@ impl Distribution for Standard { impl Distribution for Standard { fn sample(&self, rng: &mut R) -> rpc::push_block_store_state::Req { - rpc::push_block_store_state::Req(BlockStoreState { - first: rng.gen(), - last: rng.gen(), - }) + rpc::push_block_store_state::Req::new(rng.gen(), &rng.gen()) } } @@ -58,24 +54,3 @@ impl Distribution for Standard { rpc::get_block::Resp(Some(rng.gen())) } } - -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> rpc::get_batch::Req { - rpc::get_batch::Req(rng.gen()) - } -} - -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> rpc::get_batch::Resp { - rpc::get_batch::Resp(Some(rng.gen())) - } -} - -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> rpc::push_batch_store_state::Req { - rpc::push_batch_store_state::Req(BatchStoreState { - first: rng.gen(), - last: rng.gen(), - }) - } -} diff --git a/node/actors/network/src/rpc/tests.rs b/node/actors/network/src/rpc/tests.rs index d2523fc2..4e04a2e9 100644 --- a/node/actors/network/src/rpc/tests.rs +++ b/node/actors/network/src/rpc/tests.rs @@ -14,9 +14,6 @@ fn test_schema_encode_decode() { test_encode_random::(rng); test_encode_random::(rng); test_encode_random::(rng); - test_encode_random::(rng); - test_encode_random::(rng); - test_encode_random::(rng); } fn expected(res: Result<(), mux::RunError>) -> Result<(), mux::RunError> { diff --git a/node/actors/network/src/testonly.rs b/node/actors/network/src/testonly.rs index dac7ba76..64f870d9 100644 --- a/node/actors/network/src/testonly.rs +++ b/node/actors/network/src/testonly.rs @@ -18,7 +18,7 @@ use zksync_concurrency::{ io, limiter, net, scope, sync, }; use zksync_consensus_roles::{node, validator}; -use zksync_consensus_storage::{BatchStore, BlockStore}; +use zksync_consensus_storage::BlockStore; use zksync_consensus_utils::pipe; impl Distribution for Standard { @@ -59,10 +59,10 @@ pub(crate) fn make_config(key: node::SecretKey) -> Config { static_outbound: HashMap::default(), }, max_block_size: usize::MAX, - max_batch_size: usize::MAX, tcp_accept_rate: limiter::Rate::INF, rpc: RpcConfig::default(), max_block_queue_size: 10, + enable_pregenesis: true, } } @@ -160,17 +160,16 @@ pub fn new_fullnode(rng: &mut impl Rng, peer: &Config) -> Config { static_outbound: [(peer.gossip.key.public(), peer.public_addr.clone())].into(), }, max_block_size: usize::MAX, - max_batch_size: usize::MAX, tcp_accept_rate: limiter::Rate::INF, rpc: RpcConfig::default(), max_block_queue_size: 10, + enable_pregenesis: true, } } /// Runner for Instance. pub struct InstanceRunner { net_runner: Runner, - batch_store: Arc, terminate: channel::Receiver<()>, } @@ -194,8 +193,6 @@ pub struct InstanceConfig { pub cfg: Config, /// block_store pub block_store: Arc, - /// batch_store - pub batch_store: Arc, /// Attestation controller. /// It is not configured by default. /// Attestation tests should configure it and consume @@ -205,15 +202,10 @@ pub struct InstanceConfig { impl Instance { /// Constructs a new instance. - pub fn new( - cfg: Config, - block_store: Arc, - batch_store: Arc, - ) -> (Self, InstanceRunner) { + pub fn new(cfg: Config, block_store: Arc) -> (Self, InstanceRunner) { Self::new_from_config(InstanceConfig { cfg, block_store, - batch_store, attestation: attestation::Controller::new(None).into(), }) } @@ -224,7 +216,6 @@ impl Instance { let (net, net_runner) = Network::new( cfg.cfg, cfg.block_store.clone(), - cfg.batch_store.clone(), actor_pipe, cfg.attestation, ); @@ -237,7 +228,6 @@ impl Instance { }, InstanceRunner { net_runner, - batch_store: cfg.batch_store.clone(), terminate: terminate_recv, }, ) diff --git a/node/actors/network/src/tests.rs b/node/actors/network/src/tests.rs index 22d325e3..6ef2f08b 100644 --- a/node/actors/network/src/tests.rs +++ b/node/actors/network/src/tests.rs @@ -14,14 +14,13 @@ async fn test_metrics() { let setup = validator::testonly::Setup::new(rng, 3); let cfgs = testonly::new_configs(rng, &setup, 1); scope::run!(ctx, |ctx, s| async { - let store = TestMemoryStorage::new(ctx, &setup.genesis).await; + let store = TestMemoryStorage::new(ctx, &setup).await; s.spawn_bg(store.runner.run(ctx)); let nodes: Vec<_> = cfgs .into_iter() .enumerate() .map(|(i, cfg)| { - let (node, runner) = - testonly::Instance::new(cfg, store.blocks.clone(), store.batches.clone()); + let (node, runner) = testonly::Instance::new(cfg, store.blocks.clone()); s.spawn_bg(runner.run(ctx).instrument(tracing::info_span!("node", i))); node }) diff --git a/node/libs/roles/src/attester/conv.rs b/node/libs/roles/src/attester/conv.rs index 7eb3c065..fe9cc651 100644 --- a/node/libs/roles/src/attester/conv.rs +++ b/node/libs/roles/src/attester/conv.rs @@ -1,11 +1,8 @@ use super::{ AggregateSignature, Batch, BatchHash, BatchNumber, BatchQC, Msg, MsgHash, MultiSig, PublicKey, - Signature, Signed, Signers, SyncBatch, WeightedAttester, -}; -use crate::{ - proto::attester::{self as proto, Attestation}, - validator::Payload, + Signature, Signed, Signers, WeightedAttester, }; +use crate::proto::attester as proto; use anyhow::Context as _; use zksync_consensus_crypto::ByteFmt; use zksync_consensus_utils::enum_util::Variant; @@ -41,29 +38,6 @@ impl ProtoFmt for Batch { } } -impl ProtoFmt for SyncBatch { - type Proto = proto::SyncBatch; - fn read(r: &Self::Proto) -> anyhow::Result { - let payloads = &r - .payloads - .iter() - .map(|p| Payload(p.clone())) - .collect::>(); - Ok(Self { - number: BatchNumber(*required(&r.number).context("number")?), - payloads: payloads.clone(), - proof: required(&r.proof).context("proof")?.to_vec(), - }) - } - fn build(&self) -> Self::Proto { - Self::Proto { - number: Some(self.number.0), - payloads: self.payloads.iter().map(|p| p.0.clone()).collect(), - proof: Some(self.proof.clone()), - } - } -} - impl + Clone> ProtoFmt for Signed { type Proto = proto::Signed; fn read(r: &Self::Proto) -> anyhow::Result { @@ -202,7 +176,7 @@ impl ProtoFmt for BatchQC { signatures: self .signatures .iter() - .map(|(pk, sig)| Attestation { + .map(|(pk, sig)| proto::Attestation { key: Some(pk.build()), sig: Some(sig.build()), }) diff --git a/node/libs/roles/src/attester/keys/secret_key.rs b/node/libs/roles/src/attester/keys/secret_key.rs index 2e70302e..d9f6288a 100644 --- a/node/libs/roles/src/attester/keys/secret_key.rs +++ b/node/libs/roles/src/attester/keys/secret_key.rs @@ -11,7 +11,7 @@ use zksync_consensus_utils::enum_util::Variant; pub struct SecretKey(pub(crate) Arc); impl SecretKey { - /// Generates a batch secret key from a cryptographically-secure entropy source. + /// Generates a secret key from a cryptographically-secure entropy source. pub fn generate() -> Self { Self(Arc::new(secp256k1::SecretKey::generate())) } @@ -21,7 +21,7 @@ impl SecretKey { PublicKey(self.0.public()) } - /// Signs a batch message. + /// Signs a message. pub fn sign_msg(&self, msg: V) -> Signed where V: Variant, diff --git a/node/libs/roles/src/attester/messages/batch.rs b/node/libs/roles/src/attester/messages/batch.rs index 41ada768..9bb8f03a 100644 --- a/node/libs/roles/src/attester/messages/batch.rs +++ b/node/libs/roles/src/attester/messages/batch.rs @@ -1,36 +1,8 @@ use super::{GenesisHash, Signed}; -use crate::{attester, validator::Payload}; +use crate::attester; use zksync_consensus_crypto::{keccak256::Keccak256, ByteFmt, Text, TextFmt}; use zksync_consensus_utils::enum_util::Variant; -/// A batch of L2 blocks used for the peers to fetch and keep in sync. -/// -/// The use case for this is for peers to be able to catch up with L1 -/// batches they missed during periods of being offline. These will -/// come with a proof of having been included on L1, rather than an -/// attester quorum certificate. -#[derive(Clone, Debug, PartialEq, Eq, Default)] -pub struct SyncBatch { - /// The number of the batch. - pub number: BatchNumber, - /// The payloads of the blocks the batch contains. - pub payloads: Vec, - /// The proof of the batch. - /// - /// It is going to be a Merkle proof the the batch has been included - /// in the state of the L1 system contract (not the L1 state root hash). - /// It can be produced as soon as we have the commitment available - /// locally, but validation requires a trusted L1 client to look up - /// the system contract state. - pub proof: Vec, -} - -impl PartialOrd for SyncBatch { - fn partial_cmp(&self, other: &Self) -> Option { - self.number.partial_cmp(&other.number) - } -} - #[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Default)] /// A batch number. pub struct BatchNumber(pub u64); diff --git a/node/libs/roles/src/attester/testonly.rs b/node/libs/roles/src/attester/testonly.rs index ce56f3ad..9e6f7294 100644 --- a/node/libs/roles/src/attester/testonly.rs +++ b/node/libs/roles/src/attester/testonly.rs @@ -1,9 +1,7 @@ use super::{ AggregateMultiSig, AggregateSignature, Batch, BatchHash, BatchNumber, BatchQC, Committee, Msg, - MsgHash, MultiSig, PublicKey, SecretKey, Signature, Signed, Signers, SyncBatch, - WeightedAttester, + MsgHash, MultiSig, PublicKey, SecretKey, Signature, Signed, Signers, WeightedAttester, }; -use crate::validator::Payload; use bit_vec::BitVec; use rand::{ distributions::{Distribution, Standard}, @@ -57,17 +55,6 @@ impl Distribution for Standard { } } -impl Distribution for Standard { - fn sample(&self, rng: &mut R) -> SyncBatch { - let size: usize = rng.gen_range(500..1000); - SyncBatch { - number: rng.gen(), - payloads: vec![Payload((0..size).map(|_| rng.gen()).collect())], - proof: rng.gen::<[u8; 32]>().to_vec(), - } - } -} - impl Distribution for Standard { fn sample(&self, rng: &mut R) -> BatchNumber { BatchNumber(rng.gen()) diff --git a/node/libs/roles/src/proto/attester/messages.proto b/node/libs/roles/src/proto/attester/messages.proto index b6b40ec8..296a77f1 100644 --- a/node/libs/roles/src/proto/attester/messages.proto +++ b/node/libs/roles/src/proto/attester/messages.proto @@ -5,12 +5,6 @@ package zksync.roles.attester; import "zksync/roles/attester/keys.proto"; import "zksync/roles/validator/genesis.proto"; -message SyncBatch { - optional uint64 number = 1; // required - repeated bytes payloads = 2; // required - optional bytes proof = 3; // required -} - message BatchHash { optional bytes keccak256 = 1; // required } diff --git a/node/libs/roles/src/proto/validator/messages.proto b/node/libs/roles/src/proto/validator/messages.proto index 7da97655..83308704 100644 --- a/node/libs/roles/src/proto/validator/messages.proto +++ b/node/libs/roles/src/proto/validator/messages.proto @@ -22,6 +22,20 @@ message FinalBlock { optional CommitQC justification = 2; // required } +message PreGenesisBlock { + optional uint64 number = 1; // required + optional bytes payload = 2; // required + optional bytes justification = 3; // required +} + +message Block { + // required + oneof t { + FinalBlock final = 1; + PreGenesisBlock pre_genesis = 2; + } +} + message View { reserved 1,2; reserved "protocol_version","fork"; diff --git a/node/libs/roles/src/validator/conv.rs b/node/libs/roles/src/validator/conv.rs index 8d6703b0..25f2bca3 100644 --- a/node/libs/roles/src/validator/conv.rs +++ b/node/libs/roles/src/validator/conv.rs @@ -1,8 +1,9 @@ use super::{ - AggregateSignature, BlockHeader, BlockNumber, ChainId, CommitQC, Committee, ConsensusMsg, - FinalBlock, ForkNumber, Genesis, GenesisHash, GenesisRaw, LeaderCommit, LeaderPrepare, Msg, - MsgHash, NetAddress, Payload, PayloadHash, Phase, PrepareQC, ProtocolVersion, PublicKey, - ReplicaCommit, ReplicaPrepare, Signature, Signed, Signers, View, ViewNumber, WeightedValidator, + AggregateSignature, Block, BlockHeader, BlockNumber, ChainId, CommitQC, Committee, + ConsensusMsg, FinalBlock, ForkNumber, Genesis, GenesisHash, GenesisRaw, Justification, + LeaderCommit, LeaderPrepare, Msg, MsgHash, NetAddress, Payload, PayloadHash, Phase, + PreGenesisBlock, PrepareQC, ProtocolVersion, PublicKey, ReplicaCommit, ReplicaPrepare, + Signature, Signed, Signers, View, ViewNumber, WeightedValidator, }; use crate::{ attester::{self, WeightedAttester}, @@ -135,6 +136,50 @@ impl ProtoFmt for FinalBlock { } } +impl ProtoFmt for PreGenesisBlock { + type Proto = proto::PreGenesisBlock; + + fn read(r: &Self::Proto) -> anyhow::Result { + Ok(Self { + number: BlockNumber(*required(&r.number).context("number")?), + payload: Payload(required(&r.payload).context("payload")?.clone()), + justification: Justification( + required(&r.justification).context("justification")?.clone(), + ), + }) + } + + fn build(&self) -> Self::Proto { + Self::Proto { + number: Some(self.number.0), + payload: Some(self.payload.0.clone()), + justification: Some(self.justification.0.clone()), + } + } +} + +impl ProtoFmt for Block { + type Proto = proto::Block; + + fn read(r: &Self::Proto) -> anyhow::Result { + use proto::block::T; + Ok(match required(&r.t)? { + T::Final(b) => Block::Final(ProtoFmt::read(b).context("block")?), + T::PreGenesis(b) => Block::PreGenesis(ProtoFmt::read(b).context("pre_genesis_block")?), + }) + } + + fn build(&self) -> Self::Proto { + use proto::block::T; + Self::Proto { + t: Some(match self { + Block::Final(b) => T::Final(b.build()), + Block::PreGenesis(b) => T::PreGenesis(b.build()), + }), + } + } +} + impl ProtoFmt for ConsensusMsg { type Proto = proto::ConsensusMsg; diff --git a/node/libs/roles/src/validator/messages/block.rs b/node/libs/roles/src/validator/messages/block.rs index 9c9498f2..2fa3c765 100644 --- a/node/libs/roles/src/validator/messages/block.rs +++ b/node/libs/roles/src/validator/messages/block.rs @@ -162,3 +162,58 @@ pub enum BlockValidationError { #[error("failed verifying quorum certificate: {0:#?}")] Justification(#[source] CommitQCVerifyError), } + +/// TODO: docs +#[derive(Debug, Clone, PartialEq)] +pub struct Justification(pub Vec); + +/// Block before `genesis.first_block` +/// with an external (non-consensus) justification. +#[derive(Debug, Clone, PartialEq)] +pub struct PreGenesisBlock { + /// Block number. + pub number: BlockNumber, + /// Payload. + pub payload: Payload, + /// Justification. + pub justification: Justification, +} + +/// TODO: docs +#[derive(Debug, Clone, PartialEq)] +pub enum Block { + /// Block with number `=genesis.first_block`. + Final(FinalBlock), +} + +impl From for Block { + fn from(b: PreGenesisBlock) -> Self { + Self::PreGenesis(b) + } +} + +impl From for Block { + fn from(b: FinalBlock) -> Self { + Self::Final(b) + } +} + +impl Block { + /// Block number. + pub fn number(&self) -> BlockNumber { + match self { + Self::PreGenesis(b) => b.number, + Self::Final(b) => b.number(), + } + } + + /// Payload. + pub fn payload(&self) -> &Payload { + match self { + Self::PreGenesis(b) => &b.payload, + Self::Final(b) => &b.payload, + } + } +} diff --git a/node/libs/roles/src/validator/testonly.rs b/node/libs/roles/src/validator/testonly.rs index f4fdea28..39c4f8ce 100644 --- a/node/libs/roles/src/validator/testonly.rs +++ b/node/libs/roles/src/validator/testonly.rs @@ -1,10 +1,10 @@ //! Test-only utilities. use super::{ - AggregateSignature, BlockHeader, BlockNumber, ChainId, CommitQC, Committee, ConsensusMsg, - FinalBlock, ForkNumber, Genesis, GenesisHash, GenesisRaw, LeaderCommit, LeaderPrepare, Msg, - MsgHash, NetAddress, Payload, PayloadHash, Phase, PrepareQC, ProofOfPossession, - ProtocolVersion, PublicKey, ReplicaCommit, ReplicaPrepare, SecretKey, Signature, Signed, - Signers, View, ViewNumber, WeightedValidator, + AggregateSignature, Block, BlockHeader, BlockNumber, ChainId, CommitQC, Committee, + ConsensusMsg, FinalBlock, ForkNumber, Genesis, GenesisHash, GenesisRaw, Justification, + LeaderCommit, LeaderPrepare, Msg, MsgHash, NetAddress, Payload, PayloadHash, Phase, + PreGenesisBlock, PrepareQC, ProofOfPossession, ProtocolVersion, PublicKey, ReplicaCommit, + ReplicaPrepare, SecretKey, Signature, Signed, Signers, View, ViewNumber, WeightedValidator, }; use crate::{attester, validator::LeaderSelectionMode}; use bit_vec::BitVec; @@ -25,6 +25,8 @@ pub struct SetupSpec { pub fork_number: ForkNumber, /// First block. pub first_block: BlockNumber, + /// First block that exists. + pub first_pregenesis_block: BlockNumber, /// Protocol version. pub protocol_version: ProtocolVersion, /// Validator secret keys and weights. @@ -47,6 +49,7 @@ impl SetupSpec { /// New `SetupSpec`. pub fn new_with_weights(rng: &mut impl Rng, weights: Vec) -> Self { + let first_block = BlockNumber(rng.gen_range(0..100)); Self { validator_weights: weights .clone() @@ -56,7 +59,8 @@ impl SetupSpec { attester_weights: weights.into_iter().map(|w| (rng.gen(), w)).collect(), chain_id: ChainId(1337), fork_number: ForkNumber(rng.gen_range(0..100)), - first_block: BlockNumber(rng.gen_range(0..100)), + first_block, + first_pregenesis_block: BlockNumber(rng.gen_range(0..=first_block.0)), protocol_version: ProtocolVersion::CURRENT, leader_selection: LeaderSelectionMode::RoundRobin, } @@ -66,18 +70,20 @@ impl SetupSpec { impl Setup { /// New `Setup`. pub fn new(rng: &mut impl Rng, validators: usize) -> Self { - SetupSpec::new(rng, validators).into() + let spec = SetupSpec::new(rng, validators); + Self::from_spec(rng, spec) } /// New `Setup`. pub fn new_with_weights(rng: &mut impl Rng, weights: Vec) -> Self { - SetupSpec::new_with_weights(rng, weights).into() + let spec = SetupSpec::new_with_weights(rng, weights); + Self::from_spec(rng, spec) } /// Next block to finalize. pub fn next(&self) -> BlockNumber { match self.0.blocks.last() { - Some(b) => b.header().number.next(), + Some(b) => b.number().next(), None => self.0.genesis.first_block, } } @@ -90,7 +96,10 @@ impl Setup { .0 .blocks .last() - .map(|b| b.justification.view().number.next()) + .map(|b| match b { + Block::Final(b) => b.justification.view().number.next(), + Block::PreGenesis(_) => ViewNumber(0), + }) .unwrap_or(ViewNumber(0)), }; let proposal = match self.0.blocks.last() { @@ -113,10 +122,13 @@ impl Setup { ) .unwrap(); } - self.0.blocks.push(FinalBlock { - payload, - justification, - }); + self.0.blocks.push( + FinalBlock { + payload, + justification, + } + .into(), + ); } /// Pushes `count` blocks with a random payload. @@ -127,39 +139,16 @@ impl Setup { } /// Finds the block by the number. - pub fn block(&self, n: BlockNumber) -> Option<&FinalBlock> { + pub fn block(&self, n: BlockNumber) -> Option<&Block> { let first = self.0.blocks.first()?.number(); self.0.blocks.get(n.0.checked_sub(first.0)? as usize) } - - /// Pushes `count` batches with a random payload. - pub fn push_batches(&mut self, rng: &mut impl Rng, count: usize) { - for _ in 0..count { - self.push_batch(rng); - } - } - - /// Pushes a new L1 batch. - pub fn push_batch(&mut self, rng: &mut impl Rng) { - let batch_number = match self.0.batches.last() { - Some(b) => b.number.next(), - None => attester::BatchNumber(0), - }; - let size: usize = rng.gen_range(500..1000); - let payloads = vec![Payload((0..size).map(|_| rng.gen()).collect())]; - let proof = rng.gen::<[u8; 32]>().to_vec(); - let batch = attester::SyncBatch { - number: batch_number, - payloads, - proof, - }; - self.0.batches.push(batch); - } } -impl From for Setup { - fn from(spec: SetupSpec) -> Self { - Self(SetupInner { +impl Setup { + /// Generates a new `Setup` from the given `SetupSpec`. + pub fn from_spec(rng: &mut impl Rng, spec: SetupSpec) -> Self { + let mut this = Self(SetupInner { genesis: GenesisRaw { chain_id: spec.chain_id, fork_number: spec.fork_number, @@ -186,9 +175,20 @@ impl From for Setup { .with_hash(), validator_keys: spec.validator_weights.into_iter().map(|(k, _)| k).collect(), attester_keys: spec.attester_weights.into_iter().map(|(k, _)| k).collect(), - batches: vec![], blocks: vec![], - }) + }); + // Populate pregenesis blocks. + for block in spec.first_pregenesis_block.0..spec.first_block.0 { + this.0.blocks.push( + PreGenesisBlock { + number: BlockNumber(block), + payload: rng.gen(), + justification: rng.gen(), + } + .into(), + ); + } + this } } @@ -200,9 +200,7 @@ pub struct SetupInner { /// Attesters' secret keys. pub attester_keys: Vec, /// Past blocks. - pub blocks: Vec, - /// L1 batches - pub batches: Vec, + pub blocks: Vec, /// Genesis config. pub genesis: Genesis, } @@ -368,6 +366,32 @@ impl Distribution for Standard { } } +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> Justification { + let size: usize = rng.gen_range(500..1000); + Justification((0..size).map(|_| rng.gen()).collect()) + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> PreGenesisBlock { + PreGenesisBlock { + number: rng.gen(), + payload: rng.gen(), + justification: rng.gen(), + } + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> Block { + match rng.gen_range(0..2) { + 0 => Block::PreGenesis(rng.gen()), + _ => Block::Final(rng.gen()), + } + } +} + impl Distribution for Standard { fn sample(&self, rng: &mut R) -> ReplicaPrepare { ReplicaPrepare { diff --git a/node/libs/roles/src/validator/tests.rs b/node/libs/roles/src/validator/tests.rs index b1c77204..eafa3a9e 100644 --- a/node/libs/roles/src/validator/tests.rs +++ b/node/libs/roles/src/validator/tests.rs @@ -87,6 +87,8 @@ fn test_schema_encoding() { test_encode_random::(rng); test_encode_random::(rng); test_encode_random::(rng); + test_encode_random::(rng); + test_encode_random::(rng); test_encode_random::>(rng); test_encode_random::(rng); test_encode_random::(rng); diff --git a/node/libs/storage/src/batch_store/metrics.rs b/node/libs/storage/src/batch_store/metrics.rs deleted file mode 100644 index deddc351..00000000 --- a/node/libs/storage/src/batch_store/metrics.rs +++ /dev/null @@ -1,50 +0,0 @@ -//! Storage metrics. -use std::time; - -#[derive(Debug, vise::Metrics)] -#[metrics(prefix = "zksync_consensus_storage_persistent_batch_store")] -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 `next_batch_to_attest_latency()` call. - #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - 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, -} - -#[vise::register] -pub(super) static PERSISTENT_BATCH_STORE: vise::Global = vise::Global::new(); - -#[derive(Debug, vise::Metrics)] -#[metrics(prefix = "zksync_consensus_storage_batch_store")] -pub(super) struct BatchStore { - /// Overall latency of a `queue_batch()` call. - #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) queue_batch: vise::Histogram, - /// Overall latency of a `persist_batch_qc()` call. - #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) persist_batch_qc: vise::Histogram, - /// Overall latency of a `wait_until_queued()` call. - #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) wait_until_queued: vise::Histogram, - /// Overall latency of a `wait_until_persisted()` call. - #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] - pub(super) wait_until_persisted: vise::Histogram, - /// Last persisted batch QC. - pub(super) last_persisted_batch_qc: vise::Gauge, -} - -#[vise::register] -pub(super) static BATCH_STORE: vise::Global = vise::Global::new(); - -#[derive(Debug, vise::Metrics)] -#[metrics(prefix = "zksync_consensus_storage_batch_store")] -pub(super) struct BatchStoreState { - /// BatchNumber of the next batch to queue. - pub(super) next_queued_batch: vise::Gauge, - /// BatchNumber of the next batch to persist. - pub(super) next_persisted_batch: vise::Gauge, -} diff --git a/node/libs/storage/src/batch_store/mod.rs b/node/libs/storage/src/batch_store/mod.rs deleted file mode 100644 index 1643bedd..00000000 --- a/node/libs/storage/src/batch_store/mod.rs +++ /dev/null @@ -1,340 +0,0 @@ -//! 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}; - -mod metrics; - -/// State of the `BatchStore`: continuous range of batches. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct BatchStoreState { - /// Stored batch with the lowest number. - /// If last is `None`, this is the first batch that should be fetched. - pub first: attester::BatchNumber, - /// The last stored L1 batch. - /// None iff store is empty. - pub last: Option, -} - -impl BatchStoreState { - /// Checks whether batch with the given number is stored in the `BatchStore`. - pub fn contains(&self, number: attester::BatchNumber) -> bool { - let Some(last) = self.last else { return false }; - self.first <= number && number <= last - } - - /// Number of the next batch that can be stored in the `BatchStore`. - /// (i.e. `last` + 1). - pub fn next(&self) -> attester::BatchNumber { - match &self.last { - Some(last) => last.next(), - None => self.first, - } - } - - /// Verifies `BatchStoreState'. - pub fn verify(&self) -> anyhow::Result<()> { - if let Some(last) = self.last { - anyhow::ensure!( - self.first <= last, - "first batch {} has bigger number than the last batch {}", - self.first, - last - ); - } - Ok(()) - } -} - -/// Trait for the shared state of batches between the consensus and the execution layer. -#[async_trait::async_trait] -pub trait PersistentBatchStore: 'static + fmt::Debug + Send + Sync { - /// Range of batches persisted in storage. - fn persisted(&self) -> sync::watch::Receiver; - - /// Returns the [attester::SyncBatch] with the given number, which is used by peers - /// to catch up with L1 batches that they might have missed if they went offline. - async fn get_batch( - &self, - ctx: &ctx::Ctx, - number: attester::BatchNumber, - ) -> ctx::Result>; - - /// Queue the batch to be persisted in storage. - /// `queue_next_batch()` may return BEFORE the batch is actually persisted, - /// but if the call succeeded the batch is expected to be persisted eventually. - /// Implementations are only required to accept a batch directly after the previous queued - /// batch, starting with `persisted().borrow().next()`. - async fn queue_next_batch(&self, ctx: &ctx::Ctx, batch: attester::SyncBatch) - -> ctx::Result<()>; -} - -/// Inner state of the `BatchStore`. -#[derive(Debug)] -struct Inner { - /// Batches that are queued to be persisted. - /// - /// This reflects the state of the `cache`. Its source is mainly the gossip layer (the RPC protocols started in `Network::run_stream`): - /// * the node pushes `SyncBatch` records which appear in `queued` to its gossip peers - /// * the node pulls `SyncBatch` records that it needs from gossip peers that reported to have them, and adds them to `queued` - /// * the `BatchStoreRunner` looks for new items in `queued` and pushes them into the `PersistentBatchStore` - /// - /// XXX: There doesn't seem to be anything that currently actively pushes into `queued` from outside gossip, - /// like it happens with the `BlockStore::queue_block` being called from BFT. - queued: BatchStoreState, - /// Batches that are already persisted. - /// - /// This reflects the state of the database. Its source is mainly the `PersistentBatchStore`: - /// * the `BatchStoreRunner` subscribes to `PersistedBatchStore::persisted()` and copies its contents to here; - /// * it also uses the opportunity to clear items from the `cache` - /// * but notably doesn't update `queued`, which would cause the data to be gossiped - persisted: BatchStoreState, - cache: VecDeque, -} - -impl Inner { - /// Minimal number of most recent batches to keep in memory. - /// It allows to serve the recent batches to peers fast, even - /// if persistent storage reads are slow (like in RocksDB). - /// `BatchStore` may keep in memory more batches in case - /// batches are queued faster than they are persisted. - const CACHE_CAPACITY: usize = 10; - - /// Tries to push the next batch to cache. - /// Noop if provided batch is not the expected one. - /// Returns true iff cache has been modified. - fn try_push(&mut self, batch: attester::SyncBatch) -> bool { - if self.queued.next() != batch.number { - return false; - } - self.queued.last = Some(batch.number); - self.cache.push_back(batch.clone()); - self.truncate_cache(); - true - } - - #[tracing::instrument(skip_all)] - fn truncate_cache(&mut self) { - while self.cache.len() > Self::CACHE_CAPACITY - && self.persisted.contains(self.cache[0].number) - { - self.cache.pop_front(); - } - } - - fn batch(&self, n: attester::BatchNumber) -> Option { - // Subtraction is safe, because batches in cache are - // stored in increasing order of batch number. - let first = self.cache.front()?; - self.cache.get((n.0 - first.number.0) as usize).cloned() - } -} - -/// A wrapper around a PersistentBatchStore. -#[derive(Debug)] -pub struct BatchStore { - inner: sync::watch::Sender, - persistent: Box, -} - -/// Runner of the BatchStore background tasks. -#[must_use] -#[derive(Debug, Clone)] -pub struct BatchStoreRunner(Arc); - -impl BatchStoreRunner { - /// Runs the background tasks of the BatchStore. - pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { - #[vise::register] - static COLLECTOR: vise::Collector> = - vise::Collector::new(); - let store_ref = Arc::downgrade(&self.0); - let _ = COLLECTOR.before_scrape(move || Some(store_ref.upgrade()?.scrape_metrics())); - - let res = scope::run!(ctx, |ctx, s| async { - let persisted = self.0.persistent.persisted(); - let mut queue_next = persisted.borrow().next(); - // Task truncating cache whenever a batch gets persisted. - s.spawn::<()>(async { - let mut persisted = persisted; - loop { - 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 { - 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(); - - self.0.persistent.queue_next_batch(ctx, batch).await?; - - ctx::Ok(()) - } - .instrument(tracing::info_span!("queue_persist_batch_iter")) - .await?; - } - }) - .await; - match res { - Ok(()) | Err(ctx::Error::Canceled(_)) => Ok(()), - Err(ctx::Error::Internal(err)) => Err(err), - } - } -} - -impl BatchStore { - /// Constructs a BatchStore. - /// BatchStore takes ownership of the passed PersistentBatchStore, - /// i.e. caller should modify the underlying persistent storage - /// ONLY through the constructed BatchStore. - pub async fn new( - _ctx: &ctx::Ctx, - persistent: Box, - ) -> ctx::Result<(Arc, BatchStoreRunner)> { - let persisted = persistent.persisted().borrow().clone(); - persisted.verify().context("state.verify()")?; - let this = Arc::new(Self { - inner: sync::watch::channel(Inner { - queued: persisted.clone(), - persisted, - cache: VecDeque::new(), - }) - .0, - persistent, - }); - Ok((this.clone(), BatchStoreRunner(this))) - } - - /// Available batches (in memory & persisted). - pub fn queued(&self) -> BatchStoreState { - self.inner.borrow().queued.clone() - } - - /// Fetches a batch (from queue or persistent storage). - pub async fn batch( - &self, - ctx: &ctx::Ctx, - number: attester::BatchNumber, - ) -> ctx::Result> { - { - let inner = self.inner.borrow(); - if !inner.queued.contains(number) { - return Ok(None); - } - if let Some(batch) = inner.batch(number) { - return Ok(Some(batch)); - } - } - let t = metrics::PERSISTENT_BATCH_STORE.batch_latency.start(); - - let batch = self - .persistent - .get_batch(ctx, number) - .await - .wrap("persistent.get_batch()")?; - - t.observe(); - Ok(batch) - } - - /// Append batch to a queue to be persisted eventually. - /// Since persisting a batch may take a significant amount of time, - /// BatchStore contains a queue of batches waiting to be persisted. - /// `queue_batch()` adds a batch to the queue as soon as all intermediate - /// Batches are queued_state as well. Queue is unbounded, so it is caller's - /// responsibility to manage the queue size. - pub async fn queue_batch( - &self, - ctx: &ctx::Ctx, - batch: attester::SyncBatch, - _genesis: validator::Genesis, - ) -> ctx::Result<()> { - let t = metrics::BATCH_STORE.queue_batch.start(); - - // XXX: Once we can validate `SyncBatch::proof` we should do it before adding the - // batch to the cache, otherwise a malicious peer could serve us data that prevents - // other inputs from entering the queue. It will also cause it to be gossiped at the moment. - sync::wait_for(ctx, &mut self.inner.subscribe(), |inner| { - inner.queued.next() >= batch.number - }) - .await?; - - self.inner - .send_if_modified(|inner| inner.try_push(batch.clone())); - - t.observe(); - Ok(()) - } - - /// 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, - number: attester::BatchNumber, - ) -> ctx::OrCanceled { - let t = metrics::BATCH_STORE.wait_until_queued.start(); - - let state = sync::wait_for(ctx, &mut self.inner.subscribe(), |inner| { - number < inner.queued.next() - }) - .await? - .queued - .clone(); - - t.observe(); - Ok(state) - } - - /// 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, - number: attester::BatchNumber, - ) -> ctx::OrCanceled { - let t = metrics::BATCH_STORE.wait_until_persisted.start(); - - let state = sync::wait_for(ctx, &mut self.persistent.persisted(), |persisted| { - number < persisted.next() - }) - .await? - .clone(); - - t.observe(); - Ok(state) - } - - fn scrape_metrics(&self) -> metrics::BatchStoreState { - let m = metrics::BatchStoreState::default(); - let inner = self.inner.borrow(); - m.next_queued_batch.set(inner.queued.next().0); - m.next_persisted_batch.set(inner.persisted.next().0); - m - } -} diff --git a/node/libs/storage/src/block_store/metrics.rs b/node/libs/storage/src/block_store/metrics.rs index 04f87bd0..5417b209 100644 --- a/node/libs/storage/src/block_store/metrics.rs +++ b/node/libs/storage/src/block_store/metrics.rs @@ -13,9 +13,12 @@ pub(super) struct PersistentBlockStore { /// Latency of a successful `block()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] pub(super) block_latency: vise::Histogram, - /// Latency of a successful `store_next_block()` call. + /// Latency of a successful `queue_next_block()` call. #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] pub(super) queue_next_block_latency: vise::Histogram, + /// Latency of a successful `verify_pregenesis_block()` call. + #[metrics(unit = vise::Unit::Seconds, buckets = vise::Buckets::LATENCIES)] + pub(super) verify_pregenesis_block_latency: vise::Histogram, } #[vise::register] diff --git a/node/libs/storage/src/block_store/mod.rs b/node/libs/storage/src/block_store/mod.rs index 14397c19..3d316952 100644 --- a/node/libs/storage/src/block_store/mod.rs +++ b/node/libs/storage/src/block_store/mod.rs @@ -7,49 +7,86 @@ use zksync_consensus_roles::validator; mod metrics; +/// Last block in the block store (see `BlockStoreState`). +/// Note that the commit qc is required in the case the block +/// has been finalized by the consensus. +#[derive(Debug, Clone, PartialEq)] +pub enum Last { + /// `=genesis.first_block`. + Final(validator::CommitQC), +} + +impl From<&validator::Block> for Last { + fn from(b: &validator::Block) -> Last { + use validator::Block as B; + match b { + B::PreGenesis(b) => Last::PreGenesis(b.number), + B::Final(b) => Last::Final(b.justification.clone()), + } + } +} + +impl Last { + /// Converts Last to block number. + pub fn number(&self) -> validator::BlockNumber { + match self { + Last::PreGenesis(n) => *n, + Last::Final(qc) => qc.header().number, + } + } + + /// Verifies Last. + pub fn verify(&self, genesis: &validator::Genesis) -> anyhow::Result<()> { + match self { + Last::PreGenesis(n) => anyhow::ensure!(n < &genesis.first_block, "missing qc"), + Last::Final(qc) => qc.verify(genesis)?, + } + Ok(()) + } +} + /// State of the `BlockStore`: continuous range of blocks. -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Clone, PartialEq)] pub struct BlockStoreState { /// Stored block with the lowest number. - /// If last is `None`, this is the first block that should be fetched. + /// If `last` is `None`, this is the first block that should be fetched. pub first: validator::BlockNumber, /// Stored block with the highest number. + /// If it is lower than genesis.first, then it will be set to `Last::Number` and + /// the first blocks to be fetched will need to include the non-consensus justification + /// (see `PreGenesisBlock`). /// None iff store is empty. - pub last: Option, + pub last: Option, } impl BlockStoreState { /// Checks whether block with the given number is stored in the `BlockStore`. pub fn contains(&self, number: validator::BlockNumber) -> bool { let Some(last) = &self.last else { return false }; - self.first <= number && number <= last.header().number + self.first <= number && number <= last.number() } /// Number of the next block that can be stored in the `BlockStore`. /// (i.e. `last` + 1). pub fn next(&self) -> validator::BlockNumber { match &self.last { - Some(qc) => qc.header().number.next(), + Some(last) => last.number().next(), None => self.first, } } /// Verifies `BlockStoreState'. pub fn verify(&self, genesis: &validator::Genesis) -> anyhow::Result<()> { - anyhow::ensure!( - genesis.first_block <= self.first, - "first block ({}) doesn't belong to the fork (which starts at block {})", - self.first, - genesis.first_block - ); if let Some(last) = &self.last { anyhow::ensure!( - self.first <= last.header().number, + self.first <= last.number(), "first block {} has bigger number than the last block {}", self.first, - last.header().number + last.number(), ); - last.verify(genesis).context("last.verify()")?; + last.verify(genesis).context("last")?; } Ok(()) } @@ -67,6 +104,15 @@ pub trait PersistentBlockStore: 'static + fmt::Debug + Send + Sync { /// Range of blocks persisted in storage. fn persisted(&self) -> sync::watch::Receiver; + /// Verifies a pre-genesis block. + /// It may interpret `block.justification` + /// and/or consult external source of truth. + async fn verify_pregenesis_block( + &self, + ctx: &ctx::Ctx, + block: &validator::PreGenesisBlock, + ) -> ctx::Result<()>; + /// Gets a block by its number. /// All the blocks from `state()` range are expected to be available. /// Blocks that have been queued but haven't been persisted yet don't have to be available. @@ -75,56 +121,46 @@ pub trait PersistentBlockStore: 'static + fmt::Debug + Send + Sync { &self, ctx: &ctx::Ctx, number: validator::BlockNumber, - ) -> ctx::Result; + ) -> ctx::Result; /// Queue the block to be persisted in storage. /// `queue_next_block()` may return BEFORE the block is actually persisted, /// but if the call succeeded the block is expected to be persisted eventually. /// Implementations are only required to accept a block directly after the previous queued /// block, starting with `persisted().borrow().next()`. - async fn queue_next_block( - &self, - ctx: &ctx::Ctx, - block: validator::FinalBlock, - ) -> ctx::Result<()>; + async fn queue_next_block(&self, ctx: &ctx::Ctx, block: validator::Block) -> ctx::Result<()>; } #[derive(Debug)] struct Inner { queued: BlockStoreState, persisted: BlockStoreState, - cache: VecDeque, + cache: VecDeque, } -impl Inner { - /// Minimal number of most recent blocks to keep in memory. - /// It allows to serve the recent blocks to peers fast, even - /// if persistent storage reads are slow (like in RocksDB). - /// `BlockStore` may keep in memory more blocks in case - /// blocks are queued faster than they are persisted. - const CACHE_CAPACITY: usize = 100; +/// Minimal number of most recent blocks to keep in memory. +/// It allows to serve the recent blocks to peers fast, even +/// if persistent storage reads are slow (like in RocksDB). +/// `BlockStore` may keep in memory more blocks in case +/// blocks are queued faster than they are persisted. +pub(crate) const CACHE_CAPACITY: usize = 100; +impl Inner { /// Tries to push the next block to cache. /// Noop if provided block is not the expected one. /// Returns true iff cache has been modified. - fn try_push(&mut self, block: validator::FinalBlock) -> bool { + fn try_push(&mut self, block: validator::Block) -> bool { if self.queued.next() != block.number() { return false; } - self.queued.last = Some(block.justification.clone()); + self.queued.last = Some(Last::from(&block)); self.cache.push_back(block); self.truncate_cache(); true } /// 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) - ) - )] + #[tracing::instrument(skip_all)] 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"); @@ -145,18 +181,16 @@ impl Inner { /// If cache size has been exceeded, remove entries which were already persisted. fn truncate_cache(&mut self) { - while self.cache.len() > Self::CACHE_CAPACITY - && self.persisted.next() > self.cache[0].number() - { + while self.cache.len() > CACHE_CAPACITY && self.persisted.next() > self.cache[0].number() { self.cache.pop_front(); } } - fn block(&self, n: validator::BlockNumber) -> Option { - // Subtraction is safe, because blocks in cache are - // stored in increasing order of block number. + fn block(&self, n: validator::BlockNumber) -> Option { let first = self.cache.front()?; - self.cache.get((n.0 - first.number().0) as usize).cloned() + self.cache + .get(n.0.checked_sub(first.number().0)? as usize) + .cloned() } } @@ -276,7 +310,7 @@ impl BlockStore { &self, ctx: &ctx::Ctx, number: validator::BlockNumber, - ) -> ctx::Result> { + ) -> ctx::Result> { { let inner = self.inner.borrow(); if !inner.queued.contains(number) { @@ -296,18 +330,41 @@ impl BlockStore { Ok(Some(block)) } + /// Verifies the block. + pub async fn verify_block(&self, ctx: &ctx::Ctx, block: &validator::Block) -> ctx::Result<()> { + use validator::Block as B; + match &block { + B::Final(b) => b.verify(&self.genesis).context("block.verify()")?, + B::PreGenesis(b) => { + if b.number >= self.genesis.first_block { + return Err(anyhow::format_err!( + "external justification is allowed only for pre-genesis blocks" + ) + .into()); + } + let t = metrics::PERSISTENT_BLOCK_STORE + .verify_pregenesis_block_latency + .start(); + self.persistent + .verify_pregenesis_block(ctx, b) + .await + .context("verify_pregenesis_block()")?; + t.observe(); + } + } + Ok(()) + } + /// Append block to a queue to be persisted eventually. /// Since persisting a block may take a significant amount of time, /// BlockStore contains a queue of blocks waiting to be persisted. /// `queue_block()` adds a block to the queue as soon as all intermediate /// blocks are queued_state as well. Queue is unbounded, so it is caller's /// responsibility to manage the queue size. - pub async fn queue_block( - &self, - ctx: &ctx::Ctx, - block: validator::FinalBlock, - ) -> ctx::Result<()> { - block.verify(&self.genesis).context("block.verify()")?; + pub async fn queue_block(&self, ctx: &ctx::Ctx, block: validator::Block) -> ctx::Result<()> { + self.verify_block(ctx, &block) + .await + .with_wrap(|| format!("verify_block({})", block.number()))?; sync::wait_for(ctx, &mut self.inner.subscribe(), |inner| { inner.queued.next() >= block.number() }) diff --git a/node/libs/storage/src/lib.rs b/node/libs/storage/src/lib.rs index 8e78a7a1..19d4496d 100644 --- a/node/libs/storage/src/lib.rs +++ b/node/libs/storage/src/lib.rs @@ -1,6 +1,5 @@ //! Abstraction for persistent data storage. //! It provides schema-aware type-safe database access. -mod batch_store; mod block_store; pub mod proto; mod replica_store; @@ -9,7 +8,6 @@ pub mod testonly; mod tests; pub use crate::{ - batch_store::{BatchStore, BatchStoreState, PersistentBatchStore}, - block_store::{BlockStore, BlockStoreRunner, BlockStoreState, PersistentBlockStore}, + block_store::{BlockStore, BlockStoreRunner, BlockStoreState, Last, PersistentBlockStore}, replica_store::{Proposal, ReplicaState, ReplicaStore}, }; diff --git a/node/libs/storage/src/testonly/in_memory.rs b/node/libs/storage/src/testonly/in_memory.rs index 487356ab..f6719463 100644 --- a/node/libs/storage/src/testonly/in_memory.rs +++ b/node/libs/storage/src/testonly/in_memory.rs @@ -1,27 +1,19 @@ //! In-memory storage implementation. -use crate::{ - batch_store::BatchStoreState, BlockStoreState, PersistentBatchStore, PersistentBlockStore, - ReplicaState, -}; +use crate::{block_store::Last, BlockStoreState, PersistentBlockStore, ReplicaState}; use anyhow::Context as _; use std::{ - collections::VecDeque, + collections::{HashMap, VecDeque}, sync::{Arc, Mutex}, }; use zksync_concurrency::{ctx, sync}; -use zksync_consensus_roles::{attester, validator}; +use zksync_consensus_roles::{validator, validator::testonly::Setup}; #[derive(Debug)] struct BlockStoreInner { genesis: validator::Genesis, persisted: sync::watch::Sender, - blocks: Mutex>, -} - -#[derive(Debug)] -struct BatchStoreInner { - persisted: sync::watch::Sender, - batches: Mutex>, + blocks: Mutex>, + pregenesis_blocks: HashMap, } /// In-memory block store. @@ -32,18 +24,29 @@ pub struct BlockStore(Arc); #[derive(Clone, Debug, Default)] pub struct ReplicaStore(Arc>); -/// In-memory replica store. -#[derive(Clone, Debug)] -pub struct BatchStore(Arc); - impl BlockStore { /// New In-memory `BlockStore`. - pub fn new(genesis: validator::Genesis, first: validator::BlockNumber) -> Self { - assert!(genesis.first_block <= first); + pub fn new(setup: &Setup, first: validator::BlockNumber) -> Self { + assert!( + setup + .blocks + .first() + .map(|b| b.number()) + .unwrap_or(setup.genesis.first_block) + <= first + ); Self(Arc::new(BlockStoreInner { - genesis, + genesis: setup.genesis.clone(), persisted: sync::watch::channel(BlockStoreState { first, last: None }).0, blocks: Mutex::default(), + pregenesis_blocks: setup + .blocks + .iter() + .flat_map(|b| match b { + validator::Block::PreGenesis(b) => Some((b.number, b.clone())), + validator::Block::Final(_) => None, + }) + .collect(), })) } @@ -66,16 +69,6 @@ impl BlockStore { } } -impl BatchStore { - /// New In-memory `BatchStore`. - pub fn new(first: attester::BatchNumber) -> Self { - Self(Arc::new(BatchStoreInner { - persisted: sync::watch::channel(BatchStoreState { first, last: None }).0, - batches: Mutex::default(), - })) - } -} - #[async_trait::async_trait] impl PersistentBlockStore for BlockStore { async fn genesis(&self, _ctx: &ctx::Ctx) -> ctx::Result { @@ -86,25 +79,35 @@ impl PersistentBlockStore for BlockStore { self.0.persisted.subscribe() } + async fn verify_pregenesis_block( + &self, + _ctx: &ctx::Ctx, + block: &validator::PreGenesisBlock, + ) -> ctx::Result<()> { + if self.0.pregenesis_blocks.get(&block.number) != Some(block) { + return Err(anyhow::format_err!("invalid pre-genesis block").into()); + } + Ok(()) + } + async fn block( &self, _ctx: &ctx::Ctx, number: validator::BlockNumber, - ) -> ctx::Result { + ) -> ctx::Result { let blocks = self.0.blocks.lock().unwrap(); let front = blocks.front().context("not found")?; let idx = number .0 - .checked_sub(front.header().number.0) + .checked_sub(front.number().0) .context("not found")?; Ok(blocks.get(idx as usize).context("not found")?.clone()) } - async fn queue_next_block( - &self, - _ctx: &ctx::Ctx, - block: validator::FinalBlock, - ) -> ctx::Result<()> { + async fn queue_next_block(&self, ctx: &ctx::Ctx, block: validator::Block) -> ctx::Result<()> { + if let validator::Block::PreGenesis(b) = &block { + self.verify_pregenesis_block(ctx, b).await?; + } let mut blocks = self.0.blocks.lock().unwrap(); let want = self.0.persisted.borrow().next(); if block.number() < want { @@ -117,52 +120,12 @@ impl PersistentBlockStore for BlockStore { } self.0 .persisted - .send_modify(|p| p.last = Some(block.justification.clone())); + .send_modify(|p| p.last = Some(Last::from(&block))); blocks.push_back(block); Ok(()) } } -#[async_trait::async_trait] -impl PersistentBatchStore for BatchStore { - fn persisted(&self) -> sync::watch::Receiver { - self.0.persisted.subscribe() - } - - async fn get_batch( - &self, - _ctx: &ctx::Ctx, - number: attester::BatchNumber, - ) -> ctx::Result> { - let batches = self.0.batches.lock().unwrap(); - let Some(front) = batches.front() else { - return Ok(None); - }; - let Some(idx) = number.0.checked_sub(front.number.0) else { - return Ok(None); - }; - 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, - batch: attester::SyncBatch, - ) -> ctx::Result<()> { - let mut batches = self.0.batches.lock().unwrap(); - let want = self.0.persisted.borrow().next(); - if batch.number != want { - return Err(anyhow::anyhow!("got batch {:?}, want {want:?}", batch.number).into()); - } - self.0 - .persisted - .send_modify(|p| p.last = Some(batch.number)); - batches.push_back(batch); - Ok(()) - } -} - #[async_trait::async_trait] impl crate::ReplicaStore for ReplicaStore { async fn state(&self, _ctx: &ctx::Ctx) -> ctx::Result { diff --git a/node/libs/storage/src/testonly/mod.rs b/node/libs/storage/src/testonly/mod.rs index 38c4a4ba..c36d3b74 100644 --- a/node/libs/storage/src/testonly/mod.rs +++ b/node/libs/storage/src/testonly/mod.rs @@ -1,13 +1,13 @@ //! Test-only utilities. use crate::{ - batch_store::BatchStoreRunner, BatchStore, BlockStore, BlockStoreRunner, PersistentBatchStore, - PersistentBlockStore, Proposal, ReplicaState, + BlockStore, BlockStoreRunner, BlockStoreState, Last, PersistentBlockStore, Proposal, + ReplicaState, }; use anyhow::Context as _; use rand::{distributions::Standard, prelude::Distribution, Rng}; use std::sync::Arc; -use zksync_concurrency::{ctx, scope}; -use zksync_consensus_roles::{attester, validator}; +use zksync_concurrency::ctx; +use zksync_consensus_roles::{validator, validator::testonly::Setup}; pub mod in_memory; @@ -32,107 +32,76 @@ impl Distribution for Standard { } } +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> Last { + match rng.gen_range(0..2) { + 0 => Last::PreGenesis(rng.gen()), + _ => Last::Final(rng.gen()), + } + } +} + +impl Distribution for Standard { + fn sample(&self, rng: &mut R) -> BlockStoreState { + BlockStoreState { + first: rng.gen(), + last: rng.gen(), + } + } +} + /// Test-only memory storage for blocks and batches. pub struct TestMemoryStorage { /// In-memory block store with its runner. pub blocks: Arc, - /// In-memory batch store with its runner. - pub batches: Arc, /// In-memory storage runner. - pub runner: TestMemoryStorageRunner, + pub runner: BlockStoreRunner, /// The in-memory block store representing the persistent store. pub im_blocks: in_memory::BlockStore, - /// The in-memory batch store representing the persistent store. - pub im_batches: in_memory::BatchStore, -} - -/// Test-only memory storage runner wrapping both block and batch store runners. -#[derive(Clone, Debug)] -pub struct TestMemoryStorageRunner { - /// In-memory block store runner. - blocks: BlockStoreRunner, - /// In-memory batch store runner. - batches: BatchStoreRunner, -} - -impl TestMemoryStorageRunner { - /// Constructs a new in-memory store for both blocks and batches with their respective runners. - pub async fn new(blocks_runner: BlockStoreRunner, batches_runner: BatchStoreRunner) -> Self { - Self { - blocks: blocks_runner, - batches: batches_runner, - } - } - - /// Runs the storage for both blocks and batches. - pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { - scope::run!(ctx, |ctx, s| async { - s.spawn(self.blocks.run(ctx)); - s.spawn(self.batches.run(ctx)); - Ok(()) - }) - .await - } } impl TestMemoryStorage { /// Constructs a new in-memory store for both blocks and batches with their respective runners. - pub async fn new(ctx: &ctx::Ctx, genesis: &validator::Genesis) -> Self { - Self::new_store_with_first_block(ctx, genesis, genesis.first_block).await + pub async fn new(ctx: &ctx::Ctx, setup: &Setup) -> Self { + Self::new_store_with_first_block(ctx, setup, setup.genesis.first_block).await } /// Constructs a new in-memory store with a custom expected first block /// (i.e. possibly different than `genesis.fork.first_block`). pub async fn new_store_with_first_block( ctx: &ctx::Ctx, - genesis: &validator::Genesis, + setup: &Setup, first: validator::BlockNumber, ) -> Self { - let im_blocks = in_memory::BlockStore::new(genesis.clone(), first); - let im_batches = in_memory::BatchStore::new(attester::BatchNumber(0)); - Self::new_with_im(ctx, im_blocks, im_batches).await + let im_blocks = in_memory::BlockStore::new(setup, first); + Self::new_with_im(ctx, im_blocks).await } /// Constructs a new in-memory store for both blocks and batches with their respective runners. - async fn new_with_im( - ctx: &ctx::Ctx, - im_blocks: in_memory::BlockStore, - im_batches: in_memory::BatchStore, - ) -> Self { - let (blocks, blocks_runner) = BlockStore::new(ctx, Box::new(im_blocks.clone())) + async fn new_with_im(ctx: &ctx::Ctx, im_blocks: in_memory::BlockStore) -> Self { + let (blocks, runner) = BlockStore::new(ctx, Box::new(im_blocks.clone())) .await .unwrap(); - - let (batches, batches_runner) = BatchStore::new(ctx, Box::new(im_batches.clone())) - .await - .unwrap(); - - let runner = TestMemoryStorageRunner::new(blocks_runner, batches_runner).await; - Self { blocks, - batches, runner, im_blocks, - im_batches, } } } /// Dumps all the blocks stored in `store`. -pub async fn dump(ctx: &ctx::Ctx, store: &dyn PersistentBlockStore) -> Vec { - let genesis = store.genesis(ctx).await.unwrap(); +pub async fn dump(ctx: &ctx::Ctx, store: &dyn PersistentBlockStore) -> Vec { let state = store.persisted().borrow().clone(); - assert!(genesis.first_block <= state.first); let mut blocks = vec![]; let after = state .last .as_ref() - .map(|qc| qc.header().number.next()) + .map(|last| last.number().next()) .unwrap_or(state.first); for n in (state.first.0..after.0).map(validator::BlockNumber) { let block = store.block(ctx, n).await.unwrap(); - assert_eq!(block.header().number, n); + assert_eq!(block.number(), n); blocks.push(block); } if let Some(before) = state.first.prev() { @@ -142,43 +111,13 @@ pub async fn dump(ctx: &ctx::Ctx, store: &dyn PersistentBlockStore) -> Vec Vec { - // let genesis = store.genesis(ctx).await.unwrap(); - let state = store.persisted().borrow().clone(); - // assert!(genesis.first_block <= state.first); - let mut batches = vec![]; - let after = state - .last - .as_ref() - .map(|sb| sb.next()) - .unwrap_or(state.first); - for n in (state.first.0..after.0).map(attester::BatchNumber) { - let batch = store.get_batch(ctx, n).await.unwrap().unwrap(); - assert_eq!(batch.number, n); - batches.push(batch); - } - if let Some(before) = state.first.prev() { - assert!(store.get_batch(ctx, before).await.unwrap().is_none()); - } - assert!(store.get_batch(ctx, after).await.unwrap().is_none()); - batches -} - /// Verifies storage content. pub async fn verify(ctx: &ctx::Ctx, store: &BlockStore) -> anyhow::Result<()> { let range = store.queued(); for n in (range.first.0..range.next().0).map(validator::BlockNumber) { async { - store - .block(ctx, n) - .await? - .context("missing")? - .verify(store.genesis()) - .context("verify()") + let b = store.block(ctx, n).await?.context("missing")?; + store.verify_block(ctx, &b).await.context("verify_block()") } .await .context(n)?; diff --git a/node/libs/storage/src/tests.rs b/node/libs/storage/src/tests.rs index 36266bba..9216bd52 100644 --- a/node/libs/storage/src/tests.rs +++ b/node/libs/storage/src/tests.rs @@ -1,7 +1,11 @@ use super::*; use crate::{testonly::TestMemoryStorage, ReplicaState}; +use rand::Rng as _; use zksync_concurrency::{ctx, scope, sync, testonly::abort_on_panic}; -use zksync_consensus_roles::{attester::BatchNumber, validator::testonly::Setup}; +use zksync_consensus_roles::{ + validator, + validator::testonly::{Setup, SetupSpec}, +}; #[tokio::test] async fn test_inmemory_block_store() { @@ -11,10 +15,13 @@ async fn test_inmemory_block_store() { let mut setup = Setup::new(rng, 3); setup.push_blocks(rng, 5); - let store = - &testonly::in_memory::BlockStore::new(setup.genesis.clone(), setup.genesis.first_block); + let store = &testonly::in_memory::BlockStore::new( + &setup, + setup.blocks.first().as_ref().unwrap().number(), + ); let mut want = vec![]; for block in &setup.blocks { + tracing::info!("block = {:?}", block.number()); store.queue_next_block(ctx, block.clone()).await.unwrap(); sync::wait_for(ctx, &mut store.persisted(), |p| p.contains(block.number())) .await @@ -24,24 +31,34 @@ async fn test_inmemory_block_store() { } } +// Test checking that store doesn't accept pre-genesis blocks with invalid justification. #[tokio::test] -async fn test_inmemory_batch_store() { +async fn test_invalid_justification() { abort_on_panic(); let ctx = &ctx::test_root(&ctx::RealClock); let rng = &mut ctx.rng(); - let mut setup = Setup::new(rng, 3); - setup.push_batches(rng, 5); - - let store = &testonly::in_memory::BatchStore::new(BatchNumber(0)); - let mut want = vec![]; - for batch in &setup.batches { - store.queue_next_batch(ctx, batch.clone()).await.unwrap(); - sync::wait_for(ctx, &mut store.persisted(), |p| p.contains(batch.number)) + let mut spec = SetupSpec::new(rng, 1); + spec.first_block = spec.first_pregenesis_block + 2; + let setup = Setup::from_spec(rng, spec); + scope::run!(ctx, |ctx, s| async { + let store = TestMemoryStorage::new(ctx, &setup).await; + s.spawn_bg(store.runner.run(ctx)); + let store = store.blocks; + // Insert a correct block first. + store + .queue_block(ctx, setup.blocks[0].clone()) .await .unwrap(); - want.push(batch.clone()); - assert_eq!(want, testonly::dump_batch(ctx, store).await); - } + // Insert an incorrect second block. + let validator::Block::PreGenesis(mut b) = setup.blocks[1].clone() else { + panic!() + }; + b.justification = rng.gen(); + store.queue_block(ctx, b.into()).await.unwrap_err(); + Ok(()) + }) + .await + .unwrap(); } #[test] @@ -51,6 +68,41 @@ fn test_schema_encode_decode() { zksync_protobuf::testonly::test_encode_random::(rng); } +#[tokio::test] +async fn test_get_not_cached_block() { + abort_on_panic(); + let ctx = &ctx::test_root(&ctx::RealClock); + let rng = &mut ctx.rng(); + let mut setup = Setup::new(rng, 1); + setup.push_blocks(rng, block_store::CACHE_CAPACITY + 5); + scope::run!(ctx, |ctx, s| async { + let store = TestMemoryStorage::new(ctx, &setup).await; + s.spawn_bg(store.runner.run(ctx)); + // Persist more blocks than the cache size. + for block in &setup.blocks { + store.blocks.queue_block(ctx, block.clone()).await.unwrap(); + } + store + .blocks + .wait_until_persisted(ctx, setup.blocks.last().as_ref().unwrap().number()) + .await + .unwrap(); + // Request the first block (not in cache). + assert_eq!( + setup.blocks[0], + store + .blocks + .block(ctx, setup.blocks[0].number()) + .await + .unwrap() + .unwrap() + ); + Ok(()) + }) + .await + .unwrap(); +} + #[tokio::test] async fn test_state_updates() { abort_on_panic(); @@ -61,8 +113,7 @@ async fn test_state_updates() { // Create store with non-trivial first block. let first_block = &setup.blocks[2]; let store = - TestMemoryStorage::new_store_with_first_block(ctx, &setup.genesis, first_block.number()) - .await; + TestMemoryStorage::new_store_with_first_block(ctx, &setup, first_block.number()).await; scope::run!(ctx, |ctx, s| async { s.spawn_bg(store.runner.run(ctx)); let want = BlockStoreState { @@ -101,7 +152,7 @@ async fn test_state_updates() { assert_eq!( BlockStoreState { first: first_block.number(), - last: Some(block.justification.clone()), + last: Some(block.into()), }, store.blocks.queued() ); diff --git a/node/tools/src/bin/deployer.rs b/node/tools/src/bin/deployer.rs index b67c3fc8..bd5ee64f 100644 --- a/node/tools/src/bin/deployer.rs +++ b/node/tools/src/bin/deployer.rs @@ -48,7 +48,6 @@ fn generate_consensus_nodes(nodes: usize, seed_nodes_amount: Option) -> V metrics_server_addr: None, genesis: setup.genesis.clone(), max_payload_size: 1000000, - max_batch_size: 100000000, validator_key: Some(validator_keys[i].clone()), attester_key: Some(attester_keys[i].clone()), node_key: node_keys[i].clone(), diff --git a/node/tools/src/bin/localnet_config.rs b/node/tools/src/bin/localnet_config.rs index 90e603c8..0769a62f 100644 --- a/node/tools/src/bin/localnet_config.rs +++ b/node/tools/src/bin/localnet_config.rs @@ -76,7 +76,6 @@ fn main() -> anyhow::Result<()> { .map(|port| SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), port)), genesis: setup.genesis.clone(), max_payload_size: 1000000, - max_batch_size: 100000000, node_key: node_keys[i].clone(), validator_key: validator_keys.get(i).cloned(), attester_key: attester_keys.get(i).cloned(), diff --git a/node/tools/src/config.rs b/node/tools/src/config.rs index be36dc56..8399e036 100644 --- a/node/tools/src/config.rs +++ b/node/tools/src/config.rs @@ -9,15 +9,15 @@ use std::{ sync::Arc, }; use tokio_rustls::rustls::pki_types::{CertificateDer, PrivateKeyDer}; -use zksync_concurrency::{ctx, net, time}; +use zksync_concurrency::{ctx, net}; 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}; use zksync_consensus_network as network; use zksync_consensus_roles::{attester, node, validator}; -use zksync_consensus_storage::testonly::{TestMemoryStorage, TestMemoryStorageRunner}; +use zksync_consensus_storage::{BlockStore, BlockStoreRunner}; use zksync_protobuf::{ - kB, read_optional, read_optional_repr, read_required, required, ProtoFmt, ProtoRepr, + read_optional, read_optional_repr, read_required, required, ProtoFmt, ProtoRepr, }; const CRATE_VERSION: &str = env!("CARGO_PKG_VERSION"); @@ -76,7 +76,6 @@ pub struct App { pub genesis: validator::Genesis, pub max_payload_size: usize, - pub max_batch_size: usize, pub validator_key: Option, pub attester_key: Option, @@ -168,14 +167,6 @@ impl ProtoFmt for App { .and_then(|x| Ok((*x).try_into()?)) .context("max_payload_size")?; - let max_batch_size = match &r.max_batch_size { - Some(x) => (*x).try_into().context("max_payload_size")?, - // Arbitrary estimate of 100 blocks + 1kB for the merkle proof. - // NOTE: this test node currently doesn't implement batches at all. - // Once it does, do the estimates again. - None => max_payload_size * 100 + kB, - }; - Ok(Self { server_addr: read_required_text(&r.server_addr).context("server_addr")?, public_addr: net::Host(required(&r.public_addr).context("public_addr")?.clone()), @@ -185,7 +176,6 @@ impl ProtoFmt for App { genesis: read_required(&r.genesis).context("genesis")?, max_payload_size, - max_batch_size, // TODO: read secret. validator_key: read_optional_secret_text(&r.validator_secret_key) .context("validator_secret_key")?, @@ -211,7 +201,6 @@ impl ProtoFmt for App { genesis: Some(self.genesis.build()), max_payload_size: Some(self.max_payload_size.try_into().unwrap()), - max_batch_size: Some(self.max_batch_size.try_into().unwrap()), validator_secret_key: self.validator_key.as_ref().map(TextFmt::encode), attester_secret_key: self.attester_key.as_ref().map(TextFmt::encode), @@ -266,12 +255,10 @@ impl Configs { pub async fn make_executor( &self, ctx: &ctx::Ctx, - ) -> ctx::Result<(executor::Executor, TestMemoryStorageRunner)> { - let replica_store = store::RocksDB::open(self.app.genesis.clone(), &self.database).await?; - let store = TestMemoryStorage::new(ctx, &self.app.genesis).await; - + ) -> ctx::Result<(executor::Executor, BlockStoreRunner)> { + let store = store::RocksDB::open(self.app.genesis.clone(), &self.database).await?; + let (block_store, runner) = BlockStore::new(ctx, Box::new(store.clone())).await?; let attestation = Arc::new(attestation::Controller::new(self.app.attester_key.clone())); - let runner = store.runner; let e = executor::Executor { config: executor::Config { @@ -283,7 +270,6 @@ impl Configs { gossip_static_inbound: self.app.gossip_static_inbound.clone(), gossip_static_outbound: self.app.gossip_static_outbound.clone(), max_payload_size: self.app.max_payload_size, - max_batch_size: self.app.max_batch_size, rpc: executor::RpcConfig::default(), debug_page: self .app @@ -310,17 +296,15 @@ impl Configs { }) .transpose() .context("debug_page")?, - batch_poll_interval: time::Duration::seconds(1), }, - block_store: store.blocks, - batch_store: store.batches, + block_store, validator: self .app .validator_key .as_ref() .map(|key| executor::Validator { key: key.clone(), - replica_store: Box::new(replica_store), + replica_store: Box::new(store.clone()), payload_manager: Box::new(bft::testonly::RandomPayload( self.app.max_payload_size, )), diff --git a/node/tools/src/proto/mod.proto b/node/tools/src/proto/mod.proto index 2120a9ee..95091d77 100644 --- a/node/tools/src/proto/mod.proto +++ b/node/tools/src/proto/mod.proto @@ -69,8 +69,8 @@ message DebugPageConfig { // Application configuration. message AppConfig { - reserved 9,13,14,15; - reserved "debug_addr", "debug_credentials", "debug_cert_path", "debug_key_path"; + reserved 9,13,14,15,17; + reserved "debug_addr", "debug_credentials", "debug_cert_path", "debug_key_path", "max_batch_size"; // Ports @@ -98,9 +98,6 @@ message AppConfig { // Maximal size of the block payload. optional uint64 max_payload_size = 5; // required; bytes - // Maximal size of the sync batch payload. - optional uint64 max_batch_size = 17; // optional; bytes - // Validator secret key. optional string validator_secret_key = 10; // optional; ValidatorSecretKey diff --git a/node/tools/src/rpc/methods/last_committed_block.rs b/node/tools/src/rpc/methods/last_committed_block.rs index 658c1f82..1715b28d 100644 --- a/node/tools/src/rpc/methods/last_committed_block.rs +++ b/node/tools/src/rpc/methods/last_committed_block.rs @@ -14,8 +14,7 @@ pub fn callback(node_storage: Arc) -> RpcResult { .last .context("Failed to get last state") .map_err(|_| ErrorObjectOwned::from(ErrorCode::InternalError))? - .header() - .number + .number() .0; Ok(serde_json::json!({ "last_committed_block": last_committed_block_header diff --git a/node/tools/src/rpc/methods/last_view.rs b/node/tools/src/rpc/methods/last_view.rs index e0255273..ad812dd3 100644 --- a/node/tools/src/rpc/methods/last_view.rs +++ b/node/tools/src/rpc/methods/last_view.rs @@ -5,18 +5,19 @@ use jsonrpsee::{ types::{error::ErrorCode, ErrorObjectOwned}, }; use std::sync::Arc; -use zksync_consensus_storage::BlockStore; +use zksync_consensus_storage::{BlockStore, Last}; /// Last view response for /last_view endpoint. pub fn callback(node_storage: Arc) -> RpcResult { let state = node_storage.queued(); - let last_view = state + let last_view = match state .last .context("Failed to get last state") .map_err(|_| ErrorObjectOwned::from(ErrorCode::InternalError))? - .view() - .number - .0; + { + Last::PreGenesis(_) => 0, + Last::Final(qc) => qc.header().number.0, + }; // TODO(gprusak): this is the view of the last finalized block, not the current view of the // replica. Fix this. Ok(serde_json::json!({ diff --git a/node/tools/src/store.rs b/node/tools/src/store.rs index c6622b75..f4fab28e 100644 --- a/node/tools/src/store.rs +++ b/node/tools/src/store.rs @@ -8,7 +8,9 @@ use std::{ }; use zksync_concurrency::{ctx, error::Wrap as _, scope, sync}; use zksync_consensus_roles::validator; -use zksync_consensus_storage::{BlockStoreState, PersistentBlockStore, ReplicaState, ReplicaStore}; +use zksync_consensus_storage::{ + BlockStoreState, Last, PersistentBlockStore, ReplicaState, ReplicaStore, +}; /// Enum used to represent a key in the database. It also acts as a separator between different stores. #[derive(Debug, Clone, PartialEq, Eq)] @@ -88,7 +90,7 @@ impl RocksDB { }))) } - fn last_blocking(db: &rocksdb::DB) -> anyhow::Result> { + fn last_blocking(db: &rocksdb::DB) -> anyhow::Result> { let mut options = ReadOptions::default(); options.set_iterate_range(DatabaseKey::BLOCKS_START_KEY..); let Some(res) = db @@ -97,10 +99,10 @@ impl RocksDB { else { return Ok(None); }; - let (_, last) = res.context("RocksDB error reading head block")?; - let last: validator::FinalBlock = - zksync_protobuf::decode(&last).context("Failed decoding head block bytes")?; - Ok(Some(last.justification)) + let (_, raw) = res.context("RocksDB error reading head block")?; + let b: validator::Block = + zksync_protobuf::decode(&raw).context("Failed decoding head block bytes")?; + Ok(Some((&b).into())) } } @@ -124,7 +126,7 @@ impl PersistentBlockStore for RocksDB { &self, _ctx: &ctx::Ctx, number: validator::BlockNumber, - ) -> ctx::Result { + ) -> ctx::Result { scope::wait_blocking(|| { let db = self.0.db.read().unwrap(); let block = db @@ -137,12 +139,16 @@ impl PersistentBlockStore for RocksDB { .wrap(number) } - #[tracing::instrument(skip_all, fields(l2_block = %block.justification.message.proposal.number))] - async fn queue_next_block( + async fn verify_pregenesis_block( &self, _ctx: &ctx::Ctx, - block: validator::FinalBlock, + _block: &validator::PreGenesisBlock, ) -> ctx::Result<()> { + Err(anyhow::format_err!("pre-genesis blocks not supported").into()) + } + + #[tracing::instrument(skip_all, fields(l2_block = %block.number()))] + async fn queue_next_block(&self, _ctx: &ctx::Ctx, block: validator::Block) -> ctx::Result<()> { scope::wait_blocking(|| { // We use an exclusive lock so no other thread can change the expected block number // between the check and the insertion into the database. We could use a RocksDB @@ -155,10 +161,9 @@ impl PersistentBlockStore for RocksDB { "got {:?} want {want:?}", block.number() ); - let block_number = block.header().number; let mut write_batch = rocksdb::WriteBatch::default(); write_batch.put( - DatabaseKey::Block(block_number).encode_key(), + DatabaseKey::Block(block.number()).encode_key(), zksync_protobuf::encode(&block), ); // Commit the transaction. @@ -166,11 +171,11 @@ impl PersistentBlockStore for RocksDB { .context("Failed writing block to database")?; self.0 .persisted - .send_modify(|p| p.last = Some(block.justification.clone())); + .send_modify(|p| p.last = Some((&block).into())); Ok(()) }) .await - .context(block.header().number)?; + .context(block.number())?; Ok(()) } } diff --git a/node/tools/src/tests.rs b/node/tools/src/tests.rs index 50e82f66..9381700c 100644 --- a/node/tools/src/tests.rs +++ b/node/tools/src/tests.rs @@ -18,7 +18,6 @@ impl Distribution for EncodeDist { genesis: rng.gen(), max_payload_size: rng.gen(), - max_batch_size: rng.gen(), validator_key: self.sample_opt(|| rng.gen()), attester_key: self.sample_opt(|| rng.gen()), @@ -74,6 +73,9 @@ async fn test_reopen_rocksdb() { setup.push_blocks(rng, 5); let mut want = vec![]; for b in &setup.blocks { + if b.number() < setup.genesis.first_block { + continue; + } let store = store::RocksDB::open(setup.genesis.clone(), dir.path()) .await .unwrap();