From 725a6ada3847fb3ad93ffdbf2f4c3e0c14fc247c Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Fri, 10 May 2024 15:45:04 +0400 Subject: [PATCH 01/15] Extract state keeper --- Cargo.lock | 40 +++++++ Cargo.toml | 2 + core/bin/external_node/Cargo.toml | 1 + core/bin/external_node/src/main.rs | 8 +- core/lib/dal/src/blocks_dal.rs | 34 ++++++ core/lib/zksync_core/Cargo.toml | 1 + .../src/api_server/tx_sender/mod.rs | 25 ++--- .../zksync_core/src/consensus/storage/mod.rs | 10 +- .../lib/zksync_core/src/consensus/testonly.rs | 14 +-- core/lib/zksync_core/src/lib.rs | 9 +- .../zksync_core/src/sync_layer/external_io.rs | 18 ++-- .../lib/zksync_core/src/sync_layer/fetcher.rs | 23 ++-- .../zksync_core/src/sync_layer/sync_action.rs | 4 +- .../zksync_core/src/sync_layer/sync_state.rs | 3 +- core/lib/zksync_core/src/sync_layer/tests.rs | 16 ++- core/lib/zksync_core/src/utils/mod.rs | 40 +------ core/node/node_framework/Cargo.toml | 1 + .../state_keeper/main_batch_executor.rs | 2 +- .../layers/state_keeper/mempool_io.rs | 7 +- .../layers/state_keeper/mod.rs | 2 +- .../implementations/resources/state_keeper.rs | 2 +- core/node/state_keeper/Cargo.toml | 48 +++++++++ core/node/state_keeper/README.md | 5 + .../src}/batch_executor/main_executor.rs | 2 +- .../state_keeper/src}/batch_executor/mod.rs | 4 +- .../src}/batch_executor/tests/mod.rs | 0 .../tests/read_storage_factory.rs | 0 .../src}/batch_executor/tests/tester.rs | 5 +- .../state_keeper/src}/io/common/mod.rs | 2 +- .../state_keeper/src}/io/common/tests.rs | 0 .../state_keeper/src}/io/mempool.rs | 2 +- .../state_keeper/src}/io/mod.rs | 16 +-- .../state_keeper/src}/io/output_handler.rs | 2 +- .../state_keeper/src}/io/persistence.rs | 7 +- .../state_keeper/src}/io/seal_logic.rs | 2 +- .../state_keeper/src}/io/tests/mod.rs | 5 +- .../state_keeper/src}/io/tests/tester.rs | 2 +- .../state_keeper/src}/keeper.rs | 0 .../mod.rs => node/state_keeper/src/lib.rs} | 9 +- .../state_keeper/src}/mempool_actor.rs | 5 +- .../state_keeper/src}/metrics.rs | 8 +- .../src}/seal_criteria/conditional_sealer.rs | 2 +- .../src}/seal_criteria/criteria/gas.rs | 2 +- .../criteria/gas_for_batch_tip.rs | 4 +- .../criteria/geometry_seal_criteria.rs | 2 +- .../src}/seal_criteria/criteria/mod.rs | 2 +- .../seal_criteria/criteria/pubdata_bytes.rs | 4 +- .../src}/seal_criteria/criteria/slots.rs | 4 +- .../criteria/tx_encoding_size.rs | 4 +- .../state_keeper/src}/seal_criteria/mod.rs | 6 +- .../state_keeper/src}/state_keeper_storage.rs | 0 core/node/state_keeper/src/testonly/mod.rs | 101 ++++++++++++++++++ .../src/testonly/test_batch_executor.rs} | 68 ++++-------- .../state_keeper/src}/tests/mod.rs | 71 +++++------- .../state_keeper/src}/types.rs | 0 .../src}/updates/l1_batch_updates.rs | 4 +- .../src}/updates/l2_block_updates.rs | 2 +- .../state_keeper/src}/updates/mod.rs | 4 +- .../state_keeper/src}/utils.rs | 0 59 files changed, 409 insertions(+), 257 deletions(-) create mode 100644 core/node/state_keeper/Cargo.toml create mode 100644 core/node/state_keeper/README.md rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/batch_executor/main_executor.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/batch_executor/mod.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/batch_executor/tests/mod.rs (100%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/batch_executor/tests/read_storage_factory.rs (100%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/batch_executor/tests/tester.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/common/mod.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/common/tests.rs (100%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/mempool.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/mod.rs (95%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/output_handler.rs (98%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/persistence.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/seal_logic.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/tests/mod.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/io/tests/tester.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/keeper.rs (100%) rename core/{lib/zksync_core/src/state_keeper/mod.rs => node/state_keeper/src/lib.rs} (94%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/mempool_actor.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/metrics.rs (98%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/conditional_sealer.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/gas.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/gas_for_batch_tip.rs (96%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/geometry_seal_criteria.rs (98%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/mod.rs (89%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/pubdata_bytes.rs (97%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/slots.rs (95%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/criteria/tx_encoding_size.rs (97%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/seal_criteria/mod.rs (98%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/state_keeper_storage.rs (100%) create mode 100644 core/node/state_keeper/src/testonly/mod.rs rename core/{lib/zksync_core/src/state_keeper/tests/tester.rs => node/state_keeper/src/testonly/test_batch_executor.rs} (92%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/tests/mod.rs (93%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/types.rs (100%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/updates/l1_batch_updates.rs (96%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/updates/l2_block_updates.rs (98%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/updates/mod.rs (99%) rename core/{lib/zksync_core/src/state_keeper => node/state_keeper/src}/utils.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 050073afbb06..ebc36538a4e4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8365,6 +8365,7 @@ dependencies = [ "zksync_prover_interface", "zksync_shared_metrics", "zksync_state", + "zksync_state_keeper", "zksync_storage", "zksync_system_constants", "zksync_test_account", @@ -8584,6 +8585,7 @@ dependencies = [ "zksync_shared_metrics", "zksync_snapshots_applier", "zksync_state", + "zksync_state_keeper", "zksync_storage", "zksync_types", "zksync_utils", @@ -8762,6 +8764,7 @@ dependencies = [ "zksync_proof_data_handler", "zksync_protobuf_config", "zksync_state", + "zksync_state_keeper", "zksync_storage", "zksync_types", "zksync_utils", @@ -9003,6 +9006,43 @@ dependencies = [ "zksync_utils", ] +[[package]] +name = "zksync_state_keeper" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "async-trait", + "futures 0.3.28", + "hex", + "itertools 0.10.5", + "multivm", + "once_cell", + "tempfile", + "test-casing", + "thiserror", + "tokio", + "tracing", + "vise", + "vm_utils", + "zksync_config", + "zksync_contracts", + "zksync_dal", + "zksync_eth_client", + "zksync_mempool", + "zksync_node_fee_model", + "zksync_node_genesis", + "zksync_node_test_utils", + "zksync_protobuf", + "zksync_shared_metrics", + "zksync_state", + "zksync_storage", + "zksync_system_constants", + "zksync_test_account", + "zksync_types", + "zksync_utils", +] + [[package]] name = "zksync_storage" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index d0112a46bbc0..ed55711bf47c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,6 +23,7 @@ members = [ "core/node/eth_sender", "core/node/vm_runner", "core/node/test_utils", + "core/node/state_keeper", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -240,3 +241,4 @@ zksync_node_db_pruner = { path = "core/node/db_pruner" } zksync_node_fee_model = { path = "core/node/fee_model" } zksync_vm_runner = { path = "core/node/vm_runner" } zksync_node_test_utils = { path = "core/node/test_utils" } +zksync_state_keeper = { path = "core/node/state_keeper" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index b496dfd72c73..2a6f40a15ff5 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -36,6 +36,7 @@ zksync_node_genesis.workspace = true zksync_node_fee_model.workspace = true zksync_node_db_pruner.workspace = true zksync_eth_sender.workspace = true +zksync_state_keeper.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index dc9380940382..916e78d1bc17 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -30,10 +30,6 @@ use zksync_core::{ metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, reorg_detector::{self, ReorgDetector}, setup_sigint_handler, - state_keeper::{ - seal_criteria::NoopSealer, AsyncRocksdbCache, BatchExecutor, MainBatchExecutor, - OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, - }, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, SyncState, }, @@ -52,6 +48,10 @@ use zksync_health_check::{AppHealthCheck, HealthStatus, ReactiveHealthCheck}; use zksync_node_db_pruner::{DbPruner, DbPrunerConfig}; use zksync_node_fee_model::l1_gas_price::MainNodeFeeParamsFetcher; use zksync_state::{PostgresStorageCaches, RocksdbStorageOptions}; +use zksync_state_keeper::{ + seal_criteria::NoopSealer, AsyncRocksdbCache, BatchExecutor, MainBatchExecutor, OutputHandler, + StateKeeperPersistence, ZkSyncStateKeeper, +}; use zksync_storage::RocksDB; use zksync_types::L2ChainId; use zksync_utils::wait_for_tasks::ManagedTasks; diff --git a/core/lib/dal/src/blocks_dal.rs b/core/lib/dal/src/blocks_dal.rs index 4e39a7bcea31..d4c3841f5dd8 100644 --- a/core/lib/dal/src/blocks_dal.rs +++ b/core/lib/dal/src/blocks_dal.rs @@ -2,6 +2,7 @@ use std::{ collections::HashMap, convert::{Into, TryInto}, ops, + sync::atomic::{AtomicBool, Ordering}, }; use anyhow::Context as _; @@ -2230,6 +2231,39 @@ impl BlocksDal<'_, '_> { self.delete_logs_inner().await?; Ok(()) } + + /// Obtains a protocol version projected to be applied for the next L2 block. This is either the version used by the last + /// sealed L2 block, or (if there are no L2 blocks), one referenced in the snapshot recovery record. + pub async fn pending_protocol_version(&mut self) -> anyhow::Result { + static WARNED_ABOUT_NO_VERSION: AtomicBool = AtomicBool::new(false); + + let last_l2_block = self + .storage + .blocks_dal() + .get_last_sealed_l2_block_header() + .await?; + if let Some(last_l2_block) = last_l2_block { + return Ok(last_l2_block.protocol_version.unwrap_or_else(|| { + // Protocol version should be set for the most recent L2 block even in cases it's not filled + // for old L2 blocks, hence the warning. We don't want to rely on this assumption, so we treat + // the lack of it as in other similar places, replacing with the default value. + if !WARNED_ABOUT_NO_VERSION.fetch_or(true, Ordering::Relaxed) { + tracing::warn!( + "Protocol version not set for recent L2 block: {last_l2_block:?}" + ); + } + ProtocolVersionId::last_potentially_undefined() + })); + } + // No L2 blocks in the storage; use snapshot recovery information. + let snapshot_recovery = self + .storage + .snapshot_recovery_dal() + .get_applied_snapshot_status() + .await? + .context("storage contains neither L2 blocks, nor snapshot recovery info")?; + Ok(snapshot_recovery.protocol_version) + } } #[cfg(test)] diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 756d32ab74af..3755f1d8beeb 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -49,6 +49,7 @@ zksync_house_keeper.workspace = true zksync_node_genesis.workspace = true zksync_eth_sender.workspace = true zksync_node_fee_model.workspace = true +zksync_state_keeper.workspace = true multivm.workspace = true # Consensus dependenices diff --git a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs index 1a03c91bcaa6..78143978fc3c 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs +++ b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs @@ -19,6 +19,7 @@ use zksync_dal::{ }; use zksync_node_fee_model::BatchFeeModelInputProvider; use zksync_state::PostgresStorageCaches; +use zksync_state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}; use zksync_types::{ fee::{Fee, TransactionExecutionMetrics}, fee_model::BatchFeeInput, @@ -33,16 +34,12 @@ use zksync_utils::h256_to_u256; pub(super) use self::result::SubmitTxError; use self::tx_sink::TxSink; -use crate::{ - api_server::{ - execution_sandbox::{ - BlockArgs, SubmitTxStage, TransactionExecutor, TxExecutionArgs, TxSharedArgs, - VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, - }, - tx_sender::result::ApiCallResult, +use crate::api_server::{ + execution_sandbox::{ + BlockArgs, SubmitTxStage, TransactionExecutor, TxExecutionArgs, TxSharedArgs, + VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, }, - state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}, - utils::pending_protocol_version, + tx_sender::result::ApiCallResult, }; pub mod master_pool_sink; @@ -320,7 +317,7 @@ impl TxSender { let tx_hash = tx.hash(); let stage_latency = SANDBOX_METRICS.start_tx_submit_stage(tx_hash, SubmitTxStage::Validate); let mut connection = self.acquire_replica_connection().await?; - let protocol_version = pending_protocol_version(&mut connection).await?; + let protocol_version = connection.blocks_dal().pending_protocol_version().await?; drop(connection); self.validate_tx(&tx, protocol_version).await?; stage_latency.observe(); @@ -697,7 +694,9 @@ impl TxSender { let mut connection = self.acquire_replica_connection().await?; let block_args = BlockArgs::pending(&mut connection).await?; - let protocol_version = pending_protocol_version(&mut connection) + let protocol_version = connection + .blocks_dal() + .pending_protocol_version() .await .context("failed getting pending protocol version")?; let max_gas_limit = get_max_batch_gas_limit(protocol_version.into()); @@ -951,7 +950,9 @@ impl TxSender { pub async fn gas_price(&self) -> anyhow::Result { let mut connection = self.acquire_replica_connection().await?; - let protocol_version = pending_protocol_version(&mut connection) + let protocol_version = connection + .blocks_dal() + .pending_protocol_version() .await .context("failed obtaining pending protocol version")?; drop(connection); diff --git a/core/lib/zksync_core/src/consensus/storage/mod.rs b/core/lib/zksync_core/src/consensus/storage/mod.rs index 0083481deed5..83bd6a9fbb7c 100644 --- a/core/lib/zksync_core/src/consensus/storage/mod.rs +++ b/core/lib/zksync_core/src/consensus/storage/mod.rs @@ -8,17 +8,15 @@ use zksync_consensus_bft::PayloadManager; use zksync_consensus_roles::validator; use zksync_consensus_storage as storage; use zksync_dal::{consensus_dal::Payload, ConnectionPool, Core, CoreDal, DalError}; +use zksync_state_keeper::io::common::IoCursor; use zksync_types::L2BlockNumber; #[cfg(test)] mod testonly; -use crate::{ - state_keeper::io::common::IoCursor, - sync_layer::{ - fetcher::{FetchedBlock, FetchedTransaction}, - sync_action::ActionQueueSender, - }, +use crate::sync_layer::{ + fetcher::{FetchedBlock, FetchedTransaction, IoCursorExt as _}, + sync_action::ActionQueueSender, }; /// Context-aware `zksync_dal::Connection` wrapper. diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index f1996cb2bf92..27df24773509 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -11,6 +11,12 @@ use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{CoreDal, DalError}; use zksync_node_genesis::{mock_genesis_config, GenesisParams}; use zksync_node_test_utils::{create_l1_batch_metadata, create_l2_transaction}; +use zksync_state_keeper::{ + io::{IoCursor, L1BatchParams, L2BlockParams}, + seal_criteria::NoopSealer, + testonly::MockBatchExecutor, + OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, +}; use zksync_types::{ api, snapshots::SnapshotRecoveryStatus, Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, H256, @@ -23,14 +29,8 @@ use zksync_web3_decl::{ use crate::{ api_server::web3::{state::InternalApiConfig, tests::spawn_http_server}, consensus::{fetcher::P2PConfig, Fetcher, Store}, - state_keeper::{ - io::{IoCursor, L1BatchParams, L2BlockParams}, - seal_criteria::NoopSealer, - tests::MockBatchExecutor, - OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, - }, sync_layer::{ - fetcher::FetchedTransaction, + fetcher::{FetchedTransaction, IoCursorExt as _}, sync_action::{ActionQueue, ActionQueueSender, SyncAction}, ExternalIO, MainNodeClient, SyncState, }, diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index b8513c6963fb..862fbce272bf 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -78,6 +78,10 @@ use zksync_node_genesis::{ensure_genesis_state, GenesisParams}; use zksync_object_store::{ObjectStore, ObjectStoreFactory}; use zksync_shared_metrics::{InitStage, APP_METRICS}; use zksync_state::{PostgresStorageCaches, RocksdbStorageOptions}; +use zksync_state_keeper::{ + create_state_keeper, AsyncRocksdbCache, MempoolFetcher, MempoolGuard, OutputHandler, + SequencerSealer, StateKeeperPersistence, +}; use zksync_types::{ethabi::Contract, fee_model::FeeModelConfig, Address, L2ChainId}; use crate::{ @@ -90,10 +94,6 @@ use crate::{ web3::{self, mempool_cache::MempoolCache, state::InternalApiConfig, Namespace}, }, metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, - state_keeper::{ - create_state_keeper, AsyncRocksdbCache, MempoolFetcher, MempoolGuard, OutputHandler, - SequencerSealer, StateKeeperPersistence, - }, utils::ensure_l1_batch_commit_data_generation_mode, }; @@ -103,7 +103,6 @@ pub mod consistency_checker; pub mod metadata_calculator; pub mod proto; pub mod reorg_detector; -pub mod state_keeper; pub mod sync_layer; pub mod temp_config_store; pub mod utils; 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 482c0fbe2de2..d8a62f804899 100644 --- a/core/lib/zksync_core/src/sync_layer/external_io.rs +++ b/core/lib/zksync_core/src/sync_layer/external_io.rs @@ -5,6 +5,15 @@ use async_trait::async_trait; use vm_utils::storage::L1BatchParamsProvider; use zksync_contracts::{BaseSystemContracts, BaseSystemContractsHashes, SystemContractCode}; use zksync_dal::{ConnectionPool, Core, CoreDal}; +use zksync_state_keeper::{ + io::{ + common::{load_pending_batch, IoCursor}, + L1BatchParams, L2BlockParams, PendingBatchData, StateKeeperIO, + }, + metrics::KEEPER_METRICS, + seal_criteria::IoSealCriteria, + updates::UpdatesManager, +}; use zksync_types::{ protocol_upgrade::ProtocolUpgradeTx, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, Transaction, H256, @@ -15,15 +24,6 @@ use super::{ client::MainNodeClient, sync_action::{ActionQueue, SyncAction}, }; -use crate::state_keeper::{ - io::{ - common::{load_pending_batch, IoCursor}, - L1BatchParams, L2BlockParams, PendingBatchData, StateKeeperIO, - }, - metrics::KEEPER_METRICS, - seal_criteria::IoSealCriteria, - updates::UpdatesManager, -}; /// ExternalIO is the IO abstraction for the state keeper that is used in the external node. /// It receives a sequence of actions from the fetcher via the action queue and propagates it diff --git a/core/lib/zksync_core/src/sync_layer/fetcher.rs b/core/lib/zksync_core/src/sync_layer/fetcher.rs index 8f5922afa3af..51b9f7c7a060 100644 --- a/core/lib/zksync_core/src/sync_layer/fetcher.rs +++ b/core/lib/zksync_core/src/sync_layer/fetcher.rs @@ -1,5 +1,6 @@ use zksync_dal::{Connection, Core, CoreDal}; use zksync_shared_metrics::{TxStage, APP_METRICS}; +use zksync_state_keeper::io::{common::IoCursor, L1BatchParams, L2BlockParams}; use zksync_types::{ api::en::SyncBlock, block::L2BlockHasher, fee_model::BatchFeeInput, helpers::unix_timestamp_ms, Address, L1BatchNumber, L2BlockNumber, ProtocolVersionId, H256, @@ -9,12 +10,11 @@ use super::{ metrics::{L1BatchStage, FETCHER_METRICS}, sync_action::SyncAction, }; -use crate::state_keeper::io::{common::IoCursor, L1BatchParams, L2BlockParams}; /// Same as [`zksync_types::Transaction`], just with additional guarantees that the "received at" timestamp was set locally. /// We cannot transfer `Transaction`s without these timestamps, because this would break backward compatibility. #[derive(Debug, Clone)] -pub(crate) struct FetchedTransaction(zksync_types::Transaction); +pub struct FetchedTransaction(zksync_types::Transaction); impl FetchedTransaction { pub fn new(mut tx: zksync_types::Transaction) -> Self { @@ -38,7 +38,7 @@ impl From for zksync_types::Transaction { /// Common denominator for blocks fetched by an external node. #[derive(Debug)] -pub(crate) struct FetchedBlock { +pub struct FetchedBlock { pub number: L2BlockNumber, pub l1_batch_number: L1BatchNumber, pub last_in_batch: bool, @@ -97,9 +97,20 @@ impl TryFrom for FetchedBlock { } } -impl IoCursor { +/// Helper method for `IoCursor` for needs of sync layer. +#[async_trait::async_trait] +pub trait IoCursorExt: Sized { /// Loads this cursor from storage and modifies it to account for the pending L1 batch if necessary. - pub(crate) async fn for_fetcher(storage: &mut Connection<'_, Core>) -> anyhow::Result { + async fn for_fetcher(storage: &mut Connection<'_, Core>) -> anyhow::Result; + + /// Advances the cursor according to the provided fetched block and returns a sequence of `SyncAction` + /// objects to process. + fn advance(&mut self, block: FetchedBlock) -> Vec; +} + +#[async_trait::async_trait] +impl IoCursorExt for IoCursor { + async fn for_fetcher(storage: &mut Connection<'_, Core>) -> anyhow::Result { let mut this = Self::new(storage).await?; // It's important to know whether we have opened a new batch already or just sealed the previous one. // Depending on it, we must either insert `OpenBatch` item into the queue, or not. @@ -110,7 +121,7 @@ impl IoCursor { Ok(this) } - pub(crate) fn advance(&mut self, block: FetchedBlock) -> Vec { + fn advance(&mut self, block: FetchedBlock) -> Vec { assert_eq!(block.number, self.next_l2_block); let local_block_hash = block.compute_hash(self.prev_l2_block_hash); if let Some(reference_hash) = block.reference_hash { 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 820f045445d5..23d921213d52 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_action.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_action.rs @@ -1,8 +1,8 @@ use tokio::sync::mpsc; +use zksync_state_keeper::io::{L1BatchParams, L2BlockParams}; use zksync_types::{L1BatchNumber, L2BlockNumber}; use super::{fetcher::FetchedTransaction, metrics::QUEUE_METRICS}; -use crate::state_keeper::io::{L1BatchParams, L2BlockParams}; #[derive(Debug)] pub struct ActionQueueSender(mpsc::Sender); @@ -136,7 +136,7 @@ impl ActionQueue { /// An instruction for the ExternalIO to request a certain action from the state keeper. #[derive(Debug, Clone)] -pub(crate) enum SyncAction { +pub enum SyncAction { OpenBatch { params: L1BatchParams, // Additional parameters used only for sanity checks 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 68d5e4be51cd..87c8c2c14535 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_state.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_state.rs @@ -7,11 +7,10 @@ use zksync_concurrency::{ctx, sync}; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_health_check::{CheckHealth, Health, HealthStatus}; use zksync_shared_metrics::EN_METRICS; +use zksync_state_keeper::{io::IoCursor, updates::UpdatesManager, StateKeeperOutputHandler}; use zksync_types::L2BlockNumber; use zksync_web3_decl::{client::BoxedL2Client, namespaces::EthNamespaceClient}; -use crate::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. /// Data inside is expected to be updated by both `MainNodeFetcher` (on last block available on the main node) diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index c52ef05a02e6..7186c03ccb5e 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -10,6 +10,12 @@ use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{ create_l1_batch_metadata, create_l2_transaction, prepare_recovery_snapshot, }; +use zksync_state_keeper::{ + io::{L1BatchParams, L2BlockParams}, + seal_criteria::NoopSealer, + testonly::test_batch_executor::TestBatchExecutorBuilder, + OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, +}; use zksync_types::{ api, block::L2BlockHasher, @@ -19,15 +25,7 @@ use zksync_types::{ }; use super::{fetcher::FetchedTransaction, sync_action::SyncAction, *}; -use crate::{ - consensus::testonly::MockMainNodeClient, - state_keeper::{ - io::{L1BatchParams, L2BlockParams}, - seal_criteria::NoopSealer, - tests::TestBatchExecutorBuilder, - OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, - }, -}; +use crate::consensus::testonly::MockMainNodeClient; const TEST_TIMEOUT: Duration = Duration::from_secs(10); const POLL_INTERVAL: Duration = Duration::from_millis(50); diff --git a/core/lib/zksync_core/src/utils/mod.rs b/core/lib/zksync_core/src/utils/mod.rs index f2364dd1eff0..ea182a9fe235 100644 --- a/core/lib/zksync_core/src/utils/mod.rs +++ b/core/lib/zksync_core/src/utils/mod.rs @@ -1,18 +1,13 @@ //! Miscellaneous utils used by multiple components. -use std::{ - future::Future, - sync::atomic::{AtomicBool, Ordering}, - time::Duration, -}; +use std::{future::Future, time::Duration}; -use anyhow::Context as _; use async_trait::async_trait; use tokio::sync::watch; use zksync_config::configs::chain::L1BatchCommitDataGeneratorMode; use zksync_dal::{Connection, ConnectionPool, Core, CoreDal}; use zksync_eth_client::{CallFunctionArgs, ClientError, Error as EthClientError, EthInterface}; -use zksync_types::{Address, L1BatchNumber, ProtocolVersionId}; +use zksync_types::{Address, L1BatchNumber}; /// Fallible and async predicate for binary search. #[async_trait] @@ -128,37 +123,6 @@ pub(crate) async fn projected_first_l1_batch( Ok(snapshot_recovery.map_or(L1BatchNumber(0), |recovery| recovery.l1_batch_number + 1)) } -/// Obtains a protocol version projected to be applied for the next L2 block. This is either the version used by the last -/// sealed L2 block, or (if there are no L2 blocks), one referenced in the snapshot recovery record. -pub(crate) async fn pending_protocol_version( - storage: &mut Connection<'_, Core>, -) -> anyhow::Result { - static WARNED_ABOUT_NO_VERSION: AtomicBool = AtomicBool::new(false); - - let last_l2_block = storage - .blocks_dal() - .get_last_sealed_l2_block_header() - .await?; - if let Some(last_l2_block) = last_l2_block { - return Ok(last_l2_block.protocol_version.unwrap_or_else(|| { - // Protocol version should be set for the most recent L2 block even in cases it's not filled - // for old L2 blocks, hence the warning. We don't want to rely on this assumption, so we treat - // the lack of it as in other similar places, replacing with the default value. - if !WARNED_ABOUT_NO_VERSION.fetch_or(true, Ordering::Relaxed) { - tracing::warn!("Protocol version not set for recent L2 block: {last_l2_block:?}"); - } - ProtocolVersionId::last_potentially_undefined() - })); - } - // No L2 blocks in the storage; use snapshot recovery information. - let snapshot_recovery = storage - .snapshot_recovery_dal() - .get_applied_snapshot_status() - .await? - .context("storage contains neither L2 blocks, nor snapshot recovery info")?; - Ok(snapshot_recovery.protocol_version) -} - async fn get_pubdata_pricing_mode( diamond_proxy_address: Address, eth_client: &dyn EthInterface, diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index e02266c16546..9ad5d0dd1617 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -34,6 +34,7 @@ zksync_commitment_generator.workspace = true zksync_house_keeper.workspace = true zksync_node_fee_model.workspace = true zksync_eth_sender.workspace = true +zksync_state_keeper.workspace = true tracing.workspace = true thiserror.workspace = true 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 d460b0725277..216d29fd81ac 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 @@ -1,8 +1,8 @@ use std::sync::Arc; use zksync_config::{configs::chain::StateKeeperConfig, DBConfig}; -use zksync_core::state_keeper::{AsyncRocksdbCache, MainBatchExecutor}; use zksync_state::{AsyncCatchupTask, RocksdbStorageOptions}; +use zksync_state_keeper::{AsyncRocksdbCache, MainBatchExecutor}; use crate::{ implementations::resources::{ 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 0346ef2c72cc..eaf4b4204343 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 @@ -8,9 +8,8 @@ use zksync_config::{ }, ContractsConfig, }; -use zksync_core::state_keeper::{ - self, MempoolFetcher, MempoolGuard, MempoolIO, OutputHandler, SequencerSealer, - StateKeeperPersistence, +use zksync_state_keeper::{ + MempoolFetcher, MempoolGuard, MempoolIO, OutputHandler, SequencerSealer, StateKeeperPersistence, }; use crate::{ @@ -133,7 +132,7 @@ impl WiringLayer for MempoolIOLayer { } #[derive(Debug)] -struct L2BlockSealerTask(state_keeper::L2BlockSealerTask); +struct L2BlockSealerTask(zksync_state_keeper::L2BlockSealerTask); #[async_trait::async_trait] impl Task for L2BlockSealerTask { 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 ebdd96ac540f..3b6becfe73c8 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 @@ -1,7 +1,7 @@ use std::sync::Arc; use anyhow::Context; -use zksync_core::state_keeper::{ +use zksync_state_keeper::{ seal_criteria::ConditionalSealer, BatchExecutor, OutputHandler, StateKeeperIO, ZkSyncStateKeeper, }; 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 16643c81b5fa..804822154497 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,6 @@ use std::sync::Arc; -use zksync_core::state_keeper::{ +use zksync_state_keeper::{ seal_criteria::ConditionalSealer, BatchExecutor, OutputHandler, StateKeeperIO, }; diff --git a/core/node/state_keeper/Cargo.toml b/core/node/state_keeper/Cargo.toml new file mode 100644 index 000000000000..297f309d4f20 --- /dev/null +++ b/core/node/state_keeper/Cargo.toml @@ -0,0 +1,48 @@ +[package] +name = "zksync_state_keeper" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + + +[dependencies] +vise.workspace = true +multivm.workspace = true +zksync_types.workspace = true +zksync_dal.workspace = true +zksync_state.workspace = true +zksync_storage.workspace = true +zksync_mempool.workspace = true +zksync_shared_metrics.workspace = true +zksync_config.workspace = true +zksync_node_fee_model.workspace = true +zksync_utils.workspace = true +zksync_contracts.workspace = true +zksync_protobuf.workspace = true +zksync_node_test_utils.workspace = true +vm_utils.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +tokio = { workspace = true, features = ["time"] } +thiserror.workspace = true +tracing.workspace = true +once_cell.workspace = true +itertools.workspace = true +hex.workspace = true + +[dev-dependencies] +assert_matches.workspace = true +test-casing.workspace = true +tempfile.workspace = true +futures.workspace = true + +zksync_test_account.workspace = true +zksync_node_genesis.workspace = true +zksync_eth_client.workspace = true +zksync_system_constants.workspace = true diff --git a/core/node/state_keeper/README.md b/core/node/state_keeper/README.md new file mode 100644 index 000000000000..db6f87a76cc2 --- /dev/null +++ b/core/node/state_keeper/README.md @@ -0,0 +1,5 @@ +# `zksync_state_keeper` + +State keeper is the main component of the sequencer implementation. Its main responsibility is to extract transactions +from a certain source (like mempool), form them into a set of L2 blocks and L1 batches, and pass for persisting and +further processing. diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs b/core/node/state_keeper/src/batch_executor/main_executor.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs rename to core/node/state_keeper/src/batch_executor/main_executor.rs index 1d0c3b203b1f..fa3bd5197f6d 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/main_executor.rs +++ b/core/node/state_keeper/src/batch_executor/main_executor.rs @@ -21,7 +21,7 @@ use zksync_types::{vm_trace::Call, Transaction}; use zksync_utils::bytecode::CompressedBytecodeInfo; use super::{BatchExecutor, BatchExecutorHandle, Command, TxExecutionResult}; -use crate::state_keeper::{ +use crate::{ metrics::{TxExecutionStage, BATCH_TIP_METRICS, EXECUTOR_METRICS, KEEPER_METRICS}, types::ExecutionMetricsForCriteria, }; diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs b/core/node/state_keeper/src/batch_executor/mod.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs rename to core/node/state_keeper/src/batch_executor/mod.rs index 0164af10e2b7..671695503ecb 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs +++ b/core/node/state_keeper/src/batch_executor/mod.rs @@ -11,7 +11,7 @@ use tokio::{ use zksync_types::{vm_trace::Call, Transaction}; use zksync_utils::bytecode::CompressedBytecodeInfo; -use crate::state_keeper::{ +use crate::{ metrics::{ExecutorCommand, EXECUTOR_METRICS}, types::ExecutionMetricsForCriteria, }; @@ -76,7 +76,7 @@ pub struct BatchExecutorHandle { impl BatchExecutorHandle { /// Creates a batch executor handle from the provided sender and thread join handle. /// Can be used to inject an alternative batch executor implementation. - #[cfg(test)] + #[doc(hidden)] pub(super) fn from_raw(handle: JoinHandle<()>, commands: mpsc::Sender) -> Self { Self { handle, commands } } diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs b/core/node/state_keeper/src/batch_executor/tests/mod.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs rename to core/node/state_keeper/src/batch_executor/tests/mod.rs diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/read_storage_factory.rs b/core/node/state_keeper/src/batch_executor/tests/read_storage_factory.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/batch_executor/tests/read_storage_factory.rs rename to core/node/state_keeper/src/batch_executor/tests/read_storage_factory.rs diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs b/core/node/state_keeper/src/batch_executor/tests/tester.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs rename to core/node/state_keeper/src/batch_executor/tests/tester.rs index 68ee85683d14..b77d044f136f 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs +++ b/core/node/state_keeper/src/batch_executor/tests/tester.rs @@ -30,9 +30,10 @@ use super::{ read_storage_factory::{PostgresFactory, RocksdbFactory}, StorageType, }; -use crate::state_keeper::{ +use crate::{ batch_executor::{BatchExecutorHandle, TxExecutionResult}, - tests::{default_l1_batch_env, default_system_env, BASE_SYSTEM_CONTRACTS}, + testonly::BASE_SYSTEM_CONTRACTS, + tests::{default_l1_batch_env, default_system_env}, AsyncRocksdbCache, BatchExecutor, MainBatchExecutor, }; diff --git a/core/lib/zksync_core/src/state_keeper/io/common/mod.rs b/core/node/state_keeper/src/io/common/mod.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/common/mod.rs rename to core/node/state_keeper/src/io/common/mod.rs index 72b4808251ef..f521a87ab228 100644 --- a/core/lib/zksync_core/src/state_keeper/io/common/mod.rs +++ b/core/node/state_keeper/src/io/common/mod.rs @@ -81,7 +81,7 @@ impl IoCursor { /// # Errors /// /// Propagates DB errors. Also returns an error if environment doesn't correspond to a pending L1 batch. -pub(crate) async fn load_pending_batch( +pub async fn load_pending_batch( storage: &mut Connection<'_, Core>, system_env: SystemEnv, l1_batch_env: L1BatchEnv, diff --git a/core/lib/zksync_core/src/state_keeper/io/common/tests.rs b/core/node/state_keeper/src/io/common/tests.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/io/common/tests.rs rename to core/node/state_keeper/src/io/common/tests.rs diff --git a/core/lib/zksync_core/src/state_keeper/io/mempool.rs b/core/node/state_keeper/src/io/mempool.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/mempool.rs rename to core/node/state_keeper/src/io/mempool.rs index d017716f82e7..3fb1ee827c6d 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mempool.rs +++ b/core/node/state_keeper/src/io/mempool.rs @@ -21,7 +21,7 @@ use zksync_types::{ // TODO (SMA-1206): use seconds instead of milliseconds. use zksync_utils::time::millis_since_epoch; -use crate::state_keeper::{ +use crate::{ io::{ common::{load_pending_batch, poll_iters, IoCursor}, L1BatchParams, L2BlockParams, PendingBatchData, StateKeeperIO, diff --git a/core/lib/zksync_core/src/state_keeper/io/mod.rs b/core/node/state_keeper/src/io/mod.rs similarity index 95% rename from core/lib/zksync_core/src/state_keeper/io/mod.rs rename to core/node/state_keeper/src/io/mod.rs index 0b1f5fcbe95c..33809fa850c9 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mod.rs +++ b/core/node/state_keeper/src/io/mod.rs @@ -17,7 +17,7 @@ pub use self::{ }; use super::seal_criteria::IoSealCriteria; -pub(crate) mod common; +pub mod common; pub(crate) mod mempool; mod output_handler; mod persistence; @@ -46,7 +46,7 @@ pub struct PendingBatchData { #[derive(Debug, Copy, Clone, Default)] pub struct L2BlockParams { /// The timestamp of the L2 block. - pub(crate) timestamp: u64, + pub timestamp: u64, /// The maximal number of virtual blocks that can be created within this L2 block. /// During the migration from displaying users `batch.number` to L2 block number in Q3 2023 /// in order to make the process smoother for users, we temporarily display the virtual blocks for users. @@ -55,22 +55,22 @@ pub struct L2BlockParams { /// Note that it is the *maximal* number of virtual blocks that can be created within this L2 block since /// once the virtual blocks' number reaches the L2 block number, they will never be allowed to exceed those, i.e. /// any "excess" created blocks will be ignored. - pub(crate) virtual_blocks: u32, + pub virtual_blocks: u32, } /// 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, + pub protocol_version: ProtocolVersionId, /// Computational gas limit for the new L1 batch. - pub(crate) validation_computational_gas_limit: u32, + pub validation_computational_gas_limit: u32, /// Operator address (aka fee address) for the new L1 batch. - pub(crate) operator_address: Address, + pub operator_address: Address, /// Fee parameters to be used in the new L1 batch. - pub(crate) fee_input: BatchFeeInput, + pub fee_input: BatchFeeInput, /// Parameters of the first L2 block in the batch. - pub(crate) first_l2_block: L2BlockParams, + pub first_l2_block: L2BlockParams, } impl L1BatchParams { diff --git a/core/lib/zksync_core/src/state_keeper/io/output_handler.rs b/core/node/state_keeper/src/io/output_handler.rs similarity index 98% rename from core/lib/zksync_core/src/state_keeper/io/output_handler.rs rename to core/node/state_keeper/src/io/output_handler.rs index d9a6797c0e93..1d237498f7fc 100644 --- a/core/lib/zksync_core/src/state_keeper/io/output_handler.rs +++ b/core/node/state_keeper/src/io/output_handler.rs @@ -5,7 +5,7 @@ use std::fmt; use anyhow::Context as _; use async_trait::async_trait; -use crate::state_keeper::{io::IoCursor, updates::UpdatesManager}; +use crate::{io::IoCursor, updates::UpdatesManager}; /// Handler for state keeper outputs (L2 blocks and L1 batches). #[async_trait] diff --git a/core/lib/zksync_core/src/state_keeper/io/persistence.rs b/core/node/state_keeper/src/io/persistence.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/persistence.rs rename to core/node/state_keeper/src/io/persistence.rs index 7e206514bec8..726600a6a59d 100644 --- a/core/lib/zksync_core/src/state_keeper/io/persistence.rs +++ b/core/node/state_keeper/src/io/persistence.rs @@ -9,7 +9,7 @@ use zksync_dal::{ConnectionPool, Core}; use zksync_shared_metrics::{BlockStage, APP_METRICS}; use zksync_types::Address; -use crate::state_keeper::{ +use crate::{ io::StateKeeperOutputHandler, metrics::{L2BlockQueueStage, L2_BLOCK_METRICS}, updates::{L2BlockSealCommand, UpdatesManager}, @@ -256,11 +256,12 @@ mod tests { use zksync_utils::u256_to_h256; use super::*; - use crate::state_keeper::{ + use crate::{ io::L2BlockParams, + testonly::default_vm_batch_result, tests::{ create_execution_result, create_transaction, create_updates_manager, - default_l1_batch_env, default_system_env, default_vm_batch_result, Query, + default_l1_batch_env, default_system_env, Query, }, }; diff --git a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs b/core/node/state_keeper/src/io/seal_logic.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/seal_logic.rs rename to core/node/state_keeper/src/io/seal_logic.rs index d04f17b2c22f..6772e93ad46e 100644 --- a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs +++ b/core/node/state_keeper/src/io/seal_logic.rs @@ -28,7 +28,7 @@ use zksync_types::{ }; use zksync_utils::u256_to_h256; -use crate::state_keeper::{ +use crate::{ metrics::{ L1BatchSealStage, L2BlockSealStage, TxExecutionType, KEEPER_METRICS, L1_BATCH_METRICS, L2_BLOCK_METRICS, diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/node/state_keeper/src/io/tests/mod.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/tests/mod.rs rename to core/node/state_keeper/src/io/tests/mod.rs index 4814e5ebd9bc..0e7132d3d665 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/node/state_keeper/src/io/tests/mod.rs @@ -17,10 +17,11 @@ use zksync_types::{ use zksync_utils::time::seconds_since_epoch; use self::tester::Tester; -use crate::state_keeper::{ +use crate::{ io::StateKeeperIO, mempool_actor::l2_tx_filter, - tests::{create_execution_result, create_transaction, Query, BASE_SYSTEM_CONTRACTS}, + testonly::BASE_SYSTEM_CONTRACTS, + tests::{create_execution_result, create_transaction, Query}, updates::{L2BlockSealCommand, L2BlockUpdates, UpdatesManager}, StateKeeperOutputHandler, StateKeeperPersistence, }; diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs b/core/node/state_keeper/src/io/tests/tester.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/io/tests/tester.rs rename to core/node/state_keeper/src/io/tests/tester.rs index 433dba9e72ed..0f23ff8693a9 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs +++ b/core/node/state_keeper/src/io/tests/tester.rs @@ -29,7 +29,7 @@ use zksync_types::{ L2BlockNumber, L2ChainId, PriorityOpId, ProtocolVersionId, H256, }; -use crate::state_keeper::{MempoolGuard, MempoolIO}; +use crate::{MempoolGuard, MempoolIO}; #[derive(Debug)] pub struct Tester { diff --git a/core/lib/zksync_core/src/state_keeper/keeper.rs b/core/node/state_keeper/src/keeper.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/keeper.rs rename to core/node/state_keeper/src/keeper.rs diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/node/state_keeper/src/lib.rs similarity index 94% rename from core/lib/zksync_core/src/state_keeper/mod.rs rename to core/node/state_keeper/src/lib.rs index 9d2603a872ad..d9220272276c 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/node/state_keeper/src/lib.rs @@ -23,20 +23,21 @@ pub use self::{ }; mod batch_executor; -pub(crate) mod io; +pub mod io; mod keeper; mod mempool_actor; -pub(crate) mod metrics; +pub mod metrics; pub mod seal_criteria; mod state_keeper_storage; +pub mod testonly; #[cfg(test)] pub(crate) mod tests; pub(crate) mod types; -pub(crate) mod updates; +pub mod updates; pub(crate) mod utils; #[allow(clippy::too_many_arguments)] -pub(crate) async fn create_state_keeper( +pub async fn create_state_keeper( state_keeper_config: StateKeeperConfig, wallets: wallets::StateKeeper, async_cache: AsyncRocksdbCache, diff --git a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs b/core/node/state_keeper/src/mempool_actor.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/mempool_actor.rs rename to core/node/state_keeper/src/mempool_actor.rs index 8bc2498f94a7..2496655bcb94 100644 --- a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs +++ b/core/node/state_keeper/src/mempool_actor.rs @@ -14,7 +14,6 @@ use zksync_types::H256; use zksync_types::{get_nonce_key, Address, Nonce, Transaction, VmVersion}; use super::{metrics::KEEPER_METRICS, types::MempoolGuard}; -use crate::utils::pending_protocol_version; /// Creates a mempool filter for L2 transactions based on the current L1 gas price. /// The filter is used to filter out transactions from the mempool that do not cover expenses @@ -84,7 +83,9 @@ impl MempoolFetcher { let latency = KEEPER_METRICS.mempool_sync.start(); let mut storage = self.pool.connection_tagged("state_keeper").await?; let mempool_info = self.mempool.get_mempool_info(); - let protocol_version = pending_protocol_version(&mut storage) + let protocol_version = storage + .blocks_dal() + .pending_protocol_version() .await .context("failed getting pending protocol version")?; diff --git a/core/lib/zksync_core/src/state_keeper/metrics.rs b/core/node/state_keeper/src/metrics.rs similarity index 98% rename from core/lib/zksync_core/src/state_keeper/metrics.rs rename to core/node/state_keeper/src/metrics.rs index df8d83606915..11ae07eef054 100644 --- a/core/lib/zksync_core/src/state_keeper/metrics.rs +++ b/core/node/state_keeper/src/metrics.rs @@ -18,14 +18,14 @@ use super::seal_criteria::SealResolution; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "stage", rename_all = "snake_case")] -pub(crate) enum TxExecutionStage { +pub enum TxExecutionStage { Execution, TxRollback, } #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "tx_execution_type", rename_all = "snake_case")] -pub(crate) enum TxExecutionType { +pub enum TxExecutionType { L1, L2, } @@ -47,7 +47,7 @@ const INCLUSION_DELAY_BUCKETS: Buckets = Buckets::values(&[ /// General-purpose state keeper metrics. #[derive(Debug, Metrics)] #[metrics(prefix = "server_state_keeper")] -pub(crate) struct StateKeeperMetrics { +pub struct StateKeeperMetrics { /// Latency to synchronize the mempool with Postgres. #[metrics(buckets = Buckets::LATENCIES)] pub mempool_sync: Histogram, @@ -78,7 +78,7 @@ pub(crate) struct StateKeeperMetrics { } #[vise::register] -pub(crate) static KEEPER_METRICS: vise::Global = vise::Global::new(); +pub static KEEPER_METRICS: vise::Global = vise::Global::new(); /// State keeper-related gauges exposed via a collector. #[derive(Debug, Metrics)] diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs b/core/node/state_keeper/src/seal_criteria/conditional_sealer.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs rename to core/node/state_keeper/src/seal_criteria/conditional_sealer.rs index cc383ae16cb6..d29e66cd2b5b 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs +++ b/core/node/state_keeper/src/seal_criteria/conditional_sealer.rs @@ -121,7 +121,7 @@ impl SequencerSealer { } #[cfg(test)] - pub(in crate::state_keeper) fn with_sealers( + pub(crate) fn with_sealers( config: StateKeeperConfig, sealers: Vec>, ) -> Self { diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas.rs b/core/node/state_keeper/src/seal_criteria/criteria/gas.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas.rs rename to core/node/state_keeper/src/seal_criteria/criteria/gas.rs index b96d23a891b6..6677915e4e1f 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/gas.rs @@ -1,6 +1,6 @@ use zksync_types::ProtocolVersionId; -use crate::state_keeper::{ +use crate::{ seal_criteria::{SealCriterion, SealData, SealResolution, StateKeeperConfig}, utils::new_block_gas_count, }; diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas_for_batch_tip.rs b/core/node/state_keeper/src/seal_criteria/criteria/gas_for_batch_tip.rs similarity index 96% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas_for_batch_tip.rs rename to core/node/state_keeper/src/seal_criteria/criteria/gas_for_batch_tip.rs index a0a3d7ca6640..ff655880185b 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/gas_for_batch_tip.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/gas_for_batch_tip.rs @@ -1,9 +1,7 @@ use multivm::utils::gas_bootloader_batch_tip_overhead; use zksync_types::ProtocolVersionId; -use crate::state_keeper::seal_criteria::{ - SealCriterion, SealData, SealResolution, StateKeeperConfig, -}; +use crate::seal_criteria::{SealCriterion, SealData, SealResolution, StateKeeperConfig}; /// Checks whether we should exclude the transaction because we don't have enough gas for batch tip. #[derive(Debug)] diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs b/core/node/state_keeper/src/seal_criteria/criteria/geometry_seal_criteria.rs similarity index 98% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs rename to core/node/state_keeper/src/seal_criteria/criteria/geometry_seal_criteria.rs index c7166de9102d..91a7ce148cbb 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/geometry_seal_criteria.rs @@ -5,7 +5,7 @@ use zksync_config::configs::chain::StateKeeperConfig; use zksync_types::ProtocolVersionId; // Local uses -use crate::state_keeper::seal_criteria::{SealCriterion, SealData, SealResolution}; +use crate::seal_criteria::{SealCriterion, SealData, SealResolution}; // Collected vm execution metrics should fit into geometry limits. // Otherwise witness generation will fail and proof won't be generated. diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs b/core/node/state_keeper/src/seal_criteria/criteria/mod.rs similarity index 89% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs rename to core/node/state_keeper/src/seal_criteria/criteria/mod.rs index 4772de5c2650..6732a8cd2717 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/mod.rs @@ -5,7 +5,7 @@ mod pubdata_bytes; mod slots; mod tx_encoding_size; -pub(in crate::state_keeper) use self::{ +pub(crate) use self::{ gas::GasCriterion, gas_for_batch_tip::GasForBatchTipCriterion, geometry_seal_criteria::CircuitsCriterion, pubdata_bytes::PubDataBytesCriterion, slots::SlotsCriterion, tx_encoding_size::TxEncodingSizeCriterion, diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs b/core/node/state_keeper/src/seal_criteria/criteria/pubdata_bytes.rs similarity index 97% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs rename to core/node/state_keeper/src/seal_criteria/criteria/pubdata_bytes.rs index b6d6b3b94d2e..2e17bbb6d77c 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/pubdata_bytes.rs @@ -1,9 +1,7 @@ use multivm::utils::execution_metrics_bootloader_batch_tip_overhead; use zksync_types::ProtocolVersionId; -use crate::state_keeper::seal_criteria::{ - SealCriterion, SealData, SealResolution, StateKeeperConfig, -}; +use crate::seal_criteria::{SealCriterion, SealData, SealResolution, StateKeeperConfig}; #[derive(Debug)] pub struct PubDataBytesCriterion { diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs b/core/node/state_keeper/src/seal_criteria/criteria/slots.rs similarity index 95% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs rename to core/node/state_keeper/src/seal_criteria/criteria/slots.rs index 41d99b8274b7..6178f9e824d8 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/slots.rs @@ -1,9 +1,7 @@ use multivm::utils::get_bootloader_max_txs_in_batch; use zksync_types::ProtocolVersionId; -use crate::state_keeper::seal_criteria::{ - SealCriterion, SealData, SealResolution, StateKeeperConfig, -}; +use crate::seal_criteria::{SealCriterion, SealData, SealResolution, StateKeeperConfig}; /// Checks whether we should seal the block because we've run out of transaction slots. #[derive(Debug)] diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs b/core/node/state_keeper/src/seal_criteria/criteria/tx_encoding_size.rs similarity index 97% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs rename to core/node/state_keeper/src/seal_criteria/criteria/tx_encoding_size.rs index 02683e501d9b..03c2c3e14c84 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs +++ b/core/node/state_keeper/src/seal_criteria/criteria/tx_encoding_size.rs @@ -1,9 +1,7 @@ use multivm::utils::get_bootloader_encoding_space; use zksync_types::ProtocolVersionId; -use crate::state_keeper::seal_criteria::{ - SealCriterion, SealData, SealResolution, StateKeeperConfig, -}; +use crate::seal_criteria::{SealCriterion, SealData, SealResolution, StateKeeperConfig}; #[derive(Debug)] pub struct TxEncodingSizeCriterion; diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs b/core/node/state_keeper/src/seal_criteria/mod.rs similarity index 98% rename from core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs rename to core/node/state_keeper/src/seal_criteria/mod.rs index da2f22e8998a..51ad1c4ad906 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs +++ b/core/node/state_keeper/src/seal_criteria/mod.rs @@ -91,7 +91,7 @@ pub struct SealData { impl SealData { /// Creates sealing data based on the execution of a `transaction`. Assumes that all writes /// performed by the transaction are initial. - pub(crate) fn for_transaction( + pub fn for_transaction( transaction: &Transaction, tx_metrics: &TransactionExecutionMetrics, protocol_version: ProtocolVersionId, @@ -203,9 +203,7 @@ mod tests { use zksync_utils::time::seconds_since_epoch; use super::*; - use crate::state_keeper::tests::{ - create_execution_result, create_transaction, create_updates_manager, - }; + use crate::tests::{create_execution_result, create_transaction, create_updates_manager}; fn apply_tx_to_manager(tx: Transaction, manager: &mut UpdatesManager) { manager.extend_from_executed_transaction( diff --git a/core/lib/zksync_core/src/state_keeper/state_keeper_storage.rs b/core/node/state_keeper/src/state_keeper_storage.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/state_keeper_storage.rs rename to core/node/state_keeper/src/state_keeper_storage.rs diff --git a/core/node/state_keeper/src/testonly/mod.rs b/core/node/state_keeper/src/testonly/mod.rs new file mode 100644 index 000000000000..56c8a773c473 --- /dev/null +++ b/core/node/state_keeper/src/testonly/mod.rs @@ -0,0 +1,101 @@ +//! Test utilities that can be used for testing sequencer that may +//! be useful outside of this crate. + +use async_trait::async_trait; +use multivm::{ + interface::{ + CurrentExecutionState, ExecutionResult, FinishedL1Batch, L1BatchEnv, Refunds, SystemEnv, + VmExecutionResultAndLogs, VmExecutionStatistics, + }, + vm_latest::VmExecutionLogs, +}; +use once_cell::sync::Lazy; +use tokio::sync::{mpsc, watch}; +use zksync_contracts::BaseSystemContracts; + +use crate::{ + batch_executor::{BatchExecutor, BatchExecutorHandle, Command, TxExecutionResult}, + types::ExecutionMetricsForCriteria, +}; + +pub mod test_batch_executor; + +pub(super) static BASE_SYSTEM_CONTRACTS: Lazy = + Lazy::new(BaseSystemContracts::load_from_disk); + +pub(super) fn default_vm_batch_result() -> FinishedL1Batch { + FinishedL1Batch { + block_tip_execution_result: VmExecutionResultAndLogs { + result: ExecutionResult::Success { output: vec![] }, + logs: VmExecutionLogs::default(), + statistics: VmExecutionStatistics::default(), + refunds: Refunds::default(), + }, + final_execution_state: CurrentExecutionState { + events: vec![], + deduplicated_storage_log_queries: vec![], + used_contract_hashes: vec![], + user_l2_to_l1_logs: vec![], + system_logs: vec![], + total_log_queries: 0, + cycles_used: 0, + deduplicated_events_logs: vec![], + storage_refunds: Vec::new(), + pubdata_costs: Vec::new(), + }, + final_bootloader_memory: Some(vec![]), + pubdata_input: Some(vec![]), + initially_written_slots: Some(vec![]), + } +} + +/// Creates a `TxExecutionResult` object denoting a successful tx execution. +pub(crate) fn successful_exec() -> TxExecutionResult { + TxExecutionResult::Success { + tx_result: Box::new(VmExecutionResultAndLogs { + result: ExecutionResult::Success { output: vec![] }, + logs: Default::default(), + statistics: Default::default(), + refunds: Default::default(), + }), + tx_metrics: Box::new(ExecutionMetricsForCriteria { + l1_gas: Default::default(), + execution_metrics: Default::default(), + }), + compressed_bytecodes: vec![], + call_tracer_result: vec![], + gas_remaining: Default::default(), + } +} + +/// `BatchExecutor` which doesn't check anything at all. Accepts all transactions. +#[derive(Debug)] +pub struct MockBatchExecutor; + +#[async_trait] +impl BatchExecutor for MockBatchExecutor { + async fn init_batch( + &mut self, + _l1batch_params: L1BatchEnv, + _system_env: SystemEnv, + _stop_receiver: &watch::Receiver, + ) -> Option { + let (send, recv) = mpsc::channel(1); + let handle = tokio::task::spawn(async { + let mut recv = recv; + while let Some(cmd) = recv.recv().await { + match cmd { + Command::ExecuteTx(_, resp) => resp.send(successful_exec()).unwrap(), + Command::StartNextL2Block(_, resp) => resp.send(()).unwrap(), + Command::RollbackLastTx(_) => panic!("unexpected rollback"), + Command::FinishBatch(resp) => { + // Blanket result, it doesn't really matter. + resp.send(default_vm_batch_result()).unwrap(); + return; + } + } + } + }); + Some(BatchExecutorHandle::from_raw(handle, send)) + } +} diff --git a/core/lib/zksync_core/src/state_keeper/tests/tester.rs b/core/node/state_keeper/src/testonly/test_batch_executor.rs similarity index 92% rename from core/lib/zksync_core/src/state_keeper/tests/tester.rs rename to core/node/state_keeper/src/testonly/test_batch_executor.rs index ef936a956f5b..54ae71d406f9 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/node/state_keeper/src/testonly/test_batch_executor.rs @@ -1,3 +1,9 @@ +// TODO(QIT-33): Some of the interfaces are public, and some are only used in tests within this crate. +// This causes crate-local interfaces to spawn a warning without `cfg(test)`. The interfaces here must +// be revisited and properly split into "truly public" (e.g. useful for other crates to test, say, different +// IO or BatchExecutor implementations) and "local-test-only" (e.g. used only in tests within this crate). +#![allow(dead_code)] + use std::{ collections::{HashMap, HashSet, VecDeque}, convert::TryInto, @@ -8,30 +14,28 @@ use std::{ use async_trait::async_trait; use multivm::{ - interface::{ - ExecutionResult, L1BatchEnv, SystemEnv, TxExecutionMode, VmExecutionResultAndLogs, - }, + interface::{ExecutionResult, L1BatchEnv, SystemEnv, VmExecutionResultAndLogs}, vm_latest::constants::BATCH_COMPUTATIONAL_GAS_LIMIT, }; use tokio::sync::{mpsc, watch}; use zksync_contracts::BaseSystemContracts; use zksync_node_test_utils::create_l2_transaction; use zksync_types::{ - block::L2BlockExecutionData, fee_model::BatchFeeInput, protocol_upgrade::ProtocolUpgradeTx, - Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, Transaction, H256, + fee_model::BatchFeeInput, protocol_upgrade::ProtocolUpgradeTx, Address, L1BatchNumber, + L2BlockNumber, L2ChainId, ProtocolVersionId, Transaction, H256, }; -use crate::state_keeper::{ +use crate::{ batch_executor::{BatchExecutor, BatchExecutorHandle, Command, TxExecutionResult}, io::{IoCursor, L1BatchParams, L2BlockParams, PendingBatchData, StateKeeperIO}, seal_criteria::{IoSealCriteria, SequencerSealer}, - tests::{default_l1_batch_env, default_vm_batch_result, BASE_SYSTEM_CONTRACTS}, + testonly::{default_vm_batch_result, successful_exec, BASE_SYSTEM_CONTRACTS}, types::ExecutionMetricsForCriteria, updates::UpdatesManager, OutputHandler, StateKeeperOutputHandler, ZkSyncStateKeeper, }; -const FEE_ACCOUNT: Address = Address::repeat_byte(0x11); +pub const FEE_ACCOUNT: Address = Address::repeat_byte(0x11); /// Main entry for writing tests for the state keeper. /// Represents a planned sequence of actions that would happen with the state keeper. @@ -244,25 +248,6 @@ pub(crate) fn random_upgrade_tx(tx_number: u64) -> ProtocolUpgradeTx { tx } -/// Creates a `TxExecutionResult` object denoting a successful tx execution. -pub(crate) fn successful_exec() -> TxExecutionResult { - TxExecutionResult::Success { - tx_result: Box::new(VmExecutionResultAndLogs { - result: ExecutionResult::Success { output: vec![] }, - logs: Default::default(), - statistics: Default::default(), - refunds: Default::default(), - }), - tx_metrics: Box::new(ExecutionMetricsForCriteria { - l1_gas: Default::default(), - execution_metrics: Default::default(), - }), - compressed_bytecodes: vec![], - call_tracer_result: vec![], - gas_remaining: Default::default(), - } -} - /// Creates a `TxExecutionResult` object denoting a successful tx execution with the given execution metrics. pub(crate) fn successful_exec_with_metrics( tx_metrics: ExecutionMetricsForCriteria, @@ -288,23 +273,6 @@ pub(crate) fn rejected_exec() -> TxExecutionResult { } } -/// Creates a mock `PendingBatchData` object containing the provided sequence of L2 blocks. -pub(crate) fn pending_batch_data(pending_l2_blocks: Vec) -> PendingBatchData { - PendingBatchData { - l1_batch_env: default_l1_batch_env(1, 1, FEE_ACCOUNT), - system_env: SystemEnv { - zk_porter_available: false, - version: ProtocolVersionId::latest(), - base_system_smart_contracts: BASE_SYSTEM_CONTRACTS.clone(), - bootloader_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, - execution_mode: TxExecutionMode::VerifyExecute, - default_validation_computational_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, - chain_id: L2ChainId::from(270), - }, - pending_l2_blocks, - } -} - #[allow(clippy::type_complexity, clippy::large_enum_variant)] // It's OK for tests. enum ScenarioItem { /// Configures scenario to repeatedly return `None` to tx requests until the next action from the scenario happens. @@ -363,7 +331,7 @@ impl fmt::Debug for ScenarioItem { type ExpectedTransactions = VecDeque>>; #[derive(Debug, Default)] -pub(crate) struct TestBatchExecutorBuilder { +pub struct TestBatchExecutorBuilder { /// Sequence of known transaction execution results per batch. /// We need to store txs for each batch separately, since the same transaction /// can be executed in several batches (e.g. after an `ExcludeAndSeal` rollback). @@ -374,7 +342,7 @@ pub(crate) struct TestBatchExecutorBuilder { } impl TestBatchExecutorBuilder { - pub(super) fn new(scenario: &TestScenario) -> Self { + pub(crate) fn new(scenario: &TestScenario) -> Self { let mut txs = VecDeque::new(); let mut batch_txs = HashMap::new(); let mut rollback_set = HashSet::new(); @@ -429,7 +397,7 @@ impl TestBatchExecutorBuilder { } /// Adds successful transactions to be executed in a single L1 batch. - pub(crate) fn push_successful_transactions(&mut self, tx_hashes: &[H256]) { + pub fn push_successful_transactions(&mut self, tx_hashes: &[H256]) { let txs = tx_hashes .iter() .copied() @@ -576,7 +544,7 @@ impl StateKeeperOutputHandler for TestPersistence { } } -pub(super) struct TestIO { +pub(crate) struct TestIO { stop_sender: Arc>, batch_number: L1BatchNumber, timestamp: u64, @@ -602,7 +570,7 @@ impl fmt::Debug for TestIO { } impl TestIO { - pub(super) fn new( + pub(crate) fn new( stop_sender: watch::Sender, scenario: TestScenario, ) -> (Self, OutputHandler) { @@ -644,7 +612,7 @@ impl TestIO { (this, OutputHandler::new(Box::new(persistence))) } - pub(super) fn add_upgrade_tx(&mut self, version: ProtocolVersionId, tx: ProtocolUpgradeTx) { + pub fn add_upgrade_tx(&mut self, version: ProtocolVersionId, tx: ProtocolUpgradeTx) { self.protocol_upgrade_txs.insert(version, tx); } diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/node/state_keeper/src/tests/mod.rs similarity index 93% rename from core/lib/zksync_core/src/state_keeper/tests/mod.rs rename to core/node/state_keeper/src/tests/mod.rs index 3c4f71260708..2b347c0629ea 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/node/state_keeper/src/tests/mod.rs @@ -8,17 +8,14 @@ use std::{ use multivm::{ interface::{ - CurrentExecutionState, ExecutionResult, FinishedL1Batch, L1BatchEnv, L2BlockEnv, Refunds, - SystemEnv, TxExecutionMode, VmExecutionResultAndLogs, VmExecutionStatistics, + ExecutionResult, L1BatchEnv, L2BlockEnv, Refunds, SystemEnv, TxExecutionMode, + VmExecutionResultAndLogs, VmExecutionStatistics, }, vm_latest::{constants::BATCH_COMPUTATIONAL_GAS_LIMIT, VmExecutionLogs}, }; -use once_cell::sync::Lazy; use tokio::sync::watch; use zksync_config::configs::chain::StateKeeperConfig; -use zksync_contracts::BaseSystemContracts; use zksync_node_test_utils::create_l2_transaction; -use zksync_system_constants::ZKPORTER_IS_AVAILABLE; use zksync_types::{ aggregated_operations::AggregatedActionType, block::{BlockGasCount, L2BlockExecutionData, L2BlockHasher}, @@ -26,31 +23,47 @@ use zksync_types::{ tx::tx_execution_info::ExecutionMetrics, zk_evm_types::{LogQuery, Timestamp}, Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, StorageLogQuery, - StorageLogQueryType, Transaction, H256, U256, + StorageLogQueryType, Transaction, H256, U256, ZKPORTER_IS_AVAILABLE, }; -mod tester; - -use self::tester::{ - pending_batch_data, random_tx, random_upgrade_tx, rejected_exec, successful_exec, - successful_exec_with_metrics, TestIO, TestScenario, -}; -pub(crate) use self::tester::{MockBatchExecutor, TestBatchExecutorBuilder}; -use crate::state_keeper::{ +use crate::{ batch_executor::TxExecutionResult, + io::PendingBatchData, keeper::POLL_WAIT_DURATION, seal_criteria::{ criteria::{GasCriterion, SlotsCriterion}, SequencerSealer, }, + testonly::{ + successful_exec, + test_batch_executor::{ + random_tx, random_upgrade_tx, rejected_exec, successful_exec_with_metrics, + TestBatchExecutorBuilder, TestIO, TestScenario, FEE_ACCOUNT, + }, + BASE_SYSTEM_CONTRACTS, + }, types::ExecutionMetricsForCriteria, updates::UpdatesManager, utils::l1_batch_base_cost, ZkSyncStateKeeper, }; -pub(super) static BASE_SYSTEM_CONTRACTS: Lazy = - Lazy::new(BaseSystemContracts::load_from_disk); +/// Creates a mock `PendingBatchData` object containing the provided sequence of L2 blocks. +pub(crate) fn pending_batch_data(pending_l2_blocks: Vec) -> PendingBatchData { + PendingBatchData { + l1_batch_env: default_l1_batch_env(1, 1, FEE_ACCOUNT), + system_env: SystemEnv { + zk_porter_available: false, + version: ProtocolVersionId::latest(), + base_system_smart_contracts: BASE_SYSTEM_CONTRACTS.clone(), + bootloader_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, + execution_mode: TxExecutionMode::VerifyExecute, + default_validation_computational_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, + chain_id: L2ChainId::from(270), + }, + pending_l2_blocks, + } +} pub(super) fn default_system_env() -> SystemEnv { SystemEnv { @@ -89,32 +102,6 @@ pub(super) fn default_l1_batch_env( } } -pub(super) fn default_vm_batch_result() -> FinishedL1Batch { - FinishedL1Batch { - block_tip_execution_result: VmExecutionResultAndLogs { - result: ExecutionResult::Success { output: vec![] }, - logs: VmExecutionLogs::default(), - statistics: VmExecutionStatistics::default(), - refunds: Refunds::default(), - }, - final_execution_state: CurrentExecutionState { - events: vec![], - deduplicated_storage_log_queries: vec![], - used_contract_hashes: vec![], - user_l2_to_l1_logs: vec![], - system_logs: vec![], - total_log_queries: 0, - cycles_used: 0, - deduplicated_events_logs: vec![], - storage_refunds: Vec::new(), - pubdata_costs: Vec::new(), - }, - final_bootloader_memory: Some(vec![]), - pubdata_input: Some(vec![]), - initially_written_slots: Some(vec![]), - } -} - pub(super) fn create_updates_manager() -> UpdatesManager { let l1_batch_env = default_l1_batch_env(1, 1, Address::default()); UpdatesManager::new(&l1_batch_env, &default_system_env()) diff --git a/core/lib/zksync_core/src/state_keeper/types.rs b/core/node/state_keeper/src/types.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/types.rs rename to core/node/state_keeper/src/types.rs diff --git a/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs b/core/node/state_keeper/src/updates/l1_batch_updates.rs similarity index 96% rename from core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs rename to core/node/state_keeper/src/updates/l1_batch_updates.rs index 4f8c0cc4b0e7..6becfae2b7ac 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/l1_batch_updates.rs +++ b/core/node/state_keeper/src/updates/l1_batch_updates.rs @@ -6,7 +6,7 @@ use zksync_types::{ ExecuteTransactionCommon, L1BatchNumber, }; -use crate::state_keeper::{updates::l2_block_updates::L2BlockUpdates, utils::new_block_gas_count}; +use crate::{updates::l2_block_updates::L2BlockUpdates, utils::new_block_gas_count}; #[derive(Debug)] pub struct L1BatchUpdates { @@ -55,7 +55,7 @@ mod tests { use zksync_types::{L2BlockNumber, ProtocolVersionId, H256}; use super::*; - use crate::state_keeper::{ + use crate::{ tests::{create_execution_result, create_transaction}, utils::new_block_gas_count, }; diff --git a/core/lib/zksync_core/src/state_keeper/updates/l2_block_updates.rs b/core/node/state_keeper/src/updates/l2_block_updates.rs similarity index 98% rename from core/lib/zksync_core/src/state_keeper/updates/l2_block_updates.rs rename to core/node/state_keeper/src/updates/l2_block_updates.rs index 5c1ee62bba3f..a74d94be30ef 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/l2_block_updates.rs +++ b/core/node/state_keeper/src/updates/l2_block_updates.rs @@ -174,7 +174,7 @@ mod tests { use multivm::vm_latest::TransactionVmExt; use super::*; - use crate::state_keeper::tests::{create_execution_result, create_transaction}; + use crate::tests::{create_execution_result, create_transaction}; #[test] fn apply_empty_l2_tx() { diff --git a/core/lib/zksync_core/src/state_keeper/updates/mod.rs b/core/node/state_keeper/src/updates/mod.rs similarity index 99% rename from core/lib/zksync_core/src/state_keeper/updates/mod.rs rename to core/node/state_keeper/src/updates/mod.rs index bb28b262f4be..ce35dc98f033 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/mod.rs +++ b/core/node/state_keeper/src/updates/mod.rs @@ -16,7 +16,7 @@ use super::{ io::{IoCursor, L2BlockParams}, metrics::BATCH_TIP_METRICS, }; -use crate::state_keeper::types::ExecutionMetricsForCriteria; +use crate::types::ExecutionMetricsForCriteria; pub mod l1_batch_updates; pub mod l2_block_updates; @@ -199,7 +199,7 @@ pub(crate) struct L2BlockSealCommand { #[cfg(test)] mod tests { use super::*; - use crate::state_keeper::{ + use crate::{ tests::{create_execution_result, create_transaction, create_updates_manager}, utils::new_block_gas_count, }; diff --git a/core/lib/zksync_core/src/state_keeper/utils.rs b/core/node/state_keeper/src/utils.rs similarity index 100% rename from core/lib/zksync_core/src/state_keeper/utils.rs rename to core/node/state_keeper/src/utils.rs From 326a0cc8c77978cf074842837e36b16f4b46d521 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Fri, 10 May 2024 16:12:13 +0400 Subject: [PATCH 02/15] Move binary search to reorg detector --- .../lib/zksync_core/src/reorg_detector/mod.rs | 41 ++++++++++++++- .../zksync_core/src/reorg_detector/tests.rs | 9 ++++ core/lib/zksync_core/src/utils/mod.rs | 50 +------------------ 3 files changed, 49 insertions(+), 51 deletions(-) diff --git a/core/lib/zksync_core/src/reorg_detector/mod.rs b/core/lib/zksync_core/src/reorg_detector/mod.rs index 9561649c0c8c..5f6353ff685b 100644 --- a/core/lib/zksync_core/src/reorg_detector/mod.rs +++ b/core/lib/zksync_core/src/reorg_detector/mod.rs @@ -2,6 +2,7 @@ use std::{fmt, time::Duration}; use anyhow::Context as _; use async_trait::async_trait; +use futures::Future; use tokio::sync::watch; use zksync_dal::{ConnectionPool, Core, CoreDal, DalError}; use zksync_health_check::{Health, HealthStatus, HealthUpdater, ReactiveHealthCheck}; @@ -13,8 +14,6 @@ use zksync_web3_decl::{ namespaces::{EthNamespaceClient, ZksNamespaceClient}, }; -use crate::utils::binary_search_with; - #[cfg(test)] mod tests; @@ -408,3 +407,41 @@ impl ReorgDetector { Ok(()) } } + +/// Fallible and async predicate for binary search. +#[async_trait] +trait BinarySearchPredicate: Send { + type Error; + + async fn eval(&mut self, argument: u32) -> Result; +} + +#[async_trait] +impl BinarySearchPredicate for F +where + F: Send + FnMut(u32) -> Fut, + Fut: Send + Future>, +{ + type Error = E; + + async fn eval(&mut self, argument: u32) -> Result { + self(argument).await + } +} + +/// Finds the greatest `u32` value for which `f` returns `true`. +async fn binary_search_with( + mut left: u32, + mut right: u32, + mut predicate: P, +) -> Result { + while left + 1 < right { + let middle = (left + right) / 2; + if predicate.eval(middle).await? { + left = middle; + } else { + right = middle; + } + } + Ok(left) +} diff --git a/core/lib/zksync_core/src/reorg_detector/tests.rs b/core/lib/zksync_core/src/reorg_detector/tests.rs index 011ab62bcd99..e58fdde4ac59 100644 --- a/core/lib/zksync_core/src/reorg_detector/tests.rs +++ b/core/lib/zksync_core/src/reorg_detector/tests.rs @@ -19,6 +19,15 @@ use zksync_web3_decl::jsonrpsee::core::ClientError as RpcError; use super::*; +#[tokio::test] +async fn test_binary_search() { + for divergence_point in [1, 50, 51, 100] { + let mut f = |x| async move { Ok::<_, ()>(x < divergence_point) }; + let result = binary_search_with(0, 100, &mut f).await; + assert_eq!(result, Ok(divergence_point - 1)); + } +} + async fn store_l2_block(storage: &mut Connection<'_, Core>, number: u32, hash: H256) { let header = L2BlockHeader { hash, diff --git a/core/lib/zksync_core/src/utils/mod.rs b/core/lib/zksync_core/src/utils/mod.rs index ea182a9fe235..29b462c10179 100644 --- a/core/lib/zksync_core/src/utils/mod.rs +++ b/core/lib/zksync_core/src/utils/mod.rs @@ -1,52 +1,13 @@ //! Miscellaneous utils used by multiple components. -use std::{future::Future, time::Duration}; +use std::time::Duration; -use async_trait::async_trait; use tokio::sync::watch; use zksync_config::configs::chain::L1BatchCommitDataGeneratorMode; use zksync_dal::{Connection, ConnectionPool, Core, CoreDal}; use zksync_eth_client::{CallFunctionArgs, ClientError, Error as EthClientError, EthInterface}; use zksync_types::{Address, L1BatchNumber}; -/// Fallible and async predicate for binary search. -#[async_trait] -pub(crate) trait BinarySearchPredicate: Send { - type Error; - - async fn eval(&mut self, argument: u32) -> Result; -} - -#[async_trait] -impl BinarySearchPredicate for F -where - F: Send + FnMut(u32) -> Fut, - Fut: Send + Future>, -{ - type Error = E; - - async fn eval(&mut self, argument: u32) -> Result { - self(argument).await - } -} - -/// Finds the greatest `u32` value for which `f` returns `true`. -pub(crate) async fn binary_search_with( - mut left: u32, - mut right: u32, - mut predicate: P, -) -> Result { - while left + 1 < right { - let middle = (left + right) / 2; - if predicate.eval(middle).await? { - left = middle; - } else { - right = middle; - } - } - Ok(left) -} - /// Repeatedly polls the DB until there is an L1 batch. We may not have such a batch initially /// if the DB is recovered from an application-level snapshot. /// @@ -175,15 +136,6 @@ mod tests { use super::*; - #[tokio::test] - async fn test_binary_search() { - for divergence_point in [1, 50, 51, 100] { - let mut f = |x| async move { Ok::<_, ()>(x < divergence_point) }; - let result = binary_search_with(0, 100, &mut f).await; - assert_eq!(result, Ok(divergence_point - 1)); - } - } - #[tokio::test] async fn waiting_for_l1_batch_success() { let pool = ConnectionPool::::test_pool().await; From 60fc06b6fb2fdc9b850f58de7115a2ff6c4f636c Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Tue, 14 May 2024 15:37:14 +0400 Subject: [PATCH 03/15] Get rid of most utils --- core/lib/dal/src/helpers.rs | 98 +++++++++++++++ core/lib/dal/src/lib.rs | 1 + .../zksync_core/src/api_server/web3/mod.rs | 3 +- .../src/consistency_checker/mod.rs | 35 +++++- .../src/metadata_calculator/updater.rs | 3 +- .../sync_layer/batch_status_updater/mod.rs | 13 +- core/lib/zksync_core/src/utils/mod.rs | 119 +----------------- 7 files changed, 147 insertions(+), 125 deletions(-) create mode 100644 core/lib/dal/src/helpers.rs diff --git a/core/lib/dal/src/helpers.rs b/core/lib/dal/src/helpers.rs new file mode 100644 index 000000000000..d584a14270e4 --- /dev/null +++ b/core/lib/dal/src/helpers.rs @@ -0,0 +1,98 @@ +//! Various helpers for using DAL methods. + +use std::time::Duration; + +use tokio::sync::watch; +use zksync_types::L1BatchNumber; + +use crate::{ConnectionPool, Core, CoreDal}; + +/// Repeatedly polls the DB until there is an L1 batch. We may not have such a batch initially +/// if the DB is recovered from an application-level snapshot. +/// +/// Returns the number of the *earliest* L1 batch, or `None` if the stop signal is received. +pub async fn wait_for_l1_batch( + pool: &ConnectionPool, + poll_interval: Duration, + stop_receiver: &mut watch::Receiver, +) -> anyhow::Result> { + tracing::debug!("Waiting for at least one L1 batch in db in DB"); + loop { + if *stop_receiver.borrow() { + return Ok(None); + } + + let mut storage = pool.connection().await?; + let sealed_l1_batch_number = storage.blocks_dal().get_earliest_l1_batch_number().await?; + drop(storage); + + if let Some(number) = sealed_l1_batch_number { + return Ok(Some(number)); + } + + // We don't check the result: if a stop signal is received, we'll return at the start + // of the next iteration. + tokio::time::timeout(poll_interval, stop_receiver.changed()) + .await + .ok(); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use zksync_contracts::BaseSystemContractsHashes; + use zksync_types::{block::L1BatchHeader, ProtocolVersion, ProtocolVersionId, H256}; + + use crate::{ConnectionPool, Core, CoreDal}; + + #[tokio::test] + async fn waiting_for_l1_batch_success() { + let pool = ConnectionPool::::test_pool().await; + let (_stop_sender, mut stop_receiver) = watch::channel(false); + + let pool_copy = pool.clone(); + tokio::spawn(async move { + tokio::time::sleep(Duration::from_millis(25)).await; + let mut conn = pool_copy.connection().await.unwrap(); + conn.protocol_versions_dal() + .save_protocol_version_with_tx(&ProtocolVersion::default()) + .await + .unwrap(); + let header = L1BatchHeader::new( + L1BatchNumber(0), + 100, + BaseSystemContractsHashes { + bootloader: H256::repeat_byte(1), + default_aa: H256::repeat_byte(42), + }, + ProtocolVersionId::latest(), + ); + conn.blocks_dal() + .insert_mock_l1_batch(&header) + .await + .unwrap(); + }); + + let l1_batch = wait_for_l1_batch(&pool, Duration::from_millis(10), &mut stop_receiver) + .await + .unwrap(); + assert_eq!(l1_batch, Some(L1BatchNumber(0))); + } + + #[tokio::test] + async fn waiting_for_l1_batch_cancellation() { + let pool = ConnectionPool::::test_pool().await; + let (stop_sender, mut stop_receiver) = watch::channel(false); + + tokio::spawn(async move { + tokio::time::sleep(Duration::from_millis(25)).await; + stop_sender.send_replace(true); + }); + + let l1_batch = wait_for_l1_batch(&pool, Duration::from_secs(30), &mut stop_receiver) + .await + .unwrap(); + assert_eq!(l1_batch, None); + } +} diff --git a/core/lib/dal/src/lib.rs b/core/lib/dal/src/lib.rs index 5a95f9688ff6..709e4fa76a50 100644 --- a/core/lib/dal/src/lib.rs +++ b/core/lib/dal/src/lib.rs @@ -34,6 +34,7 @@ pub mod eth_sender_dal; pub mod events_dal; pub mod events_web3_dal; pub mod factory_deps_dal; +pub mod helpers; pub mod metrics; mod models; pub mod proof_generation_dal; diff --git a/core/lib/zksync_core/src/api_server/web3/mod.rs b/core/lib/zksync_core/src/api_server/web3/mod.rs index 00050fb85812..ec1bc49fe0b3 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/mod.rs @@ -10,7 +10,7 @@ use tokio::{ }; use tower_http::{cors::CorsLayer, metrics::InFlightRequestsLayer}; use zksync_config::configs::api::{MaxResponseSize, MaxResponseSizeOverrides}; -use zksync_dal::{ConnectionPool, Core}; +use zksync_dal::{helpers::wait_for_l1_batch, ConnectionPool, Core}; use zksync_health_check::{HealthStatus, HealthUpdater, ReactiveHealthCheck}; use zksync_types::L2BlockNumber; use zksync_web3_decl::{ @@ -48,7 +48,6 @@ use crate::{ tx_sender::TxSender, }, sync_layer::SyncState, - utils::wait_for_l1_batch, }; pub mod backend_jsonrpsee; diff --git a/core/lib/zksync_core/src/consistency_checker/mod.rs b/core/lib/zksync_core/src/consistency_checker/mod.rs index 9b94a9c5ef67..a42a32baa39e 100644 --- a/core/lib/zksync_core/src/consistency_checker/mod.rs +++ b/core/lib/zksync_core/src/consistency_checker/mod.rs @@ -15,8 +15,6 @@ use zksync_types::{ L1BatchNumber, ProtocolVersionId, H256, U256, }; -use crate::utils::wait_for_l1_batch_with_metadata; - #[cfg(test)] mod tests; @@ -650,3 +648,36 @@ impl ConsistencyChecker { Ok(()) } } + +/// Repeatedly polls the DB until there is an L1 batch with metadata. We may not have such a batch initially +/// if the DB is recovered from an application-level snapshot. +/// +/// Returns the number of the *earliest* L1 batch with metadata, or `None` if the stop signal is received. +async fn wait_for_l1_batch_with_metadata( + pool: &ConnectionPool, + poll_interval: Duration, + stop_receiver: &mut watch::Receiver, +) -> anyhow::Result> { + loop { + if *stop_receiver.borrow() { + return Ok(None); + } + + let mut storage = pool.connection().await?; + let sealed_l1_batch_number = storage + .blocks_dal() + .get_earliest_l1_batch_number_with_metadata() + .await?; + drop(storage); + + if let Some(number) = sealed_l1_batch_number { + return Ok(Some(number)); + } + tracing::debug!( + "No L1 batches with metadata are present in DB; trying again in {poll_interval:?}" + ); + tokio::time::timeout(poll_interval, stop_receiver.changed()) + .await + .ok(); + } +} diff --git a/core/lib/zksync_core/src/metadata_calculator/updater.rs b/core/lib/zksync_core/src/metadata_calculator/updater.rs index b4f7d90117a2..9ae274ea94e8 100644 --- a/core/lib/zksync_core/src/metadata_calculator/updater.rs +++ b/core/lib/zksync_core/src/metadata_calculator/updater.rs @@ -5,7 +5,7 @@ use std::{ops, sync::Arc, time::Instant}; use anyhow::Context as _; use futures::{future, FutureExt}; use tokio::sync::watch; -use zksync_dal::{Connection, ConnectionPool, Core, CoreDal}; +use zksync_dal::{helpers::wait_for_l1_batch, Connection, ConnectionPool, Core, CoreDal}; use zksync_merkle_tree::domain::TreeMetadata; use zksync_object_store::ObjectStore; use zksync_types::{ @@ -18,7 +18,6 @@ use super::{ metrics::{TreeUpdateStage, METRICS}, MetadataCalculator, }; -use crate::utils::wait_for_l1_batch; #[derive(Debug)] pub(super) struct TreeUpdater { diff --git a/core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs b/core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs index 34a970e41217..f57e4c76d7c0 100644 --- a/core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs +++ b/core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs @@ -22,7 +22,6 @@ use zksync_web3_decl::{ }; use super::metrics::{FetchStage, FETCHER_METRICS}; -use crate::utils::projected_first_l1_batch; #[cfg(test)] mod tests; @@ -485,3 +484,15 @@ impl BatchStatusUpdater { Ok(()) } } + +/// Returns the projected number of the first locally available L1 batch. The L1 batch is **not** +/// guaranteed to be present in the storage! +async fn projected_first_l1_batch( + storage: &mut Connection<'_, Core>, +) -> anyhow::Result { + let snapshot_recovery = storage + .snapshot_recovery_dal() + .get_applied_snapshot_status() + .await?; + Ok(snapshot_recovery.map_or(L1BatchNumber(0), |recovery| recovery.l1_batch_number + 1)) +} diff --git a/core/lib/zksync_core/src/utils/mod.rs b/core/lib/zksync_core/src/utils/mod.rs index 29b462c10179..76ca38204269 100644 --- a/core/lib/zksync_core/src/utils/mod.rs +++ b/core/lib/zksync_core/src/utils/mod.rs @@ -1,88 +1,8 @@ //! Miscellaneous utils used by multiple components. -use std::time::Duration; - -use tokio::sync::watch; use zksync_config::configs::chain::L1BatchCommitDataGeneratorMode; -use zksync_dal::{Connection, ConnectionPool, Core, CoreDal}; use zksync_eth_client::{CallFunctionArgs, ClientError, Error as EthClientError, EthInterface}; -use zksync_types::{Address, L1BatchNumber}; - -/// Repeatedly polls the DB until there is an L1 batch. We may not have such a batch initially -/// if the DB is recovered from an application-level snapshot. -/// -/// Returns the number of the *earliest* L1 batch, or `None` if the stop signal is received. -pub(crate) async fn wait_for_l1_batch( - pool: &ConnectionPool, - poll_interval: Duration, - stop_receiver: &mut watch::Receiver, -) -> anyhow::Result> { - tracing::debug!("Waiting for at least one L1 batch in db in DB"); - loop { - if *stop_receiver.borrow() { - return Ok(None); - } - - let mut storage = pool.connection().await?; - let sealed_l1_batch_number = storage.blocks_dal().get_earliest_l1_batch_number().await?; - drop(storage); - - if let Some(number) = sealed_l1_batch_number { - return Ok(Some(number)); - } - - // We don't check the result: if a stop signal is received, we'll return at the start - // of the next iteration. - tokio::time::timeout(poll_interval, stop_receiver.changed()) - .await - .ok(); - } -} - -/// Repeatedly polls the DB until there is an L1 batch with metadata. We may not have such a batch initially -/// if the DB is recovered from an application-level snapshot. -/// -/// Returns the number of the *earliest* L1 batch with metadata, or `None` if the stop signal is received. -pub(crate) async fn wait_for_l1_batch_with_metadata( - pool: &ConnectionPool, - poll_interval: Duration, - stop_receiver: &mut watch::Receiver, -) -> anyhow::Result> { - loop { - if *stop_receiver.borrow() { - return Ok(None); - } - - let mut storage = pool.connection().await?; - let sealed_l1_batch_number = storage - .blocks_dal() - .get_earliest_l1_batch_number_with_metadata() - .await?; - drop(storage); - - if let Some(number) = sealed_l1_batch_number { - return Ok(Some(number)); - } - tracing::debug!( - "No L1 batches with metadata are present in DB; trying again in {poll_interval:?}" - ); - tokio::time::timeout(poll_interval, stop_receiver.changed()) - .await - .ok(); - } -} - -/// Returns the projected number of the first locally available L1 batch. The L1 batch is **not** -/// guaranteed to be present in the storage! -pub(crate) async fn projected_first_l1_batch( - storage: &mut Connection<'_, Core>, -) -> anyhow::Result { - let snapshot_recovery = storage - .snapshot_recovery_dal() - .get_applied_snapshot_status() - .await?; - Ok(snapshot_recovery.map_or(L1BatchNumber(0), |recovery| recovery.l1_batch_number + 1)) -} +use zksync_types::Address; async fn get_pubdata_pricing_mode( diamond_proxy_address: Address, @@ -131,47 +51,10 @@ mod tests { use jsonrpsee::types::ErrorObject; use zksync_eth_client::{clients::MockEthereum, ClientError}; - use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_types::{ethabi, U256}; use super::*; - #[tokio::test] - async fn waiting_for_l1_batch_success() { - let pool = ConnectionPool::::test_pool().await; - let (_stop_sender, mut stop_receiver) = watch::channel(false); - - let pool_copy = pool.clone(); - tokio::spawn(async move { - tokio::time::sleep(Duration::from_millis(25)).await; - let mut storage = pool_copy.connection().await.unwrap(); - insert_genesis_batch(&mut storage, &GenesisParams::mock()) - .await - .unwrap(); - }); - - let l1_batch = wait_for_l1_batch(&pool, Duration::from_millis(10), &mut stop_receiver) - .await - .unwrap(); - assert_eq!(l1_batch, Some(L1BatchNumber(0))); - } - - #[tokio::test] - async fn waiting_for_l1_batch_cancellation() { - let pool = ConnectionPool::::test_pool().await; - let (stop_sender, mut stop_receiver) = watch::channel(false); - - tokio::spawn(async move { - tokio::time::sleep(Duration::from_millis(25)).await; - stop_sender.send_replace(true); - }); - - let l1_batch = wait_for_l1_batch(&pool, Duration::from_secs(30), &mut stop_receiver) - .await - .unwrap(); - assert_eq!(l1_batch, None); - } - fn mock_ethereum(token: ethabi::Token, err: Option) -> MockEthereum { let err_mutex = Mutex::new(err); MockEthereum::default().with_fallible_call_handler(move |_, _| { From ba566e88a3513a00aaa4aeac8db6b7389cefea5f Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Tue, 14 May 2024 15:52:55 +0400 Subject: [PATCH 04/15] Extract reorg detector --- Cargo.lock | 22 +++++++++++++ Cargo.toml | 2 ++ core/bin/external_node/Cargo.toml | 1 + core/bin/external_node/src/main.rs | 4 +-- core/lib/zksync_core/src/lib.rs | 1 - core/node/reorg_detector/Cargo.toml | 31 +++++++++++++++++++ core/node/reorg_detector/README.md | 3 ++ .../mod.rs => node/reorg_detector/src/lib.rs} | 3 +- .../reorg_detector/src}/tests.rs | 0 9 files changed, 62 insertions(+), 5 deletions(-) create mode 100644 core/node/reorg_detector/Cargo.toml create mode 100644 core/node/reorg_detector/README.md rename core/{lib/zksync_core/src/reorg_detector/mod.rs => node/reorg_detector/src/lib.rs} (99%) rename core/{lib/zksync_core/src/reorg_detector => node/reorg_detector/src}/tests.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 468d2aa2f97b..4bd695a96a39 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8583,6 +8583,7 @@ dependencies = [ "zksync_node_genesis", "zksync_object_store", "zksync_protobuf_config", + "zksync_reorg_detector", "zksync_shared_metrics", "zksync_snapshots_applier", "zksync_state", @@ -8924,6 +8925,27 @@ dependencies = [ "zksync_utils", ] +[[package]] +name = "zksync_reorg_detector" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "async-trait", + "serde_json", + "test-casing", + "thiserror", + "tokio", + "tracing", + "zksync_dal", + "zksync_health_check", + "zksync_node_genesis", + "zksync_node_test_utils", + "zksync_shared_metrics", + "zksync_types", + "zksync_web3_decl", +] + [[package]] name = "zksync_server" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index ed55711bf47c..54ffa8da257f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,6 +24,7 @@ members = [ "core/node/vm_runner", "core/node/test_utils", "core/node/state_keeper", + "core/node/reorg_detector", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -242,3 +243,4 @@ zksync_node_fee_model = { path = "core/node/fee_model" } zksync_vm_runner = { path = "core/node/vm_runner" } zksync_node_test_utils = { path = "core/node/test_utils" } zksync_state_keeper = { path = "core/node/state_keeper" } +zksync_reorg_detector = { path = "core/node/reorg_detector" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index 2a6f40a15ff5..96fd04e0fb57 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -37,6 +37,7 @@ zksync_node_fee_model.workspace = true zksync_node_db_pruner.workspace = true zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true +zksync_reorg_detector.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 37e691e5622b..fbcea280177e 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -30,7 +30,6 @@ use zksync_core::{ consensus, consistency_checker::ConsistencyChecker, metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, - reorg_detector::{self, ReorgDetector}, setup_sigint_handler, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, SyncState, @@ -49,6 +48,7 @@ use zksync_eth_sender::l1_batch_commit_data_generator::{ use zksync_health_check::{AppHealthCheck, HealthStatus, ReactiveHealthCheck}; use zksync_node_db_pruner::{DbPruner, DbPrunerConfig}; use zksync_node_fee_model::l1_gas_price::MainNodeFeeParamsFetcher; +use zksync_reorg_detector::ReorgDetector; use zksync_state::{PostgresStorageCaches, RocksdbStorageOptions}; use zksync_state_keeper::{ seal_criteria::NoopSealer, AsyncRocksdbCache, BatchExecutor, MainBatchExecutor, OutputHandler, @@ -971,7 +971,7 @@ async fn run_node( // will be able to operate normally afterwards. match reorg_detector.check_consistency().await { Ok(()) => {} - Err(reorg_detector::Error::ReorgDetected(last_correct_l1_batch)) => { + Err(zksync_reorg_detector::Error::ReorgDetected(last_correct_l1_batch)) => { tracing::info!("Reverting to l1 batch number {last_correct_l1_batch}"); reverter.roll_back(last_correct_l1_batch).await?; tracing::info!("Revert successfully completed"); diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 2e96d229976e..f16b035b4c93 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -108,7 +108,6 @@ pub mod consensus; pub mod consistency_checker; pub mod metadata_calculator; pub mod proto; -pub mod reorg_detector; pub mod sync_layer; pub mod temp_config_store; pub mod utils; diff --git a/core/node/reorg_detector/Cargo.toml b/core/node/reorg_detector/Cargo.toml new file mode 100644 index 000000000000..75e2eb3c0ece --- /dev/null +++ b/core/node/reorg_detector/Cargo.toml @@ -0,0 +1,31 @@ +[package] +name = "zksync_reorg_detector" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_dal.workspace = true +zksync_health_check.workspace = true +zksync_shared_metrics.workspace = true +zksync_types.workspace = true +zksync_web3_decl.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +tokio = { workspace = true, features = ["time"] } +thiserror.workspace = true +tracing.workspace = true +serde_json.workspace = true + +[dev-dependencies] +assert_matches.workspace = true +test-casing.workspace = true + +zksync_node_genesis.workspace = true +zksync_node_test_utils.workspace = true diff --git a/core/node/reorg_detector/README.md b/core/node/reorg_detector/README.md new file mode 100644 index 000000000000..9e8cadc7f5cb --- /dev/null +++ b/core/node/reorg_detector/README.md @@ -0,0 +1,3 @@ +# `zksync_reorg_detector` + +A component to detect divergencies between local and remote instances of ZK Stack node. diff --git a/core/lib/zksync_core/src/reorg_detector/mod.rs b/core/node/reorg_detector/src/lib.rs similarity index 99% rename from core/lib/zksync_core/src/reorg_detector/mod.rs rename to core/node/reorg_detector/src/lib.rs index 5f6353ff685b..bc0541cc89a0 100644 --- a/core/lib/zksync_core/src/reorg_detector/mod.rs +++ b/core/node/reorg_detector/src/lib.rs @@ -1,8 +1,7 @@ -use std::{fmt, time::Duration}; +use std::{fmt, future::Future, time::Duration}; use anyhow::Context as _; use async_trait::async_trait; -use futures::Future; use tokio::sync::watch; use zksync_dal::{ConnectionPool, Core, CoreDal, DalError}; use zksync_health_check::{Health, HealthStatus, HealthUpdater, ReactiveHealthCheck}; diff --git a/core/lib/zksync_core/src/reorg_detector/tests.rs b/core/node/reorg_detector/src/tests.rs similarity index 100% rename from core/lib/zksync_core/src/reorg_detector/tests.rs rename to core/node/reorg_detector/src/tests.rs From 6fe07fd7b5285ba8a833e9f5d61bb1240af803ed Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Tue, 14 May 2024 16:02:14 +0400 Subject: [PATCH 05/15] Extract consistency checker --- Cargo.lock | 28 ++++++++++++++ Cargo.toml | 2 + core/bin/external_node/Cargo.toml | 1 + core/bin/external_node/src/main.rs | 2 +- core/lib/zksync_core/src/lib.rs | 1 - core/node/consistency_checker/Cargo.toml | 36 ++++++++++++++++++ core/node/consistency_checker/README.md | 3 ++ .../consistency_checker/src/lib.rs} | 0 ...it_l1_batch_200000_testnet_goerli.calldata | Bin ...it_l1_batch_351000-351004_mainnet.calldata | Bin ...mit_l1_batch_4470_testnet_sepolia.calldata | Bin .../consistency_checker/src}/tests/mod.rs | 0 core/node/node_framework/Cargo.toml | 1 + .../layers/consistency_checker.rs | 2 +- 14 files changed, 73 insertions(+), 3 deletions(-) create mode 100644 core/node/consistency_checker/Cargo.toml create mode 100644 core/node/consistency_checker/README.md rename core/{lib/zksync_core/src/consistency_checker/mod.rs => node/consistency_checker/src/lib.rs} (100%) rename core/{lib/zksync_core/src/consistency_checker => node/consistency_checker/src}/tests/commit_l1_batch_200000_testnet_goerli.calldata (100%) rename core/{lib/zksync_core/src/consistency_checker => node/consistency_checker/src}/tests/commit_l1_batch_351000-351004_mainnet.calldata (100%) rename core/{lib/zksync_core/src/consistency_checker => node/consistency_checker/src}/tests/commit_l1_batch_4470_testnet_sepolia.calldata (100%) rename core/{lib/zksync_core/src/consistency_checker => node/consistency_checker/src}/tests/mod.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 4bd695a96a39..1f3ead4ec071 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8246,6 +8246,32 @@ dependencies = [ "zksync_concurrency", ] +[[package]] +name = "zksync_consistency_checker" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "chrono", + "once_cell", + "serde", + "test-casing", + "thiserror", + "tokio", + "tracing", + "zksync_config", + "zksync_contracts", + "zksync_dal", + "zksync_eth_client", + "zksync_eth_sender", + "zksync_health_check", + "zksync_l1_contract_interface", + "zksync_node_genesis", + "zksync_node_test_utils", + "zksync_shared_metrics", + "zksync_types", +] + [[package]] name = "zksync_contract_verifier" version = "0.1.0" @@ -8570,6 +8596,7 @@ dependencies = [ "zksync_concurrency", "zksync_config", "zksync_consensus_roles", + "zksync_consistency_checker", "zksync_contracts", "zksync_core", "zksync_dal", @@ -8751,6 +8778,7 @@ dependencies = [ "zksync_commitment_generator", "zksync_concurrency", "zksync_config", + "zksync_consistency_checker", "zksync_contracts", "zksync_core", "zksync_dal", diff --git a/Cargo.toml b/Cargo.toml index 54ffa8da257f..b953470653e0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,6 +25,7 @@ members = [ "core/node/test_utils", "core/node/state_keeper", "core/node/reorg_detector", + "core/node/consistency_checker", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -244,3 +245,4 @@ zksync_vm_runner = { path = "core/node/vm_runner" } zksync_node_test_utils = { path = "core/node/test_utils" } zksync_state_keeper = { path = "core/node/state_keeper" } zksync_reorg_detector = { path = "core/node/reorg_detector" } +zksync_consistency_checker = { path = "core/node/consistency_checker" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index 96fd04e0fb57..632a3d50b501 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -38,6 +38,7 @@ zksync_node_db_pruner.workspace = true zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true zksync_reorg_detector.workspace = true +zksync_consistency_checker.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index fbcea280177e..2dc01a22aac7 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -19,6 +19,7 @@ use zksync_concurrency::{ctx, scope}; use zksync_config::configs::{ api::MerkleTreeApiConfig, chain::L1BatchCommitDataGeneratorMode, database::MerkleTreeMode, }; +use zksync_consistency_checker::ConsistencyChecker; use zksync_core::{ api_server::{ execution_sandbox::VmConcurrencyLimiter, @@ -28,7 +29,6 @@ use zksync_core::{ web3::{mempool_cache::MempoolCache, ApiBuilder, Namespace}, }, consensus, - consistency_checker::ConsistencyChecker, metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, setup_sigint_handler, sync_layer::{ diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index f16b035b4c93..ed57d308fda1 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -105,7 +105,6 @@ use crate::{ pub mod api_server; pub mod consensus; -pub mod consistency_checker; pub mod metadata_calculator; pub mod proto; pub mod sync_layer; diff --git a/core/node/consistency_checker/Cargo.toml b/core/node/consistency_checker/Cargo.toml new file mode 100644 index 000000000000..41fe90fabe27 --- /dev/null +++ b/core/node/consistency_checker/Cargo.toml @@ -0,0 +1,36 @@ +[package] +name = "zksync_consistency_checker" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_contracts.workspace = true +zksync_dal.workspace = true +zksync_eth_client.workspace = true +zksync_eth_sender.workspace = true +zksync_health_check.workspace = true +zksync_l1_contract_interface.workspace = true +zksync_shared_metrics.workspace = true +zksync_types.workspace = true + +anyhow.workspace = true +serde.workspace = true +tokio = { workspace = true, features = ["time"] } +tracing.workspace = true +thiserror.workspace = true + +[dev-dependencies] +assert_matches.workspace = true +test-casing.workspace = true +chrono.workspace = true +once_cell.workspace = true + +zksync_node_genesis.workspace = true +zksync_node_test_utils.workspace = true +zksync_config.workspace = true diff --git a/core/node/consistency_checker/README.md b/core/node/consistency_checker/README.md new file mode 100644 index 000000000000..1c2a621640f1 --- /dev/null +++ b/core/node/consistency_checker/README.md @@ -0,0 +1,3 @@ +# `zksync_consistency_checker` + +Component responsible for ensuring consistency between a local and remote instance of ZK Stack. diff --git a/core/lib/zksync_core/src/consistency_checker/mod.rs b/core/node/consistency_checker/src/lib.rs similarity index 100% rename from core/lib/zksync_core/src/consistency_checker/mod.rs rename to core/node/consistency_checker/src/lib.rs diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_200000_testnet_goerli.calldata b/core/node/consistency_checker/src/tests/commit_l1_batch_200000_testnet_goerli.calldata similarity index 100% rename from core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_200000_testnet_goerli.calldata rename to core/node/consistency_checker/src/tests/commit_l1_batch_200000_testnet_goerli.calldata diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_351000-351004_mainnet.calldata b/core/node/consistency_checker/src/tests/commit_l1_batch_351000-351004_mainnet.calldata similarity index 100% rename from core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_351000-351004_mainnet.calldata rename to core/node/consistency_checker/src/tests/commit_l1_batch_351000-351004_mainnet.calldata diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_4470_testnet_sepolia.calldata b/core/node/consistency_checker/src/tests/commit_l1_batch_4470_testnet_sepolia.calldata similarity index 100% rename from core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_4470_testnet_sepolia.calldata rename to core/node/consistency_checker/src/tests/commit_l1_batch_4470_testnet_sepolia.calldata diff --git a/core/lib/zksync_core/src/consistency_checker/tests/mod.rs b/core/node/consistency_checker/src/tests/mod.rs similarity index 100% rename from core/lib/zksync_core/src/consistency_checker/tests/mod.rs rename to core/node/consistency_checker/src/tests/mod.rs diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index 9ad5d0dd1617..eeb704ee7724 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -35,6 +35,7 @@ zksync_house_keeper.workspace = true zksync_node_fee_model.workspace = true zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true +zksync_consistency_checker.workspace = true tracing.workspace = true thiserror.workspace = true diff --git a/core/node/node_framework/src/implementations/layers/consistency_checker.rs b/core/node/node_framework/src/implementations/layers/consistency_checker.rs index 43afeaca47bd..4805bbba54f6 100644 --- a/core/node/node_framework/src/implementations/layers/consistency_checker.rs +++ b/core/node/node_framework/src/implementations/layers/consistency_checker.rs @@ -1,4 +1,4 @@ -use zksync_core::consistency_checker::ConsistencyChecker; +use zksync_consistency_checker::ConsistencyChecker; use zksync_types::Address; use crate::{ From 1ea5fa258501eb1cff3cb87eb0580d8202238d81 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Tue, 14 May 2024 17:27:42 +0400 Subject: [PATCH 06/15] Extract metadata calculator --- Cargo.lock | 37 +++++++++++++++ Cargo.toml | 2 + core/bin/external_node/Cargo.toml | 1 + core/bin/external_node/src/main.rs | 9 ++-- core/lib/zksync_core/Cargo.toml | 1 + core/lib/zksync_core/src/api_server/mod.rs | 1 - .../zksync_core/src/api_server/web3/mod.rs | 2 +- .../src/api_server/web3/namespaces/zks.rs | 12 +++-- .../zksync_core/src/api_server/web3/state.rs | 2 +- core/lib/zksync_core/src/lib.rs | 6 +-- core/node/metadata_calculator/Cargo.toml | 46 +++++++++++++++++++ core/node/metadata_calculator/README.md | 6 +++ .../src/api_server}/metrics.rs | 0 .../src/api_server}/mod.rs | 2 +- .../src/api_server}/tests.rs | 4 +- .../metadata_calculator/src}/helpers.rs | 6 +-- .../metadata_calculator/src/lib.rs} | 7 ++- .../metadata_calculator/src}/metrics.rs | 0 .../metadata_calculator/src}/pruning.rs | 2 +- .../metadata_calculator/src}/recovery/mod.rs | 0 .../src}/recovery/tests.rs | 2 +- .../metadata_calculator/src}/tests.rs | 0 .../metadata_calculator/src}/updater.rs | 0 core/node/node_framework/Cargo.toml | 1 + .../node/node_framework/examples/main_node.rs | 2 +- .../layers/metadata_calculator.rs | 2 +- .../layers/web3_api/tree_api_client.rs | 2 +- .../src/implementations/resources/web3_api.rs | 2 +- 28 files changed, 127 insertions(+), 30 deletions(-) create mode 100644 core/node/metadata_calculator/Cargo.toml create mode 100644 core/node/metadata_calculator/README.md rename core/{lib/zksync_core/src/api_server/tree => node/metadata_calculator/src/api_server}/metrics.rs (100%) rename core/{lib/zksync_core/src/api_server/tree => node/metadata_calculator/src/api_server}/mod.rs (99%) rename core/{lib/zksync_core/src/api_server/tree => node/metadata_calculator/src/api_server}/tests.rs (96%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/helpers.rs (99%) rename core/{lib/zksync_core/src/metadata_calculator/mod.rs => node/metadata_calculator/src/lib.rs} (98%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/metrics.rs (100%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/pruning.rs (99%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/recovery/mod.rs (100%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/recovery/tests.rs (99%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/tests.rs (100%) rename core/{lib/zksync_core/src/metadata_calculator => node/metadata_calculator/src}/updater.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 1f3ead4ec071..3e68234bb001 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8380,6 +8380,7 @@ dependencies = [ "zksync_l1_contract_interface", "zksync_mempool", "zksync_merkle_tree", + "zksync_metadata_calculator", "zksync_mini_merkle_tree", "zksync_node_fee_model", "zksync_node_genesis", @@ -8605,6 +8606,7 @@ dependencies = [ "zksync_eth_sender", "zksync_health_check", "zksync_l1_contract_interface", + "zksync_metadata_calculator", "zksync_node_db_pruner", "zksync_node_fee_model", "zksync_node_genesis", @@ -8708,6 +8710,40 @@ dependencies = [ "zksync_utils", ] +[[package]] +name = "zksync_metadata_calculator" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "async-trait", + "axum", + "futures 0.3.28", + "itertools 0.10.5", + "once_cell", + "reqwest", + "serde", + "serde_json", + "tempfile", + "test-casing", + "thiserror", + "tokio", + "tracing", + "vise", + "zksync_config", + "zksync_dal", + "zksync_health_check", + "zksync_merkle_tree", + "zksync_node_genesis", + "zksync_node_test_utils", + "zksync_object_store", + "zksync_prover_interface", + "zksync_shared_metrics", + "zksync_storage", + "zksync_types", + "zksync_utils", +] + [[package]] name = "zksync_mini_merkle_tree" version = "0.1.0" @@ -8789,6 +8825,7 @@ dependencies = [ "zksync_eth_watch", "zksync_health_check", "zksync_house_keeper", + "zksync_metadata_calculator", "zksync_node_fee_model", "zksync_object_store", "zksync_proof_data_handler", diff --git a/Cargo.toml b/Cargo.toml index b953470653e0..bb34118fb790 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,6 +26,7 @@ members = [ "core/node/state_keeper", "core/node/reorg_detector", "core/node/consistency_checker", + "core/node/metadata_calculator", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -246,3 +247,4 @@ zksync_node_test_utils = { path = "core/node/test_utils" } zksync_state_keeper = { path = "core/node/state_keeper" } zksync_reorg_detector = { path = "core/node/reorg_detector" } zksync_consistency_checker = { path = "core/node/consistency_checker" } +zksync_metadata_calculator = { path = "core/node/metadata_calculator" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index 632a3d50b501..8a3ec5f7e6c1 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -39,6 +39,7 @@ zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true zksync_reorg_detector.workspace = true zksync_consistency_checker.workspace = true +zksync_metadata_calculator.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 2dc01a22aac7..6b1508b9f04f 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -24,13 +24,10 @@ use zksync_core::{ api_server::{ execution_sandbox::VmConcurrencyLimiter, healthcheck::HealthCheckHandle, - tree::{TreeApiClient, TreeApiHttpClient}, tx_sender::{proxy::TxProxy, ApiContracts, TxSenderBuilder}, web3::{mempool_cache::MempoolCache, ApiBuilder, Namespace}, }, - consensus, - metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, - setup_sigint_handler, + consensus, setup_sigint_handler, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, SyncState, }, @@ -46,6 +43,10 @@ use zksync_eth_sender::l1_batch_commit_data_generator::{ ValidiumModeL1BatchCommitDataGenerator, }; use zksync_health_check::{AppHealthCheck, HealthStatus, ReactiveHealthCheck}; +use zksync_metadata_calculator::{ + api_server::{TreeApiClient, TreeApiHttpClient}, + MetadataCalculator, MetadataCalculatorConfig, +}; use zksync_node_db_pruner::{DbPruner, DbPrunerConfig}; use zksync_node_fee_model::l1_gas_price::MainNodeFeeParamsFetcher; use zksync_reorg_detector::ReorgDetector; diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 3755f1d8beeb..9acaaa54a6e5 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -50,6 +50,7 @@ zksync_node_genesis.workspace = true zksync_eth_sender.workspace = true zksync_node_fee_model.workspace = true zksync_state_keeper.workspace = true +zksync_metadata_calculator.workspace = true multivm.workspace = true # Consensus dependenices diff --git a/core/lib/zksync_core/src/api_server/mod.rs b/core/lib/zksync_core/src/api_server/mod.rs index cc6ec239473b..a8c4304ccba2 100644 --- a/core/lib/zksync_core/src/api_server/mod.rs +++ b/core/lib/zksync_core/src/api_server/mod.rs @@ -5,6 +5,5 @@ mod utils; pub mod contract_verification; pub mod execution_sandbox; pub mod healthcheck; -pub mod tree; pub mod tx_sender; pub mod web3; diff --git a/core/lib/zksync_core/src/api_server/web3/mod.rs b/core/lib/zksync_core/src/api_server/web3/mod.rs index ec1bc49fe0b3..42d7dd599658 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/mod.rs @@ -12,6 +12,7 @@ use tower_http::{cors::CorsLayer, metrics::InFlightRequestsLayer}; use zksync_config::configs::api::{MaxResponseSize, MaxResponseSizeOverrides}; use zksync_dal::{helpers::wait_for_l1_batch, ConnectionPool, Core}; use zksync_health_check::{HealthStatus, HealthUpdater, ReactiveHealthCheck}; +use zksync_metadata_calculator::api_server::TreeApiClient; use zksync_types::L2BlockNumber; use zksync_web3_decl::{ jsonrpsee::{ @@ -44,7 +45,6 @@ use self::{ use crate::{ api_server::{ execution_sandbox::{BlockStartInfo, VmConcurrencyBarrier}, - tree::TreeApiClient, tx_sender::TxSender, }, sync_layer::SyncState, diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs b/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs index 339961f7dd60..fca41af1e62e 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs +++ b/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs @@ -3,6 +3,7 @@ use std::{collections::HashMap, convert::TryInto}; use anyhow::Context as _; use multivm::interface::VmExecutionResultAndLogs; use zksync_dal::{Connection, Core, CoreDal, DalError}; +use zksync_metadata_calculator::api_server::TreeApiError; use zksync_mini_merkle_tree::MiniMerkleTree; use zksync_system_constants::DEFAULT_L2_TX_GAS_PER_PUBDATA_BYTE; use zksync_types::{ @@ -28,10 +29,7 @@ use zksync_web3_decl::{ types::{Address, Token, H256}, }; -use crate::api_server::{ - tree::TreeApiError, - web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, RpcState}, -}; +use crate::api_server::web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, RpcState}; #[derive(Debug)] pub(crate) struct ZksNamespace { @@ -509,6 +507,12 @@ impl ZksNamespace { }; } Err(TreeApiError::Internal(err)) => return Err(Web3Error::InternalError(err)), + Err(_) => { + // This branch is not expected to be executed, but has to be provided since the error is non-exhaustive. + return Err(Web3Error::InternalError(anyhow::anyhow!( + "Unspecified tree API error" + ))); + } }; let storage_proof = proofs diff --git a/core/lib/zksync_core/src/api_server/web3/state.rs b/core/lib/zksync_core/src/api_server/web3/state.rs index fd11ac012ed6..5b68f0f503ea 100644 --- a/core/lib/zksync_core/src/api_server/web3/state.rs +++ b/core/lib/zksync_core/src/api_server/web3/state.rs @@ -17,6 +17,7 @@ use zksync_config::{ GenesisConfig, }; use zksync_dal::{Connection, ConnectionPool, Core, CoreDal, DalError}; +use zksync_metadata_calculator::api_server::TreeApiClient; use zksync_types::{ api, l2::L2Tx, transaction_request::CallRequest, Address, L1BatchNumber, L1ChainId, L2BlockNumber, L2ChainId, H256, U256, U64, @@ -32,7 +33,6 @@ use super::{ use crate::{ api_server::{ execution_sandbox::{BlockArgs, BlockArgsError, BlockStartInfo}, - tree::TreeApiClient, tx_sender::{tx_sink::TxSink, TxSender}, }, sync_layer::SyncState, diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index ed57d308fda1..cec391b3a8fe 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -71,6 +71,9 @@ use zksync_house_keeper::{ periodic_job::PeriodicJob, waiting_to_queued_fri_witness_job_mover::WaitingToQueuedFriWitnessJobMover, }; +use zksync_metadata_calculator::{ + api_server::TreeApiHttpClient, MetadataCalculator, MetadataCalculatorConfig, +}; use zksync_node_fee_model::{ l1_gas_price::{ GasAdjusterSingleton, PubdataPricing, RollupPubdataPricing, ValidiumPubdataPricing, @@ -95,17 +98,14 @@ use crate::{ contract_verification, execution_sandbox::{VmConcurrencyBarrier, VmConcurrencyLimiter}, healthcheck::HealthCheckHandle, - tree::TreeApiHttpClient, tx_sender::{ApiContracts, TxSender, TxSenderBuilder, TxSenderConfig}, web3::{self, mempool_cache::MempoolCache, state::InternalApiConfig, Namespace}, }, - metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, utils::ensure_l1_batch_commit_data_generation_mode, }; pub mod api_server; pub mod consensus; -pub mod metadata_calculator; pub mod proto; pub mod sync_layer; pub mod temp_config_store; diff --git a/core/node/metadata_calculator/Cargo.toml b/core/node/metadata_calculator/Cargo.toml new file mode 100644 index 000000000000..3dcfcd89c211 --- /dev/null +++ b/core/node/metadata_calculator/Cargo.toml @@ -0,0 +1,46 @@ +[package] +name = "zksync_metadata_calculator" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_dal.workspace = true +zksync_health_check.workspace = true +zksync_merkle_tree.workspace = true +zksync_types.workspace = true +zksync_config.workspace = true +zksync_storage.workspace = true +zksync_shared_metrics.workspace = true +zksync_utils.workspace = true +zksync_object_store.workspace = true +vise.workspace = true + +async-trait.workspace = true +anyhow.workspace = true +serde.workspace = true +tokio = { workspace = true, features = ["time"] } +thiserror.workspace = true +tracing.workspace = true +once_cell.workspace = true +futures.workspace = true + +# dependencies for the tree API server +reqwest.workspace = true +axum.workspace = true +serde_json.workspace = true + +[dev-dependencies] +zksync_node_genesis.workspace = true +zksync_node_test_utils.workspace = true +zksync_prover_interface.workspace = true + +assert_matches.workspace = true +tempfile.workspace = true +test-casing.workspace = true +itertools.workspace = true diff --git a/core/node/metadata_calculator/README.md b/core/node/metadata_calculator/README.md new file mode 100644 index 000000000000..35880000b98c --- /dev/null +++ b/core/node/metadata_calculator/README.md @@ -0,0 +1,6 @@ +# `zksync_metadata_calculator` + +Component responsible for calculating any supplimentary metadata required for L1 batches. +Most notably, this is component responsible for maintaining the Merkle Tree. + +Additionally, this crate provides ability to spawn the Merkle Tree API server. diff --git a/core/lib/zksync_core/src/api_server/tree/metrics.rs b/core/node/metadata_calculator/src/api_server/metrics.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/tree/metrics.rs rename to core/node/metadata_calculator/src/api_server/metrics.rs diff --git a/core/lib/zksync_core/src/api_server/tree/mod.rs b/core/node/metadata_calculator/src/api_server/mod.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/tree/mod.rs rename to core/node/metadata_calculator/src/api_server/mod.rs index bb6f2528672e..c427397b72c0 100644 --- a/core/lib/zksync_core/src/api_server/tree/mod.rs +++ b/core/node/metadata_calculator/src/api_server/mod.rs @@ -17,7 +17,7 @@ use zksync_merkle_tree::NoVersionError; use zksync_types::{L1BatchNumber, H256, U256}; use self::metrics::{MerkleTreeApiMethod, API_METRICS}; -use crate::metadata_calculator::{AsyncTreeReader, LazyAsyncTreeReader, MerkleTreeInfo}; +use crate::{AsyncTreeReader, LazyAsyncTreeReader, MerkleTreeInfo}; mod metrics; #[cfg(test)] diff --git a/core/lib/zksync_core/src/api_server/tree/tests.rs b/core/node/metadata_calculator/src/api_server/tests.rs similarity index 96% rename from core/lib/zksync_core/src/api_server/tree/tests.rs rename to core/node/metadata_calculator/src/api_server/tests.rs index b0ae8a104561..ce7ad03ada07 100644 --- a/core/lib/zksync_core/src/api_server/tree/tests.rs +++ b/core/node/metadata_calculator/src/api_server/tests.rs @@ -7,9 +7,7 @@ use tempfile::TempDir; use zksync_dal::{ConnectionPool, Core}; use super::*; -use crate::metadata_calculator::tests::{ - gen_storage_logs, reset_db_state, run_calculator, setup_calculator, -}; +use crate::tests::{gen_storage_logs, reset_db_state, run_calculator, setup_calculator}; #[tokio::test] async fn merkle_tree_api() { diff --git a/core/lib/zksync_core/src/metadata_calculator/helpers.rs b/core/node/metadata_calculator/src/helpers.rs similarity index 99% rename from core/lib/zksync_core/src/metadata_calculator/helpers.rs rename to core/node/metadata_calculator/src/helpers.rs index b85224ab0520..52cb18ea4458 100644 --- a/core/lib/zksync_core/src/metadata_calculator/helpers.rs +++ b/core/node/metadata_calculator/src/helpers.rs @@ -349,7 +349,7 @@ pub struct LazyAsyncTreeReader(pub(super) watch::Receiver Option { + pub fn read(&self) -> Option { self.0.borrow().clone() } @@ -616,9 +616,7 @@ mod tests { use zksync_types::{StorageKey, StorageLog}; use super::*; - use crate::metadata_calculator::tests::{ - extend_db_state, gen_storage_logs, mock_config, reset_db_state, - }; + use crate::tests::{extend_db_state, gen_storage_logs, mock_config, reset_db_state}; impl L1BatchWithLogs { /// Old, slower method of loading storage logs. We want to test its equivalence to the new implementation. diff --git a/core/lib/zksync_core/src/metadata_calculator/mod.rs b/core/node/metadata_calculator/src/lib.rs similarity index 98% rename from core/lib/zksync_core/src/metadata_calculator/mod.rs rename to core/node/metadata_calculator/src/lib.rs index cab3ceb27c87..9f3b0a113a72 100644 --- a/core/lib/zksync_core/src/metadata_calculator/mod.rs +++ b/core/node/metadata_calculator/src/lib.rs @@ -17,15 +17,18 @@ use zksync_dal::{ConnectionPool, Core}; use zksync_health_check::{CheckHealth, HealthUpdater, ReactiveHealthCheck}; use zksync_object_store::ObjectStore; -pub(crate) use self::helpers::{AsyncTreeReader, MerkleTreeInfo}; -pub use self::{helpers::LazyAsyncTreeReader, pruning::MerkleTreePruningTask}; use self::{ helpers::{create_db, Delayer, GenericAsyncTree, MerkleTreeHealth, MerkleTreeHealthCheck}, metrics::{ConfigLabels, METRICS}, pruning::PruningHandles, updater::TreeUpdater, }; +pub use self::{ + helpers::{AsyncTreeReader, LazyAsyncTreeReader, MerkleTreeInfo}, + pruning::MerkleTreePruningTask, +}; +pub mod api_server; mod helpers; mod metrics; mod pruning; diff --git a/core/lib/zksync_core/src/metadata_calculator/metrics.rs b/core/node/metadata_calculator/src/metrics.rs similarity index 100% rename from core/lib/zksync_core/src/metadata_calculator/metrics.rs rename to core/node/metadata_calculator/src/metrics.rs diff --git a/core/lib/zksync_core/src/metadata_calculator/pruning.rs b/core/node/metadata_calculator/src/pruning.rs similarity index 99% rename from core/lib/zksync_core/src/metadata_calculator/pruning.rs rename to core/node/metadata_calculator/src/pruning.rs index 9bed06084263..2e15e22e8292 100644 --- a/core/lib/zksync_core/src/metadata_calculator/pruning.rs +++ b/core/node/metadata_calculator/src/pruning.rs @@ -151,7 +151,7 @@ mod tests { use zksync_types::{L1BatchNumber, L2BlockNumber}; use super::*; - use crate::metadata_calculator::{ + use crate::{ tests::{extend_db_state_from_l1_batch, gen_storage_logs, mock_config, reset_db_state}, MetadataCalculator, }; diff --git a/core/lib/zksync_core/src/metadata_calculator/recovery/mod.rs b/core/node/metadata_calculator/src/recovery/mod.rs similarity index 100% rename from core/lib/zksync_core/src/metadata_calculator/recovery/mod.rs rename to core/node/metadata_calculator/src/recovery/mod.rs diff --git a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs b/core/node/metadata_calculator/src/recovery/tests.rs similarity index 99% rename from core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs rename to core/node/metadata_calculator/src/recovery/tests.rs index d340bd578235..3e2978cd8ccf 100644 --- a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs +++ b/core/node/metadata_calculator/src/recovery/tests.rs @@ -18,7 +18,7 @@ use zksync_node_test_utils::prepare_recovery_snapshot; use zksync_types::{L1BatchNumber, ProtocolVersionId, StorageLog}; use super::*; -use crate::metadata_calculator::{ +use crate::{ helpers::create_db, tests::{ extend_db_state, extend_db_state_from_l1_batch, gen_storage_logs, mock_config, diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/node/metadata_calculator/src/tests.rs similarity index 100% rename from core/lib/zksync_core/src/metadata_calculator/tests.rs rename to core/node/metadata_calculator/src/tests.rs diff --git a/core/lib/zksync_core/src/metadata_calculator/updater.rs b/core/node/metadata_calculator/src/updater.rs similarity index 100% rename from core/lib/zksync_core/src/metadata_calculator/updater.rs rename to core/node/metadata_calculator/src/updater.rs diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index eeb704ee7724..5e8ce5d49777 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -36,6 +36,7 @@ zksync_node_fee_model.workspace = true zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true zksync_consistency_checker.workspace = true +zksync_metadata_calculator.workspace = true tracing.workspace = true thiserror.workspace = true diff --git a/core/node/node_framework/examples/main_node.rs b/core/node/node_framework/examples/main_node.rs index fa783a163ff3..7a2857a340f2 100644 --- a/core/node/node_framework/examples/main_node.rs +++ b/core/node/node_framework/examples/main_node.rs @@ -24,10 +24,10 @@ use zksync_core::{ tx_sender::{ApiContracts, TxSenderConfig}, web3::{state::InternalApiConfig, Namespace}, }, - metadata_calculator::MetadataCalculatorConfig, temp_config_store::decode_yaml_repr, }; use zksync_env_config::FromEnv; +use zksync_metadata_calculator::MetadataCalculatorConfig; use zksync_node_framework::{ implementations::layers::{ circuit_breaker_checker::CircuitBreakerCheckerLayer, diff --git a/core/node/node_framework/src/implementations/layers/metadata_calculator.rs b/core/node/node_framework/src/implementations/layers/metadata_calculator.rs index 7909856d460f..4b1e1d00cb5e 100644 --- a/core/node/node_framework/src/implementations/layers/metadata_calculator.rs +++ b/core/node/node_framework/src/implementations/layers/metadata_calculator.rs @@ -5,7 +5,7 @@ use std::{ use anyhow::Context as _; use zksync_config::configs::{api::MerkleTreeApiConfig, database::MerkleTreeMode}; -use zksync_core::metadata_calculator::{ +use zksync_metadata_calculator::{ LazyAsyncTreeReader, MetadataCalculator, MetadataCalculatorConfig, }; use zksync_storage::RocksDB; diff --git a/core/node/node_framework/src/implementations/layers/web3_api/tree_api_client.rs b/core/node/node_framework/src/implementations/layers/web3_api/tree_api_client.rs index 6380e0bb445a..065eabf6170b 100644 --- a/core/node/node_framework/src/implementations/layers/web3_api/tree_api_client.rs +++ b/core/node/node_framework/src/implementations/layers/web3_api/tree_api_client.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use zksync_core::api_server::tree::TreeApiHttpClient; +use zksync_metadata_calculator::api_server::TreeApiHttpClient; use crate::{ implementations::resources::{ diff --git a/core/node/node_framework/src/implementations/resources/web3_api.rs b/core/node/node_framework/src/implementations/resources/web3_api.rs index 68d343b9b0ca..d9d71220949f 100644 --- a/core/node/node_framework/src/implementations/resources/web3_api.rs +++ b/core/node/node_framework/src/implementations/resources/web3_api.rs @@ -1,10 +1,10 @@ use std::sync::Arc; use zksync_core::api_server::{ - tree::TreeApiClient, tx_sender::{tx_sink::TxSink, TxSender}, web3::mempool_cache::MempoolCache, }; +use zksync_metadata_calculator::api_server::TreeApiClient; use crate::resource::Resource; From 0a4d53d92ed8a10bd367d8d8543e30528e517ffb Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 09:48:41 +0400 Subject: [PATCH 07/15] Detach sync_layer from consensus --- .../lib/zksync_core/src/consensus/testonly.rs | 127 +----------------- core/lib/zksync_core/src/sync_layer/mod.rs | 1 + .../zksync_core/src/sync_layer/testonly.rs | 83 ++++++++++++ core/lib/zksync_core/src/sync_layer/tests.rs | 40 +++++- 4 files changed, 128 insertions(+), 123 deletions(-) create mode 100644 core/lib/zksync_core/src/sync_layer/testonly.rs diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 1f3cfa2a8538..63c32213f2e3 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -1,17 +1,16 @@ //! Utilities for testing the consensus module. -use std::{collections::HashMap, sync::Arc}; +use std::sync::Arc; use anyhow::Context as _; use rand::Rng; use zksync_concurrency::{ctx, error::Wrap as _, scope, sync, time}; -use zksync_config::{configs, configs::consensus as config, GenesisConfig}; +use zksync_config::{configs, configs::consensus as config}; use zksync_consensus_crypto::TextFmt as _; use zksync_consensus_network as network; use zksync_consensus_roles::validator; -use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{CoreDal, DalError}; -use zksync_node_genesis::{mock_genesis_config, GenesisParams}; +use zksync_node_genesis::GenesisParams; use zksync_node_test_utils::{create_l1_batch_metadata, create_l2_transaction}; use zksync_state_keeper::{ io::{IoCursor, L1BatchParams, L2BlockParams}, @@ -19,14 +18,8 @@ use zksync_state_keeper::{ testonly::MockBatchExecutor, OutputHandler, StateKeeperPersistence, ZkSyncStateKeeper, }; -use zksync_types::{ - api, snapshots::SnapshotRecoveryStatus, Address, L1BatchNumber, L2BlockNumber, L2ChainId, - ProtocolVersionId, H256, -}; -use zksync_web3_decl::{ - client::{Client, DynClient, L2}, - error::{EnrichedClientError, EnrichedClientResult}, -}; +use zksync_types::{Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId}; +use zksync_web3_decl::client::{Client, DynClient, L2}; use crate::{ api_server::web3::{state::InternalApiConfig, tests::spawn_http_server}, @@ -34,119 +27,11 @@ use crate::{ sync_layer::{ fetcher::{FetchedTransaction, IoCursorExt as _}, sync_action::{ActionQueue, ActionQueueSender, SyncAction}, + testonly::MockMainNodeClient, ExternalIO, MainNodeClient, SyncState, }, }; -#[derive(Debug, Default)] -pub(crate) struct MockMainNodeClient { - l2_blocks: Vec, - block_number_offset: u32, - protocol_versions: HashMap, - system_contracts: HashMap>, -} - -impl MockMainNodeClient { - pub fn for_snapshot_recovery(snapshot: &SnapshotRecoveryStatus) -> Self { - // This block may be requested during node initialization - let last_l2_block_in_snapshot_batch = api::en::SyncBlock { - number: snapshot.l2_block_number, - l1_batch_number: snapshot.l1_batch_number, - last_in_batch: true, - timestamp: snapshot.l2_block_timestamp, - l1_gas_price: 2, - l2_fair_gas_price: 3, - fair_pubdata_price: Some(24), - base_system_contracts_hashes: BaseSystemContractsHashes::default(), - operator_address: Address::repeat_byte(2), - transactions: Some(vec![]), - virtual_blocks: Some(0), - hash: Some(snapshot.l2_block_hash), - protocol_version: ProtocolVersionId::latest(), - }; - - Self { - l2_blocks: vec![last_l2_block_in_snapshot_batch], - block_number_offset: snapshot.l2_block_number.0, - ..Self::default() - } - } - - pub fn insert_protocol_version(&mut self, version: api::ProtocolVersion) { - self.system_contracts - .insert(version.base_system_contracts.bootloader, vec![]); - self.system_contracts - .insert(version.base_system_contracts.default_aa, vec![]); - self.protocol_versions.insert(version.version_id, version); - } -} - -#[async_trait::async_trait] -impl MainNodeClient for MockMainNodeClient { - async fn fetch_system_contract_by_hash( - &self, - hash: H256, - ) -> EnrichedClientResult>> { - Ok(self.system_contracts.get(&hash).cloned()) - } - - async fn fetch_genesis_contract_bytecode( - &self, - _address: Address, - ) -> EnrichedClientResult>> { - Err(EnrichedClientError::custom( - "not implemented", - "fetch_genesis_contract_bytecode", - )) - } - - async fn fetch_protocol_version( - &self, - protocol_version: ProtocolVersionId, - ) -> EnrichedClientResult> { - let protocol_version = protocol_version as u16; - Ok(self.protocol_versions.get(&protocol_version).cloned()) - } - - async fn fetch_l2_block_number(&self) -> EnrichedClientResult { - if let Some(number) = self.l2_blocks.len().checked_sub(1) { - Ok(L2BlockNumber(number as u32)) - } else { - Err(EnrichedClientError::custom( - "not implemented", - "fetch_l2_block_number", - )) - } - } - - async fn fetch_l2_block( - &self, - number: L2BlockNumber, - with_transactions: bool, - ) -> EnrichedClientResult> { - let Some(block_index) = number.0.checked_sub(self.block_number_offset) else { - return Ok(None); - }; - let Some(mut block) = self.l2_blocks.get(block_index as usize).cloned() else { - return Ok(None); - }; - if !with_transactions { - block.transactions = None; - } - Ok(Some(block)) - } - - async fn fetch_consensus_genesis( - &self, - ) -> EnrichedClientResult> { - unimplemented!() - } - - async fn fetch_genesis_config(&self) -> EnrichedClientResult { - Ok(mock_genesis_config()) - } -} - /// Fake StateKeeper for tests. pub(super) struct StateKeeper { // Batch of the `last_block`. diff --git a/core/lib/zksync_core/src/sync_layer/mod.rs b/core/lib/zksync_core/src/sync_layer/mod.rs index a8e840c96099..7bb825fc8bb6 100644 --- a/core/lib/zksync_core/src/sync_layer/mod.rs +++ b/core/lib/zksync_core/src/sync_layer/mod.rs @@ -6,6 +6,7 @@ pub mod genesis; mod metrics; pub(crate) mod sync_action; mod sync_state; +pub mod testonly; #[cfg(test)] mod tests; diff --git a/core/lib/zksync_core/src/sync_layer/testonly.rs b/core/lib/zksync_core/src/sync_layer/testonly.rs new file mode 100644 index 000000000000..6d84e2c1ee1c --- /dev/null +++ b/core/lib/zksync_core/src/sync_layer/testonly.rs @@ -0,0 +1,83 @@ +use std::collections::HashMap; + +use zksync_config::GenesisConfig; +use zksync_eth_client::EnrichedClientError; +use zksync_node_genesis::mock_genesis_config; +use zksync_types::{api, Address, L2BlockNumber, ProtocolVersionId, H256}; +use zksync_web3_decl::error::EnrichedClientResult; + +use super::MainNodeClient; + +#[derive(Debug, Default)] +pub(crate) struct MockMainNodeClient { + pub l2_blocks: Vec, + pub block_number_offset: u32, + pub protocol_versions: HashMap, + pub system_contracts: HashMap>, +} + +#[async_trait::async_trait] +impl MainNodeClient for MockMainNodeClient { + async fn fetch_system_contract_by_hash( + &self, + hash: H256, + ) -> EnrichedClientResult>> { + Ok(self.system_contracts.get(&hash).cloned()) + } + + async fn fetch_genesis_contract_bytecode( + &self, + _address: Address, + ) -> EnrichedClientResult>> { + Err(EnrichedClientError::custom( + "not implemented", + "fetch_genesis_contract_bytecode", + )) + } + + async fn fetch_protocol_version( + &self, + protocol_version: ProtocolVersionId, + ) -> EnrichedClientResult> { + let protocol_version = protocol_version as u16; + Ok(self.protocol_versions.get(&protocol_version).cloned()) + } + + async fn fetch_l2_block_number(&self) -> EnrichedClientResult { + if let Some(number) = self.l2_blocks.len().checked_sub(1) { + Ok(L2BlockNumber(number as u32)) + } else { + Err(EnrichedClientError::custom( + "not implemented", + "fetch_l2_block_number", + )) + } + } + + async fn fetch_l2_block( + &self, + number: L2BlockNumber, + with_transactions: bool, + ) -> EnrichedClientResult> { + let Some(block_index) = number.0.checked_sub(self.block_number_offset) else { + return Ok(None); + }; + let Some(mut block) = self.l2_blocks.get(block_index as usize).cloned() else { + return Ok(None); + }; + if !with_transactions { + block.transactions = None; + } + Ok(Some(block)) + } + + async fn fetch_consensus_genesis( + &self, + ) -> EnrichedClientResult> { + unimplemented!() + } + + async fn fetch_genesis_config(&self) -> EnrichedClientResult { + Ok(mock_genesis_config()) + } +} diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 7186c03ccb5e..47c98d5cb693 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -24,8 +24,9 @@ use zksync_types::{ Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, Transaction, H256, }; -use super::{fetcher::FetchedTransaction, sync_action::SyncAction, *}; -use crate::consensus::testonly::MockMainNodeClient; +use super::{ + fetcher::FetchedTransaction, sync_action::SyncAction, testonly::MockMainNodeClient, *, +}; const TEST_TIMEOUT: Duration = Duration::from_secs(10); const POLL_INTERVAL: Duration = Duration::from_millis(50); @@ -48,6 +49,41 @@ fn open_l1_batch(number: u32, timestamp: u64, first_l2_block_number: u32) -> Syn } } +impl MockMainNodeClient { + pub(crate) fn for_snapshot_recovery(snapshot: &SnapshotRecoveryStatus) -> Self { + // This block may be requested during node initialization + let last_l2_block_in_snapshot_batch = api::en::SyncBlock { + number: snapshot.l2_block_number, + l1_batch_number: snapshot.l1_batch_number, + last_in_batch: true, + timestamp: snapshot.l2_block_timestamp, + l1_gas_price: 2, + l2_fair_gas_price: 3, + fair_pubdata_price: Some(24), + base_system_contracts_hashes: BaseSystemContractsHashes::default(), + operator_address: Address::repeat_byte(2), + transactions: Some(vec![]), + virtual_blocks: Some(0), + hash: Some(snapshot.l2_block_hash), + protocol_version: ProtocolVersionId::latest(), + }; + + Self { + l2_blocks: vec![last_l2_block_in_snapshot_batch], + block_number_offset: snapshot.l2_block_number.0, + ..Self::default() + } + } + + pub fn insert_protocol_version(&mut self, version: api::ProtocolVersion) { + self.system_contracts + .insert(version.base_system_contracts.bootloader, vec![]); + self.system_contracts + .insert(version.base_system_contracts.default_aa, vec![]); + self.protocol_versions.insert(version.version_id, version); + } +} + #[derive(Debug)] pub(super) struct StateKeeperHandles { pub stop_sender: watch::Sender, From d694b203357a70dec3e445720d48d7d490829ddb Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 12:03:07 +0400 Subject: [PATCH 08/15] Extract sync_layer -> zksync_node_sync --- Cargo.lock | 34 +++++++++++++++ Cargo.toml | 2 + core/bin/external_node/Cargo.toml | 1 + core/bin/external_node/src/init.rs | 2 +- core/bin/external_node/src/main.rs | 6 +-- core/lib/zksync_core/Cargo.toml | 1 + .../zksync_core/src/api_server/web3/mod.rs | 10 ++--- .../zksync_core/src/api_server/web3/state.rs | 10 ++--- core/lib/zksync_core/src/consensus/en.rs | 10 ++--- core/lib/zksync_core/src/consensus/era.rs | 2 +- .../zksync_core/src/consensus/storage/mod.rs | 2 +- .../lib/zksync_core/src/consensus/testonly.rs | 12 +++--- core/lib/zksync_core/src/lib.rs | 1 - core/node/node_framework/Cargo.toml | 1 + .../src/implementations/layers/consensus.rs | 6 +-- .../implementations/resources/action_queue.rs | 2 +- .../implementations/resources/sync_state.rs | 2 +- core/node/node_sync/Cargo.toml | 41 +++++++++++++++++++ core/node/node_sync/README.md | 3 ++ .../src}/batch_status_updater/mod.rs | 0 .../src}/batch_status_updater/tests.rs | 2 +- .../node_sync/src}/client.rs | 0 .../node_sync/src}/external_io.rs | 0 .../node_sync/src}/fetcher.rs | 0 .../node_sync/src}/genesis.rs | 0 .../mod.rs => node/node_sync/src/lib.rs} | 2 +- .../node_sync/src}/metrics.rs | 0 .../node_sync/src}/sync_action.rs | 2 +- .../node_sync/src}/sync_state.rs | 12 +++--- .../node_sync/src}/testonly.rs | 2 +- .../node_sync/src}/tests.rs | 0 31 files changed, 121 insertions(+), 47 deletions(-) create mode 100644 core/node/node_sync/Cargo.toml create mode 100644 core/node/node_sync/README.md rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/batch_status_updater/mod.rs (100%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/batch_status_updater/tests.rs (99%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/client.rs (100%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/external_io.rs (100%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/fetcher.rs (100%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/genesis.rs (100%) rename core/{lib/zksync_core/src/sync_layer/mod.rs => node/node_sync/src/lib.rs} (96%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/metrics.rs (100%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/sync_action.rs (99%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/sync_state.rs (96%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/testonly.rs (98%) rename core/{lib/zksync_core/src/sync_layer => node/node_sync/src}/tests.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 200767e8f8be..c13ffc946fc8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8394,6 +8394,7 @@ dependencies = [ "zksync_mini_merkle_tree", "zksync_node_fee_model", "zksync_node_genesis", + "zksync_node_sync", "zksync_node_test_utils", "zksync_object_store", "zksync_proof_data_handler", @@ -8622,6 +8623,7 @@ dependencies = [ "zksync_node_db_pruner", "zksync_node_fee_model", "zksync_node_genesis", + "zksync_node_sync", "zksync_object_store", "zksync_protobuf_config", "zksync_reorg_detector", @@ -8838,6 +8840,7 @@ dependencies = [ "zksync_house_keeper", "zksync_metadata_calculator", "zksync_node_fee_model", + "zksync_node_sync", "zksync_object_store", "zksync_proof_data_handler", "zksync_protobuf_config", @@ -8870,6 +8873,37 @@ dependencies = [ "zksync_utils", ] +[[package]] +name = "zksync_node_sync" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "async-trait", + "chrono", + "serde", + "test-casing", + "thiserror", + "tokio", + "tracing", + "vise", + "vm_utils", + "zksync_concurrency", + "zksync_config", + "zksync_contracts", + "zksync_dal", + "zksync_eth_client", + "zksync_health_check", + "zksync_node_genesis", + "zksync_node_test_utils", + "zksync_shared_metrics", + "zksync_state_keeper", + "zksync_system_constants", + "zksync_types", + "zksync_utils", + "zksync_web3_decl", +] + [[package]] name = "zksync_node_test_utils" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 1620d0179c33..0a33f5b4ec29 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,6 +27,7 @@ members = [ "core/node/reorg_detector", "core/node/consistency_checker", "core/node/metadata_calculator", + "core/node/node_sync", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -251,3 +252,4 @@ zksync_state_keeper = { path = "core/node/state_keeper" } zksync_reorg_detector = { path = "core/node/reorg_detector" } zksync_consistency_checker = { path = "core/node/consistency_checker" } zksync_metadata_calculator = { path = "core/node/metadata_calculator" } +zksync_node_sync = { path = "core/node/node_sync" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index e45daf5200d4..29203cb73721 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -39,6 +39,7 @@ zksync_state_keeper.workspace = true zksync_reorg_detector.workspace = true zksync_consistency_checker.workspace = true zksync_metadata_calculator.workspace = true +zksync_node_sync.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/init.rs b/core/bin/external_node/src/init.rs index 397adc07ec69..0f4ae9a80362 100644 --- a/core/bin/external_node/src/init.rs +++ b/core/bin/external_node/src/init.rs @@ -3,9 +3,9 @@ use std::time::Instant; use anyhow::Context as _; -use zksync_core::sync_layer::genesis::perform_genesis_if_needed; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_health_check::AppHealthCheck; +use zksync_node_sync::genesis::perform_genesis_if_needed; use zksync_object_store::ObjectStoreFactory; use zksync_shared_metrics::{SnapshotRecoveryStage, APP_METRICS}; use zksync_snapshots_applier::{SnapshotsApplierConfig, SnapshotsApplierTask}; diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index b5fa575cb79b..43a8452a18c1 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -20,9 +20,6 @@ use zksync_core::{ web3::{mempool_cache::MempoolCache, ApiBuilder, Namespace}, }, consensus, setup_sigint_handler, - sync_layer::{ - batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, SyncState, - }, utils::L1BatchCommitmentModeValidationTask, }; use zksync_dal::{metrics::PostgresMetrics, ConnectionPool, Core, CoreDal}; @@ -37,6 +34,9 @@ use zksync_metadata_calculator::{ }; use zksync_node_db_pruner::{DbPruner, DbPrunerConfig}; use zksync_node_fee_model::l1_gas_price::MainNodeFeeParamsFetcher; +use zksync_node_sync::{ + batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, ActionQueue, SyncState, +}; use zksync_reorg_detector::ReorgDetector; use zksync_state::{PostgresStorageCaches, RocksdbStorageOptions}; use zksync_state_keeper::{ diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index abf95b43b54e..64e40e9858b1 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -53,6 +53,7 @@ zksync_eth_sender.workspace = true zksync_node_fee_model.workspace = true zksync_state_keeper.workspace = true zksync_metadata_calculator.workspace = true +zksync_node_sync.workspace = true multivm.workspace = true # Consensus dependenices diff --git a/core/lib/zksync_core/src/api_server/web3/mod.rs b/core/lib/zksync_core/src/api_server/web3/mod.rs index 42d7dd599658..54528286c3d0 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/mod.rs @@ -42,13 +42,11 @@ use self::{ pubsub::{EthSubscribe, EthSubscriptionIdProvider, PubSubEvent}, state::{Filters, InternalApiConfig, RpcState, SealedL2BlockNumber}, }; -use crate::{ - api_server::{ - execution_sandbox::{BlockStartInfo, VmConcurrencyBarrier}, - tx_sender::TxSender, - }, - sync_layer::SyncState, +use crate::api_server::{ + execution_sandbox::{BlockStartInfo, VmConcurrencyBarrier}, + tx_sender::TxSender, }; +use zksync_node_sync::SyncState; pub mod backend_jsonrpsee; pub mod mempool_cache; diff --git a/core/lib/zksync_core/src/api_server/web3/state.rs b/core/lib/zksync_core/src/api_server/web3/state.rs index cee2b7f67ec5..35b1cdc78c46 100644 --- a/core/lib/zksync_core/src/api_server/web3/state.rs +++ b/core/lib/zksync_core/src/api_server/web3/state.rs @@ -18,6 +18,7 @@ use zksync_config::{ }; use zksync_dal::{Connection, ConnectionPool, Core, CoreDal, DalError}; use zksync_metadata_calculator::api_server::TreeApiClient; +use zksync_node_sync::SyncState; use zksync_types::{ api, commitment::L1BatchCommitmentMode, l2::L2Tx, transaction_request::CallRequest, Address, L1BatchNumber, L1ChainId, L2BlockNumber, L2ChainId, H256, U256, U64, @@ -30,12 +31,9 @@ use super::{ metrics::{FilterType, FILTER_METRICS}, TypedFilter, }; -use crate::{ - api_server::{ - execution_sandbox::{BlockArgs, BlockArgsError, BlockStartInfo}, - tx_sender::{tx_sink::TxSink, TxSender}, - }, - sync_layer::SyncState, +use crate::api_server::{ + execution_sandbox::{BlockArgs, BlockArgsError, BlockStartInfo}, + tx_sender::{tx_sink::TxSink, TxSender}, }; #[derive(Debug)] diff --git a/core/lib/zksync_core/src/consensus/en.rs b/core/lib/zksync_core/src/consensus/en.rs index adf246c34a15..48775bd4288e 100644 --- a/core/lib/zksync_core/src/consensus/en.rs +++ b/core/lib/zksync_core/src/consensus/en.rs @@ -3,16 +3,14 @@ use zksync_concurrency::{ctx, error::Wrap as _, scope, time}; use zksync_consensus_executor as executor; use zksync_consensus_roles::validator; use zksync_consensus_storage::BlockStore; +use zksync_node_sync::{ + fetcher::FetchedBlock, sync_action::ActionQueueSender, MainNodeClient, SyncState, +}; use zksync_types::L2BlockNumber; use zksync_web3_decl::client::{DynClient, L2}; use super::{config, storage::Store, ConnectionPool, ConsensusConfig, ConsensusSecrets}; -use crate::{ - consensus::storage, - sync_layer::{ - fetcher::FetchedBlock, sync_action::ActionQueueSender, MainNodeClient, SyncState, - }, -}; +use crate::consensus::storage; /// External node. pub(super) struct EN { diff --git a/core/lib/zksync_core/src/consensus/era.rs b/core/lib/zksync_core/src/consensus/era.rs index f042fef2fad7..3b532c88ba7c 100644 --- a/core/lib/zksync_core/src/consensus/era.rs +++ b/core/lib/zksync_core/src/consensus/era.rs @@ -10,7 +10,7 @@ use zksync_dal::Core; use zksync_web3_decl::client::{DynClient, L2}; use super::{en, storage::ConnectionPool}; -use crate::sync_layer::{sync_action::ActionQueueSender, SyncState}; +use zksync_node_sync::{sync_action::ActionQueueSender, SyncState}; /// Runs the consensus task in the main node mode. pub async fn run_main_node( diff --git a/core/lib/zksync_core/src/consensus/storage/mod.rs b/core/lib/zksync_core/src/consensus/storage/mod.rs index 257ee1baf62b..6a9fb6930121 100644 --- a/core/lib/zksync_core/src/consensus/storage/mod.rs +++ b/core/lib/zksync_core/src/consensus/storage/mod.rs @@ -11,7 +11,7 @@ use zksync_state_keeper::io::common::IoCursor; use zksync_types::L2BlockNumber; use super::config; -use crate::sync_layer::{ +use zksync_node_sync::{ fetcher::{FetchedBlock, FetchedTransaction, IoCursorExt as _}, sync_action::ActionQueueSender, SyncState, diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 63c32213f2e3..819cbc626cf5 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -11,6 +11,12 @@ use zksync_consensus_network as network; use zksync_consensus_roles::validator; use zksync_dal::{CoreDal, DalError}; use zksync_node_genesis::GenesisParams; +use zksync_node_sync::{ + fetcher::{FetchedTransaction, IoCursorExt as _}, + sync_action::{ActionQueue, ActionQueueSender, SyncAction}, + testonly::MockMainNodeClient, + ExternalIO, MainNodeClient, SyncState, +}; use zksync_node_test_utils::{create_l1_batch_metadata, create_l2_transaction}; use zksync_state_keeper::{ io::{IoCursor, L1BatchParams, L2BlockParams}, @@ -24,12 +30,6 @@ use zksync_web3_decl::client::{Client, DynClient, L2}; use crate::{ api_server::web3::{state::InternalApiConfig, tests::spawn_http_server}, consensus::{en, ConnectionPool}, - sync_layer::{ - fetcher::{FetchedTransaction, IoCursorExt as _}, - sync_action::{ActionQueue, ActionQueueSender, SyncAction}, - testonly::MockMainNodeClient, - ExternalIO, MainNodeClient, SyncState, - }, }; /// Fake StateKeeper for tests. diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index b7947e947590..48f50a5122b5 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -87,7 +87,6 @@ use crate::{ pub mod api_server; pub mod consensus; pub mod proto; -pub mod sync_layer; pub mod tee_verifier_input_producer; pub mod temp_config_store; pub mod utils; diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index c4edacfbf16c..70014773bd85 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -37,6 +37,7 @@ zksync_eth_sender.workspace = true zksync_state_keeper.workspace = true zksync_consistency_checker.workspace = true zksync_metadata_calculator.workspace = true +zksync_node_sync.workspace = true tracing.workspace = true thiserror.workspace = true diff --git a/core/node/node_framework/src/implementations/layers/consensus.rs b/core/node/node_framework/src/implementations/layers/consensus.rs index 3af88b0505f3..e3dc747b0c2b 100644 --- a/core/node/node_framework/src/implementations/layers/consensus.rs +++ b/core/node/node_framework/src/implementations/layers/consensus.rs @@ -1,11 +1,9 @@ use anyhow::Context as _; use zksync_concurrency::{ctx, scope}; use zksync_config::configs::consensus::{ConsensusConfig, ConsensusSecrets}; -use zksync_core::{ - consensus, - sync_layer::{ActionQueueSender, SyncState}, -}; +use zksync_core::consensus; use zksync_dal::{ConnectionPool, Core}; +use zksync_node_sync::{ActionQueueSender, SyncState}; use zksync_web3_decl::client::{DynClient, L2}; use crate::{ diff --git a/core/node/node_framework/src/implementations/resources/action_queue.rs b/core/node/node_framework/src/implementations/resources/action_queue.rs index 18d995688830..164f8dca3109 100644 --- a/core/node/node_framework/src/implementations/resources/action_queue.rs +++ b/core/node/node_framework/src/implementations/resources/action_queue.rs @@ -1,4 +1,4 @@ -use zksync_core::sync_layer::ActionQueueSender; +use zksync_node_sync::ActionQueueSender; use crate::resource::{Resource, Unique}; diff --git a/core/node/node_framework/src/implementations/resources/sync_state.rs b/core/node/node_framework/src/implementations/resources/sync_state.rs index e5c54cf1bfe9..87eb565ac077 100644 --- a/core/node/node_framework/src/implementations/resources/sync_state.rs +++ b/core/node/node_framework/src/implementations/resources/sync_state.rs @@ -1,4 +1,4 @@ -use zksync_core::sync_layer::SyncState; +use zksync_node_sync::SyncState; use crate::resource::Resource; diff --git a/core/node/node_sync/Cargo.toml b/core/node/node_sync/Cargo.toml new file mode 100644 index 000000000000..248478abddf5 --- /dev/null +++ b/core/node/node_sync/Cargo.toml @@ -0,0 +1,41 @@ +[package] +name = "zksync_node_sync" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_config.workspace = true +zksync_contracts.workspace = true +zksync_dal.workspace = true +zksync_node_genesis.workspace = true +zksync_system_constants.workspace = true +zksync_types.workspace = true +zksync_state_keeper.workspace = true +zksync_shared_metrics.workspace = true +zksync_web3_decl.workspace = true +zksync_health_check.workspace = true +zksync_utils.workspace = true +zksync_eth_client.workspace = true +zksync_concurrency.workspace = true +vise.workspace = true +vm_utils.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +chrono.workspace = true +tracing.workspace = true +serde.workspace = true +tokio = { workspace = true, features = ["time"] } +thiserror.workspace = true + +[dev-dependencies] +zksync_node_test_utils.workspace = true + +assert_matches.workspace = true +test-casing.workspace = true diff --git a/core/node/node_sync/README.md b/core/node/node_sync/README.md new file mode 100644 index 000000000000..afbe1b378a3c --- /dev/null +++ b/core/node/node_sync/README.md @@ -0,0 +1,3 @@ +# `zksync_node_sync` + +A set of primitives and components that enable synchronization of ZK Stack nodes replicas. diff --git a/core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs b/core/node/node_sync/src/batch_status_updater/mod.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/batch_status_updater/mod.rs rename to core/node/node_sync/src/batch_status_updater/mod.rs diff --git a/core/lib/zksync_core/src/sync_layer/batch_status_updater/tests.rs b/core/node/node_sync/src/batch_status_updater/tests.rs similarity index 99% rename from core/lib/zksync_core/src/sync_layer/batch_status_updater/tests.rs rename to core/node/node_sync/src/batch_status_updater/tests.rs index be0df56dfbf3..f3850ccfe362 100644 --- a/core/lib/zksync_core/src/sync_layer/batch_status_updater/tests.rs +++ b/core/node/node_sync/src/batch_status_updater/tests.rs @@ -11,7 +11,7 @@ use zksync_node_test_utils::{create_l1_batch, create_l2_block, prepare_recovery_ use zksync_types::{Address, ProtocolVersionId}; use super::*; -use crate::sync_layer::metrics::L1BatchStage; +use crate::metrics::L1BatchStage; async fn seal_l1_batch(storage: &mut Connection<'_, Core>, number: L1BatchNumber) { let mut storage = storage.start_transaction().await.unwrap(); diff --git a/core/lib/zksync_core/src/sync_layer/client.rs b/core/node/node_sync/src/client.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/client.rs rename to core/node/node_sync/src/client.rs diff --git a/core/lib/zksync_core/src/sync_layer/external_io.rs b/core/node/node_sync/src/external_io.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/external_io.rs rename to core/node/node_sync/src/external_io.rs diff --git a/core/lib/zksync_core/src/sync_layer/fetcher.rs b/core/node/node_sync/src/fetcher.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/fetcher.rs rename to core/node/node_sync/src/fetcher.rs diff --git a/core/lib/zksync_core/src/sync_layer/genesis.rs b/core/node/node_sync/src/genesis.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/genesis.rs rename to core/node/node_sync/src/genesis.rs diff --git a/core/lib/zksync_core/src/sync_layer/mod.rs b/core/node/node_sync/src/lib.rs similarity index 96% rename from core/lib/zksync_core/src/sync_layer/mod.rs rename to core/node/node_sync/src/lib.rs index 7bb825fc8bb6..663a924e52b7 100644 --- a/core/lib/zksync_core/src/sync_layer/mod.rs +++ b/core/node/node_sync/src/lib.rs @@ -4,7 +4,7 @@ pub mod external_io; pub mod fetcher; pub mod genesis; mod metrics; -pub(crate) mod sync_action; +pub mod sync_action; mod sync_state; pub mod testonly; #[cfg(test)] diff --git a/core/lib/zksync_core/src/sync_layer/metrics.rs b/core/node/node_sync/src/metrics.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/metrics.rs rename to core/node/node_sync/src/metrics.rs diff --git a/core/lib/zksync_core/src/sync_layer/sync_action.rs b/core/node/node_sync/src/sync_action.rs similarity index 99% rename from core/lib/zksync_core/src/sync_layer/sync_action.rs rename to core/node/node_sync/src/sync_action.rs index 23d921213d52..09d49943a454 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_action.rs +++ b/core/node/node_sync/src/sync_action.rs @@ -13,7 +13,7 @@ impl ActionQueueSender { /// Requires that the actions are in the correct order: starts with a new open L1 batch / L2 block, /// followed by 0 or more transactions, have mandatory `SealL2Block` and optional `SealBatch` at the end. /// Would panic if the order is incorrect. - pub(crate) async fn push_actions(&self, actions: Vec) { + pub async fn push_actions(&self, actions: Vec) { Self::check_action_sequence(&actions).unwrap(); for action in actions { self.0.send(action).await.expect("EN sync logic panicked"); diff --git a/core/lib/zksync_core/src/sync_layer/sync_state.rs b/core/node/node_sync/src/sync_state.rs similarity index 96% rename from core/lib/zksync_core/src/sync_layer/sync_state.rs rename to core/node/node_sync/src/sync_state.rs index af71480fe81f..53f13dee517d 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_state.rs +++ b/core/node/node_sync/src/sync_state.rs @@ -34,15 +34,15 @@ impl Default for SyncState { const SYNC_L2_BLOCK_DELTA: u32 = 10; impl SyncState { - pub(crate) fn get_main_node_block(&self) -> L2BlockNumber { + pub fn get_main_node_block(&self) -> L2BlockNumber { self.0.borrow().main_node_block.unwrap_or_default() } - pub(crate) fn get_local_block(&self) -> L2BlockNumber { + pub fn get_local_block(&self) -> L2BlockNumber { self.0.borrow().local_block.unwrap_or_default() } - pub(crate) async fn wait_for_local_block(&self, want: L2BlockNumber) { + pub async fn wait_for_local_block(&self, want: L2BlockNumber) { self.0 .subscribe() .wait_for(|inner| matches!(inner.local_block, Some(got) if got >= want)) @@ -50,7 +50,7 @@ impl SyncState { .unwrap(); } - pub(crate) async fn wait_for_main_node_block( + pub async fn wait_for_main_node_block( &self, ctx: &ctx::Ctx, want: L2BlockNumber, @@ -64,7 +64,7 @@ impl SyncState { Ok(()) } - pub(crate) fn set_main_node_block(&self, block: L2BlockNumber) { + pub fn set_main_node_block(&self, block: L2BlockNumber) { self.0.send_modify(|inner| inner.set_main_node_block(block)); } @@ -72,7 +72,7 @@ impl SyncState { self.0.send_modify(|inner| inner.set_local_block(block)); } - pub(crate) fn is_synced(&self) -> bool { + pub fn is_synced(&self) -> bool { self.0.borrow().is_synced().0 } diff --git a/core/lib/zksync_core/src/sync_layer/testonly.rs b/core/node/node_sync/src/testonly.rs similarity index 98% rename from core/lib/zksync_core/src/sync_layer/testonly.rs rename to core/node/node_sync/src/testonly.rs index 6d84e2c1ee1c..502da17ffd98 100644 --- a/core/lib/zksync_core/src/sync_layer/testonly.rs +++ b/core/node/node_sync/src/testonly.rs @@ -9,7 +9,7 @@ use zksync_web3_decl::error::EnrichedClientResult; use super::MainNodeClient; #[derive(Debug, Default)] -pub(crate) struct MockMainNodeClient { +pub struct MockMainNodeClient { pub l2_blocks: Vec, pub block_number_offset: u32, pub protocol_versions: HashMap, diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/node/node_sync/src/tests.rs similarity index 100% rename from core/lib/zksync_core/src/sync_layer/tests.rs rename to core/node/node_sync/src/tests.rs From 3682b4834216a851ff915e6515019a0105aae806 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 14:25:56 +0400 Subject: [PATCH 09/15] Extract consensus, api_server, contract_verification_server --- Cargo.lock | 106 +++++++++++ Cargo.toml | 6 + core/bin/external_node/Cargo.toml | 2 + core/bin/external_node/src/config/mod.rs | 12 +- core/bin/external_node/src/main.rs | 18 +- core/lib/dal/src/helpers.rs | 2 +- core/lib/zksync_core/Cargo.toml | 3 + core/lib/zksync_core/src/lib.rs | 79 +++----- core/node/api_server/Cargo.toml | 60 ++++++ core/node/api_server/README.md | 3 + .../src}/execution_sandbox/apply.rs | 0 .../src}/execution_sandbox/error.rs | 0 .../src}/execution_sandbox/execute.rs | 10 +- .../api_server/src}/execution_sandbox/mod.rs | 3 +- .../src}/execution_sandbox/testonly.rs | 20 +- .../src}/execution_sandbox/tests.rs | 2 +- .../src}/execution_sandbox/tracers.rs | 0 .../src}/execution_sandbox/validate.rs | 1 - .../src}/execution_sandbox/vm_metrics.rs | 11 +- .../api_server/src}/healthcheck.rs | 0 .../mod.rs => node/api_server/src/lib.rs} | 1 - .../src}/tx_sender/master_pool_sink.rs | 2 +- .../api_server/src}/tx_sender/mod.rs | 48 ++++- .../api_server/src}/tx_sender/proxy.rs | 0 .../api_server/src}/tx_sender/result.rs | 2 +- .../api_server/src}/tx_sender/tests.rs | 37 +--- .../api_server/src}/tx_sender/tx_sink.rs | 0 .../api_server/src}/utils.rs | 0 .../src}/web3/backend_jsonrpsee/metadata.rs | 8 +- .../src}/web3/backend_jsonrpsee/middleware.rs | 2 +- .../src}/web3/backend_jsonrpsee/mod.rs | 2 +- .../backend_jsonrpsee/namespaces/debug.rs | 2 +- .../web3/backend_jsonrpsee/namespaces/en.rs | 2 +- .../web3/backend_jsonrpsee/namespaces/eth.rs | 2 +- .../web3/backend_jsonrpsee/namespaces/mod.rs | 0 .../web3/backend_jsonrpsee/namespaces/net.rs | 2 +- .../backend_jsonrpsee/namespaces/snapshots.rs | 2 +- .../web3/backend_jsonrpsee/namespaces/web3.rs | 2 +- .../web3/backend_jsonrpsee/namespaces/zks.rs | 2 +- .../src}/web3/backend_jsonrpsee/testonly.rs | 0 .../api_server/src}/web3/mempool_cache.rs | 0 .../api_server/src}/web3/metrics.rs | 10 +- .../api_server/src}/web3/mod.rs | 11 +- .../api_server/src}/web3/namespaces/debug.rs | 2 +- .../api_server/src}/web3/namespaces/en.rs | 2 +- .../api_server/src}/web3/namespaces/eth.rs | 2 +- .../api_server/src}/web3/namespaces/mod.rs | 0 .../api_server/src}/web3/namespaces/net.rs | 0 .../src}/web3/namespaces/snapshots.rs | 2 +- .../api_server/src}/web3/namespaces/web3.rs | 0 .../api_server/src}/web3/namespaces/zks.rs | 2 +- .../api_server/src}/web3/pubsub.rs | 2 +- .../api_server/src}/web3/state.rs | 2 +- core/node/api_server/src/web3/testonly.rs | 178 ++++++++++++++++++ .../api_server/src}/web3/tests/debug.rs | 0 .../api_server/src}/web3/tests/filters.rs | 0 .../api_server/src}/web3/tests/mod.rs | 136 +------------ .../api_server/src}/web3/tests/snapshots.rs | 0 .../api_server/src}/web3/tests/vm.rs | 0 .../api_server/src}/web3/tests/ws.rs | 2 +- core/node/consensus/Cargo.toml | 41 ++++ core/node/consensus/README.md | 3 + .../consensus/src}/config.rs | 0 .../consensus => node/consensus/src}/en.rs | 2 +- .../consensus => node/consensus/src}/era.rs | 2 +- .../mod.rs => node/consensus/src/lib.rs} | 3 +- .../consensus/src}/storage/mod.rs | 8 +- .../consensus/src}/storage/testonly.rs | 0 .../consensus/src}/testonly.rs | 6 +- .../consensus => node/consensus/src}/tests.rs | 0 .../contract_verification_server/Cargo.toml | 24 +++ .../contract_verification_server/README.md | 3 + .../src}/api_decl.rs | 0 .../src}/api_impl.rs | 0 .../contract_verification_server/src/lib.rs} | 0 .../src}/metrics.rs | 0 core/node/metadata_calculator/README.md | 4 +- core/node/node_framework/Cargo.toml | 3 + .../node/node_framework/examples/main_node.rs | 12 +- .../src/implementations/layers/consensus.rs | 4 +- .../layers/contract_verification_api.rs | 2 +- .../layers/healtcheck_server.rs | 2 +- .../implementations/layers/web3_api/caches.rs | 2 +- .../implementations/layers/web3_api/server.rs | 2 +- .../layers/web3_api/tx_sender.rs | 2 +- .../layers/web3_api/tx_sink.rs | 2 +- .../src/implementations/resources/web3_api.rs | 4 +- 87 files changed, 604 insertions(+), 332 deletions(-) create mode 100644 core/node/api_server/Cargo.toml create mode 100644 core/node/api_server/README.md rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/apply.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/error.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/execute.rs (97%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/mod.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/testonly.rs (88%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/tests.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/tracers.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/validate.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/execution_sandbox/vm_metrics.rs (97%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/healthcheck.rs (100%) rename core/{lib/zksync_core/src/api_server/mod.rs => node/api_server/src/lib.rs} (85%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/master_pool_sink.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/mod.rs (96%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/proxy.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/result.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/tests.rs (83%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/tx_sender/tx_sink.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/utils.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/metadata.rs (96%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/middleware.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/mod.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/debug.rs (97%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/en.rs (96%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/eth.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/mod.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/net.rs (90%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/snapshots.rs (92%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/web3.rs (84%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/namespaces/zks.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/backend_jsonrpsee/testonly.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/mempool_cache.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/metrics.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/mod.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/debug.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/en.rs (98%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/eth.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/mod.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/net.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/snapshots.rs (97%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/web3.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/namespaces/zks.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/pubsub.rs (99%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/state.rs (99%) create mode 100644 core/node/api_server/src/web3/testonly.rs rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/debug.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/filters.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/mod.rs (88%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/snapshots.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/vm.rs (100%) rename core/{lib/zksync_core/src/api_server => node/api_server/src}/web3/tests/ws.rs (99%) create mode 100644 core/node/consensus/Cargo.toml create mode 100644 core/node/consensus/README.md rename core/{lib/zksync_core/src/consensus => node/consensus/src}/config.rs (100%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/en.rs (99%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/era.rs (100%) rename core/{lib/zksync_core/src/consensus/mod.rs => node/consensus/src/lib.rs} (98%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/storage/mod.rs (100%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/storage/testonly.rs (100%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/testonly.rs (99%) rename core/{lib/zksync_core/src/consensus => node/consensus/src}/tests.rs (100%) create mode 100644 core/node/contract_verification_server/Cargo.toml create mode 100644 core/node/contract_verification_server/README.md rename core/{lib/zksync_core/src/api_server/contract_verification => node/contract_verification_server/src}/api_decl.rs (100%) rename core/{lib/zksync_core/src/api_server/contract_verification => node/contract_verification_server/src}/api_impl.rs (100%) rename core/{lib/zksync_core/src/api_server/contract_verification/mod.rs => node/contract_verification_server/src/lib.rs} (100%) rename core/{lib/zksync_core/src/api_server/contract_verification => node/contract_verification_server/src}/metrics.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index c13ffc946fc8..a410d8af3e83 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8282,6 +8282,23 @@ dependencies = [ "zksync_types", ] +[[package]] +name = "zksync_contract_verification_server" +version = "0.1.0" +dependencies = [ + "anyhow", + "axum", + "serde", + "serde_json", + "tokio", + "tower-http", + "tracing", + "vise", + "zksync_config", + "zksync_dal", + "zksync_types", +] + [[package]] name = "zksync_contract_verifier" version = "0.1.0" @@ -8378,6 +8395,7 @@ dependencies = [ "zksync_consensus_roles", "zksync_consensus_storage", "zksync_consensus_utils", + "zksync_contract_verification_server", "zksync_contracts", "zksync_dal", "zksync_db_connection", @@ -8392,6 +8410,8 @@ dependencies = [ "zksync_merkle_tree", "zksync_metadata_calculator", "zksync_mini_merkle_tree", + "zksync_node_api_server", + "zksync_node_consensus", "zksync_node_fee_model", "zksync_node_genesis", "zksync_node_sync", @@ -8620,6 +8640,8 @@ dependencies = [ "zksync_health_check", "zksync_l1_contract_interface", "zksync_metadata_calculator", + "zksync_node_api_server", + "zksync_node_consensus", "zksync_node_db_pruner", "zksync_node_fee_model", "zksync_node_genesis", @@ -8767,6 +8789,87 @@ dependencies = [ "zksync_crypto", ] +[[package]] +name = "zksync_node_api_server" +version = "0.1.0" +dependencies = [ + "anyhow", + "assert_matches", + "async-trait", + "axum", + "chrono", + "futures 0.3.28", + "governor", + "hex", + "itertools 0.10.5", + "jsonrpsee", + "lru", + "multivm", + "once_cell", + "pin-project-lite", + "rand 0.8.5", + "reqwest", + "serde", + "serde_json", + "test-casing", + "thiserror", + "thread_local", + "tokio", + "tower", + "tower-http", + "tracing", + "vise", + "zksync_config", + "zksync_contracts", + "zksync_dal", + "zksync_health_check", + "zksync_metadata_calculator", + "zksync_mini_merkle_tree", + "zksync_node_fee_model", + "zksync_node_genesis", + "zksync_node_sync", + "zksync_node_test_utils", + "zksync_protobuf", + "zksync_shared_metrics", + "zksync_state", + "zksync_state_keeper", + "zksync_system_constants", + "zksync_types", + "zksync_utils", + "zksync_web3_decl", +] + +[[package]] +name = "zksync_node_consensus" +version = "0.1.0" +dependencies = [ + "anyhow", + "async-trait", + "rand 0.8.5", + "secrecy", + "test-casing", + "tokio", + "tracing", + "zksync_concurrency", + "zksync_config", + "zksync_consensus_bft", + "zksync_consensus_crypto", + "zksync_consensus_executor", + "zksync_consensus_network", + "zksync_consensus_roles", + "zksync_consensus_storage", + "zksync_consensus_utils", + "zksync_dal", + "zksync_node_api_server", + "zksync_node_genesis", + "zksync_node_sync", + "zksync_node_test_utils", + "zksync_protobuf", + "zksync_state_keeper", + "zksync_types", + "zksync_web3_decl", +] + [[package]] name = "zksync_node_db_pruner" version = "0.1.0" @@ -8828,6 +8931,7 @@ dependencies = [ "zksync_concurrency", "zksync_config", "zksync_consistency_checker", + "zksync_contract_verification_server", "zksync_contracts", "zksync_core", "zksync_dal", @@ -8839,6 +8943,8 @@ dependencies = [ "zksync_health_check", "zksync_house_keeper", "zksync_metadata_calculator", + "zksync_node_api_server", + "zksync_node_consensus", "zksync_node_fee_model", "zksync_node_sync", "zksync_object_store", diff --git a/Cargo.toml b/Cargo.toml index 0a33f5b4ec29..268997afcb1c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,6 +28,9 @@ members = [ "core/node/consistency_checker", "core/node/metadata_calculator", "core/node/node_sync", + "core/node/consensus", + "core/node/contract_verification_server", + "core/node/api_server", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -253,3 +256,6 @@ zksync_reorg_detector = { path = "core/node/reorg_detector" } zksync_consistency_checker = { path = "core/node/consistency_checker" } zksync_metadata_calculator = { path = "core/node/metadata_calculator" } zksync_node_sync = { path = "core/node/node_sync" } +zksync_node_consensus = { path = "core/node/consensus" } +zksync_contract_verification_server = { path = "core/node/contract_verification_server" } +zksync_node_api_server = { path = "core/node/api_server" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index 29203cb73721..2e87b02e9e4b 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -40,6 +40,8 @@ zksync_reorg_detector.workspace = true zksync_consistency_checker.workspace = true zksync_metadata_calculator.workspace = true zksync_node_sync.workspace = true +zksync_node_api_server.workspace = true +zksync_node_consensus.workspace = true vlog.workspace = true zksync_concurrency.workspace = true diff --git a/core/bin/external_node/src/config/mod.rs b/core/bin/external_node/src/config/mod.rs index 7f6be8273b35..13e504ce06d2 100644 --- a/core/bin/external_node/src/config/mod.rs +++ b/core/bin/external_node/src/config/mod.rs @@ -14,15 +14,13 @@ use zksync_config::{ }, ObjectStoreConfig, }; -use zksync_core::{ - api_server::{ - tx_sender::TxSenderConfig, - web3::{state::InternalApiConfig, Namespace}, - }, - temp_config_store::decode_yaml_repr, -}; +use zksync_core::temp_config_store::decode_yaml_repr; #[cfg(test)] use zksync_dal::{ConnectionPool, Core}; +use zksync_node_api_server::{ + tx_sender::TxSenderConfig, + web3::{state::InternalApiConfig, Namespace}, +}; use zksync_protobuf_config::proto; use zksync_snapshots_applier::SnapshotsApplierConfig; use zksync_types::{ diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 43a8452a18c1..ac885b0101e7 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -12,16 +12,7 @@ use zksync_commitment_generator::CommitmentGenerator; use zksync_concurrency::{ctx, scope}; use zksync_config::configs::{api::MerkleTreeApiConfig, database::MerkleTreeMode}; use zksync_consistency_checker::ConsistencyChecker; -use zksync_core::{ - api_server::{ - execution_sandbox::VmConcurrencyLimiter, - healthcheck::HealthCheckHandle, - tx_sender::{proxy::TxProxy, ApiContracts, TxSenderBuilder}, - web3::{mempool_cache::MempoolCache, ApiBuilder, Namespace}, - }, - consensus, setup_sigint_handler, - utils::L1BatchCommitmentModeValidationTask, -}; +use zksync_core::{setup_sigint_handler, utils::L1BatchCommitmentModeValidationTask}; use zksync_dal::{metrics::PostgresMetrics, ConnectionPool, Core, CoreDal}; use zksync_db_connection::{ connection_pool::ConnectionPoolBuilder, healthcheck::ConnectionPoolHealthCheck, @@ -32,6 +23,13 @@ use zksync_metadata_calculator::{ api_server::{TreeApiClient, TreeApiHttpClient}, MetadataCalculator, MetadataCalculatorConfig, }; +use zksync_node_api_server::{ + execution_sandbox::VmConcurrencyLimiter, + healthcheck::HealthCheckHandle, + tx_sender::{proxy::TxProxy, ApiContracts, TxSenderBuilder}, + web3::{mempool_cache::MempoolCache, ApiBuilder, Namespace}, +}; +use zksync_node_consensus as consensus; use zksync_node_db_pruner::{DbPruner, DbPrunerConfig}; use zksync_node_fee_model::l1_gas_price::MainNodeFeeParamsFetcher; use zksync_node_sync::{ diff --git a/core/lib/dal/src/helpers.rs b/core/lib/dal/src/helpers.rs index d584a14270e4..e8e11f1cc5f3 100644 --- a/core/lib/dal/src/helpers.rs +++ b/core/lib/dal/src/helpers.rs @@ -40,10 +40,10 @@ pub async fn wait_for_l1_batch( #[cfg(test)] mod tests { - use super::*; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{block::L1BatchHeader, ProtocolVersion, ProtocolVersionId, H256}; + use super::*; use crate::{ConnectionPool, Core, CoreDal}; #[tokio::test] diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 64e40e9858b1..511e13e0898e 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -54,6 +54,9 @@ zksync_node_fee_model.workspace = true zksync_state_keeper.workspace = true zksync_metadata_calculator.workspace = true zksync_node_sync.workspace = true +zksync_node_consensus.workspace = true +zksync_contract_verification_server.workspace = true +zksync_node_api_server.workspace = true multivm.workspace = true # Consensus dependenices diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 48f50a5122b5..85a7ce0c234d 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -8,7 +8,6 @@ use std::{ }; use anyhow::Context as _; -use api_server::tx_sender::master_pool_sink::MasterPoolSink; use prometheus_exporter::PrometheusExporterConfig; use prover_dal::Prover; use temp_config_store::Secrets; @@ -56,9 +55,13 @@ use zksync_house_keeper::{ use zksync_metadata_calculator::{ api_server::TreeApiHttpClient, MetadataCalculator, MetadataCalculatorConfig, }; +use zksync_node_api_server::{ + healthcheck::HealthCheckHandle, + tx_sender::{build_tx_sender, TxSenderConfig}, + web3::{self, mempool_cache::MempoolCache, state::InternalApiConfig, Namespace}, +}; use zksync_node_fee_model::{ - l1_gas_price::GasAdjusterSingleton, ApiFeeInputProvider, BatchFeeModelInputProvider, - MainNodeFeeInputProvider, + l1_gas_price::GasAdjusterSingleton, BatchFeeModelInputProvider, MainNodeFeeInputProvider, }; use zksync_node_genesis::{ensure_genesis_state, GenesisParams}; use zksync_object_store::{ObjectStore, ObjectStoreFactory}; @@ -67,25 +70,16 @@ use zksync_shared_metrics::{InitStage, APP_METRICS}; use zksync_state::{PostgresStorageCaches, RocksdbStorageOptions}; use zksync_state_keeper::{ create_state_keeper, AsyncRocksdbCache, MempoolFetcher, MempoolGuard, OutputHandler, - SequencerSealer, StateKeeperPersistence, + StateKeeperPersistence, }; use zksync_types::{ethabi::Contract, fee_model::FeeModelConfig, Address, L2ChainId}; use zksync_web3_decl::client::Client; use crate::{ - api_server::{ - contract_verification, - execution_sandbox::{VmConcurrencyBarrier, VmConcurrencyLimiter}, - healthcheck::HealthCheckHandle, - tx_sender::{ApiContracts, TxSender, TxSenderBuilder, TxSenderConfig}, - web3::{self, mempool_cache::MempoolCache, state::InternalApiConfig, Namespace}, - }, tee_verifier_input_producer::TeeVerifierInputProducer, utils::L1BatchCommitmentModeValidationTask, }; -pub mod api_server; -pub mod consensus; pub mod proto; pub mod tee_verifier_input_producer; pub mod temp_config_store; @@ -460,15 +454,17 @@ pub async fn initialize_components( if components.contains(&Component::ContractVerificationApi) { let started_at = Instant::now(); tracing::info!("initializing contract verification REST API"); - task_futures.push(tokio::spawn(contract_verification::start_server( - connection_pool.clone(), - replica_connection_pool.clone(), - configs - .contract_verifier - .clone() - .context("Contract verifier")?, - stop_receiver.clone(), - ))); + task_futures.push(tokio::spawn( + zksync_contract_verification_server::start_server( + connection_pool.clone(), + replica_connection_pool.clone(), + configs + .contract_verifier + .clone() + .context("Contract verifier")?, + stop_receiver.clone(), + ), + )); let elapsed = started_at.elapsed(); APP_METRICS.init_latency[&InitStage::ContractVerificationApi].set(elapsed); tracing::info!("initialized contract verification REST API in {elapsed:?}"); @@ -548,7 +544,9 @@ pub async fn initialize_components( // but we only need to wait for stop signal once, and it will be propagated to all child contexts. let root_ctx = ctx::root(); scope::run!(&root_ctx, |ctx, s| async move { - s.spawn_bg(consensus::era::run_main_node(ctx, cfg, secrets, pool)); + s.spawn_bg(zksync_node_consensus::era::run_main_node( + ctx, cfg, secrets, pool, + )); let _ = stop_receiver.wait_for(|stop| *stop).await?; Ok(()) }) @@ -1213,41 +1211,6 @@ fn build_storage_caches( Ok(storage_caches) } -async fn build_tx_sender( - tx_sender_config: &TxSenderConfig, - web3_json_config: &Web3JsonRpcConfig, - state_keeper_config: &StateKeeperConfig, - replica_pool: ConnectionPool, - master_pool: ConnectionPool, - batch_fee_model_input_provider: Arc, - storage_caches: PostgresStorageCaches, -) -> (TxSender, VmConcurrencyBarrier) { - let sequencer_sealer = SequencerSealer::new(state_keeper_config.clone()); - let master_pool_sink = MasterPoolSink::new(master_pool); - let tx_sender_builder = TxSenderBuilder::new( - tx_sender_config.clone(), - replica_pool.clone(), - Arc::new(master_pool_sink), - ) - .with_sealer(Arc::new(sequencer_sealer)); - - let max_concurrency = web3_json_config.vm_concurrency_limit(); - let (vm_concurrency_limiter, vm_barrier) = VmConcurrencyLimiter::new(max_concurrency); - - let batch_fee_input_provider = - ApiFeeInputProvider::new(batch_fee_model_input_provider, replica_pool); - - let tx_sender = tx_sender_builder - .build( - Arc::new(batch_fee_input_provider), - Arc::new(vm_concurrency_limiter), - ApiContracts::load_from_disk(), - storage_caches, - ) - .await; - (tx_sender, vm_barrier) -} - #[allow(clippy::too_many_arguments)] async fn run_http_api( task_futures: &mut Vec>>, diff --git a/core/node/api_server/Cargo.toml b/core/node/api_server/Cargo.toml new file mode 100644 index 000000000000..9009c66e1469 --- /dev/null +++ b/core/node/api_server/Cargo.toml @@ -0,0 +1,60 @@ +[package] +name = "zksync_node_api_server" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_config.workspace = true +zksync_contracts.workspace = true +zksync_types.workspace = true +zksync_dal.workspace = true +zksync_node_sync.workspace = true +zksync_health_check.workspace = true +zksync_node_fee_model.workspace = true +zksync_state_keeper.workspace = true +zksync_shared_metrics.workspace = true +zksync_state.workspace = true +zksync_system_constants.workspace = true +zksync_metadata_calculator.workspace = true +zksync_web3_decl.workspace = true +zksync_utils.workspace = true +zksync_protobuf.workspace = true +zksync_mini_merkle_tree.workspace = true +multivm.workspace = true +vise.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +axum.workspace = true +chrono.workspace = true +futures.workspace = true +tokio = { workspace = true, features = ["rt", "time"] } +tracing.workspace = true +thiserror.workspace = true +once_cell.workspace = true +rand.workspace = true +serde.workspace = true +serde_json.workspace = true +itertools.workspace = true +thread_local.workspace = true +governor.workspace = true +pin-project-lite.workspace = true +hex.workspace = true +jsonrpsee.workspace = true +reqwest.workspace = true +tower.workspace = true +tower-http.workspace = true +lru.workspace = true + +[dev-dependencies] +zksync_node_genesis.workspace = true +zksync_node_test_utils.workspace = true + +assert_matches.workspace = true +test-casing.workspace = true diff --git a/core/node/api_server/README.md b/core/node/api_server/README.md new file mode 100644 index 000000000000..27dbb57b2c89 --- /dev/null +++ b/core/node/api_server/README.md @@ -0,0 +1,3 @@ +# `zksync_node_api_server` + +Implementation of Web3 API used by ZK Stack nodes. diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs b/core/node/api_server/src/execution_sandbox/apply.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs rename to core/node/api_server/src/execution_sandbox/apply.rs diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/error.rs b/core/node/api_server/src/execution_sandbox/error.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/execution_sandbox/error.rs rename to core/node/api_server/src/execution_sandbox/error.rs diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs b/core/node/api_server/src/execution_sandbox/execute.rs similarity index 97% rename from core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs rename to core/node/api_server/src/execution_sandbox/execute.rs index afdfbffd6fc8..2fd5b376acb3 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs +++ b/core/node/api_server/src/execution_sandbox/execute.rs @@ -14,9 +14,10 @@ use zksync_types::{ PackedEthSignature, Transaction, U256, }; -#[cfg(test)] -use super::testonly::MockTransactionExecutor; -use super::{apply, vm_metrics, ApiTracer, BlockArgs, TxSharedArgs, VmPermit}; +use super::{ + apply, testonly::MockTransactionExecutor, vm_metrics, ApiTracer, BlockArgs, TxSharedArgs, + VmPermit, +}; #[derive(Debug)] pub(crate) struct TxExecutionArgs { @@ -90,7 +91,7 @@ pub(crate) struct TransactionExecutionOutput { #[derive(Debug)] pub(crate) enum TransactionExecutor { Real, - #[cfg(test)] + #[doc(hidden)] // Intended for tests only Mock(MockTransactionExecutor), } @@ -113,7 +114,6 @@ impl TransactionExecutor { block_args: BlockArgs, custom_tracers: Vec, ) -> anyhow::Result { - #[cfg(test)] if let Self::Mock(mock_executor) = self { return mock_executor.execute_tx(&tx, &block_args); } diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs b/core/node/api_server/src/execution_sandbox/mod.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs rename to core/node/api_server/src/execution_sandbox/mod.rs index 42de8e285d91..72c6ba9789f5 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs +++ b/core/node/api_server/src/execution_sandbox/mod.rs @@ -26,8 +26,7 @@ use super::tx_sender::MultiVMBaseSystemContracts; mod apply; mod error; mod execute; -#[cfg(test)] -pub(super) mod testonly; +pub mod testonly; #[cfg(test)] mod tests; mod tracers; diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/testonly.rs b/core/node/api_server/src/execution_sandbox/testonly.rs similarity index 88% rename from core/lib/zksync_core/src/api_server/execution_sandbox/testonly.rs rename to core/node/api_server/src/execution_sandbox/testonly.rs index 8662101d5068..f027acc6d625 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/testonly.rs +++ b/core/node/api_server/src/execution_sandbox/testonly.rs @@ -13,7 +13,7 @@ use super::{ type TxResponseFn = dyn Fn(&Transaction, &BlockArgs) -> VmExecutionResultAndLogs + Send + Sync; -pub(crate) struct MockTransactionExecutor { +pub struct MockTransactionExecutor { call_responses: Box, tx_responses: Box, } @@ -43,20 +43,23 @@ impl Default for MockTransactionExecutor { } impl MockTransactionExecutor { - pub fn set_call_responses(&mut self, responses: F) + #[cfg(test)] + pub(crate) fn set_call_responses(&mut self, responses: F) where F: Fn(&Transaction, &BlockArgs) -> ExecutionResult + 'static + Send + Sync, { self.call_responses = self.wrap_responses(responses); } - pub fn set_tx_responses(&mut self, responses: F) + #[cfg(test)] + pub(crate) fn set_tx_responses(&mut self, responses: F) where F: Fn(&Transaction, &BlockArgs) -> ExecutionResult + 'static + Send + Sync, { self.tx_responses = self.wrap_responses(responses); } + #[cfg(test)] fn wrap_responses(&mut self, responses: F) -> Box where F: Fn(&Transaction, &BlockArgs) -> ExecutionResult + 'static + Send + Sync, @@ -73,14 +76,19 @@ impl MockTransactionExecutor { ) } - pub fn set_tx_responses_with_logs(&mut self, responses: F) + #[cfg(test)] + pub(crate) fn set_tx_responses_with_logs(&mut self, responses: F) where F: Fn(&Transaction, &BlockArgs) -> VmExecutionResultAndLogs + 'static + Send + Sync, { self.tx_responses = Box::new(responses); } - pub fn validate_tx(&self, tx: L2Tx, block_args: &BlockArgs) -> Result<(), ValidationError> { + pub(crate) fn validate_tx( + &self, + tx: L2Tx, + block_args: &BlockArgs, + ) -> Result<(), ValidationError> { let result = (self.tx_responses)(&tx.into(), block_args); match result.result { ExecutionResult::Success { .. } => Ok(()), @@ -90,7 +98,7 @@ impl MockTransactionExecutor { } } - pub fn execute_tx( + pub(crate) fn execute_tx( &self, tx: &Transaction, block_args: &BlockArgs, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/tests.rs b/core/node/api_server/src/execution_sandbox/tests.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/execution_sandbox/tests.rs rename to core/node/api_server/src/execution_sandbox/tests.rs index a3e6bb6ad0cf..9abe97f90254 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/tests.rs +++ b/core/node/api_server/src/execution_sandbox/tests.rs @@ -6,7 +6,7 @@ use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; use super::*; -use crate::api_server::{execution_sandbox::apply::apply_vm_in_sandbox, tx_sender::ApiContracts}; +use crate::{execution_sandbox::apply::apply_vm_in_sandbox, tx_sender::ApiContracts}; #[tokio::test] async fn creating_block_args() { diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs b/core/node/api_server/src/execution_sandbox/tracers.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs rename to core/node/api_server/src/execution_sandbox/tracers.rs diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs b/core/node/api_server/src/execution_sandbox/validate.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs rename to core/node/api_server/src/execution_sandbox/validate.rs index 5540fa3ec36a..0356ac74c5c1 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs +++ b/core/node/api_server/src/execution_sandbox/validate.rs @@ -40,7 +40,6 @@ impl TransactionExecutor { block_args: BlockArgs, computational_gas_limit: u32, ) -> Result<(), ValidationError> { - #[cfg(test)] if let Self::Mock(mock) = self { return mock.validate_tx(tx, &block_args); } diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs b/core/node/api_server/src/execution_sandbox/vm_metrics.rs similarity index 97% rename from core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs rename to core/node/api_server/src/execution_sandbox/vm_metrics.rs index 6c1f92ed68b4..33100169e392 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs +++ b/core/node/api_server/src/execution_sandbox/vm_metrics.rs @@ -14,7 +14,7 @@ use zksync_types::{ }; use zksync_utils::bytecode::bytecode_len_in_bytes; -use crate::api_server::utils::ReportFilter; +use crate::utils::ReportFilter; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "type", rename_all = "snake_case")] @@ -87,7 +87,7 @@ pub(super) enum SandboxStage { #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "stage", rename_all = "snake_case")] -pub(in crate::api_server) enum SubmitTxStage { +pub(crate) enum SubmitTxStage { #[metrics(name = "1_validate")] Validate, #[metrics(name = "2_dry_run")] @@ -102,7 +102,7 @@ pub(in crate::api_server) enum SubmitTxStage { #[must_use = "should be `observe()`d"] #[derive(Debug)] -pub(in crate::api_server) struct SubmitTxLatencyObserver<'a> { +pub(crate) struct SubmitTxLatencyObserver<'a> { inner: Option>, tx_hash: H256, stage: SubmitTxStage, @@ -145,7 +145,7 @@ impl Drop for SubmitTxLatencyObserver<'_> { #[derive(Debug, Metrics)] #[metrics(prefix = "api_web3")] -pub(in crate::api_server) struct SandboxMetrics { +pub(crate) struct SandboxMetrics { #[metrics(buckets = Buckets::LATENCIES)] pub(super) sandbox: Family>, #[metrics(buckets = Buckets::linear(0.0..=2_000.0, 200.0))] @@ -171,8 +171,7 @@ impl SandboxMetrics { } #[vise::register] -pub(in crate::api_server) static SANDBOX_METRICS: vise::Global = - vise::Global::new(); +pub(crate) static SANDBOX_METRICS: vise::Global = vise::Global::new(); #[derive(Debug, Metrics)] #[metrics(prefix = "api_execution")] diff --git a/core/lib/zksync_core/src/api_server/healthcheck.rs b/core/node/api_server/src/healthcheck.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/healthcheck.rs rename to core/node/api_server/src/healthcheck.rs diff --git a/core/lib/zksync_core/src/api_server/mod.rs b/core/node/api_server/src/lib.rs similarity index 85% rename from core/lib/zksync_core/src/api_server/mod.rs rename to core/node/api_server/src/lib.rs index a8c4304ccba2..00a3a5632ceb 100644 --- a/core/lib/zksync_core/src/api_server/mod.rs +++ b/core/node/api_server/src/lib.rs @@ -2,7 +2,6 @@ #[macro_use] mod utils; -pub mod contract_verification; pub mod execution_sandbox; pub mod healthcheck; pub mod tx_sender; diff --git a/core/lib/zksync_core/src/api_server/tx_sender/master_pool_sink.rs b/core/node/api_server/src/tx_sender/master_pool_sink.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/tx_sender/master_pool_sink.rs rename to core/node/api_server/src/tx_sender/master_pool_sink.rs index 9938870c0b7a..b7478b9c9711 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/master_pool_sink.rs +++ b/core/node/api_server/src/tx_sender/master_pool_sink.rs @@ -6,7 +6,7 @@ use zksync_shared_metrics::{TxStage, APP_METRICS}; use zksync_types::{fee::TransactionExecutionMetrics, l2::L2Tx, Address, Nonce, H256}; use super::{tx_sink::TxSink, SubmitTxError}; -use crate::api_server::web3::metrics::API_METRICS; +use crate::web3::metrics::API_METRICS; /// Wrapper for the master DB pool that allows to submit transactions to the mempool. #[derive(Debug)] diff --git a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs b/core/node/api_server/src/tx_sender/mod.rs similarity index 96% rename from core/lib/zksync_core/src/api_server/tx_sender/mod.rs rename to core/node/api_server/src/tx_sender/mod.rs index 78143978fc3c..9e6bd86415f9 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs +++ b/core/node/api_server/src/tx_sender/mod.rs @@ -17,9 +17,12 @@ use zksync_contracts::BaseSystemContracts; use zksync_dal::{ transactions_dal::L2TxSubmissionResult, Connection, ConnectionPool, Core, CoreDal, }; -use zksync_node_fee_model::BatchFeeModelInputProvider; +use zksync_node_fee_model::{ApiFeeInputProvider, BatchFeeModelInputProvider}; use zksync_state::PostgresStorageCaches; -use zksync_state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}; +use zksync_state_keeper::{ + seal_criteria::{ConditionalSealer, NoopSealer, SealData}, + SequencerSealer, +}; use zksync_types::{ fee::{Fee, TransactionExecutionMetrics}, fee_model::BatchFeeInput, @@ -33,11 +36,11 @@ use zksync_types::{ use zksync_utils::h256_to_u256; pub(super) use self::result::SubmitTxError; -use self::tx_sink::TxSink; -use crate::api_server::{ +use self::{master_pool_sink::MasterPoolSink, tx_sink::TxSink}; +use crate::{ execution_sandbox::{ BlockArgs, SubmitTxStage, TransactionExecutor, TxExecutionArgs, TxSharedArgs, - VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, + VmConcurrencyBarrier, VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, }, tx_sender::result::ApiCallResult, }; @@ -49,6 +52,41 @@ mod result; pub(crate) mod tests; pub mod tx_sink; +pub async fn build_tx_sender( + tx_sender_config: &TxSenderConfig, + web3_json_config: &Web3JsonRpcConfig, + state_keeper_config: &StateKeeperConfig, + replica_pool: ConnectionPool, + master_pool: ConnectionPool, + batch_fee_model_input_provider: Arc, + storage_caches: PostgresStorageCaches, +) -> (TxSender, VmConcurrencyBarrier) { + let sequencer_sealer = SequencerSealer::new(state_keeper_config.clone()); + let master_pool_sink = MasterPoolSink::new(master_pool); + let tx_sender_builder = TxSenderBuilder::new( + tx_sender_config.clone(), + replica_pool.clone(), + Arc::new(master_pool_sink), + ) + .with_sealer(Arc::new(sequencer_sealer)); + + let max_concurrency = web3_json_config.vm_concurrency_limit(); + let (vm_concurrency_limiter, vm_barrier) = VmConcurrencyLimiter::new(max_concurrency); + + let batch_fee_input_provider = + ApiFeeInputProvider::new(batch_fee_model_input_provider, replica_pool); + + let tx_sender = tx_sender_builder + .build( + Arc::new(batch_fee_input_provider), + Arc::new(vm_concurrency_limiter), + ApiContracts::load_from_disk(), + storage_caches, + ) + .await; + (tx_sender, vm_barrier) +} + #[derive(Debug, Clone)] pub struct MultiVMBaseSystemContracts { /// Contracts to be used for pre-virtual-blocks protocol versions. diff --git a/core/lib/zksync_core/src/api_server/tx_sender/proxy.rs b/core/node/api_server/src/tx_sender/proxy.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/tx_sender/proxy.rs rename to core/node/api_server/src/tx_sender/proxy.rs diff --git a/core/lib/zksync_core/src/api_server/tx_sender/result.rs b/core/node/api_server/src/tx_sender/result.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/tx_sender/result.rs rename to core/node/api_server/src/tx_sender/result.rs index 18dbf7bbf893..a003b640525d 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/result.rs +++ b/core/node/api_server/src/tx_sender/result.rs @@ -3,7 +3,7 @@ use thiserror::Error; use zksync_types::{l2::error::TxCheckError, U256}; use zksync_web3_decl::error::EnrichedClientError; -use crate::api_server::execution_sandbox::{SandboxExecutionError, ValidationError}; +use crate::execution_sandbox::{SandboxExecutionError, ValidationError}; /// Errors that con occur submitting a transaction or estimating gas for its execution. #[derive(Debug, Error)] diff --git a/core/lib/zksync_core/src/api_server/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs similarity index 83% rename from core/lib/zksync_core/src/api_server/tx_sender/tests.rs rename to core/node/api_server/src/tx_sender/tests.rs index 4c8709f1e404..897808447e7c 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -2,7 +2,6 @@ use assert_matches::assert_matches; use multivm::interface::ExecutionResult; -use zksync_config::configs::wallets::Wallets; use zksync_node_fee_model::MockBatchFeeParamsProvider; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; @@ -10,42 +9,10 @@ use zksync_types::{get_nonce_key, L1BatchNumber, L2BlockNumber, StorageLog}; use zksync_utils::u256_to_h256; use super::*; -use crate::api_server::execution_sandbox::{ - testonly::MockTransactionExecutor, VmConcurrencyBarrier, +use crate::{ + execution_sandbox::testonly::MockTransactionExecutor, web3::testonly::create_test_tx_sender, }; -pub(crate) async fn create_test_tx_sender( - pool: ConnectionPool, - l2_chain_id: L2ChainId, - tx_executor: TransactionExecutor, -) -> (TxSender, VmConcurrencyBarrier) { - let web3_config = Web3JsonRpcConfig::for_tests(); - let state_keeper_config = StateKeeperConfig::for_tests(); - let wallets = Wallets::for_tests(); - let tx_sender_config = TxSenderConfig::new( - &state_keeper_config, - &web3_config, - wallets.state_keeper.unwrap().fee_account.address(), - l2_chain_id, - ); - - let storage_caches = PostgresStorageCaches::new(1, 1); - let batch_fee_model_input_provider = Arc::new(MockBatchFeeParamsProvider::default()); - let (mut tx_sender, vm_barrier) = crate::build_tx_sender( - &tx_sender_config, - &web3_config, - &state_keeper_config, - pool.clone(), - pool, - batch_fee_model_input_provider, - storage_caches, - ) - .await; - - Arc::get_mut(&mut tx_sender.0).unwrap().executor = tx_executor; - (tx_sender, vm_barrier) -} - #[tokio::test] async fn getting_nonce_for_account() { let l2_chain_id = L2ChainId::default(); diff --git a/core/lib/zksync_core/src/api_server/tx_sender/tx_sink.rs b/core/node/api_server/src/tx_sender/tx_sink.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/tx_sender/tx_sink.rs rename to core/node/api_server/src/tx_sender/tx_sink.rs diff --git a/core/lib/zksync_core/src/api_server/utils.rs b/core/node/api_server/src/utils.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/utils.rs rename to core/node/api_server/src/utils.rs diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/metadata.rs b/core/node/api_server/src/web3/backend_jsonrpsee/metadata.rs similarity index 96% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/metadata.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/metadata.rs index a2f8082aeeaa..d5b8d90fdf98 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/metadata.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/metadata.rs @@ -11,7 +11,7 @@ use zksync_web3_decl::{ #[cfg(test)] use super::testonly::RecordedMethodCalls; -use crate::api_server::web3::metrics::{ObservedRpcParams, API_METRICS}; +use crate::web3::metrics::{ObservedRpcParams, API_METRICS}; /// Metadata assigned to a JSON-RPC method call. #[derive(Debug, Clone)] @@ -60,7 +60,7 @@ impl Drop for CurrentMethodGuard<'_> { // is being polled. We use the drop guard pattern to handle corner cases like the handler panicking. // Method handlers are wrapped using RPC-level middleware in `jsonrpsee`. #[derive(Debug, Default)] -pub(crate) struct MethodTracer { +pub struct MethodTracer { inner: ThreadLocal, #[cfg(test)] recorder: RecordedMethodCalls, @@ -113,11 +113,11 @@ impl MethodTracer { #[cfg(test)] impl MethodTracer { /// Copies current method metadata. - pub fn meta(&self) -> Option { + pub(crate) fn meta(&self) -> Option { self.inner.get_or_default().borrow().clone() } - pub fn recorded_calls(&self) -> &RecordedMethodCalls { + pub(crate) fn recorded_calls(&self) -> &RecordedMethodCalls { &self.recorder } } diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/middleware.rs b/core/node/api_server/src/web3/backend_jsonrpsee/middleware.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/middleware.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/middleware.rs index 664dda3f81ab..17d4d3398908 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/middleware.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/middleware.rs @@ -30,7 +30,7 @@ use zksync_web3_decl::jsonrpsee::{ }; use super::metadata::{MethodCall, MethodTracer}; -use crate::api_server::web3::metrics::{ObservedRpcParams, API_METRICS}; +use crate::web3::metrics::{ObservedRpcParams, API_METRICS}; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "transport", rename_all = "snake_case")] diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/mod.rs b/core/node/api_server/src/web3/backend_jsonrpsee/mod.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/mod.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/mod.rs index 060ee68d3bb7..76beb0f7a3a8 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/mod.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/mod.rs @@ -13,7 +13,7 @@ pub(crate) use self::{ CorrelationMiddleware, LimitMiddleware, MetadataLayer, ShutdownMiddleware, TrafficTracker, }, }; -use crate::api_server::tx_sender::SubmitTxError; +use crate::tx_sender::SubmitTxError; mod metadata; mod middleware; diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/debug.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/debug.rs similarity index 97% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/debug.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/debug.rs index a628c3f8c7bb..9c637b88c80e 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/debug.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/debug.rs @@ -9,7 +9,7 @@ use zksync_web3_decl::{ namespaces::debug::DebugNamespaceServer, }; -use crate::api_server::web3::namespaces::DebugNamespace; +use crate::web3::namespaces::DebugNamespace; #[async_trait] impl DebugNamespaceServer for DebugNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/en.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/en.rs similarity index 96% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/en.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/en.rs index f9de0638f47c..4bde945498b3 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/en.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/en.rs @@ -5,7 +5,7 @@ use zksync_web3_decl::{ namespaces::en::EnNamespaceServer, }; -use crate::api_server::web3::namespaces::EnNamespace; +use crate::web3::namespaces::EnNamespace; #[async_trait] impl EnNamespaceServer for EnNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/eth.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/eth.rs index 03abcef7799b..e242d2734ccb 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/eth.rs @@ -13,7 +13,7 @@ use zksync_web3_decl::{ types::{Filter, FilterChanges}, }; -use crate::api_server::web3::EthNamespace; +use crate::web3::EthNamespace; #[async_trait] impl EthNamespaceServer for EthNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/mod.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/mod.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/mod.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/mod.rs diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/net.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/net.rs similarity index 90% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/net.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/net.rs index 8b3743ac483a..449951ddcbae 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/net.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/net.rs @@ -1,7 +1,7 @@ use zksync_types::U256; use zksync_web3_decl::{jsonrpsee::core::RpcResult, namespaces::net::NetNamespaceServer}; -use crate::api_server::web3::NetNamespace; +use crate::web3::NetNamespace; impl NetNamespaceServer for NetNamespace { fn version(&self) -> RpcResult { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/snapshots.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/snapshots.rs similarity index 92% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/snapshots.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/snapshots.rs index f2c0a80a122e..824e09e5d0e7 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/snapshots.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/snapshots.rs @@ -5,7 +5,7 @@ use zksync_types::{ }; use zksync_web3_decl::{jsonrpsee::core::RpcResult, namespaces::SnapshotsNamespaceServer}; -use crate::api_server::web3::namespaces::SnapshotsNamespace; +use crate::web3::namespaces::SnapshotsNamespace; #[async_trait] impl SnapshotsNamespaceServer for SnapshotsNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/web3.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/web3.rs similarity index 84% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/web3.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/web3.rs index 8de5cd3fe2c9..14d1bebfb039 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/web3.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/web3.rs @@ -1,6 +1,6 @@ use zksync_web3_decl::{jsonrpsee::core::RpcResult, namespaces::web3::Web3NamespaceServer}; -use crate::api_server::web3::Web3Namespace; +use crate::web3::Web3Namespace; impl Web3NamespaceServer for Web3Namespace { fn client_version(&self) -> RpcResult { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/zks.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/namespaces/zks.rs index a119422c97fc..2d1047fb0b05 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs +++ b/core/node/api_server/src/web3/backend_jsonrpsee/namespaces/zks.rs @@ -18,7 +18,7 @@ use zksync_web3_decl::{ types::Token, }; -use crate::api_server::web3::ZksNamespace; +use crate::web3::ZksNamespace; #[async_trait] impl ZksNamespaceServer for ZksNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/testonly.rs b/core/node/api_server/src/web3/backend_jsonrpsee/testonly.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/testonly.rs rename to core/node/api_server/src/web3/backend_jsonrpsee/testonly.rs diff --git a/core/lib/zksync_core/src/api_server/web3/mempool_cache.rs b/core/node/api_server/src/web3/mempool_cache.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/mempool_cache.rs rename to core/node/api_server/src/web3/mempool_cache.rs diff --git a/core/lib/zksync_core/src/api_server/web3/metrics.rs b/core/node/api_server/src/web3/metrics.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/web3/metrics.rs rename to core/node/api_server/src/web3/metrics.rs index f2ec4a683c97..a8d6c0d5851a 100644 --- a/core/lib/zksync_core/src/api_server/web3/metrics.rs +++ b/core/node/api_server/src/web3/metrics.rs @@ -13,7 +13,7 @@ use super::{ backend_jsonrpsee::MethodMetadata, ApiTransport, InternalApiConfig, OptionalApiParams, TypedFilter, }; -use crate::api_server::utils::ReportFilter; +use crate::utils::ReportFilter; /// Observed version of RPC parameters. Have a bounded upper-limit size (256 bytes), so that we don't over-allocate. #[derive(Debug)] @@ -81,7 +81,7 @@ impl<'a> ObservedRpcParams<'a> { #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "scheme", rename_all = "UPPERCASE")] -pub(in crate::api_server) enum ApiTransportLabel { +pub(crate) enum ApiTransportLabel { Http, Ws, } @@ -235,7 +235,7 @@ const RESPONSE_SIZE_BUCKETS: Buckets = Buckets::exponential(1.0..=1_048_576.0, 4 /// General-purpose API server metrics. #[derive(Debug, Metrics)] #[metrics(prefix = "api")] -pub(in crate::api_server) struct ApiMetrics { +pub(crate) struct ApiMetrics { /// Web3 server configuration. web3_info: Family>, @@ -411,11 +411,11 @@ impl ApiMetrics { } #[vise::register] -pub(in crate::api_server) static API_METRICS: vise::Global = vise::Global::new(); +pub(crate) static API_METRICS: vise::Global = vise::Global::new(); #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] #[metrics(label = "subscription_type", rename_all = "snake_case")] -pub(super) enum SubscriptionType { +pub enum SubscriptionType { Blocks, Txs, Logs, diff --git a/core/lib/zksync_core/src/api_server/web3/mod.rs b/core/node/api_server/src/web3/mod.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/mod.rs rename to core/node/api_server/src/web3/mod.rs index 54528286c3d0..0f9126344791 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/node/api_server/src/web3/mod.rs @@ -13,6 +13,7 @@ use zksync_config::configs::api::{MaxResponseSize, MaxResponseSizeOverrides}; use zksync_dal::{helpers::wait_for_l1_batch, ConnectionPool, Core}; use zksync_health_check::{HealthStatus, HealthUpdater, ReactiveHealthCheck}; use zksync_metadata_calculator::api_server::TreeApiClient; +use zksync_node_sync::SyncState; use zksync_types::L2BlockNumber; use zksync_web3_decl::{ jsonrpsee::{ @@ -42,11 +43,10 @@ use self::{ pubsub::{EthSubscribe, EthSubscriptionIdProvider, PubSubEvent}, state::{Filters, InternalApiConfig, RpcState, SealedL2BlockNumber}, }; -use crate::api_server::{ +use crate::{ execution_sandbox::{BlockStartInfo, VmConcurrencyBarrier}, tx_sender::TxSender, }; -use zksync_node_sync::SyncState; pub mod backend_jsonrpsee; pub mod mempool_cache; @@ -54,6 +54,7 @@ pub(super) mod metrics; pub mod namespaces; mod pubsub; pub mod state; +pub mod testonly; #[cfg(test)] pub(crate) mod tests; @@ -281,13 +282,15 @@ impl ApiBuilder { self } - #[cfg(test)] + // Intended for tests only. + #[doc(hidden)] fn with_pub_sub_events(mut self, sender: mpsc::UnboundedSender) -> Self { self.optional.pub_sub_events_sender = Some(sender); self } - #[cfg(test)] + // Intended for tests only. + #[doc(hidden)] fn with_method_tracer(mut self, method_tracer: Arc) -> Self { self.method_tracer = method_tracer; self diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs b/core/node/api_server/src/web3/namespaces/debug.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs rename to core/node/api_server/src/web3/namespaces/debug.rs index aee048a16890..4b998adcfeb8 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs +++ b/core/node/api_server/src/web3/namespaces/debug.rs @@ -16,7 +16,7 @@ use zksync_types::{ }; use zksync_web3_decl::error::Web3Error; -use crate::api_server::{ +use crate::{ execution_sandbox::{ApiTracer, TxSharedArgs}, tx_sender::{ApiContracts, TxSenderConfig}, web3::{backend_jsonrpsee::MethodTracer, state::RpcState}, diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/en.rs b/core/node/api_server/src/web3/namespaces/en.rs similarity index 98% rename from core/lib/zksync_core/src/api_server/web3/namespaces/en.rs rename to core/node/api_server/src/web3/namespaces/en.rs index 621b62a53ec1..321e407e14b3 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/en.rs +++ b/core/node/api_server/src/web3/namespaces/en.rs @@ -4,7 +4,7 @@ use zksync_dal::{CoreDal, DalError}; use zksync_types::{api::en, tokens::TokenInfo, Address, L1BatchNumber, L2BlockNumber, H256}; use zksync_web3_decl::error::Web3Error; -use crate::api_server::web3::{backend_jsonrpsee::MethodTracer, state::RpcState}; +use crate::web3::{backend_jsonrpsee::MethodTracer, state::RpcState}; /// Namespace for External Node unique methods. /// Main use case for it is the EN synchronization. diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs b/core/node/api_server/src/web3/namespaces/eth.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs rename to core/node/api_server/src/web3/namespaces/eth.rs index 45e938cf2163..10c5cb22d697 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs +++ b/core/node/api_server/src/web3/namespaces/eth.rs @@ -18,7 +18,7 @@ use zksync_web3_decl::{ types::{Address, Block, Filter, FilterChanges, Log, U64}, }; -use crate::api_server::web3::{ +use crate::web3::{ backend_jsonrpsee::MethodTracer, metrics::API_METRICS, state::RpcState, TypedFilter, }; diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/mod.rs b/core/node/api_server/src/web3/namespaces/mod.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/namespaces/mod.rs rename to core/node/api_server/src/web3/namespaces/mod.rs diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/net.rs b/core/node/api_server/src/web3/namespaces/net.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/namespaces/net.rs rename to core/node/api_server/src/web3/namespaces/net.rs diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs b/core/node/api_server/src/web3/namespaces/snapshots.rs similarity index 97% rename from core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs rename to core/node/api_server/src/web3/namespaces/snapshots.rs index 9397d24479df..0586940b0eb0 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs +++ b/core/node/api_server/src/web3/namespaces/snapshots.rs @@ -6,7 +6,7 @@ use zksync_types::{ }; use zksync_web3_decl::error::Web3Error; -use crate::api_server::web3::{backend_jsonrpsee::MethodTracer, state::RpcState}; +use crate::web3::{backend_jsonrpsee::MethodTracer, state::RpcState}; #[derive(Debug, Clone)] pub(crate) struct SnapshotsNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/web3.rs b/core/node/api_server/src/web3/namespaces/web3.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/namespaces/web3.rs rename to core/node/api_server/src/web3/namespaces/web3.rs diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs b/core/node/api_server/src/web3/namespaces/zks.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs rename to core/node/api_server/src/web3/namespaces/zks.rs index fca41af1e62e..0cdbcf603299 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/zks.rs +++ b/core/node/api_server/src/web3/namespaces/zks.rs @@ -29,7 +29,7 @@ use zksync_web3_decl::{ types::{Address, Token, H256}, }; -use crate::api_server::web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, RpcState}; +use crate::web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, RpcState}; #[derive(Debug)] pub(crate) struct ZksNamespace { diff --git a/core/lib/zksync_core/src/api_server/web3/pubsub.rs b/core/node/api_server/src/web3/pubsub.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/pubsub.rs rename to core/node/api_server/src/web3/pubsub.rs index 0ce5698ecea6..f3fa564c3e44 100644 --- a/core/lib/zksync_core/src/api_server/web3/pubsub.rs +++ b/core/node/api_server/src/web3/pubsub.rs @@ -41,7 +41,7 @@ impl IdProvider for EthSubscriptionIdProvider { /// Events emitted by the subscription logic. Only used in WebSocket server tests so far. #[derive(Debug)] -pub(super) enum PubSubEvent { +pub enum PubSubEvent { Subscribed(SubscriptionType), NotifyIterationFinished(SubscriptionType), L2BlockAdvanced(SubscriptionType, L2BlockNumber), diff --git a/core/lib/zksync_core/src/api_server/web3/state.rs b/core/node/api_server/src/web3/state.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/state.rs rename to core/node/api_server/src/web3/state.rs index 35b1cdc78c46..276e0b6755e7 100644 --- a/core/lib/zksync_core/src/api_server/web3/state.rs +++ b/core/node/api_server/src/web3/state.rs @@ -31,7 +31,7 @@ use super::{ metrics::{FilterType, FILTER_METRICS}, TypedFilter, }; -use crate::api_server::{ +use crate::{ execution_sandbox::{BlockArgs, BlockArgsError, BlockStartInfo}, tx_sender::{tx_sink::TxSink, TxSender}, }; diff --git a/core/node/api_server/src/web3/testonly.rs b/core/node/api_server/src/web3/testonly.rs new file mode 100644 index 000000000000..566db4d73f3e --- /dev/null +++ b/core/node/api_server/src/web3/testonly.rs @@ -0,0 +1,178 @@ +//! Test utilities useful for writing unit tests outside of this crate. + +use std::{pin::Pin, time::Instant}; + +use tokio::sync::watch; +use zksync_config::configs::{api::Web3JsonRpcConfig, chain::StateKeeperConfig, wallets::Wallets}; +use zksync_dal::ConnectionPool; +use zksync_health_check::CheckHealth; +use zksync_node_fee_model::MockBatchFeeParamsProvider; +use zksync_state::PostgresStorageCaches; +use zksync_types::L2ChainId; + +use super::{metrics::ApiTransportLabel, *}; +use crate::{ + execution_sandbox::{testonly::MockTransactionExecutor, TransactionExecutor}, + tx_sender::TxSenderConfig, +}; + +const TEST_TIMEOUT: Duration = Duration::from_secs(90); +const POLL_INTERVAL: Duration = Duration::from_millis(50); + +pub(crate) async fn create_test_tx_sender( + pool: ConnectionPool, + l2_chain_id: L2ChainId, + tx_executor: TransactionExecutor, +) -> (TxSender, VmConcurrencyBarrier) { + let web3_config = Web3JsonRpcConfig::for_tests(); + let state_keeper_config = StateKeeperConfig::for_tests(); + let wallets = Wallets::for_tests(); + let tx_sender_config = TxSenderConfig::new( + &state_keeper_config, + &web3_config, + wallets.state_keeper.unwrap().fee_account.address(), + l2_chain_id, + ); + + let storage_caches = PostgresStorageCaches::new(1, 1); + let batch_fee_model_input_provider = Arc::new(MockBatchFeeParamsProvider::default()); + let (mut tx_sender, vm_barrier) = crate::tx_sender::build_tx_sender( + &tx_sender_config, + &web3_config, + &state_keeper_config, + pool.clone(), + pool, + batch_fee_model_input_provider, + storage_caches, + ) + .await; + + Arc::get_mut(&mut tx_sender.0).unwrap().executor = tx_executor; + (tx_sender, vm_barrier) +} + +impl ApiServerHandles { + /// Waits until the server health check reports the ready state. Must be called once per server instance. + pub async fn wait_until_ready(&mut self) -> SocketAddr { + let started_at = Instant::now(); + loop { + assert!( + started_at.elapsed() <= TEST_TIMEOUT, + "Timed out waiting for API server" + ); + let health = self.health_check.check_health().await; + if health.status().is_healthy() { + break; + } + tokio::time::sleep(POLL_INTERVAL).await; + } + + let mut local_addr_future = Pin::new(&mut self.local_addr); + local_addr_future + .as_mut() + .await + .expect("API server panicked"); + local_addr_future.output_mut().copied().unwrap() + } + + pub async fn shutdown(self) { + let stop_server = async { + for task in self.tasks { + match task.await { + Ok(Ok(())) => { /* Task successfully completed */ } + Err(err) if err.is_cancelled() => { + // Task was canceled since the server runtime which runs the task was dropped. + // This is fine. + } + Err(err) => panic!("Server task panicked: {err:?}"), + Ok(Err(err)) => panic!("Server task failed: {err:?}"), + } + } + }; + tokio::time::timeout(TEST_TIMEOUT, stop_server) + .await + .unwrap_or_else(|_| panic!("panicking at {}", chrono::Utc::now())); + } +} + +pub async fn spawn_http_server( + api_config: InternalApiConfig, + pool: ConnectionPool, + tx_executor: MockTransactionExecutor, + method_tracer: Arc, + stop_receiver: watch::Receiver, +) -> ApiServerHandles { + spawn_server( + ApiTransportLabel::Http, + api_config, + pool, + None, + tx_executor, + method_tracer, + stop_receiver, + ) + .await + .0 +} + +pub async fn spawn_ws_server( + api_config: InternalApiConfig, + pool: ConnectionPool, + stop_receiver: watch::Receiver, + websocket_requests_per_minute_limit: Option, +) -> (ApiServerHandles, mpsc::UnboundedReceiver) { + spawn_server( + ApiTransportLabel::Ws, + api_config, + pool, + websocket_requests_per_minute_limit, + MockTransactionExecutor::default(), + Arc::default(), + stop_receiver, + ) + .await +} + +async fn spawn_server( + transport: ApiTransportLabel, + api_config: InternalApiConfig, + pool: ConnectionPool, + websocket_requests_per_minute_limit: Option, + tx_executor: MockTransactionExecutor, + method_tracer: Arc, + stop_receiver: watch::Receiver, +) -> (ApiServerHandles, mpsc::UnboundedReceiver) { + let (tx_sender, vm_barrier) = + create_test_tx_sender(pool.clone(), api_config.l2_chain_id, tx_executor.into()).await; + let (pub_sub_events_sender, pub_sub_events_receiver) = mpsc::unbounded_channel(); + + let mut namespaces = Namespace::DEFAULT.to_vec(); + namespaces.extend([Namespace::Debug, Namespace::Snapshots]); + + let server_builder = match transport { + ApiTransportLabel::Http => ApiBuilder::jsonrpsee_backend(api_config, pool).http(0), + ApiTransportLabel::Ws => { + let mut builder = ApiBuilder::jsonrpsee_backend(api_config, pool) + .ws(0) + .with_subscriptions_limit(100); + if let Some(websocket_requests_per_minute_limit) = websocket_requests_per_minute_limit { + builder = builder + .with_websocket_requests_per_minute_limit(websocket_requests_per_minute_limit); + } + builder + } + }; + let server_handles = server_builder + .with_polling_interval(POLL_INTERVAL) + .with_tx_sender(tx_sender) + .with_vm_barrier(vm_barrier) + .with_pub_sub_events(pub_sub_events_sender) + .with_method_tracer(method_tracer) + .enable_api_namespaces(namespaces) + .build() + .expect("Unable to build API server") + .run(stop_receiver) + .await + .expect("Failed spawning JSON-RPC server"); + (server_handles, pub_sub_events_receiver) +} diff --git a/core/lib/zksync_core/src/api_server/web3/tests/debug.rs b/core/node/api_server/src/web3/tests/debug.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/tests/debug.rs rename to core/node/api_server/src/web3/tests/debug.rs diff --git a/core/lib/zksync_core/src/api_server/web3/tests/filters.rs b/core/node/api_server/src/web3/tests/filters.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/tests/filters.rs rename to core/node/api_server/src/web3/tests/filters.rs diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/node/api_server/src/web3/tests/mod.rs similarity index 88% rename from core/lib/zksync_core/src/api_server/web3/tests/mod.rs rename to core/node/api_server/src/web3/tests/mod.rs index a0088c6cb3ef..f24522e75364 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/node/api_server/src/web3/tests/mod.rs @@ -2,9 +2,7 @@ use std::{ collections::{HashMap, HashSet}, net::Ipv4Addr, num::NonZeroUsize, - pin::Pin, slice, - time::Instant, }; use assert_matches::assert_matches; @@ -25,7 +23,6 @@ use zksync_config::{ GenesisConfig, }; use zksync_dal::{transactions_dal::L2TxSubmissionResult, Connection, ConnectionPool, CoreDal}; -use zksync_health_check::CheckHealth; use zksync_node_genesis::{insert_genesis_batch, mock_genesis_config, GenesisParams}; use zksync_node_test_utils::{ create_l1_batch, create_l1_batch_metadata, create_l2_block, create_l2_transaction, @@ -53,9 +50,10 @@ use zksync_web3_decl::{ namespaces::{EnNamespaceClient, EthNamespaceClient, ZksNamespaceClient}, }; -use super::{metrics::ApiTransportLabel, *}; -use crate::api_server::{ - execution_sandbox::testonly::MockTransactionExecutor, tx_sender::tests::create_test_tx_sender, +use super::*; +use crate::{ + execution_sandbox::testonly::MockTransactionExecutor, + web3::testonly::{spawn_http_server, spawn_ws_server}, }; mod debug; @@ -127,132 +125,6 @@ async fn setting_response_size_limits() { server_handle.stop().ok(); } -impl ApiServerHandles { - /// Waits until the server health check reports the ready state. Must be called once per server instance. - pub(crate) async fn wait_until_ready(&mut self) -> SocketAddr { - let started_at = Instant::now(); - loop { - assert!( - started_at.elapsed() <= TEST_TIMEOUT, - "Timed out waiting for API server" - ); - let health = self.health_check.check_health().await; - if health.status().is_healthy() { - break; - } - tokio::time::sleep(POLL_INTERVAL).await; - } - - let mut local_addr_future = Pin::new(&mut self.local_addr); - local_addr_future - .as_mut() - .await - .expect("API server panicked"); - local_addr_future.output_mut().copied().unwrap() - } - - pub(crate) async fn shutdown(self) { - let stop_server = async { - for task in self.tasks { - match task.await { - Ok(Ok(())) => { /* Task successfully completed */ } - Err(err) if err.is_cancelled() => { - // Task was canceled since the server runtime which runs the task was dropped. - // This is fine. - } - Err(err) => panic!("Server task panicked: {err:?}"), - Ok(Err(err)) => panic!("Server task failed: {err:?}"), - } - } - }; - tokio::time::timeout(TEST_TIMEOUT, stop_server) - .await - .unwrap_or_else(|_| panic!("panicking at {}", chrono::Utc::now())); - } -} - -pub(crate) async fn spawn_http_server( - api_config: InternalApiConfig, - pool: ConnectionPool, - tx_executor: MockTransactionExecutor, - method_tracer: Arc, - stop_receiver: watch::Receiver, -) -> ApiServerHandles { - spawn_server( - ApiTransportLabel::Http, - api_config, - pool, - None, - tx_executor, - method_tracer, - stop_receiver, - ) - .await - .0 -} - -async fn spawn_ws_server( - api_config: InternalApiConfig, - pool: ConnectionPool, - stop_receiver: watch::Receiver, - websocket_requests_per_minute_limit: Option, -) -> (ApiServerHandles, mpsc::UnboundedReceiver) { - spawn_server( - ApiTransportLabel::Ws, - api_config, - pool, - websocket_requests_per_minute_limit, - MockTransactionExecutor::default(), - Arc::default(), - stop_receiver, - ) - .await -} - -async fn spawn_server( - transport: ApiTransportLabel, - api_config: InternalApiConfig, - pool: ConnectionPool, - websocket_requests_per_minute_limit: Option, - tx_executor: MockTransactionExecutor, - method_tracer: Arc, - stop_receiver: watch::Receiver, -) -> (ApiServerHandles, mpsc::UnboundedReceiver) { - let (tx_sender, vm_barrier) = - create_test_tx_sender(pool.clone(), api_config.l2_chain_id, tx_executor.into()).await; - let (pub_sub_events_sender, pub_sub_events_receiver) = mpsc::unbounded_channel(); - - let mut namespaces = Namespace::DEFAULT.to_vec(); - namespaces.extend([Namespace::Debug, Namespace::Snapshots]); - - let server_builder = match transport { - ApiTransportLabel::Http => ApiBuilder::jsonrpsee_backend(api_config, pool).http(0), - ApiTransportLabel::Ws => { - let mut builder = ApiBuilder::jsonrpsee_backend(api_config, pool) - .ws(0) - .with_subscriptions_limit(100); - if let Some(websocket_requests_per_minute_limit) = websocket_requests_per_minute_limit { - builder = builder - .with_websocket_requests_per_minute_limit(websocket_requests_per_minute_limit); - } - builder - } - }; - let server_handles = server_builder - .with_polling_interval(POLL_INTERVAL) - .with_tx_sender(tx_sender) - .with_vm_barrier(vm_barrier) - .with_pub_sub_events(pub_sub_events_sender) - .with_method_tracer(method_tracer) - .enable_api_namespaces(namespaces) - .build() - .expect("Unable to build API server") - .run(stop_receiver) - .await - .expect("Failed spawning JSON-RPC server"); - (server_handles, pub_sub_events_receiver) -} - #[async_trait] trait HttpTest: Send + Sync { /// Prepares the storage before the server is started. The default implementation performs genesis. diff --git a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs b/core/node/api_server/src/web3/tests/snapshots.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs rename to core/node/api_server/src/web3/tests/snapshots.rs diff --git a/core/lib/zksync_core/src/api_server/web3/tests/vm.rs b/core/node/api_server/src/web3/tests/vm.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/web3/tests/vm.rs rename to core/node/api_server/src/web3/tests/vm.rs diff --git a/core/lib/zksync_core/src/api_server/web3/tests/ws.rs b/core/node/api_server/src/web3/tests/ws.rs similarity index 99% rename from core/lib/zksync_core/src/api_server/web3/tests/ws.rs rename to core/node/api_server/src/web3/tests/ws.rs index 97190a72e96a..ff3fc465811b 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/ws.rs +++ b/core/node/api_server/src/web3/tests/ws.rs @@ -20,7 +20,7 @@ use zksync_web3_decl::{ }; use super::*; -use crate::api_server::web3::metrics::SubscriptionType; +use crate::web3::metrics::SubscriptionType; #[allow(clippy::needless_pass_by_ref_mut)] // false positive async fn wait_for_subscription( diff --git a/core/node/consensus/Cargo.toml b/core/node/consensus/Cargo.toml new file mode 100644 index 000000000000..9cfb3c86b0be --- /dev/null +++ b/core/node/consensus/Cargo.toml @@ -0,0 +1,41 @@ +[package] +name = "zksync_node_consensus" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_config.workspace = true +zksync_concurrency.workspace = true +zksync_consensus_crypto.workspace = true +zksync_consensus_network.workspace = true +zksync_consensus_roles.workspace = true +zksync_consensus_storage.workspace = true +zksync_consensus_executor.workspace = true +zksync_consensus_bft.workspace = true +zksync_consensus_utils.workspace = true +zksync_protobuf.workspace = true +zksync_dal.workspace = true +zksync_state_keeper.workspace = true +zksync_node_sync.workspace = true +zksync_types.workspace = true +zksync_web3_decl.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +secrecy.workspace = true +tracing.workspace = true + +[dev-dependencies] +zksync_node_genesis.workspace = true +zksync_node_test_utils.workspace = true +zksync_node_api_server.workspace = true + +tokio.workspace = true +test-casing.workspace = true +rand.workspace = true diff --git a/core/node/consensus/README.md b/core/node/consensus/README.md new file mode 100644 index 000000000000..191104555ad8 --- /dev/null +++ b/core/node/consensus/README.md @@ -0,0 +1,3 @@ +# `zksync_node_consensus` + +Set of components that add the support of consensus to the ZK Stack. diff --git a/core/lib/zksync_core/src/consensus/config.rs b/core/node/consensus/src/config.rs similarity index 100% rename from core/lib/zksync_core/src/consensus/config.rs rename to core/node/consensus/src/config.rs diff --git a/core/lib/zksync_core/src/consensus/en.rs b/core/node/consensus/src/en.rs similarity index 99% rename from core/lib/zksync_core/src/consensus/en.rs rename to core/node/consensus/src/en.rs index 48775bd4288e..685bc982bd07 100644 --- a/core/lib/zksync_core/src/consensus/en.rs +++ b/core/node/consensus/src/en.rs @@ -10,7 +10,7 @@ use zksync_types::L2BlockNumber; use zksync_web3_decl::client::{DynClient, L2}; use super::{config, storage::Store, ConnectionPool, ConsensusConfig, ConsensusSecrets}; -use crate::consensus::storage; +use crate::storage; /// External node. pub(super) struct EN { diff --git a/core/lib/zksync_core/src/consensus/era.rs b/core/node/consensus/src/era.rs similarity index 100% rename from core/lib/zksync_core/src/consensus/era.rs rename to core/node/consensus/src/era.rs index 3b532c88ba7c..5cf537f65300 100644 --- a/core/lib/zksync_core/src/consensus/era.rs +++ b/core/node/consensus/src/era.rs @@ -7,10 +7,10 @@ use zksync_concurrency::ctx; use zksync_config::configs::consensus::{ConsensusConfig, ConsensusSecrets}; use zksync_dal::Core; +use zksync_node_sync::{sync_action::ActionQueueSender, SyncState}; use zksync_web3_decl::client::{DynClient, L2}; use super::{en, storage::ConnectionPool}; -use zksync_node_sync::{sync_action::ActionQueueSender, SyncState}; /// Runs the consensus task in the main node mode. pub async fn run_main_node( diff --git a/core/lib/zksync_core/src/consensus/mod.rs b/core/node/consensus/src/lib.rs similarity index 98% rename from core/lib/zksync_core/src/consensus/mod.rs rename to core/node/consensus/src/lib.rs index 35bee505ec46..b076b26e2749 100644 --- a/core/lib/zksync_core/src/consensus/mod.rs +++ b/core/node/consensus/src/lib.rs @@ -3,13 +3,14 @@ #![allow(clippy::redundant_locals)] #![allow(clippy::needless_pass_by_ref_mut)] use anyhow::Context as _; -use storage::{ConnectionPool, Store}; use zksync_concurrency::{ctx, error::Wrap as _, scope}; use zksync_config::configs::consensus::{ConsensusConfig, ConsensusSecrets}; use zksync_consensus_executor as executor; use zksync_consensus_roles::validator; use zksync_consensus_storage::BlockStore; +use crate::storage::{ConnectionPool, Store}; + mod config; mod en; pub mod era; diff --git a/core/lib/zksync_core/src/consensus/storage/mod.rs b/core/node/consensus/src/storage/mod.rs similarity index 100% rename from core/lib/zksync_core/src/consensus/storage/mod.rs rename to core/node/consensus/src/storage/mod.rs index 6a9fb6930121..658c7a887d5f 100644 --- a/core/lib/zksync_core/src/consensus/storage/mod.rs +++ b/core/node/consensus/src/storage/mod.rs @@ -7,15 +7,15 @@ use zksync_consensus_bft::PayloadManager; use zksync_consensus_roles::validator; use zksync_consensus_storage as storage; use zksync_dal::{consensus_dal::Payload, Core, CoreDal, DalError}; -use zksync_state_keeper::io::common::IoCursor; -use zksync_types::L2BlockNumber; - -use super::config; use zksync_node_sync::{ fetcher::{FetchedBlock, FetchedTransaction, IoCursorExt as _}, sync_action::ActionQueueSender, SyncState, }; +use zksync_state_keeper::io::common::IoCursor; +use zksync_types::L2BlockNumber; + +use super::config; #[cfg(test)] mod testonly; diff --git a/core/lib/zksync_core/src/consensus/storage/testonly.rs b/core/node/consensus/src/storage/testonly.rs similarity index 100% rename from core/lib/zksync_core/src/consensus/storage/testonly.rs rename to core/node/consensus/src/storage/testonly.rs diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/node/consensus/src/testonly.rs similarity index 99% rename from core/lib/zksync_core/src/consensus/testonly.rs rename to core/node/consensus/src/testonly.rs index 819cbc626cf5..ff1af127c038 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/node/consensus/src/testonly.rs @@ -10,6 +10,7 @@ use zksync_consensus_crypto::TextFmt as _; use zksync_consensus_network as network; use zksync_consensus_roles::validator; use zksync_dal::{CoreDal, DalError}; +use zksync_node_api_server::web3::{state::InternalApiConfig, testonly::spawn_http_server}; use zksync_node_genesis::GenesisParams; use zksync_node_sync::{ fetcher::{FetchedTransaction, IoCursorExt as _}, @@ -27,10 +28,7 @@ use zksync_state_keeper::{ use zksync_types::{Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId}; use zksync_web3_decl::client::{Client, DynClient, L2}; -use crate::{ - api_server::web3::{state::InternalApiConfig, tests::spawn_http_server}, - consensus::{en, ConnectionPool}, -}; +use crate::{en, ConnectionPool}; /// Fake StateKeeper for tests. pub(super) struct StateKeeper { diff --git a/core/lib/zksync_core/src/consensus/tests.rs b/core/node/consensus/src/tests.rs similarity index 100% rename from core/lib/zksync_core/src/consensus/tests.rs rename to core/node/consensus/src/tests.rs diff --git a/core/node/contract_verification_server/Cargo.toml b/core/node/contract_verification_server/Cargo.toml new file mode 100644 index 000000000000..ee38d30906fb --- /dev/null +++ b/core/node/contract_verification_server/Cargo.toml @@ -0,0 +1,24 @@ +[package] +name = "zksync_contract_verification_server" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_config.workspace = true +zksync_dal.workspace = true +zksync_types.workspace = true +vise.workspace = true + +anyhow.workspace = true +axum.workspace = true +tokio = { workspace = true, features = ["time"] } +tower-http.workspace = true +tracing.workspace = true +serde.workspace = true +serde_json.workspace = true diff --git a/core/node/contract_verification_server/README.md b/core/node/contract_verification_server/README.md new file mode 100644 index 000000000000..ab18bbcde432 --- /dev/null +++ b/core/node/contract_verification_server/README.md @@ -0,0 +1,3 @@ +# `zksync_contract_verification_server` + +Implementation of the backend used for contract verification. diff --git a/core/lib/zksync_core/src/api_server/contract_verification/api_decl.rs b/core/node/contract_verification_server/src/api_decl.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/contract_verification/api_decl.rs rename to core/node/contract_verification_server/src/api_decl.rs diff --git a/core/lib/zksync_core/src/api_server/contract_verification/api_impl.rs b/core/node/contract_verification_server/src/api_impl.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/contract_verification/api_impl.rs rename to core/node/contract_verification_server/src/api_impl.rs diff --git a/core/lib/zksync_core/src/api_server/contract_verification/mod.rs b/core/node/contract_verification_server/src/lib.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/contract_verification/mod.rs rename to core/node/contract_verification_server/src/lib.rs diff --git a/core/lib/zksync_core/src/api_server/contract_verification/metrics.rs b/core/node/contract_verification_server/src/metrics.rs similarity index 100% rename from core/lib/zksync_core/src/api_server/contract_verification/metrics.rs rename to core/node/contract_verification_server/src/metrics.rs diff --git a/core/node/metadata_calculator/README.md b/core/node/metadata_calculator/README.md index 35880000b98c..15e7a6b92b9c 100644 --- a/core/node/metadata_calculator/README.md +++ b/core/node/metadata_calculator/README.md @@ -1,6 +1,6 @@ # `zksync_metadata_calculator` -Component responsible for calculating any supplimentary metadata required for L1 batches. -Most notably, this is component responsible for maintaining the Merkle Tree. +Component responsible for calculating any supplimentary metadata required for L1 batches. Most notably, this is +component responsible for maintaining the Merkle Tree. Additionally, this crate provides ability to spawn the Merkle Tree API server. diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index 70014773bd85..4579655e3a61 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -38,6 +38,9 @@ zksync_state_keeper.workspace = true zksync_consistency_checker.workspace = true zksync_metadata_calculator.workspace = true zksync_node_sync.workspace = true +zksync_node_api_server.workspace = true +zksync_node_consensus.workspace = true +zksync_contract_verification_server.workspace = true tracing.workspace = true thiserror.workspace = true diff --git a/core/node/node_framework/examples/main_node.rs b/core/node/node_framework/examples/main_node.rs index cdf2c3a7e848..e3d42e1828bc 100644 --- a/core/node/node_framework/examples/main_node.rs +++ b/core/node/node_framework/examples/main_node.rs @@ -19,15 +19,13 @@ use zksync_config::{ ApiConfig, ContractVerifierConfig, ContractsConfig, DBConfig, EthConfig, EthWatchConfig, GasAdjusterConfig, GenesisConfig, ObjectStoreConfig, PostgresConfig, }; -use zksync_core::{ - api_server::{ - tx_sender::{ApiContracts, TxSenderConfig}, - web3::{state::InternalApiConfig, Namespace}, - }, - temp_config_store::decode_yaml_repr, -}; +use zksync_core::temp_config_store::decode_yaml_repr; use zksync_env_config::FromEnv; use zksync_metadata_calculator::MetadataCalculatorConfig; +use zksync_node_api_server::{ + tx_sender::{ApiContracts, TxSenderConfig}, + web3::{state::InternalApiConfig, Namespace}, +}; use zksync_node_framework::{ implementations::layers::{ circuit_breaker_checker::CircuitBreakerCheckerLayer, diff --git a/core/node/node_framework/src/implementations/layers/consensus.rs b/core/node/node_framework/src/implementations/layers/consensus.rs index e3dc747b0c2b..5a91e796eb5f 100644 --- a/core/node/node_framework/src/implementations/layers/consensus.rs +++ b/core/node/node_framework/src/implementations/layers/consensus.rs @@ -1,8 +1,8 @@ use anyhow::Context as _; use zksync_concurrency::{ctx, scope}; use zksync_config::configs::consensus::{ConsensusConfig, ConsensusSecrets}; -use zksync_core::consensus; use zksync_dal::{ConnectionPool, Core}; +use zksync_node_consensus as consensus; use zksync_node_sync::{ActionQueueSender, SyncState}; use zksync_web3_decl::client::{DynClient, L2}; @@ -160,7 +160,7 @@ impl Task for FetcherTask { // but we only need to wait for stop signal once, and it will be propagated to all child contexts. let root_ctx = ctx::root(); scope::run!(&root_ctx, |ctx, s| async { - s.spawn_bg(zksync_core::consensus::era::run_en( + s.spawn_bg(consensus::era::run_en( &root_ctx, self.config, self.pool, diff --git a/core/node/node_framework/src/implementations/layers/contract_verification_api.rs b/core/node/node_framework/src/implementations/layers/contract_verification_api.rs index 879cb7d616a5..2e0dcf540ea6 100644 --- a/core/node/node_framework/src/implementations/layers/contract_verification_api.rs +++ b/core/node/node_framework/src/implementations/layers/contract_verification_api.rs @@ -51,7 +51,7 @@ impl Task for ContractVerificationApiTask { } async fn run(self: Box, stop_receiver: StopReceiver) -> anyhow::Result<()> { - zksync_core::api_server::contract_verification::start_server( + zksync_contract_verification_server::start_server( self.master_pool, self.replica_pool, self.config, diff --git a/core/node/node_framework/src/implementations/layers/healtcheck_server.rs b/core/node/node_framework/src/implementations/layers/healtcheck_server.rs index 0c6a9b38dfa6..34c41fd70a97 100644 --- a/core/node/node_framework/src/implementations/layers/healtcheck_server.rs +++ b/core/node/node_framework/src/implementations/layers/healtcheck_server.rs @@ -1,8 +1,8 @@ use std::sync::Arc; use zksync_config::configs::api::HealthCheckConfig; -use zksync_core::api_server::healthcheck::HealthCheckHandle; use zksync_health_check::AppHealthCheck; +use zksync_node_api_server::healthcheck::HealthCheckHandle; use crate::{ implementations::resources::healthcheck::AppHealthCheckResource, diff --git a/core/node/node_framework/src/implementations/layers/web3_api/caches.rs b/core/node/node_framework/src/implementations/layers/web3_api/caches.rs index 812d898c2005..7c6d160c3339 100644 --- a/core/node/node_framework/src/implementations/layers/web3_api/caches.rs +++ b/core/node/node_framework/src/implementations/layers/web3_api/caches.rs @@ -1,6 +1,6 @@ use std::time::Duration; -use zksync_core::api_server::web3::mempool_cache::{self, MempoolCache}; +use zksync_node_api_server::web3::mempool_cache::{self, MempoolCache}; use crate::{ implementations::resources::{ diff --git a/core/node/node_framework/src/implementations/layers/web3_api/server.rs b/core/node/node_framework/src/implementations/layers/web3_api/server.rs index 6c29dca5837a..08eaa4b80444 100644 --- a/core/node/node_framework/src/implementations/layers/web3_api/server.rs +++ b/core/node/node_framework/src/implementations/layers/web3_api/server.rs @@ -3,7 +3,7 @@ use std::{num::NonZeroU32, time::Duration}; use tokio::{sync::oneshot, task::JoinHandle}; use zksync_circuit_breaker::replication_lag::ReplicationLagChecker; use zksync_config::configs::api::MaxResponseSize; -use zksync_core::api_server::web3::{state::InternalApiConfig, ApiBuilder, ApiServer, Namespace}; +use zksync_node_api_server::web3::{state::InternalApiConfig, ApiBuilder, ApiServer, Namespace}; use crate::{ implementations::resources::{ diff --git a/core/node/node_framework/src/implementations/layers/web3_api/tx_sender.rs b/core/node/node_framework/src/implementations/layers/web3_api/tx_sender.rs index 35e7aee08403..eea9148f6a6e 100644 --- a/core/node/node_framework/src/implementations/layers/web3_api/tx_sender.rs +++ b/core/node/node_framework/src/implementations/layers/web3_api/tx_sender.rs @@ -1,6 +1,6 @@ use std::{fmt, sync::Arc}; -use zksync_core::api_server::{ +use zksync_node_api_server::{ execution_sandbox::{VmConcurrencyBarrier, VmConcurrencyLimiter}, tx_sender::{ApiContracts, TxSenderBuilder, TxSenderConfig}, }; diff --git a/core/node/node_framework/src/implementations/layers/web3_api/tx_sink.rs b/core/node/node_framework/src/implementations/layers/web3_api/tx_sink.rs index bbac8599ab59..df4812b3c098 100644 --- a/core/node/node_framework/src/implementations/layers/web3_api/tx_sink.rs +++ b/core/node/node_framework/src/implementations/layers/web3_api/tx_sink.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use zksync_core::api_server::tx_sender::{master_pool_sink::MasterPoolSink, proxy::TxProxy}; +use zksync_node_api_server::tx_sender::{master_pool_sink::MasterPoolSink, proxy::TxProxy}; use crate::{ implementations::resources::{ diff --git a/core/node/node_framework/src/implementations/resources/web3_api.rs b/core/node/node_framework/src/implementations/resources/web3_api.rs index d9d71220949f..ba555ccca0ef 100644 --- a/core/node/node_framework/src/implementations/resources/web3_api.rs +++ b/core/node/node_framework/src/implementations/resources/web3_api.rs @@ -1,10 +1,10 @@ use std::sync::Arc; -use zksync_core::api_server::{ +use zksync_metadata_calculator::api_server::TreeApiClient; +use zksync_node_api_server::{ tx_sender::{tx_sink::TxSink, TxSender}, web3::mempool_cache::MempoolCache, }; -use zksync_metadata_calculator::api_server::TreeApiClient; use crate::resource::Resource; From 6e03fa31cfb90a019079a24c944641145b241a2d Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 15:34:03 +0400 Subject: [PATCH 10/15] Extract tee verifier input producer --- Cargo.lock | 22 +++++++++++++++ Cargo.toml | 2 ++ core/lib/zksync_core/Cargo.toml | 1 + core/lib/zksync_core/src/lib.rs | 7 ++--- core/node/shared_metrics/Cargo.toml | 1 - .../tee_verifier_input_producer/Cargo.toml | 28 +++++++++++++++++++ .../tee_verifier_input_producer/README.md | 3 ++ .../tee_verifier_input_producer/src/lib.rs} | 0 .../src}/metrics.rs | 0 9 files changed, 58 insertions(+), 6 deletions(-) create mode 100644 core/node/tee_verifier_input_producer/Cargo.toml create mode 100644 core/node/tee_verifier_input_producer/README.md rename core/{lib/zksync_core/src/tee_verifier_input_producer/mod.rs => node/tee_verifier_input_producer/src/lib.rs} (100%) rename core/{lib/zksync_core/src/tee_verifier_input_producer => node/tee_verifier_input_producer/src}/metrics.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index ed7344fc65b0..2bd0fda9a7f3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8430,6 +8430,7 @@ dependencies = [ "zksync_storage", "zksync_system_constants", "zksync_tee_verifier", + "zksync_tee_verifier_input_producer", "zksync_test_account", "zksync_types", "zksync_utils", @@ -9330,6 +9331,27 @@ dependencies = [ "zksync_utils", ] +[[package]] +name = "zksync_tee_verifier_input_producer" +version = "0.1.0" +dependencies = [ + "anyhow", + "async-trait", + "multivm", + "tokio", + "tracing", + "vise", + "vm_utils", + "zksync_dal", + "zksync_object_store", + "zksync_prover_interface", + "zksync_queued_job_processor", + "zksync_state", + "zksync_tee_verifier", + "zksync_types", + "zksync_utils", +] + [[package]] name = "zksync_test_account" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 268997afcb1c..09d79dcd14cb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -31,6 +31,7 @@ members = [ "core/node/consensus", "core/node/contract_verification_server", "core/node/api_server", + "core/node/tee_verifier_input_producer", # Libraries "core/lib/db_connection", "core/lib/zksync_core", @@ -259,3 +260,4 @@ zksync_node_sync = { path = "core/node/node_sync" } zksync_node_consensus = { path = "core/node/consensus" } zksync_contract_verification_server = { path = "core/node/contract_verification_server" } zksync_node_api_server = { path = "core/node/api_server" } +zksync_tee_verifier_input_producer = { path = "core/node/tee_verifier_input_producer" } diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 511e13e0898e..36583682f1bb 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -57,6 +57,7 @@ zksync_node_sync.workspace = true zksync_node_consensus.workspace = true zksync_contract_verification_server.workspace = true zksync_node_api_server.workspace = true +zksync_tee_verifier_input_producer.workspace = true multivm.workspace = true # Consensus dependenices diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 9a322a5badc3..9631724f837e 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -72,16 +72,13 @@ use zksync_state_keeper::{ create_state_keeper, io::seal_logic::l2_block_seal_subtasks::L2BlockSealProcess, AsyncRocksdbCache, MempoolFetcher, MempoolGuard, OutputHandler, StateKeeperPersistence, }; +use zksync_tee_verifier_input_producer::TeeVerifierInputProducer; use zksync_types::{ethabi::Contract, fee_model::FeeModelConfig, Address, L2ChainId}; use zksync_web3_decl::client::Client; -use crate::{ - tee_verifier_input_producer::TeeVerifierInputProducer, - utils::L1BatchCommitmentModeValidationTask, -}; +use crate::utils::L1BatchCommitmentModeValidationTask; pub mod proto; -pub mod tee_verifier_input_producer; pub mod temp_config_store; pub mod utils; diff --git a/core/node/shared_metrics/Cargo.toml b/core/node/shared_metrics/Cargo.toml index 5ba0ebfbfca1..c6d60828b40a 100644 --- a/core/node/shared_metrics/Cargo.toml +++ b/core/node/shared_metrics/Cargo.toml @@ -9,7 +9,6 @@ license.workspace = true keywords.workspace = true categories.workspace = true - [dependencies] vise.workspace = true zksync_types.workspace = true diff --git a/core/node/tee_verifier_input_producer/Cargo.toml b/core/node/tee_verifier_input_producer/Cargo.toml new file mode 100644 index 000000000000..49856f5c7022 --- /dev/null +++ b/core/node/tee_verifier_input_producer/Cargo.toml @@ -0,0 +1,28 @@ +[package] +name = "zksync_tee_verifier_input_producer" +version = "0.1.0" +edition.workspace = true +authors.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +keywords.workspace = true +categories.workspace = true + +[dependencies] +zksync_dal.workspace = true +zksync_object_store.workspace = true +zksync_prover_interface.workspace = true +zksync_queued_job_processor.workspace = true +zksync_state.workspace = true +zksync_tee_verifier.workspace = true +zksync_types.workspace = true +zksync_utils.workspace = true +multivm.workspace = true +vm_utils.workspace = true +vise.workspace = true + +anyhow.workspace = true +async-trait.workspace = true +tracing.workspace = true +tokio = { workspace = true, features = ["time"] } diff --git a/core/node/tee_verifier_input_producer/README.md b/core/node/tee_verifier_input_producer/README.md new file mode 100644 index 000000000000..75a2029985cc --- /dev/null +++ b/core/node/tee_verifier_input_producer/README.md @@ -0,0 +1,3 @@ +# `zksync_tee_verifier_input_producer` + +Component responsible for producing inputs for verification of execution in TEE. diff --git a/core/lib/zksync_core/src/tee_verifier_input_producer/mod.rs b/core/node/tee_verifier_input_producer/src/lib.rs similarity index 100% rename from core/lib/zksync_core/src/tee_verifier_input_producer/mod.rs rename to core/node/tee_verifier_input_producer/src/lib.rs diff --git a/core/lib/zksync_core/src/tee_verifier_input_producer/metrics.rs b/core/node/tee_verifier_input_producer/src/metrics.rs similarity index 100% rename from core/lib/zksync_core/src/tee_verifier_input_producer/metrics.rs rename to core/node/tee_verifier_input_producer/src/metrics.rs From 70902e5e65f3d8571dce6153238c898e0d9d4e0f Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 15:36:32 +0400 Subject: [PATCH 11/15] Extract L1BatchCommitmentModeValidationTask --- Cargo.lock | 4 ++++ core/bin/external_node/src/main.rs | 6 ++++-- core/lib/zksync_core/src/lib.rs | 7 +++---- core/node/commitment_generator/Cargo.toml | 5 +++++ core/node/commitment_generator/src/lib.rs | 1 + .../commitment_generator/src/validation_task.rs} | 2 -- 6 files changed, 17 insertions(+), 8 deletions(-) rename core/{lib/zksync_core/src/utils/mod.rs => node/commitment_generator/src/validation_task.rs} (99%) diff --git a/Cargo.lock b/Cargo.lock index 2bd0fda9a7f3..ea9da8ace721 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8076,6 +8076,7 @@ dependencies = [ "circuit_sequencer_api 0.1.41", "circuit_sequencer_api 0.1.50", "itertools 0.10.5", + "jsonrpsee", "multivm", "serde_json", "tokio", @@ -8084,11 +8085,14 @@ dependencies = [ "zk_evm 1.3.3 (git+https://github.com/matter-labs/era-zk_evm.git?tag=v1.3.3-rc2)", "zk_evm 1.4.1", "zk_evm 1.5.0", + "zksync_contracts", "zksync_dal", + "zksync_eth_client", "zksync_health_check", "zksync_l1_contract_interface", "zksync_types", "zksync_utils", + "zksync_web3_decl", ] [[package]] diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index ac885b0101e7..6dee870f8f80 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -8,11 +8,13 @@ use tokio::{ task::{self, JoinHandle}, }; use zksync_block_reverter::{BlockReverter, NodeRole}; -use zksync_commitment_generator::CommitmentGenerator; +use zksync_commitment_generator::{ + validation_task::L1BatchCommitmentModeValidationTask, CommitmentGenerator, +}; use zksync_concurrency::{ctx, scope}; use zksync_config::configs::{api::MerkleTreeApiConfig, database::MerkleTreeMode}; use zksync_consistency_checker::ConsistencyChecker; -use zksync_core::{setup_sigint_handler, utils::L1BatchCommitmentModeValidationTask}; +use zksync_core::setup_sigint_handler; use zksync_dal::{metrics::PostgresMetrics, ConnectionPool, Core, CoreDal}; use zksync_db_connection::{ connection_pool::ConnectionPoolBuilder, healthcheck::ConnectionPoolHealthCheck, diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 9631724f837e..fa75ce76a712 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -19,7 +19,9 @@ use zksync_circuit_breaker::{ l1_txs::FailedL1TransactionChecker, replication_lag::ReplicationLagChecker, CircuitBreakerChecker, CircuitBreakers, }; -use zksync_commitment_generator::CommitmentGenerator; +use zksync_commitment_generator::{ + validation_task::L1BatchCommitmentModeValidationTask, CommitmentGenerator, +}; use zksync_concurrency::{ctx, scope}; use zksync_config::{ configs::{ @@ -76,11 +78,8 @@ use zksync_tee_verifier_input_producer::TeeVerifierInputProducer; use zksync_types::{ethabi::Contract, fee_model::FeeModelConfig, Address, L2ChainId}; use zksync_web3_decl::client::Client; -use crate::utils::L1BatchCommitmentModeValidationTask; - pub mod proto; pub mod temp_config_store; -pub mod utils; /// Inserts the initial information about zkSync tokens into the database. pub async fn genesis_init( diff --git a/core/node/commitment_generator/Cargo.toml b/core/node/commitment_generator/Cargo.toml index a15266c7a679..45c62161e3f2 100644 --- a/core/node/commitment_generator/Cargo.toml +++ b/core/node/commitment_generator/Cargo.toml @@ -16,6 +16,8 @@ zksync_dal.workspace = true zksync_health_check.workspace = true zksync_l1_contract_interface.workspace = true zksync_utils.workspace = true +zksync_eth_client.workspace = true +zksync_contracts.workspace = true multivm.workspace = true circuit_sequencer_api_1_4_0.workspace = true circuit_sequencer_api_1_4_1.workspace = true @@ -31,3 +33,6 @@ tracing.workspace = true itertools.workspace = true serde_json.workspace = true +[dev-dependencies] +jsonrpsee.workspace = true +zksync_web3_decl.workspace = true diff --git a/core/node/commitment_generator/src/lib.rs b/core/node/commitment_generator/src/lib.rs index 4c6e48daa1ae..866ef572b065 100644 --- a/core/node/commitment_generator/src/lib.rs +++ b/core/node/commitment_generator/src/lib.rs @@ -26,6 +26,7 @@ use crate::{ mod metrics; mod utils; +pub mod validation_task; const SLEEP_INTERVAL: Duration = Duration::from_millis(100); diff --git a/core/lib/zksync_core/src/utils/mod.rs b/core/node/commitment_generator/src/validation_task.rs similarity index 99% rename from core/lib/zksync_core/src/utils/mod.rs rename to core/node/commitment_generator/src/validation_task.rs index e9816d8cf3c2..06a82c7ca46a 100644 --- a/core/lib/zksync_core/src/utils/mod.rs +++ b/core/node/commitment_generator/src/validation_task.rs @@ -1,5 +1,3 @@ -//! Miscellaneous utils used by multiple components. - use std::time::Duration; use tokio::sync::watch; From 67db50987934e72b1739523b5c0ac2fa3f104944 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 15:45:29 +0400 Subject: [PATCH 12/15] Rename zksync_core -> zksync_core_leftovers --- Cargo.lock | 10 +++++----- Cargo.toml | 4 ++-- core/bin/external_node/Cargo.toml | 2 +- core/bin/external_node/src/config/mod.rs | 2 +- core/bin/external_node/src/main.rs | 2 +- core/bin/genesis_generator/Cargo.toml | 2 +- core/bin/genesis_generator/src/main.rs | 2 +- core/bin/zksync_server/Cargo.toml | 2 +- core/bin/zksync_server/src/config.rs | 2 +- core/bin/zksync_server/src/main.rs | 2 +- .../{zksync_core => zksync_core_leftovers}/Cargo.toml | 4 ++-- .../{zksync_core => zksync_core_leftovers}/build.rs | 0 .../{zksync_core => zksync_core_leftovers}/src/lib.rs | 0 .../src/proto/mod.proto | 0 .../src/proto/mod.rs | 0 .../src/temp_config_store/mod.rs | 0 core/node/node_framework/Cargo.toml | 2 +- core/node/node_framework/examples/main_node.rs | 2 +- 18 files changed, 19 insertions(+), 19 deletions(-) rename core/lib/{zksync_core => zksync_core_leftovers}/Cargo.toml (98%) rename core/lib/{zksync_core => zksync_core_leftovers}/build.rs (100%) rename core/lib/{zksync_core => zksync_core_leftovers}/src/lib.rs (100%) rename core/lib/{zksync_core => zksync_core_leftovers}/src/proto/mod.proto (100%) rename core/lib/{zksync_core => zksync_core_leftovers}/src/proto/mod.rs (100%) rename core/lib/{zksync_core => zksync_core_leftovers}/src/temp_config_store/mod.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index ea9da8ace721..4e59195981a1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2359,7 +2359,7 @@ dependencies = [ "tracing", "zksync_config", "zksync_contracts", - "zksync_core", + "zksync_core_leftovers", "zksync_dal", "zksync_env_config", "zksync_node_genesis", @@ -8349,7 +8349,7 @@ dependencies = [ ] [[package]] -name = "zksync_core" +name = "zksync_core_leftovers" version = "0.1.0" dependencies = [ "anyhow", @@ -8638,7 +8638,7 @@ dependencies = [ "zksync_consensus_roles", "zksync_consistency_checker", "zksync_contracts", - "zksync_core", + "zksync_core_leftovers", "zksync_dal", "zksync_db_connection", "zksync_eth_client", @@ -8939,7 +8939,7 @@ dependencies = [ "zksync_consistency_checker", "zksync_contract_verification_server", "zksync_contracts", - "zksync_core", + "zksync_core_leftovers", "zksync_dal", "zksync_db_connection", "zksync_env_config", @@ -9186,7 +9186,7 @@ dependencies = [ "zksync_consensus_crypto", "zksync_consensus_executor", "zksync_consensus_roles", - "zksync_core", + "zksync_core_leftovers", "zksync_env_config", "zksync_eth_client", "zksync_node_genesis", diff --git a/Cargo.toml b/Cargo.toml index 09d79dcd14cb..d894be260df9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,7 +34,7 @@ members = [ "core/node/tee_verifier_input_producer", # Libraries "core/lib/db_connection", - "core/lib/zksync_core", + "core/lib/zksync_core_leftovers", "core/lib/basic_types", "core/lib/config", "core/lib/constants", @@ -211,7 +211,7 @@ zksync_basic_types = { path = "core/lib/basic_types" } zksync_circuit_breaker = { path = "core/lib/circuit_breaker" } zksync_config = { path = "core/lib/config" } zksync_contracts = { path = "core/lib/contracts" } -zksync_core = { path = "core/lib/zksync_core" } +zksync_core_leftovers = { path = "core/lib/zksync_core_leftovers" } zksync_crypto = { path = "core/lib/crypto" } zksync_dal = { path = "core/lib/dal" } zksync_db_connection = { path = "core/lib/db_connection" } diff --git a/core/bin/external_node/Cargo.toml b/core/bin/external_node/Cargo.toml index 2e87b02e9e4b..3ca6d958a598 100644 --- a/core/bin/external_node/Cargo.toml +++ b/core/bin/external_node/Cargo.toml @@ -11,7 +11,7 @@ categories.workspace = true publish = false [dependencies] -zksync_core.workspace = true +zksync_core_leftovers.workspace = true zksync_commitment_generator.workspace = true zksync_dal.workspace = true zksync_db_connection.workspace = true diff --git a/core/bin/external_node/src/config/mod.rs b/core/bin/external_node/src/config/mod.rs index 13e504ce06d2..87f5ef63ddf2 100644 --- a/core/bin/external_node/src/config/mod.rs +++ b/core/bin/external_node/src/config/mod.rs @@ -14,7 +14,7 @@ use zksync_config::{ }, ObjectStoreConfig, }; -use zksync_core::temp_config_store::decode_yaml_repr; +use zksync_core_leftovers::temp_config_store::decode_yaml_repr; #[cfg(test)] use zksync_dal::{ConnectionPool, Core}; use zksync_node_api_server::{ diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 6dee870f8f80..f239ff125fb6 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -14,7 +14,7 @@ use zksync_commitment_generator::{ use zksync_concurrency::{ctx, scope}; use zksync_config::configs::{api::MerkleTreeApiConfig, database::MerkleTreeMode}; use zksync_consistency_checker::ConsistencyChecker; -use zksync_core::setup_sigint_handler; +use zksync_core_leftovers::setup_sigint_handler; use zksync_dal::{metrics::PostgresMetrics, ConnectionPool, Core, CoreDal}; use zksync_db_connection::{ connection_pool::ConnectionPoolBuilder, healthcheck::ConnectionPoolHealthCheck, diff --git a/core/bin/genesis_generator/Cargo.toml b/core/bin/genesis_generator/Cargo.toml index 8b91505abf09..e6ac400c0ff0 100644 --- a/core/bin/genesis_generator/Cargo.toml +++ b/core/bin/genesis_generator/Cargo.toml @@ -17,7 +17,7 @@ zksync_env_config.workspace = true zksync_protobuf_config.workspace = true zksync_utils.workspace = true zksync_types.workspace = true -zksync_core.workspace = true +zksync_core_leftovers.workspace = true zksync_dal.workspace = true zksync_contracts.workspace = true zksync_protobuf.workspace = true diff --git a/core/bin/genesis_generator/src/main.rs b/core/bin/genesis_generator/src/main.rs index dfad2bd13333..0639ddaac78c 100644 --- a/core/bin/genesis_generator/src/main.rs +++ b/core/bin/genesis_generator/src/main.rs @@ -9,7 +9,7 @@ use clap::Parser; use serde_yaml::Serializer; use zksync_config::{GenesisConfig, PostgresConfig}; use zksync_contracts::BaseSystemContracts; -use zksync_core::temp_config_store::decode_yaml_repr; +use zksync_core_leftovers::temp_config_store::decode_yaml_repr; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_env_config::FromEnv; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; diff --git a/core/bin/zksync_server/Cargo.toml b/core/bin/zksync_server/Cargo.toml index deef059dc960..118288dfe671 100644 --- a/core/bin/zksync_server/Cargo.toml +++ b/core/bin/zksync_server/Cargo.toml @@ -18,7 +18,7 @@ zksync_protobuf_config.workspace = true zksync_storage.workspace = true zksync_utils.workspace = true zksync_types.workspace = true -zksync_core.workspace = true +zksync_core_leftovers.workspace = true zksync_node_genesis.workspace = true # Consensus dependenices diff --git a/core/bin/zksync_server/src/config.rs b/core/bin/zksync_server/src/config.rs index fadcced7942d..8ee796e74c66 100644 --- a/core/bin/zksync_server/src/config.rs +++ b/core/bin/zksync_server/src/config.rs @@ -1,6 +1,6 @@ use anyhow::Context as _; use zksync_config::configs::consensus::{ConsensusConfig, ConsensusSecrets}; -use zksync_core::temp_config_store::decode_yaml_repr; +use zksync_core_leftovers::temp_config_store::decode_yaml_repr; use zksync_protobuf_config::proto; pub(crate) fn read_consensus_secrets() -> anyhow::Result> { diff --git a/core/bin/zksync_server/src/main.rs b/core/bin/zksync_server/src/main.rs index 749e4753d087..1c3bac4d456e 100644 --- a/core/bin/zksync_server/src/main.rs +++ b/core/bin/zksync_server/src/main.rs @@ -18,7 +18,7 @@ use zksync_config::{ ApiConfig, ContractVerifierConfig, DBConfig, EthConfig, EthWatchConfig, GasAdjusterConfig, GenesisConfig, ObjectStoreConfig, PostgresConfig, SnapshotsCreatorConfig, }; -use zksync_core::{ +use zksync_core_leftovers::{ genesis_init, initialize_components, is_genesis_needed, setup_sigint_handler, temp_config_store::{decode_yaml, decode_yaml_repr, Secrets, TempConfigStore}, Component, Components, diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core_leftovers/Cargo.toml similarity index 98% rename from core/lib/zksync_core/Cargo.toml rename to core/lib/zksync_core_leftovers/Cargo.toml index 36583682f1bb..09b8fec422df 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core_leftovers/Cargo.toml @@ -1,5 +1,5 @@ [package] -name = "zksync_core" +name = "zksync_core_leftovers" version = "0.1.0" edition.workspace = true authors.workspace = true @@ -9,7 +9,7 @@ license.workspace = true keywords.workspace = true categories.workspace = true -links = "zksync_core_proto" +links = "zksync_core_leftovers_proto" [dependencies] vise.workspace = true diff --git a/core/lib/zksync_core/build.rs b/core/lib/zksync_core_leftovers/build.rs similarity index 100% rename from core/lib/zksync_core/build.rs rename to core/lib/zksync_core_leftovers/build.rs diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core_leftovers/src/lib.rs similarity index 100% rename from core/lib/zksync_core/src/lib.rs rename to core/lib/zksync_core_leftovers/src/lib.rs diff --git a/core/lib/zksync_core/src/proto/mod.proto b/core/lib/zksync_core_leftovers/src/proto/mod.proto similarity index 100% rename from core/lib/zksync_core/src/proto/mod.proto rename to core/lib/zksync_core_leftovers/src/proto/mod.proto diff --git a/core/lib/zksync_core/src/proto/mod.rs b/core/lib/zksync_core_leftovers/src/proto/mod.rs similarity index 100% rename from core/lib/zksync_core/src/proto/mod.rs rename to core/lib/zksync_core_leftovers/src/proto/mod.rs diff --git a/core/lib/zksync_core/src/temp_config_store/mod.rs b/core/lib/zksync_core_leftovers/src/temp_config_store/mod.rs similarity index 100% rename from core/lib/zksync_core/src/temp_config_store/mod.rs rename to core/lib/zksync_core_leftovers/src/temp_config_store/mod.rs diff --git a/core/node/node_framework/Cargo.toml b/core/node/node_framework/Cargo.toml index 4579655e3a61..8a39c48de783 100644 --- a/core/node/node_framework/Cargo.toml +++ b/core/node/node_framework/Cargo.toml @@ -20,7 +20,7 @@ zksync_config.workspace = true zksync_protobuf_config.workspace = true zksync_state.workspace = true zksync_object_store.workspace = true -zksync_core.workspace = true +zksync_core_leftovers.workspace = true zksync_storage.workspace = true zksync_eth_client.workspace = true zksync_contracts.workspace = true diff --git a/core/node/node_framework/examples/main_node.rs b/core/node/node_framework/examples/main_node.rs index e3d42e1828bc..8d34ec163d34 100644 --- a/core/node/node_framework/examples/main_node.rs +++ b/core/node/node_framework/examples/main_node.rs @@ -19,7 +19,7 @@ use zksync_config::{ ApiConfig, ContractVerifierConfig, ContractsConfig, DBConfig, EthConfig, EthWatchConfig, GasAdjusterConfig, GenesisConfig, ObjectStoreConfig, PostgresConfig, }; -use zksync_core::temp_config_store::decode_yaml_repr; +use zksync_core_leftovers::temp_config_store::decode_yaml_repr; use zksync_env_config::FromEnv; use zksync_metadata_calculator::MetadataCalculatorConfig; use zksync_node_api_server::{ From d0b9b1b045e8834adcdd56c40e9e5cd9906d14e4 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 15:49:17 +0400 Subject: [PATCH 13/15] Spelling --- core/node/metadata_calculator/README.md | 2 +- core/node/state_keeper/src/testonly/test_batch_executor.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/node/metadata_calculator/README.md b/core/node/metadata_calculator/README.md index 15e7a6b92b9c..38c96c2c4b2b 100644 --- a/core/node/metadata_calculator/README.md +++ b/core/node/metadata_calculator/README.md @@ -1,6 +1,6 @@ # `zksync_metadata_calculator` -Component responsible for calculating any supplimentary metadata required for L1 batches. Most notably, this is +Component responsible for calculating any supplementary metadata required for L1 batches. Most notably, this is component responsible for maintaining the Merkle Tree. Additionally, this crate provides ability to spawn the Merkle Tree API server. diff --git a/core/node/state_keeper/src/testonly/test_batch_executor.rs b/core/node/state_keeper/src/testonly/test_batch_executor.rs index 54ae71d406f9..44294a639cf0 100644 --- a/core/node/state_keeper/src/testonly/test_batch_executor.rs +++ b/core/node/state_keeper/src/testonly/test_batch_executor.rs @@ -1,7 +1,7 @@ // TODO(QIT-33): Some of the interfaces are public, and some are only used in tests within this crate. // This causes crate-local interfaces to spawn a warning without `cfg(test)`. The interfaces here must // be revisited and properly split into "truly public" (e.g. useful for other crates to test, say, different -// IO or BatchExecutor implementations) and "local-test-only" (e.g. used only in tests within this crate). +// IO or `BatchExecutor` implementations) and "local-test-only" (e.g. used only in tests within this crate). #![allow(dead_code)] use std::{ From 0800a5d0b49ecf29e473b7e561c7e52612a5d3b1 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 15:51:08 +0400 Subject: [PATCH 14/15] Alter log levels --- etc/env/base/rust.toml | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/etc/env/base/rust.toml b/etc/env/base/rust.toml index ba38b517249a..ee4a69721cd3 100644 --- a/etc/env/base/rust.toml +++ b/etc/env/base/rust.toml @@ -17,10 +17,20 @@ zksync_proof_data_handler=info,\ zksync_shared_metrics=info,\ zksync_node_test_utils=info,\ zksync_vm_runner=info,\ +zksync_node_test_utils=info,\ +zksync_state_keeper=info,\ +zksync_reorg_detector=info,\ +zksync_consistency_checker=info,\ +zksync_metadata_calculator=info,\ +zksync_node_sync=info,\ +zksync_node_consensus=info,\ +zksync_contract_verification_server=info,\ +zksync_node_api_server=info,\ +zksync_tee_verifier_input_producer=info,\ zksync_consensus_bft=info,\ zksync_consensus_network=info,\ zksync_consensus_storage=info,\ -zksync_core=debug,\ +zksync_core_leftovers=debug,\ zksync_server=debug,\ zksync_contract_verifier=debug,\ zksync_dal=info,\ From aa3be82072ae4cccd45310fb0cfeeb48ea59e090 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 15 May 2024 16:50:31 +0400 Subject: [PATCH 15/15] Update file-based config as well --- etc/env/file_based/general.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/etc/env/file_based/general.yaml b/etc/env/file_based/general.yaml index 9b5b7cafbf27..ca5a9f18069d 100644 --- a/etc/env/file_based/general.yaml +++ b/etc/env/file_based/general.yaml @@ -316,7 +316,7 @@ prometheus: observability: log_format: plain - log_directives: "zksync_node_framework=info,zksync_block_reverter=info,zksync_commitment_generator=info,zksync_node_db_pruner=info,zksync_eth_sender=info,zksync_node_fee_model=info,zksync_node_genesis=info,zksync_house_keeper=info,zksync_proof_data_handler=info,zksync_shared_metrics=info,zksync_node_test_utils=info,zksync_vm_runner=info,zksync_consensus_bft=info,zksync_consensus_network=info,zksync_consensus_storage=info,zksync_core=debug,zksync_server=debug,zksync_contract_verifier=debug,zksync_dal=info,zksync_db_connection=info,zksync_eth_client=info,zksync_eth_watch=debug,zksync_storage=info,zksync_db_manager=info,zksync_merkle_tree=info,zksync_state=debug,zksync_utils=debug,zksync_queued_job_processor=info,zksync_types=info,zksync_mempool=debug,loadnext=info,vm=info,zksync_object_store=info,zksync_external_node=info,zksync_witness_generator=info,zksync_prover_fri=info,zksync_witness_vector_generator=info,zksync_web3_decl=debug,zksync_health_check=debug,zksync_proof_fri_compressor=info,vise_exporter=debug,snapshots_creator=debug" + log_directives: "zksync_node_test_utils=info,zksync_state_keeper=info,zksync_reorg_detector=info,zksync_consistency_checker=info,zksync_metadata_calculator=info,zksync_node_sync=info,zksync_node_consensus=info,zksync_contract_verification_server=info,zksync_node_api_server=info,zksync_tee_verifier_input_producer=info,zksync_node_framework=info,zksync_block_reverter=info,zksync_commitment_generator=info,zksync_node_db_pruner=info,zksync_eth_sender=info,zksync_node_fee_model=info,zksync_node_genesis=info,zksync_house_keeper=info,zksync_proof_data_handler=info,zksync_shared_metrics=info,zksync_node_test_utils=info,zksync_vm_runner=info,zksync_consensus_bft=info,zksync_consensus_network=info,zksync_consensus_storage=info,zksync_core_leftovers=debug,zksync_server=debug,zksync_contract_verifier=debug,zksync_dal=info,zksync_db_connection=info,zksync_eth_client=info,zksync_eth_watch=debug,zksync_storage=info,zksync_db_manager=info,zksync_merkle_tree=info,zksync_state=debug,zksync_utils=debug,zksync_queued_job_processor=info,zksync_types=info,zksync_mempool=debug,loadnext=info,vm=info,zksync_object_store=info,zksync_external_node=info,zksync_witness_generator=info,zksync_prover_fri=info,zksync_witness_vector_generator=info,zksync_web3_decl=debug,zksync_health_check=debug,zksync_proof_fri_compressor=info,vise_exporter=debug,snapshots_creator=debug" sentry: url: unset panic_interval: 1800