diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index c9edc4770581..d815619934f1 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -23,8 +23,8 @@ use zksync_core::{ reorg_detector::ReorgDetector, setup_sigint_handler, state_keeper::{ - L1BatchExecutorBuilder, MainBatchExecutorBuilder, MiniblockSealer, MiniblockSealerHandle, - ZkSyncStateKeeper, + seal_criteria::NoopSealer, L1BatchExecutorBuilder, MainBatchExecutorBuilder, + MiniblockSealer, MiniblockSealerHandle, ZkSyncStateKeeper, }, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, fetcher::FetcherCursor, @@ -92,7 +92,12 @@ async fn build_state_keeper( ) .await; - ZkSyncStateKeeper::without_sealer(stop_receiver, Box::new(io), batch_executor_base) + ZkSyncStateKeeper::new( + stop_receiver, + Box::new(io), + batch_executor_base, + Box::new(NoopSealer), + ) } async fn init_tasks( diff --git a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs index ad332a899308..f99b3cba5ed7 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs +++ b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs @@ -39,7 +39,7 @@ use crate::{ }, l1_gas_price::L1GasPriceProvider, metrics::{TxStage, APP_METRICS}, - state_keeper::seal_criteria::{ConditionalSealer, SealData}, + state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}, }; mod proxy; @@ -139,9 +139,8 @@ pub struct TxSenderBuilder { master_connection_pool: Option, /// Proxy to submit transactions to the network. If not set, `master_connection_pool` must be set. proxy: Option, - /// Actual state keeper configuration, required for tx verification. - /// If not set, transactions would not be checked against seal criteria. - state_keeper_config: Option, + /// Batch sealer used to check whether transaction can be executed by the sequencer. + sealer: Option>, } impl TxSenderBuilder { @@ -151,10 +150,15 @@ impl TxSenderBuilder { replica_connection_pool, master_connection_pool: None, proxy: None, - state_keeper_config: None, + sealer: None, } } + pub fn with_sealer(mut self, sealer: Arc) -> Self { + self.sealer = Some(sealer); + self + } + pub fn with_tx_proxy(mut self, main_node_url: &str) -> Self { self.proxy = Some(TxProxy::new(main_node_url)); self @@ -165,11 +169,6 @@ impl TxSenderBuilder { self } - pub fn with_state_keeper_config(mut self, state_keeper_config: StateKeeperConfig) -> Self { - self.state_keeper_config = Some(state_keeper_config); - self - } - pub async fn build( self, l1_gas_price_source: Arc, @@ -182,6 +181,9 @@ impl TxSenderBuilder { "Either master connection pool or proxy must be set" ); + // Use noop sealer if no sealer was explicitly provided. + let sealer = self.sealer.unwrap_or_else(|| Arc::new(NoopSealer)); + TxSender(Arc::new(TxSenderInner { sender_config: self.config, master_connection_pool: self.master_connection_pool, @@ -189,9 +191,9 @@ impl TxSenderBuilder { l1_gas_price_source, api_contracts, proxy: self.proxy, - state_keeper_config: self.state_keeper_config, vm_concurrency_limiter, storage_caches, + sealer, })) } } @@ -241,14 +243,12 @@ pub struct TxSenderInner { pub(super) api_contracts: ApiContracts, /// Optional transaction proxy to be used for transaction submission. pub(super) proxy: Option, - /// An up-to-date version of the state keeper config. - /// This field may be omitted on the external node, since the configuration may change unexpectedly. - /// If this field is set to `None`, `TxSender` will assume that any transaction is executable. - state_keeper_config: Option, /// Used to limit the amount of VMs that can be executed simultaneously. pub(super) vm_concurrency_limiter: Arc, // Caches used in VM execution. storage_caches: PostgresStorageCaches, + /// Batch sealer used to check whether transaction can be executed by the sequencer. + sealer: Arc, } #[derive(Clone)] @@ -850,13 +850,6 @@ impl TxSender { tx_metrics: &TransactionExecutionMetrics, log_message: bool, ) -> Result<(), SubmitTxError> { - let Some(sk_config) = &self.0.state_keeper_config else { - // No config provided, so we can't check if transaction satisfies the seal criteria. - // We assume that it's executable, and if it's not, it will be caught by the main server - // (where this check is always performed). - return Ok(()); - }; - // Hash is not computable for the provided `transaction` during gas estimation (it doesn't have // its input data set). Since we don't log a hash in this case anyway, we just use a dummy value. let tx_hash = if log_message { @@ -870,8 +863,10 @@ impl TxSender { // still reject them as it's not. let protocol_version = ProtocolVersionId::latest(); let seal_data = SealData::for_transaction(transaction, tx_metrics, protocol_version); - if let Some(reason) = - ConditionalSealer::find_unexecutable_reason(sk_config, &seal_data, protocol_version) + if let Some(reason) = self + .0 + .sealer + .find_unexecutable_reason(&seal_data, protocol_version) { let message = format!( "Tx is Unexecutable because of {reason}; inputs for decision: {seal_data:?}" diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 4195cd76a054..9de6f7085e20 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -12,6 +12,7 @@ use zksync_types::{ use crate::{ genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ + seal_criteria::NoopSealer, tests::{create_l1_batch_metadata, create_l2_transaction, MockBatchExecutorBuilder}, MiniblockSealer, ZkSyncStateKeeper, }, @@ -355,10 +356,11 @@ impl StateKeeperRunner { s.spawn_bg(miniblock_sealer.run()); s.spawn_bg(run_mock_metadata_calculator(ctx, pool.clone())); s.spawn_bg( - ZkSyncStateKeeper::without_sealer( + ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(MockBatchExecutorBuilder), + Box::new(NoopSealer), ) .run(), ); diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 936ad4cfbe21..8269793a40b1 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -70,7 +70,9 @@ use crate::{ MetadataCalculator, MetadataCalculatorConfig, MetadataCalculatorModeConfig, }, metrics::{InitStage, APP_METRICS}, - state_keeper::{create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer}, + state_keeper::{ + create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer, SequencerSealer, + }, }; pub mod api_server; @@ -1032,9 +1034,10 @@ async fn build_tx_sender( l1_gas_price_provider: Arc, storage_caches: PostgresStorageCaches, ) -> (TxSender, VmConcurrencyBarrier) { + let sequencer_sealer = SequencerSealer::new(state_keeper_config.clone()); let tx_sender_builder = TxSenderBuilder::new(tx_sender_config.clone(), replica_pool) .with_main_connection_pool(master_pool) - .with_state_keeper_config(state_keeper_config.clone()); + .with_sealer(Arc::new(sequencer_sealer)); let max_concurrency = web3_json_config.vm_concurrency_limit(); let (vm_concurrency_limiter, vm_barrier) = VmConcurrencyLimiter::new(max_concurrency); diff --git a/core/lib/zksync_core/src/state_keeper/keeper.rs b/core/lib/zksync_core/src/state_keeper/keeper.rs index 3cc153120c26..209809d33f95 100644 --- a/core/lib/zksync_core/src/state_keeper/keeper.rs +++ b/core/lib/zksync_core/src/state_keeper/keeper.rs @@ -58,7 +58,7 @@ pub struct ZkSyncStateKeeper { stop_receiver: watch::Receiver, io: Box, batch_executor_base: Box, - sealer: Option, + sealer: Box, } impl ZkSyncStateKeeper { @@ -66,26 +66,13 @@ impl ZkSyncStateKeeper { stop_receiver: watch::Receiver, io: Box, batch_executor_base: Box, - sealer: ConditionalSealer, + sealer: Box, ) -> Self { Self { stop_receiver, io, batch_executor_base, - sealer: Some(sealer), - } - } - - pub fn without_sealer( - stop_receiver: watch::Receiver, - io: Box, - batch_executor_base: Box, - ) -> Self { - Self { - stop_receiver, - io, - batch_executor_base, - sealer: None, + sealer, } } @@ -651,18 +638,14 @@ impl ZkSyncStateKeeper { writes_metrics: block_writes_metrics, }; - if let Some(sealer) = &self.sealer { - sealer.should_seal_l1_batch( - self.io.current_l1_batch_number().0, - updates_manager.batch_timestamp() as u128 * 1_000, - updates_manager.pending_executed_transactions_len() + 1, - &block_data, - &tx_data, - updates_manager.protocol_version(), - ) - } else { - SealResolution::NoSeal - } + self.sealer.should_seal_l1_batch( + self.io.current_l1_batch_number().0, + updates_manager.batch_timestamp() as u128 * 1_000, + updates_manager.pending_executed_transactions_len() + 1, + &block_data, + &tx_data, + updates_manager.protocol_version(), + ) } }; (resolution, exec_result) diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index 6175befb1baf..d9f09ba49c6e 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -16,7 +16,7 @@ pub use self::{ keeper::ZkSyncStateKeeper, }; pub(crate) use self::{ - mempool_actor::MempoolFetcher, seal_criteria::ConditionalSealer, types::MempoolGuard, + mempool_actor::MempoolFetcher, seal_criteria::SequencerSealer, types::MempoolGuard, }; use crate::l1_gas_price::L1GasPriceProvider; @@ -26,7 +26,7 @@ pub(crate) mod io; mod keeper; mod mempool_actor; pub(crate) mod metrics; -pub(crate) mod seal_criteria; +pub mod seal_criteria; #[cfg(test)] pub(crate) mod tests; pub(crate) mod types; @@ -76,11 +76,11 @@ pub(crate) async fn create_state_keeper( ) .await; - let sealer = ConditionalSealer::new(state_keeper_config); + let sealer = SequencerSealer::new(state_keeper_config); ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(batch_executor_base), - sealer, + Box::new(sealer), ) } diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs index 2b5fb9fba485..f239f108f5ba 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs @@ -1,7 +1,10 @@ //! This module represents the conditional sealer, which can decide whether the batch //! should be sealed after executing a particular transaction. -//! It is used on the main node to decide when the batch should be sealed (as opposed to the external node, -//! which unconditionally follows the instructions from the main node). +//! +//! The conditional sealer abstraction allows to implement different sealing strategies, e.g. the actual +//! sealing strategy for the main node or noop sealer for the external node. + +use std::fmt; use zksync_config::configs::chain::StateKeeperConfig; use zksync_types::ProtocolVersionId; @@ -9,28 +12,51 @@ use zksync_types::ProtocolVersionId; use super::{criteria, SealCriterion, SealData, SealResolution, AGGREGATION_METRICS}; /// Checks if an L1 batch should be sealed after executing a transaction. +pub trait ConditionalSealer: 'static + fmt::Debug + Send + Sync { + /// Finds a reason why a transaction with the specified `data` is unexecutable. + /// + /// Can be used to determine whether the transaction can be executed by the sequencer. + fn find_unexecutable_reason( + &self, + data: &SealData, + protocol_version: ProtocolVersionId, + ) -> Option<&'static str>; + + /// Returns the action that should be taken by the state keeper after executing a transaction. + fn should_seal_l1_batch( + &self, + l1_batch_number: u32, + block_open_timestamp_ms: u128, + tx_count: usize, + block_data: &SealData, + tx_data: &SealData, + protocol_version: ProtocolVersionId, + ) -> SealResolution; +} + +/// Implementation of [`ConditionalSealer`] used by the main node. +/// Internally uses a set of [`SealCriterion`]s to determine whether the batch should be sealed. /// /// The checks are deterministic, i.e., should depend solely on execution metrics and [`StateKeeperConfig`]. /// Non-deterministic seal criteria are expressed using [`IoSealCriteria`](super::IoSealCriteria). #[derive(Debug)] -pub struct ConditionalSealer { +pub struct SequencerSealer { config: StateKeeperConfig, sealers: Vec>, } -impl ConditionalSealer { - /// Finds a reason why a transaction with the specified `data` is unexecutable. - pub(crate) fn find_unexecutable_reason( - config: &StateKeeperConfig, +impl ConditionalSealer for SequencerSealer { + fn find_unexecutable_reason( + &self, data: &SealData, protocol_version: ProtocolVersionId, ) -> Option<&'static str> { - for sealer in &Self::default_sealers() { + for sealer in &self.sealers { const MOCK_BLOCK_TIMESTAMP: u128 = 0; const TX_COUNT: usize = 1; let resolution = sealer.should_seal( - config, + &self.config, MOCK_BLOCK_TIMESTAMP, TX_COUNT, data, @@ -44,20 +70,7 @@ impl ConditionalSealer { None } - pub(crate) fn new(config: StateKeeperConfig) -> Self { - let sealers = Self::default_sealers(); - Self { config, sealers } - } - - #[cfg(test)] - pub(in crate::state_keeper) fn with_sealers( - config: StateKeeperConfig, - sealers: Vec>, - ) -> Self { - Self { config, sealers } - } - - pub fn should_seal_l1_batch( + fn should_seal_l1_batch( &self, l1_batch_number: u32, block_open_timestamp_ms: u128, @@ -99,6 +112,21 @@ impl ConditionalSealer { } final_seal_resolution } +} + +impl SequencerSealer { + pub(crate) fn new(config: StateKeeperConfig) -> Self { + let sealers = Self::default_sealers(); + Self { config, sealers } + } + + #[cfg(test)] + pub(in crate::state_keeper) fn with_sealers( + config: StateKeeperConfig, + sealers: Vec>, + ) -> Self { + Self { config, sealers } + } fn default_sealers() -> Vec> { vec![ @@ -114,3 +142,31 @@ impl ConditionalSealer { ] } } + +/// Implementation of [`ConditionalSealer`] that never seals the batch. +/// Can be used in contexts where, for example, state keeper configuration is not available, +/// or the decision to seal batch is taken by some other component. +#[derive(Debug)] +pub struct NoopSealer; + +impl ConditionalSealer for NoopSealer { + fn find_unexecutable_reason( + &self, + _data: &SealData, + _protocol_version: ProtocolVersionId, + ) -> Option<&'static str> { + None + } + + fn should_seal_l1_batch( + &self, + _l1_batch_number: u32, + _block_open_timestamp_ms: u128, + _tx_count: usize, + _block_data: &SealData, + _tx_data: &SealData, + _protocol_version: ProtocolVersionId, + ) -> SealResolution { + SealResolution::NoSeal + } +} diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs index 99cb25c654d8..bf44c7af0ecf 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs @@ -25,7 +25,7 @@ use zksync_utils::time::millis_since; mod conditional_sealer; pub(super) mod criteria; -pub(crate) use self::conditional_sealer::ConditionalSealer; +pub use self::conditional_sealer::{ConditionalSealer, NoopSealer, SequencerSealer}; use super::{extractors, metrics::AGGREGATION_METRICS, updates::UpdatesManager}; use crate::gas_tracker::{gas_count_from_tx_and_metrics, gas_count_from_writes}; @@ -104,7 +104,7 @@ impl SealData { } } -pub(super) trait SealCriterion: fmt::Debug + Send + 'static { +pub(super) trait SealCriterion: fmt::Debug + Send + Sync + 'static { fn should_seal( &self, config: &StateKeeperConfig, diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index c725cdcf010b..044e65742e78 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -42,7 +42,7 @@ use crate::{ keeper::POLL_WAIT_DURATION, seal_criteria::{ criteria::{GasCriterion, SlotsCriterion}, - ConditionalSealer, + SequencerSealer, }, types::ExecutionMetricsForCriteria, updates::UpdatesManager, @@ -250,7 +250,7 @@ async fn sealed_by_number_of_txs() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) @@ -271,7 +271,7 @@ async fn sealed_by_gas() { close_block_at_gas_percentage: 0.5, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(GasCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(GasCriterion)]); let l1_gas_per_tx = BlockGasCount { commit: 1, // Both txs together with block_base_cost would bring it over the block 31_001 commit bound. @@ -320,7 +320,7 @@ async fn sealed_by_gas_then_by_num_tx() { transaction_slots: 3, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers( + let sealer = SequencerSealer::with_sealers( config, vec![Box::new(GasCriterion), Box::new(SlotsCriterion)], ); @@ -357,7 +357,7 @@ async fn batch_sealed_before_miniblock_does() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); // Miniblock sealer will not return true before the batch is sealed because the batch only has 2 txs. TestScenario::new() @@ -382,7 +382,7 @@ async fn rejected_tx() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let rejected_tx = random_tx(1); TestScenario::new() @@ -404,7 +404,7 @@ async fn bootloader_tip_out_of_gas_flow() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let first_tx = random_tx(1); let bootloader_out_of_gas_tx = random_tx(2); @@ -442,7 +442,7 @@ async fn pending_batch_is_applied() { transaction_slots: 3, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let pending_batch = pending_batch_data(vec![ MiniblockExecutionData { @@ -500,7 +500,7 @@ async fn unconditional_sealing() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_l1_batch_when(move |_| batch_seal_trigger_checker.load(Ordering::Relaxed)) @@ -530,7 +530,7 @@ async fn miniblock_timestamp_after_pending_batch() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let pending_batch = pending_batch_data(vec![MiniblockExecutionData { number: MiniblockNumber(1), @@ -574,7 +574,7 @@ async fn time_is_monotonic() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) @@ -625,7 +625,7 @@ async fn protocol_upgrade() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) diff --git a/core/lib/zksync_core/src/state_keeper/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/tests/tester.rs index 2c26370a22a9..ce0293e7e679 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/tester.rs @@ -23,7 +23,7 @@ use zksync_types::{ use crate::state_keeper::{ batch_executor::{BatchExecutorHandle, Command, L1BatchExecutorBuilder, TxExecutionResult}, io::{MiniblockParams, PendingBatchData, StateKeeperIO}, - seal_criteria::{ConditionalSealer, IoSealCriteria}, + seal_criteria::{IoSealCriteria, SequencerSealer}, tests::{ create_l2_transaction, default_l1_batch_env, default_vm_block_result, BASE_SYSTEM_CONTRACTS, }, @@ -189,7 +189,7 @@ impl TestScenario { /// Launches the test. /// Provided `SealManager` is expected to be externally configured to adhere the written scenario logic. - pub(crate) async fn run(self, sealer: ConditionalSealer) { + pub(crate) async fn run(self, sealer: SequencerSealer) { assert!(!self.actions.is_empty(), "Test scenario can't be empty"); let batch_executor_base = TestBatchExecutorBuilder::new(&self); @@ -199,7 +199,7 @@ impl TestScenario { stop_receiver, Box::new(io), Box::new(batch_executor_base), - sealer, + Box::new(sealer), ); let sk_thread = tokio::spawn(sk.run()); diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 5b36f3c039bb..619ee282e718 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -19,6 +19,7 @@ use crate::{ consensus::testonly::MockMainNodeClient, genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ + seal_criteria::NoopSealer, tests::{create_l1_batch_metadata, create_l2_transaction, TestBatchExecutorBuilder}, MiniblockSealer, ZkSyncStateKeeper, }, @@ -77,10 +78,11 @@ impl StateKeeperHandles { batch_executor_base.push_successful_transactions(tx_hashes_in_l1_batch); } - let state_keeper = ZkSyncStateKeeper::without_sealer( + let state_keeper = ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(batch_executor_base), + Box::new(NoopSealer), ); Self { stop_sender, diff --git a/spellcheck/era.dic b/spellcheck/era.dic index a054a5930270..9c0f6bb87401 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -607,4 +607,5 @@ dal codebase compactions M6 -compiler_common \ No newline at end of file +compiler_common +noop \ No newline at end of file