diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 9056350afdd..85887ed1557 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -5,7 +5,7 @@ use clap::Parser; use metrics::EN_METRICS; use prometheus_exporter::PrometheusExporterConfig; use tokio::{sync::watch, task}; -use zksync_basic_types::{Address, L2ChainId}; +use zksync_basic_types::L2ChainId; use zksync_concurrency::{ctx, limiter, scope, time}; use zksync_config::configs::database::MerkleTreeMode; use zksync_core::{ @@ -25,8 +25,8 @@ use zksync_core::{ reorg_detector::ReorgDetector, setup_sigint_handler, state_keeper::{ - seal_criteria::NoopSealer, BatchExecutor, MainBatchExecutor, MiniblockSealer, - MiniblockSealerHandle, ZkSyncStateKeeper, + seal_criteria::NoopSealer, BatchExecutor, MainBatchExecutor, OutputHandler, + StateKeeperPersistence, ZkSyncStateKeeper, }, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, @@ -62,25 +62,16 @@ async fn build_state_keeper( state_keeper_db_path: String, config: &ExternalNodeConfig, connection_pool: ConnectionPool, - sync_state: SyncState, - l2_erc20_bridge_addr: Address, - miniblock_sealer_handle: MiniblockSealerHandle, + output_handler: OutputHandler, stop_receiver: watch::Receiver, chain_id: L2ChainId, ) -> anyhow::Result { - // These config values are used on the main node, and depending on these values certain transactions can - // be *rejected* (that is, not included into the block). However, external node only mirrors what the main - // node has already executed, so we can safely set these values to the maximum possible values - if the main - // node has already executed the transaction, then the external node must execute it too. - let max_allowed_l2_tx_gas_limit = u32::MAX.into(); - let validation_computational_gas_limit = u32::MAX; // We only need call traces on the external node if the `debug_` namespace is enabled. let save_call_traces = config.optional.api_namespaces().contains(&Namespace::Debug); let batch_executor_base: Box = Box::new(MainBatchExecutor::new( state_keeper_db_path, connection_pool.clone(), - max_allowed_l2_tx_gas_limit, save_call_traces, false, config.optional.enum_index_migration_chunk_size, @@ -91,13 +82,9 @@ async fn build_state_keeper( let main_node_client = ::json_rpc(&main_node_url) .context("Failed creating JSON-RPC client for main node")?; let io = ExternalIO::new( - miniblock_sealer_handle, connection_pool, action_queue, - sync_state, Box::new(main_node_client), - l2_erc20_bridge_addr, - validation_computational_gas_limit, chain_id, ) .await @@ -107,6 +94,7 @@ async fn build_state_keeper( stop_receiver, Box::new(io), batch_executor_base, + output_handler, Arc::new(NoopSealer), )) } @@ -134,8 +122,9 @@ async fn init_tasks( app_health.insert_custom_component(Arc::new(sync_state.clone())); let (action_queue_sender, action_queue) = ActionQueue::new(); - let (miniblock_sealer, miniblock_sealer_handle) = MiniblockSealer::new( + let (persistence, miniblock_sealer) = StateKeeperPersistence::new( connection_pool.clone(), + config.remote.l2_erc20_bridge_addr, config.optional.miniblock_seal_queue_capacity, ); task_handles.push(tokio::spawn(miniblock_sealer.run())); @@ -157,14 +146,14 @@ async fn init_tasks( } })); + let output_handler = OutputHandler::new(Box::new(persistence.with_tx_insertion())) + .with_handler(Box::new(sync_state.clone())); let state_keeper = build_state_keeper( action_queue, config.required.state_cache_path.clone(), config, connection_pool.clone(), - sync_state.clone(), - config.remote.l2_erc20_bridge_addr, - miniblock_sealer_handle, + output_handler, stop_receiver.clone(), config.remote.l2_chain_id, ) diff --git a/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json b/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json deleted file mode 100644 index bb47b8254ac..00000000000 --- a/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json +++ /dev/null @@ -1,34 +0,0 @@ -{ - "db_name": "PostgreSQL", - "query": "\n SELECT\n bootloader_code_hash,\n default_account_code_hash,\n id\n FROM\n protocol_versions\n WHERE\n timestamp <= $1\n ORDER BY\n id DESC\n LIMIT\n 1\n ", - "describe": { - "columns": [ - { - "ordinal": 0, - "name": "bootloader_code_hash", - "type_info": "Bytea" - }, - { - "ordinal": 1, - "name": "default_account_code_hash", - "type_info": "Bytea" - }, - { - "ordinal": 2, - "name": "id", - "type_info": "Int4" - } - ], - "parameters": { - "Left": [ - "Int8" - ] - }, - "nullable": [ - false, - false, - false - ] - }, - "hash": "2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef" -} diff --git a/core/lib/dal/.sqlx/query-d50555b3b7a1feadd34a2fe90b205ed6a7335a7092c7dc2b81337535e4d489d5.json b/core/lib/dal/.sqlx/query-d50555b3b7a1feadd34a2fe90b205ed6a7335a7092c7dc2b81337535e4d489d5.json new file mode 100644 index 00000000000..7a5668b6df6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-d50555b3b7a1feadd34a2fe90b205ed6a7335a7092c7dc2b81337535e4d489d5.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id\n FROM\n protocol_versions\n WHERE\n timestamp <= $1\n ORDER BY\n id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "d50555b3b7a1feadd34a2fe90b205ed6a7335a7092c7dc2b81337535e4d489d5" +} diff --git a/core/lib/dal/src/protocol_versions_dal.rs b/core/lib/dal/src/protocol_versions_dal.rs index 13100470736..1c4930c86b1 100644 --- a/core/lib/dal/src/protocol_versions_dal.rs +++ b/core/lib/dal/src/protocol_versions_dal.rs @@ -138,15 +138,13 @@ impl ProtocolVersionsDal<'_, '_> { db_transaction.commit().await.unwrap(); } - pub async fn base_system_contracts_by_timestamp( + pub async fn protocol_version_id_by_timestamp( &mut self, current_timestamp: u64, - ) -> anyhow::Result<(BaseSystemContracts, ProtocolVersionId)> { + ) -> sqlx::Result { let row = sqlx::query!( r#" SELECT - bootloader_code_hash, - default_account_code_hash, id FROM protocol_versions @@ -160,21 +158,9 @@ impl ProtocolVersionsDal<'_, '_> { current_timestamp as i64 ) .fetch_one(self.storage.conn()) - .await - .context("cannot fetch system contract hashes")?; + .await?; - let protocol_version = (row.id as u16) - .try_into() - .context("bogus protocol version ID")?; - let contracts = self - .storage - .factory_deps_dal() - .get_base_system_contracts( - H256::from_slice(&row.bootloader_code_hash), - H256::from_slice(&row.default_account_code_hash), - ) - .await?; - Ok((contracts, protocol_version)) + ProtocolVersionId::try_from(row.id as u16).map_err(|err| sqlx::Error::Decode(err.into())) } pub async fn load_base_system_contracts_by_version_id( diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 341a106270b..ef84aeb341e 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -1,4 +1,5 @@ //! Utilities for testing the consensus module. + use std::{collections::HashMap, sync::Arc}; use anyhow::Context as _; @@ -22,8 +23,10 @@ use crate::{ consensus::{fetcher::P2PConfig, Fetcher, Store}, genesis::{mock_genesis_config, GenesisParams}, state_keeper::{ - io::common::IoCursor, seal_criteria::NoopSealer, tests::MockBatchExecutor, MiniblockSealer, - ZkSyncStateKeeper, + io::{IoCursor, L1BatchParams, MiniblockParams}, + seal_criteria::NoopSealer, + tests::MockBatchExecutor, + OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, }, sync_layer::{ fetcher::FetchedTransaction, @@ -154,7 +157,6 @@ pub(super) struct StateKeeper { fee_per_gas: u64, gas_per_pubdata: u64, - sync_state: SyncState, actions_sender: ActionQueueSender, addr: sync::watch::Receiver>, store: Store, @@ -163,7 +165,6 @@ pub(super) struct StateKeeper { /// Fake StateKeeper task to be executed in the background. pub(super) struct StateKeeperRunner { actions_queue: ActionQueue, - sync_state: SyncState, store: Store, addr: sync::watch::Sender>, } @@ -193,7 +194,6 @@ impl StateKeeper { .await? .context("pending_batch_exists()")?; let (actions_sender, actions_queue) = ActionQueue::new(); - let sync_state = SyncState::default(); let addr = sync::watch::channel(None).0; Ok(( Self { @@ -204,12 +204,10 @@ impl StateKeeper { fee_per_gas: 10, gas_per_pubdata: 100, actions_sender, - sync_state: sync_state.clone(), addr: addr.subscribe(), store: store.clone(), }, StateKeeperRunner { - sync_state, actions_queue, store: store.clone(), addr, @@ -224,22 +222,28 @@ impl StateKeeper { self.last_timestamp += 5; self.batch_sealed = false; SyncAction::OpenBatch { + params: L1BatchParams { + protocol_version: ProtocolVersionId::latest(), + validation_computational_gas_limit: u32::MAX, + operator_address: GenesisParams::mock().config().fee_account, + fee_input: Default::default(), + first_miniblock: MiniblockParams { + timestamp: self.last_timestamp, + virtual_blocks: 1, + }, + }, number: self.last_batch, - timestamp: self.last_timestamp, - l1_gas_price: 2, - l2_fair_gas_price: 3, - fair_pubdata_price: Some(24), - operator_address: GenesisParams::mock().config().fee_account, - protocol_version: ProtocolVersionId::latest(), - first_miniblock_info: (self.last_block, 1), + first_miniblock_number: self.last_block, } } else { self.last_block += 1; self.last_timestamp += 2; SyncAction::Miniblock { + params: MiniblockParams { + timestamp: self.last_timestamp, + virtual_blocks: 0, + }, number: self.last_block, - timestamp: self.last_timestamp, - virtual_blocks: 0, } } } @@ -260,7 +264,7 @@ impl StateKeeper { pub async fn seal_batch(&mut self) { // Each batch ends with an empty block (aka fictive block). let mut actions = vec![self.open_block()]; - actions.push(SyncAction::SealBatch { virtual_blocks: 0 }); + actions.push(SyncAction::SealBatch); self.actions_sender.push_actions(actions).await; self.batch_sealed = true; } @@ -302,7 +306,7 @@ impl StateKeeper { Fetcher { store: self.store, client: Box::new(client), - sync_state: self.sync_state, + sync_state: SyncState::default(), limiter: unbounded_limiter(ctx), } .run_centralized(ctx, self.actions_sender) @@ -319,7 +323,7 @@ impl StateKeeper { Fetcher { store: self.store, client: Box::new(client), - sync_state: self.sync_state, + sync_state: SyncState::default(), limiter: unbounded_limiter(ctx), } .run_p2p(ctx, self.actions_sender, cfg) @@ -367,17 +371,13 @@ impl StateKeeperRunner { pub async fn run(self, ctx: &ctx::Ctx) -> anyhow::Result<()> { let res = scope::run!(ctx, |ctx, s| async { let (stop_send, stop_recv) = sync::watch::channel(false); - let (miniblock_sealer, miniblock_sealer_handle) = - MiniblockSealer::new(self.store.0.clone(), 5); + let (persistence, miniblock_sealer) = + StateKeeperPersistence::new(self.store.0.clone(), Address::repeat_byte(11), 5); let io = ExternalIO::new( - miniblock_sealer_handle, self.store.0.clone(), self.actions_queue, - self.sync_state, Box::::default(), - Address::repeat_byte(11), - u32::MAX, L2ChainId::default(), ) .await?; @@ -402,6 +402,7 @@ impl StateKeeperRunner { stop_recv, Box::new(io), Box::new(MockBatchExecutor), + OutputHandler::new(Box::new(persistence.with_tx_insertion())), Arc::new(NoopSealer), ) .run() diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index dd40acce947..3e898e25845 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -76,7 +76,8 @@ use crate::{ metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, metrics::{InitStage, APP_METRICS}, state_keeper::{ - create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer, SequencerSealer, + create_state_keeper, MempoolFetcher, MempoolGuard, OutputHandler, SequencerSealer, + StateKeeperPersistence, }, }; @@ -765,14 +766,15 @@ async fn add_state_keeper_to_task_futures( .build() .await .context("failed to build miniblock_sealer_pool")?; - let (miniblock_sealer, miniblock_sealer_handle) = MiniblockSealer::new( + let (persistence, miniblock_sealer) = StateKeeperPersistence::new( miniblock_sealer_pool, + contracts_config.l2_erc20_bridge_addr, state_keeper_config.miniblock_seal_queue_capacity, ); + let persistence = persistence.with_object_store(object_store); task_futures.push(tokio::spawn(miniblock_sealer.run())); let state_keeper = create_state_keeper( - contracts_config, state_keeper_config, db_config, network_config, @@ -780,8 +782,7 @@ async fn add_state_keeper_to_task_futures( state_keeper_pool.clone(), mempool.clone(), batch_fee_input_provider.clone(), - miniblock_sealer_handle, - object_store, + OutputHandler::new(Box::new(persistence)), stop_receiver.clone(), ) .await; diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs index e49b69ea679..38c6e974468 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs @@ -14,7 +14,7 @@ use once_cell::sync::OnceCell; use tokio::sync::{mpsc, watch}; use zksync_dal::{ConnectionPool, Core}; use zksync_state::{RocksdbStorage, StorageView, WriteStorage}; -use zksync_types::{vm_trace::Call, Transaction, U256}; +use zksync_types::{vm_trace::Call, Transaction}; use zksync_utils::bytecode::CompressedBytecodeInfo; use super::{BatchExecutor, BatchExecutorHandle, Command, TxExecutionResult}; @@ -33,7 +33,6 @@ pub struct MainBatchExecutor { state_keeper_db_path: String, pool: ConnectionPool, save_call_traces: bool, - max_allowed_tx_gas_limit: U256, upload_witness_inputs_to_gcs: bool, enum_index_migration_chunk_size: usize, optional_bytecode_compression: bool, @@ -43,7 +42,6 @@ impl MainBatchExecutor { pub fn new( state_keeper_db_path: String, pool: ConnectionPool, - max_allowed_tx_gas_limit: U256, save_call_traces: bool, upload_witness_inputs_to_gcs: bool, enum_index_migration_chunk_size: usize, @@ -53,7 +51,6 @@ impl MainBatchExecutor { state_keeper_db_path, pool, save_call_traces, - max_allowed_tx_gas_limit, upload_witness_inputs_to_gcs, enum_index_migration_chunk_size, optional_bytecode_compression, @@ -84,7 +81,6 @@ impl BatchExecutor for MainBatchExecutor { let (commands_sender, commands_receiver) = mpsc::channel(1); let executor = CommandReceiver { save_call_traces: self.save_call_traces, - max_allowed_tx_gas_limit: self.max_allowed_tx_gas_limit, optional_bytecode_compression: self.optional_bytecode_compression, commands: commands_receiver, }; @@ -114,7 +110,6 @@ impl BatchExecutor for MainBatchExecutor { #[derive(Debug)] struct CommandReceiver { save_call_traces: bool, - max_allowed_tx_gas_limit: U256, optional_bytecode_compression: bool, commands: mpsc::Receiver, } @@ -179,20 +174,6 @@ impl CommandReceiver { // Save pre-`execute_next_tx` VM snapshot. vm.make_snapshot(); - // Reject transactions with too big gas limit. - // They are also rejected on the API level, but - // we need to secure ourselves in case some tx will somehow get into mempool. - if tx.gas_limit() > self.max_allowed_tx_gas_limit { - tracing::warn!( - "Found tx with too big gas limit in state keeper, hash: {:?}, gas_limit: {}", - tx.hash(), - tx.gas_limit() - ); - return TxExecutionResult::RejectedByVm { - reason: Halt::TooBigGasLimit, - }; - } - // Execute the transaction. let latency = KEEPER_METRICS.tx_execution_time[&TxExecutionStage::Execution].start(); let (tx_result, compressed_bytecodes, call_tracer_result) = diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs index 1fc7289cf94..c74816891c0 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs @@ -374,7 +374,6 @@ async fn bootloader_out_of_gas_for_any_tx() { TestConfig { save_call_traces: false, vm_gas_limit: Some(10), - max_allowed_tx_gas_limit: u32::MAX, validation_computational_gas_limit: u32::MAX, upload_witness_inputs_to_gcs: false, }, @@ -411,7 +410,6 @@ async fn bootloader_tip_out_of_gas() { tester.set_config(TestConfig { save_call_traces: false, vm_gas_limit: Some(vm_block_res.block_tip_execution_result.statistics.gas_used - 10), - max_allowed_tx_gas_limit: u32::MAX, validation_computational_gas_limit: u32::MAX, upload_witness_inputs_to_gcs: false, }); diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs index 28214c22075..4560913d33a 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs @@ -42,7 +42,6 @@ const CHAIN_ID: u32 = 270; pub(super) struct TestConfig { pub(super) save_call_traces: bool, pub(super) vm_gas_limit: Option, - pub(super) max_allowed_tx_gas_limit: u32, pub(super) validation_computational_gas_limit: u32, pub(super) upload_witness_inputs_to_gcs: bool, } @@ -54,7 +53,6 @@ impl TestConfig { Self { vm_gas_limit: None, save_call_traces: false, - max_allowed_tx_gas_limit: config.max_allowed_l2_tx_gas_limit, validation_computational_gas_limit: config.validation_computational_gas_limit, upload_witness_inputs_to_gcs: false, } @@ -106,7 +104,6 @@ impl Tester { let mut builder = MainBatchExecutor::new( self.db_dir.path().to_str().unwrap().to_owned(), self.pool.clone(), - self.config.max_allowed_tx_gas_limit.into(), self.config.save_call_traces, self.config.upload_witness_inputs_to_gcs, 100, diff --git a/core/lib/zksync_core/src/state_keeper/io/common/mod.rs b/core/lib/zksync_core/src/state_keeper/io/common/mod.rs index 1ba0890867a..e285ad9492e 100644 --- a/core/lib/zksync_core/src/state_keeper/io/common/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/common/mod.rs @@ -21,7 +21,7 @@ pub(crate) fn poll_iters(delay_interval: Duration, max_wait: Duration) -> usize /// Cursor of the miniblock / L1 batch progress used by [`StateKeeperIO`](super::StateKeeperIO) implementations. #[derive(Debug)] -pub(crate) struct IoCursor { +pub struct IoCursor { pub next_miniblock: MiniblockNumber, pub prev_miniblock_hash: H256, pub prev_miniblock_timestamp: u64, diff --git a/core/lib/zksync_core/src/state_keeper/io/fee_address_migration.rs b/core/lib/zksync_core/src/state_keeper/io/fee_address_migration.rs index 160d64a0297..d1f80c32776 100644 --- a/core/lib/zksync_core/src/state_keeper/io/fee_address_migration.rs +++ b/core/lib/zksync_core/src/state_keeper/io/fee_address_migration.rs @@ -41,7 +41,6 @@ pub(crate) async fn migrate_pending_miniblocks( /// Runs the migration for non-pending miniblocks. Should be run as a background task. pub(crate) async fn migrate_miniblocks( pool: ConnectionPool, - last_miniblock: MiniblockNumber, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { // `migrate_miniblocks_inner` assumes that miniblocks start from the genesis (i.e., no snapshot recovery). @@ -58,6 +57,11 @@ pub(crate) async fn migrate_miniblocks( tracing::info!("Detected snapshot recovery; fee address migration is skipped as no-op"); return Ok(()); } + let last_miniblock = storage + .blocks_dal() + .get_sealed_miniblock_number() + .await? + .context("storage is empty, but there's no snapshot recovery data")?; drop(storage); let MigrationOutput { diff --git a/core/lib/zksync_core/src/state_keeper/io/mempool.rs b/core/lib/zksync_core/src/state_keeper/io/mempool.rs index ab4cef13f08..606090a2bb5 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mempool.rs +++ b/core/lib/zksync_core/src/state_keeper/io/mempool.rs @@ -7,18 +7,15 @@ use std::{ use anyhow::Context as _; use async_trait::async_trait; -use multivm::{ - interface::{FinishedL1Batch, L1BatchEnv, SystemEnv}, - utils::derive_base_fee_and_gas_per_pubdata, -}; -use vm_utils::storage::{l1_batch_params, L1BatchParamsProvider}; +use multivm::{interface::Halt, utils::derive_base_fee_and_gas_per_pubdata}; +use vm_utils::storage::L1BatchParamsProvider; use zksync_config::configs::chain::StateKeeperConfig; +use zksync_contracts::BaseSystemContracts; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_mempool::L2TxFilter; -use zksync_object_store::ObjectStore; use zksync_types::{ - protocol_upgrade::ProtocolUpgradeTx, witness_block_state::WitnessBlockState, Address, - L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, Transaction, H256, + protocol_upgrade::ProtocolUpgradeTx, Address, L1BatchNumber, L2ChainId, MiniblockNumber, + ProtocolVersionId, Transaction, H256, U256, }; // TODO (SMA-1206): use seconds instead of milliseconds. use zksync_utils::time::millis_since_epoch; @@ -29,18 +26,17 @@ use crate::{ extractors, io::{ common::{load_pending_batch, poll_iters, IoCursor}, - fee_address_migration, MiniblockParams, MiniblockSealerHandle, PendingBatchData, - StateKeeperIO, + fee_address_migration, L1BatchParams, MiniblockParams, PendingBatchData, StateKeeperIO, }, mempool_actor::l2_tx_filter, metrics::KEEPER_METRICS, seal_criteria::{IoSealCriteria, TimeoutSealer}, - updates::{MiniblockUpdates, UpdatesManager}, + updates::UpdatesManager, MempoolGuard, }, }; -/// Mempool-based IO for the state keeper. +/// Mempool-based sequencer for the state keeper. /// Receives transactions from the database through the mempool filtering logic. /// Decides which batch parameters should be used for the new batch. /// This is an IO for the main server application. @@ -48,21 +44,15 @@ use crate::{ pub struct MempoolIO { mempool: MempoolGuard, pool: ConnectionPool, - object_store: Arc, timeout_sealer: TimeoutSealer, filter: L2TxFilter, - current_miniblock_number: MiniblockNumber, - prev_miniblock_hash: H256, - prev_miniblock_timestamp: u64, l1_batch_params_provider: L1BatchParamsProvider, - miniblock_sealer_handle: MiniblockSealerHandle, - current_l1_batch_number: L1BatchNumber, fee_account: Address, validation_computational_gas_limit: u32, + max_allowed_tx_gas_limit: U256, delay_interval: Duration, // Used to keep track of gas prices to set accepted price per pubdata byte in blocks. batch_fee_input_provider: Arc, - l2_erc20_bridge_addr: Address, chain_id: L2ChainId, virtual_blocks_interval: u32, @@ -82,29 +72,26 @@ impl IoSealCriteria for MempoolIO { #[async_trait] impl StateKeeperIO for MempoolIO { - fn current_l1_batch_number(&self) -> L1BatchNumber { - self.current_l1_batch_number - } - - fn current_miniblock_number(&self) -> MiniblockNumber { - self.current_miniblock_number + fn chain_id(&self) -> L2ChainId { + self.chain_id } - async fn load_pending_batch(&mut self) -> anyhow::Result> { + async fn initialize(&mut self) -> anyhow::Result<(IoCursor, Option)> { let mut storage = self.pool.connection_tagged("state_keeper").await?; + let cursor = IoCursor::new(&mut storage).await?; let pending_miniblock_header = self .l1_batch_params_provider - .load_first_miniblock_in_batch(&mut storage, self.current_l1_batch_number) + .load_first_miniblock_in_batch(&mut storage, cursor.l1_batch) .await .with_context(|| { format!( "failed loading first miniblock for L1 batch #{}", - self.current_l1_batch_number + cursor.l1_batch ) })?; let Some(pending_miniblock_header) = pending_miniblock_header else { - return Ok(None); + return Ok((cursor, None)); }; let (system_env, l1_batch_env) = self @@ -116,18 +103,13 @@ impl StateKeeperIO for MempoolIO { self.chain_id, ) .await - .with_context(|| { - format!( - "failed loading params for L1 batch #{}", - self.current_l1_batch_number - ) - })?; + .with_context(|| format!("failed loading params for L1 batch #{}", cursor.l1_batch))?; let pending_batch_data = load_pending_batch(&mut storage, system_env, l1_batch_env) .await .with_context(|| { format!( "failed loading data for re-execution for pending L1 batch #{}", - self.current_l1_batch_number + cursor.l1_batch ) })?; @@ -146,17 +128,21 @@ impl StateKeeperIO for MempoolIO { gas_per_pubdata: gas_per_pubdata as u32, }; - Ok(Some(PendingBatchData { - l1_batch_env, - system_env, - pending_miniblocks, - })) + Ok(( + cursor, + Some(PendingBatchData { + l1_batch_env, + system_env, + pending_miniblocks, + }), + )) } async fn wait_for_new_batch_params( &mut self, + cursor: &IoCursor, max_wait: Duration, - ) -> anyhow::Result> { + ) -> anyhow::Result> { let deadline = Instant::now() + max_wait; // Block until at least one transaction in the mempool can match the filter (or timeout happens). @@ -165,25 +151,25 @@ impl StateKeeperIO for MempoolIO { // We cannot create two L1 batches or miniblocks with the same timestamp (forbidden by the bootloader). // Hence, we wait until the current timestamp is larger than the timestamp of the previous miniblock. // We can use `timeout_at` since `sleep_past` is cancel-safe; it only uses `sleep()` async calls. - let current_timestamp = tokio::time::timeout_at( + let timestamp = tokio::time::timeout_at( deadline.into(), - sleep_past(self.prev_miniblock_timestamp, self.current_miniblock_number), + sleep_past(cursor.prev_miniblock_timestamp, cursor.next_miniblock), ); - let Some(current_timestamp) = current_timestamp.await.ok() else { + let Some(timestamp) = timestamp.await.ok() else { return Ok(None); }; tracing::trace!( "Fee input for L1 batch #{} is {:#?}", - self.current_l1_batch_number.0, + cursor.l1_batch, self.filter.fee_input ); let mut storage = self.pool.connection_tagged("state_keeper").await?; - let (base_system_contracts, protocol_version) = storage + let protocol_version = storage .protocol_versions_dal() - .base_system_contracts_by_timestamp(current_timestamp) + .protocol_version_id_by_timestamp(timestamp) .await - .context("Failed loading base system contracts")?; + .context("Failed loading protocol version")?; drop(storage); // We create a new filter each time, since parameters may change and a previously @@ -198,69 +184,80 @@ impl StateKeeperIO for MempoolIO { continue; } - // We only need to get the root hash when we're certain that we have a new transaction. - let prev_l1_batch_hash = self.wait_for_previous_l1_batch_hash().await?; - return Ok(Some(l1_batch_params( - self.current_l1_batch_number, - self.fee_account, - current_timestamp, - prev_l1_batch_hash, - self.filter.fee_input, - self.current_miniblock_number, - self.prev_miniblock_hash, - base_system_contracts, - self.validation_computational_gas_limit, + return Ok(Some(L1BatchParams { protocol_version, - self.get_virtual_blocks_count(true, self.current_miniblock_number.0), - self.chain_id, - ))); + validation_computational_gas_limit: self.validation_computational_gas_limit, + operator_address: self.fee_account, + fee_input: self.filter.fee_input, + first_miniblock: MiniblockParams { + timestamp, + virtual_blocks: self.get_virtual_blocks_count(true, cursor.next_miniblock), + }, + })); } Ok(None) } - // Returns the pair of timestamp and the number of virtual blocks to be produced in this miniblock async fn wait_for_new_miniblock_params( &mut self, + cursor: &IoCursor, max_wait: Duration, ) -> anyhow::Result> { // We must provide different timestamps for each miniblock. // If miniblock sealing interval is greater than 1 second then `sleep_past` won't actually sleep. let timeout_result = tokio::time::timeout( max_wait, - sleep_past(self.prev_miniblock_timestamp, self.current_miniblock_number), + sleep_past(cursor.prev_miniblock_timestamp, cursor.next_miniblock), ) .await; let Ok(timestamp) = timeout_result else { return Ok(None); }; - let virtual_blocks = self.get_virtual_blocks_count(false, self.current_miniblock_number.0); + let virtual_blocks = self.get_virtual_blocks_count(false, cursor.next_miniblock); Ok(Some(MiniblockParams { timestamp, virtual_blocks, })) } - async fn wait_for_next_tx(&mut self, max_wait: Duration) -> Option { - for _ in 0..poll_iters(self.delay_interval, max_wait) { + async fn wait_for_next_tx( + &mut self, + max_wait: Duration, + ) -> anyhow::Result> { + let started_at = Instant::now(); + while started_at.elapsed() <= max_wait { let get_latency = KEEPER_METRICS.get_tx_from_mempool.start(); - let res = self.mempool.next_transaction(&self.filter); + let maybe_tx = self.mempool.next_transaction(&self.filter); get_latency.observe(); - if let Some(res) = res { - return Some(res); + + if let Some(tx) = maybe_tx { + // Reject transactions with too big gas limit. They are also rejected on the API level, but + // we need to secure ourselves in case some tx will somehow get into mempool. + if tx.gas_limit() > self.max_allowed_tx_gas_limit { + tracing::warn!( + "Found tx with too big gas limit in state keeper, hash: {:?}, gas_limit: {}", + tx.hash(), + tx.gas_limit() + ); + self.reject(&tx, &Halt::TooBigGasLimit.to_string()).await?; + continue; + } + return Ok(Some(tx)); } else { tokio::time::sleep(self.delay_interval).await; continue; } } - None + Ok(None) } - async fn rollback(&mut self, tx: Transaction) { + async fn rollback(&mut self, tx: Transaction) -> anyhow::Result<()> { // Reset nonces in the mempool. self.mempool.rollback(&tx); // Insert the transaction back. self.mempool.insert(vec![tx], HashMap::new()); + Ok(()) } async fn reject(&mut self, rejected: &Transaction, error: &str) -> anyhow::Result<()> { @@ -286,78 +283,35 @@ impl StateKeeperIO for MempoolIO { Ok(()) } - async fn seal_miniblock(&mut self, updates_manager: &UpdatesManager) { - let command = updates_manager.seal_miniblock_command( - self.current_l1_batch_number, - self.current_miniblock_number, - self.l2_erc20_bridge_addr, - false, - ); - self.miniblock_sealer_handle.submit(command).await; - self.update_miniblock_fields(&updates_manager.miniblock); - } - - async fn seal_l1_batch( + async fn load_base_system_contracts( &mut self, - witness_block_state: Option, - updates_manager: UpdatesManager, - l1_batch_env: &L1BatchEnv, - finished_batch: FinishedL1Batch, - ) -> anyhow::Result<()> { - assert_eq!( - updates_manager.batch_timestamp(), - l1_batch_env.timestamp, - "Batch timestamps don't match, batch number {}", - self.current_l1_batch_number() - ); - - // We cannot start sealing an L1 batch until we've sealed all miniblocks included in it. - self.miniblock_sealer_handle.wait_for_all_commands().await; - - if let Some(witness_witness_block_state) = witness_block_state { - match self - .object_store - .put(self.current_l1_batch_number(), &witness_witness_block_state) - .await - { - Ok(path) => { - tracing::debug!("Successfully uploaded witness block start state to Object Store to path = '{path}'"); - } - Err(e) => { - tracing::error!( - "Failed to upload witness block start state to Object Store: {e:?}" - ); - } - } - } - - let pool = self.pool.clone(); - let mut storage = pool.connection_tagged("state_keeper").await?; - - let fictive_miniblock = updates_manager - .seal_l1_batch( - &mut storage, - self.current_miniblock_number, - l1_batch_env, - finished_batch, - self.l2_erc20_bridge_addr, - ) - .await; - self.update_miniblock_fields(&fictive_miniblock); - self.current_l1_batch_number += 1; - Ok(()) + protocol_version: ProtocolVersionId, + _cursor: &IoCursor, + ) -> anyhow::Result { + self.pool + .connection_tagged("state_keeper") + .await? + .protocol_versions_dal() + .load_base_system_contracts_by_version_id(protocol_version as u16) + .await + .context("failed loading base system contracts")? + .with_context(|| { + format!( + "no base system contracts persisted for protocol version {protocol_version:?}" + ) + }) } - async fn load_previous_batch_version_id(&mut self) -> anyhow::Result { + async fn load_batch_version_id( + &mut self, + number: L1BatchNumber, + ) -> anyhow::Result { let mut storage = self.pool.connection_tagged("state_keeper").await?; - let prev_l1_batch_number = self.current_l1_batch_number - 1; self.l1_batch_params_provider - .load_l1_batch_protocol_version(&mut storage, prev_l1_batch_number) + .load_l1_batch_protocol_version(&mut storage, number) .await - .with_context(|| { - format!("failed loading protocol version for L1 batch #{prev_l1_batch_number}") - })? - .with_context(|| format!("L1 batch #{prev_l1_batch_number} misses protocol version")) + .with_context(|| format!("failed loading protocol version for L1 batch #{number}"))? + .with_context(|| format!("L1 batch #{number} misses protocol version")) } async fn load_upgrade_tx( @@ -370,6 +324,27 @@ impl StateKeeperIO for MempoolIO { .get_protocol_upgrade_tx(version_id) .await) } + + async fn load_batch_state_hash( + &mut self, + l1_batch_number: L1BatchNumber, + ) -> anyhow::Result { + tracing::trace!("Getting L1 batch hash for L1 batch #{l1_batch_number}"); + let wait_latency = KEEPER_METRICS.wait_for_prev_hash_time.start(); + + let mut storage = self.pool.connection_tagged("state_keeper").await?; + let (batch_state_hash, _) = self + .l1_batch_params_provider + .wait_for_l1_batch_params(&mut storage, l1_batch_number) + .await + .with_context(|| format!("error waiting for params for L1 batch #{l1_batch_number}"))?; + + wait_latency.observe(); + tracing::trace!( + "Got L1 batch state hash: {batch_state_hash:?} for L1 batch #{l1_batch_number}" + ); + Ok(batch_state_hash) + } } /// Sleeps until the current timestamp is larger than the provided `timestamp`. @@ -420,17 +395,12 @@ async fn sleep_past(timestamp: u64, miniblock: MiniblockNumber) -> u64 { } impl MempoolIO { - #[allow(clippy::too_many_arguments)] pub async fn new( mempool: MempoolGuard, - object_store: Arc, - miniblock_sealer_handle: MiniblockSealerHandle, batch_fee_input_provider: Arc, pool: ConnectionPool, config: &StateKeeperConfig, delay_interval: Duration, - l2_erc20_bridge_addr: Address, - validation_computational_gas_limit: u32, chain_id: L2ChainId, ) -> anyhow::Result { anyhow::ensure!( @@ -443,9 +413,6 @@ impl MempoolIO { ); let mut storage = pool.connection_tagged("state_keeper").await?; - let cursor = IoCursor::new(&mut storage) - .await - .context("failed initializing I/O cursor")?; let l1_batch_params_provider = L1BatchParamsProvider::new(&mut storage) .await .context("failed initializing L1 batch params provider")?; @@ -454,71 +421,34 @@ impl MempoolIO { Ok(Self { mempool, - object_store, pool, timeout_sealer: TimeoutSealer::new(config), filter: L2TxFilter::default(), // ^ Will be initialized properly on the first newly opened batch - current_l1_batch_number: cursor.l1_batch, - miniblock_sealer_handle, - current_miniblock_number: cursor.next_miniblock, - prev_miniblock_hash: cursor.prev_miniblock_hash, - prev_miniblock_timestamp: cursor.prev_miniblock_timestamp, l1_batch_params_provider, fee_account: config.fee_account_addr, - validation_computational_gas_limit, + validation_computational_gas_limit: config.validation_computational_gas_limit, + max_allowed_tx_gas_limit: config.max_allowed_l2_tx_gas_limit.into(), delay_interval, batch_fee_input_provider, - l2_erc20_bridge_addr, chain_id, virtual_blocks_interval: config.virtual_blocks_interval, virtual_blocks_per_miniblock: config.virtual_blocks_per_miniblock, }) } - fn update_miniblock_fields(&mut self, miniblock: &MiniblockUpdates) { - assert_eq!( - miniblock.number, self.current_miniblock_number.0, - "Attempted to seal a miniblock with unexpected number" - ); - self.current_miniblock_number += 1; - self.prev_miniblock_hash = miniblock.get_miniblock_hash(); - self.prev_miniblock_timestamp = miniblock.timestamp; - } - - async fn wait_for_previous_l1_batch_hash(&self) -> anyhow::Result { - tracing::trace!( - "Getting previous L1 batch hash for L1 batch #{}", - self.current_l1_batch_number - ); - let wait_latency = KEEPER_METRICS.wait_for_prev_hash_time.start(); - - let mut storage = self.pool.connection_tagged("state_keeper").await?; - let prev_l1_batch_number = self.current_l1_batch_number - 1; - let (batch_hash, _) = self - .l1_batch_params_provider - .wait_for_l1_batch_params(&mut storage, prev_l1_batch_number) - .await - .with_context(|| { - format!("error waiting for params for L1 batch #{prev_l1_batch_number}") - })?; - - wait_latency.observe(); - tracing::trace!( - "Got previous L1 batch hash: {batch_hash:?} for L1 batch #{}", - self.current_l1_batch_number - ); - Ok(batch_hash) - } - /// "virtual_blocks_per_miniblock" will be created either if the miniblock_number % virtual_blocks_interval == 0 or /// the miniblock is the first one in the batch. /// For instance: /// 1) If we want to have virtual block speed the same as the batch speed, virtual_block_interval = 10^9 and virtual_blocks_per_miniblock = 1 /// 2) If we want to have roughly 1 virtual block per 2 miniblocks, we need to have virtual_block_interval = 2, and virtual_blocks_per_miniblock = 1 /// 3) If we want to have 4 virtual blocks per miniblock, we need to have virtual_block_interval = 1, and virtual_blocks_per_miniblock = 4. - fn get_virtual_blocks_count(&self, first_in_batch: bool, miniblock_number: u32) -> u32 { - if first_in_batch || miniblock_number % self.virtual_blocks_interval == 0 { + fn get_virtual_blocks_count( + &self, + first_in_batch: bool, + miniblock_number: MiniblockNumber, + ) -> u32 { + if first_in_batch || miniblock_number.0 % self.virtual_blocks_interval == 0 { return self.virtual_blocks_per_miniblock; } 0 @@ -531,10 +461,6 @@ impl MempoolIO { pub(super) fn filter(&self) -> &L2TxFilter { &self.filter } - - pub(super) fn set_prev_miniblock_timestamp(&mut self, timestamp: u64) { - self.prev_miniblock_timestamp = timestamp; - } } #[cfg(test)] diff --git a/core/lib/zksync_core/src/state_keeper/io/mod.rs b/core/lib/zksync_core/src/state_keeper/io/mod.rs index 747b576029e..d52f301868d 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/mod.rs @@ -1,27 +1,27 @@ -use std::{ - fmt, - time::{Duration, Instant}, -}; +use std::{fmt, time::Duration}; +use anyhow::Context as _; use async_trait::async_trait; -use multivm::interface::{FinishedL1Batch, L1BatchEnv, SystemEnv}; -use tokio::sync::{mpsc, oneshot}; -use zksync_dal::{ConnectionPool, Core}; +use multivm::interface::{L1BatchEnv, SystemEnv}; +use vm_utils::storage::l1_batch_params; +use zksync_contracts::BaseSystemContracts; use zksync_types::{ - block::MiniblockExecutionData, protocol_upgrade::ProtocolUpgradeTx, - witness_block_state::WitnessBlockState, L1BatchNumber, MiniblockNumber, ProtocolVersionId, - Transaction, + block::MiniblockExecutionData, fee_model::BatchFeeInput, protocol_upgrade::ProtocolUpgradeTx, + Address, L1BatchNumber, L2ChainId, ProtocolVersionId, Transaction, H256, }; -use super::{ - metrics::{MiniblockQueueStage, MINIBLOCK_METRICS}, - seal_criteria::IoSealCriteria, - updates::{MiniblockSealCommand, UpdatesManager}, +pub use self::{ + common::IoCursor, + output_handler::{OutputHandler, StateKeeperOutputHandler}, + persistence::{MiniblockSealerTask, StateKeeperPersistence}, }; +use super::seal_criteria::IoSealCriteria; pub(crate) mod common; pub(crate) mod fee_address_migration; pub(crate) mod mempool; +mod output_handler; +mod persistence; pub(crate) mod seal_logic; #[cfg(test)] mod tests; @@ -59,245 +59,132 @@ pub struct MiniblockParams { pub(crate) virtual_blocks: u32, } -/// `StateKeeperIO` provides the interactive layer for the state keeper: -/// it's used to receive volatile parameters (such as batch parameters), and also it's used to perform -/// mutable operations on the persistent state (e.g. persist executed batches). +/// Parameters for a new L1 batch returned by [`StateKeeperIO::wait_for_new_batch_params()`]. +#[derive(Debug, Clone)] +pub struct L1BatchParams { + /// Protocol version for the new L1 batch. + pub(crate) protocol_version: ProtocolVersionId, + /// Computational gas limit for the new L1 batch. + pub(crate) validation_computational_gas_limit: u32, + /// Operator address (aka fee address) for the new L1 batch. + pub(crate) operator_address: Address, + /// Fee parameters to be used in the new L1 batch. + pub(crate) fee_input: BatchFeeInput, + /// Parameters of the first miniblock in the batch. + pub(crate) first_miniblock: MiniblockParams, +} + +impl L1BatchParams { + pub(crate) fn into_env( + self, + chain_id: L2ChainId, + contracts: BaseSystemContracts, + cursor: &IoCursor, + previous_batch_hash: H256, + ) -> (SystemEnv, L1BatchEnv) { + l1_batch_params( + cursor.l1_batch, + self.operator_address, + self.first_miniblock.timestamp, + previous_batch_hash, + self.fee_input, + cursor.next_miniblock, + cursor.prev_miniblock_hash, + contracts, + self.validation_computational_gas_limit, + self.protocol_version, + self.first_miniblock.virtual_blocks, + chain_id, + ) + } +} + +/// Provides the interactive layer for the state keeper: +/// it's used to receive volatile parameters (such as batch parameters) and sequence transactions +/// providing miniblock and L1 batch boundaries for them. /// /// All errors returned from this method are treated as unrecoverable. #[async_trait] -pub trait StateKeeperIO: 'static + Send + IoSealCriteria { - /// Returns the number of the currently processed L1 batch. - fn current_l1_batch_number(&self) -> L1BatchNumber; - /// Returns the number of the currently processed miniblock (aka L2 block). - fn current_miniblock_number(&self) -> MiniblockNumber; +pub trait StateKeeperIO: 'static + Send + fmt::Debug + IoSealCriteria { + /// Returns the ID of the L2 chain. This ID is supposed to be static. + fn chain_id(&self) -> L2ChainId; /// Returns the data on the batch that was not sealed before the server restart. /// See `PendingBatchData` doc-comment for details. - async fn load_pending_batch(&mut self) -> anyhow::Result>; + async fn initialize(&mut self) -> anyhow::Result<(IoCursor, Option)>; /// Blocks for up to `max_wait` until the parameters for the next L1 batch are available. /// Returns the data required to initialize the VM for the next batch. async fn wait_for_new_batch_params( &mut self, + cursor: &IoCursor, max_wait: Duration, - ) -> anyhow::Result>; + ) -> anyhow::Result>; /// Blocks for up to `max_wait` until the parameters for the next miniblock are available. async fn wait_for_new_miniblock_params( &mut self, + cursor: &IoCursor, max_wait: Duration, ) -> anyhow::Result>; /// Blocks for up to `max_wait` until the next transaction is available for execution. /// Returns `None` if no transaction became available until the timeout. - async fn wait_for_next_tx(&mut self, max_wait: Duration) -> Option; + async fn wait_for_next_tx(&mut self, max_wait: Duration) + -> anyhow::Result>; /// Marks the transaction as "not executed", so it can be retrieved from the IO again. - async fn rollback(&mut self, tx: Transaction); + async fn rollback(&mut self, tx: Transaction) -> anyhow::Result<()>; /// Marks the transaction as "rejected", e.g. one that is not correct and can't be executed. async fn reject(&mut self, tx: &Transaction, error: &str) -> anyhow::Result<()>; - /// Marks the miniblock (aka L2 block) as sealed. Returns the timestamp for the next miniblock. - async fn seal_miniblock(&mut self, updates_manager: &UpdatesManager); - /// Marks the L1 batch as sealed. - async fn seal_l1_batch( + /// Loads base system contracts with the specified version. + async fn load_base_system_contracts( &mut self, - witness_block_state: Option, - updates_manager: UpdatesManager, - l1_batch_env: &L1BatchEnv, - finished_batch: FinishedL1Batch, - ) -> anyhow::Result<()>; - - /// Loads protocol version of the previous l1 batch. - async fn load_previous_batch_version_id(&mut self) -> anyhow::Result; + protocol_version: ProtocolVersionId, + cursor: &IoCursor, + ) -> anyhow::Result; + /// Loads protocol version of the specified L1 batch, which is guaranteed to exist in the storage. + async fn load_batch_version_id( + &mut self, + number: L1BatchNumber, + ) -> anyhow::Result; /// Loads protocol upgrade tx for given version. async fn load_upgrade_tx( &mut self, version_id: ProtocolVersionId, ) -> anyhow::Result>; + /// Loads state hash for the L1 batch with the specified number. The batch is guaranteed to be present + /// in the storage. + async fn load_batch_state_hash(&mut self, number: L1BatchNumber) -> anyhow::Result; } -impl fmt::Debug for dyn StateKeeperIO { - fn fmt(&self, formatter: &mut fmt::Formatter<'_>) -> fmt::Result { - formatter - .debug_struct("StateKeeperIO") - .field("current_l1_batch_number", &self.current_l1_batch_number()) - .field("current_miniblock_number", &self.current_miniblock_number()) - .finish() - } -} - -/// A command together with the return address allowing to track command processing completion. -#[derive(Debug)] -struct Completable { - command: T, - completion_sender: oneshot::Sender<()>, -} - -/// Handle for [`MiniblockSealer`] allowing to submit [`MiniblockSealCommand`]s. -#[derive(Debug)] -pub struct MiniblockSealerHandle { - commands_sender: mpsc::Sender>, - latest_completion_receiver: Option>, - // If true, `submit()` will wait for the operation to complete. - is_sync: bool, -} - -impl MiniblockSealerHandle { - const SHUTDOWN_MSG: &'static str = "miniblock sealer unexpectedly shut down"; - - /// Submits a new sealing `command` to the sealer that this handle is attached to. - /// - /// If there are currently too many unprocessed commands, this method will wait until - /// enough of them are processed (i.e., there is back pressure). - pub(crate) async fn submit(&mut self, command: MiniblockSealCommand) { - let miniblock_number = command.miniblock_number; - tracing::debug!( - "Enqueuing sealing command for miniblock #{miniblock_number} with #{} txs (L1 batch #{})", - command.miniblock.executed_transactions.len(), - command.l1_batch_number - ); - - let start = Instant::now(); - let (completion_sender, completion_receiver) = oneshot::channel(); - self.latest_completion_receiver = Some(completion_receiver); - let command = Completable { - command, - completion_sender, +impl dyn StateKeeperIO { + pub(super) async fn wait_for_new_batch_env( + &mut self, + cursor: &IoCursor, + max_wait: Duration, + ) -> anyhow::Result> { + let Some(params) = self.wait_for_new_batch_params(cursor, max_wait).await? else { + return Ok(None); }; - self.commands_sender - .send(command) + let contracts = self + .load_base_system_contracts(params.protocol_version, cursor) .await - .expect(Self::SHUTDOWN_MSG); - - let elapsed = start.elapsed(); - let queue_capacity = self.commands_sender.capacity(); - tracing::debug!( - "Enqueued sealing command for miniblock #{miniblock_number} (took {elapsed:?}; \ - available queue capacity: {queue_capacity})" - ); - - if self.is_sync { - self.wait_for_all_commands().await; - } else { - MINIBLOCK_METRICS.seal_queue_capacity.set(queue_capacity); - MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::Submit].observe(elapsed); - } - } - - /// Waits until all previously submitted commands are fully processed by the sealer. - pub async fn wait_for_all_commands(&mut self) { - tracing::debug!( - "Requested waiting for miniblock seal queue to empty; current available capacity: {}", - self.commands_sender.capacity() - ); - - let start = Instant::now(); - let completion_receiver = self.latest_completion_receiver.take(); - if let Some(completion_receiver) = completion_receiver { - completion_receiver.await.expect(Self::SHUTDOWN_MSG); - } - - let elapsed = start.elapsed(); - tracing::debug!("Miniblock seal queue is emptied (took {elapsed:?})"); - - // Since this method called from outside is essentially a no-op if `self.is_sync`, - // we don't report its metrics in this case. - if !self.is_sync { - MINIBLOCK_METRICS - .seal_queue_capacity - .set(self.commands_sender.capacity()); - MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::WaitForAllCommands] - .observe(elapsed); - } - } -} - -/// Component responsible for sealing miniblocks (i.e., storing their data to Postgres). -#[derive(Debug)] -pub struct MiniblockSealer { - pool: ConnectionPool, - is_sync: bool, - // Weak sender handle to get queue capacity stats. - commands_sender: mpsc::WeakSender>, - commands_receiver: mpsc::Receiver>, -} - -impl MiniblockSealer { - /// Creates a sealer that will use the provided Postgres connection and will have the specified - /// `command_capacity` for unprocessed sealing commands. - pub fn new( - pool: ConnectionPool, - mut command_capacity: usize, - ) -> (Self, MiniblockSealerHandle) { - let is_sync = command_capacity == 0; - command_capacity = command_capacity.max(1); - - let (commands_sender, commands_receiver) = mpsc::channel(command_capacity); - let this = Self { - pool, - is_sync, - commands_sender: commands_sender.downgrade(), - commands_receiver, - }; - let handle = MiniblockSealerHandle { - commands_sender, - latest_completion_receiver: None, - is_sync, - }; - (this, handle) - } - - /// Seals miniblocks as they are received from the [`MiniblockSealerHandle`]. This should be run - /// on a separate Tokio task. - pub async fn run(mut self) -> anyhow::Result<()> { - if self.is_sync { - tracing::info!("Starting synchronous miniblock sealer"); - } else if let Some(sender) = self.commands_sender.upgrade() { - tracing::info!( - "Starting async miniblock sealer with queue capacity {}", - sender.max_capacity() - ); - } else { - tracing::warn!("Miniblock sealer not started, since its handle is already dropped"); - } - - let mut miniblock_seal_delta: Option = None; - // Commands must be processed sequentially: a later miniblock cannot be saved before - // an earlier one. - while let Some(completable) = self.next_command().await { - let mut conn = self.pool.connection_tagged("state_keeper").await.unwrap(); - completable.command.seal(&mut conn).await; - if let Some(delta) = miniblock_seal_delta { - MINIBLOCK_METRICS.seal_delta.observe(delta.elapsed()); - } - miniblock_seal_delta = Some(Instant::now()); - - completable.completion_sender.send(()).ok(); - // ^ We don't care whether anyone listens to the processing progress - } - Ok(()) - } - - async fn next_command(&mut self) -> Option> { - tracing::debug!("Polling miniblock seal queue for next command"); - let start = Instant::now(); - let command = self.commands_receiver.recv().await; - let elapsed = start.elapsed(); - - if let Some(completable) = &command { - tracing::debug!( - "Received command to seal miniblock #{} (polling took {elapsed:?})", - completable.command.miniblock_number - ); - } - - if !self.is_sync { - MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::NextCommand] - .observe(elapsed); - if let Some(sender) = self.commands_sender.upgrade() { - MINIBLOCK_METRICS.seal_queue_capacity.set(sender.capacity()); - } - } - command + .with_context(|| { + format!( + "failed loading system contracts for protocol version {:?}", + params.protocol_version + ) + })?; + let previous_batch_hash = self + .load_batch_state_hash(cursor.l1_batch - 1) + .await + .context("cannot load state hash for previous L1 batch")?; + Ok(Some(params.into_env( + self.chain_id(), + contracts, + cursor, + previous_batch_hash, + ))) } } diff --git a/core/lib/zksync_core/src/state_keeper/io/output_handler.rs b/core/lib/zksync_core/src/state_keeper/io/output_handler.rs new file mode 100644 index 00000000000..671b4b49856 --- /dev/null +++ b/core/lib/zksync_core/src/state_keeper/io/output_handler.rs @@ -0,0 +1,105 @@ +//! Handling outputs produced by the state keeper. + +use std::fmt; + +use anyhow::Context as _; +use async_trait::async_trait; +use zksync_types::witness_block_state::WitnessBlockState; + +use crate::state_keeper::{io::IoCursor, updates::UpdatesManager}; + +/// Handler for state keeper outputs (miniblocks and L1 batches). +#[async_trait] +pub trait StateKeeperOutputHandler: 'static + Send + fmt::Debug { + /// Initializes this handler. This method will be called on state keeper initialization before any other calls. + /// The default implementation does nothing. + async fn initialize(&mut self, _cursor: &IoCursor) -> anyhow::Result<()> { + Ok(()) + } + + /// Handles a miniblock (aka L2 block) produced by the state keeper. + async fn handle_miniblock(&mut self, updates_manager: &UpdatesManager) -> anyhow::Result<()>; + + /// Handles an L1 batch produced by the state keeper. + async fn handle_l1_batch( + &mut self, + _witness_block_state: Option<&WitnessBlockState>, + _updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + Ok(()) + } +} + +/// Compound output handler plugged into the state keeper. +/// +/// This handle aggregates one or more [`StateKeeperOutputHandler`]s executing their hooks +/// on each new miniblock / L1 batch produced by the state keeper. These are executed sequentially in the order +/// handlers were inserted into this `OutputHandler`. Errors from handlers are bubbled up to the state keeper level, +/// meaning that if a handler fails, the corresponding hook won't run for subsequent handlers. +#[derive(Debug)] +pub struct OutputHandler { + inner: Vec>, +} + +impl OutputHandler { + /// Creates an output handler consisting of a single handler. + pub fn new(main_handler: Box) -> Self { + Self { + inner: vec![main_handler], + } + } + + /// Adds a new handler. Its hooks will be executed after all handlers inserted previously. + #[must_use] + pub fn with_handler(mut self, handler: Box) -> Self { + self.inner.push(handler); + self + } + + pub(crate) async fn initialize(&mut self, cursor: &IoCursor) -> anyhow::Result<()> { + for handler in &mut self.inner { + handler + .initialize(cursor) + .await + .with_context(|| format!("failed initializing handler {handler:?}"))?; + } + Ok(()) + } + + pub(crate) async fn handle_miniblock( + &mut self, + updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + for handler in &mut self.inner { + handler + .handle_miniblock(updates_manager) + .await + .with_context(|| { + format!( + "failed handling miniblock {:?} on handler {handler:?}", + updates_manager.miniblock + ) + })?; + } + Ok(()) + } + + pub(crate) async fn handle_l1_batch( + &mut self, + witness_block_state: Option<&WitnessBlockState>, + updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + for handler in &mut self.inner { + handler + .handle_l1_batch(witness_block_state, updates_manager) + .await + .with_context(|| { + format!( + "failed handling L1 batch #{} on handler {handler:?}", + updates_manager.l1_batch.number + ) + })?; + } + Ok(()) + } +} diff --git a/core/lib/zksync_core/src/state_keeper/io/persistence.rs b/core/lib/zksync_core/src/state_keeper/io/persistence.rs new file mode 100644 index 00000000000..d864d22e786 --- /dev/null +++ b/core/lib/zksync_core/src/state_keeper/io/persistence.rs @@ -0,0 +1,435 @@ +//! State keeper persistence logic. + +use std::{sync::Arc, time::Instant}; + +use anyhow::Context as _; +use async_trait::async_trait; +use tokio::sync::{mpsc, oneshot}; +use zksync_dal::{ConnectionPool, Core}; +use zksync_object_store::ObjectStore; +use zksync_types::{witness_block_state::WitnessBlockState, Address}; + +use crate::{ + metrics::{BlockStage, APP_METRICS}, + state_keeper::{ + io::StateKeeperOutputHandler, + metrics::{MiniblockQueueStage, MINIBLOCK_METRICS}, + updates::{MiniblockSealCommand, UpdatesManager}, + }, +}; + +/// A command together with the return address allowing to track command processing completion. +#[derive(Debug)] +struct Completable { + command: T, + completion_sender: oneshot::Sender<()>, +} + +/// Canonical [`HandleStateKeeperOutput`] implementation that stores processed miniblocks and L1 batches to Postgres. +#[derive(Debug)] +pub struct StateKeeperPersistence { + pool: ConnectionPool, + object_store: Option>, // FIXME (PLA-857): remove from the state keeper + l2_erc20_bridge_addr: Address, + pre_insert_txs: bool, + commands_sender: mpsc::Sender>, + latest_completion_receiver: Option>, + // If true, `submit_miniblock()` will wait for the operation to complete. + is_sync: bool, +} + +impl StateKeeperPersistence { + const SHUTDOWN_MSG: &'static str = "miniblock sealer unexpectedly shut down"; + + /// Creates a sealer that will use the provided Postgres connection and will have the specified + /// `command_capacity` for unprocessed sealing commands. + pub fn new( + pool: ConnectionPool, + l2_erc20_bridge_addr: Address, + mut command_capacity: usize, + ) -> (Self, MiniblockSealerTask) { + let is_sync = command_capacity == 0; + command_capacity = command_capacity.max(1); + + let (commands_sender, commands_receiver) = mpsc::channel(command_capacity); + let sealer = MiniblockSealerTask { + pool: pool.clone(), + is_sync, + commands_sender: commands_sender.downgrade(), + commands_receiver, + }; + let this = Self { + pool, + object_store: None, + l2_erc20_bridge_addr, + pre_insert_txs: false, + commands_sender, + latest_completion_receiver: None, + is_sync, + }; + (this, sealer) + } + + pub fn with_tx_insertion(mut self) -> Self { + self.pre_insert_txs = true; + self + } + + pub fn with_object_store(mut self, object_store: Arc) -> Self { + self.object_store = Some(object_store); + self + } + + /// Submits a new sealing `command` to the sealer that this handle is attached to. + /// + /// If there are currently too many unprocessed commands, this method will wait until + /// enough of them are processed (i.e., there is back pressure). + async fn submit_miniblock(&mut self, command: MiniblockSealCommand) { + let miniblock_number = command.miniblock.number; + tracing::debug!( + "Enqueuing sealing command for miniblock #{miniblock_number} with #{} txs (L1 batch #{})", + command.miniblock.executed_transactions.len(), + command.l1_batch_number + ); + + let start = Instant::now(); + let (completion_sender, completion_receiver) = oneshot::channel(); + self.latest_completion_receiver = Some(completion_receiver); + let command = Completable { + command, + completion_sender, + }; + self.commands_sender + .send(command) + .await + .expect(Self::SHUTDOWN_MSG); + + let elapsed = start.elapsed(); + let queue_capacity = self.commands_sender.capacity(); + tracing::debug!( + "Enqueued sealing command for miniblock #{miniblock_number} (took {elapsed:?}; \ + available queue capacity: {queue_capacity})" + ); + + if self.is_sync { + self.wait_for_all_commands().await; + } else { + MINIBLOCK_METRICS.seal_queue_capacity.set(queue_capacity); + MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::Submit].observe(elapsed); + } + } + + /// Waits until all previously submitted commands are fully processed by the sealer. + async fn wait_for_all_commands(&mut self) { + tracing::debug!( + "Requested waiting for miniblock seal queue to empty; current available capacity: {}", + self.commands_sender.capacity() + ); + + let start = Instant::now(); + let completion_receiver = self.latest_completion_receiver.take(); + if let Some(completion_receiver) = completion_receiver { + completion_receiver.await.expect(Self::SHUTDOWN_MSG); + } + + let elapsed = start.elapsed(); + tracing::debug!("Miniblock seal queue is emptied (took {elapsed:?})"); + + // Since this method called from outside is essentially a no-op if `self.is_sync`, + // we don't report its metrics in this case. + if !self.is_sync { + MINIBLOCK_METRICS + .seal_queue_capacity + .set(self.commands_sender.capacity()); + MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::WaitForAllCommands] + .observe(elapsed); + } + } +} + +#[async_trait] +impl StateKeeperOutputHandler for StateKeeperPersistence { + async fn handle_miniblock(&mut self, updates_manager: &UpdatesManager) -> anyhow::Result<()> { + let command = + updates_manager.seal_miniblock_command(self.l2_erc20_bridge_addr, self.pre_insert_txs); + self.submit_miniblock(command).await; + Ok(()) + } + + async fn handle_l1_batch( + &mut self, + witness_block_state: Option<&WitnessBlockState>, + updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + // We cannot start sealing an L1 batch until we've sealed all miniblocks included in it. + self.wait_for_all_commands().await; + + if let Some(witness_block_state) = witness_block_state { + let store = self + .object_store + .as_deref() + .context("object store not set when saving `WitnessBlockState`")?; + match store + .put(updates_manager.l1_batch.number, witness_block_state) + .await + { + Ok(path) => { + tracing::debug!("Successfully uploaded witness block start state to Object Store to path = '{path}'"); + } + Err(e) => { + tracing::error!( + "Failed to upload witness block start state to Object Store: {e:?}" + ); + } + } + } + + let pool = self.pool.clone(); + let mut storage = pool.connection_tagged("state_keeper").await?; + updates_manager + .seal_l1_batch(&mut storage, self.l2_erc20_bridge_addr) + .await; + APP_METRICS.block_number[&BlockStage::Sealed].set(updates_manager.l1_batch.number.0.into()); + Ok(()) + } +} + +/// Component responsible for sealing miniblocks (i.e., storing their data to Postgres). +#[derive(Debug)] +pub struct MiniblockSealerTask { + pool: ConnectionPool, + is_sync: bool, + // Weak sender handle to get queue capacity stats. + commands_sender: mpsc::WeakSender>, + commands_receiver: mpsc::Receiver>, +} + +impl MiniblockSealerTask { + /// Seals miniblocks as they are received from the [`StateKeeperPersistence`]. This should be run + /// on a separate Tokio task. + pub async fn run(mut self) -> anyhow::Result<()> { + if self.is_sync { + tracing::info!("Starting synchronous miniblock sealer"); + } else if let Some(sender) = self.commands_sender.upgrade() { + tracing::info!( + "Starting async miniblock sealer with queue capacity {}", + sender.max_capacity() + ); + } else { + tracing::warn!("Miniblock sealer not started, since its handle is already dropped"); + } + + let mut miniblock_seal_delta: Option = None; + // Commands must be processed sequentially: a later miniblock cannot be saved before + // an earlier one. + while let Some(completable) = self.next_command().await { + let mut storage = self.pool.connection_tagged("state_keeper").await?; + completable.command.seal(&mut storage).await; + if let Some(delta) = miniblock_seal_delta { + MINIBLOCK_METRICS.seal_delta.observe(delta.elapsed()); + } + miniblock_seal_delta = Some(Instant::now()); + + completable.completion_sender.send(()).ok(); + // ^ We don't care whether anyone listens to the processing progress + } + Ok(()) + } + + async fn next_command(&mut self) -> Option> { + tracing::debug!("Polling miniblock seal queue for next command"); + let start = Instant::now(); + let command = self.commands_receiver.recv().await; + let elapsed = start.elapsed(); + + if let Some(completable) = &command { + tracing::debug!( + "Received command to seal miniblock #{} (polling took {elapsed:?})", + completable.command.miniblock.number + ); + } + + if !self.is_sync { + MINIBLOCK_METRICS.seal_queue_latency[&MiniblockQueueStage::NextCommand] + .observe(elapsed); + if let Some(sender) = self.commands_sender.upgrade() { + MINIBLOCK_METRICS.seal_queue_capacity.set(sender.capacity()); + } + } + command + } +} + +#[cfg(test)] +mod tests { + use futures::FutureExt; + use multivm::zk_evm_latest::ethereum_types::H256; + use zksync_dal::CoreDal; + use zksync_types::{ + block::BlockGasCount, tx::ExecutionMetrics, L1BatchNumber, MiniblockNumber, + }; + + use super::*; + use crate::{ + genesis::{insert_genesis_batch, GenesisParams}, + state_keeper::{ + io::MiniblockParams, + tests::{ + create_execution_result, create_transaction, create_updates_manager, + default_l1_batch_env, default_system_env, default_vm_block_result, + }, + }, + }; + + async fn test_miniblock_and_l1_batch_processing( + pool: ConnectionPool, + miniblock_sealer_capacity: usize, + ) { + let mut storage = pool.connection().await.unwrap(); + insert_genesis_batch(&mut storage, &GenesisParams::mock()) + .await + .unwrap(); + // Save metadata for the genesis L1 batch so that we don't hang in `seal_l1_batch`. + storage + .blocks_dal() + .set_l1_batch_hash(L1BatchNumber(0), H256::zero()) + .await + .unwrap(); + drop(storage); + + let (mut persistence, miniblock_sealer) = StateKeeperPersistence::new( + pool.clone(), + Address::default(), + miniblock_sealer_capacity, + ); + tokio::spawn(miniblock_sealer.run()); + + let l1_batch_env = default_l1_batch_env(1, 1, Address::random()); + let mut updates = UpdatesManager::new(&l1_batch_env, &default_system_env()); + + let tx = create_transaction(10, 100); + updates.extend_from_executed_transaction( + tx, + create_execution_result(0, []), + vec![], + BlockGasCount::default(), + ExecutionMetrics::default(), + vec![], + ); + persistence.handle_miniblock(&updates).await.unwrap(); + updates.push_miniblock(MiniblockParams { + timestamp: 1, + virtual_blocks: 1, + }); + + updates.finish_batch(default_vm_block_result()); + persistence.handle_l1_batch(None, &updates).await.unwrap(); + + // Check that miniblock #1 and L1 batch #1 are persisted. + let mut storage = pool.connection().await.unwrap(); + assert_eq!( + storage + .blocks_dal() + .get_sealed_miniblock_number() + .await + .unwrap(), + Some(MiniblockNumber(2)) // + fictive miniblock + ); + let l1_batch_header = storage + .blocks_dal() + .get_l1_batch_header(L1BatchNumber(1)) + .await + .unwrap() + .expect("No L1 batch #1"); + assert_eq!(l1_batch_header.l2_tx_count, 1); + } + + #[tokio::test] + async fn miniblock_and_l1_batch_processing() { + let pool = ConnectionPool::constrained_test_pool(1).await; + test_miniblock_and_l1_batch_processing(pool, 1).await; + } + + #[tokio::test] + async fn miniblock_and_l1_batch_processing_with_sync_sealer() { + let pool = ConnectionPool::constrained_test_pool(1).await; + test_miniblock_and_l1_batch_processing(pool, 0).await; + } + + #[tokio::test] + async fn miniblock_sealer_handle_blocking() { + let pool = ConnectionPool::constrained_test_pool(1).await; + let (mut persistence, mut sealer) = + StateKeeperPersistence::new(pool, Address::default(), 1); + + // The first command should be successfully submitted immediately. + let mut updates_manager = create_updates_manager(); + let seal_command = updates_manager.seal_miniblock_command(Address::default(), false); + persistence.submit_miniblock(seal_command).await; + + // The second command should lead to blocking + updates_manager.push_miniblock(MiniblockParams { + timestamp: 2, + virtual_blocks: 1, + }); + let seal_command = updates_manager.seal_miniblock_command(Address::default(), false); + { + let submit_future = persistence.submit_miniblock(seal_command); + futures::pin_mut!(submit_future); + + assert!((&mut submit_future).now_or_never().is_none()); + // ...until miniblock #1 is processed + let command = sealer.commands_receiver.recv().await.unwrap(); + command.completion_sender.send(()).unwrap_err(); // completion receiver should be dropped + submit_future.await; + } + + { + let wait_future = persistence.wait_for_all_commands(); + futures::pin_mut!(wait_future); + assert!((&mut wait_future).now_or_never().is_none()); + let command = sealer.commands_receiver.recv().await.unwrap(); + command.completion_sender.send(()).unwrap(); + wait_future.await; + } + + // Check that `wait_for_all_commands()` state is reset after use. + persistence.wait_for_all_commands().await; + + updates_manager.push_miniblock(MiniblockParams { + timestamp: 3, + virtual_blocks: 1, + }); + let seal_command = updates_manager.seal_miniblock_command(Address::default(), false); + persistence.submit_miniblock(seal_command).await; + let command = sealer.commands_receiver.recv().await.unwrap(); + command.completion_sender.send(()).unwrap(); + persistence.wait_for_all_commands().await; + } + + #[tokio::test] + async fn miniblock_sealer_handle_parallel_processing() { + let pool = ConnectionPool::constrained_test_pool(1).await; + let (mut persistence, mut sealer) = + StateKeeperPersistence::new(pool, Address::default(), 5); + + // 5 miniblock sealing commands can be submitted without blocking. + let mut updates_manager = create_updates_manager(); + for i in 1..=5 { + let seal_command = updates_manager.seal_miniblock_command(Address::default(), false); + updates_manager.push_miniblock(MiniblockParams { + timestamp: i, + virtual_blocks: 1, + }); + persistence.submit_miniblock(seal_command).await; + } + + for i in 1..=5 { + let command = sealer.commands_receiver.recv().await.unwrap(); + assert_eq!(command.command.miniblock.number, MiniblockNumber(i)); + command.completion_sender.send(()).ok(); + } + + persistence.wait_for_all_commands().await; + } +} diff --git a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs index c874ba1c9b7..b21ca35228e 100644 --- a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs +++ b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs @@ -4,10 +4,7 @@ use std::time::{Duration, Instant}; use itertools::Itertools; -use multivm::{ - interface::{FinishedL1Batch, L1BatchEnv}, - utils::get_max_gas_per_pubdata_byte, -}; +use multivm::utils::get_max_gas_per_pubdata_byte; use zksync_dal::{Connection, Core, CoreDal}; use zksync_types::{ block::{unpack_block_info, L1BatchHeader, MiniblockHeader}, @@ -23,9 +20,9 @@ use zksync_types::{ TransactionExecutionResult, }, zk_evm_types::LogQuery, - AccountTreeId, Address, ExecuteTransactionCommon, L1BatchNumber, L1BlockNumber, - MiniblockNumber, ProtocolVersionId, StorageKey, StorageLog, StorageLogQuery, Transaction, - VmEvent, CURRENT_VIRTUAL_BLOCK_INFO_POSITION, H256, SYSTEM_CONTEXT_ADDRESS, + AccountTreeId, Address, ExecuteTransactionCommon, L1BlockNumber, ProtocolVersionId, StorageKey, + StorageLog, StorageLogQuery, Transaction, VmEvent, CURRENT_VIRTUAL_BLOCK_INFO_POSITION, H256, + SYSTEM_CONTEXT_ADDRESS, }; use zksync_utils::{h256_to_u256, u256_to_h256}; @@ -36,8 +33,7 @@ use crate::{ L1BatchSealStage, MiniblockSealStage, TxExecutionType, KEEPER_METRICS, L1_BATCH_METRICS, MINIBLOCK_METRICS, }, - types::ExecutionMetricsForCriteria, - updates::{MiniblockSealCommand, MiniblockUpdates, UpdatesManager}, + updates::{MiniblockSealCommand, UpdatesManager}, }, }; @@ -45,34 +41,22 @@ impl UpdatesManager { /// Persists an L1 batch in the storage. /// This action includes a creation of an empty "fictive" miniblock that contains /// the events generated during the bootloader "tip phase". Returns updates for this fictive miniblock. - #[must_use = "fictive miniblock must be used to update I/O params"] - pub(crate) async fn seal_l1_batch( - mut self, + pub(super) async fn seal_l1_batch( + &self, storage: &mut Connection<'_, Core>, - current_miniblock_number: MiniblockNumber, - l1_batch_env: &L1BatchEnv, - finished_batch: FinishedL1Batch, l2_erc20_bridge_addr: Address, - ) -> MiniblockUpdates { + ) { let started_at = Instant::now(); - let progress = L1_BATCH_METRICS.start(L1BatchSealStage::VmFinalization); + let finished_batch = self + .l1_batch + .finished + .as_ref() + .expect("L1 batch is not actually finished"); let mut transaction = storage.start_transaction().await.unwrap(); - progress.observe(None); let progress = L1_BATCH_METRICS.start(L1BatchSealStage::FictiveMiniblock); - let ExecutionMetricsForCriteria { - l1_gas: batch_tip_l1_gas, - execution_metrics: batch_tip_execution_metrics, - } = ExecutionMetricsForCriteria::new(None, &finished_batch.block_tip_execution_result); - self.extend_from_fictive_transaction( - finished_batch.block_tip_execution_result, - batch_tip_l1_gas, - batch_tip_execution_metrics, - ); // Seal fictive miniblock with last events and storage logs. let miniblock_command = self.seal_miniblock_command( - l1_batch_env.number, - current_miniblock_number, l2_erc20_bridge_addr, false, // fictive miniblocks don't have txs, so it's fine to pass `false` here. ); @@ -110,29 +94,35 @@ impl UpdatesManager { .user_l2_to_l1_logs .len(), event_count = finished_batch.final_execution_state.events.len(), - current_l1_batch_number = l1_batch_env.number + current_l1_batch_number = self.l1_batch.number ); let progress = L1_BATCH_METRICS.start(L1BatchSealStage::InsertL1BatchHeader); let l2_to_l1_messages = extract_long_l2_to_l1_messages(&finished_batch.final_execution_state.events); let l1_batch = L1BatchHeader { - number: l1_batch_env.number, - timestamp: l1_batch_env.timestamp, + number: self.l1_batch.number, + timestamp: self.batch_timestamp(), priority_ops_onchain_data: self.l1_batch.priority_ops_onchain_data.clone(), l1_tx_count: l1_tx_count as u16, l2_tx_count: l2_tx_count as u16, - l2_to_l1_logs: finished_batch.final_execution_state.user_l2_to_l1_logs, + l2_to_l1_logs: finished_batch + .final_execution_state + .user_l2_to_l1_logs + .clone(), l2_to_l1_messages, bloom: Default::default(), - used_contract_hashes: finished_batch.final_execution_state.used_contract_hashes, + used_contract_hashes: finished_batch + .final_execution_state + .used_contract_hashes + .clone(), base_system_contracts_hashes: self.base_system_contract_hashes(), protocol_version: Some(self.protocol_version()), - system_logs: finished_batch.final_execution_state.system_logs, - pubdata_input: finished_batch.pubdata_input, + system_logs: finished_batch.final_execution_state.system_logs.clone(), + pubdata_input: finished_batch.pubdata_input.clone(), }; - let events_queue = finished_batch + let events_queue = &finished_batch .final_execution_state .deduplicated_events_logs; @@ -146,7 +136,7 @@ impl UpdatesManager { &l1_batch, &final_bootloader_memory, self.pending_l1_gas_count(), - &events_queue, + events_queue, &finished_batch.final_execution_state.storage_refunds, self.pending_execution_metrics().circuit_statistic, ) @@ -157,7 +147,7 @@ impl UpdatesManager { let progress = L1_BATCH_METRICS.start(L1BatchSealStage::SetL1BatchNumberForMiniblocks); transaction .blocks_dal() - .mark_miniblocks_as_executed_in_l1_batch(l1_batch_env.number) + .mark_miniblocks_as_executed_in_l1_batch(self.l1_batch.number) .await .unwrap(); progress.observe(None); @@ -166,7 +156,7 @@ impl UpdatesManager { transaction .transactions_dal() .mark_txs_as_executed_in_l1_batch( - l1_batch_env.number, + self.l1_batch.number, &self.l1_batch.executed_transactions, ) .await; @@ -176,11 +166,11 @@ impl UpdatesManager { let (deduplicated_writes, protective_reads): (Vec<_>, Vec<_>) = finished_batch .final_execution_state .deduplicated_storage_log_queries - .into_iter() + .iter() .partition(|log_query| log_query.rw_flag); transaction .storage_logs_dedup_dal() - .insert_protective_reads(l1_batch_env.number, &protective_reads) + .insert_protective_reads(self.l1_batch.number, &protective_reads) .await .unwrap(); progress.observe(protective_reads.len()); @@ -213,7 +203,7 @@ impl UpdatesManager { transaction .storage_logs_dedup_dal() - .insert_initial_writes(l1_batch_env.number, &written_storage_keys) + .insert_initial_writes(self.l1_batch.number, &written_storage_keys) .await .unwrap(); progress.observe(deduplicated_writes.len()); @@ -230,20 +220,12 @@ impl UpdatesManager { "Results of in-flight and common deduplications are mismatched" ); - self.report_l1_batch_metrics( - started_at, - l1_batch_env.number, - l1_batch_env.timestamp, - &writes_metrics, - ); - miniblock_command.miniblock + self.report_l1_batch_metrics(started_at, &writes_metrics); } fn report_l1_batch_metrics( &self, started_at: Instant, - current_l1_batch_number: L1BatchNumber, - batch_timestamp: u64, writes_metrics: &DeduplicatedWritesMetrics, ) { L1_BATCH_METRICS @@ -256,6 +238,7 @@ impl UpdatesManager { .transactions_in_l1_batch .observe(self.l1_batch.executed_transactions.len()); + let batch_timestamp = self.batch_timestamp(); let l1_batch_latency = unix_timestamp_ms().saturating_sub(batch_timestamp * 1_000) as f64 / 1_000.0; APP_METRICS.block_latency[&BlockStage::Sealed] @@ -263,12 +246,12 @@ impl UpdatesManager { let elapsed = started_at.elapsed(); L1_BATCH_METRICS.sealed_time.observe(elapsed); - tracing::debug!("Sealed L1 batch {current_l1_batch_number} in {elapsed:?}"); + tracing::debug!("Sealed L1 batch {} in {elapsed:?}", self.l1_batch.number); } } impl MiniblockSealCommand { - pub async fn seal(&self, storage: &mut Connection<'_, Core>) { + pub(super) async fn seal(&self, storage: &mut Connection<'_, Core>) { self.seal_inner(storage, false).await; } @@ -327,7 +310,7 @@ impl MiniblockSealCommand { } let l1_batch_number = self.l1_batch_number; - let miniblock_number = self.miniblock_number; + let miniblock_number = self.miniblock.number; let started_at = Instant::now(); let progress = MINIBLOCK_METRICS.start(MiniblockSealStage::InsertMiniblockHeader, is_fictive); @@ -636,7 +619,7 @@ impl MiniblockSealCommand { } fn report_miniblock_metrics(&self, started_at: Instant, latest_virtual_block_number: u64) { - let miniblock_number = self.miniblock_number; + let miniblock_number = self.miniblock.number; MINIBLOCK_METRICS .transactions_in_miniblock diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index d8f0fc1c513..82a108bf2b5 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -1,6 +1,5 @@ use std::time::Duration; -use futures::FutureExt; use multivm::utils::derive_base_fee_and_gas_per_pubdata; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{ConnectionPool, Core, CoreDal}; @@ -10,21 +9,19 @@ use zksync_types::{ fee::TransactionExecutionMetrics, fee_model::{BatchFeeInput, PubdataIndependentBatchFeeModelInput}, tx::ExecutionMetrics, - AccountTreeId, Address, L1BatchNumber, MiniblockNumber, ProtocolVersionId, StorageKey, VmEvent, - H256, U256, + AccountTreeId, Address, L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, + StorageKey, VmEvent, H256, U256, }; use zksync_utils::time::seconds_since_epoch; use self::tester::Tester; use crate::{ state_keeper::{ - io::{MiniblockParams, MiniblockSealer, StateKeeperIO}, + io::StateKeeperIO, mempool_actor::l2_tx_filter, - tests::{ - create_execution_result, create_transaction, create_updates_manager, - default_l1_batch_env, default_system_env, default_vm_block_result, Query, - }, + tests::{create_execution_result, create_transaction, Query, BASE_SYSTEM_CONTRACTS}, updates::{MiniblockSealCommand, MiniblockUpdates, UpdatesManager}, + StateKeeperOutputHandler, StateKeeperPersistence, }, utils::testonly::prepare_recovery_snapshot, }; @@ -39,7 +36,7 @@ async fn test_filter_initialization() { // Genesis is needed for proper mempool initialization. tester.genesis(&connection_pool).await; - let (mempool, _) = tester.create_test_mempool_io(connection_pool, 1).await; + let (mempool, _) = tester.create_test_mempool_io(connection_pool).await; // Upon initialization, the filter should be set to the default values. assert_eq!(mempool.filter(), &L2TxFilter::default()); @@ -75,11 +72,11 @@ async fn test_filter_with_pending_batch() { .insert_miniblock(&connection_pool, 2, 10, fee_input) .await; - let (mut mempool, _) = tester.create_test_mempool_io(connection_pool, 1).await; + let (mut mempool, _) = tester.create_test_mempool_io(connection_pool).await; // Before the mempool knows there is a pending batch, the filter is still set to the default values. assert_eq!(mempool.filter(), &L2TxFilter::default()); - mempool.load_pending_batch().await.unwrap(); + mempool.initialize().await.unwrap(); let (want_base_fee, want_gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata(fee_input, ProtocolVersionId::latest().into()); let want_filter = L2TxFilter { @@ -114,7 +111,8 @@ async fn test_filter_with_no_pending_batch() { .await; // Create a mempool without pending batch and ensure that filter is not initialized just yet. - let (mut mempool, mut guard) = tester.create_test_mempool_io(connection_pool, 1).await; + let (mut mempool, mut guard) = tester.create_test_mempool_io(connection_pool).await; + let (io_cursor, _) = mempool.initialize().await.unwrap(); assert_eq!(mempool.filter(), &L2TxFilter::default()); // Insert a transaction that matches the expected filter. @@ -127,7 +125,7 @@ async fn test_filter_with_no_pending_batch() { // Now, given that there is a transaction matching the expected filter, waiting for the new batch params // should succeed and initialize the filter. mempool - .wait_for_new_batch_params(Duration::from_secs(10)) + .wait_for_new_batch_params(&io_cursor, Duration::from_secs(10)) .await .expect("No batch params in the test mempool"); assert_eq!(mempool.filter(), &want_filter); @@ -152,7 +150,8 @@ async fn test_timestamps_are_distinct( .insert_sealed_batch(&connection_pool, 1, &[tx_result]) .await; - let (mut mempool, mut guard) = tester.create_test_mempool_io(connection_pool, 1).await; + let (mut mempool, mut guard) = tester.create_test_mempool_io(connection_pool).await; + let (io_cursor, _) = mempool.initialize().await.unwrap(); // Insert a transaction to trigger L1 batch creation. let tx_filter = l2_tx_filter( &tester.create_batch_fee_input_provider().await, @@ -161,12 +160,12 @@ async fn test_timestamps_are_distinct( .await; tester.insert_tx(&mut guard, tx_filter.fee_per_gas, tx_filter.gas_per_pubdata); - let (_, l1_batch_env) = mempool - .wait_for_new_batch_params(Duration::from_secs(10)) + let l1_batch_params = mempool + .wait_for_new_batch_params(&io_cursor, Duration::from_secs(10)) .await .unwrap() .expect("No batch params in the test mempool"); - assert!(l1_batch_env.timestamp > prev_miniblock_timestamp); + assert!(l1_batch_params.first_miniblock.timestamp > prev_miniblock_timestamp); } #[tokio::test] @@ -200,7 +199,13 @@ async fn l1_batch_timestamp_respects_prev_miniblock_with_clock_skew() { #[tokio::test] async fn processing_storage_logs_when_sealing_miniblock() { let connection_pool = ConnectionPool::::constrained_test_pool(1).await; - let mut miniblock = MiniblockUpdates::new(0, 1, H256::zero(), 1, ProtocolVersionId::latest()); + let mut miniblock = MiniblockUpdates::new( + 0, + MiniblockNumber(3), + H256::zero(), + 1, + ProtocolVersionId::latest(), + ); let tx = create_transaction(10, 100); let storage_logs = [ @@ -246,7 +251,6 @@ async fn processing_storage_logs_when_sealing_miniblock() { let l1_batch_number = L1BatchNumber(2); let seal_command = MiniblockSealCommand { l1_batch_number, - miniblock_number: MiniblockNumber(3), miniblock, first_tx_index: 0, fee_account_address: Address::repeat_byte(0x23), @@ -298,7 +302,13 @@ async fn processing_storage_logs_when_sealing_miniblock() { async fn processing_events_when_sealing_miniblock() { let pool = ConnectionPool::::constrained_test_pool(1).await; let l1_batch_number = L1BatchNumber(2); - let mut miniblock = MiniblockUpdates::new(0, 1, H256::zero(), 1, ProtocolVersionId::latest()); + let mut miniblock = MiniblockUpdates::new( + 0, + MiniblockNumber(3), + H256::zero(), + 1, + ProtocolVersionId::latest(), + ); let events = (0_u8..10).map(|i| VmEvent { location: (l1_batch_number, u32::from(i / 4)), @@ -321,10 +331,8 @@ async fn processing_events_when_sealing_miniblock() { ); } - let miniblock_number = MiniblockNumber(3); let seal_command = MiniblockSealCommand { l1_batch_number, - miniblock_number, miniblock, first_tx_index: 0, fee_account_address: Address::repeat_byte(0x23), @@ -347,7 +355,7 @@ async fn processing_events_when_sealing_miniblock() { let logs = conn .events_web3_dal() - .get_all_logs(miniblock_number - 1) + .get_all_logs(seal_command.miniblock.number - 1) .await .unwrap(); @@ -358,81 +366,6 @@ async fn processing_events_when_sealing_miniblock() { } } -async fn test_miniblock_and_l1_batch_processing( - pool: ConnectionPool, - miniblock_sealer_capacity: usize, -) { - let tester = Tester::new(); - - // Genesis is needed for proper mempool initialization. - tester.genesis(&pool).await; - let mut storage = pool.connection().await.unwrap(); - // Save metadata for the genesis L1 batch so that we don't hang in `seal_l1_batch`. - storage - .blocks_dal() - .set_l1_batch_hash(L1BatchNumber(0), H256::zero()) - .await - .unwrap(); - drop(storage); - - let (mut mempool, _) = tester - .create_test_mempool_io(pool.clone(), miniblock_sealer_capacity) - .await; - - let l1_batch_env = default_l1_batch_env(1, 1, Address::random()); - let mut updates = UpdatesManager::new(&l1_batch_env, &default_system_env()); - - let tx = create_transaction(10, 100); - updates.extend_from_executed_transaction( - tx, - create_execution_result(0, []), - vec![], - BlockGasCount::default(), - ExecutionMetrics::default(), - vec![], - ); - mempool.seal_miniblock(&updates).await; - updates.push_miniblock(MiniblockParams { - timestamp: 1, - virtual_blocks: 1, - }); - - let finished_batch = default_vm_block_result(); - mempool - .seal_l1_batch(None, updates, &l1_batch_env, finished_batch) - .await - .unwrap(); - - // Check that miniblock #1 and L1 batch #1 are persisted. - let mut conn = pool.connection().await.unwrap(); - assert_eq!( - conn.blocks_dal() - .get_sealed_miniblock_number() - .await - .unwrap(), - Some(MiniblockNumber(2)) // + fictive miniblock - ); - let l1_batch_header = conn - .blocks_dal() - .get_l1_batch_header(L1BatchNumber(1)) - .await - .unwrap() - .expect("No L1 batch #1"); - assert_eq!(l1_batch_header.l2_tx_count, 1); -} - -#[tokio::test] -async fn miniblock_and_l1_batch_processing() { - let pool = ConnectionPool::::constrained_test_pool(1).await; - test_miniblock_and_l1_batch_processing(pool, 1).await; -} - -#[tokio::test] -async fn miniblock_and_l1_batch_processing_with_sync_sealer() { - let pool = ConnectionPool::::constrained_test_pool(1).await; - test_miniblock_and_l1_batch_processing(pool, 0).await; -} - #[tokio::test] async fn miniblock_processing_after_snapshot_recovery() { let connection_pool = ConnectionPool::::test_pool().await; @@ -441,18 +374,15 @@ async fn miniblock_processing_after_snapshot_recovery() { prepare_recovery_snapshot(&mut storage, L1BatchNumber(23), MiniblockNumber(42), &[]).await; let tester = Tester::new(); - let (mut mempool, mut mempool_guard) = tester - .create_test_mempool_io(connection_pool.clone(), 0) - .await; + let (mut mempool, mut mempool_guard) = + tester.create_test_mempool_io(connection_pool.clone()).await; + let (cursor, maybe_pending_batch) = mempool.initialize().await.unwrap(); assert_eq!( - mempool.current_miniblock_number(), + cursor.next_miniblock, snapshot_recovery.miniblock_number + 1 ); - assert_eq!( - mempool.current_l1_batch_number(), - snapshot_recovery.l1_batch_number + 1 - ); - assert!(mempool.load_pending_batch().await.unwrap().is_none()); + assert_eq!(cursor.l1_batch, snapshot_recovery.l1_batch_number + 1); + assert!(maybe_pending_batch.is_none()); // Insert a transaction into the mempool in order to open a new batch. let tx_filter = l2_tx_filter( @@ -471,21 +401,23 @@ async fn miniblock_processing_after_snapshot_recovery() { .await .unwrap(); - let (system_env, l1_batch_env) = mempool - .wait_for_new_batch_params(Duration::from_secs(10)) + let previous_batch_hash = mempool + .load_batch_state_hash(snapshot_recovery.l1_batch_number) + .await + .unwrap(); + assert_eq!(previous_batch_hash, snapshot_recovery.l1_batch_root_hash); + + let l1_batch_params = mempool + .wait_for_new_batch_params(&cursor, Duration::from_secs(10)) .await .unwrap() .expect("no batch params generated"); - assert_eq!(l1_batch_env.number, snapshot_recovery.l1_batch_number + 1); - assert_eq!( - l1_batch_env.previous_batch_hash, - Some(snapshot_recovery.l1_batch_root_hash) + let (system_env, l1_batch_env) = l1_batch_params.into_env( + L2ChainId::default(), + BASE_SYSTEM_CONTRACTS.clone(), + &cursor, + previous_batch_hash, ); - assert_eq!( - l1_batch_env.first_l2_block.prev_block_hash, - snapshot_recovery.miniblock_hash - ); - let mut updates = UpdatesManager::new(&l1_batch_env, &system_env); let tx_hash = tx.hash(); @@ -497,7 +429,11 @@ async fn miniblock_processing_after_snapshot_recovery() { ExecutionMetrics::default(), vec![], ); - mempool.seal_miniblock(&updates).await; + + let (mut persistence, miniblock_sealer) = + StateKeeperPersistence::new(connection_pool.clone(), Address::default(), 0); + tokio::spawn(miniblock_sealer.run()); + persistence.handle_miniblock(&updates).await.unwrap(); // Check that the miniblock is persisted and has correct data. let persisted_miniblock = storage @@ -532,23 +468,15 @@ async fn miniblock_processing_after_snapshot_recovery() { assert_eq!(miniblock_transactions[0].hash(), tx_hash); // Emulate node restart. - let (mut mempool, _) = tester - .create_test_mempool_io(connection_pool.clone(), 0) - .await; + let (mut mempool, _) = tester.create_test_mempool_io(connection_pool.clone()).await; + let (cursor, maybe_pending_batch) = mempool.initialize().await.unwrap(); assert_eq!( - mempool.current_miniblock_number(), + cursor.next_miniblock, snapshot_recovery.miniblock_number + 2 ); - assert_eq!( - mempool.current_l1_batch_number(), - snapshot_recovery.l1_batch_number + 1 - ); + assert_eq!(cursor.l1_batch, snapshot_recovery.l1_batch_number + 1); - let pending_batch = mempool - .load_pending_batch() - .await - .unwrap() - .expect("no pending batch"); + let pending_batch = maybe_pending_batch.expect("no pending batch"); assert_eq!( pending_batch.l1_batch_env.number, snapshot_recovery.l1_batch_number + 1 @@ -574,89 +502,6 @@ async fn miniblock_processing_after_snapshot_recovery() { assert_eq!(pending_batch.pending_miniblocks[0].txs[0].hash(), tx_hash); } -#[tokio::test] -async fn miniblock_sealer_handle_blocking() { - let pool = ConnectionPool::::constrained_test_pool(1).await; - let (mut sealer, mut sealer_handle) = MiniblockSealer::new(pool, 1); - - // The first command should be successfully submitted immediately. - let updates_manager = create_updates_manager(); - let seal_command = updates_manager.seal_miniblock_command( - L1BatchNumber(1), - MiniblockNumber(1), - Address::default(), - false, - ); - sealer_handle.submit(seal_command).await; - - // The second command should lead to blocking - let seal_command = updates_manager.seal_miniblock_command( - L1BatchNumber(1), - MiniblockNumber(2), - Address::default(), - false, - ); - { - let submit_future = sealer_handle.submit(seal_command); - futures::pin_mut!(submit_future); - - assert!((&mut submit_future).now_or_never().is_none()); - // ...until miniblock #1 is processed - let command = sealer.commands_receiver.recv().await.unwrap(); - command.completion_sender.send(()).unwrap_err(); // completion receiver should be dropped - submit_future.await; - } - - { - let wait_future = sealer_handle.wait_for_all_commands(); - futures::pin_mut!(wait_future); - assert!((&mut wait_future).now_or_never().is_none()); - let command = sealer.commands_receiver.recv().await.unwrap(); - command.completion_sender.send(()).unwrap(); - wait_future.await; - } - - // Check that `wait_for_all_commands()` state is reset after use. - sealer_handle.wait_for_all_commands().await; - - let seal_command = updates_manager.seal_miniblock_command( - L1BatchNumber(2), - MiniblockNumber(3), - Address::default(), - false, - ); - sealer_handle.submit(seal_command).await; - let command = sealer.commands_receiver.recv().await.unwrap(); - command.completion_sender.send(()).unwrap(); - sealer_handle.wait_for_all_commands().await; -} - -#[tokio::test] -async fn miniblock_sealer_handle_parallel_processing() { - let pool = ConnectionPool::::constrained_test_pool(1).await; - let (mut sealer, mut sealer_handle) = MiniblockSealer::new(pool, 5); - - // 5 miniblock sealing commands can be submitted without blocking. - for i in 1..=5 { - let updates_manager = create_updates_manager(); - let seal_command = updates_manager.seal_miniblock_command( - L1BatchNumber(1), - MiniblockNumber(i), - Address::default(), - false, - ); - sealer_handle.submit(seal_command).await; - } - - for i in 1..=5 { - let command = sealer.commands_receiver.recv().await.unwrap(); - assert_eq!(command.command.miniblock_number, MiniblockNumber(i)); - command.completion_sender.send(()).ok(); - } - - sealer_handle.wait_for_all_commands().await; -} - /// Ensure that subsequent miniblocks that belong to the same L1 batch have different timestamps #[tokio::test] async fn different_timestamp_for_miniblocks_in_same_batch() { @@ -665,12 +510,13 @@ async fn different_timestamp_for_miniblocks_in_same_batch() { // Genesis is needed for proper mempool initialization. tester.genesis(&connection_pool).await; - let (mut mempool, _) = tester.create_test_mempool_io(connection_pool, 1).await; + let (mut mempool, _) = tester.create_test_mempool_io(connection_pool).await; + let (mut io_cursor, _) = mempool.initialize().await.unwrap(); let current_timestamp = seconds_since_epoch(); - mempool.set_prev_miniblock_timestamp(current_timestamp); + io_cursor.prev_miniblock_timestamp = current_timestamp; let miniblock_params = mempool - .wait_for_new_miniblock_params(Duration::from_secs(10)) + .wait_for_new_miniblock_params(&io_cursor, Duration::from_secs(10)) .await .unwrap() .expect("no new miniblock params"); diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs index 154d011e141..13c7e830d82 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs @@ -10,7 +10,6 @@ use zksync_config::{ use zksync_contracts::BaseSystemContracts; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_eth_client::clients::MockEthereum; -use zksync_object_store::ObjectStoreFactory; use zksync_types::{ block::MiniblockHeader, fee::TransactionExecutionMetrics, @@ -26,7 +25,7 @@ use crate::{ fee_model::MainNodeFeeInputProvider, genesis::create_genesis_l1_batch, l1_gas_price::GasAdjuster, - state_keeper::{io::MiniblockSealer, MempoolGuard, MempoolIO}, + state_keeper::{MempoolGuard, MempoolIO}, utils::testonly::{ create_l1_batch, create_l2_transaction, create_miniblock, execute_l2_transaction, }, @@ -92,7 +91,6 @@ impl Tester { pub(super) async fn create_test_mempool_io( &self, pool: ConnectionPool, - miniblock_sealer_capacity: usize, ) -> (MempoolIO, MempoolGuard) { let gas_adjuster = Arc::new(self.create_gas_adjuster().await); let batch_fee_input_provider = MainNodeFeeInputProvider::new( @@ -103,29 +101,20 @@ impl Tester { ); let mempool = MempoolGuard::new(PriorityOpId(0), 100); - let (miniblock_sealer, miniblock_sealer_handle) = - MiniblockSealer::new(pool.clone(), miniblock_sealer_capacity); - tokio::spawn(miniblock_sealer.run()); - let config = StateKeeperConfig { minimal_l2_gas_price: self.minimal_l2_gas_price(), virtual_blocks_interval: 1, virtual_blocks_per_miniblock: 1, fee_account_addr: Address::repeat_byte(0x11), // Maintain implicit invariant: fee address is never `Address::zero()` - ..StateKeeperConfig::default() + validation_computational_gas_limit: BLOCK_GAS_LIMIT, + ..StateKeeperConfig::for_tests() }; - let object_store = ObjectStoreFactory::mock().create_store().await; - let l2_erc20_bridge_addr = Address::repeat_byte(0x5a); // Isn't relevant. let io = MempoolIO::new( mempool.clone(), - object_store, - miniblock_sealer_handle, Arc::new(batch_fee_input_provider), pool, &config, Duration::from_secs(1), - l2_erc20_bridge_addr, - BLOCK_GAS_LIMIT, L2ChainId::from(270), ) .await diff --git a/core/lib/zksync_core/src/state_keeper/keeper.rs b/core/lib/zksync_core/src/state_keeper/keeper.rs index e8bce4c320b..53ea9d08f22 100644 --- a/core/lib/zksync_core/src/state_keeper/keeper.rs +++ b/core/lib/zksync_core/src/state_keeper/keeper.rs @@ -18,13 +18,16 @@ use zksync_types::{ use super::{ batch_executor::{BatchExecutor, BatchExecutorHandle, TxExecutionResult}, extractors, - io::{MiniblockParams, PendingBatchData, StateKeeperIO}, + io::{ + fee_address_migration, IoCursor, MiniblockParams, OutputHandler, PendingBatchData, + StateKeeperIO, + }, metrics::{AGGREGATION_METRICS, KEEPER_METRICS, L1_BATCH_METRICS}, seal_criteria::{ConditionalSealer, SealData, SealResolution}, types::ExecutionMetricsForCriteria, updates::UpdatesManager, }; -use crate::{gas_tracker::gas_count_from_writes, state_keeper::io::fee_address_migration}; +use crate::gas_tracker::gas_count_from_writes; /// Amount of time to block on waiting for some resource. The exact value is not really important, /// we only need it to not block on waiting indefinitely and be able to process cancellation requests. @@ -61,6 +64,7 @@ impl Error { pub struct ZkSyncStateKeeper { stop_receiver: watch::Receiver, io: Box, + output_handler: OutputHandler, batch_executor_base: Box, sealer: Arc, } @@ -68,14 +72,16 @@ pub struct ZkSyncStateKeeper { impl ZkSyncStateKeeper { pub fn new( stop_receiver: watch::Receiver, - io: Box, + sequencer: Box, batch_executor_base: Box, + output_handler: OutputHandler, sealer: Arc, ) -> Self { Self { stop_receiver, - io, + io: sequencer, batch_executor_base, + output_handler, sealer, } } @@ -85,11 +91,9 @@ impl ZkSyncStateKeeper { &self, pool: ConnectionPool, ) -> impl Future> { - let last_miniblock = self.io.current_miniblock_number() - 1; let mut stop_receiver = self.stop_receiver.clone(); async move { - fee_address_migration::migrate_miniblocks(pool, last_miniblock, stop_receiver.clone()) - .await?; + fee_address_migration::migrate_miniblocks(pool, stop_receiver.clone()).await?; // Since this is run as a task, we don't want it to exit on success (this would shut down the node). // We still want for the task to be cancellation-aware, so we just wait until a stop signal is sent. stop_receiver.changed().await.ok(); @@ -110,18 +114,14 @@ impl ZkSyncStateKeeper { /// Fallible version of `run` routine that allows to easily exit upon cancellation. async fn run_inner(&mut self) -> Result { + let (cursor, pending_batch_params) = self.io.initialize().await?; + self.output_handler.initialize(&cursor).await?; tracing::info!( "Starting state keeper. Next l1 batch to seal: {}, Next miniblock to seal: {}", - self.io.current_l1_batch_number(), - self.io.current_miniblock_number() + cursor.l1_batch, + cursor.next_miniblock ); - let pending_batch_params = self - .io - .load_pending_batch() - .await - .context("failed loading pending L1 batch")?; - // Re-execute pending batch if it exists. Otherwise, initialize a new batch. let PendingBatchData { mut l1_batch_env, @@ -143,7 +143,7 @@ impl ZkSyncStateKeeper { None => { tracing::info!("There is no open pending batch, starting a new empty batch"); let (system_env, l1_batch_env) = self - .wait_for_new_batch_params() + .wait_for_new_batch_env(&cursor) .await .map_err(|e| e.context("wait_for_new_batch_params()"))?; PendingBatchData { @@ -182,10 +182,11 @@ impl ZkSyncStateKeeper { // Finish current batch. if !updates_manager.miniblock.executed_transactions.is_empty() { - self.io.seal_miniblock(&updates_manager).await; - // We've sealed the miniblock that we had, but we still need to setup the timestamp + self.seal_miniblock(&updates_manager).await?; + // We've sealed the miniblock that we had, but we still need to set up the timestamp // for the fictive miniblock. - let new_miniblock_params = self.wait_for_new_miniblock_params().await?; + let new_miniblock_params = + self.wait_for_new_miniblock_params(&updates_manager).await?; Self::start_next_miniblock( new_miniblock_params, &mut updates_manager, @@ -193,24 +194,24 @@ impl ZkSyncStateKeeper { ) .await; } + let (finished_batch, witness_block_state) = batch_executor.finish_batch().await; let sealed_batch_protocol_version = updates_manager.protocol_version(); - self.io - .seal_l1_batch( - witness_block_state, - updates_manager, - &l1_batch_env, - finished_batch, - ) + updates_manager.finish_batch(finished_batch); + self.output_handler + .handle_l1_batch(witness_block_state.as_ref(), &updates_manager) .await .with_context(|| format!("failed sealing L1 batch {l1_batch_env:?}"))?; + if let Some(delta) = l1_batch_seal_delta { L1_BATCH_METRICS.seal_delta.observe(delta.elapsed()); } l1_batch_seal_delta = Some(Instant::now()); // Start the new batch. - (system_env, l1_batch_env) = self.wait_for_new_batch_params().await?; + let mut next_cursor = updates_manager.io_cursor(); + next_cursor.l1_batch += 1; + (system_env, l1_batch_env) = self.wait_for_new_batch_env(&next_cursor).await?; updates_manager = UpdatesManager::new(&l1_batch_env, &system_env); batch_executor = self .batch_executor_base @@ -245,10 +246,11 @@ impl ZkSyncStateKeeper { // there has to be a setChainId upgrade transaction after the chain genesis. // It has to be the first transaction of the first batch. // The setChainId upgrade does not bump the protocol version, but attaches an upgrade - // transaction to the genesis protocol version version. + // transaction to the genesis protocol version. let first_batch_in_shared_bridge = l1_batch_number == L1BatchNumber(1) && !protocol_version.is_pre_shared_bridge(); - let previous_batch_protocol_version = self.io.load_previous_batch_version_id().await?; + let previous_batch_protocol_version = + self.io.load_batch_version_id(l1_batch_number - 1).await?; let version_changed = protocol_version != previous_batch_protocol_version; let mut protocol_upgrade_tx = if version_changed || first_batch_in_shared_bridge { @@ -294,25 +296,32 @@ impl ZkSyncStateKeeper { .with_context(|| format!("failed loading upgrade transaction for {protocol_version:?}")) } - async fn wait_for_new_batch_params(&mut self) -> Result<(SystemEnv, L1BatchEnv), Error> { + async fn wait_for_new_batch_env( + &mut self, + cursor: &IoCursor, + ) -> Result<(SystemEnv, L1BatchEnv), Error> { while !self.is_canceled() { - if let Some(params) = self + if let Some(envs) = self .io - .wait_for_new_batch_params(POLL_WAIT_DURATION) + .wait_for_new_batch_env(cursor, POLL_WAIT_DURATION) .await - .context("error waiting for new L1 batch params")? + .context("error waiting for new L1 batch environment")? { - return Ok(params); + return Ok(envs); } } Err(Error::Canceled) } - async fn wait_for_new_miniblock_params(&mut self) -> Result { + async fn wait_for_new_miniblock_params( + &mut self, + updates: &UpdatesManager, + ) -> Result { + let cursor = updates.io_cursor(); while !self.is_canceled() { if let Some(params) = self .io - .wait_for_new_miniblock_params(POLL_WAIT_DURATION) + .wait_for_new_miniblock_params(&cursor, POLL_WAIT_DURATION) .await .context("error waiting for new miniblock params")? { @@ -333,6 +342,18 @@ impl ZkSyncStateKeeper { .await; } + async fn seal_miniblock(&mut self, updates_manager: &UpdatesManager) -> anyhow::Result<()> { + self.output_handler + .handle_miniblock(updates_manager) + .await + .with_context(|| { + format!( + "handling miniblock #{} failed", + updates_manager.miniblock.number + ) + }) + } + /// Applies the "pending state" on the `UpdatesManager`. /// Pending state means transactions that were executed before the server restart. Before we continue processing the /// batch, we need to restore the state. We must ensure that every transaction is executed successfully. @@ -414,7 +435,7 @@ impl ZkSyncStateKeeper { status: {exec_result_status:?}. L1 gas spent: {tx_l1_gas_this_tx:?}, total in L1 batch: {pending_l1_gas:?}, \ tx execution metrics: {tx_execution_metrics:?}, block execution metrics: {block_execution_metrics:?}", idx_in_l1_batch = updates_manager.pending_executed_transactions_len(), - l1_batch_number = self.io.current_l1_batch_number().0, + l1_batch_number = updates_manager.l1_batch.number, idx_in_miniblock = updates_manager.miniblock.executed_transactions.len(), pending_l1_gas = updates_manager.pending_l1_gas_count(), block_execution_metrics = updates_manager.pending_execution_metrics() @@ -428,7 +449,7 @@ impl ZkSyncStateKeeper { // We've processed all the miniblocks, and right now we're initializing the next *actual* miniblock. let new_miniblock_params = self - .wait_for_new_miniblock_params() + .wait_for_new_miniblock_params(updates_manager) .await .map_err(|e| e.context("wait_for_new_miniblock_params"))?; Self::start_next_miniblock(new_miniblock_params, updates_manager, batch_executor).await; @@ -454,7 +475,7 @@ impl ZkSyncStateKeeper { { tracing::debug!( "L1 batch #{} should be sealed unconditionally as per sealing rules", - self.io.current_l1_batch_number() + updates_manager.l1_batch.number ); return Ok(()); } @@ -462,19 +483,19 @@ impl ZkSyncStateKeeper { if self.io.should_seal_miniblock(updates_manager) { tracing::debug!( "Miniblock #{} (L1 batch #{}) should be sealed as per sealing rules", - self.io.current_miniblock_number(), - self.io.current_l1_batch_number() + updates_manager.miniblock.number, + updates_manager.l1_batch.number ); - self.io.seal_miniblock(updates_manager).await; + self.seal_miniblock(updates_manager).await?; let new_miniblock_params = self - .wait_for_new_miniblock_params() + .wait_for_new_miniblock_params(updates_manager) .await .map_err(|e| e.context("wait_for_new_miniblock_params"))?; tracing::debug!( "Initialized new miniblock #{} (L1 batch #{}) with timestamp {}", - self.io.current_miniblock_number(), - self.io.current_l1_batch_number(), + updates_manager.miniblock.number, + updates_manager.l1_batch.number, extractors::display_timestamp(new_miniblock_params.timestamp) ); Self::start_next_miniblock(new_miniblock_params, updates_manager, batch_executor) @@ -482,7 +503,12 @@ impl ZkSyncStateKeeper { } let waiting_latency = KEEPER_METRICS.waiting_for_tx.start(); - let Some(tx) = self.io.wait_for_next_tx(POLL_WAIT_DURATION).await else { + let Some(tx) = self + .io + .wait_for_next_tx(POLL_WAIT_DURATION) + .await + .context("error waiting for next transaction")? + else { waiting_latency.observe(); tracing::trace!("No new transactions. Waiting!"); continue; @@ -523,7 +549,10 @@ impl ZkSyncStateKeeper { } SealResolution::ExcludeAndSeal => { batch_executor.rollback_last_tx().await; - self.io.rollback(tx).await; + self.io + .rollback(tx) + .await + .context("failed rolling back transaction")?; } SealResolution::Unexecutable(reason) => { batch_executor.rollback_last_tx().await; @@ -538,7 +567,7 @@ impl ZkSyncStateKeeper { tracing::debug!( "L1 batch #{} should be sealed with resolution {seal_resolution:?} after executing \ transaction {tx_hash}", - self.io.current_l1_batch_number() + updates_manager.l1_batch.number ); return Ok(()); } @@ -674,9 +703,9 @@ impl ZkSyncStateKeeper { tx.initiator_account(), tx.is_l1(), updates_manager.pending_executed_transactions_len() + 1, - self.io.current_l1_batch_number().0, + updates_manager.l1_batch.number, updates_manager.miniblock.executed_transactions.len() + 1, - self.io.current_miniblock_number().0, + updates_manager.miniblock.number, tx_execution_status, tx_l1_gas_this_tx, updates_manager.pending_l1_gas_count() + tx_l1_gas_this_tx, @@ -722,7 +751,7 @@ impl ZkSyncStateKeeper { }; self.sealer.should_seal_l1_batch( - self.io.current_l1_batch_number().0, + updates_manager.l1_batch.number.0, updates_manager.batch_timestamp() as u128 * 1_000, updates_manager.pending_executed_transactions_len() + 1, &block_data, diff --git a/core/lib/zksync_core/src/state_keeper/metrics.rs b/core/lib/zksync_core/src/state_keeper/metrics.rs index 1445f6584a0..6635587f30c 100644 --- a/core/lib/zksync_core/src/state_keeper/metrics.rs +++ b/core/lib/zksync_core/src/state_keeper/metrics.rs @@ -175,7 +175,6 @@ pub(super) static AGGREGATION_METRICS: vise::Global = vise #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "stage", rename_all = "snake_case")] pub(super) enum L1BatchSealStage { - VmFinalization, FictiveMiniblock, LogDeduplication, InsertL1BatchHeader, diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index 8037d00dd22..16288ea44ac 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -3,14 +3,16 @@ use std::sync::Arc; use tokio::sync::watch; use zksync_config::{ configs::chain::{MempoolConfig, NetworkConfig, StateKeeperConfig}, - ContractsConfig, DBConfig, + DBConfig, }; use zksync_dal::{ConnectionPool, Core}; -use zksync_object_store::ObjectStore; pub use self::{ batch_executor::{main_executor::MainBatchExecutor, BatchExecutor}, - io::{mempool::MempoolIO, MiniblockSealer, MiniblockSealerHandle, StateKeeperIO}, + io::{ + mempool::MempoolIO, MiniblockSealerTask, OutputHandler, StateKeeperIO, + StateKeeperOutputHandler, StateKeeperPersistence, + }, keeper::ZkSyncStateKeeper, mempool_actor::MempoolFetcher, seal_criteria::SequencerSealer, @@ -32,7 +34,6 @@ pub(crate) mod updates; #[allow(clippy::too_many_arguments)] pub(crate) async fn create_state_keeper( - contracts_config: &ContractsConfig, state_keeper_config: StateKeeperConfig, db_config: &DBConfig, network_config: &NetworkConfig, @@ -40,14 +41,12 @@ pub(crate) async fn create_state_keeper( pool: ConnectionPool, mempool: MempoolGuard, batch_fee_input_provider: Arc, - miniblock_sealer_handle: MiniblockSealerHandle, - object_store: Arc, + output_handler: OutputHandler, stop_receiver: watch::Receiver, ) -> ZkSyncStateKeeper { let batch_executor_base = MainBatchExecutor::new( db_config.state_keeper_db_path.clone(), pool.clone(), - state_keeper_config.max_allowed_l2_tx_gas_limit.into(), state_keeper_config.save_call_traces, state_keeper_config.upload_witness_inputs_to_gcs, state_keeper_config.enum_index_migration_chunk_size(), @@ -56,14 +55,10 @@ pub(crate) async fn create_state_keeper( let io = MempoolIO::new( mempool, - object_store, - miniblock_sealer_handle, batch_fee_input_provider, pool, &state_keeper_config, mempool_config.delay_interval(), - contracts_config.l2_erc20_bridge_addr, - state_keeper_config.validation_computational_gas_limit, network_config.zksync_network_id, ) .await @@ -74,6 +69,7 @@ pub(crate) async fn create_state_keeper( stop_receiver, Box::new(io), Box::new(batch_executor_base), + output_handler, Arc::new(sealer), ) } 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 9ce22ea819c..8fb5054a1ab 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -251,7 +251,7 @@ async fn sealed_by_gas() { }) .next_tx("Second tx", random_tx(1), execution_result) .miniblock_sealed("Miniblock 2") - .batch_sealed_with("Batch sealed with both txs", |_, updates, _| { + .batch_sealed_with("Batch sealed with both txs", |updates| { assert_eq!( updates.l1_batch.l1_gas_count, BlockGasCount { @@ -429,7 +429,7 @@ async fn pending_batch_is_applied() { "Only one transaction should be in miniblock" ); }) - .batch_sealed_with("Batch sealed with all 3 txs", |_, updates, _| { + .batch_sealed_with("Batch sealed with all 3 txs", |updates| { assert_eq!( updates.l1_batch.executed_transactions.len(), 3, @@ -448,7 +448,7 @@ async fn load_upgrade_tx() { let batch_executor_base = TestBatchExecutorBuilder::new(&scenario); let (stop_sender, stop_receiver) = watch::channel(false); - let mut io = TestIO::new(stop_sender, scenario); + let (mut io, output_handler) = TestIO::new(stop_sender, scenario); io.add_upgrade_tx(ProtocolVersionId::latest(), random_upgrade_tx(1)); io.add_upgrade_tx(ProtocolVersionId::next(), random_upgrade_tx(2)); @@ -456,6 +456,7 @@ async fn load_upgrade_tx() { stop_receiver, Box::new(io), Box::new(batch_executor_base), + output_handler, Arc::new(sealer), ); @@ -545,8 +546,8 @@ async fn miniblock_timestamp_after_pending_batch() { successful_exec(), ) .miniblock_sealed_with("Miniblock with a single tx", move |updates| { - assert!( - updates.miniblock.timestamp == 1, + assert_eq!( + updates.miniblock.timestamp, 2, "Timestamp for the new block must be taken from the test IO" ); }) @@ -597,7 +598,7 @@ async fn time_is_monotonic() { ); timestamp_second_miniblock.store(updates.miniblock.timestamp, Ordering::Relaxed); }) - .batch_sealed_with("Batch 1", move |_, updates, _| { + .batch_sealed_with("Batch 1", move |updates| { // Timestamp from the currently stored miniblock would be used in the fictive miniblock. // It should be correct as well. let min_expected = timestamp_third_miniblock.load(Ordering::Relaxed); @@ -629,7 +630,7 @@ async fn protocol_upgrade() { .increment_protocol_version("Increment protocol version") .next_tx("Second tx", random_tx(2), successful_exec()) .miniblock_sealed("Miniblock 2") - .batch_sealed_with("Batch 1", move |_, updates, _| { + .batch_sealed_with("Batch 1", move |updates| { assert_eq!( updates.protocol_version(), ProtocolVersionId::latest(), 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 14959130fe9..a3a1243e5c3 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/tester.rs @@ -1,20 +1,20 @@ use std::{ collections::{HashMap, HashSet, VecDeque}, convert::TryInto, - fmt, - sync::Arc, + fmt, mem, + sync::{Arc, Mutex}, time::{Duration, Instant}, }; use async_trait::async_trait; use multivm::{ interface::{ - ExecutionResult, FinishedL1Batch, L1BatchEnv, L2BlockEnv, SystemEnv, TxExecutionMode, - VmExecutionResultAndLogs, + ExecutionResult, L1BatchEnv, SystemEnv, TxExecutionMode, VmExecutionResultAndLogs, }, vm_latest::constants::BLOCK_GAS_LIMIT, }; use tokio::sync::{mpsc, watch}; +use zksync_contracts::BaseSystemContracts; use zksync_types::{ block::MiniblockExecutionData, fee_model::BatchFeeInput, protocol_upgrade::ProtocolUpgradeTx, witness_block_state::WitnessBlockState, Address, L1BatchNumber, L2ChainId, MiniblockNumber, @@ -24,12 +24,12 @@ use zksync_types::{ use crate::{ state_keeper::{ batch_executor::{BatchExecutor, BatchExecutorHandle, Command, TxExecutionResult}, - io::{MiniblockParams, PendingBatchData, StateKeeperIO}, + io::{IoCursor, L1BatchParams, MiniblockParams, PendingBatchData, StateKeeperIO}, seal_criteria::{IoSealCriteria, SequencerSealer}, tests::{default_l1_batch_env, default_vm_block_result, BASE_SYSTEM_CONTRACTS}, types::ExecutionMetricsForCriteria, updates::UpdatesManager, - ZkSyncStateKeeper, + OutputHandler, StateKeeperOutputHandler, ZkSyncStateKeeper, }, utils::testonly::create_l2_transaction, }; @@ -166,7 +166,7 @@ impl TestScenario { /// additional assertions on the sealed batch. pub(crate) fn batch_sealed_with(mut self, description: &'static str, f: F) -> Self where - F: FnOnce(&VmExecutionResultAndLogs, &UpdatesManager, &L1BatchEnv) + Send + 'static, + F: FnOnce(&UpdatesManager) + Send + 'static, { self.actions .push_back(ScenarioItem::BatchSeal(description, Some(Box::new(f)))); @@ -196,14 +196,15 @@ impl TestScenario { let batch_executor_base = TestBatchExecutorBuilder::new(&self); let (stop_sender, stop_receiver) = watch::channel(false); - let io = TestIO::new(stop_sender, self); - let sk = ZkSyncStateKeeper::new( + let (io, output_handler) = TestIO::new(stop_sender, self); + let state_keeper = ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(batch_executor_base), + output_handler, Arc::new(sealer), ); - let sk_thread = tokio::spawn(sk.run()); + let sk_thread = tokio::spawn(state_keeper.run()); // We must assume that *theoretically* state keeper may ignore the stop signal from IO once scenario is // completed, so we spawn it in a separate thread to not get test stuck. @@ -324,35 +325,42 @@ enum ScenarioItem { ), BatchSeal( &'static str, - Option>, + Option>, ), } -impl std::fmt::Debug for ScenarioItem { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { +impl fmt::Debug for ScenarioItem { + fn fmt(&self, formatter: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Self::NoTxsUntilNextAction(descr) => { - f.debug_tuple("NoTxsUntilNextAction").field(descr).finish() - } - Self::IncrementProtocolVersion(descr) => f + Self::NoTxsUntilNextAction(descr) => formatter + .debug_tuple("NoTxsUntilNextAction") + .field(descr) + .finish(), + Self::IncrementProtocolVersion(descr) => formatter .debug_tuple("IncrementProtocolVersion") .field(descr) .finish(), - Self::Tx(descr, tx, result) => f + Self::Tx(descr, tx, result) => formatter .debug_tuple("Tx") .field(descr) .field(tx) .field(result) .finish(), - Self::Rollback(descr, tx) => f.debug_tuple("Rollback").field(descr).field(tx).finish(), - Self::Reject(descr, tx, err) => f + Self::Rollback(descr, tx) => formatter + .debug_tuple("Rollback") + .field(descr) + .field(tx) + .finish(), + Self::Reject(descr, tx, err) => formatter .debug_tuple("Reject") .field(descr) .field(tx) .field(err) .finish(), - Self::MiniblockSeal(descr, _) => f.debug_tuple("MiniblockSeal").field(descr).finish(), - Self::BatchSeal(descr, _) => f.debug_tuple("BatchSeal").field(descr).finish(), + Self::MiniblockSeal(descr, _) => { + formatter.debug_tuple("MiniblockSeal").field(descr).finish() + } + Self::BatchSeal(descr, _) => formatter.debug_tuple("BatchSeal").field(descr).finish(), } } } @@ -409,14 +417,14 @@ impl TestBatchExecutorBuilder { ScenarioItem::Reject(_, tx, _) => { rollback_set.insert(tx.hash()); } - ScenarioItem::BatchSeal(_, _) => txs.push_back(std::mem::take(&mut batch_txs)), + ScenarioItem::BatchSeal(_, _) => txs.push_back(mem::take(&mut batch_txs)), _ => {} } } // Some batch seal may not be included into scenario, dump such txs if they exist. if !batch_txs.is_empty() { - txs.push_back(std::mem::take(&mut batch_txs)); + txs.push_back(mem::take(&mut batch_txs)); } // After sealing the batch, state keeper initialized a new one, so we need to create an empty set // for the initialization of the "next-to-last" batch. @@ -529,37 +537,120 @@ impl TestBatchExecutor { } #[derive(Debug)] +pub(super) struct TestPersistence { + actions: Arc>>, + stop_sender: Arc>, +} + +impl TestPersistence { + fn pop_next_item(&self, request: &str) -> ScenarioItem { + let mut actions = self.actions.lock().expect("scenario queue is poisoned"); + let action = actions + .pop_front() + .unwrap_or_else(|| panic!("no action for request: {request}")); + // If that was a last action, tell the state keeper to stop after that. + if actions.is_empty() { + self.stop_sender.send_replace(true); + } + action + } +} + +#[async_trait] +impl StateKeeperOutputHandler for TestPersistence { + async fn handle_miniblock(&mut self, updates_manager: &UpdatesManager) -> anyhow::Result<()> { + let action = self.pop_next_item("seal_miniblock"); + let ScenarioItem::MiniblockSeal(_, check_fn) = action else { + anyhow::bail!("Unexpected action: {:?}", action); + }; + if let Some(check_fn) = check_fn { + check_fn(updates_manager); + } + Ok(()) + } + + async fn handle_l1_batch( + &mut self, + _witness_block_state: Option<&WitnessBlockState>, + updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + let action = self.pop_next_item("seal_l1_batch"); + let ScenarioItem::BatchSeal(_, check_fn) = action else { + anyhow::bail!("Unexpected action: {:?}", action); + }; + if let Some(check_fn) = check_fn { + check_fn(updates_manager); + } + Ok(()) + } +} + pub(super) struct TestIO { - stop_sender: watch::Sender, + stop_sender: Arc>, batch_number: L1BatchNumber, timestamp: u64, fee_input: BatchFeeInput, miniblock_number: MiniblockNumber, fee_account: Address, - scenario: TestScenario, + pending_batch: Option, + l1_batch_seal_fn: Box, + miniblock_seal_fn: Box, + actions: Arc>>, /// Internal flag that is being set if scenario was configured to return `None` to all the transaction /// requests until some other action happens. skipping_txs: bool, protocol_version: ProtocolVersionId, - previous_batch_protocol_version: ProtocolVersionId, + previous_batch_protocol_version: ProtocolVersionId, // FIXME: not updated protocol_upgrade_txs: HashMap, } +impl fmt::Debug for TestIO { + fn fmt(&self, formatter: &mut fmt::Formatter<'_>) -> fmt::Result { + formatter.debug_struct("TestIO").finish_non_exhaustive() + } +} + impl TestIO { - pub(super) fn new(stop_sender: watch::Sender, scenario: TestScenario) -> Self { - Self { + pub(super) fn new( + stop_sender: watch::Sender, + scenario: TestScenario, + ) -> (Self, OutputHandler) { + let stop_sender = Arc::new(stop_sender); + let actions = Arc::new(Mutex::new(scenario.actions)); + let persistence = TestPersistence { + stop_sender: stop_sender.clone(), + actions: actions.clone(), + }; + + let (miniblock_number, timestamp) = if let Some(pending_batch) = &scenario.pending_batch { + let last_pending_miniblock = pending_batch + .pending_miniblocks + .last() + .expect("pending batch should have at least one miniblock"); + ( + last_pending_miniblock.number + 1, + last_pending_miniblock.timestamp + 1, + ) + } else { + (MiniblockNumber(1), 1) + }; + let this = Self { stop_sender, batch_number: L1BatchNumber(1), - timestamp: 1, + timestamp, fee_input: BatchFeeInput::default(), - miniblock_number: MiniblockNumber(1), + pending_batch: scenario.pending_batch, + l1_batch_seal_fn: scenario.l1_batch_seal_fn, + miniblock_seal_fn: scenario.miniblock_seal_fn, + actions, + miniblock_number, fee_account: FEE_ACCOUNT, - scenario, skipping_txs: false, protocol_version: ProtocolVersionId::latest(), previous_batch_protocol_version: ProtocolVersionId::latest(), protocol_upgrade_txs: HashMap::default(), - } + }; + (this, OutputHandler::new(Box::new(persistence))) } pub(super) fn add_upgrade_tx(&mut self, version: ProtocolVersionId, tx: ProtocolUpgradeTx) { @@ -567,104 +658,108 @@ impl TestIO { } fn pop_next_item(&mut self, request: &str) -> ScenarioItem { - if self.scenario.actions.is_empty() { - panic!( - "Test scenario is empty, but the following action was done by the state keeper: {}", - request - ); - } - - let action = self.scenario.actions.pop_front().unwrap(); - if matches!(action, ScenarioItem::NoTxsUntilNextAction(_)) { - self.skipping_txs = true; - // This is a mock item, so pop an actual one for the IO to process. - return self.pop_next_item(request); - } - - if matches!(action, ScenarioItem::IncrementProtocolVersion(_)) { - self.protocol_version = (self.protocol_version as u16 + 1) - .try_into() - .expect("Cannot increment latest version"); - // This is a mock item, so pop an actual one for the IO to process. - return self.pop_next_item(request); - } + let mut actions = self.actions.lock().expect("scenario queue is poisoned"); + loop { + let action = actions.pop_front().unwrap_or_else(|| { + panic!( + "Test scenario is empty, but the following action was done by the state keeper: {request}" + ); + }); + // If that was a last action, tell the state keeper to stop after that. + if actions.is_empty() { + self.stop_sender.send_replace(true); + } - // If that was a last action, tell the state keeper to stop after that. - if self.scenario.actions.is_empty() { - self.stop_sender.send(true).unwrap(); + match &action { + ScenarioItem::NoTxsUntilNextAction(_) => { + self.skipping_txs = true; + // This is a mock item, so pop an actual one for the IO to process. + continue; + } + ScenarioItem::IncrementProtocolVersion(_) => { + self.protocol_version = (self.protocol_version as u16 + 1) + .try_into() + .expect("Cannot increment latest version"); + // This is a mock item, so pop an actual one for the IO to process. + continue; + } + _ => break action, + } } - action } } impl IoSealCriteria for TestIO { fn should_seal_l1_batch_unconditionally(&mut self, manager: &UpdatesManager) -> bool { - (self.scenario.l1_batch_seal_fn)(manager) + (self.l1_batch_seal_fn)(manager) } fn should_seal_miniblock(&mut self, manager: &UpdatesManager) -> bool { - (self.scenario.miniblock_seal_fn)(manager) + (self.miniblock_seal_fn)(manager) } } #[async_trait] impl StateKeeperIO for TestIO { - fn current_l1_batch_number(&self) -> L1BatchNumber { - self.batch_number - } - - fn current_miniblock_number(&self) -> MiniblockNumber { - self.miniblock_number + fn chain_id(&self) -> L2ChainId { + L2ChainId::default() } - async fn load_pending_batch(&mut self) -> anyhow::Result> { - Ok(self.scenario.pending_batch.take()) + async fn initialize(&mut self) -> anyhow::Result<(IoCursor, Option)> { + let cursor = IoCursor { + next_miniblock: self.miniblock_number, + prev_miniblock_hash: H256::zero(), + prev_miniblock_timestamp: self.timestamp.saturating_sub(1), + l1_batch: self.batch_number, + }; + let pending_batch = self.pending_batch.take(); + Ok((cursor, pending_batch)) } async fn wait_for_new_batch_params( &mut self, + cursor: &IoCursor, _max_wait: Duration, - ) -> anyhow::Result> { - let first_miniblock_info = L2BlockEnv { - number: self.miniblock_number.0, - timestamp: self.timestamp, - prev_block_hash: H256::zero(), - max_virtual_blocks_to_create: 1, - }; - Ok(Some(( - SystemEnv { - zk_porter_available: false, - version: self.protocol_version, - base_system_smart_contracts: BASE_SYSTEM_CONTRACTS.clone(), - gas_limit: BLOCK_GAS_LIMIT, - execution_mode: TxExecutionMode::VerifyExecute, - default_validation_computational_gas_limit: BLOCK_GAS_LIMIT, - chain_id: L2ChainId::from(270), - }, - L1BatchEnv { - previous_batch_hash: Some(H256::zero()), - number: self.batch_number, + ) -> anyhow::Result> { + assert_eq!(cursor.next_miniblock, self.miniblock_number); + assert_eq!(cursor.l1_batch, self.batch_number); + + let params = L1BatchParams { + protocol_version: self.protocol_version, + validation_computational_gas_limit: BLOCK_GAS_LIMIT, + operator_address: self.fee_account, + fee_input: self.fee_input, + first_miniblock: MiniblockParams { timestamp: self.timestamp, - fee_input: self.fee_input, - fee_account: self.fee_account, - enforced_base_fee: None, - first_l2_block: first_miniblock_info, + virtual_blocks: 1, }, - ))) + }; + self.miniblock_number += 1; + self.timestamp += 1; + self.batch_number += 1; + Ok(Some(params)) } async fn wait_for_new_miniblock_params( &mut self, + cursor: &IoCursor, _max_wait: Duration, ) -> anyhow::Result> { - Ok(Some(MiniblockParams { + assert_eq!(cursor.next_miniblock, self.miniblock_number); + let params = MiniblockParams { timestamp: self.timestamp, // 1 is just a constant used for tests. virtual_blocks: 1, - })) + }; + self.miniblock_number += 1; + self.timestamp += 1; + Ok(Some(params)) } - async fn wait_for_next_tx(&mut self, max_wait: Duration) -> Option { + async fn wait_for_next_tx( + &mut self, + max_wait: Duration, + ) -> anyhow::Result> { let action = self.pop_next_item("wait_for_next_tx"); // Check whether we should ignore tx requests. @@ -672,18 +767,18 @@ impl StateKeeperIO for TestIO { // As per expectation, we should provide a delay given by the state keeper. tokio::time::sleep(max_wait).await; // Return the action to the scenario as we don't use it. - self.scenario.actions.push_front(action); - return None; + self.actions.lock().unwrap().push_front(action); + return Ok(None); } // We shouldn't, process normally. let ScenarioItem::Tx(_, tx, _) = action else { panic!("Unexpected action: {:?}", action); }; - Some(tx) + Ok(Some(tx)) } - async fn rollback(&mut self, tx: Transaction) { + async fn rollback(&mut self, tx: Transaction) -> anyhow::Result<()> { let action = self.pop_next_item("rollback"); let ScenarioItem::Rollback(_, expected_tx) = action else { panic!("Unexpected action: {:?}", action); @@ -693,6 +788,7 @@ impl StateKeeperIO for TestIO { "Incorrect transaction has been rolled back" ); self.skipping_txs = false; + Ok(()) } async fn reject(&mut self, tx: &Transaction, error: &str) -> anyhow::Result<()> { @@ -713,47 +809,18 @@ impl StateKeeperIO for TestIO { Ok(()) } - async fn seal_miniblock(&mut self, updates_manager: &UpdatesManager) { - let action = self.pop_next_item("seal_miniblock"); - let ScenarioItem::MiniblockSeal(_, check_fn) = action else { - panic!("Unexpected action: {:?}", action); - }; - if let Some(check_fn) = check_fn { - check_fn(updates_manager); - } - self.miniblock_number += 1; - self.timestamp += 1; - self.skipping_txs = false; - } - - async fn seal_l1_batch( + async fn load_base_system_contracts( &mut self, - _witness_block_state: Option, - updates_manager: UpdatesManager, - l1_batch_env: &L1BatchEnv, - finished_batch: FinishedL1Batch, - ) -> anyhow::Result<()> { - let action = self.pop_next_item("seal_l1_batch"); - let ScenarioItem::BatchSeal(_, check_fn) = action else { - anyhow::bail!("Unexpected action: {:?}", action); - }; - if let Some(check_fn) = check_fn { - check_fn( - &finished_batch.block_tip_execution_result, - &updates_manager, - l1_batch_env, - ); - } - - self.miniblock_number += 1; // Seal the fictive miniblock. - self.batch_number += 1; - self.previous_batch_protocol_version = self.protocol_version; - self.timestamp += 1; - self.skipping_txs = false; - Ok(()) + _protocol_version: ProtocolVersionId, + _cursor: &IoCursor, + ) -> anyhow::Result { + Ok(BASE_SYSTEM_CONTRACTS.clone()) } - async fn load_previous_batch_version_id(&mut self) -> anyhow::Result { + async fn load_batch_version_id( + &mut self, + _number: L1BatchNumber, + ) -> anyhow::Result { Ok(self.previous_batch_protocol_version) } @@ -763,6 +830,13 @@ impl StateKeeperIO for TestIO { ) -> anyhow::Result> { Ok(self.protocol_upgrade_txs.get(&version_id).cloned()) } + + async fn load_batch_state_hash( + &mut self, + _l1_batch_number: L1BatchNumber, + ) -> anyhow::Result { + Ok(H256::zero()) + } } /// `BatchExecutor` which doesn't check anything at all. Accepts all transactions. diff --git a/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs b/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs index 7f18edb3320..baaa7f987ee 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs +++ b/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs @@ -1,31 +1,36 @@ +use multivm::interface::FinishedL1Batch; use zksync_types::{ block::BlockGasCount, priority_op_onchain_data::PriorityOpOnchainData, tx::{tx_execution_info::ExecutionMetrics, TransactionExecutionResult}, - ExecuteTransactionCommon, + ExecuteTransactionCommon, L1BatchNumber, }; use super::miniblock_updates::MiniblockUpdates; use crate::gas_tracker::new_block_gas_count; -#[derive(Debug, Clone, PartialEq)] +#[derive(Debug)] pub struct L1BatchUpdates { + pub number: L1BatchNumber, pub executed_transactions: Vec, pub priority_ops_onchain_data: Vec, pub block_execution_metrics: ExecutionMetrics, // how much L1 gas will it take to submit this block? pub l1_gas_count: BlockGasCount, pub txs_encoding_size: usize, + pub finished: Option, } impl L1BatchUpdates { - pub(crate) fn new() -> Self { + pub(crate) fn new(number: L1BatchNumber) -> Self { Self { + number, executed_transactions: Default::default(), priority_ops_onchain_data: Default::default(), block_execution_metrics: Default::default(), l1_gas_count: new_block_gas_count(), txs_encoding_size: 0, + finished: None, } } @@ -48,7 +53,7 @@ impl L1BatchUpdates { #[cfg(test)] mod tests { use multivm::vm_latest::TransactionVmExt; - use zksync_types::{ProtocolVersionId, H256}; + use zksync_types::{MiniblockNumber, ProtocolVersionId, H256}; use super::*; use crate::{ @@ -58,8 +63,13 @@ mod tests { #[test] fn apply_miniblock_with_empty_tx() { - let mut miniblock_accumulator = - MiniblockUpdates::new(0, 0, H256::zero(), 1, ProtocolVersionId::latest()); + let mut miniblock_accumulator = MiniblockUpdates::new( + 0, + MiniblockNumber(0), + H256::zero(), + 1, + ProtocolVersionId::latest(), + ); let tx = create_transaction(10, 100); let expected_tx_size = tx.bootloader_encoding_size(); @@ -72,7 +82,7 @@ mod tests { vec![], ); - let mut l1_batch_accumulator = L1BatchUpdates::new(); + let mut l1_batch_accumulator = L1BatchUpdates::new(L1BatchNumber(1)); l1_batch_accumulator.extend_from_sealed_miniblock(miniblock_accumulator); assert_eq!(l1_batch_accumulator.executed_transactions.len(), 1); diff --git a/core/lib/zksync_core/src/state_keeper/updates/miniblock_updates.rs b/core/lib/zksync_core/src/state_keeper/updates/miniblock_updates.rs index 4dd561e72aa..db1ab91529c 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/miniblock_updates.rs +++ b/core/lib/zksync_core/src/state_keeper/updates/miniblock_updates.rs @@ -27,7 +27,7 @@ pub struct MiniblockUpdates { pub block_execution_metrics: ExecutionMetrics, pub txs_encoding_size: usize, pub timestamp: u64, - pub number: u32, + pub number: MiniblockNumber, pub prev_block_hash: H256, pub virtual_blocks: u32, pub protocol_version: ProtocolVersionId, @@ -36,7 +36,7 @@ pub struct MiniblockUpdates { impl MiniblockUpdates { pub(crate) fn new( timestamp: u64, - number: u32, + number: MiniblockNumber, prev_block_hash: H256, virtual_blocks: u32, protocol_version: ProtocolVersionId, @@ -148,11 +148,7 @@ impl MiniblockUpdates { /// Calculates miniblock hash based on the protocol version. pub(crate) fn get_miniblock_hash(&self) -> H256 { - let mut digest = MiniblockHasher::new( - MiniblockNumber(self.number), - self.timestamp, - self.prev_block_hash, - ); + let mut digest = MiniblockHasher::new(self.number, self.timestamp, self.prev_block_hash); for tx in &self.executed_transactions { digest.push_tx_hash(tx.hash); } @@ -161,7 +157,7 @@ impl MiniblockUpdates { pub(crate) fn get_miniblock_env(&self) -> L2BlockEnv { L2BlockEnv { - number: self.number, + number: self.number.0, timestamp: self.timestamp, prev_block_hash: self.prev_block_hash, max_virtual_blocks_to_create: self.virtual_blocks, @@ -178,8 +174,13 @@ mod tests { #[test] fn apply_empty_l2_tx() { - let mut accumulator = - MiniblockUpdates::new(0, 0, H256::random(), 0, ProtocolVersionId::latest()); + let mut accumulator = MiniblockUpdates::new( + 0, + MiniblockNumber(0), + H256::random(), + 0, + ProtocolVersionId::latest(), + ); let tx = create_transaction(10, 100); let bootloader_encoding_size = tx.bootloader_encoding_size(); accumulator.extend_from_executed_transaction( diff --git a/core/lib/zksync_core/src/state_keeper/updates/mod.rs b/core/lib/zksync_core/src/state_keeper/updates/mod.rs index 0e839584d8b..d0aabf83a6d 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/updates/mod.rs @@ -1,5 +1,5 @@ use multivm::{ - interface::{L1BatchEnv, SystemEnv, VmExecutionResultAndLogs}, + interface::{FinishedL1Batch, L1BatchEnv, SystemEnv, VmExecutionResultAndLogs}, utils::get_batch_base_fee, }; use zksync_contracts::BaseSystemContractsHashes; @@ -12,8 +12,11 @@ use zksync_types::{ use zksync_utils::bytecode::CompressedBytecodeInfo; pub(crate) use self::{l1_batch_updates::L1BatchUpdates, miniblock_updates::MiniblockUpdates}; -use super::io::MiniblockParams; -use crate::state_keeper::metrics::BATCH_TIP_METRICS; +use super::{ + io::{IoCursor, MiniblockParams}, + metrics::BATCH_TIP_METRICS, +}; +use crate::state_keeper::types::ExecutionMetricsForCriteria; pub mod l1_batch_updates; pub mod miniblock_updates; @@ -24,7 +27,7 @@ pub mod miniblock_updates; /// `L1BatchUpdates` keeps updates for the already sealed mini-blocks of the pending L1 batch. /// `UpdatesManager` manages the state of both of these accumulators to be consistent /// and provides information about the pending state of the current L1 batch. -#[derive(Debug, Clone, PartialEq)] +#[derive(Debug)] pub struct UpdatesManager { batch_timestamp: u64, fee_account_address: Address, @@ -47,10 +50,10 @@ impl UpdatesManager { base_fee_per_gas: get_batch_base_fee(l1_batch_env, protocol_version.into()), protocol_version, base_system_contract_hashes: system_env.base_system_smart_contracts.hashes(), - l1_batch: L1BatchUpdates::new(), + l1_batch: L1BatchUpdates::new(l1_batch_env.number), miniblock: MiniblockUpdates::new( l1_batch_env.first_l2_block.timestamp, - l1_batch_env.first_l2_block.number, + MiniblockNumber(l1_batch_env.first_l2_block.number), l1_batch_env.first_l2_block.prev_block_hash, l1_batch_env.first_l2_block.max_virtual_blocks_to_create, protocol_version, @@ -67,16 +70,22 @@ impl UpdatesManager { self.base_system_contract_hashes } + pub(crate) fn io_cursor(&self) -> IoCursor { + IoCursor { + next_miniblock: self.miniblock.number + 1, + prev_miniblock_hash: self.miniblock.get_miniblock_hash(), + prev_miniblock_timestamp: self.miniblock.timestamp, + l1_batch: self.l1_batch.number, + } + } + pub(crate) fn seal_miniblock_command( &self, - l1_batch_number: L1BatchNumber, - miniblock_number: MiniblockNumber, l2_erc20_bridge_addr: Address, pre_insert_txs: bool, ) -> MiniblockSealCommand { MiniblockSealCommand { - l1_batch_number, - miniblock_number, + l1_batch_number: self.l1_batch.number, miniblock: self.miniblock.clone(), first_tx_index: self.l1_batch.executed_transactions.len(), fee_account_address: self.fee_account_address, @@ -114,21 +123,27 @@ impl UpdatesManager { ); } - pub(crate) fn extend_from_fictive_transaction( - &mut self, - result: VmExecutionResultAndLogs, - l1_gas_count: BlockGasCount, - execution_metrics: ExecutionMetrics, - ) { + pub(crate) fn finish_batch(&mut self, finished_batch: FinishedL1Batch) { + assert!( + self.l1_batch.finished.is_none(), + "Cannot finish already finished batch" + ); + + let result = &finished_batch.block_tip_execution_result; + let batch_tip_metrics = ExecutionMetricsForCriteria::new(None, result); + let before = self.storage_writes_deduplicator.metrics(); self.storage_writes_deduplicator .apply(&result.logs.storage_logs); let after = self.storage_writes_deduplicator.metrics(); - BATCH_TIP_METRICS.observe_writes_metrics(&before, &after, self.protocol_version()); - self.miniblock - .extend_from_fictive_transaction(result, l1_gas_count, execution_metrics); + self.miniblock.extend_from_fictive_transaction( + result.clone(), + batch_tip_metrics.l1_gas, + batch_tip_metrics.execution_metrics, + ); + self.l1_batch.finished = Some(finished_batch); } /// Pushes a new miniblock with the specified timestamp into this manager. The previously @@ -167,7 +182,6 @@ impl UpdatesManager { #[derive(Debug)] pub(crate) struct MiniblockSealCommand { pub l1_batch_number: L1BatchNumber, - pub miniblock_number: MiniblockNumber, pub miniblock: MiniblockUpdates, pub first_tx_index: usize, pub fee_account_address: Address, diff --git a/core/lib/zksync_core/src/sync_layer/external_io.rs b/core/lib/zksync_core/src/sync_layer/external_io.rs index 54d1c8325d9..a8cb8fac015 100644 --- a/core/lib/zksync_core/src/sync_layer/external_io.rs +++ b/core/lib/zksync_core/src/sync_layer/external_io.rs @@ -2,13 +2,11 @@ use std::{collections::HashMap, time::Duration}; use anyhow::Context as _; use async_trait::async_trait; -use multivm::interface::{FinishedL1Batch, L1BatchEnv, SystemEnv}; -use vm_utils::storage::{l1_batch_params, L1BatchParamsProvider}; -use zksync_contracts::{BaseSystemContracts, SystemContractCode}; +use vm_utils::storage::L1BatchParamsProvider; +use zksync_contracts::{BaseSystemContracts, BaseSystemContractsHashes, SystemContractCode}; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_types::{ - ethabi::Address, fee_model::BatchFeeInput, protocol_upgrade::ProtocolUpgradeTx, - witness_block_state::WitnessBlockState, L1BatchNumber, L2ChainId, MiniblockNumber, + protocol_upgrade::ProtocolUpgradeTx, L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, Transaction, H256, }; use zksync_utils::bytes_to_be_words; @@ -16,20 +14,15 @@ use zksync_utils::bytes_to_be_words; use super::{ client::MainNodeClient, sync_action::{ActionQueue, SyncAction}, - SyncState, }; -use crate::{ - metrics::{BlockStage, APP_METRICS}, - state_keeper::{ - io::{ - common::{load_pending_batch, poll_iters, IoCursor}, - fee_address_migration, MiniblockParams, MiniblockSealerHandle, PendingBatchData, - StateKeeperIO, - }, - metrics::KEEPER_METRICS, - seal_criteria::IoSealCriteria, - updates::{MiniblockUpdates, UpdatesManager}, +use crate::state_keeper::{ + io::{ + common::{load_pending_batch, poll_iters, IoCursor}, + fee_address_migration, L1BatchParams, MiniblockParams, PendingBatchData, StateKeeperIO, }, + metrics::KEEPER_METRICS, + seal_criteria::IoSealCriteria, + updates::UpdatesManager, }; /// The interval between the action queue polling attempts for the new actions. @@ -43,40 +36,21 @@ const POLL_INTERVAL: Duration = Duration::from_millis(100); /// to the one in the mempool IO (which is used in the main node). #[derive(Debug)] pub struct ExternalIO { - miniblock_sealer_handle: MiniblockSealerHandle, pool: ConnectionPool, - - current_l1_batch_number: L1BatchNumber, - current_miniblock_number: MiniblockNumber, - prev_miniblock_hash: H256, l1_batch_params_provider: L1BatchParamsProvider, actions: ActionQueue, - sync_state: SyncState, main_node_client: Box, - - /// Required to extract newly added tokens. - l2_erc20_bridge_addr: Address, - // TODO it's required for system env, probably we have to get rid of getting system env - validation_computational_gas_limit: u32, chain_id: L2ChainId, } impl ExternalIO { - #[allow(clippy::too_many_arguments)] pub async fn new( - miniblock_sealer_handle: MiniblockSealerHandle, pool: ConnectionPool, actions: ActionQueue, - sync_state: SyncState, main_node_client: Box, - l2_erc20_bridge_addr: Address, - validation_computational_gas_limit: u32, chain_id: L2ChainId, ) -> anyhow::Result { let mut storage = pool.connection_tagged("sync_layer").await?; - let cursor = IoCursor::new(&mut storage) - .await - .context("failed initializing I/O cursor")?; let l1_batch_params_provider = L1BatchParamsProvider::new(&mut storage) .await .context("failed initializing L1 batch params provider")?; @@ -85,123 +59,20 @@ impl ExternalIO { fee_address_migration::migrate_pending_miniblocks(&mut storage).await?; drop(storage); - tracing::info!( - "Initialized the ExternalIO: current L1 batch number {}, current miniblock number {}", - cursor.l1_batch, - cursor.next_miniblock, - ); - - sync_state.set_local_block(MiniblockNumber(cursor.next_miniblock.saturating_sub(1))); - Ok(Self { - miniblock_sealer_handle, pool, - current_l1_batch_number: cursor.l1_batch, - current_miniblock_number: cursor.next_miniblock, - prev_miniblock_hash: cursor.prev_miniblock_hash, l1_batch_params_provider, actions, - sync_state, main_node_client, - l2_erc20_bridge_addr, - validation_computational_gas_limit, chain_id, }) } - fn update_miniblock_fields(&mut self, miniblock: &MiniblockUpdates) { - assert_eq!( - miniblock.number, self.current_miniblock_number.0, - "Attempted to seal a miniblock with unexpected number" - ); - // Mimic the metric emitted by the main node to reuse existing Grafana charts. - APP_METRICS.block_number[&BlockStage::Sealed].set(self.current_l1_batch_number.0.into()); - self.sync_state - .set_local_block(self.current_miniblock_number); - self.current_miniblock_number += 1; - self.prev_miniblock_hash = miniblock.get_miniblock_hash(); - } - - async fn wait_for_previous_l1_batch_hash(&self) -> anyhow::Result { - tracing::info!( - "Getting previous L1 batch hash for L1 batch #{}", - self.current_l1_batch_number - ); - let mut storage = self.pool.connection_tagged("sync_layer").await?; - let wait_latency = KEEPER_METRICS.wait_for_prev_hash_time.start(); - let prev_l1_batch_number = self.current_l1_batch_number - 1; - let (hash, _) = self - .l1_batch_params_provider - .wait_for_l1_batch_params(&mut storage, prev_l1_batch_number) - .await - .with_context(|| { - format!("error waiting for params for L1 batch #{prev_l1_batch_number}") - })?; - wait_latency.observe(); - Ok(hash) - } - - async fn load_base_system_contracts_by_version_id( + async fn get_base_system_contract( &self, - id: ProtocolVersionId, - ) -> anyhow::Result { - let base_system_contracts = self - .pool - .connection_tagged("sync_layer") - .await? - .protocol_versions_dal() - .load_base_system_contracts_by_version_id(id as u16) - .await - .context("failed loading base system contracts")?; - - Ok(match base_system_contracts { - Some(version) => version, - None => { - tracing::info!("Fetching protocol version {id:?} from the main node"); - - let protocol_version = self - .main_node_client - .fetch_protocol_version(id) - .await - .context("failed to fetch protocol version from the main node")? - .context("protocol version is missing on the main node")?; - self.pool - .connection_tagged("sync_layer") - .await? - .protocol_versions_dal() - .save_protocol_version( - protocol_version - .version_id - .try_into() - .context("cannot convert protocol version")?, - protocol_version.timestamp, - protocol_version.verification_keys_hashes, - protocol_version.base_system_contracts, - protocol_version.l2_system_upgrade_tx_hash, - ) - .await; - - let bootloader = self - .get_base_system_contract(protocol_version.base_system_contracts.bootloader) - .await - .with_context(|| { - format!("cannot fetch bootloader code for {protocol_version:?}") - })?; - let default_aa = self - .get_base_system_contract(protocol_version.base_system_contracts.default_aa) - .await - .with_context(|| { - format!("cannot fetch default AA code for {protocol_version:?}") - })?; - BaseSystemContracts { - bootloader, - default_aa, - } - } - }) - } - - async fn get_base_system_contract(&self, hash: H256) -> anyhow::Result { + hash: H256, + current_miniblock_number: MiniblockNumber, + ) -> anyhow::Result { let bytecode = self .pool .connection_tagged("sync_layer") @@ -232,7 +103,7 @@ impl ExternalIO { .await? .factory_deps_dal() .insert_factory_deps( - self.current_miniblock_number, + current_miniblock_number, &HashMap::from([(hash, contract_bytecode.clone())]), ) .await @@ -248,42 +119,49 @@ impl ExternalIO { impl IoSealCriteria for ExternalIO { fn should_seal_l1_batch_unconditionally(&mut self, _manager: &UpdatesManager) -> bool { - matches!( - self.actions.peek_action(), - Some(SyncAction::SealBatch { .. }) - ) + if !matches!(self.actions.peek_action(), Some(SyncAction::SealBatch)) { + return false; + } + self.actions.pop_action(); + true } fn should_seal_miniblock(&mut self, _manager: &UpdatesManager) -> bool { - matches!(self.actions.peek_action(), Some(SyncAction::SealMiniblock)) + if !matches!(self.actions.peek_action(), Some(SyncAction::SealMiniblock)) { + return false; + } + self.actions.pop_action(); + true } } #[async_trait] impl StateKeeperIO for ExternalIO { - fn current_l1_batch_number(&self) -> L1BatchNumber { - self.current_l1_batch_number + fn chain_id(&self) -> L2ChainId { + self.chain_id } - fn current_miniblock_number(&self) -> MiniblockNumber { - self.current_miniblock_number - } - - async fn load_pending_batch(&mut self) -> anyhow::Result> { + async fn initialize(&mut self) -> anyhow::Result<(IoCursor, Option)> { let mut storage = self.pool.connection_tagged("sync_layer").await?; + let cursor = IoCursor::new(&mut storage).await?; + tracing::info!( + "Initialized the ExternalIO: current L1 batch number {}, current miniblock number {}", + cursor.l1_batch, + cursor.next_miniblock, + ); let pending_miniblock_header = self .l1_batch_params_provider - .load_first_miniblock_in_batch(&mut storage, self.current_l1_batch_number) + .load_first_miniblock_in_batch(&mut storage, cursor.l1_batch) .await .with_context(|| { format!( "failed loading first miniblock for L1 batch #{}", - self.current_l1_batch_number + cursor.l1_batch ) })?; let Some(mut pending_miniblock_header) = pending_miniblock_header else { - return Ok(None); + return Ok((cursor, None)); }; if !pending_miniblock_header.has_protocol_version() { @@ -299,11 +177,13 @@ impl StateKeeperIO for ExternalIO { })?; // Loading base system contracts will insert protocol version in the database if it's not present there. let protocol_version = sync_block.protocol_version; - self.load_base_system_contracts_by_version_id(protocol_version) + drop(storage); + self.load_base_system_contracts(protocol_version, &cursor) .await .with_context(|| { format!("cannot load base system contracts for {protocol_version:?}") })?; + storage = self.pool.connection_tagged("sync_layer").await?; storage .blocks_dal() .set_protocol_version_for_pending_miniblocks(protocol_version) @@ -317,14 +197,14 @@ impl StateKeeperIO for ExternalIO { .load_l1_batch_params( &mut storage, &pending_miniblock_header, - self.validation_computational_gas_limit, + super::VALIDATION_COMPUTATIONAL_GAS_LIMIT, self.chain_id, ) .await .with_context(|| { format!( "failed loading parameters for pending L1 batch #{}", - self.current_l1_batch_number + cursor.l1_batch ) })?; let data = load_pending_batch(&mut storage, system_env, l1_batch_env) @@ -332,65 +212,36 @@ impl StateKeeperIO for ExternalIO { .with_context(|| { format!( "failed loading data for re-execution for pending L1 batch #{}", - self.current_l1_batch_number + cursor.l1_batch ) })?; - Ok(Some(data)) + Ok((cursor, Some(data))) } async fn wait_for_new_batch_params( &mut self, + cursor: &IoCursor, max_wait: Duration, - ) -> anyhow::Result> { + ) -> anyhow::Result> { tracing::debug!("Waiting for the new batch params"); for _ in 0..poll_iters(POLL_INTERVAL, max_wait) { match self.actions.pop_action() { Some(SyncAction::OpenBatch { + params, number, - timestamp, - l1_gas_price, - l2_fair_gas_price, - fair_pubdata_price, - operator_address, - protocol_version, - first_miniblock_info: (miniblock_number, virtual_blocks), + first_miniblock_number, }) => { anyhow::ensure!( - number == self.current_l1_batch_number, + number == cursor.l1_batch, "Batch number mismatch: expected {}, got {number}", - self.current_l1_batch_number + cursor.l1_batch ); - let previous_l1_batch_hash = self.wait_for_previous_l1_batch_hash().await?; - tracing::info!( - "Previous L1 batch hash: {previous_l1_batch_hash:?}, previous miniblock hash: {:?}", - self.prev_miniblock_hash + anyhow::ensure!( + first_miniblock_number == cursor.next_miniblock, + "Miniblock number mismatch: expected {}, got {first_miniblock_number}", + cursor.next_miniblock ); - - let base_system_contracts = self - .load_base_system_contracts_by_version_id(protocol_version) - .await - .with_context(|| { - format!("cannot load base system contracts for {protocol_version:?}") - })?; - return Ok(Some(l1_batch_params( - number, - operator_address, - timestamp, - previous_l1_batch_hash, - BatchFeeInput::for_protocol_version( - protocol_version, - l2_fair_gas_price, - fair_pubdata_price, - l1_gas_price, - ), - miniblock_number, - self.prev_miniblock_hash, - base_system_contracts, - self.validation_computational_gas_limit, - protocol_version, - virtual_blocks, - self.chain_id, - ))); + return Ok(Some(params)); } Some(other) => { anyhow::bail!("unexpected action in the action queue: {other:?}"); @@ -405,37 +256,20 @@ impl StateKeeperIO for ExternalIO { async fn wait_for_new_miniblock_params( &mut self, + cursor: &IoCursor, max_wait: Duration, ) -> anyhow::Result> { // Wait for the next miniblock to appear in the queue. let actions = &mut self.actions; for _ in 0..poll_iters(POLL_INTERVAL, max_wait) { - match actions.peek_action() { - Some(SyncAction::Miniblock { - number, - timestamp, - virtual_blocks, - }) => { - self.actions.pop_action(); // We found the miniblock, remove it from the queue. + match actions.pop_action() { + Some(SyncAction::Miniblock { params, number }) => { anyhow::ensure!( - number == self.current_miniblock_number, + number == cursor.next_miniblock, "Miniblock number mismatch: expected {}, got {number}", - self.current_miniblock_number + cursor.next_miniblock ); - return Ok(Some(MiniblockParams { - timestamp, - virtual_blocks, - })); - } - Some(SyncAction::SealBatch { virtual_blocks, .. }) => { - // We've reached the next batch, so this situation would be handled by the batch sealer. - // No need to pop the action from the queue. - // It also doesn't matter which timestamp we return, since there will be no more miniblocks in this - // batch. We return 0 to make it easy to detect if it ever appears somewhere. - return Ok(Some(MiniblockParams { - timestamp: 0, - virtual_blocks, - })); + return Ok(Some(params)); } Some(other) => { anyhow::bail!( @@ -450,35 +284,43 @@ impl StateKeeperIO for ExternalIO { Ok(None) } - async fn wait_for_next_tx(&mut self, max_wait: Duration) -> Option { + async fn wait_for_next_tx( + &mut self, + max_wait: Duration, + ) -> anyhow::Result> { let actions = &mut self.actions; tracing::debug!( "Waiting for the new tx, next action is {:?}", actions.peek_action() ); for _ in 0..poll_iters(POLL_INTERVAL, max_wait) { - // We keep polling until we get any item from the queue. - // Once we have the item, it'll be either a transaction, or a seal request. - // Whatever item it is, we don't have to poll anymore and may exit, thus double option use. match actions.peek_action() { Some(SyncAction::Tx(_)) => { let SyncAction::Tx(tx) = actions.pop_action().unwrap() else { unreachable!() }; - return Some(Transaction::from(*tx)); + return Ok(Some(Transaction::from(*tx))); + } + Some(SyncAction::SealMiniblock | SyncAction::SealBatch) => { + // No more transactions in the current miniblock; the state keeper should seal it. + return Ok(None); + } + Some(other) => { + anyhow::bail!( + "Unexpected action in the queue while waiting for the next transaction: {other:?}" + ); } _ => { tokio::time::sleep(POLL_INTERVAL).await; - continue; } } } - None + Ok(None) } - async fn rollback(&mut self, tx: Transaction) { + async fn rollback(&mut self, tx: Transaction) -> anyhow::Result<()> { // We are replaying the already sealed batches so no rollbacks are expected to occur. - panic!("Rollback requested. Transaction hash: {:?}", tx.hash()); + anyhow::bail!("Rollback requested. Transaction hash: {:?}", tx.hash()); } async fn reject(&mut self, tx: &Transaction, error: &str) -> anyhow::Result<()> { @@ -490,70 +332,75 @@ impl StateKeeperIO for ExternalIO { ); } - async fn seal_miniblock(&mut self, updates_manager: &UpdatesManager) { - let action = self.actions.pop_action(); - let Some(SyncAction::SealMiniblock) = action else { - panic!("State keeper requested to seal miniblock, but the next action is {action:?}"); - }; - - // Now transactions are stored, and we may mark them as executed. - let command = updates_manager.seal_miniblock_command( - self.current_l1_batch_number, - self.current_miniblock_number, - self.l2_erc20_bridge_addr, - true, - ); - self.miniblock_sealer_handle.submit(command).await; - tracing::info!("Miniblock {} is sealed", self.current_miniblock_number); - self.update_miniblock_fields(&updates_manager.miniblock); - } - - async fn seal_l1_batch( + async fn load_base_system_contracts( &mut self, - // needed as part of the interface, to be removed once we transition to Merkle Paths - _witness_block_state: Option, - updates_manager: UpdatesManager, - l1_batch_env: &L1BatchEnv, - finished_batch: FinishedL1Batch, - ) -> anyhow::Result<()> { - let action = self.actions.pop_action(); - let Some(SyncAction::SealBatch { .. }) = action else { - anyhow::bail!( - "State keeper requested to seal the batch, but the next action is {action:?}" - ); - }; + protocol_version: ProtocolVersionId, + cursor: &IoCursor, + ) -> anyhow::Result { + let base_system_contracts = self + .pool + .connection_tagged("sync_layer") + .await? + .protocol_versions_dal() + .load_base_system_contracts_by_version_id(protocol_version as u16) + .await + .context("failed loading base system contracts")?; - // We cannot start sealing an L1 batch until we've sealed all miniblocks included in it. - self.miniblock_sealer_handle.wait_for_all_commands().await; + if let Some(contracts) = base_system_contracts { + return Ok(contracts); + } + tracing::info!("Fetching protocol version {protocol_version:?} from the main node"); - let mut storage = self.pool.connection_tagged("sync_layer").await?; - let fictive_miniblock = updates_manager - .seal_l1_batch( - &mut storage, - self.current_miniblock_number, - l1_batch_env, - finished_batch, - self.l2_erc20_bridge_addr, + let protocol_version = self + .main_node_client + .fetch_protocol_version(protocol_version) + .await + .context("failed to fetch protocol version from the main node")? + .context("protocol version is missing on the main node")?; + self.pool + .connection_tagged("sync_layer") + .await? + .protocol_versions_dal() + .save_protocol_version( + protocol_version + .version_id + .try_into() + .context("cannot convert protocol version")?, + protocol_version.timestamp, + protocol_version.verification_keys_hashes, + protocol_version.base_system_contracts, + protocol_version.l2_system_upgrade_tx_hash, ) .await; - drop(storage); - self.update_miniblock_fields(&fictive_miniblock); - tracing::info!("Batch {} is sealed", self.current_l1_batch_number); - self.current_l1_batch_number += 1; - Ok(()) + let BaseSystemContractsHashes { + bootloader, + default_aa, + } = protocol_version.base_system_contracts; + let bootloader = self + .get_base_system_contract(bootloader, cursor.next_miniblock) + .await + .with_context(|| format!("cannot fetch bootloader code for {protocol_version:?}"))?; + let default_aa = self + .get_base_system_contract(default_aa, cursor.next_miniblock) + .await + .with_context(|| format!("cannot fetch default AA code for {protocol_version:?}"))?; + Ok(BaseSystemContracts { + bootloader, + default_aa, + }) } - async fn load_previous_batch_version_id(&mut self) -> anyhow::Result { + async fn load_batch_version_id( + &mut self, + number: L1BatchNumber, + ) -> anyhow::Result { let mut storage = self.pool.connection_tagged("sync_layer").await?; - let prev_l1_batch_number = self.current_l1_batch_number - 1; self.l1_batch_params_provider - .load_l1_batch_protocol_version(&mut storage, prev_l1_batch_number) + .load_l1_batch_protocol_version(&mut storage, number) .await - .with_context(|| { - format!("failed loading protocol version for L1 batch #{prev_l1_batch_number}") - })? - .with_context(|| format!("L1 batch #{prev_l1_batch_number} misses protocol version")) + .with_context(|| format!("failed loading protocol version for L1 batch #{number}"))? + .with_context(|| format!("L1 batch #{number} misses protocol version")) } async fn load_upgrade_tx( @@ -563,4 +410,20 @@ impl StateKeeperIO for ExternalIO { // External node will fetch upgrade tx from the main node Ok(None) } + + async fn load_batch_state_hash( + &mut self, + l1_batch_number: L1BatchNumber, + ) -> anyhow::Result { + tracing::info!("Getting L1 batch hash for L1 batch #{l1_batch_number}"); + let mut storage = self.pool.connection_tagged("sync_layer").await?; + let wait_latency = KEEPER_METRICS.wait_for_prev_hash_time.start(); + let (hash, _) = self + .l1_batch_params_provider + .wait_for_l1_batch_params(&mut storage, l1_batch_number) + .await + .with_context(|| format!("error waiting for params for L1 batch #{l1_batch_number}"))?; + wait_latency.observe(); + Ok(hash) + } } diff --git a/core/lib/zksync_core/src/sync_layer/fetcher.rs b/core/lib/zksync_core/src/sync_layer/fetcher.rs index 455912006d0..f7837e6c1cc 100644 --- a/core/lib/zksync_core/src/sync_layer/fetcher.rs +++ b/core/lib/zksync_core/src/sync_layer/fetcher.rs @@ -1,8 +1,8 @@ use anyhow::Context as _; use zksync_dal::{Connection, Core, CoreDal}; use zksync_types::{ - api::en::SyncBlock, block::MiniblockHasher, helpers::unix_timestamp_ms, Address, L1BatchNumber, - MiniblockNumber, ProtocolVersionId, H256, + api::en::SyncBlock, block::MiniblockHasher, fee_model::BatchFeeInput, + helpers::unix_timestamp_ms, Address, L1BatchNumber, MiniblockNumber, ProtocolVersionId, H256, }; use super::{ @@ -11,7 +11,7 @@ use super::{ }; use crate::{ metrics::{TxStage, APP_METRICS}, - state_keeper::io::common::IoCursor, + state_keeper::io::{common::IoCursor, L1BatchParams, MiniblockParams}, }; /// Same as [`zksync_types::Transaction`], just with additional guarantees that the "received at" timestamp was set locally. @@ -147,15 +147,23 @@ impl IoCursor { ); new_actions.push(SyncAction::OpenBatch { + params: L1BatchParams { + protocol_version: block.protocol_version, + validation_computational_gas_limit: super::VALIDATION_COMPUTATIONAL_GAS_LIMIT, + operator_address: block.operator_address, + fee_input: BatchFeeInput::for_protocol_version( + block.protocol_version, + block.l2_fair_gas_price, + block.fair_pubdata_price, + block.l1_gas_price, + ), + first_miniblock: MiniblockParams { + timestamp: block.timestamp, + virtual_blocks: block.virtual_blocks, + }, + }, number: block.l1_batch_number, - timestamp: block.timestamp, - l1_gas_price: block.l1_gas_price, - l2_fair_gas_price: block.l2_fair_gas_price, - fair_pubdata_price: block.fair_pubdata_price, - operator_address: block.operator_address, - protocol_version: block.protocol_version, - // `block.virtual_blocks` can be `None` only for old VM versions where it's not used, so it's fine to provide any number. - first_miniblock_info: (block.number, block.virtual_blocks), + first_miniblock_number: block.number, }); FETCHER_METRICS.l1_batch[&L1BatchStage::Open].set(block.l1_batch_number.0.into()); self.l1_batch += 1; @@ -163,10 +171,11 @@ impl IoCursor { // New batch implicitly means a new miniblock, so we only need to push the miniblock action // if it's not a new batch. new_actions.push(SyncAction::Miniblock { + params: MiniblockParams { + timestamp: block.timestamp, + virtual_blocks: block.virtual_blocks, + }, number: block.number, - timestamp: block.timestamp, - // `block.virtual_blocks` can be `None` only for old VM versions where it's not used, so it's fine to provide any number. - virtual_blocks: block.virtual_blocks, }); FETCHER_METRICS.miniblock.set(block.number.0.into()); } @@ -178,10 +187,7 @@ impl IoCursor { // Last miniblock of the batch is a "fictive" miniblock and would be replicated locally. // We don't need to seal it explicitly, so we only put the seal miniblock command if it's not the last miniblock. if block.last_in_batch { - new_actions.push(SyncAction::SealBatch { - // `block.virtual_blocks` can be `None` only for old VM versions where it's not used, so it's fine to provide any number. - virtual_blocks: block.virtual_blocks, - }); + new_actions.push(SyncAction::SealBatch); } else { new_actions.push(SyncAction::SealMiniblock); } diff --git a/core/lib/zksync_core/src/sync_layer/mod.rs b/core/lib/zksync_core/src/sync_layer/mod.rs index e216ef4f8c5..c5ebd52ea43 100644 --- a/core/lib/zksync_core/src/sync_layer/mod.rs +++ b/core/lib/zksync_core/src/sync_layer/mod.rs @@ -13,3 +13,10 @@ pub use self::{ client::MainNodeClient, external_io::ExternalIO, sync_action::ActionQueue, sync_state::SyncState, }; + +/// Validation gas limit used by the external node. +// This config value is used on the main node, and depending on these values certain transactions can +// be *rejected* (that is, not included into the block). However, external node only mirrors what the main +// node has already executed, so we can safely set this value to the maximum possible values – if the main +// node has already executed the transaction, then the external node must execute it too. +const VALIDATION_COMPUTATIONAL_GAS_LIMIT: u32 = u32::MAX; diff --git a/core/lib/zksync_core/src/sync_layer/sync_action.rs b/core/lib/zksync_core/src/sync_layer/sync_action.rs index c52a400b9ab..7b009872f7a 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_action.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_action.rs @@ -1,7 +1,8 @@ use tokio::sync::mpsc; -use zksync_types::{Address, L1BatchNumber, MiniblockNumber, ProtocolVersionId}; +use zksync_types::{L1BatchNumber, MiniblockNumber}; use super::{fetcher::FetchedTransaction, metrics::QUEUE_METRICS}; +use crate::state_keeper::io::{L1BatchParams, MiniblockParams}; #[derive(Debug)] pub struct ActionQueueSender(mpsc::Sender); @@ -47,7 +48,7 @@ impl ActionQueueSender { return Err(format!("Unexpected Tx: {:?}", actions)); } } - SyncAction::SealMiniblock | SyncAction::SealBatch { .. } => { + SyncAction::SealMiniblock | SyncAction::SealBatch => { if !opened || miniblock_sealed { return Err(format!("Unexpected SealMiniblock/SealBatch: {:?}", actions)); } @@ -110,20 +111,15 @@ impl ActionQueue { #[derive(Debug, Clone)] pub(crate) enum SyncAction { OpenBatch { + params: L1BatchParams, + // Additional parameters used only for sanity checks number: L1BatchNumber, - timestamp: u64, - l1_gas_price: u64, - l2_fair_gas_price: u64, - fair_pubdata_price: Option, - operator_address: Address, - protocol_version: ProtocolVersionId, - // Miniblock number and virtual blocks count. - first_miniblock_info: (MiniblockNumber, u32), + first_miniblock_number: MiniblockNumber, }, Miniblock { + params: MiniblockParams, + // Additional parameters used only for sanity checks number: MiniblockNumber, - timestamp: u64, - virtual_blocks: u32, }, Tx(Box), /// We need an explicit action for the miniblock sealing, since we fetch the whole miniblocks and already know @@ -132,10 +128,7 @@ pub(crate) enum SyncAction { /// the next one is sealed on the main node. SealMiniblock, /// Similarly to `SealMiniblock` we must be able to seal the batch even if there is no next miniblock yet. - SealBatch { - /// Virtual blocks count for the fictive miniblock. - virtual_blocks: u32, - }, + SealBatch, } impl From for SyncAction { @@ -146,28 +139,34 @@ impl From for SyncAction { #[cfg(test)] mod tests { - use zksync_types::{l2::L2Tx, H256}; + use zksync_types::{fee_model::BatchFeeInput, l2::L2Tx, Address, ProtocolVersionId, H256}; use super::*; fn open_batch() -> SyncAction { SyncAction::OpenBatch { - number: 1.into(), - timestamp: 1, - l1_gas_price: 1, - l2_fair_gas_price: 1, - fair_pubdata_price: Some(1), - operator_address: Default::default(), - protocol_version: ProtocolVersionId::latest(), - first_miniblock_info: (1.into(), 1), + params: L1BatchParams { + protocol_version: ProtocolVersionId::latest(), + validation_computational_gas_limit: u32::MAX, + operator_address: Address::default(), + fee_input: BatchFeeInput::default(), + first_miniblock: MiniblockParams { + timestamp: 1, + virtual_blocks: 1, + }, + }, + number: L1BatchNumber(1), + first_miniblock_number: MiniblockNumber(1), } } fn miniblock() -> SyncAction { SyncAction::Miniblock { + params: MiniblockParams { + timestamp: 1, + virtual_blocks: 1, + }, number: 1.into(), - timestamp: 1, - virtual_blocks: 1, } } @@ -192,7 +191,7 @@ mod tests { } fn seal_batch() -> SyncAction { - SyncAction::SealBatch { virtual_blocks: 1 } + SyncAction::SealBatch } #[test] diff --git a/core/lib/zksync_core/src/sync_layer/sync_state.rs b/core/lib/zksync_core/src/sync_layer/sync_state.rs index 74c191c1ef0..03f8b4e9ca1 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_state.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_state.rs @@ -4,9 +4,12 @@ use async_trait::async_trait; use serde::Serialize; use zksync_concurrency::{ctx, sync}; use zksync_health_check::{CheckHealth, Health, HealthStatus}; -use zksync_types::MiniblockNumber; +use zksync_types::{witness_block_state::WitnessBlockState, MiniblockNumber}; -use crate::metrics::EN_METRICS; +use crate::{ + metrics::EN_METRICS, + state_keeper::{io::IoCursor, updates::UpdatesManager, StateKeeperOutputHandler}, +}; /// `SyncState` is a structure that holds the state of the syncing process. /// The intended use case is to signalize to Web3 API whether the node is fully synced. @@ -36,6 +39,15 @@ impl SyncState { self.0.borrow().local_block.unwrap_or_default() } + #[cfg(test)] + pub(crate) async fn wait_for_local_block(&self, want: MiniblockNumber) { + self.0 + .subscribe() + .wait_for(|inner| matches!(inner.local_block, Some(got) if got >= want)) + .await + .unwrap(); + } + pub(crate) async fn wait_for_main_node_block( &self, ctx: &ctx::Ctx, @@ -44,7 +56,7 @@ impl SyncState { sync::wait_for( ctx, &mut self.0.subscribe(), - |s| matches!(s.main_node_block, Some(got) if got >= want), + |inner| matches!(inner.main_node_block, Some(got) if got >= want), ) .await?; Ok(()) @@ -54,7 +66,7 @@ impl SyncState { self.0.send_modify(|inner| inner.set_main_node_block(block)); } - pub(super) fn set_local_block(&self, block: MiniblockNumber) { + fn set_local_block(&self, block: MiniblockNumber) { self.0.send_modify(|inner| inner.set_local_block(block)); } @@ -63,6 +75,31 @@ impl SyncState { } } +#[async_trait] +impl StateKeeperOutputHandler for SyncState { + async fn initialize(&mut self, cursor: &IoCursor) -> anyhow::Result<()> { + let sealed_block_number = cursor.next_miniblock.saturating_sub(1); + self.set_local_block(MiniblockNumber(sealed_block_number)); + Ok(()) + } + + async fn handle_miniblock(&mut self, updates_manager: &UpdatesManager) -> anyhow::Result<()> { + let sealed_block_number = updates_manager.miniblock.number; + self.set_local_block(sealed_block_number); + Ok(()) + } + + async fn handle_l1_batch( + &mut self, + _witness_block_state: Option<&WitnessBlockState>, + updates_manager: &UpdatesManager, + ) -> anyhow::Result<()> { + let sealed_block_number = updates_manager.miniblock.number; + self.set_local_block(sealed_block_number); + Ok(()) + } +} + #[derive(Clone, Debug, Default)] pub(crate) struct SyncStateInner { pub(crate) main_node_block: Option, diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 55f3d74a45f..7fe4da79188 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -1,10 +1,6 @@ //! High-level sync layer tests. -use std::{ - iter, - sync::Arc, - time::{Duration, Instant}, -}; +use std::{iter, sync::Arc, time::Duration}; use test_casing::test_casing; use tokio::{sync::watch, task::JoinHandle}; @@ -23,8 +19,10 @@ use crate::{ consensus::testonly::MockMainNodeClient, genesis::{insert_genesis_batch, GenesisParams}, state_keeper::{ - seal_criteria::NoopSealer, tests::TestBatchExecutorBuilder, MiniblockSealer, - ZkSyncStateKeeper, + io::{L1BatchParams, MiniblockParams}, + seal_criteria::NoopSealer, + tests::TestBatchExecutorBuilder, + OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, }, utils::testonly::{create_l1_batch_metadata, create_l2_transaction, prepare_recovery_snapshot}, }; @@ -35,14 +33,18 @@ pub(crate) const OPERATOR_ADDRESS: Address = Address::repeat_byte(1); fn open_l1_batch(number: u32, timestamp: u64, first_miniblock_number: u32) -> SyncAction { SyncAction::OpenBatch { + params: L1BatchParams { + protocol_version: ProtocolVersionId::latest(), + validation_computational_gas_limit: u32::MAX, + operator_address: OPERATOR_ADDRESS, + fee_input: BatchFeeInput::pubdata_independent(2, 3, 4), + first_miniblock: MiniblockParams { + timestamp, + virtual_blocks: 1, + }, + }, number: L1BatchNumber(number), - timestamp, - l1_gas_price: 2, - l2_fair_gas_price: 3, - fair_pubdata_price: Some(4), - operator_address: OPERATOR_ADDRESS, - protocol_version: ProtocolVersionId::latest(), - first_miniblock_info: (MiniblockNumber(first_miniblock_number), 1), + first_miniblock_number: MiniblockNumber(first_miniblock_number), } } @@ -65,16 +67,16 @@ impl StateKeeperHandles { assert!(tx_hashes.iter().all(|tx_hashes| !tx_hashes.is_empty())); let sync_state = SyncState::default(); - let (miniblock_sealer, miniblock_sealer_handle) = MiniblockSealer::new(pool.clone(), 5); + let (persistence, miniblock_sealer) = + StateKeeperPersistence::new(pool.clone(), Address::repeat_byte(1), 5); + let output_handler = OutputHandler::new(Box::new(persistence.with_tx_insertion())) + .with_handler(Box::new(sync_state.clone())); + tokio::spawn(miniblock_sealer.run()); let io = ExternalIO::new( - miniblock_sealer_handle, pool, actions, - sync_state.clone(), Box::new(main_node_client), - Address::repeat_byte(1), - u32::MAX, L2ChainId::default(), ) .await @@ -90,8 +92,10 @@ impl StateKeeperHandles { stop_receiver, Box::new(io), Box::new(batch_executor_base), + output_handler, Arc::new(NoopSealer), ); + Self { stop_sender, sync_state, @@ -100,28 +104,23 @@ impl StateKeeperHandles { } /// Waits for the given condition. - pub async fn wait(self, mut condition: impl FnMut(&SyncState) -> bool) { - let started_at = Instant::now(); - loop { - assert!( - started_at.elapsed() <= TEST_TIMEOUT, - "Timed out waiting for miniblock to be sealed" - ); - if self.task.is_finished() { - match self.task.await { - Err(err) => panic!("State keeper panicked: {}", err), - Ok(Err(err)) => panic!("State keeper finished with an error: {}", err), + pub async fn wait_for_local_block(mut self, want: MiniblockNumber) { + tokio::select! { + task_result = &mut self.task => { + match task_result { + Err(err) => panic!("State keeper panicked: {err}"), + Ok(Err(err)) => panic!("State keeper finished with an error: {err:?}"), Ok(Ok(())) => unreachable!(), } } - if condition(&self.sync_state) { - break; + () = tokio::time::sleep(TEST_TIMEOUT) => { + panic!("Timed out waiting for miniblock to be sealed"); + } + () = self.sync_state.wait_for_local_block(want) => { + self.stop_sender.send_replace(true); + self.task.await.unwrap().unwrap(); } - tokio::time::sleep(POLL_INTERVAL).await; } - - self.stop_sender.send_replace(true); - self.task.await.unwrap().unwrap(); } } @@ -194,7 +193,7 @@ async fn external_io_basics(snapshot_recovery: bool) { actions_sender.push_actions(actions).await; // Wait until the miniblock is sealed. state_keeper - .wait(|state| state.get_local_block() == snapshot.miniblock_number + 1) + .wait_for_local_block(snapshot.miniblock_number + 1) .await; // Check that the miniblock is persisted. @@ -249,11 +248,8 @@ async fn external_io_works_without_local_protocol_version(snapshot_recovery: boo snapshot.miniblock_timestamp + 1, snapshot.miniblock_number.0 + 1, ); - if let SyncAction::OpenBatch { - protocol_version, .. - } = &mut open_l1_batch - { - *protocol_version = ProtocolVersionId::next(); + if let SyncAction::OpenBatch { params, .. } = &mut open_l1_batch { + params.protocol_version = ProtocolVersionId::next(); } else { unreachable!(); }; @@ -285,7 +281,7 @@ async fn external_io_works_without_local_protocol_version(snapshot_recovery: boo actions_sender.push_actions(actions).await; // Wait until the miniblock is sealed. state_keeper - .wait(|state| state.get_local_block() == snapshot.miniblock_number + 1) + .wait_for_local_block(snapshot.miniblock_number + 1) .await; // Check that the miniblock and the protocol version for it are persisted. @@ -341,9 +337,11 @@ pub(super) async fn run_state_keeper_with_multiple_miniblocks( .collect(); let open_miniblock = SyncAction::Miniblock { + params: MiniblockParams { + timestamp: snapshot.miniblock_timestamp + 2, + virtual_blocks: 1, + }, number: snapshot.miniblock_number + 2, - timestamp: snapshot.miniblock_timestamp + 2, - virtual_blocks: 1, }; let more_txs = (0..3).map(|_| { let tx = create_l2_transaction(10, 100); @@ -366,7 +364,7 @@ pub(super) async fn run_state_keeper_with_multiple_miniblocks( actions_sender.push_actions(second_miniblock_actions).await; // Wait until both miniblocks are sealed. state_keeper - .wait(|state| state.get_local_block() == snapshot.miniblock_number + 2) + .wait_for_local_block(snapshot.miniblock_number + 2) .await; (snapshot, tx_hashes) } @@ -431,21 +429,23 @@ async fn test_external_io_recovery( let state_keeper = StateKeeperHandles::new(pool.clone(), client, action_queue, &[&tx_hashes]).await; // Check that the state keeper state is restored. - assert_eq!( - state_keeper.sync_state.get_local_block(), - snapshot.miniblock_number + 2 - ); + state_keeper + .sync_state + .wait_for_local_block(snapshot.miniblock_number + 2) + .await; // Send new actions and wait until the new miniblock is sealed. let open_miniblock = SyncAction::Miniblock { + params: MiniblockParams { + timestamp: snapshot.miniblock_timestamp + 3, + virtual_blocks: 1, + }, number: snapshot.miniblock_number + 3, - timestamp: snapshot.miniblock_timestamp + 3, - virtual_blocks: 1, }; let actions = vec![open_miniblock, new_tx.into(), SyncAction::SealMiniblock]; actions_sender.push_actions(actions).await; state_keeper - .wait(|state| state.get_local_block() == snapshot.miniblock_number + 3) + .wait_for_local_block(snapshot.miniblock_number + 3) .await; let mut storage = pool.connection().await.unwrap(); @@ -507,12 +507,13 @@ pub(super) async fn run_state_keeper_with_multiple_l1_batches( let first_l1_batch_actions = vec![l1_batch, first_tx.into(), SyncAction::SealMiniblock]; let fictive_miniblock = SyncAction::Miniblock { + params: MiniblockParams { + timestamp: snapshot.miniblock_timestamp + 2, + virtual_blocks: 0, + }, number: snapshot.miniblock_number + 2, - timestamp: snapshot.miniblock_timestamp + 2, - virtual_blocks: 0, }; - let seal_l1_batch = SyncAction::SealBatch { virtual_blocks: 0 }; - let fictive_miniblock_actions = vec![fictive_miniblock, seal_l1_batch]; + let fictive_miniblock_actions = vec![fictive_miniblock, SyncAction::SealBatch]; let l1_batch = open_l1_batch( snapshot.l1_batch_number.0 + 2, @@ -542,7 +543,7 @@ pub(super) async fn run_state_keeper_with_multiple_l1_batches( )); // Wait until the miniblocks are sealed. state_keeper - .wait(|state| state.get_local_block() == snapshot.miniblock_number + 3) + .wait_for_local_block(snapshot.miniblock_number + 3) .await; hash_task.await.unwrap(); diff --git a/core/node/node_framework/src/implementations/layers/state_keeper/main_batch_executor.rs b/core/node/node_framework/src/implementations/layers/state_keeper/main_batch_executor.rs index 075ae88934e..02f8a496f5f 100644 --- a/core/node/node_framework/src/implementations/layers/state_keeper/main_batch_executor.rs +++ b/core/node/node_framework/src/implementations/layers/state_keeper/main_batch_executor.rs @@ -35,7 +35,6 @@ impl WiringLayer for MainBatchExecutorLayer { let builder = MainBatchExecutor::new( self.db_config.state_keeper_db_path, master_pool.get_singleton().await?, - self.state_keeper_config.max_allowed_l2_tx_gas_limit.into(), self.state_keeper_config.save_call_traces, self.state_keeper_config.upload_witness_inputs_to_gcs, self.state_keeper_config.enum_index_migration_chunk_size(), diff --git a/core/node/node_framework/src/implementations/layers/state_keeper/mempool_io.rs b/core/node/node_framework/src/implementations/layers/state_keeper/mempool_io.rs index 057bc95ea67..76a444c3b6d 100644 --- a/core/node/node_framework/src/implementations/layers/state_keeper/mempool_io.rs +++ b/core/node/node_framework/src/implementations/layers/state_keeper/mempool_io.rs @@ -6,7 +6,8 @@ use zksync_config::{ ContractsConfig, }; use zksync_core::state_keeper::{ - MempoolFetcher, MempoolGuard, MempoolIO, MiniblockSealer, SequencerSealer, + self, MempoolFetcher, MempoolGuard, MempoolIO, OutputHandler, SequencerSealer, + StateKeeperPersistence, }; use crate::{ @@ -14,7 +15,7 @@ use crate::{ fee_input::FeeInputResource, object_store::ObjectStoreResource, pools::MasterPoolResource, - state_keeper::{ConditionalSealerResource, StateKeeperIOResource}, + state_keeper::{ConditionalSealerResource, OutputHandlerResource, StateKeeperIOResource}, }, resource::Unique, service::{ServiceContext, StopReceiver}, @@ -76,13 +77,17 @@ impl WiringLayer for MempoolIOLayer { let master_pool = context.get_resource::().await?; // Create miniblock sealer task. - let (miniblock_sealer, miniblock_sealer_handle) = MiniblockSealer::new( + let (persistence, miniblock_sealer) = StateKeeperPersistence::new( master_pool .get_singleton() .await .context("Get master pool")?, + self.contracts_config.l2_erc20_bridge_addr, self.state_keeper_config.miniblock_seal_queue_capacity, ); + let persistence = persistence.with_object_store(object_store); + let output_handler = OutputHandler::new(Box::new(persistence)); + context.insert_resource(OutputHandlerResource(Unique::new(output_handler)))?; context.add_task(Box::new(MiniblockSealerTask(miniblock_sealer))); // Create mempool fetcher task. @@ -106,14 +111,10 @@ impl WiringLayer for MempoolIOLayer { .context("Get master pool")?; let io = MempoolIO::new( mempool_guard, - object_store, - miniblock_sealer_handle, batch_fee_input_provider, mempool_db_pool, &self.state_keeper_config, self.mempool_config.delay_interval(), - self.contracts_config.l2_erc20_bridge_addr, - self.state_keeper_config.validation_computational_gas_limit, self.network_config.zksync_network_id, ) .await?; @@ -128,7 +129,7 @@ impl WiringLayer for MempoolIOLayer { } #[derive(Debug)] -struct MiniblockSealerTask(MiniblockSealer); +struct MiniblockSealerTask(state_keeper::MiniblockSealerTask); #[async_trait::async_trait] impl Task for MiniblockSealerTask { diff --git a/core/node/node_framework/src/implementations/layers/state_keeper/mod.rs b/core/node/node_framework/src/implementations/layers/state_keeper/mod.rs index 45c8449868a..ebdd96ac540 100644 --- a/core/node/node_framework/src/implementations/layers/state_keeper/mod.rs +++ b/core/node/node_framework/src/implementations/layers/state_keeper/mod.rs @@ -2,7 +2,8 @@ use std::sync::Arc; use anyhow::Context; use zksync_core::state_keeper::{ - seal_criteria::ConditionalSealer, BatchExecutor, StateKeeperIO, ZkSyncStateKeeper, + seal_criteria::ConditionalSealer, BatchExecutor, OutputHandler, StateKeeperIO, + ZkSyncStateKeeper, }; use zksync_storage::RocksDB; @@ -11,7 +12,8 @@ pub mod mempool_io; use crate::{ implementations::resources::state_keeper::{ - BatchExecutorResource, ConditionalSealerResource, StateKeeperIOResource, + BatchExecutorResource, ConditionalSealerResource, OutputHandlerResource, + StateKeeperIOResource, }, service::{ServiceContext, StopReceiver}, task::Task, @@ -45,11 +47,18 @@ impl WiringLayer for StateKeeperLayer { .0 .take() .context("L1BatchExecutorBuilder was provided but taken by some other task")?; + let output_handler = context + .get_resource::() + .await? + .0 + .take() + .context("HandleStateKeeperOutput was provided but taken by another task")?; let sealer = context.get_resource::().await?.0; context.add_task(Box::new(StateKeeperTask { io, batch_executor_base, + output_handler, sealer, })); Ok(()) @@ -60,6 +69,7 @@ impl WiringLayer for StateKeeperLayer { struct StateKeeperTask { io: Box, batch_executor_base: Box, + output_handler: OutputHandler, sealer: Arc, } @@ -74,6 +84,7 @@ impl Task for StateKeeperTask { stop_receiver.0, self.io, self.batch_executor_base, + self.output_handler, self.sealer, ); let result = state_keeper.run().await; diff --git a/core/node/node_framework/src/implementations/resources/state_keeper.rs b/core/node/node_framework/src/implementations/resources/state_keeper.rs index 67d147fa208..13e78143cf7 100644 --- a/core/node/node_framework/src/implementations/resources/state_keeper.rs +++ b/core/node/node_framework/src/implementations/resources/state_keeper.rs @@ -1,6 +1,8 @@ use std::sync::Arc; -use zksync_core::state_keeper::{seal_criteria::ConditionalSealer, BatchExecutor, StateKeeperIO}; +use zksync_core::state_keeper::{ + seal_criteria::ConditionalSealer, BatchExecutor, OutputHandler, StateKeeperIO, +}; use crate::resource::{Resource, ResourceId, Unique}; @@ -22,6 +24,15 @@ impl Resource for BatchExecutorResource { } } +#[derive(Debug, Clone)] +pub struct OutputHandlerResource(pub Unique); + +impl Resource for OutputHandlerResource { + fn resource_id() -> ResourceId { + "state_keeper/output_handler".into() + } +} + #[derive(Debug, Clone)] pub struct ConditionalSealerResource(pub Arc);