From e19d6544abc4c35528227f35b61ae1220005f13d Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 3 Jan 2024 15:41:52 +0400 Subject: [PATCH 01/49] refactor(metadata_calculator): Make config owned (#808) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Makes metadata calculator config owned + makes it use `Box` instead of `ObjectStoreFactory`. ## Why ❔ - Prerequisite for ZK Stack modular thingy. - Makes it easy to store the config object in a dynamic context. - Makes it more abstract w.r.t. resources required for instantiation. ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/main.rs | 8 ++-- core/lib/zksync_core/src/lib.rs | 6 +-- .../src/metadata_calculator/mod.rs | 39 +++++++++---------- .../src/metadata_calculator/recovery/tests.rs | 2 +- .../src/metadata_calculator/tests.rs | 9 +++-- 5 files changed, 30 insertions(+), 34 deletions(-) diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index de05efe396c7..c9edc4770581 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -181,11 +181,9 @@ async fn init_tasks( stop_receiver.clone(), ); - let metadata_calculator = MetadataCalculator::new(&MetadataCalculatorConfig { - db_path: &config.required.merkle_tree_path, - mode: MetadataCalculatorModeConfig::Full { - store_factory: None, - }, + let metadata_calculator = MetadataCalculator::new(MetadataCalculatorConfig { + db_path: config.required.merkle_tree_path.clone(), + mode: MetadataCalculatorModeConfig::Full { object_store: None }, delay_interval: config.optional.metadata_calculator_delay(), max_l1_batches_per_iter: config.optional.max_l1_batches_per_tree_iter, multi_get_chunk_size: config.optional.merkle_tree_multi_get_chunk_size, diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 133814c0abaa..936ad4cfbe21 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -774,7 +774,7 @@ async fn add_trees_to_task_futures( let mode = match db_config.merkle_tree.mode { MerkleTreeMode::Lightweight => MetadataCalculatorModeConfig::Lightweight, MerkleTreeMode::Full => MetadataCalculatorModeConfig::Full { - store_factory: Some(store_factory), + object_store: Some(store_factory.create_store().await), }, }; @@ -800,7 +800,7 @@ async fn run_tree( db_config: &DBConfig, api_config: Option<&MerkleTreeApiConfig>, operation_manager: &OperationsManagerConfig, - mode: MetadataCalculatorModeConfig<'_>, + mode: MetadataCalculatorModeConfig, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { let started_at = Instant::now(); @@ -813,7 +813,7 @@ async fn run_tree( let config = MetadataCalculatorConfig::for_main_node(&db_config.merkle_tree, operation_manager, mode); - let metadata_calculator = MetadataCalculator::new(&config).await; + let metadata_calculator = MetadataCalculator::new(config).await; if let Some(api_config) = api_config { let address = (Ipv4Addr::UNSPECIFIED, api_config.port).into(); let tree_reader = metadata_calculator.tree_reader(); diff --git a/core/lib/zksync_core/src/metadata_calculator/mod.rs b/core/lib/zksync_core/src/metadata_calculator/mod.rs index 0cfea020203c..c0fb5142eb6d 100644 --- a/core/lib/zksync_core/src/metadata_calculator/mod.rs +++ b/core/lib/zksync_core/src/metadata_calculator/mod.rs @@ -14,7 +14,7 @@ use zksync_config::configs::{ use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_health_check::{HealthUpdater, ReactiveHealthCheck}; use zksync_merkle_tree::domain::TreeMetadata; -use zksync_object_store::{ObjectStore, ObjectStoreFactory}; +use zksync_object_store::ObjectStore; use zksync_types::{ block::L1BatchHeader, commitment::{L1BatchCommitment, L1BatchMetadata}, @@ -37,20 +37,20 @@ pub(crate) mod tests; mod updater; /// Part of [`MetadataCalculator`] related to the operation mode of the Merkle tree. -#[derive(Debug, Clone, Copy)] -pub enum MetadataCalculatorModeConfig<'a> { +#[derive(Debug)] +pub enum MetadataCalculatorModeConfig { /// In this mode, `MetadataCalculator` computes Merkle tree root hashes and some auxiliary information /// for L1 batches, but not witness inputs. Lightweight, /// In this mode, `MetadataCalculator` will compute commitments and witness inputs for all storage operations - /// and optionally put witness inputs into the object store as provided by `store_factory` (e.g., GCS). + /// and optionally put witness inputs into the object store (e.g., GCS). Full { - store_factory: Option<&'a ObjectStoreFactory>, + object_store: Option>, }, } -impl MetadataCalculatorModeConfig<'_> { - fn to_mode(self) -> MerkleTreeMode { +impl MetadataCalculatorModeConfig { + fn to_mode(&self) -> MerkleTreeMode { if matches!(self, Self::Full { .. }) { MerkleTreeMode::Full } else { @@ -61,11 +61,11 @@ impl MetadataCalculatorModeConfig<'_> { /// Configuration of [`MetadataCalculator`]. #[derive(Debug)] -pub struct MetadataCalculatorConfig<'a> { +pub struct MetadataCalculatorConfig { /// Filesystem path to the RocksDB instance that stores the tree. - pub db_path: &'a str, + pub db_path: String, /// Configuration of the Merkle tree mode. - pub mode: MetadataCalculatorModeConfig<'a>, + pub mode: MetadataCalculatorModeConfig, /// Interval between polling Postgres for updates if no progress was made by the tree. pub delay_interval: Duration, /// Maximum number of L1 batches to get from Postgres on a single update iteration. @@ -82,14 +82,14 @@ pub struct MetadataCalculatorConfig<'a> { pub stalled_writes_timeout: Duration, } -impl<'a> MetadataCalculatorConfig<'a> { +impl MetadataCalculatorConfig { pub(crate) fn for_main_node( - merkle_tree_config: &'a MerkleTreeConfig, - operation_config: &'a OperationsManagerConfig, - mode: MetadataCalculatorModeConfig<'a>, + merkle_tree_config: &MerkleTreeConfig, + operation_config: &OperationsManagerConfig, + mode: MetadataCalculatorModeConfig, ) -> Self { Self { - db_path: &merkle_tree_config.path, + db_path: merkle_tree_config.path.clone(), mode, delay_interval: operation_config.delay_interval(), max_l1_batches_per_iter: merkle_tree_config.max_l1_batches_per_iter, @@ -113,7 +113,7 @@ pub struct MetadataCalculator { impl MetadataCalculator { /// Creates a calculator with the specified `config`. - pub async fn new(config: &MetadataCalculatorConfig<'_>) -> Self { + pub async fn new(config: MetadataCalculatorConfig) -> Self { assert!( config.max_l1_batches_per_iter > 0, "Maximum L1 batches per iteration is misconfigured to be 0; please update it to positive value" @@ -121,15 +121,12 @@ impl MetadataCalculator { let mode = config.mode.to_mode(); let object_store = match config.mode { - MetadataCalculatorModeConfig::Full { store_factory } => match store_factory { - Some(f) => Some(f.create_store().await), - None => None, - }, + MetadataCalculatorModeConfig::Full { object_store } => object_store, MetadataCalculatorModeConfig::Lightweight => None, }; let db = create_db( - config.db_path.into(), + config.db_path.clone().into(), config.block_cache_capacity, config.memtable_capacity, config.stalled_writes_timeout, diff --git a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs index d121990dc283..082f684bbc9c 100644 --- a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs @@ -294,7 +294,7 @@ async fn entire_recovery_workflow(case: RecoveryWorkflowCase) { &OperationsManagerConfig { delay_interval: 50 }, MetadataCalculatorModeConfig::Lightweight, ); - let mut calculator = MetadataCalculator::new(&calculator_config).await; + let mut calculator = MetadataCalculator::new(calculator_config).await; let (delay_sx, mut delay_rx) = mpsc::unbounded_channel(); calculator.delayer.delay_notifier = delay_sx; diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index 8c8121fac65a..1a5d2a6d7704 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -363,10 +363,11 @@ pub(crate) async fn setup_calculator( db_path: &Path, pool: &ConnectionPool, ) -> (MetadataCalculator, Box) { - let store_factory = &ObjectStoreFactory::mock(); + let store_factory = ObjectStoreFactory::mock(); + let store = store_factory.create_store().await; let (merkle_tree_config, operation_manager) = create_config(db_path); let mode = MetadataCalculatorModeConfig::Full { - store_factory: Some(store_factory), + object_store: Some(store), }; let calculator = setup_calculator_with_options(&merkle_tree_config, &operation_manager, pool, mode).await; @@ -395,11 +396,11 @@ async fn setup_calculator_with_options( merkle_tree_config: &MerkleTreeConfig, operation_config: &OperationsManagerConfig, pool: &ConnectionPool, - mode: MetadataCalculatorModeConfig<'_>, + mode: MetadataCalculatorModeConfig, ) -> MetadataCalculator { let calculator_config = MetadataCalculatorConfig::for_main_node(merkle_tree_config, operation_config, mode); - let metadata_calculator = MetadataCalculator::new(&calculator_config).await; + let metadata_calculator = MetadataCalculator::new(calculator_config).await; let mut storage = pool.access_storage().await.unwrap(); if storage.blocks_dal().is_genesis_needed().await.unwrap() { From cbe8c7fb14e44fdf112f919b48008cf2c74ee8ef Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Wed, 3 Jan 2024 15:42:02 +0400 Subject: [PATCH 02/49] refactor(state_keeper): Abstract ConditionalSealer (#803) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Defines a new trait: `ConditionalSealer` and provides two implementations: `SequencerSealer` and `NoopSealer`. ## Why ❔ - Prerequisite for ZK Stack configuration system. - Leaves a single constructor for the state keeper. - Removes `StateKeeperConfig` use from `TxSender`. - Potentially makes it possible to create `ConditionalSealer` that contains some kind of sealing logic but does not rely on `StateKeeperConfig`. ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/main.rs | 11 +- .../src/api_server/tx_sender/mod.rs | 43 ++++---- .../lib/zksync_core/src/consensus/testonly.rs | 4 +- core/lib/zksync_core/src/lib.rs | 7 +- .../zksync_core/src/state_keeper/keeper.rs | 39 ++----- core/lib/zksync_core/src/state_keeper/mod.rs | 8 +- .../seal_criteria/conditional_sealer.rs | 102 ++++++++++++++---- .../src/state_keeper/seal_criteria/mod.rs | 4 +- .../zksync_core/src/state_keeper/tests/mod.rs | 24 ++--- .../src/state_keeper/tests/tester.rs | 6 +- core/lib/zksync_core/src/sync_layer/tests.rs | 4 +- spellcheck/era.dic | 3 +- 12 files changed, 151 insertions(+), 104 deletions(-) diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index c9edc4770581..d815619934f1 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -23,8 +23,8 @@ use zksync_core::{ reorg_detector::ReorgDetector, setup_sigint_handler, state_keeper::{ - L1BatchExecutorBuilder, MainBatchExecutorBuilder, MiniblockSealer, MiniblockSealerHandle, - ZkSyncStateKeeper, + seal_criteria::NoopSealer, L1BatchExecutorBuilder, MainBatchExecutorBuilder, + MiniblockSealer, MiniblockSealerHandle, ZkSyncStateKeeper, }, sync_layer::{ batch_status_updater::BatchStatusUpdater, external_io::ExternalIO, fetcher::FetcherCursor, @@ -92,7 +92,12 @@ async fn build_state_keeper( ) .await; - ZkSyncStateKeeper::without_sealer(stop_receiver, Box::new(io), batch_executor_base) + ZkSyncStateKeeper::new( + stop_receiver, + Box::new(io), + batch_executor_base, + Box::new(NoopSealer), + ) } async fn init_tasks( diff --git a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs index ad332a899308..f99b3cba5ed7 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/mod.rs +++ b/core/lib/zksync_core/src/api_server/tx_sender/mod.rs @@ -39,7 +39,7 @@ use crate::{ }, l1_gas_price::L1GasPriceProvider, metrics::{TxStage, APP_METRICS}, - state_keeper::seal_criteria::{ConditionalSealer, SealData}, + state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}, }; mod proxy; @@ -139,9 +139,8 @@ pub struct TxSenderBuilder { master_connection_pool: Option, /// Proxy to submit transactions to the network. If not set, `master_connection_pool` must be set. proxy: Option, - /// Actual state keeper configuration, required for tx verification. - /// If not set, transactions would not be checked against seal criteria. - state_keeper_config: Option, + /// Batch sealer used to check whether transaction can be executed by the sequencer. + sealer: Option>, } impl TxSenderBuilder { @@ -151,10 +150,15 @@ impl TxSenderBuilder { replica_connection_pool, master_connection_pool: None, proxy: None, - state_keeper_config: None, + sealer: None, } } + pub fn with_sealer(mut self, sealer: Arc) -> Self { + self.sealer = Some(sealer); + self + } + pub fn with_tx_proxy(mut self, main_node_url: &str) -> Self { self.proxy = Some(TxProxy::new(main_node_url)); self @@ -165,11 +169,6 @@ impl TxSenderBuilder { self } - pub fn with_state_keeper_config(mut self, state_keeper_config: StateKeeperConfig) -> Self { - self.state_keeper_config = Some(state_keeper_config); - self - } - pub async fn build( self, l1_gas_price_source: Arc, @@ -182,6 +181,9 @@ impl TxSenderBuilder { "Either master connection pool or proxy must be set" ); + // Use noop sealer if no sealer was explicitly provided. + let sealer = self.sealer.unwrap_or_else(|| Arc::new(NoopSealer)); + TxSender(Arc::new(TxSenderInner { sender_config: self.config, master_connection_pool: self.master_connection_pool, @@ -189,9 +191,9 @@ impl TxSenderBuilder { l1_gas_price_source, api_contracts, proxy: self.proxy, - state_keeper_config: self.state_keeper_config, vm_concurrency_limiter, storage_caches, + sealer, })) } } @@ -241,14 +243,12 @@ pub struct TxSenderInner { pub(super) api_contracts: ApiContracts, /// Optional transaction proxy to be used for transaction submission. pub(super) proxy: Option, - /// An up-to-date version of the state keeper config. - /// This field may be omitted on the external node, since the configuration may change unexpectedly. - /// If this field is set to `None`, `TxSender` will assume that any transaction is executable. - state_keeper_config: Option, /// Used to limit the amount of VMs that can be executed simultaneously. pub(super) vm_concurrency_limiter: Arc, // Caches used in VM execution. storage_caches: PostgresStorageCaches, + /// Batch sealer used to check whether transaction can be executed by the sequencer. + sealer: Arc, } #[derive(Clone)] @@ -850,13 +850,6 @@ impl TxSender { tx_metrics: &TransactionExecutionMetrics, log_message: bool, ) -> Result<(), SubmitTxError> { - let Some(sk_config) = &self.0.state_keeper_config else { - // No config provided, so we can't check if transaction satisfies the seal criteria. - // We assume that it's executable, and if it's not, it will be caught by the main server - // (where this check is always performed). - return Ok(()); - }; - // Hash is not computable for the provided `transaction` during gas estimation (it doesn't have // its input data set). Since we don't log a hash in this case anyway, we just use a dummy value. let tx_hash = if log_message { @@ -870,8 +863,10 @@ impl TxSender { // still reject them as it's not. let protocol_version = ProtocolVersionId::latest(); let seal_data = SealData::for_transaction(transaction, tx_metrics, protocol_version); - if let Some(reason) = - ConditionalSealer::find_unexecutable_reason(sk_config, &seal_data, protocol_version) + if let Some(reason) = self + .0 + .sealer + .find_unexecutable_reason(&seal_data, protocol_version) { let message = format!( "Tx is Unexecutable because of {reason}; inputs for decision: {seal_data:?}" diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 4195cd76a054..9de6f7085e20 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -12,6 +12,7 @@ use zksync_types::{ use crate::{ genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ + seal_criteria::NoopSealer, tests::{create_l1_batch_metadata, create_l2_transaction, MockBatchExecutorBuilder}, MiniblockSealer, ZkSyncStateKeeper, }, @@ -355,10 +356,11 @@ impl StateKeeperRunner { s.spawn_bg(miniblock_sealer.run()); s.spawn_bg(run_mock_metadata_calculator(ctx, pool.clone())); s.spawn_bg( - ZkSyncStateKeeper::without_sealer( + ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(MockBatchExecutorBuilder), + Box::new(NoopSealer), ) .run(), ); diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 936ad4cfbe21..8269793a40b1 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -70,7 +70,9 @@ use crate::{ MetadataCalculator, MetadataCalculatorConfig, MetadataCalculatorModeConfig, }, metrics::{InitStage, APP_METRICS}, - state_keeper::{create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer}, + state_keeper::{ + create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer, SequencerSealer, + }, }; pub mod api_server; @@ -1032,9 +1034,10 @@ async fn build_tx_sender( l1_gas_price_provider: Arc, storage_caches: PostgresStorageCaches, ) -> (TxSender, VmConcurrencyBarrier) { + let sequencer_sealer = SequencerSealer::new(state_keeper_config.clone()); let tx_sender_builder = TxSenderBuilder::new(tx_sender_config.clone(), replica_pool) .with_main_connection_pool(master_pool) - .with_state_keeper_config(state_keeper_config.clone()); + .with_sealer(Arc::new(sequencer_sealer)); let max_concurrency = web3_json_config.vm_concurrency_limit(); let (vm_concurrency_limiter, vm_barrier) = VmConcurrencyLimiter::new(max_concurrency); diff --git a/core/lib/zksync_core/src/state_keeper/keeper.rs b/core/lib/zksync_core/src/state_keeper/keeper.rs index 3cc153120c26..209809d33f95 100644 --- a/core/lib/zksync_core/src/state_keeper/keeper.rs +++ b/core/lib/zksync_core/src/state_keeper/keeper.rs @@ -58,7 +58,7 @@ pub struct ZkSyncStateKeeper { stop_receiver: watch::Receiver, io: Box, batch_executor_base: Box, - sealer: Option, + sealer: Box, } impl ZkSyncStateKeeper { @@ -66,26 +66,13 @@ impl ZkSyncStateKeeper { stop_receiver: watch::Receiver, io: Box, batch_executor_base: Box, - sealer: ConditionalSealer, + sealer: Box, ) -> Self { Self { stop_receiver, io, batch_executor_base, - sealer: Some(sealer), - } - } - - pub fn without_sealer( - stop_receiver: watch::Receiver, - io: Box, - batch_executor_base: Box, - ) -> Self { - Self { - stop_receiver, - io, - batch_executor_base, - sealer: None, + sealer, } } @@ -651,18 +638,14 @@ impl ZkSyncStateKeeper { writes_metrics: block_writes_metrics, }; - if let Some(sealer) = &self.sealer { - sealer.should_seal_l1_batch( - self.io.current_l1_batch_number().0, - updates_manager.batch_timestamp() as u128 * 1_000, - updates_manager.pending_executed_transactions_len() + 1, - &block_data, - &tx_data, - updates_manager.protocol_version(), - ) - } else { - SealResolution::NoSeal - } + self.sealer.should_seal_l1_batch( + self.io.current_l1_batch_number().0, + updates_manager.batch_timestamp() as u128 * 1_000, + updates_manager.pending_executed_transactions_len() + 1, + &block_data, + &tx_data, + updates_manager.protocol_version(), + ) } }; (resolution, exec_result) diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index 6175befb1baf..d9f09ba49c6e 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -16,7 +16,7 @@ pub use self::{ keeper::ZkSyncStateKeeper, }; pub(crate) use self::{ - mempool_actor::MempoolFetcher, seal_criteria::ConditionalSealer, types::MempoolGuard, + mempool_actor::MempoolFetcher, seal_criteria::SequencerSealer, types::MempoolGuard, }; use crate::l1_gas_price::L1GasPriceProvider; @@ -26,7 +26,7 @@ pub(crate) mod io; mod keeper; mod mempool_actor; pub(crate) mod metrics; -pub(crate) mod seal_criteria; +pub mod seal_criteria; #[cfg(test)] pub(crate) mod tests; pub(crate) mod types; @@ -76,11 +76,11 @@ pub(crate) async fn create_state_keeper( ) .await; - let sealer = ConditionalSealer::new(state_keeper_config); + let sealer = SequencerSealer::new(state_keeper_config); ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(batch_executor_base), - sealer, + Box::new(sealer), ) } diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs index 2b5fb9fba485..f239f108f5ba 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs @@ -1,7 +1,10 @@ //! This module represents the conditional sealer, which can decide whether the batch //! should be sealed after executing a particular transaction. -//! It is used on the main node to decide when the batch should be sealed (as opposed to the external node, -//! which unconditionally follows the instructions from the main node). +//! +//! The conditional sealer abstraction allows to implement different sealing strategies, e.g. the actual +//! sealing strategy for the main node or noop sealer for the external node. + +use std::fmt; use zksync_config::configs::chain::StateKeeperConfig; use zksync_types::ProtocolVersionId; @@ -9,28 +12,51 @@ use zksync_types::ProtocolVersionId; use super::{criteria, SealCriterion, SealData, SealResolution, AGGREGATION_METRICS}; /// Checks if an L1 batch should be sealed after executing a transaction. +pub trait ConditionalSealer: 'static + fmt::Debug + Send + Sync { + /// Finds a reason why a transaction with the specified `data` is unexecutable. + /// + /// Can be used to determine whether the transaction can be executed by the sequencer. + fn find_unexecutable_reason( + &self, + data: &SealData, + protocol_version: ProtocolVersionId, + ) -> Option<&'static str>; + + /// Returns the action that should be taken by the state keeper after executing a transaction. + fn should_seal_l1_batch( + &self, + l1_batch_number: u32, + block_open_timestamp_ms: u128, + tx_count: usize, + block_data: &SealData, + tx_data: &SealData, + protocol_version: ProtocolVersionId, + ) -> SealResolution; +} + +/// Implementation of [`ConditionalSealer`] used by the main node. +/// Internally uses a set of [`SealCriterion`]s to determine whether the batch should be sealed. /// /// The checks are deterministic, i.e., should depend solely on execution metrics and [`StateKeeperConfig`]. /// Non-deterministic seal criteria are expressed using [`IoSealCriteria`](super::IoSealCriteria). #[derive(Debug)] -pub struct ConditionalSealer { +pub struct SequencerSealer { config: StateKeeperConfig, sealers: Vec>, } -impl ConditionalSealer { - /// Finds a reason why a transaction with the specified `data` is unexecutable. - pub(crate) fn find_unexecutable_reason( - config: &StateKeeperConfig, +impl ConditionalSealer for SequencerSealer { + fn find_unexecutable_reason( + &self, data: &SealData, protocol_version: ProtocolVersionId, ) -> Option<&'static str> { - for sealer in &Self::default_sealers() { + for sealer in &self.sealers { const MOCK_BLOCK_TIMESTAMP: u128 = 0; const TX_COUNT: usize = 1; let resolution = sealer.should_seal( - config, + &self.config, MOCK_BLOCK_TIMESTAMP, TX_COUNT, data, @@ -44,20 +70,7 @@ impl ConditionalSealer { None } - pub(crate) fn new(config: StateKeeperConfig) -> Self { - let sealers = Self::default_sealers(); - Self { config, sealers } - } - - #[cfg(test)] - pub(in crate::state_keeper) fn with_sealers( - config: StateKeeperConfig, - sealers: Vec>, - ) -> Self { - Self { config, sealers } - } - - pub fn should_seal_l1_batch( + fn should_seal_l1_batch( &self, l1_batch_number: u32, block_open_timestamp_ms: u128, @@ -99,6 +112,21 @@ impl ConditionalSealer { } final_seal_resolution } +} + +impl SequencerSealer { + pub(crate) fn new(config: StateKeeperConfig) -> Self { + let sealers = Self::default_sealers(); + Self { config, sealers } + } + + #[cfg(test)] + pub(in crate::state_keeper) fn with_sealers( + config: StateKeeperConfig, + sealers: Vec>, + ) -> Self { + Self { config, sealers } + } fn default_sealers() -> Vec> { vec![ @@ -114,3 +142,31 @@ impl ConditionalSealer { ] } } + +/// Implementation of [`ConditionalSealer`] that never seals the batch. +/// Can be used in contexts where, for example, state keeper configuration is not available, +/// or the decision to seal batch is taken by some other component. +#[derive(Debug)] +pub struct NoopSealer; + +impl ConditionalSealer for NoopSealer { + fn find_unexecutable_reason( + &self, + _data: &SealData, + _protocol_version: ProtocolVersionId, + ) -> Option<&'static str> { + None + } + + fn should_seal_l1_batch( + &self, + _l1_batch_number: u32, + _block_open_timestamp_ms: u128, + _tx_count: usize, + _block_data: &SealData, + _tx_data: &SealData, + _protocol_version: ProtocolVersionId, + ) -> SealResolution { + SealResolution::NoSeal + } +} diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs index 99cb25c654d8..bf44c7af0ecf 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/mod.rs @@ -25,7 +25,7 @@ use zksync_utils::time::millis_since; mod conditional_sealer; pub(super) mod criteria; -pub(crate) use self::conditional_sealer::ConditionalSealer; +pub use self::conditional_sealer::{ConditionalSealer, NoopSealer, SequencerSealer}; use super::{extractors, metrics::AGGREGATION_METRICS, updates::UpdatesManager}; use crate::gas_tracker::{gas_count_from_tx_and_metrics, gas_count_from_writes}; @@ -104,7 +104,7 @@ impl SealData { } } -pub(super) trait SealCriterion: fmt::Debug + Send + 'static { +pub(super) trait SealCriterion: fmt::Debug + Send + Sync + 'static { fn should_seal( &self, config: &StateKeeperConfig, diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index c725cdcf010b..044e65742e78 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -42,7 +42,7 @@ use crate::{ keeper::POLL_WAIT_DURATION, seal_criteria::{ criteria::{GasCriterion, SlotsCriterion}, - ConditionalSealer, + SequencerSealer, }, types::ExecutionMetricsForCriteria, updates::UpdatesManager, @@ -250,7 +250,7 @@ async fn sealed_by_number_of_txs() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) @@ -271,7 +271,7 @@ async fn sealed_by_gas() { close_block_at_gas_percentage: 0.5, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(GasCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(GasCriterion)]); let l1_gas_per_tx = BlockGasCount { commit: 1, // Both txs together with block_base_cost would bring it over the block 31_001 commit bound. @@ -320,7 +320,7 @@ async fn sealed_by_gas_then_by_num_tx() { transaction_slots: 3, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers( + let sealer = SequencerSealer::with_sealers( config, vec![Box::new(GasCriterion), Box::new(SlotsCriterion)], ); @@ -357,7 +357,7 @@ async fn batch_sealed_before_miniblock_does() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); // Miniblock sealer will not return true before the batch is sealed because the batch only has 2 txs. TestScenario::new() @@ -382,7 +382,7 @@ async fn rejected_tx() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let rejected_tx = random_tx(1); TestScenario::new() @@ -404,7 +404,7 @@ async fn bootloader_tip_out_of_gas_flow() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let first_tx = random_tx(1); let bootloader_out_of_gas_tx = random_tx(2); @@ -442,7 +442,7 @@ async fn pending_batch_is_applied() { transaction_slots: 3, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let pending_batch = pending_batch_data(vec![ MiniblockExecutionData { @@ -500,7 +500,7 @@ async fn unconditional_sealing() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_l1_batch_when(move |_| batch_seal_trigger_checker.load(Ordering::Relaxed)) @@ -530,7 +530,7 @@ async fn miniblock_timestamp_after_pending_batch() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); let pending_batch = pending_batch_data(vec![MiniblockExecutionData { number: MiniblockNumber(1), @@ -574,7 +574,7 @@ async fn time_is_monotonic() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) @@ -625,7 +625,7 @@ async fn protocol_upgrade() { transaction_slots: 2, ..StateKeeperConfig::default() }; - let sealer = ConditionalSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); + let sealer = SequencerSealer::with_sealers(config, vec![Box::new(SlotsCriterion)]); TestScenario::new() .seal_miniblock_when(|updates| updates.miniblock.executed_transactions.len() == 1) diff --git a/core/lib/zksync_core/src/state_keeper/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/tests/tester.rs index 2c26370a22a9..ce0293e7e679 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/tester.rs @@ -23,7 +23,7 @@ use zksync_types::{ use crate::state_keeper::{ batch_executor::{BatchExecutorHandle, Command, L1BatchExecutorBuilder, TxExecutionResult}, io::{MiniblockParams, PendingBatchData, StateKeeperIO}, - seal_criteria::{ConditionalSealer, IoSealCriteria}, + seal_criteria::{IoSealCriteria, SequencerSealer}, tests::{ create_l2_transaction, default_l1_batch_env, default_vm_block_result, BASE_SYSTEM_CONTRACTS, }, @@ -189,7 +189,7 @@ impl TestScenario { /// Launches the test. /// Provided `SealManager` is expected to be externally configured to adhere the written scenario logic. - pub(crate) async fn run(self, sealer: ConditionalSealer) { + pub(crate) async fn run(self, sealer: SequencerSealer) { assert!(!self.actions.is_empty(), "Test scenario can't be empty"); let batch_executor_base = TestBatchExecutorBuilder::new(&self); @@ -199,7 +199,7 @@ impl TestScenario { stop_receiver, Box::new(io), Box::new(batch_executor_base), - sealer, + Box::new(sealer), ); let sk_thread = tokio::spawn(sk.run()); diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 5b36f3c039bb..619ee282e718 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -19,6 +19,7 @@ use crate::{ consensus::testonly::MockMainNodeClient, genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ + seal_criteria::NoopSealer, tests::{create_l1_batch_metadata, create_l2_transaction, TestBatchExecutorBuilder}, MiniblockSealer, ZkSyncStateKeeper, }, @@ -77,10 +78,11 @@ impl StateKeeperHandles { batch_executor_base.push_successful_transactions(tx_hashes_in_l1_batch); } - let state_keeper = ZkSyncStateKeeper::without_sealer( + let state_keeper = ZkSyncStateKeeper::new( stop_receiver, Box::new(io), Box::new(batch_executor_base), + Box::new(NoopSealer), ); Self { stop_sender, diff --git a/spellcheck/era.dic b/spellcheck/era.dic index a054a5930270..9c0f6bb87401 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -607,4 +607,5 @@ dal codebase compactions M6 -compiler_common \ No newline at end of file +compiler_common +noop \ No newline at end of file From 8be1925b18dcbf268eb03b8ea5f07adfd5330876 Mon Sep 17 00:00:00 2001 From: EmilLuta Date: Wed, 3 Jan 2024 12:52:15 +0100 Subject: [PATCH 03/49] feat(prover): Remove old prover (#810) This is one of many commits, most notable upcoming: - remove database tables via migrations - move prover related data from core to prover (I.E. prover_utils) This comes as part of migration to boojum. Old prover is decomissioned; this PR removes what's left of main components of old prover subsystems. It includes: - remove old prover - remove old prover protocol version - untangle old prover and fri prover common dependencies - and many more ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .dockerignore | 1 - .gitignore | 1 - Cargo.lock | 20 - Cargo.toml | 1 - .../Cargo.toml | 37 - .../README.md | 39 - .../data/verification_0_key.json | 399 ------ .../data/verification_10_key.json | 399 ------ .../data/verification_11_key.json | 399 ------ .../data/verification_12_key.json | 399 ------ .../data/verification_13_key.json | 399 ------ .../data/verification_14_key.json | 399 ------ .../data/verification_15_key.json | 399 ------ .../data/verification_16_key.json | 399 ------ .../data/verification_17_key.json | 399 ------ .../data/verification_18_key.json | 399 ------ .../data/verification_1_key.json | 399 ------ .../data/verification_2_key.json | 399 ------ .../data/verification_3_key.json | 399 ------ .../data/verification_4_key.json | 399 ------ .../data/verification_5_key.json | 399 ------ .../data/verification_6_key.json | 399 ------ .../data/verification_7_key.json | 399 ------ .../data/verification_8_key.json | 399 ------ .../data/verification_9_key.json | 399 ------ .../src/commitment_generator.rs | 37 - .../src/json_to_binary_vk_converter.rs | 31 - .../src/lib.rs | 188 --- .../src/main.rs | 48 - .../src/tests.rs | 68 - core/bin/zksync_server/src/main.rs | 6 +- core/lib/config/src/configs/fri_prover.rs | 1 + core/lib/config/src/configs/mod.rs | 4 - core/lib/config/src/configs/prover.rs | 61 - core/lib/config/src/configs/prover_group.rs | 66 - core/lib/config/src/lib.rs | 2 +- core/lib/dal/sqlx-data.json | 1128 ++--------------- core/lib/dal/src/gpu_prover_queue_dal.rs | 218 ---- core/lib/dal/src/lib.rs | 16 +- core/lib/dal/src/protocol_versions_dal.rs | 104 +- core/lib/dal/src/prover_dal.rs | 724 ----------- core/lib/dal/src/tests/mod.rs | 149 +-- core/lib/env_config/src/fri_prover.rs | 3 + core/lib/env_config/src/lib.rs | 2 - core/lib/env_config/src/prover.rs | 197 --- core/lib/env_config/src/prover_group.rs | 149 --- core/lib/prover_utils/src/region_fetcher.rs | 66 +- core/lib/zksync_core/Cargo.toml | 1 - .../house_keeper/gpu_prover_queue_monitor.rs | 66 - core/lib/zksync_core/src/house_keeper/mod.rs | 3 - .../house_keeper/prover_job_retry_manager.rs | 56 - .../src/house_keeper/prover_queue_monitor.rs | 82 -- .../waiting_to_queued_witness_job_mover.rs | 95 -- .../witness_generator_queue_monitor.rs | 121 -- core/lib/zksync_core/src/lib.rs | 27 - core/lib/zksync_core/src/temp_config_store.rs | 6 +- docker/prover-gar/Dockerfile | 21 - docker/prover/Dockerfile | 63 - docker/server-v2/Dockerfile | 1 - docs/guides/launch.md | 23 - etc/env/base/fri_prover.toml | 1 + etc/env/base/prover.toml | 74 -- etc/env/base/prover_group.toml | 17 - etc/env/base/rust.toml | 3 - infrastructure/zk/src/config.ts | 2 - prover/Cargo.lock | 248 +--- prover/Cargo.toml | 2 - prover/README.md | 12 - prover/prover/Cargo.toml | 57 - prover/prover/README.md | 8 - prover/prover/src/artifact_provider.rs | 25 - prover/prover/src/main.rs | 26 - prover/prover/src/metrics.rs | 41 - prover/prover/src/prover.rs | 286 ----- prover/prover/src/prover_params.rs | 35 - prover/prover/src/run.rs | 255 ---- prover/prover/src/socket_listener.rs | 122 -- .../src/synthesized_circuit_provider.rs | 81 -- prover/prover_fri/src/main.rs | 15 +- .../setup_key_generator_and_server/Cargo.toml | 42 - .../data/.gitkeep | 0 .../setup_key_generator_and_server/src/lib.rs | 63 - .../src/main.rs | 68 - prover/witness_vector_generator/src/main.rs | 9 +- 84 files changed, 154 insertions(+), 12851 deletions(-) delete mode 100644 core/bin/verification_key_generator_and_server/Cargo.toml delete mode 100644 core/bin/verification_key_generator_and_server/README.md delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_0_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_10_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_11_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_12_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_13_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_14_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_15_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_16_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_17_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_18_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_1_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_2_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_3_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_4_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_5_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_6_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_7_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_8_key.json delete mode 100644 core/bin/verification_key_generator_and_server/data/verification_9_key.json delete mode 100644 core/bin/verification_key_generator_and_server/src/commitment_generator.rs delete mode 100644 core/bin/verification_key_generator_and_server/src/json_to_binary_vk_converter.rs delete mode 100644 core/bin/verification_key_generator_and_server/src/lib.rs delete mode 100644 core/bin/verification_key_generator_and_server/src/main.rs delete mode 100644 core/bin/verification_key_generator_and_server/src/tests.rs delete mode 100644 core/lib/config/src/configs/prover.rs delete mode 100644 core/lib/config/src/configs/prover_group.rs delete mode 100644 core/lib/dal/src/gpu_prover_queue_dal.rs delete mode 100644 core/lib/dal/src/prover_dal.rs delete mode 100644 core/lib/env_config/src/prover.rs delete mode 100644 core/lib/env_config/src/prover_group.rs delete mode 100644 core/lib/zksync_core/src/house_keeper/gpu_prover_queue_monitor.rs delete mode 100644 core/lib/zksync_core/src/house_keeper/prover_job_retry_manager.rs delete mode 100644 core/lib/zksync_core/src/house_keeper/prover_queue_monitor.rs delete mode 100644 core/lib/zksync_core/src/house_keeper/waiting_to_queued_witness_job_mover.rs delete mode 100644 core/lib/zksync_core/src/house_keeper/witness_generator_queue_monitor.rs delete mode 100644 docker/prover-gar/Dockerfile delete mode 100644 docker/prover/Dockerfile delete mode 100644 etc/env/base/prover.toml delete mode 100644 etc/env/base/prover_group.toml delete mode 100644 prover/prover/Cargo.toml delete mode 100644 prover/prover/README.md delete mode 100644 prover/prover/src/artifact_provider.rs delete mode 100644 prover/prover/src/main.rs delete mode 100644 prover/prover/src/metrics.rs delete mode 100644 prover/prover/src/prover.rs delete mode 100644 prover/prover/src/prover_params.rs delete mode 100644 prover/prover/src/run.rs delete mode 100644 prover/prover/src/socket_listener.rs delete mode 100644 prover/prover/src/synthesized_circuit_provider.rs delete mode 100644 prover/setup_key_generator_and_server/Cargo.toml delete mode 100644 prover/setup_key_generator_and_server/data/.gitkeep delete mode 100644 prover/setup_key_generator_and_server/src/lib.rs delete mode 100644 prover/setup_key_generator_and_server/src/main.rs diff --git a/.dockerignore b/.dockerignore index 67be353d0bd4..8268fb1049e6 100644 --- a/.dockerignore +++ b/.dockerignore @@ -37,6 +37,5 @@ contracts/.git !etc/multivm_bootloaders !cargo !bellman-cuda -!core/bin/verification_key_generator_and_server/data/ !prover/vk_setup_data_generator_server_fri/data/ !.github/release-please/manifest.json diff --git a/.gitignore b/.gitignore index eff8079e75db..20c5973e8f48 100644 --- a/.gitignore +++ b/.gitignore @@ -27,7 +27,6 @@ todo Cargo.lock !/Cargo.lock -!/core/bin/verification_key_generator_and_server/Cargo.lock !/infrastructure/zksync-crypto/Cargo.lock !/prover/Cargo.lock diff --git a/Cargo.lock b/Cargo.lock index c794cc5fe803..d6f5176bf778 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8508,7 +8508,6 @@ dependencies = [ "zksync_test_account", "zksync_types", "zksync_utils", - "zksync_verification_key_generator_and_server", "zksync_web3_decl", ] @@ -8925,25 +8924,6 @@ dependencies = [ "zksync_basic_types", ] -[[package]] -name = "zksync_verification_key_generator_and_server" -version = "0.1.0" -dependencies = [ - "anyhow", - "bincode", - "circuit_testing", - "ff_ce", - "hex", - "itertools", - "once_cell", - "serde_json", - "structopt", - "tracing", - "vlog", - "zksync_prover_utils", - "zksync_types", -] - [[package]] name = "zksync_web3_decl" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 34115e84bead..151921aef57e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -8,7 +8,6 @@ members = [ "core/bin/snapshots_creator", "core/bin/storage_logs_dedup_migration", "core/bin/system-constants-generator", - "core/bin/verification_key_generator_and_server", "core/bin/verified_sources_fetcher", "core/bin/zksync_server", # Libraries diff --git a/core/bin/verification_key_generator_and_server/Cargo.toml b/core/bin/verification_key_generator_and_server/Cargo.toml deleted file mode 100644 index b49683424a40..000000000000 --- a/core/bin/verification_key_generator_and_server/Cargo.toml +++ /dev/null @@ -1,37 +0,0 @@ -[package] -name = "zksync_verification_key_generator_and_server" -version = "0.1.0" -edition = "2018" -license = "MIT OR Apache-2.0" - -[lib] -name = "zksync_verification_key_server" -path = "src/lib.rs" - -[[bin]] -name = "zksync_verification_key_generator" -path = "src/main.rs" - -[[bin]] -name = "zksync_json_to_binary_vk_converter" -path = "src/json_to_binary_vk_converter.rs" - -[[bin]] -name = "zksync_commitment_generator" -path = "src/commitment_generator.rs" - -[dependencies] -zksync_types = { path = "../../lib/types" } -zksync_prover_utils = { path = "../../lib/prover_utils" } -vlog = { path = "../../lib/vlog" } -circuit_testing = { git = "https://github.com/matter-labs/era-circuit_testing.git", branch = "main" } -itertools = "0.10.5" -bincode = "1.3.3" - -anyhow = "1.0" -serde_json = "1.0.85" -hex = "0.4.3" -structopt = "0.3.26" -ff = { package = "ff_ce", version = "0.14.1" } -once_cell = "1.8.0" -tracing = "0.1" diff --git a/core/bin/verification_key_generator_and_server/README.md b/core/bin/verification_key_generator_and_server/README.md deleted file mode 100644 index efe7d3f99a4e..000000000000 --- a/core/bin/verification_key_generator_and_server/README.md +++ /dev/null @@ -1,39 +0,0 @@ -# Verification keys - -We currently have around 20 different circuits like: Scheduler, Leaf, KeccakPrecompile etc (for the full list - look at -CircuitType enum in sync_vm repo). - -Each such circuit requires a separate verification key. - -This crate fulfills 2 roles: - -- it has the binaries that can generate the updated versions of the keys (for example if VM code changes) -- it provides the libraries that can be used by other components that need to use these keys (for example provers) - - behaving like a key server. - -Moreover, all these keys are submitted as code within the repo in `verification_XX_key.json` files. - -## zksync_verification_key_server - -This is the library that can be used by other components to fetch the verification key for a given circuit (using -`get_vk_for_circuit_type` function). - -## zksync_verification_key_generator - -The main binary that generates verification key for given circuits. Most of the heavy lifting is done by the -`create_vk_for_padding_size_log_2` method from circuit_testing repo. - -The results are written to the `verification_XX_key.json` files in the current repository. - -## zksync_json_to_binary_vk_converter - -Converts the local json verification keys into the binary format (and stores them in the output directory). - -## zksync_commitment_generator - -This tool takes the 3 commitments (one for all the basic circuits, one for node and one for leaf), computed based on the -current verification keys - and updates the contract.toml config file (which is located in etc/env/base/contracts.toml). - -These commitments are later used in one of the circuit breakers - to compare their values to the commitments that L1 -contract holds (so that we can 'panic' properly - if we notice that our server's commitments differ from the L1 -contracts - which would result in failed L1 transactions). diff --git a/core/bin/verification_key_generator_and_server/data/verification_0_key.json b/core/bin/verification_key_generator_and_server/data/verification_0_key.json deleted file mode 100644 index c3262193a4fd..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_0_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 14745348174000482855, - 2839037062185937123, - 3369862715588854899, - 1495909583940713128 - ], - "y": [ - 6859454683840363585, - 11340551061368171664, - 9528805406487149561, - 3414144677220223705 - ], - "infinity": false - }, - { - "x": [ - 9215749870136224396, - 18418669114332753377, - 13140219601461030180, - 2381098845928447331 - ], - "y": [ - 8834765081837029169, - 4424842234296363904, - 13294547557836067005, - 414624398145171890 - ], - "infinity": false - }, - { - "x": [ - 2148575411987453084, - 16730180692461995258, - 12423475767707134837, - 3014264170083149730 - ], - "y": [ - 10870860158804422503, - 14060279526953529989, - 2266257082861680293, - 22356173050560284 - ], - "infinity": false - }, - { - "x": [ - 17803008042411335770, - 5713064950476621403, - 17979342410816871746, - 491265656076548841 - ], - "y": [ - 9823492080506672630, - 3637386621225409615, - 8776978043600973097, - 2514196809208915768 - ], - "infinity": false - }, - { - "x": [ - 3768479078383323179, - 16153057542709544671, - 10578964798085613273, - 2831188075764800753 - ], - "y": [ - 2387514805820590694, - 15085489652142686165, - 8141513931186597223, - 1582376980242699819 - ], - "infinity": false - }, - { - "x": [ - 5395455814671474247, - 5013790368139874617, - 8671649443504728767, - 839142828943885970 - ], - "y": [ - 11231626069154926735, - 5078347962234771017, - 17373886182204596447, - 513647957075879347 - ], - "infinity": false - }, - { - "x": [ - 8940485327950054531, - 9156997542069636576, - 14316753178152000598, - 3357551869664255582 - ], - "y": [ - 14102490706504125272, - 4494991810930729808, - 15532318871086968466, - 1537365238286274178 - ], - "infinity": false - }, - { - "x": [ - 13914906478277300859, - 6213896071228541481, - 4364409818367302306, - 659097390118096039 - ], - "y": [ - 7328372274594390887, - 2650332638498669615, - 15455628473476960005, - 3119379427019958230 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 9438200511694036157, - 11094162170960057340, - 9123678872696723713, - 2950597355117190054 - ], - "y": [ - 6153972960518016517, - 8045683598100955864, - 13410633858416643489, - 988361678931464913 - ], - "infinity": false - }, - { - "x": [ - 805964423710846142, - 13603470797942296854, - 11292123377140077447, - 1455913517812009773 - ], - "y": [ - 4541622738043214385, - 8186357170000535775, - 4765839113294831637, - 3026863977499737494 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 1851039213129741497, - 11907960788190413713, - 2882727828085561070, - 1451278944954982956 - ], - "y": [ - 15245785050592773860, - 1774295027236395480, - 3373069120056880915, - 1080245109458702174 - ], - "infinity": false - }, - { - "x": [ - 9366052859968548005, - 12275028918364559591, - 2472023927159177225, - 1052535074027277666 - ], - "y": [ - 2428574557555628629, - 15067392861858369528, - 16949255188095910778, - 2297925771936569168 - ], - "infinity": false - }, - { - "x": [ - 17016009610362956206, - 4047659663396753591, - 1832464593155416403, - 2725142957049914767 - ], - "y": [ - 12447928856414787240, - 3072280375285720285, - 12294239288643819494, - 613511140380288958 - ], - "infinity": false - }, - { - "x": [ - 6312774245791141720, - 496150993329472460, - 12773767122915456934, - 3404402910494500531 - ], - "y": [ - 13852578578747731084, - 9030931732410275304, - 17159996848865265705, - 1696956882146098553 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 1073530, - "lookup_selector_commitment": { - "x": [ - 4441974708940861232, - 11325614820129407652, - 7273013871150456559, - 2270181644629652201 - ], - "y": [ - 3070631142979677922, - 15247189094202672776, - 12651459662740804392, - 1832216259472686694 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 7312875299592476003, - 313526216906044060, - 13914875394436353152, - 3424388477700656316 - ], - "y": [ - 2572062173996296044, - 5984767625164919974, - 12005537293370417131, - 616463121946800406 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_10_key.json b/core/bin/verification_key_generator_and_server/data/verification_10_key.json deleted file mode 100644 index ec9d3727bff9..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_10_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 4364720487844379181, - 17010766725144227333, - 1022678199111276314, - 1146578362772127376 - ], - "y": [ - 10340654727439455072, - 12691578856596245032, - 837883495763401146, - 2135776887902289239 - ], - "infinity": false - }, - { - "x": [ - 14564870240038241482, - 16001391704613609683, - 16397364612792898214, - 1316914335235774452 - ], - "y": [ - 2386942353392090183, - 4642131766714508143, - 16789479723446408276, - 2261353401184907401 - ], - "infinity": false - }, - { - "x": [ - 6081056006818109026, - 14051483412950926523, - 8605392534710099348, - 1527183574619010123 - ], - "y": [ - 3896696527234063839, - 12862398541231039501, - 1005646628007936886, - 3479645512156004366 - ], - "infinity": false - }, - { - "x": [ - 11266242489999219523, - 8100856016495224488, - 6788749864393617587, - 482299081118345826 - ], - "y": [ - 225211373180020785, - 6498635074385582091, - 4274055525472487569, - 2578651815252093838 - ], - "infinity": false - }, - { - "x": [ - 10378455392293934375, - 13391940670290769236, - 10463014668466536299, - 472544008986099462 - ], - "y": [ - 1502016714118108544, - 14252801754530793876, - 2203844491975584716, - 1116114255465135672 - ], - "infinity": false - }, - { - "x": [ - 9703616742074407567, - 9691703077434834222, - 7366620887865105973, - 36165572355418066 - ], - "y": [ - 7430304832706471782, - 5173267152399523091, - 14416699599905226230, - 2681204653630184824 - ], - "infinity": false - }, - { - "x": [ - 9347312215430913530, - 13606433894103359668, - 14013475178334262360, - 2947181048682744075 - ], - "y": [ - 4001199390012145932, - 4622813642635649819, - 16433672063298879053, - 1247842462976799965 - ], - "infinity": false - }, - { - "x": [ - 1639425503718708209, - 8242804754724970899, - 11043260258533730377, - 2245145560504199089 - ], - "y": [ - 14202551139064230506, - 4307109380979442947, - 13141687433511141087, - 1913204959448290015 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 17540836040216578409, - 14577118461028955096, - 2300935836423716880, - 427649651480863044 - ], - "y": [ - 13066723755606073272, - 17324941433857131282, - 1679499122173566851, - 3298750515604566671 - ], - "infinity": false - }, - { - "x": [ - 14709152157752642079, - 13510549649315108277, - 3019440420162710858, - 627188607991231846 - ], - "y": [ - 16615706301470133997, - 915024441316023047, - 13798541787831785917, - 3340602253234308653 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 12626704863583094704, - 3308474372162220296, - 16088806788444947642, - 636430705662147361 - ], - "y": [ - 17052785040105865748, - 11203395113497209978, - 2939609765212411460, - 3167290643533167611 - ], - "infinity": false - }, - { - "x": [ - 3075146465184418179, - 11559452521956513155, - 1656597085428845901, - 1618447062156730856 - ], - "y": [ - 2010693621773175313, - 2977509893150409878, - 9431891659616951962, - 1776222288355278384 - ], - "infinity": false - }, - { - "x": [ - 6408318860212838666, - 9847136022608767026, - 18080834927350013528, - 3306285138140631107 - ], - "y": [ - 16064928058583899597, - 461689523483649779, - 13572099112445223829, - 1563453638232968523 - ], - "infinity": false - }, - { - "x": [ - 327171445663828020, - 12706053900720413614, - 9237483585964880752, - 1960293149538216528 - ], - "y": [ - 11030775691809003651, - 11089052388657955457, - 3209890793790993499, - 1198867574642866523 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 5202052, - "lookup_selector_commitment": { - "x": [ - 781239045644769777, - 14316527640474633593, - 2443643435827373112, - 3049372365263474427 - ], - "y": [ - 4073012743593667819, - 16009537994875540924, - 11173412503242869179, - 1513208421597995174 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 697552212563769686, - 7709943502535418760, - 15019345407325619175, - 3433081085078580257 - ], - "y": [ - 8668947019840357731, - 14698901351824712883, - 15088598879190660424, - 2873081208166433946 - ], - "infinity": false - }, - { - "x": [ - 7893133928909060673, - 7064922516930129957, - 3592836702741304814, - 2239702595710114437 - ], - "y": [ - 7691360541875191519, - 11379321785127235277, - 6653616064071569031, - 2555434628517540774 - ], - "infinity": false - }, - { - "x": [ - 6243944238013052821, - 7908243182210136125, - 17178099109525791299, - 2553622184721264566 - ], - "y": [ - 736121280088239428, - 6158073429758170526, - 11217302997977204117, - 2594798912020899417 - ], - "infinity": false - }, - { - "x": [ - 2064240298596094591, - 16917726764104887991, - 11042784977532408536, - 3377647228930170830 - ], - "y": [ - 10635525052494768819, - 387400048616497096, - 9379200582543310995, - 1571766153703296253 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 7603211811706190713, - 2486982239745271096, - 11528266448545919500, - 3080741880407152411 - ], - "y": [ - 7967754771633653173, - 6016822892450616749, - 9688696792558711613, - 2682562048141398047 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_11_key.json b/core/bin/verification_key_generator_and_server/data/verification_11_key.json deleted file mode 100644 index ec60b1b5c70c..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_11_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 6404793958941109752, - 600086648940026770, - 17621036346050218167, - 648286585825030202 - ], - "y": [ - 15536368541166505022, - 13874331483468128999, - 15299774519724050181, - 694528839710637549 - ], - "infinity": false - }, - { - "x": [ - 8437895530551083583, - 9515418928119648176, - 13043255827139294721, - 2995712510038409810 - ], - "y": [ - 2599666661350767554, - 5213004864468121936, - 3448071048439343925, - 3372727479169634860 - ], - "infinity": false - }, - { - "x": [ - 4949545806128010634, - 7991544258837652527, - 13984289231122041826, - 435264553263929947 - ], - "y": [ - 5315155210033461895, - 5269954775753247626, - 8365554241810378947, - 3038338810517586456 - ], - "infinity": false - }, - { - "x": [ - 10765735847634894938, - 996016141851615448, - 17905928073714218280, - 1382306444325686451 - ], - "y": [ - 2138154197587423296, - 10332772886666867909, - 18365120064743353477, - 3036329558617382049 - ], - "infinity": false - }, - { - "x": [ - 10826908009799408310, - 17008417534705779156, - 6763973494549063072, - 2085829964414931488 - ], - "y": [ - 8778528796073273991, - 3575354418973385595, - 7700555759899743641, - 2991788183234680231 - ], - "infinity": false - }, - { - "x": [ - 4838537981048085423, - 17733460364049897496, - 2406410363431464143, - 317979983533551325 - ], - "y": [ - 1063783130085451648, - 17468950496650586998, - 1638492556781126884, - 2655791721465286744 - ], - "infinity": false - }, - { - "x": [ - 9900079822056413611, - 2971494295919434281, - 3851188096409515874, - 1674965457600938162 - ], - "y": [ - 278026997091552202, - 4169606578927284200, - 4285297176993939496, - 1835673146863992148 - ], - "infinity": false - }, - { - "x": [ - 14972922803706426724, - 1950002897609593521, - 14885502244328862256, - 2711533695106895845 - ], - "y": [ - 6445273103061253271, - 13093783937225622775, - 16913300898726970338, - 3338984185497324237 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 7023363902839996761, - 10470701207992157969, - 15655647820064667897, - 1574806151825297776 - ], - "y": [ - 5374465760860613169, - 17808737811039085287, - 9497881147171478776, - 2496973717640690197 - ], - "infinity": false - }, - { - "x": [ - 11667333913021610767, - 981513539224109240, - 906325130343873228, - 2938085706999497365 - ], - "y": [ - 12114685726509803851, - 8176447551157079615, - 4677211732718215770, - 612959750791398009 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 5178916486603003859, - 12440762249350081718, - 17531240512375127539, - 562979322442547791 - ], - "y": [ - 13269831614205338393, - 14075713698585784838, - 5009519510530479124, - 346033861980045408 - ], - "infinity": false - }, - { - "x": [ - 9815443577325313677, - 10727907015331332054, - 7582395371050260833, - 1746872659838481572 - ], - "y": [ - 3973552805135639320, - 14426732004648741961, - 8133164322153358522, - 2668541869556858228 - ], - "infinity": false - }, - { - "x": [ - 4868257934818957423, - 11529848268525929099, - 7089666284160764141, - 796901367628793969 - ], - "y": [ - 991195814042705325, - 1559922382138761102, - 15616159453482282503, - 1031107741111093289 - ], - "infinity": false - }, - { - "x": [ - 17936772813090339705, - 10208762457499980701, - 14796710996322725970, - 638550977107438851 - ], - "y": [ - 5073905611192321777, - 2956648407808816974, - 7778989780119416172, - 2955106321082932072 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 7960377, - "lookup_selector_commitment": { - "x": [ - 1083743271968869166, - 3134203175755215736, - 5835502497758804469, - 3010956977291777466 - ], - "y": [ - 3645612220088813035, - 32844736552579976, - 5426466326302260857, - 1489565191618899261 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 5825422128268478267, - 9219263846299851036, - 3879231702557190566, - 1702488722758880769 - ], - "y": [ - 18311881100262470992, - 5742998199368802392, - 18106865487471159417, - 502191980176920012 - ], - "infinity": false - }, - { - "x": [ - 17195892082859417081, - 7890531942603584793, - 2381805632820057528, - 3173232410464566465 - ], - "y": [ - 16359614627947132075, - 3459600273035137079, - 4550762061432972122, - 3394559699318358224 - ], - "infinity": false - }, - { - "x": [ - 1716103379277390185, - 18097936269579187542, - 16357329729761063450, - 1508640059338197502 - ], - "y": [ - 11014806739603983364, - 4396503314588777389, - 9397245609635151055, - 1703957955248411380 - ], - "infinity": false - }, - { - "x": [ - 4770171350693477354, - 17110558673192292253, - 9799800677557311408, - 761984875463445481 - ], - "y": [ - 1560561403388310063, - 31331275310848146, - 287152055803835484, - 457826332542037277 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 11327495732840772606, - 7407664417001729515, - 9486600059857658309, - 3060296564241189838 - ], - "y": [ - 7624492872489320847, - 18248981556039704277, - 3877205757853252152, - 939885486002612376 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_12_key.json b/core/bin/verification_key_generator_and_server/data/verification_12_key.json deleted file mode 100644 index fec076f39eda..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_12_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 456514006020943025, - 9595480195714948127, - 12254096252487404245, - 1742692690750856358 - ], - "y": [ - 16294223586064957217, - 3958270970168887906, - 11264067544872898258, - 1692817687935973108 - ], - "infinity": false - }, - { - "x": [ - 1359655052308122459, - 13840124148496555776, - 1774237333490664500, - 2964872651584750318 - ], - "y": [ - 11907598503482948769, - 8700506041798646988, - 15081040576888859990, - 3096802642049924528 - ], - "infinity": false - }, - { - "x": [ - 2884314851670818573, - 13442465544210396156, - 5937955495868181363, - 2486997439179977778 - ], - "y": [ - 9309776793338098458, - 14492906371677122697, - 8837309186596588911, - 1081143755093508499 - ], - "infinity": false - }, - { - "x": [ - 2655654413304275855, - 4244723109566147837, - 12150359360501203194, - 3338981627918702615 - ], - "y": [ - 2522870072161287404, - 17341373219317210182, - 13058930363994599297, - 210373422168410518 - ], - "infinity": false - }, - { - "x": [ - 16728834675380740056, - 2139390496020366235, - 9480389182940223467, - 2279560291896695719 - ], - "y": [ - 12461418813218976432, - 357566005384566098, - 5295578385080568808, - 1801243085576438875 - ], - "infinity": false - }, - { - "x": [ - 8716201428771436123, - 3392394702404760386, - 9990956922582058945, - 1388317411153212399 - ], - "y": [ - 11666415392681680155, - 10553517485129490455, - 16061047708722635939, - 2386622646140901822 - ], - "infinity": false - }, - { - "x": [ - 16162432560623854812, - 15537581062716888632, - 12927223782958923606, - 2800634589869451227 - ], - "y": [ - 5345141365329635916, - 2224393250977631865, - 396527108738048188, - 2298318725146167177 - ], - "infinity": false - }, - { - "x": [ - 18372685954785469756, - 10436523365152935441, - 15509622927999798123, - 2050428620045833325 - ], - "y": [ - 4996265985148335658, - 6073112270434155721, - 4873288683270752338, - 503179567393027927 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 4986139828502830074, - 8644425445976253042, - 4851433922656693398, - 1419574698085640872 - ], - "y": [ - 16192186537521161947, - 16183885683582261905, - 1655718756619164666, - 3420236094426390604 - ], - "infinity": false - }, - { - "x": [ - 10727231722644915889, - 13777116005624794169, - 1422623412369619026, - 1701279717637612575 - ], - "y": [ - 6503647097427010249, - 6381043883853023011, - 15391366286376907281, - 1261207976874708261 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 11852073725466955067, - 179170887563176222, - 17529899074897279348, - 2496783194148289461 - ], - "y": [ - 15490041181991978284, - 6745436372504113852, - 7017978386715410058, - 3482556315200370895 - ], - "infinity": false - }, - { - "x": [ - 1330152738947291505, - 1668990644246591877, - 6805443255260621096, - 1309987766073890626 - ], - "y": [ - 18322300356676620444, - 8225233874302527542, - 5744327785164342590, - 410571567010522636 - ], - "infinity": false - }, - { - "x": [ - 13968210937929584911, - 17067601391996082961, - 4861463652254416951, - 2147834012714370408 - ], - "y": [ - 9012483356698219484, - 8660929519763525826, - 17744882010750642463, - 331423342438323189 - ], - "infinity": false - }, - { - "x": [ - 1352282553299127274, - 8587971715415488300, - 2471024479841756772, - 1239586065229072559 - ], - "y": [ - 1597792022909153930, - 5020991346876715357, - 5622801511814109910, - 1916460940163680567 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 46287674, - "lookup_selector_commitment": { - "x": [ - 11573469000684493293, - 15304040816406013002, - 9206902553183544808, - 2597693769113957036 - ], - "y": [ - 10538181061926273477, - 5239567589495426242, - 3627181047901924882, - 302644994241575377 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 5134795695995115566, - 12287750992060803275, - 3112021177339560487, - 2737779104829043419 - ], - "y": [ - 12960786984497012138, - 17246059378047870426, - 11486754204718893642, - 46104506716724806 - ], - "infinity": false - }, - { - "x": [ - 148472607159578301, - 1393814398025790148, - 13651878286378332448, - 3460878321325997474 - ], - "y": [ - 10791022888598424744, - 1931353219232076143, - 12342018346439101174, - 23632989633122111 - ], - "infinity": false - }, - { - "x": [ - 1355031833403957875, - 10754997913401276231, - 8672292473740482178, - 3014145653612856517 - ], - "y": [ - 3728402825933673134, - 16492594359417243041, - 14619929139939206930, - 2894280666048705144 - ], - "infinity": false - }, - { - "x": [ - 11362104917939269301, - 3050269804312222606, - 17884269955997757593, - 2804911625130359365 - ], - "y": [ - 9563576475625880180, - 9736108320914226650, - 11545696954602328389, - 1108440262014676246 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 5367643753678334453, - 18149093736372716410, - 1335188566370936146, - 668596617655217713 - ], - "y": [ - 9984652217894703540, - 16253861114794085212, - 2139268495406835151, - 710303505771002735 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_13_key.json b/core/bin/verification_key_generator_and_server/data/verification_13_key.json deleted file mode 100644 index 73ffbd212002..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_13_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 17551054392858982554, - 6093238351564742844, - 9461983640740135929, - 665917981733823732 - ], - "y": [ - 5039211542045701927, - 14102316155129161178, - 7599318237652648682, - 1484263542771007309 - ], - "infinity": false - }, - { - "x": [ - 14015566113565304739, - 12895182424777444911, - 5150482782915031712, - 3280776276671330755 - ], - "y": [ - 5503211683737487414, - 5857977821275887356, - 1294122171191120577, - 2917900236095606783 - ], - "infinity": false - }, - { - "x": [ - 11180353512945796758, - 5467792637578213396, - 14862660111090994534, - 1678570344676416345 - ], - "y": [ - 16496106534540891926, - 4355829424666415263, - 8379906815867503783, - 2141225531456729878 - ], - "infinity": false - }, - { - "x": [ - 10512618919562577175, - 8909238001556772501, - 8669074760108324520, - 3259590816167766101 - ], - "y": [ - 15477336671232249792, - 10209451912771766896, - 13672268903388741173, - 682487251336397201 - ], - "infinity": false - }, - { - "x": [ - 14233534177298597555, - 14428793231398751908, - 18070433438826750034, - 1176819688107481869 - ], - "y": [ - 9251234182098356520, - 17131606126090989402, - 17185633762130361526, - 70013401388751862 - ], - "infinity": false - }, - { - "x": [ - 14148566925658671094, - 812517577375883951, - 5030512299767107864, - 44275794325016754 - ], - "y": [ - 3275438385460491589, - 12366768737850140720, - 10754478223029148744, - 64366431004577735 - ], - "infinity": false - }, - { - "x": [ - 5646513434714516506, - 12578668031398681290, - 6956692825033783810, - 536471110695536326 - ], - "y": [ - 876079378616587621, - 9787032999740439668, - 14965634813605966164, - 367083452910738472 - ], - "infinity": false - }, - { - "x": [ - 10902302115259229513, - 14044271471332330954, - 14571826360674828773, - 733766328575554031 - ], - "y": [ - 8186695183963076514, - 621472878958955881, - 14756382569165412398, - 3165780226323675661 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 17780673306296332984, - 10355922416617009060, - 5077451999006954761, - 2644291606399153501 - ], - "y": [ - 884498752701137122, - 731399349168706916, - 4286165746592754883, - 3279732117855760703 - ], - "infinity": false - }, - { - "x": [ - 11012802284910829398, - 7859388231941271159, - 17586341808458361180, - 1386364899721133297 - ], - "y": [ - 15634369655108108777, - 3858480397682251762, - 17706291110507066608, - 1663421415693803071 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 18134041530736321349, - 4345724579806003155, - 2324407857452293002, - 2319164124977213120 - ], - "y": [ - 14302129084811449335, - 8588677756442252515, - 3323846949783670865, - 2109729211841784387 - ], - "infinity": false - }, - { - "x": [ - 14486843004985564085, - 10799247040254992370, - 7658639806933647132, - 2215292564171027727 - ], - "y": [ - 14258341133968554193, - 11685656973533320944, - 14111972937744219524, - 1172604679688980794 - ], - "infinity": false - }, - { - "x": [ - 12872375111956991701, - 14049784009914403066, - 15325016171856456312, - 2811875539960405333 - ], - "y": [ - 5711194902040443430, - 13827091592207472460, - 17950028361571343192, - 1672758585097311581 - ], - "infinity": false - }, - { - "x": [ - 11717525586585736911, - 730672019767199816, - 3010255132348992613, - 2780587454575324896 - ], - "y": [ - 1473124157542628664, - 1573646910034288561, - 10026766074599473146, - 563223750818543582 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 42547753, - "lookup_selector_commitment": { - "x": [ - 4539928924349895484, - 2792770915461027618, - 11611697420465472575, - 1384307956752801018 - ], - "y": [ - 8840366360901511807, - 8892919985613263102, - 11941090149541110830, - 1930352681887390920 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 4121704254446914578, - 13863658665929861884, - 15362282368839162345, - 2762703036966024619 - ], - "y": [ - 102846692212239082, - 14904466746900448136, - 16872429770359000841, - 1687152581020907098 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_14_key.json b/core/bin/verification_key_generator_and_server/data/verification_14_key.json deleted file mode 100644 index e8c42d407e35..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_14_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 6916434521451934576, - 614815553772638285, - 3742595993843812033, - 2823214088432624432 - ], - "y": [ - 11642815096362884283, - 18063950820723921281, - 6353943092001719992, - 3201898419478369298 - ], - "infinity": false - }, - { - "x": [ - 10647237757917239762, - 1269177049592707998, - 2650053775033150725, - 582198744757304104 - ], - "y": [ - 9804667267596536998, - 493663115027956828, - 13953159385227792767, - 1568248765042207679 - ], - "infinity": false - }, - { - "x": [ - 7910659438561833906, - 12456422925439856914, - 10869604528749370003, - 1213616301038416610 - ], - "y": [ - 2606202790862698157, - 6809934263763206210, - 17472080335242458272, - 2884639755368519501 - ], - "infinity": false - }, - { - "x": [ - 14211325859682683183, - 11018598407116786751, - 10064425366978091674, - 2748595948091261209 - ], - "y": [ - 13960202853590116423, - 1211975538022172568, - 16303435518817750320, - 1634234707214097860 - ], - "infinity": false - }, - { - "x": [ - 4528591178982443847, - 16310104707629911601, - 5532120103079323919, - 1347877820087040669 - ], - "y": [ - 17983603511717948746, - 9529659424488112452, - 7820918413906679254, - 1819855238351369466 - ], - "infinity": false - }, - { - "x": [ - 14415562798118912210, - 6550719056383417327, - 424281724891761932, - 1264340531903932141 - ], - "y": [ - 7768057951329404686, - 15024442753889769568, - 9676935351692818899, - 1492251668690310932 - ], - "infinity": false - }, - { - "x": [ - 2619366878850208112, - 12150914745315976156, - 8375197026043390274, - 1935272977563031501 - ], - "y": [ - 5381369692389055354, - 17978011500330472972, - 17420193441326928998, - 479187691463910357 - ], - "infinity": false - }, - { - "x": [ - 8720830951139717797, - 15985700059986022675, - 11876530273787337931, - 421322430672290976 - ], - "y": [ - 9700690437922183179, - 1976785701667862157, - 16634886936358874061, - 3002178567925406588 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 8284083154661042764, - 11776500066398184343, - 868620904897679124, - 2988582549909766892 - ], - "y": [ - 10794129605563176627, - 15487634480061313925, - 17194646451372113884, - 2087686927573540537 - ], - "infinity": false - }, - { - "x": [ - 7916190330285050096, - 11731220788334102406, - 6221883233572429550, - 2552280229203107267 - ], - "y": [ - 10510502959728300366, - 14682539966609739595, - 8275243146917870162, - 164811532254637923 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 195850038587200624, - 10136289160450054078, - 4386512701252721226, - 219366815902177323 - ], - "y": [ - 12042545079209848932, - 599057886584676736, - 14545610403811537682, - 498958995843318019 - ], - "infinity": false - }, - { - "x": [ - 4721932753701441297, - 1676671918244393403, - 6943597542294442696, - 50994782040503038 - ], - "y": [ - 8321420884695240511, - 10606883887907326697, - 11471075822795411018, - 1311422627151559437 - ], - "infinity": false - }, - { - "x": [ - 85448132386017640, - 13016912343020112485, - 11647418800345296605, - 1741562939125330787 - ], - "y": [ - 10753835454658443286, - 8646325836340244979, - 7348777908140142985, - 2196062626460604424 - ], - "infinity": false - }, - { - "x": [ - 2125624295892265840, - 12754141819506101591, - 8789168208880604752, - 947087620272222934 - ], - "y": [ - 12566258871261234263, - 12307504590191426495, - 6700589767183706452, - 1828704371386663334 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 42212029, - "lookup_selector_commitment": { - "x": [ - 7709849601046260359, - 6836713108454667472, - 17360769186231334246, - 2348971634881039863 - ], - "y": [ - 13380830060569421804, - 15446653016734774164, - 17884501636917484387, - 1386904567459265970 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 6960699536013090594, - 2075384204892265266, - 12053931571725248687, - 1371193846897305849 - ], - "y": [ - 8904850119058507432, - 10465598889525773001, - 16159541505228012497, - 1982452464017823539 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_15_key.json b/core/bin/verification_key_generator_and_server/data/verification_15_key.json deleted file mode 100644 index 356dbb3c531a..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_15_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 3227382513538635502, - 10189582412003011525, - 1928710987967879299, - 1641062823248805930 - ], - "y": [ - 3271795224553087841, - 14036363906521936156, - 10253705337161624780, - 3091191233208402889 - ], - "infinity": false - }, - { - "x": [ - 3541471743181642086, - 8117051273006688414, - 685909872467163024, - 2614724468827209722 - ], - "y": [ - 1096952120887201428, - 8197980407203032569, - 3949713006885563085, - 2838982585728277197 - ], - "infinity": false - }, - { - "x": [ - 12432945880074879560, - 13444859845042471186, - 16599097070979057001, - 3064039790213026567 - ], - "y": [ - 3745088406100356357, - 11715355314289478148, - 2282946417129489745, - 1619614407449915711 - ], - "infinity": false - }, - { - "x": [ - 6864310053920223866, - 11095455024311706186, - 12229748247000682102, - 2475016349586561501 - ], - "y": [ - 2946781066962542712, - 14275500021265062654, - 7624481756022778467, - 1439658776940615826 - ], - "infinity": false - }, - { - "x": [ - 13589273139905087785, - 10411035015021574213, - 7322465558208873130, - 1805943743448229826 - ], - "y": [ - 13035238946064559886, - 8309482746549063820, - 14229757515324464781, - 1676135665275665956 - ], - "infinity": false - }, - { - "x": [ - 84006308859404982, - 13783127238980064918, - 14101945786439708601, - 3343881426944938693 - ], - "y": [ - 11959320721291234482, - 7288504259378326725, - 9638777183731403514, - 1648453409181088010 - ], - "infinity": false - }, - { - "x": [ - 10987163680360734145, - 3374907765066907489, - 14421201974855570464, - 3148542489906320493 - ], - "y": [ - 17180031485000081847, - 1609372527008367113, - 6050341427989573858, - 477684541505306009 - ], - "infinity": false - }, - { - "x": [ - 2257028353691713628, - 6330174784373016532, - 1686021628649718039, - 2159927805963705967 - ], - "y": [ - 10814125155819336479, - 9673780307204445954, - 7995606758095566598, - 2252251279727988680 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 12209724104183572477, - 11631007075974892904, - 18407423517909669447, - 1123848354500646471 - ], - "y": [ - 4749227851055533192, - 16918951234067984229, - 5345146076707243019, - 2836719468222132526 - ], - "infinity": false - }, - { - "x": [ - 7250866110466496804, - 16022969863388101391, - 16334300930347324147, - 2232272485807431638 - ], - "y": [ - 257675104580526310, - 8044331403028603186, - 2070174268860891010, - 412313474208091695 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 6736882681315025594, - 13400430183084617843, - 17182588928882896917, - 413858188107207402 - ], - "y": [ - 11944170108613027081, - 10598841640624895850, - 9086311820289524704, - 994240611047161478 - ], - "infinity": false - }, - { - "x": [ - 9500318283622871785, - 5480449932874899465, - 13224510306395939252, - 1891329668301281157 - ], - "y": [ - 7314078756040350933, - 1023294602177498218, - 16475078688698425911, - 1793945182112302214 - ], - "infinity": false - }, - { - "x": [ - 17207548058425781429, - 2519222249126358251, - 16087595361924038018, - 3470846273906312296 - ], - "y": [ - 7578361094884620755, - 7082109151721400218, - 13675372677342046523, - 3204472226310685459 - ], - "infinity": false - }, - { - "x": [ - 7036282717341939568, - 3035419720331773758, - 6765191455902729185, - 1301973211946290083 - ], - "y": [ - 697377419426635450, - 14612037890797520515, - 11746079616766057625, - 1031190413179598818 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 6391155, - "lookup_selector_commitment": { - "x": [ - 17111915492430945419, - 17971275185478677346, - 14211391044159602918, - 2381455978713737016 - ], - "y": [ - 13971515893527127207, - 7078722574057096191, - 6337080743811431820, - 757015217034494132 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 5825422128268478267, - 9219263846299851036, - 3879231702557190566, - 1702488722758880769 - ], - "y": [ - 18311881100262470992, - 5742998199368802392, - 18106865487471159417, - 502191980176920012 - ], - "infinity": false - }, - { - "x": [ - 17195892082859417081, - 7890531942603584793, - 2381805632820057528, - 3173232410464566465 - ], - "y": [ - 16359614627947132075, - 3459600273035137079, - 4550762061432972122, - 3394559699318358224 - ], - "infinity": false - }, - { - "x": [ - 1716103379277390185, - 18097936269579187542, - 16357329729761063450, - 1508640059338197502 - ], - "y": [ - 11014806739603983364, - 4396503314588777389, - 9397245609635151055, - 1703957955248411380 - ], - "infinity": false - }, - { - "x": [ - 4770171350693477354, - 17110558673192292253, - 9799800677557311408, - 761984875463445481 - ], - "y": [ - 1560561403388310063, - 31331275310848146, - 287152055803835484, - 457826332542037277 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 12452920133699897102, - 6896642231513345496, - 4655495116895575043, - 1453525729114564853 - ], - "y": [ - 3574087764464303986, - 10141819911397868785, - 2342639320036978232, - 556196027732983028 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_16_key.json b/core/bin/verification_key_generator_and_server/data/verification_16_key.json deleted file mode 100644 index 356dbb3c531a..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_16_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 3227382513538635502, - 10189582412003011525, - 1928710987967879299, - 1641062823248805930 - ], - "y": [ - 3271795224553087841, - 14036363906521936156, - 10253705337161624780, - 3091191233208402889 - ], - "infinity": false - }, - { - "x": [ - 3541471743181642086, - 8117051273006688414, - 685909872467163024, - 2614724468827209722 - ], - "y": [ - 1096952120887201428, - 8197980407203032569, - 3949713006885563085, - 2838982585728277197 - ], - "infinity": false - }, - { - "x": [ - 12432945880074879560, - 13444859845042471186, - 16599097070979057001, - 3064039790213026567 - ], - "y": [ - 3745088406100356357, - 11715355314289478148, - 2282946417129489745, - 1619614407449915711 - ], - "infinity": false - }, - { - "x": [ - 6864310053920223866, - 11095455024311706186, - 12229748247000682102, - 2475016349586561501 - ], - "y": [ - 2946781066962542712, - 14275500021265062654, - 7624481756022778467, - 1439658776940615826 - ], - "infinity": false - }, - { - "x": [ - 13589273139905087785, - 10411035015021574213, - 7322465558208873130, - 1805943743448229826 - ], - "y": [ - 13035238946064559886, - 8309482746549063820, - 14229757515324464781, - 1676135665275665956 - ], - "infinity": false - }, - { - "x": [ - 84006308859404982, - 13783127238980064918, - 14101945786439708601, - 3343881426944938693 - ], - "y": [ - 11959320721291234482, - 7288504259378326725, - 9638777183731403514, - 1648453409181088010 - ], - "infinity": false - }, - { - "x": [ - 10987163680360734145, - 3374907765066907489, - 14421201974855570464, - 3148542489906320493 - ], - "y": [ - 17180031485000081847, - 1609372527008367113, - 6050341427989573858, - 477684541505306009 - ], - "infinity": false - }, - { - "x": [ - 2257028353691713628, - 6330174784373016532, - 1686021628649718039, - 2159927805963705967 - ], - "y": [ - 10814125155819336479, - 9673780307204445954, - 7995606758095566598, - 2252251279727988680 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 12209724104183572477, - 11631007075974892904, - 18407423517909669447, - 1123848354500646471 - ], - "y": [ - 4749227851055533192, - 16918951234067984229, - 5345146076707243019, - 2836719468222132526 - ], - "infinity": false - }, - { - "x": [ - 7250866110466496804, - 16022969863388101391, - 16334300930347324147, - 2232272485807431638 - ], - "y": [ - 257675104580526310, - 8044331403028603186, - 2070174268860891010, - 412313474208091695 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 6736882681315025594, - 13400430183084617843, - 17182588928882896917, - 413858188107207402 - ], - "y": [ - 11944170108613027081, - 10598841640624895850, - 9086311820289524704, - 994240611047161478 - ], - "infinity": false - }, - { - "x": [ - 9500318283622871785, - 5480449932874899465, - 13224510306395939252, - 1891329668301281157 - ], - "y": [ - 7314078756040350933, - 1023294602177498218, - 16475078688698425911, - 1793945182112302214 - ], - "infinity": false - }, - { - "x": [ - 17207548058425781429, - 2519222249126358251, - 16087595361924038018, - 3470846273906312296 - ], - "y": [ - 7578361094884620755, - 7082109151721400218, - 13675372677342046523, - 3204472226310685459 - ], - "infinity": false - }, - { - "x": [ - 7036282717341939568, - 3035419720331773758, - 6765191455902729185, - 1301973211946290083 - ], - "y": [ - 697377419426635450, - 14612037890797520515, - 11746079616766057625, - 1031190413179598818 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 6391155, - "lookup_selector_commitment": { - "x": [ - 17111915492430945419, - 17971275185478677346, - 14211391044159602918, - 2381455978713737016 - ], - "y": [ - 13971515893527127207, - 7078722574057096191, - 6337080743811431820, - 757015217034494132 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 5825422128268478267, - 9219263846299851036, - 3879231702557190566, - 1702488722758880769 - ], - "y": [ - 18311881100262470992, - 5742998199368802392, - 18106865487471159417, - 502191980176920012 - ], - "infinity": false - }, - { - "x": [ - 17195892082859417081, - 7890531942603584793, - 2381805632820057528, - 3173232410464566465 - ], - "y": [ - 16359614627947132075, - 3459600273035137079, - 4550762061432972122, - 3394559699318358224 - ], - "infinity": false - }, - { - "x": [ - 1716103379277390185, - 18097936269579187542, - 16357329729761063450, - 1508640059338197502 - ], - "y": [ - 11014806739603983364, - 4396503314588777389, - 9397245609635151055, - 1703957955248411380 - ], - "infinity": false - }, - { - "x": [ - 4770171350693477354, - 17110558673192292253, - 9799800677557311408, - 761984875463445481 - ], - "y": [ - 1560561403388310063, - 31331275310848146, - 287152055803835484, - 457826332542037277 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 12452920133699897102, - 6896642231513345496, - 4655495116895575043, - 1453525729114564853 - ], - "y": [ - 3574087764464303986, - 10141819911397868785, - 2342639320036978232, - 556196027732983028 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_17_key.json b/core/bin/verification_key_generator_and_server/data/verification_17_key.json deleted file mode 100644 index 4886f501712e..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_17_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 17914331890341023175, - 5200903915088916638, - 7417971632353510341, - 989671567770015891 - ], - "y": [ - 2927207345798721401, - 12686845373576710402, - 977520799157489114, - 1882223742569339495 - ], - "infinity": false - }, - { - "x": [ - 17162848902278956536, - 16169550484471334725, - 10830640611178609260, - 1347016616567630867 - ], - "y": [ - 6224316231648682710, - 10518372790293065661, - 4887066336660303630, - 703109868065750569 - ], - "infinity": false - }, - { - "x": [ - 15783141083967762454, - 16153855592853073081, - 5667838393811413602, - 1552498518850981979 - ], - "y": [ - 4220445586486275972, - 13196202402039716924, - 17506868028821343237, - 2718319833724164541 - ], - "infinity": false - }, - { - "x": [ - 4896615254637588846, - 5804270398165250639, - 10274952983674590649, - 1937027782721476561 - ], - "y": [ - 14180244016629518742, - 1376497406583367686, - 11268467489552574214, - 2331396669725958189 - ], - "infinity": false - }, - { - "x": [ - 191294939748295885, - 2804205121966814820, - 3897841028303648224, - 3406986167359695085 - ], - "y": [ - 6000542982074572633, - 1697448874567677325, - 10313504031977824294, - 320347014349001728 - ], - "infinity": false - }, - { - "x": [ - 6817435454105168413, - 15823888625999007373, - 9766931118761036330, - 3392959293697897728 - ], - "y": [ - 3549039265311512008, - 4758653036115592629, - 219467419355603781, - 83059544477934848 - ], - "infinity": false - }, - { - "x": [ - 5038171725639341807, - 6859992384823395611, - 15284967171349293554, - 16807092603996758 - ], - "y": [ - 16504201956683368367, - 12931995037356002803, - 16812826192957092842, - 3169839139097845275 - ], - "infinity": false - }, - { - "x": [ - 7140480682142203727, - 9518528852331365100, - 6189914959408603471, - 535939568308325781 - ], - "y": [ - 5944679084532939174, - 17280810090456322382, - 3743919877743496107, - 1235924204609568068 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 1929812895882850703, - 10386198218814398503, - 17007521659662498274, - 1093092717342753672 - ], - "y": [ - 14834187133095267171, - 15506032964234961178, - 7626816120460943443, - 871778379365004315 - ], - "infinity": false - }, - { - "x": [ - 15660406110329165813, - 8146521122567923995, - 2421739551937359002, - 3037598346026174089 - ], - "y": [ - 526124545966722472, - 1168331442853419483, - 4128095883471549051, - 2951909971734725955 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 6206240620508019400, - 3690935139087147193, - 15230272164329216928, - 2140680869789406894 - ], - "y": [ - 14967331981004447304, - 1624146052760537503, - 8986435052862626311, - 334011853307313390 - ], - "infinity": false - }, - { - "x": [ - 4342223064246074020, - 2037946044543710684, - 9057698479075332373, - 1955362957846693345 - ], - "y": [ - 13253375713250043938, - 6754658208742468331, - 9339617748652368850, - 3066524060291544175 - ], - "infinity": false - }, - { - "x": [ - 17765629723696241082, - 14243015821582305127, - 922013493526048847, - 186830516636733479 - ], - "y": [ - 14465184942185208224, - 11235596895177038197, - 5490682932088517686, - 1253279069662324930 - ], - "infinity": false - }, - { - "x": [ - 9369367805867402420, - 12663806522952881709, - 10184609326459106945, - 1664572000409921348 - ], - "y": [ - 4383960972942823390, - 6526609131568596717, - 1343118583674917141, - 113408414321095416 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 6306340, - "lookup_selector_commitment": { - "x": [ - 8662938005624859815, - 9126108646717466191, - 14321121874090966307, - 2777446762308933634 - ], - "y": [ - 12555265159079607081, - 9054928862248682392, - 2784170007581120117, - 1769718192676345815 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 12644448349947379666, - 16345179309557779118, - 10854030671875297787, - 1358228639202695992 - ], - "y": [ - 2673142241557152443, - 11674634738064487673, - 12992693662201776412, - 1888958170754620568 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_18_key.json b/core/bin/verification_key_generator_and_server/data/verification_18_key.json deleted file mode 100644 index 0987039dd1fa..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_18_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 8828437332483635107, - 13777915698231175292, - 11504510351588004199, - 2516385517175522236 - ], - "y": [ - 1530453459325046685, - 2126477283125660971, - 6874073688275717548, - 2971751478402184988 - ], - "infinity": false - }, - { - "x": [ - 3490885152333630169, - 4123320877294819459, - 5138828731030738163, - 3039569146695764058 - ], - "y": [ - 10725322881860790776, - 1512262420257872325, - 10563843054743673205, - 447776577449487981 - ], - "infinity": false - }, - { - "x": [ - 14957646468235752771, - 6216555943494703122, - 7827110015048654177, - 2702223139144227095 - ], - "y": [ - 505353369980003046, - 9687811614109626117, - 5346740791392836415, - 1340467989233731971 - ], - "infinity": false - }, - { - "x": [ - 3201028595190213325, - 9659059230246338206, - 901122635500995415, - 765851963674764103 - ], - "y": [ - 10609226610841230792, - 8145519080052709505, - 17851750066177581293, - 362176586681460505 - ], - "infinity": false - }, - { - "x": [ - 13374935211181268625, - 1347742735582506393, - 4588995338963087243, - 94453217016201562 - ], - "y": [ - 4077548225372117006, - 11859845367084549583, - 2736752177668563039, - 1134818940315684409 - ], - "infinity": false - }, - { - "x": [ - 9467178015658262369, - 10545965721679492606, - 5726831550010619228, - 2051827871593168334 - ], - "y": [ - 6169140154733194545, - 5574043976386236933, - 12140759986363309479, - 1521273866181786590 - ], - "infinity": false - }, - { - "x": [ - 9642818207174528085, - 15617465062711953088, - 11263174413902929450, - 639683138088730423 - ], - "y": [ - 15150652293369779803, - 11338278639695990684, - 12204993260723588081, - 2039902155290309382 - ], - "infinity": false - }, - { - "x": [ - 7292405600450693833, - 573142590034645775, - 1583019100043676600, - 1978695840953226358 - ], - "y": [ - 5154489367309996043, - 8763740977657654022, - 9821219773990064941, - 2636875463267519559 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 2075450237700219880, - 2920304484074114568, - 8294843245052708759, - 555293007149161182 - ], - "y": [ - 6360019558055677441, - 7673047654179899818, - 10263007591992092214, - 2148859098846651643 - ], - "infinity": false - }, - { - "x": [ - 3970783323754285443, - 13019363829879217592, - 18197490676081603277, - 630296172623407012 - ], - "y": [ - 7987745494904024640, - 9631048689610078757, - 1592818072678520163, - 2678374240960081558 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 3055966415338102721, - 18231075292903695376, - 9187400351012014001, - 2311743062653684305 - ], - "y": [ - 2553578246375478674, - 930511927228692161, - 2271826946385879571, - 3124263363559878329 - ], - "infinity": false - }, - { - "x": [ - 6936812562216228782, - 15195638439305648290, - 17827467578192758430, - 2674740411261002393 - ], - "y": [ - 9738743088557108685, - 17225541903460577384, - 16627013813461429872, - 494410407050490065 - ], - "infinity": false - }, - { - "x": [ - 10570962909758341245, - 18167360144953681397, - 2744925075742623060, - 736412139310579435 - ], - "y": [ - 13849279071386536985, - 10093748777935480433, - 904764951143479286, - 138814932031469939 - ], - "infinity": false - }, - { - "x": [ - 4533871929444677010, - 10106157783629999301, - 4178648893377901718, - 3164693318611048089 - ], - "y": [ - 12699039702383686311, - 4388078229442418460, - 8961813905523894854, - 570254591975307765 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 18884644, - "lookup_selector_commitment": { - "x": [ - 15022814412717317376, - 17444332185630324119, - 14685665421775887958, - 906494215348891007 - ], - "y": [ - 9833778905776399360, - 1648124311168457783, - 3500435402371619753, - 2370413643071351216 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 8321950609730151216, - 18010887235457883784, - 17038267498493175776, - 1380842840607309871 - ], - "y": [ - 3264160671000273944, - 16611917363401804468, - 8505391859632632917, - 2149881676646664319 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_1_key.json b/core/bin/verification_key_generator_and_server/data/verification_1_key.json deleted file mode 100644 index 0310303d2a53..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_1_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 7601801432079276288, - 15201863322122857773, - 8806193975262404580, - 2590787273683229105 - ], - "y": [ - 16702527967956763728, - 6181870639994435984, - 1867123357108619315, - 2767403024411663364 - ], - "infinity": false - }, - { - "x": [ - 2455316591212726341, - 2027771240685247927, - 10685588854446154162, - 3030775657966372875 - ], - "y": [ - 18300009037843703356, - 1612973442135305251, - 10693350009422283513, - 1442590213691840716 - ], - "infinity": false - }, - { - "x": [ - 12311884457715965312, - 10390638194798557018, - 11306832124741148566, - 300716765354847473 - ], - "y": [ - 9707964220031061231, - 14753080439380196493, - 5717535245627190368, - 702219636062983319 - ], - "infinity": false - }, - { - "x": [ - 7758453297146426337, - 1673770484163252092, - 14607544807007157753, - 857313958429629763 - ], - "y": [ - 14921629410308576937, - 15298335487420996140, - 2704982045392946878, - 2611590721009022852 - ], - "infinity": false - }, - { - "x": [ - 14311011031579784592, - 15625526098906078640, - 1319146597092063841, - 774276845418764858 - ], - "y": [ - 3893523842912943845, - 18146056093503974553, - 11030513442747849089, - 389965813625175232 - ], - "infinity": false - }, - { - "x": [ - 7007915445081129178, - 2401922490835966325, - 418720827124106725, - 2770268368066902308 - ], - "y": [ - 12116308634970006696, - 14528630571959109449, - 9950799281726780069, - 724152027617190422 - ], - "infinity": false - }, - { - "x": [ - 2442021019274420960, - 16295185893380203674, - 2439146651414642189, - 2243335375830582173 - ], - "y": [ - 3782090054162740071, - 4704457281172608987, - 4410900061257118309, - 764611777065564766 - ], - "infinity": false - }, - { - "x": [ - 17964884224938230037, - 7876675311267561320, - 16762398450655445790, - 1210707988542142007 - ], - "y": [ - 10470358785861361347, - 9485656365593190672, - 6046378362748740079, - 2457285875935475197 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 17157526827088368172, - 11284084393440625999, - 9351565798611728109, - 3234841809825307363 - ], - "y": [ - 8319704714678793930, - 4159327153032521498, - 15356346081767327573, - 3239913585027348493 - ], - "infinity": false - }, - { - "x": [ - 15456321646261647359, - 15891438700803416959, - 3317730603133051465, - 2641175705943818316 - ], - "y": [ - 1411951218052246200, - 1661720531643832913, - 13537400120511760371, - 2292851110898807736 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 10328956753700766823, - 2827084848292920926, - 6753362467616392790, - 3266354497443915853 - ], - "y": [ - 4786671171082888838, - 11071539213550223285, - 3886224490311829958, - 1435384580945051012 - ], - "infinity": false - }, - { - "x": [ - 6970901872301032061, - 11845499850875638451, - 12523013241874863158, - 564589203700245768 - ], - "y": [ - 9149991346853645253, - 10833082414663634622, - 10032445307744641248, - 3184550747076826571 - ], - "infinity": false - }, - { - "x": [ - 2899501934612768796, - 7289832407727333580, - 15398305180487198919, - 2955735241334744486 - ], - "y": [ - 4963499698281910643, - 5723522390488208800, - 3637467607919864741, - 339118267031086794 - ], - "infinity": false - }, - { - "x": [ - 16561673014946600686, - 6893642268089467710, - 11554023210615815565, - 122477375056362239 - ], - "y": [ - 15978560303000591303, - 6087766803442805629, - 6114779478264008006, - 2753348573959524636 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 30899639, - "lookup_selector_commitment": { - "x": [ - 4819118611809066421, - 16205075690681881406, - 8088108199972047891, - 2462381205202312681 - ], - "y": [ - 9403235417076804812, - 11746452954984920263, - 5479393366572364588, - 2168476120537571525 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 1589280911861251894, - 2000192568988587993, - 18399902493387281635, - 1843483375839232315 - ], - "y": [ - 14712825033319581746, - 11500494123399487569, - 4370642671010258701, - 567620704393396341 - ], - "infinity": false - }, - { - "x": [ - 0, - 0, - 0, - 0 - ], - "y": [ - 1, - 0, - 0, - 0 - ], - "infinity": true - }, - { - "x": [ - 0, - 0, - 0, - 0 - ], - "y": [ - 1, - 0, - 0, - 0 - ], - "infinity": true - }, - { - "x": [ - 5989740765536181742, - 7510673671757970234, - 7988398980529338112, - 2047433943537325290 - ], - "y": [ - 14952889876146512965, - 17141012675484923048, - 328206788961236528, - 866564802795139 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 4824978155651454377, - 12191454623887257586, - 12973919510878979890, - 52932438992466171 - ], - "y": [ - 17857145998747603901, - 2092039184434926372, - 11018504664231591204, - 1321736242331612854 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_2_key.json b/core/bin/verification_key_generator_and_server/data/verification_2_key.json deleted file mode 100644 index 79b16257213f..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_2_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 5518783475412319303, - 13900056820557691891, - 3293972357974626054, - 2215936931279678502 - ], - "y": [ - 7955917949806788616, - 13341003959544330056, - 2090626280536970058, - 340565138339520735 - ], - "infinity": false - }, - { - "x": [ - 14185170917510557830, - 8046892618400404954, - 16599645397148333553, - 2994187418830549588 - ], - "y": [ - 7234254448777026502, - 8445782435526889669, - 14116370103157060862, - 2248206929083565209 - ], - "infinity": false - }, - { - "x": [ - 11154659552703848544, - 12941656139895069323, - 17062140236305086427, - 722110816848028084 - ], - "y": [ - 5009717036998782771, - 827592822749515890, - 15966856850732642654, - 618036931564479654 - ], - "infinity": false - }, - { - "x": [ - 5157594213696692987, - 15014090155482426422, - 706425002062263449, - 3203486979181293219 - ], - "y": [ - 14363949081622225749, - 9001876918808042476, - 1615414451418136701, - 444697301726425121 - ], - "infinity": false - }, - { - "x": [ - 9176460251336839321, - 17295305184785757140, - 7831134341003191604, - 2666806971657364559 - ], - "y": [ - 2598277252699259004, - 11916936738177575234, - 2912317122505195338, - 2404138220482962548 - ], - "infinity": false - }, - { - "x": [ - 11575910134534349159, - 14192914809594698195, - 18267718409201448839, - 142641722814285206 - ], - "y": [ - 5883506329268908990, - 2832339585209792351, - 14642260147093833347, - 392817691249359885 - ], - "infinity": false - }, - { - "x": [ - 12908012748245269010, - 6525727331816152736, - 16979431824428028279, - 2845131870310951239 - ], - "y": [ - 1571963770034876851, - 17602700402136611105, - 13310928253737079884, - 3347891464097055062 - ], - "infinity": false - }, - { - "x": [ - 832167803175150309, - 11457734167413059640, - 13250442890410377059, - 2814079984479722654 - ], - "y": [ - 1463471541691279258, - 1744973157713476297, - 1204969522442685286, - 1269233371856967282 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 10352656458395970023, - 3995520406692994966, - 13084432248093257522, - 2302839365715839904 - ], - "y": [ - 8225034751786073151, - 16771047952615636124, - 616708265068224682, - 186403683175385821 - ], - "infinity": false - }, - { - "x": [ - 4270731028924703792, - 3128341040439802084, - 15083522049785140229, - 2261189689222904761 - ], - "y": [ - 8781157350107493893, - 14766318733918494793, - 9428422381369337621, - 419743052593117743 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 11112968480130414212, - 11913364106966677596, - 36671493864905181, - 496058283903160224 - ], - "y": [ - 9691136012048916590, - 12909186572206021308, - 1700657689434945171, - 3072265811815532764 - ], - "infinity": false - }, - { - "x": [ - 11360744654540534278, - 9830357778413675465, - 5192069313646589173, - 113131628631742646 - ], - "y": [ - 5515513518975242303, - 323890392099446701, - 2255482865429449468, - 2322464724330067577 - ], - "infinity": false - }, - { - "x": [ - 3414259545645111239, - 5416149397109634837, - 12993204506510556426, - 2894091844446687144 - ], - "y": [ - 4731949297479191167, - 1043460441127916951, - 16890401788673829290, - 1356564712828723527 - ], - "infinity": false - }, - { - "x": [ - 8993182433738017869, - 11441314659459910136, - 8181494681500166120, - 1591321336872387140 - ], - "y": [ - 5278254820002084488, - 17932571960593236295, - 7626453034762681225, - 3463596506399756742 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 30783671, - "lookup_selector_commitment": { - "x": [ - 1336161834228740427, - 15823221750660268452, - 13689567356831376139, - 1839611883700311389 - ], - "y": [ - 14875759795137726191, - 20318096045504920, - 8816565555629805366, - 75556627728969178 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 1589280911861251894, - 2000192568988587993, - 18399902493387281635, - 1843483375839232315 - ], - "y": [ - 14712825033319581746, - 11500494123399487569, - 4370642671010258701, - 567620704393396341 - ], - "infinity": false - }, - { - "x": [ - 0, - 0, - 0, - 0 - ], - "y": [ - 1, - 0, - 0, - 0 - ], - "infinity": true - }, - { - "x": [ - 0, - 0, - 0, - 0 - ], - "y": [ - 1, - 0, - 0, - 0 - ], - "infinity": true - }, - { - "x": [ - 5989740765536181742, - 7510673671757970234, - 7988398980529338112, - 2047433943537325290 - ], - "y": [ - 14952889876146512965, - 17141012675484923048, - 328206788961236528, - 866564802795139 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 3408213281770836085, - 15382444791373914560, - 16110552627056571461, - 1161688479331593061 - ], - "y": [ - 13379188756114722390, - 12926267823879081751, - 14282599792449107495, - 3244837013658545871 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_3_key.json b/core/bin/verification_key_generator_and_server/data/verification_3_key.json deleted file mode 100644 index 613c65dec32a..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_3_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 4247884029119603815, - 14048318895702359089, - 1617022869923646571, - 1004300266779052296 - ], - "y": [ - 17868528514201987465, - 4244261302597587354, - 10221573892940475912, - 2482382880446840010 - ], - "infinity": false - }, - { - "x": [ - 6238506840459074871, - 18254983327500098151, - 12976360180164130634, - 1219856697105853614 - ], - "y": [ - 1359994609126438238, - 17827470346804056210, - 16773833510918183872, - 2604619773311417557 - ], - "infinity": false - }, - { - "x": [ - 5480908979724966765, - 3393255975447524652, - 10371160681199271551, - 3483125449532424455 - ], - "y": [ - 6910224697959110691, - 8190986918875328214, - 18233342390114194740, - 371038657258361111 - ], - "infinity": false - }, - { - "x": [ - 1589636458242554884, - 17321835409586313003, - 13993520794641679178, - 1266542986497561712 - ], - "y": [ - 5397891169353072140, - 5878548729835574296, - 15706893227817678651, - 1769961527856953483 - ], - "infinity": false - }, - { - "x": [ - 17541435070606794744, - 2655627213950653916, - 11216216944579921605, - 1313780180047509779 - ], - "y": [ - 16950319453735037870, - 1632204383055288188, - 15201163922365522932, - 2864472556240937346 - ], - "infinity": false - }, - { - "x": [ - 11997977223945303553, - 14325590013978700522, - 15557533141347230729, - 3289139360100222484 - ], - "y": [ - 2276406350677881932, - 12276125258173429823, - 6135372778488654786, - 2960027660870022236 - ], - "infinity": false - }, - { - "x": [ - 8889079782908651911, - 9444258938063781000, - 6152157289837951831, - 2046144251434758098 - ], - "y": [ - 3506685845878604982, - 480610274681523215, - 17898829927408725055, - 478373452366390807 - ], - "infinity": false - }, - { - "x": [ - 9543795530837745598, - 5641706788025454992, - 2058665597673045347, - 3199980849578540913 - ], - "y": [ - 2134420461745303677, - 11079036403297001210, - 13973590059437528369, - 2236186172656440899 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 17082763384512425754, - 5415974525679408765, - 2982831717715582652, - 2185533346241584143 - ], - "y": [ - 889517497459248510, - 11305258809453581163, - 14785916458686019285, - 712045239932611417 - ], - "infinity": false - }, - { - "x": [ - 1486326951928055275, - 17648143945822975405, - 8789056175543467342, - 1582641302957127155 - ], - "y": [ - 16130216435506275947, - 186882025793811656, - 5333388052689527168, - 2555185016165074595 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 6775436174991417687, - 1962133343483010121, - 3639644700285584252, - 2751431324201714590 - ], - "y": [ - 16721581791017871189, - 2572212631009994187, - 12263629829130796245, - 1194783809693078725 - ], - "infinity": false - }, - { - "x": [ - 9781583375044732502, - 17099127122236789849, - 15683598159868779227, - 2137916464125382410 - ], - "y": [ - 11971077938028623721, - 14460546631248863771, - 3674726360546135290, - 2587006282919627488 - ], - "infinity": false - }, - { - "x": [ - 2258960665841769264, - 11476106728738999555, - 2154715457718708453, - 1652460267728538717 - ], - "y": [ - 591013691648424928, - 2747643213972148016, - 4382285331965077793, - 700518369290275435 - ], - "infinity": false - }, - { - "x": [ - 17029386353507514799, - 12736838109975824615, - 17948233540620781856, - 1661567367117856229 - ], - "y": [ - 5088293739561490025, - 257269786506894093, - 7029871828271960168, - 2982592857123453815 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 15390957, - "lookup_selector_commitment": { - "x": [ - 3143229288506876352, - 14398478555351850494, - 17971061391349533728, - 2397240458539623423 - ], - "y": [ - 2507720097747632492, - 4897824016944146490, - 8535810669426357324, - 2617442440174156771 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 12925597216490182210, - 13030942092034120135, - 17733316148446765999, - 112547709703624791 - ], - "y": [ - 13293415162200038331, - 13010565234555563811, - 15476251035925496743, - 2588541998389664114 - ], - "infinity": false - }, - { - "x": [ - 11118240121224901946, - 9394562257959111170, - 9026436993514314918, - 1751747619588842429 - ], - "y": [ - 6039590802345873394, - 17531716309156986038, - 1711770599161550805, - 1941094644175870288 - ], - "infinity": false - }, - { - "x": [ - 17999903301086933877, - 10468070608989378923, - 3479353092436121335, - 607756992244480908 - ], - "y": [ - 10863079642303790364, - 4737012301447477097, - 4605789209164294308, - 1430572887755557386 - ], - "infinity": false - }, - { - "x": [ - 4609762018249049814, - 4113097757442144437, - 4725434011535510809, - 2977599521231955696 - ], - "y": [ - 14636094180551257630, - 8819447661702130886, - 1091706295519429215, - 56675985696303183 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 7406705046881629689, - 13550366909312172285, - 11707241152492715411, - 1951231993396003315 - ], - "y": [ - 649840467305243342, - 10916062129580101841, - 7643158916474300887, - 1216418901317802861 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_4_key.json b/core/bin/verification_key_generator_and_server/data/verification_4_key.json deleted file mode 100644 index 8d42dcd66a75..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_4_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 15923176050075197, - 8963905519117333456, - 5333091548039957996, - 1660697180439834807 - ], - "y": [ - 13105864494044341635, - 10079874572012628853, - 4164109084931753781, - 1860950003357484648 - ], - "infinity": false - }, - { - "x": [ - 8216018177730810417, - 13660800917029254431, - 2933384097067755755, - 2823425599268575868 - ], - "y": [ - 8768863192718196559, - 10146282684570870426, - 8275806247588563419, - 605489936306033583 - ], - "infinity": false - }, - { - "x": [ - 4277344855257545209, - 11172040917478096607, - 4489086903928758598, - 289283798032159440 - ], - "y": [ - 10444137083253378550, - 12133212848977612596, - 6748791972701343485, - 286274227999569844 - ], - "infinity": false - }, - { - "x": [ - 8861797510071553254, - 12734094237204882518, - 13692967202881086499, - 641906135411222522 - ], - "y": [ - 6831762763487302461, - 11965405347371646114, - 6218256502970252800, - 3201462388136754725 - ], - "infinity": false - }, - { - "x": [ - 12385743015818134054, - 16282219738575446638, - 3256359841301423419, - 505673042938576760 - ], - "y": [ - 6744956686738207932, - 8994291190634790001, - 16789606231722015883, - 2027930268272962928 - ], - "infinity": false - }, - { - "x": [ - 13671822069226357541, - 818021157447551159, - 10542481209144358852, - 2459295197762128786 - ], - "y": [ - 1072649761929447549, - 6089126583512618706, - 1178131210084507361, - 1066836948212725576 - ], - "infinity": false - }, - { - "x": [ - 16878956366815094090, - 364977461173568122, - 5439594588743996145, - 1265442855735725449 - ], - "y": [ - 11461704536083653156, - 660278441271820299, - 4314245569905306892, - 1438663846765259508 - ], - "infinity": false - }, - { - "x": [ - 9038539654045396650, - 539827912679485452, - 15399544523862100757, - 1256406598444490417 - ], - "y": [ - 5422113905848106255, - 4943961807853536385, - 10022409325033689104, - 3200702511424842211 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 7750990741566547331, - 12040155777441846781, - 3000981333322867315, - 2393292192734976436 - ], - "y": [ - 3394853839941291504, - 944019051205640111, - 1104911864338577098, - 2127308956089601096 - ], - "infinity": false - }, - { - "x": [ - 4735140124663926465, - 16935779121597983173, - 17111626619540374574, - 2327973550601526140 - ], - "y": [ - 8990848735371189388, - 4589751206662798166, - 7575424772436241307, - 2798852347400154642 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 4765077060699177749, - 15235935045874519477, - 2022237788491579392, - 354385727984957703 - ], - "y": [ - 11620113321350620961, - 2521830680983779826, - 14047226057605943635, - 2718701882953208503 - ], - "infinity": false - }, - { - "x": [ - 12967015398643083015, - 1100660813730542482, - 7835181433213557652, - 803165211156388599 - ], - "y": [ - 8557385569712401227, - 535900682745452035, - 16083571717847325979, - 396765644246918860 - ], - "infinity": false - }, - { - "x": [ - 6868107733370365435, - 17106601841261210672, - 12219407605084986215, - 2345246684976405066 - ], - "y": [ - 17532412968783851743, - 9996315626158111485, - 17970945522106166231, - 1003764081419207606 - ], - "infinity": false - }, - { - "x": [ - 7011201477832405407, - 8818123127103997131, - 2979445003396953339, - 318603240233076406 - ], - "y": [ - 11712108043964996282, - 3474989587891133574, - 3983451673298542860, - 1181581919257021598 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 8484642, - "lookup_selector_commitment": { - "x": [ - 27459247093738343, - 1785927757103538268, - 14972116880195568621, - 1034224917068963325 - ], - "y": [ - 17453858127001596558, - 6200103235089742197, - 16245568162666829501, - 651193715230511441 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 697552212563769686, - 7709943502535418760, - 15019345407325619175, - 3433081085078580257 - ], - "y": [ - 8668947019840357731, - 14698901351824712883, - 15088598879190660424, - 2873081208166433946 - ], - "infinity": false - }, - { - "x": [ - 7893133928909060673, - 7064922516930129957, - 3592836702741304814, - 2239702595710114437 - ], - "y": [ - 7691360541875191519, - 11379321785127235277, - 6653616064071569031, - 2555434628517540774 - ], - "infinity": false - }, - { - "x": [ - 6243944238013052821, - 7908243182210136125, - 17178099109525791299, - 2553622184721264566 - ], - "y": [ - 736121280088239428, - 6158073429758170526, - 11217302997977204117, - 2594798912020899417 - ], - "infinity": false - }, - { - "x": [ - 2064240298596094591, - 16917726764104887991, - 11042784977532408536, - 3377647228930170830 - ], - "y": [ - 10635525052494768819, - 387400048616497096, - 9379200582543310995, - 1571766153703296253 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 14868101692362122308, - 8135288013508071846, - 9460482611527381887, - 512823635961282598 - ], - "y": [ - 8358211286664762188, - 3532634521932288534, - 5862145521507736138, - 1807935137626658536 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_5_key.json b/core/bin/verification_key_generator_and_server/data/verification_5_key.json deleted file mode 100644 index b9a31b919f1c..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_5_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 12322129650547620518, - 4320033807979823995, - 4503809593276792861, - 630958448551597950 - ], - "y": [ - 4947307957322067889, - 1897773243457379956, - 1563584362302565484, - 802109862761172056 - ], - "infinity": false - }, - { - "x": [ - 5860641327684713918, - 16885915425353665713, - 7037370194263044401, - 1837438863045303696 - ], - "y": [ - 13386292219804271609, - 4960073609197619993, - 7328379249582994262, - 191728769121948464 - ], - "infinity": false - }, - { - "x": [ - 9390502900121613993, - 17218409610830310329, - 4830832371938391322, - 1805131323553685028 - ], - "y": [ - 15707040961083920686, - 16216062707384374953, - 16957058843586642758, - 1341814870249072628 - ], - "infinity": false - }, - { - "x": [ - 969252611989285232, - 181405773082212747, - 11110666465356509832, - 1888802363524687207 - ], - "y": [ - 5293477339288357424, - 12076391347720360980, - 11422893229655154394, - 3165450734777404812 - ], - "infinity": false - }, - { - "x": [ - 642192487369089358, - 9585449571929647331, - 3847960352134961209, - 984199510163128792 - ], - "y": [ - 13950390676065893881, - 975256099594703300, - 253120832016214204, - 1860679841584192219 - ], - "infinity": false - }, - { - "x": [ - 3564548447861991296, - 6278944799487206913, - 1163701992635366786, - 3214877162977671335 - ], - "y": [ - 13131873482361140204, - 14012120801722220187, - 13254371011592477950, - 1082108070640175604 - ], - "infinity": false - }, - { - "x": [ - 14190764189814537607, - 18412181832598818289, - 17213387738194113336, - 1662783623959823461 - ], - "y": [ - 7987199081435644988, - 17119136750046780209, - 8770669323846078492, - 3183489396270587333 - ], - "infinity": false - }, - { - "x": [ - 14638218826597535389, - 16409988612234258347, - 5025411344133541245, - 603088654230685360 - ], - "y": [ - 12538363432956258836, - 6558875956959901550, - 2415879426147965883, - 750702584304895055 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 2599908293582905760, - 13534206398743622493, - 15926090086034346074, - 467418127379229858 - ], - "y": [ - 9529512934078774185, - 1459270552041127965, - 13418846370362665102, - 2270996612016337371 - ], - "infinity": false - }, - { - "x": [ - 7264275706530137047, - 5590205367072257545, - 17891440127697345143, - 360638857846382524 - ], - "y": [ - 17983779934218975397, - 1625779403076670241, - 1474025795387210129, - 1716171421120825643 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 9354841115000244260, - 12887310615208346489, - 1120617137774653400, - 424227936372254439 - ], - "y": [ - 3626714025954019309, - 4480975902927818206, - 10093567956580931634, - 2779897825000836477 - ], - "infinity": false - }, - { - "x": [ - 1864884782104066211, - 1247154271168453374, - 9982166936353409582, - 1177339527115773898 - ], - "y": [ - 9932597332303163060, - 1888682277213109000, - 11684220277443154622, - 3062389133489783806 - ], - "infinity": false - }, - { - "x": [ - 9943021177878836437, - 9004866876172522532, - 14085451328492136137, - 1567186274425392936 - ], - "y": [ - 7148906168793986389, - 4780330524752436486, - 10067456648871712650, - 179752856567560382 - ], - "infinity": false - }, - { - "x": [ - 14745822832390509907, - 13862030626549782961, - 10000268356302875837, - 705042314567833799 - ], - "y": [ - 11091254259539384938, - 11733968109785394056, - 11099103738494585500, - 1527456782567955191 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 35330543, - "lookup_selector_commitment": { - "x": [ - 12333191731462980214, - 17841370099698959347, - 12878670991018181621, - 2894319630687016858 - ], - "y": [ - 76816727314643395, - 3214684791046221459, - 878301108738499830, - 126016925902987736 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 911668445361375614, - 12752365066512000136, - 11550232015863976467, - 2053619216798992367 - ], - "y": [ - 4194339833917391280, - 1643071887467668153, - 3377480965202592691, - 1664272901450533719 - ], - "infinity": false - }, - { - "x": [ - 2999316735203966181, - 5189676006781764591, - 14324679313847304783, - 1264086978509739587 - ], - "y": [ - 8714172036038650967, - 10907167170124829028, - 8950970593162102458, - 1596853051185997037 - ], - "infinity": false - }, - { - "x": [ - 1146500486770850326, - 13562754408872334896, - 14063471769392190265, - 3387351506820193517 - ], - "y": [ - 6677788829230735422, - 15425668102208730571, - 5341291772716012975, - 539156410041791428 - ], - "infinity": false - }, - { - "x": [ - 18159886519320172405, - 4286826840324377773, - 16364826089434525345, - 228697666397725767 - ], - "y": [ - 4850633487261444791, - 6327421534074497160, - 12883776034588695446, - 1510314148471267214 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 18245233954308230592, - 8193493714287610439, - 6521078295132558240, - 861511081336275611 - ], - "y": [ - 4275834222266292944, - 13179071278128968874, - 5943013356852335765, - 2456639561657053045 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_6_key.json b/core/bin/verification_key_generator_and_server/data/verification_6_key.json deleted file mode 100644 index 34419df17702..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_6_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 11033020679838791108, - 14920056278440370765, - 8156477685651219112, - 2935096142913695825 - ], - "y": [ - 12780055516709256833, - 966513406268819160, - 9584266886886532866, - 892347068344972829 - ], - "infinity": false - }, - { - "x": [ - 4044870432040348042, - 10630300946926732771, - 3143480015080245177, - 323917785885883620 - ], - "y": [ - 2297905282612888789, - 8206728682979815807, - 10628767928228215441, - 3062326525278498604 - ], - "infinity": false - }, - { - "x": [ - 14760731158538087565, - 9176522400170689419, - 9855180338242634009, - 2456568616568530201 - ], - "y": [ - 5168103953295979961, - 397013651969935557, - 13864468728668213717, - 2925074735515169158 - ], - "infinity": false - }, - { - "x": [ - 13613691592548742743, - 11339389230513898784, - 4864282628000142183, - 2568915564796772962 - ], - "y": [ - 13074021698952750513, - 14891339562597317806, - 6145754680491802845, - 913243322463864468 - ], - "infinity": false - }, - { - "x": [ - 9607983563343027008, - 1604609357347728263, - 6735137627175405143, - 91305611485454778 - ], - "y": [ - 2068449139446365265, - 6171753015906067998, - 16290186276604645197, - 420889087081901603 - ], - "infinity": false - }, - { - "x": [ - 15994614598808477960, - 5137738490508028659, - 6599503545391493738, - 3293094250487745346 - ], - "y": [ - 3246688300070721763, - 8836841286539929132, - 1231014124908407748, - 3042941126579517307 - ], - "infinity": false - }, - { - "x": [ - 12550390789117808745, - 14001030013656521177, - 16383284077678821701, - 1815317458772356897 - ], - "y": [ - 10125044837604978181, - 7468984969058409331, - 592554137766258541, - 2877688586321491725 - ], - "infinity": false - }, - { - "x": [ - 12238091769471133989, - 184716847866634800, - 5888077423956723698, - 609118759536864800 - ], - "y": [ - 7725369615076384544, - 7561073323636510559, - 10473734750023783127, - 861766554781597742 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 1206127807467530207, - 3510053718168412786, - 7933459343694333819, - 3179950874373950282 - ], - "y": [ - 5784856107466398982, - 395767970566909293, - 11244200096534021583, - 2068407511544404377 - ], - "infinity": false - }, - { - "x": [ - 4044617248058764838, - 11957266999135308674, - 17621747993137866783, - 990156155955733134 - ], - "y": [ - 17234504892477991728, - 17558826298225495489, - 9349531438753716103, - 2656409262947709594 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 4308597000331285311, - 12130199317436319902, - 3842336010209461436, - 191866453597778475 - ], - "y": [ - 2144400171783010971, - 13016087318985913183, - 7166370365336301922, - 2216888390030560212 - ], - "infinity": false - }, - { - "x": [ - 4661184458541745063, - 12423889401726065791, - 11959346001895915074, - 779668716585305501 - ], - "y": [ - 16401363790535442499, - 7367694133722005848, - 8015837005184593399, - 454166987511489961 - ], - "infinity": false - }, - { - "x": [ - 858215262803403659, - 1405268530667707386, - 7763962169005921611, - 2845435536097215865 - ], - "y": [ - 10639490331338262540, - 6397733211512468794, - 968161689973799899, - 2054756257253905633 - ], - "infinity": false - }, - { - "x": [ - 17338818659525246480, - 13318488425310212471, - 10548319374858973842, - 87084958643052105 - ], - "y": [ - 2279840344577984658, - 15197280761751903251, - 16019225334594459873, - 149925650787595538 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 3054916, - "lookup_selector_commitment": { - "x": [ - 4844230422625825285, - 956290027823441223, - 763010695794739308, - 2426170829255106638 - ], - "y": [ - 13850520521470006763, - 9003994589054655373, - 10310690204425503422, - 3012516431885755457 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 5825422128268478267, - 9219263846299851036, - 3879231702557190566, - 1702488722758880769 - ], - "y": [ - 18311881100262470992, - 5742998199368802392, - 18106865487471159417, - 502191980176920012 - ], - "infinity": false - }, - { - "x": [ - 17195892082859417081, - 7890531942603584793, - 2381805632820057528, - 3173232410464566465 - ], - "y": [ - 16359614627947132075, - 3459600273035137079, - 4550762061432972122, - 3394559699318358224 - ], - "infinity": false - }, - { - "x": [ - 1716103379277390185, - 18097936269579187542, - 16357329729761063450, - 1508640059338197502 - ], - "y": [ - 11014806739603983364, - 4396503314588777389, - 9397245609635151055, - 1703957955248411380 - ], - "infinity": false - }, - { - "x": [ - 4770171350693477354, - 17110558673192292253, - 9799800677557311408, - 761984875463445481 - ], - "y": [ - 1560561403388310063, - 31331275310848146, - 287152055803835484, - 457826332542037277 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 16775586915653722908, - 9787338077086882544, - 8381721730521821042, - 2974660093975661578 - ], - "y": [ - 3011389235487891234, - 15409507493813096391, - 17416460976276029026, - 324418288749844627 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_7_key.json b/core/bin/verification_key_generator_and_server/data/verification_7_key.json deleted file mode 100644 index 406afcf4f0fe..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_7_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 14104278525941001335, - 6652111379088654370, - 12369045377338511525, - 969809670184836151 - ], - "y": [ - 10111598525423302991, - 15018239425425696172, - 3683372413830991953, - 1023765059890131543 - ], - "infinity": false - }, - { - "x": [ - 11576486884237685781, - 16315823052257401029, - 9860864515877414033, - 3179959598270002012 - ], - "y": [ - 487035971539979311, - 5573003039451484772, - 15711637819381564577, - 1904127920269177012 - ], - "infinity": false - }, - { - "x": [ - 18299921128106602792, - 211731469708793711, - 17645028854462121436, - 675870769139913517 - ], - "y": [ - 15146647508675165454, - 18353083579110652488, - 12704645658780892142, - 2929235299763077823 - ], - "infinity": false - }, - { - "x": [ - 11570586127780196277, - 2363872676317471379, - 7386811009552915084, - 959006902628416514 - ], - "y": [ - 17455735716787098890, - 14879699386306994564, - 5628100821420984321, - 2862659911936763739 - ], - "infinity": false - }, - { - "x": [ - 8746328571248006135, - 17089435014355939378, - 8764506524471462449, - 1810135458362589443 - ], - "y": [ - 14070512019208911265, - 8756287737315170424, - 14821473955626613, - 1559545289765661890 - ], - "infinity": false - }, - { - "x": [ - 2113591086436573082, - 12629483649401688389, - 11845953673798951216, - 3081238281103628853 - ], - "y": [ - 727696133406005469, - 14413827745813557208, - 6425035421156126073, - 291513487083052109 - ], - "infinity": false - }, - { - "x": [ - 15346257923988607256, - 10403316660718504706, - 7158515894996917286, - 2702098910103276762 - ], - "y": [ - 16559143492878738107, - 12716298061927369795, - 12296985344891017351, - 2814996798832983835 - ], - "infinity": false - }, - { - "x": [ - 2213195001372039295, - 8878300942582564036, - 10524986226191936528, - 1815326540993196034 - ], - "y": [ - 11397120982692424098, - 4455537142488107627, - 14205354993332845055, - 2313809587433567240 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 14849046431510808003, - 11699893139960418168, - 6000246307731364190, - 3362832011707902866 - ], - "y": [ - 3242560497217933852, - 11672398501106836413, - 987926723326096281, - 2451226739475091625 - ], - "infinity": false - }, - { - "x": [ - 9272095445402359796, - 1201046264826394411, - 7424934554242366462, - 1125893484262333608 - ], - "y": [ - 15903920299684884420, - 17703294385387204708, - 2256937129195345942, - 1905295733884217610 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 7591926766688292250, - 10457199375342460747, - 3214976192729961314, - 1412860682249358355 - ], - "y": [ - 16894260140402496006, - 3666374878391815131, - 15124268261678582348, - 1340579262756129480 - ], - "infinity": false - }, - { - "x": [ - 2963934507934439034, - 17415763666461861018, - 6331792462137338053, - 3122358526111186727 - ], - "y": [ - 15040784043381591388, - 7188410244350767315, - 14077554108063383431, - 1704329843327300001 - ], - "infinity": false - }, - { - "x": [ - 7967507884960122293, - 13509230570773443525, - 11125712791473385552, - 2241808950326876268 - ], - "y": [ - 10594180941877323940, - 17179032413109513856, - 17941607623778808075, - 646138820984886096 - ], - "infinity": false - }, - { - "x": [ - 4729534828155895283, - 15489050734511381239, - 4847364931161261393, - 2461584260035042491 - ], - "y": [ - 15255817542606978857, - 6517429187947361297, - 17127878630247240853, - 3389541567226838859 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 40724289, - "lookup_selector_commitment": { - "x": [ - 5449769839889646584, - 2072406321611922291, - 9391796773218391195, - 2377769168011090955 - ], - "y": [ - 1789189431152658324, - 2639430755172378798, - 136577695530283091, - 3045539535973502646 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 12639039925867405095, - 9606685454938605275, - 7802675863289639223, - 1948831418843225802 - ], - "y": [ - 11059150608777595761, - 10458812733010634961, - 16772660325487078311, - 340608886692078192 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_8_key.json b/core/bin/verification_key_generator_and_server/data/verification_8_key.json deleted file mode 100644 index b8511e17b755..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_8_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 1834112096176967541, - 5137529514715617427, - 6540843391881340212, - 3033401888759110412 - ], - "y": [ - 8910602970094475216, - 13169513767982514776, - 5761530093694221441, - 2733318557350866268 - ], - "infinity": false - }, - { - "x": [ - 4701064149158432365, - 5425087325981406309, - 7911131985858828309, - 1683257627049186617 - ], - "y": [ - 13565328904521460918, - 17013189171844282257, - 4897087111183007258, - 2345861178674095559 - ], - "infinity": false - }, - { - "x": [ - 17285353863442654170, - 17787410547699779811, - 4803131526909484890, - 1607731426619418092 - ], - "y": [ - 3219378920021652314, - 11046862703797106703, - 10595836629242151972, - 2970963661532337787 - ], - "infinity": false - }, - { - "x": [ - 6619857367954187649, - 8023974497004524989, - 10088058961892288757, - 938018804109053807 - ], - "y": [ - 15549411064757453720, - 1776820811429478220, - 8222111141823917842, - 290593315633281086 - ], - "infinity": false - }, - { - "x": [ - 3338931670632164423, - 11330459786926502111, - 13560408114559586439, - 233279858410037466 - ], - "y": [ - 9757980615881472290, - 6475296714459436577, - 15954545788543926629, - 2522580407814024231 - ], - "infinity": false - }, - { - "x": [ - 2168501453409628158, - 16417992951888116942, - 1994813140597965849, - 1938552030580060698 - ], - "y": [ - 2393885012813093493, - 5109365147685051030, - 4449898145078443978, - 996506294158321126 - ], - "infinity": false - }, - { - "x": [ - 8163446935422765754, - 17127634458571165785, - 18101155318188210010, - 1502677094108070955 - ], - "y": [ - 4184320355428455210, - 15479528531137595907, - 8455846016430686855, - 2570922865513301289 - ], - "infinity": false - }, - { - "x": [ - 407579941387952352, - 17088458915370169940, - 16892753644011369852, - 2421666516533613805 - ], - "y": [ - 597435837737447683, - 18122233368438707442, - 4844832744563923839, - 396103093107107006 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 16242434178832819081, - 2218928756172422054, - 5871927983870638422, - 810020555846721779 - ], - "y": [ - 9387856576677982883, - 5119490172321159350, - 14295435318421985120, - 1325809191818871673 - ], - "infinity": false - }, - { - "x": [ - 5933965238687071287, - 10681704800081225943, - 14555731010498897395, - 959799154476325145 - ], - "y": [ - 1501632601560034962, - 9401704677918783964, - 12292111854761501889, - 858616662661742045 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 12841507457971520539, - 6525486152471484441, - 3744486588589217686, - 2769451038405535407 - ], - "y": [ - 14145668232228974364, - 9864097401535863500, - 12665512227995054273, - 1710776254334161256 - ], - "infinity": false - }, - { - "x": [ - 12108157388466567796, - 12008825937320240484, - 11228446795405478904, - 1520424921904150640 - ], - "y": [ - 18157047055378899649, - 10836823561088895074, - 583613418617515639, - 2570085764232471205 - ], - "infinity": false - }, - { - "x": [ - 3117226099128838157, - 10181632193024509490, - 1215328570209780930, - 1536961491401844084 - ], - "y": [ - 11646905141441654681, - 6168936708987385450, - 14459621573162108487, - 2047975568887748173 - ], - "infinity": false - }, - { - "x": [ - 12034664246790330785, - 12032082546920592595, - 12002839514296456095, - 3009479689157977152 - ], - "y": [ - 180421277197569955, - 5815678523367268562, - 11718416396488597085, - 408186057258055191 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 34384753, - "lookup_selector_commitment": { - "x": [ - 3872970821419373956, - 13556503327407661223, - 12832313376327677595, - 211677646774476601 - ], - "y": [ - 17281673428499585093, - 235933066531227024, - 17890327653152417391, - 2551853991532334733 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 14943975734974680929, - 9516136771242606543, - 6695719565456036638, - 3449077049666620393 - ], - "y": [ - 11678209093898264827, - 4499447145490933412, - 6317798459829178953, - 1439219764789809864 - ], - "infinity": false - }, - { - "x": [ - 13501290183905491407, - 17914451638435951710, - 5188762915201956497, - 1220375585898114161 - ], - "y": [ - 14519533874806433487, - 409100046306023, - 2203176115240501563, - 3105700623762337563 - ], - "infinity": false - }, - { - "x": [ - 13968159480895722732, - 6973568812120893251, - 6250254745096478587, - 2299355969860561070 - ], - "y": [ - 7695944005480078577, - 12009671787784557856, - 13727042561077817002, - 219052945806305675 - ], - "infinity": false - }, - { - "x": [ - 4871629130106420314, - 4091595855728790015, - 1851744390500340594, - 3123168382710331270 - ], - "y": [ - 9703969956757970162, - 1215036492891076659, - 11876727836856213678, - 2640893636590396388 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 10299044894603982393, - 4664166516779563250, - 13124827128688646542, - 3361599897730972314 - ], - "y": [ - 18259946931458798404, - 10145479316480429602, - 15446978899103328376, - 265382288883021070 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/data/verification_9_key.json b/core/bin/verification_key_generator_and_server/data/verification_9_key.json deleted file mode 100644 index 75de5f75c78d..000000000000 --- a/core/bin/verification_key_generator_and_server/data/verification_9_key.json +++ /dev/null @@ -1,399 +0,0 @@ -{ - "n": 67108863, - "num_inputs": 1, - "state_width": 4, - "num_witness_polys": 0, - "gate_setup_commitments": [ - { - "x": [ - 15041888416700822899, - 15908701850433687369, - 6928173929840686173, - 501601364708497325 - ], - "y": [ - 9443860646360881208, - 15174745959183347299, - 3341918218952258763, - 1470216750942469587 - ], - "infinity": false - }, - { - "x": [ - 1713492202424532619, - 5921868784153327820, - 3919870428680620477, - 2459274846398943915 - ], - "y": [ - 8012717129874416534, - 13032363221581987781, - 9462161206147300944, - 1151760065513271967 - ], - "infinity": false - }, - { - "x": [ - 6636128327108235840, - 9362733145474272574, - 7779132015244601843, - 474802631021936400 - ], - "y": [ - 3900992471196218787, - 113851245079995197, - 7493904056590361535, - 3140468871801097229 - ], - "infinity": false - }, - { - "x": [ - 4340102674797800902, - 8715432707094353745, - 4331145745081713603, - 45456583984841487 - ], - "y": [ - 18326546742044058782, - 15443239165658185296, - 9765917874876721196, - 687859761729374839 - ], - "infinity": false - }, - { - "x": [ - 10804694580890857975, - 10550068287306981825, - 14956274043654722561, - 3060589920124935341 - ], - "y": [ - 17010223672048359580, - 263749806111642373, - 8349695975133446526, - 2826070525773268002 - ], - "infinity": false - }, - { - "x": [ - 16133249269780245267, - 4275571784340824698, - 6262619645627758753, - 3231281899173719188 - ], - "y": [ - 11839616617849449709, - 7142633755989890055, - 10840735473548209733, - 2847350786075278882 - ], - "infinity": false - }, - { - "x": [ - 16258572583186965203, - 1354691125575792689, - 17235265854934968790, - 1252220109588505888 - ], - "y": [ - 9336541637487074271, - 18402912967310224930, - 13223187653117829136, - 2979297976786733465 - ], - "infinity": false - }, - { - "x": [ - 8525686695522099028, - 4103157564078645049, - 18392570749492199187, - 2911539491816599180 - ], - "y": [ - 114653447583918953, - 10470307038453386601, - 11189850644566793538, - 1298227034210846592 - ], - "infinity": false - } - ], - "gate_selectors_commitments": [ - { - "x": [ - 2069700145549311928, - 4250782333685017927, - 14207216715687122978, - 1145927286048477791 - ], - "y": [ - 9341202692364554712, - 12346939747104737180, - 2826478533799125818, - 2279570556437452275 - ], - "infinity": false - }, - { - "x": [ - 12388902775325386546, - 1277383964095999647, - 10535796018183893831, - 3359866702323175506 - ], - "y": [ - 16500893366957272235, - 2806147688388338314, - 8233156072220488773, - 2867848844627212711 - ], - "infinity": false - } - ], - "permutation_commitments": [ - { - "x": [ - 17521183961631816299, - 18327810537117645266, - 16586212795163003556, - 3052771534158410452 - ], - "y": [ - 8441310283734453731, - 14146088755801181801, - 17480253356603213989, - 3217948944323396651 - ], - "infinity": false - }, - { - "x": [ - 16076801532842923524, - 7514743296775639295, - 2571323986448120255, - 184367540214459973 - ], - "y": [ - 13389643967183613114, - 17108261756464256828, - 11145735340309739417, - 2142196980030893874 - ], - "infinity": false - }, - { - "x": [ - 8034683328666433725, - 5436036566901194392, - 18053257213361014053, - 2821377847227509494 - ], - "y": [ - 14471305228212723444, - 8894846184648865892, - 7047725473055235530, - 2413388400332075493 - ], - "infinity": false - }, - { - "x": [ - 14026981588443304814, - 14671946927765496183, - 13387079215022495926, - 2554705188091675830 - ], - "y": [ - 440116222237740520, - 1630168477189852269, - 17833425794232523381, - 908824471705597078 - ], - "infinity": false - } - ], - "total_lookup_entries_length": 41494904, - "lookup_selector_commitment": { - "x": [ - 13889323383351416990, - 17887386740570674124, - 5463612855590268091, - 2434255340534820869 - ], - "y": [ - 2436699678434218349, - 11251365794004058995, - 11023509005141034197, - 2867854671852170604 - ], - "infinity": false - }, - "lookup_tables_commitments": [ - { - "x": [ - 631990924006796604, - 16139625628991115157, - 13331739325995827711, - 1062301837743594995 - ], - "y": [ - 15303054606290800139, - 15906872095881647437, - 7093896572295020249, - 1342952934989901142 - ], - "infinity": false - }, - { - "x": [ - 7983921919542246393, - 13296544189644416678, - 17081022784392007697, - 1980832835348244027 - ], - "y": [ - 10874958134865200330, - 7702740658637630534, - 14052057929798961943, - 3193353539419869016 - ], - "infinity": false - }, - { - "x": [ - 1114587284824996932, - 4636906500482867924, - 15328247172597030456, - 87946895873973686 - ], - "y": [ - 15573033830207915877, - 5194694185599035278, - 2562407345425607214, - 2782078999306862675 - ], - "infinity": false - }, - { - "x": [ - 18225112781127431982, - 18048613958187123807, - 7325490730844456621, - 1953409020724855888 - ], - "y": [ - 7577000130125917198, - 6193701449695751861, - 4102082927677054717, - 395350071385269650 - ], - "infinity": false - } - ], - "lookup_table_type_commitment": { - "x": [ - 3832160677272803715, - 2122279734318217808, - 811690144328522684, - 1416829483108546006 - ], - "y": [ - 10041279311991435550, - 14702496983143623186, - 4419862575487552747, - 1429817244630465543 - ], - "infinity": false - }, - "non_residues": [ - [ - 5, - 0, - 0, - 0 - ], - [ - 7, - 0, - 0, - 0 - ], - [ - 10, - 0, - 0, - 0 - ] - ], - "g2_elements": [ - { - "x": { - "c0": [ - 5106727233969649389, - 7440829307424791261, - 4785637993704342649, - 1729627375292849782 - ], - "c1": [ - 10945020018377822914, - 17413811393473931026, - 8241798111626485029, - 1841571559660931130 - ] - }, - "y": { - "c0": [ - 5541340697920699818, - 16416156555105522555, - 5380518976772849807, - 1353435754470862315 - ], - "c1": [ - 6173549831154472795, - 13567992399387660019, - 17050234209342075797, - 650358724130500725 - ] - }, - "infinity": false - }, - { - "x": { - "c0": [ - 9089143573911733168, - 11482283522806384523, - 13585589533905622862, - 79029415676722370 - ], - "c1": [ - 5692040832573735873, - 16884514497384809355, - 16717166481813659368, - 2742131088506155463 - ] - }, - "y": { - "c0": [ - 9604638503594647125, - 1289961608472612514, - 6217038149984805214, - 2521661352385209130 - ], - "c1": [ - 17168069778630926308, - 11309277837895768996, - 15154989611154567813, - 359271377050603491 - ] - }, - "infinity": false - } - ] -} \ No newline at end of file diff --git a/core/bin/verification_key_generator_and_server/src/commitment_generator.rs b/core/bin/verification_key_generator_and_server/src/commitment_generator.rs deleted file mode 100644 index ed859bcb4366..000000000000 --- a/core/bin/verification_key_generator_and_server/src/commitment_generator.rs +++ /dev/null @@ -1,37 +0,0 @@ -use anyhow::Context as _; -use zksync_prover_utils::vk_commitment_helper::{ - get_toml_formatted_value, read_contract_toml, write_contract_toml, -}; -use zksync_verification_key_server::generate_commitments; - -fn main() -> anyhow::Result<()> { - tracing::info!("Starting commitment generation!"); - read_and_update_contract_toml() -} - -fn read_and_update_contract_toml() -> anyhow::Result<()> { - let mut contract_doc = read_contract_toml().context("read_contract_toml()")?; - let ( - basic_circuit_commitment_hex, - leaf_aggregation_commitment_hex, - node_aggregation_commitment_hex, - ) = generate_commitments(); - contract_doc["contracts"]["RECURSION_CIRCUITS_SET_VKS_HASH"] = - get_toml_formatted_value(basic_circuit_commitment_hex); - contract_doc["contracts"]["RECURSION_LEAF_LEVEL_VK_HASH"] = - get_toml_formatted_value(leaf_aggregation_commitment_hex); - contract_doc["contracts"]["RECURSION_NODE_LEVEL_VK_HASH"] = - get_toml_formatted_value(node_aggregation_commitment_hex); - tracing::info!("Updated toml content: {:?}", contract_doc.to_string()); - write_contract_toml(contract_doc).context("write_contract_toml") -} - -#[cfg(test)] -mod test { - use super::*; - - #[test] - fn test_read_and_update_contract_toml() { - read_and_update_contract_toml().unwrap(); - } -} diff --git a/core/bin/verification_key_generator_and_server/src/json_to_binary_vk_converter.rs b/core/bin/verification_key_generator_and_server/src/json_to_binary_vk_converter.rs deleted file mode 100644 index c04a67128334..000000000000 --- a/core/bin/verification_key_generator_and_server/src/json_to_binary_vk_converter.rs +++ /dev/null @@ -1,31 +0,0 @@ -use std::{fs::File, io::BufWriter}; - -use bincode::serialize_into; -use structopt::StructOpt; -use zksync_verification_key_server::get_vk_for_circuit_type; - -#[derive(Debug, StructOpt)] -#[structopt( - name = "json existing json VK's to binary vk", - about = "converter tool" -)] -struct Opt { - /// Binary output path of verification keys. - #[structopt(short)] - output_bin_path: String, -} - -fn main() { - let opt = Opt::from_args(); - println!("Converting existing json keys to binary"); - generate_bin_vks(opt.output_bin_path); -} - -fn generate_bin_vks(output_path: String) { - for circuit_type in 1..=18 { - let filename = format!("{}/verification_{}.key", output_path, circuit_type); - let vk = get_vk_for_circuit_type(circuit_type); - let mut f = BufWriter::new(File::create(filename).unwrap()); - serialize_into(&mut f, &vk).unwrap(); - } -} diff --git a/core/bin/verification_key_generator_and_server/src/lib.rs b/core/bin/verification_key_generator_and_server/src/lib.rs deleted file mode 100644 index 20260a30b20b..000000000000 --- a/core/bin/verification_key_generator_and_server/src/lib.rs +++ /dev/null @@ -1,188 +0,0 @@ -use std::{collections::HashMap, path::Path, str::FromStr}; - -use ff::to_hex; -use itertools::Itertools; -use once_cell::sync::Lazy; -use structopt::lazy_static::lazy_static; -use zksync_types::{ - circuit::{ - GEOMETRY_CONFIG, LEAF_CIRCUIT_INDEX, LEAF_SPLITTING_FACTOR, NODE_CIRCUIT_INDEX, - NODE_SPLITTING_FACTOR, SCHEDULER_CIRCUIT_INDEX, SCHEDULER_UPPER_BOUND, - }, - protocol_version::{L1VerifierConfig, VerifierParams}, - vk_transform::generate_vk_commitment, - zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncCircuit, - bellman::{bn256::Bn256, plonk::better_better_cs::setup::VerificationKey}, - witness, - witness::{ - full_block_artifact::BlockBasicCircuits, - oracle::VmWitnessOracle, - recursive_aggregation::{erase_vk_type, padding_aggregations}, - vk_set_generator::circuits_for_vk_generation, - }, - }, - H256, -}; - -#[cfg(test)] -mod tests; - -lazy_static! { - static ref COMMITMENTS: Lazy = Lazy::new(|| { circuit_commitments() }); -} - -pub fn get_vks_for_basic_circuits( -) -> HashMap>>> { - // 3-17 are the ids of basic circuits - (3..=18) - .map(|circuit_type| (circuit_type, get_vk_for_circuit_type(circuit_type))) - .collect() -} - -pub fn get_vk_for_circuit_type( - circuit_type: u8, -) -> VerificationKey>> { - let filepath = get_file_path(circuit_type); - tracing::info!("Fetching verification key from path: {}", filepath); - let text = std::fs::read_to_string(&filepath) - .unwrap_or_else(|_| panic!("Failed reading verification key from path: {}", filepath)); - serde_json::from_str::>>>( - &text, - ) - .unwrap_or_else(|_| { - panic!( - "Failed deserializing verification key from path: {}", - filepath - ) - }) -} - -pub fn save_vk_for_circuit_type( - circuit_type: u8, - vk: VerificationKey>>, -) { - let filepath = get_file_path(circuit_type); - tracing::info!("saving verification key to: {}", filepath); - std::fs::write(filepath, serde_json::to_string_pretty(&vk).unwrap()).unwrap(); -} - -pub fn get_ordered_vks_for_basic_circuits( - circuits: &BlockBasicCircuits, - verification_keys: &HashMap< - u8, - VerificationKey>>, - >, -) -> Vec>>> { - circuits - .clone() - .into_flattened_set() - .iter() - .map(|circuit| { - let circuit_id = circuit.numeric_circuit_type(); - verification_keys - .get(&circuit_id) - .unwrap_or_else(|| { - panic!("no VK for circuit number {:?}", circuit.short_description()) - }) - .clone() - }) - .collect() -} - -pub fn get_vks_for_commitment( - verification_keys: HashMap< - u8, - VerificationKey>>, - >, -) -> Vec>>> { - // We need all the vks sorted by their respective circuit ids - verification_keys - .into_iter() - .sorted_by_key(|(id, _)| *id) - .map(|(_, val)| val) - .collect() -} - -pub fn get_circuits_for_vk() -> Vec>> { - ensure_setup_key_exist(); - let padding_aggregations = padding_aggregations(NODE_SPLITTING_FACTOR); - circuits_for_vk_generation( - GEOMETRY_CONFIG, - LEAF_SPLITTING_FACTOR, - NODE_SPLITTING_FACTOR, - SCHEDULER_UPPER_BOUND, - padding_aggregations, - ) -} - -fn ensure_setup_key_exist() { - if !Path::new("setup_2^26.key").exists() { - panic!("File setup_2^26.key is required to be present in current directory for verification keys generation. \ndownload from https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^26.key"); - } -} -fn get_file_path(circuit_type: u8) -> String { - let zksync_home = std::env::var("ZKSYNC_HOME").unwrap_or_else(|_| "/".into()); - format!( - "{}/core/bin/verification_key_generator_and_server/data/verification_{}_key.json", - zksync_home, circuit_type - ) -} - -pub fn generate_commitments() -> (String, String, String) { - let (_, basic_circuit_commitment, _) = - witness::recursive_aggregation::form_base_circuits_committment(get_vks_for_commitment( - get_vks_for_basic_circuits(), - )); - - let leaf_aggregation_vk = get_vk_for_circuit_type(LEAF_CIRCUIT_INDEX); - let node_aggregation_vk = get_vk_for_circuit_type(NODE_CIRCUIT_INDEX); - - let (_, leaf_aggregation_vk_commitment) = - witness::recursive_aggregation::compute_vk_encoding_and_committment(erase_vk_type( - leaf_aggregation_vk, - )); - - let (_, node_aggregation_vk_commitment) = - witness::recursive_aggregation::compute_vk_encoding_and_committment(erase_vk_type( - node_aggregation_vk, - )); - let basic_circuit_commitment_hex = format!("0x{}", to_hex(&basic_circuit_commitment)); - let leaf_aggregation_commitment_hex = format!("0x{}", to_hex(&leaf_aggregation_vk_commitment)); - let node_aggregation_commitment_hex = format!("0x{}", to_hex(&node_aggregation_vk_commitment)); - tracing::info!( - "basic circuit commitment {:?}", - basic_circuit_commitment_hex - ); - tracing::info!( - "leaf aggregation commitment {:?}", - leaf_aggregation_commitment_hex - ); - tracing::info!( - "node aggregation commitment {:?}", - node_aggregation_commitment_hex - ); - ( - basic_circuit_commitment_hex, - leaf_aggregation_commitment_hex, - node_aggregation_commitment_hex, - ) -} - -fn circuit_commitments() -> L1VerifierConfig { - let (basic, leaf, node) = generate_commitments(); - let scheduler = generate_vk_commitment(get_vk_for_circuit_type(SCHEDULER_CIRCUIT_INDEX)); - L1VerifierConfig { - params: VerifierParams { - recursion_node_level_vk_hash: H256::from_str(&node).expect("invalid node commitment"), - recursion_leaf_level_vk_hash: H256::from_str(&leaf).expect("invalid leaf commitment"), - recursion_circuits_set_vks_hash: H256::from_str(&basic) - .expect("invalid basic commitment"), - }, - recursion_scheduler_level_vk_hash: scheduler, - } -} - -pub fn get_cached_commitments() -> L1VerifierConfig { - **COMMITMENTS -} diff --git a/core/bin/verification_key_generator_and_server/src/main.rs b/core/bin/verification_key_generator_and_server/src/main.rs deleted file mode 100644 index b64e5757fceb..000000000000 --- a/core/bin/verification_key_generator_and_server/src/main.rs +++ /dev/null @@ -1,48 +0,0 @@ -use std::{collections::HashSet, env}; - -use zksync_types::zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncCircuit, - bellman::{ - bn256::Bn256, plonk::better_better_cs::cs::PlonkCsWidth4WithNextStepAndCustomGatesParams, - }, - witness::oracle::VmWitnessOracle, -}; -use zksync_verification_key_server::{get_circuits_for_vk, save_vk_for_circuit_type}; - -/// Creates verification keys for the given circuit. -fn main() { - let args: Vec = env::args().collect(); - - let circuit_types: HashSet = if args.len() > 1 { - [get_and_ensure_valid_circuit_type(args[1].clone())].into() - } else { - (3..17).collect() - }; - tracing::info!("Starting verification key generation!"); - get_circuits_for_vk() - .into_iter() - .filter(|c| circuit_types.contains(&c.numeric_circuit_type())) - .for_each(generate_verification_key); -} - -fn get_and_ensure_valid_circuit_type(circuit_type: String) -> u8 { - tracing::info!("Received circuit_type: {:?}", circuit_type); - circuit_type - .parse::() - .expect("Please specify a circuit type in range [1, 17]") -} - -fn generate_verification_key(circuit: ZkSyncCircuit>) { - let res = circuit_testing::create_vk_for_padding_size_log_2::< - Bn256, - _, - PlonkCsWidth4WithNextStepAndCustomGatesParams, - >(circuit.clone(), 26) - .unwrap(); - save_vk_for_circuit_type(circuit.numeric_circuit_type(), res); - tracing::info!( - "Finished VK generation for circuit {:?} (id {:?})", - circuit.short_description(), - circuit.numeric_circuit_type() - ); -} diff --git a/core/bin/verification_key_generator_and_server/src/tests.rs b/core/bin/verification_key_generator_and_server/src/tests.rs deleted file mode 100644 index f0fea866de6d..000000000000 --- a/core/bin/verification_key_generator_and_server/src/tests.rs +++ /dev/null @@ -1,68 +0,0 @@ -use std::collections::HashMap; - -use itertools::Itertools; -use serde_json::Value; -use zksync_types::zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncCircuit, - bellman::{bn256::Bn256, plonk::better_better_cs::setup::VerificationKey}, - witness::oracle::VmWitnessOracle, -}; - -use crate::{get_vk_for_circuit_type, get_vks_for_basic_circuits, get_vks_for_commitment}; - -#[test] -fn test_get_vk_for_circuit_type() { - for circuit_type in 1..=18 { - get_vk_for_circuit_type(circuit_type); - } -} - -#[test] -fn test_get_vks_for_basic_circuits() { - let circuit_type_to_vk = get_vks_for_basic_circuits(); - let circuit_types: Vec = circuit_type_to_vk.into_keys().sorted().collect::>(); - let expected: Vec = (3..=18).collect(); - assert_eq!( - expected, circuit_types, - "circuit types must be in the range [3, 17]" - ); -} - -#[test] -fn test_get_vks_for_commitment() { - let vk_5 = get_vk_for_circuit_type(5); - let vk_2 = get_vk_for_circuit_type(2); - let vk_3 = get_vk_for_circuit_type(3); - let map = HashMap::from([ - (5u8, vk_5.clone()), - (2u8, vk_2.clone()), - (3u8, vk_3.clone()), - ]); - let vks = get_vks_for_commitment(map); - let expected = vec![vk_2, vk_3, vk_5]; - compare_vks( - expected, - vks, - "expected verification key to be in order 2, 3, 5", - ); -} - -fn get_vk_json(vk: &VerificationKey>>) -> Value { - serde_json::to_value(vk).unwrap() -} - -fn get_vk_jsons( - vks: Vec>>>, -) -> Vec { - vks.into_iter().map(|vk| get_vk_json(&vk)).collect() -} - -fn compare_vks( - first: Vec>>>, - second: Vec>>>, - error_message: &str, -) { - let first_json = get_vk_jsons(first); - let second_json = get_vk_jsons(second); - assert_eq!(first_json, second_json, "{:?}", error_message); -} diff --git a/core/bin/zksync_server/src/main.rs b/core/bin/zksync_server/src/main.rs index a7825da919a2..ffaa08ea090d 100644 --- a/core/bin/zksync_server/src/main.rs +++ b/core/bin/zksync_server/src/main.rs @@ -12,10 +12,10 @@ use zksync_config::{ fri_prover_group::FriProverGroupConfig, house_keeper::HouseKeeperConfig, FriProofCompressorConfig, FriProverConfig, FriWitnessGeneratorConfig, PrometheusConfig, - ProofDataHandlerConfig, ProverGroupConfig, WitnessGeneratorConfig, + ProofDataHandlerConfig, WitnessGeneratorConfig, }, ApiConfig, ContractsConfig, DBConfig, ETHClientConfig, ETHSenderConfig, ETHWatchConfig, - GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, ProverConfigs, + GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, }; use zksync_core::{ genesis_init, initialize_components, is_genesis_needed, setup_sigint_handler, @@ -110,7 +110,6 @@ async fn main() -> anyhow::Result<()> { fri_witness_generator_config: FriWitnessGeneratorConfig::from_env().ok(), prometheus_config: PrometheusConfig::from_env().ok(), proof_data_handler_config: ProofDataHandlerConfig::from_env().ok(), - prover_group_config: ProverGroupConfig::from_env().ok(), witness_generator_config: WitnessGeneratorConfig::from_env().ok(), api_config: ApiConfig::from_env().ok(), contracts_config: ContractsConfig::from_env().ok(), @@ -119,7 +118,6 @@ async fn main() -> anyhow::Result<()> { eth_sender_config: ETHSenderConfig::from_env().ok(), eth_watch_config: ETHWatchConfig::from_env().ok(), gas_adjuster_config: GasAdjusterConfig::from_env().ok(), - prover_configs: ProverConfigs::from_env().ok(), object_store_config: ObjectStoreConfig::from_env().ok(), }; diff --git a/core/lib/config/src/configs/fri_prover.rs b/core/lib/config/src/configs/fri_prover.rs index 44521ee36576..a5e99a407374 100644 --- a/core/lib/config/src/configs/fri_prover.rs +++ b/core/lib/config/src/configs/fri_prover.rs @@ -22,6 +22,7 @@ pub struct FriProverConfig { pub witness_vector_generator_thread_count: Option, pub queue_capacity: usize, pub witness_vector_receiver_port: u16, + pub zone_read_url: String, // whether to write to public GCS bucket for https://github.com/matter-labs/era-boojum-validator-cli pub shall_save_to_public_bucket: bool, diff --git a/core/lib/config/src/configs/mod.rs b/core/lib/config/src/configs/mod.rs index f6f58f012388..3066ee5d5c69 100644 --- a/core/lib/config/src/configs/mod.rs +++ b/core/lib/config/src/configs/mod.rs @@ -16,8 +16,6 @@ pub use self::{ fri_witness_vector_generator::FriWitnessVectorGeneratorConfig, object_store::ObjectStoreConfig, proof_data_handler::ProofDataHandlerConfig, - prover::{ProverConfig, ProverConfigs}, - prover_group::ProverGroupConfig, snapshots_creator::SnapshotsCreatorConfig, utils::PrometheusConfig, witness_generator::WitnessGeneratorConfig, @@ -41,8 +39,6 @@ pub mod fri_witness_vector_generator; pub mod house_keeper; pub mod object_store; pub mod proof_data_handler; -pub mod prover; -pub mod prover_group; pub mod snapshots_creator; pub mod utils; pub mod witness_generator; diff --git a/core/lib/config/src/configs/prover.rs b/core/lib/config/src/configs/prover.rs deleted file mode 100644 index 45ed7100f9f4..000000000000 --- a/core/lib/config/src/configs/prover.rs +++ /dev/null @@ -1,61 +0,0 @@ -use std::time::Duration; - -use serde::Deserialize; - -/// Configuration for the prover application -#[derive(Debug, Deserialize, Clone, PartialEq)] -pub struct ProverConfig { - /// Port to which the Prometheus exporter server is listening. - pub prometheus_port: u16, - /// Currently only a single (largest) key is supported. We'll support different ones in the future - pub initial_setup_key_path: String, - /// https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2\^26.key - pub key_download_url: String, - /// Max time for proof to be generated - pub generation_timeout_in_secs: u16, - /// Number of threads to be used concurrent proof generation. - pub number_of_threads: u16, - /// Max attempts for generating proof - pub max_attempts: u32, - // Polling time in mill-seconds. - pub polling_duration_in_millis: u64, - // Path to setup keys for individual circuit. - pub setup_keys_path: String, - // Group id for this prover, provers running the same circuit types shall have same group id. - pub specialized_prover_group_id: u8, - // Number of setup-keys kept in memory without swapping - // number_of_setup_slots = (R-C*A-4)/S - // R is available ram - // C is the number of parallel synth - // A is the size of Assembly that is 12gb - // S is the size of the Setup that is 20gb - // constant 4 is for the data copy with gpu - pub number_of_setup_slots: u8, - /// Port at which server would be listening to receive incoming assembly - pub assembly_receiver_port: u16, - /// Socket polling time for receiving incoming assembly - pub assembly_receiver_poll_time_in_millis: u64, - /// maximum number of assemblies that are kept in memory, - pub assembly_queue_capacity: usize, -} - -/// Prover configs for different machine types that are currently supported. -#[derive(Debug, Deserialize, Clone, PartialEq)] -pub struct ProverConfigs { - // used by witness-generator - pub non_gpu: ProverConfig, - // https://gcloud-compute.com/a2-highgpu-2g.html - pub two_gpu_forty_gb_mem: ProverConfig, - // https://gcloud-compute.com/a2-ultragpu-1g.html - pub one_gpu_eighty_gb_mem: ProverConfig, - // https://gcloud-compute.com/a2-ultragpu-2g.html - pub two_gpu_eighty_gb_mem: ProverConfig, - // https://gcloud-compute.com/a2-ultragpu-4g.html - pub four_gpu_eighty_gb_mem: ProverConfig, -} - -impl ProverConfig { - pub fn proof_generation_timeout(&self) -> Duration { - Duration::from_secs(self.generation_timeout_in_secs as u64) - } -} diff --git a/core/lib/config/src/configs/prover_group.rs b/core/lib/config/src/configs/prover_group.rs deleted file mode 100644 index 2d40d47ba8c1..000000000000 --- a/core/lib/config/src/configs/prover_group.rs +++ /dev/null @@ -1,66 +0,0 @@ -use serde::Deserialize; - -/// Configuration for the grouping of specialized provers. -/// This config would be used by circuit-synthesizer and provers. -#[derive(Debug, Deserialize, Clone, PartialEq)] -pub struct ProverGroupConfig { - pub group_0_circuit_ids: Vec, - pub group_1_circuit_ids: Vec, - pub group_2_circuit_ids: Vec, - pub group_3_circuit_ids: Vec, - pub group_4_circuit_ids: Vec, - pub group_5_circuit_ids: Vec, - pub group_6_circuit_ids: Vec, - pub group_7_circuit_ids: Vec, - pub group_8_circuit_ids: Vec, - pub group_9_circuit_ids: Vec, - pub region_read_url: String, - // This is used while running the provers/synthesizer in non-gcp cloud env. - pub region_override: Option, - pub zone_read_url: String, - // This is used while running the provers/synthesizer in non-gcp cloud env. - pub zone_override: Option, - pub synthesizer_per_gpu: u16, -} - -impl ProverGroupConfig { - pub fn get_circuit_ids_for_group_id(&self, group_id: u8) -> Option> { - match group_id { - 0 => Some(self.group_0_circuit_ids.clone()), - 1 => Some(self.group_1_circuit_ids.clone()), - 2 => Some(self.group_2_circuit_ids.clone()), - 3 => Some(self.group_3_circuit_ids.clone()), - 4 => Some(self.group_4_circuit_ids.clone()), - 5 => Some(self.group_5_circuit_ids.clone()), - 6 => Some(self.group_6_circuit_ids.clone()), - 7 => Some(self.group_7_circuit_ids.clone()), - 8 => Some(self.group_8_circuit_ids.clone()), - 9 => Some(self.group_9_circuit_ids.clone()), - _ => None, - } - } - - pub fn is_specialized_group_id(&self, group_id: u8) -> bool { - group_id <= 9 - } - - pub fn get_group_id_for_circuit_id(&self, circuit_id: u8) -> Option { - let configs = [ - &self.group_0_circuit_ids, - &self.group_1_circuit_ids, - &self.group_2_circuit_ids, - &self.group_3_circuit_ids, - &self.group_4_circuit_ids, - &self.group_5_circuit_ids, - &self.group_6_circuit_ids, - &self.group_7_circuit_ids, - &self.group_8_circuit_ids, - &self.group_9_circuit_ids, - ]; - configs - .iter() - .enumerate() - .find(|(_, group)| group.contains(&circuit_id)) - .map(|(group_id, _)| group_id as u8) - } -} diff --git a/core/lib/config/src/lib.rs b/core/lib/config/src/lib.rs index edb596be4984..f329fbf4f4b4 100644 --- a/core/lib/config/src/lib.rs +++ b/core/lib/config/src/lib.rs @@ -3,7 +3,7 @@ pub use crate::configs::{ ApiConfig, ChainConfig, ContractVerifierConfig, ContractsConfig, DBConfig, ETHClientConfig, ETHSenderConfig, ETHWatchConfig, GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, - ProverConfig, ProverConfigs, SnapshotsCreatorConfig, + SnapshotsCreatorConfig, }; pub mod configs; diff --git a/core/lib/dal/sqlx-data.json b/core/lib/dal/sqlx-data.json index 36b8bad5787f..1caa9f54a1e8 100644 --- a/core/lib/dal/sqlx-data.json +++ b/core/lib/dal/sqlx-data.json @@ -391,19 +391,6 @@ }, "query": "\n SELECT\n *\n FROM\n call_traces\n WHERE\n tx_hash IN (\n SELECT\n hash\n FROM\n transactions\n WHERE\n miniblock_number = $1\n )\n " }, - "0587fadb4f7a014caddf9e540cd2a1ece830de8777d945d48bd9c796fefb3253": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n id = $2\n " - }, "07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa": { "describe": { "columns": [ @@ -426,24 +413,6 @@ }, "query": "INSERT INTO eth_txs_history (eth_tx_id, base_fee_per_gas, priority_fee_per_gas, tx_hash, signed_raw_tx, created_at, updated_at, confirmed_at) VALUES ($1, 0, 0, $2, '\\x00', now(), now(), $3) RETURNING id" }, - "08024b2ba970d2fdac2a71c9c7c73be42a289d034670ca4e644f5df1e4614ddf": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text", - "Int4", - "Bytea", - "Int4", - "Text", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n prover_jobs (\n l1_batch_number,\n circuit_type,\n sequence_number,\n prover_input,\n aggregation_round,\n circuit_input_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number, aggregation_round, sequence_number) DO NOTHING\n " - }, "083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643": { "describe": { "columns": [ @@ -982,30 +951,6 @@ }, "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = 'reserved',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n gpu_prover_queue_fri\n WHERE\n specialized_prover_group_id = $2\n AND zone = $3\n AND (\n instance_status = 'available'\n OR (\n instance_status = 'reserved'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n )\n ORDER BY\n updated_at ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n gpu_prover_queue_fri.*\n " }, - "0e1317c908de3d9b9b87b51802cbe545198d7debecd65dc2165731c8a0c0f508": { - "describe": { - "columns": [ - { - "name": "l1_batch_number!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_type", - "ordinal": 1, - "type_info": "Text" - } - ], - "nullable": [ - null, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MIN(l1_batch_number) AS \"l1_batch_number!\",\n circuit_type\n FROM\n prover_jobs\n WHERE\n aggregation_round = 0\n AND (\n status = 'queued'\n OR status = 'in_progress'\n OR status = 'in_gpu_proof'\n OR status = 'failed'\n )\n GROUP BY\n circuit_type\n " - }, "10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730": { "describe": { "columns": [ @@ -3654,32 +3599,6 @@ }, "query": "\n UPDATE l1_batches\n SET\n commitment = $2,\n aux_data_hash = $3,\n updated_at = NOW()\n WHERE\n number = $1\n " }, - "3263423b6097fac01eadd978b826b831321c10f91b87cea38dc8a7377da9385e": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_input_blob_url", - "ordinal": 1, - "type_info": "Text" - } - ], - "nullable": [ - false, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n id,\n circuit_input_blob_url\n FROM\n prover_jobs\n WHERE\n status = 'successful'\n AND circuit_input_blob_url IS NOT NULL\n AND updated_at < NOW() - INTERVAL '30 days'\n LIMIT\n $1;\n " - }, "32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732": { "describe": { "columns": [], @@ -3762,21 +3681,6 @@ }, "query": "\n SELECT\n hashed_key,\n l1_batch_number,\n INDEX\n FROM\n initial_writes\n WHERE\n hashed_key = ANY ($1::bytea[])\n " }, - "3502a673e04b57bfde096303d7643092702c835069cc055e01f382bc56681401": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Time", - "Bytea", - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1,\n result = $2,\n proccesed_by = $3\n WHERE\n id = $4\n " - }, "35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530": { "describe": { "columns": [], @@ -4066,26 +3970,6 @@ }, "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n miniblocks\n WHERE\n number = $1\n AND consensus IS NOT NULL\n " }, - "373f6339a61c6ac74080f855fcc25dab33355eefdce69255bc7106675b0e5641": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n prover_protocol_versions\n WHERE\n id = $1\n " - }, "38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c": { "describe": { "columns": [ @@ -4235,24 +4119,6 @@ }, "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n l1_batch_number = $1\n AND status != 'successful'\n AND status != 'in_progress'\n " }, - "40c82325e05572db9c3a4ca8cc347617ed18495ef147b3ecfacdd89f54957b6a": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n prover_protocol_versions (\n id,\n timestamp,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n verifier_address,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, NOW())\n " - }, "41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151": { "describe": { "columns": [ @@ -4739,30 +4605,6 @@ }, "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n " }, - "4f9b84e4ee54902edb3738ec111268d1266a05f4d931dd874baceedf5444efa4": { - "describe": { - "columns": [ - { - "name": "l1_batch_number!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "aggregation_round", - "ordinal": 1, - "type_info": "Int4" - } - ], - "nullable": [ - null, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(l1_batch_number) AS \"l1_batch_number!\",\n aggregation_round\n FROM\n prover_jobs\n WHERE\n status = 'successful'\n GROUP BY\n aggregation_round\n " - }, "525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63": { "describe": { "columns": [ @@ -5724,30 +5566,37 @@ }, "query": "\n INSERT INTO\n eth_txs (\n raw_tx,\n nonce,\n tx_type,\n contract_address,\n predicted_gas_cost,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW(), NOW())\n RETURNING\n *\n " }, - "684775aaed3d7f3f5580363e5180a04e7a1af1057995805cb6fd35d0b810e734": { + "68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d": { "describe": { - "columns": [], - "nullable": [], + "columns": [ + { + "name": "count!", + "ordinal": 0, + "type_info": "Int8" + } + ], + "nullable": [ + null + ], "parameters": { "Left": [ - "Text", - "Text", - "Int4", - "Int4", - "Text", - "Text" + "Int8", + "Bytea", + "Bytea", + "Bytea", + "Bytea" ] } }, - "query": "\n UPDATE gpu_prover_queue\n SET\n instance_status = $1,\n updated_at = NOW(),\n queue_free_slots = $4\n WHERE\n instance_host = $2::TEXT::inet\n AND instance_port = $3\n AND region = $5\n AND zone = $6\n " + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n WHERE\n number = $1\n AND hash = $2\n AND merkle_root_hash = $3\n AND parent_hash = $4\n AND l2_l1_merkle_root = $5\n " }, - "68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d": { + "68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7": { "describe": { "columns": [ { - "name": "count!", + "name": "is_replaced!", "ordinal": 0, - "type_info": "Int8" + "type_info": "Bool" } ], "nullable": [ @@ -5755,255 +5604,109 @@ ], "parameters": { "Left": [ + "Bytea", + "Bytea", "Int8", "Bytea", + "Numeric", + "Numeric", + "Numeric", + "Numeric", + "Bytea", + "Jsonb", + "Int4", + "Bytea", + "Numeric", "Bytea", "Bytea", - "Bytea" + "Int8", + "Int4", + "Int4", + "Timestamp" ] } }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n WHERE\n number = $1\n AND hash = $2\n AND merkle_root_hash = $3\n AND parent_hash = $4\n AND l2_l1_merkle_root = $5\n " + "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n nonce,\n signature,\n gas_limit,\n max_fee_per_gas,\n max_priority_fee_per_gas,\n gas_per_pubdata_limit,\n input,\n data,\n tx_format,\n contract_address,\n value,\n paymaster,\n paymaster_input,\n execution_info,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n FALSE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n $19,\n NOW(),\n NOW()\n )\n ON CONFLICT (initiator_address, nonce) DO\n UPDATE\n SET\n hash = $1,\n signature = $4,\n gas_limit = $5,\n max_fee_per_gas = $6,\n max_priority_fee_per_gas = $7,\n gas_per_pubdata_limit = $8,\n input = $9,\n data = $10,\n tx_format = $11,\n contract_address = $12,\n value = $13,\n paymaster = $14,\n paymaster_input = $15,\n execution_info = JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n in_mempool = FALSE,\n received_at = $19,\n created_at = NOW(),\n updated_at = NOW(),\n error = NULL\n WHERE\n transactions.is_priority = FALSE\n AND transactions.miniblock_number IS NULL\n RETURNING\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n transactions.initiator_address = $2\n AND transactions.nonce = $3\n ) IS NOT NULL AS \"is_replaced!\"\n " + }, + "6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [ + "Text", + "Int4", + "Int2", + "Text" + ] + } + }, + "query": "\n INSERT INTO\n gpu_prover_queue_fri (\n instance_host,\n instance_port,\n instance_status,\n specialized_prover_group_id,\n zone,\n created_at,\n updated_at\n )\n VALUES\n (CAST($1::TEXT AS inet), $2, 'available', $3, $4, NOW(), NOW())\n ON CONFLICT (instance_host, instance_port, zone) DO\n UPDATE\n SET\n instance_status = 'available',\n specialized_prover_group_id = $3,\n zone = $4,\n updated_at = NOW()\n " + }, + "6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [] + } + }, + "query": "\n DELETE FROM eth_txs\n WHERE\n id >= (\n SELECT\n MIN(id)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n )\n " + }, + "6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [] + } + }, + "query": "\n DELETE FROM basic_witness_input_producer_jobs\n " }, - "68a937c42e280690a7a63eeec6883d30eeb6e614ca75edf582b44378c0a698ed": { + "6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [ + "Int4" + ] + } + }, + "query": "\n UPDATE miniblocks\n SET\n protocol_version = $1\n WHERE\n l1_batch_number IS NULL\n " + }, + "6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6": { "describe": { "columns": [ { - "name": "id", + "name": "number", "ordinal": 0, "type_info": "Int8" }, { - "name": "l1_batch_number", + "name": "timestamp", "ordinal": 1, "type_info": "Int8" }, { - "name": "circuit_type", + "name": "hash", "ordinal": 2, - "type_info": "Text" + "type_info": "Bytea" }, { - "name": "prover_input", + "name": "l1_tx_count", "ordinal": 3, - "type_info": "Bytea" + "type_info": "Int4" }, { - "name": "status", + "name": "l2_tx_count", "ordinal": 4, - "type_info": "Text" + "type_info": "Int4" }, { - "name": "error", + "name": "base_fee_per_gas", "ordinal": 5, - "type_info": "Text" - }, - { - "name": "processing_started_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "created_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 8, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 9, - "type_info": "Time" - }, - { - "name": "aggregation_round", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "result", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "sequence_number", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "attempts", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "circuit_input_blob_url", - "ordinal": 14, - "type_info": "Text" - }, - { - "name": "proccesed_by", - "ordinal": 15, - "type_info": "Text" - }, - { - "name": "is_blob_cleaned", - "ordinal": 16, - "type_info": "Bool" - }, - { - "name": "protocol_version", - "ordinal": 17, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - true, - false, - false, - false, - false, - true, - false, - false, - true, - true, - false, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n prover_jobs\n WHERE\n id = $1\n " - }, - "68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7": { - "describe": { - "columns": [ - { - "name": "is_replaced!", - "ordinal": 0, - "type_info": "Bool" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Int8", - "Bytea", - "Numeric", - "Numeric", - "Numeric", - "Numeric", - "Bytea", - "Jsonb", - "Int4", - "Bytea", - "Numeric", - "Bytea", - "Bytea", - "Int8", - "Int4", - "Int4", - "Timestamp" - ] - } - }, - "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n nonce,\n signature,\n gas_limit,\n max_fee_per_gas,\n max_priority_fee_per_gas,\n gas_per_pubdata_limit,\n input,\n data,\n tx_format,\n contract_address,\n value,\n paymaster,\n paymaster_input,\n execution_info,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n FALSE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n $19,\n NOW(),\n NOW()\n )\n ON CONFLICT (initiator_address, nonce) DO\n UPDATE\n SET\n hash = $1,\n signature = $4,\n gas_limit = $5,\n max_fee_per_gas = $6,\n max_priority_fee_per_gas = $7,\n gas_per_pubdata_limit = $8,\n input = $9,\n data = $10,\n tx_format = $11,\n contract_address = $12,\n value = $13,\n paymaster = $14,\n paymaster_input = $15,\n execution_info = JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n in_mempool = FALSE,\n received_at = $19,\n created_at = NOW(),\n updated_at = NOW(),\n error = NULL\n WHERE\n transactions.is_priority = FALSE\n AND transactions.miniblock_number IS NULL\n RETURNING\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n transactions.initiator_address = $2\n AND transactions.nonce = $3\n ) IS NOT NULL AS \"is_replaced!\"\n " - }, - "6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int4", - "Int2", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n gpu_prover_queue_fri (\n instance_host,\n instance_port,\n instance_status,\n specialized_prover_group_id,\n zone,\n created_at,\n updated_at\n )\n VALUES\n (CAST($1::TEXT AS inet), $2, 'available', $3, $4, NOW(), NOW())\n ON CONFLICT (instance_host, instance_port, zone) DO\n UPDATE\n SET\n instance_status = 'available',\n specialized_prover_group_id = $3,\n zone = $4,\n updated_at = NOW()\n " - }, - "6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n DELETE FROM eth_txs\n WHERE\n id >= (\n SELECT\n MIN(id)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n )\n " - }, - "6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n DELETE FROM basic_witness_input_producer_jobs\n " - }, - "6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n UPDATE miniblocks\n SET\n protocol_version = $1\n WHERE\n l1_batch_number IS NULL\n " - }, - "6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "base_fee_per_gas", - "ordinal": 5, - "type_info": "Numeric" + "type_info": "Numeric" }, { "name": "l1_gas_price", @@ -6031,98 +5734,9 @@ "type_info": "Int4" }, { - "name": "virtual_blocks", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "6f6f60e7139fc789ca420d8610985a918e90b4e7087a98356ab19e22783c88cd": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "instance_host", - "ordinal": 1, - "type_info": "Inet" - }, - { - "name": "instance_port", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "instance_status", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 4, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 5, - "type_info": "Timestamp" - }, - { - "name": "processing_started_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "queue_free_slots", - "ordinal": 7, - "type_info": "Int4" - }, - { - "name": "queue_capacity", - "ordinal": 8, - "type_info": "Int4" - }, - { - "name": "specialized_prover_group_id", - "ordinal": 9, - "type_info": "Int2" - }, - { - "name": "region", - "ordinal": 10, - "type_info": "Text" - }, - { - "name": "zone", - "ordinal": 11, - "type_info": "Text" - }, - { - "name": "num_gpu", - "ordinal": 12, - "type_info": "Int2" + "name": "virtual_blocks", + "ordinal": 11, + "type_info": "Int8" } ], "nullable": [ @@ -6132,24 +5746,18 @@ false, false, false, + false, + false, true, true, true, - true, - false, - false, - true + false ], "parameters": { - "Left": [ - "Interval", - "Int2", - "Text", - "Text" - ] + "Left": [] } }, - "query": "\n UPDATE gpu_prover_queue\n SET\n instance_status = 'reserved',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n gpu_prover_queue\n WHERE\n specialized_prover_group_id = $2\n AND region = $3\n AND zone = $4\n AND (\n instance_status = 'available'\n OR (\n instance_status = 'reserved'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n )\n ORDER BY\n updated_at ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n gpu_prover_queue.*\n " + "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n ORDER BY\n number DESC\n LIMIT\n 1\n " }, "708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d": { "describe": { @@ -7130,24 +6738,6 @@ }, "query": "\n INSERT INTO\n snapshots (\n l1_batch_number,\n storage_logs_filepaths,\n factory_deps_filepath,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, NOW(), NOW())\n " }, - "831f7bec105541bd3ff9bcf6940d6b6b9d558224ad2d8ed079a68c7e339ded6b": { - "describe": { - "columns": [ - { - "name": "l1_batch_number?", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MIN(l1_batch_number) AS \"l1_batch_number?\"\n FROM\n (\n SELECT\n MIN(l1_batch_number) AS \"l1_batch_number\"\n FROM\n prover_jobs\n WHERE\n status = 'successful'\n OR aggregation_round < 3\n GROUP BY\n l1_batch_number\n HAVING\n MAX(aggregation_round) < 3\n ) AS inn\n " - }, "83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8": { "describe": { "columns": [ @@ -7785,36 +7375,6 @@ }, "query": "\n SELECT\n protocol_version\n FROM\n miniblocks\n WHERE\n number = $1\n " }, - "8a773618c9df11217467222c9117d6868fbf88ee21d8868a7d133e7cebb3d20e": { - "describe": { - "columns": [ - { - "name": "successful_limit!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "queued_limit!", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "max_block!", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - null, - null, - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n (\n SELECT\n l1_batch_number\n FROM\n prover_jobs\n WHERE\n status NOT IN ('successful', 'skipped')\n ORDER BY\n l1_batch_number\n LIMIT\n 1\n ) AS \"successful_limit!\",\n (\n SELECT\n l1_batch_number\n FROM\n prover_jobs\n WHERE\n status <> 'queued'\n ORDER BY\n l1_batch_number DESC\n LIMIT\n 1\n ) AS \"queued_limit!\",\n (\n SELECT\n MAX(l1_batch_number) AS \"max!\"\n FROM\n prover_jobs\n ) AS \"max_block!\"\n " - }, "8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db": { "describe": { "columns": [ @@ -8187,36 +7747,6 @@ }, "query": "\n INSERT INTO\n witness_inputs_fri (\n l1_batch_number,\n merkle_tree_paths_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " }, - "9c0c3e5edce083804f49137eb3b01c0b73dfb30bdb9e11fcbf370d599344f20e": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_type!", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "status!", - "ordinal": 2, - "type_info": "Text" - } - ], - "nullable": [ - null, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\",\n circuit_type AS \"circuit_type!\",\n status AS \"status!\"\n FROM\n prover_jobs\n WHERE\n status <> 'skipped'\n AND status <> 'successful'\n GROUP BY\n circuit_type,\n status\n " - }, "9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86": { "describe": { "columns": [ @@ -8585,22 +8115,6 @@ }, "query": "\n SELECT\n (\n SELECT\n l1_batch_number\n FROM\n miniblocks\n WHERE\n number = $1\n ) AS \"block_batch?\",\n (\n SELECT\n MAX(number) + 1\n FROM\n l1_batches\n ) AS \"max_batch?\"\n " }, - "ad53e912e68d81628089ae68aaa4154b988ce8ed67af02f4254717a1cdd3da7e": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int4", - "Int4", - "Text", - "Text" - ] - } - }, - "query": "\n UPDATE gpu_prover_queue\n SET\n instance_status = 'available',\n updated_at = NOW(),\n queue_free_slots = $3\n WHERE\n instance_host = $1::TEXT::inet\n AND instance_port = $2\n AND instance_status = 'full'\n AND region = $4\n AND zone = $5\n " - }, "ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2": { "describe": { "columns": [ @@ -9596,29 +9110,6 @@ }, "query": "\n SELECT\n address,\n key,\n value\n FROM\n storage_logs\n WHERE\n miniblock_number BETWEEN (\n SELECT\n MIN(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n ) AND (\n SELECT\n MAX(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n )\n ORDER BY\n miniblock_number,\n operation_number\n " }, - "c3b76b8030d4b5266242619f091c5cffe6869add3e43c71390a9a921b8ff48c5": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n id\n FROM\n prover_protocol_versions\n WHERE\n recursion_circuits_set_vks_hash = $1\n AND recursion_leaf_level_vk_hash = $2\n AND recursion_node_level_vk_hash = $3\n AND recursion_scheduler_level_vk_hash = $4\n " - }, "c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7": { "describe": { "columns": [], @@ -9698,30 +9189,6 @@ }, "query": "\n UPDATE scheduler_dependency_tracker_fri\n SET\n status = 'queuing'\n WHERE\n l1_batch_number IN (\n SELECT\n l1_batch_number\n FROM\n scheduler_dependency_tracker_fri\n WHERE\n status != 'queued'\n AND circuit_1_final_prover_job_id IS NOT NULL\n AND circuit_2_final_prover_job_id IS NOT NULL\n AND circuit_3_final_prover_job_id IS NOT NULL\n AND circuit_4_final_prover_job_id IS NOT NULL\n AND circuit_5_final_prover_job_id IS NOT NULL\n AND circuit_6_final_prover_job_id IS NOT NULL\n AND circuit_7_final_prover_job_id IS NOT NULL\n AND circuit_8_final_prover_job_id IS NOT NULL\n AND circuit_9_final_prover_job_id IS NOT NULL\n AND circuit_10_final_prover_job_id IS NOT NULL\n AND circuit_11_final_prover_job_id IS NOT NULL\n AND circuit_12_final_prover_job_id IS NOT NULL\n AND circuit_13_final_prover_job_id IS NOT NULL\n )\n RETURNING\n l1_batch_number;\n " }, - "c689a86a40c882a0d990a17ecc32290b2a47043fc88d4a2232461434a3f4a57d": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status!", - "ordinal": 1, - "type_info": "Text" - } - ], - "nullable": [ - null, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\",\n status AS \"status!\"\n FROM\n prover_jobs\n GROUP BY\n status\n " - }, "c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355": { "describe": { "columns": [ @@ -9788,162 +9255,39 @@ } ], "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status <> 'successful'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " - }, - "cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n timestamp,\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "cd76f54e1b9b4c0cf3044d3b767714e290f88ea1f20092a0278718fecda63caf": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_type", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "prover_input", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "status", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "error", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "processing_started_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "created_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 8, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 9, - "type_info": "Time" - }, - { - "name": "aggregation_round", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "result", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "sequence_number", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "attempts", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "circuit_input_blob_url", - "ordinal": 14, - "type_info": "Text" - }, - { - "name": "proccesed_by", - "ordinal": 15, - "type_info": "Text" - }, + false + ], + "parameters": { + "Left": [] + } + }, + "query": "\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status <> 'successful'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " + }, + "cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4": { + "describe": { + "columns": [ { - "name": "is_blob_cleaned", - "ordinal": 16, - "type_info": "Bool" + "name": "timestamp", + "ordinal": 0, + "type_info": "Int8" }, { - "name": "protocol_version", - "ordinal": 17, - "type_info": "Int4" + "name": "hash", + "ordinal": 1, + "type_info": "Bytea" } ], "nullable": [ - false, - false, - false, - false, - false, - true, - true, - false, - false, - false, - false, - true, - false, - false, - true, - true, false, true ], "parameters": { "Left": [ - "TextArray", - "Int4Array" + "Int8" ] } }, - "query": "\n UPDATE prover_jobs\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id = (\n SELECT\n id\n FROM\n prover_jobs\n WHERE\n circuit_type = ANY ($1)\n AND status = 'queued'\n AND protocol_version = ANY ($2)\n ORDER BY\n aggregation_round DESC,\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs.*\n " + "query": "\n SELECT\n timestamp,\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n " }, "cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c": { "describe": { @@ -10275,128 +9619,6 @@ }, "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n " }, - "d17221312e645b0287ff9238954512b528e7928087351a32c96b44d538dfb9ee": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_type", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "prover_input", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "status", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "error", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "processing_started_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "created_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 8, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 9, - "type_info": "Time" - }, - { - "name": "aggregation_round", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "result", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "sequence_number", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "attempts", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "circuit_input_blob_url", - "ordinal": 14, - "type_info": "Text" - }, - { - "name": "proccesed_by", - "ordinal": 15, - "type_info": "Text" - }, - { - "name": "is_blob_cleaned", - "ordinal": 16, - "type_info": "Bool" - }, - { - "name": "protocol_version", - "ordinal": 17, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - true, - false, - false, - false, - false, - true, - false, - false, - true, - true, - false, - true - ], - "parameters": { - "Left": [ - "Int4Array" - ] - } - }, - "query": "\n UPDATE prover_jobs\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id = (\n SELECT\n id\n FROM\n prover_jobs\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n aggregation_round DESC,\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs.*\n " - }, "d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283": { "describe": { "columns": [ @@ -11360,33 +10582,6 @@ }, "query": "\n DELETE FROM transactions\n WHERE\n miniblock_number IS NULL\n AND received_at < NOW() - $1::INTERVAL\n AND is_priority = FALSE\n AND error IS NULL\n RETURNING\n hash\n " }, - "e6a3efeffe1b3520cc9b1751e2c842c27546b3fd41f7d8a784ca58579856621b": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "attempts", - "ordinal": 1, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n RETURNING\n l1_batch_number,\n attempts\n " - }, "e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705": { "describe": { "columns": [ @@ -11715,24 +10910,6 @@ }, "query": "\n INSERT INTO\n proof_compression_jobs_fri (l1_batch_number, fri_proof_blob_url, status, created_at, updated_at)\n VALUES\n ($1, $2, $3, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " }, - "efe2a4ce4ba09e40ac7401f19ac5a42a0d521ffa33594c7861d786741d303f30": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int4", - "Int4", - "Int2", - "Text", - "Text", - "Int2" - ] - } - }, - "query": "\n INSERT INTO\n gpu_prover_queue (\n instance_host,\n instance_port,\n queue_capacity,\n queue_free_slots,\n instance_status,\n specialized_prover_group_id,\n region,\n zone,\n num_gpu,\n created_at,\n updated_at\n )\n VALUES\n (CAST($1::TEXT AS inet), $2, $3, $3, 'available', $4, $5, $6, $7, NOW(), NOW())\n ON CONFLICT (instance_host, instance_port, region, zone) DO\n UPDATE\n SET\n instance_status = 'available',\n queue_capacity = $3,\n queue_free_slots = $3,\n specialized_prover_group_id = $4,\n region = $5,\n zone = $6,\n num_gpu = $7,\n updated_at = NOW()\n " - }, "f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66": { "describe": { "columns": [ @@ -11767,36 +10944,6 @@ }, "query": "\n WITH\n sl AS (\n SELECT\n (\n SELECT\n ARRAY[hashed_key, value] AS kv\n FROM\n storage_logs\n WHERE\n storage_logs.miniblock_number = $1\n AND storage_logs.hashed_key >= u.start_key\n AND storage_logs.hashed_key <= u.end_key\n ORDER BY\n storage_logs.hashed_key\n LIMIT\n 1\n )\n FROM\n UNNEST($2::bytea[], $3::bytea[]) AS u (start_key, end_key)\n )\n SELECT\n sl.kv[1] AS \"hashed_key?\",\n sl.kv[2] AS \"value?\",\n initial_writes.index\n FROM\n sl\n LEFT OUTER JOIN initial_writes ON initial_writes.hashed_key = sl.kv[1]\n " }, - "f1478830e5f95cbcd0da01d3457cbb9cb8da439c7ab28fa865f53908621dc4c5": { - "describe": { - "columns": [ - { - "name": "region", - "ordinal": 0, - "type_info": "Text" - }, - { - "name": "zone", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "total_gpus", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n region,\n zone,\n SUM(num_gpu) AS total_gpus\n FROM\n gpu_prover_queue\n GROUP BY\n region,\n zone\n " - }, "f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a": { "describe": { "columns": [ @@ -11837,39 +10984,6 @@ }, "query": "\n SELECT\n l1_address\n FROM\n tokens\n WHERE\n market_volume > $1\n " }, - "f303c53843a58fac4fcdedbfe2b2b33ad609b7df8e55a0cf214ea1ec6421e57a": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int4" - ] - } - }, - "query": "\n UPDATE prover_jobs\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'in_gpu_proof'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n id,\n status,\n attempts\n " - }, "f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb": { "describe": { "columns": [ diff --git a/core/lib/dal/src/gpu_prover_queue_dal.rs b/core/lib/dal/src/gpu_prover_queue_dal.rs deleted file mode 100644 index b424edd57456..000000000000 --- a/core/lib/dal/src/gpu_prover_queue_dal.rs +++ /dev/null @@ -1,218 +0,0 @@ -use std::{collections::HashMap, time::Duration}; - -use zksync_types::proofs::{GpuProverInstanceStatus, SocketAddress}; - -use crate::{time_utils::pg_interval_from_duration, StorageProcessor}; - -#[derive(Debug)] -pub struct GpuProverQueueDal<'a, 'c> { - pub(crate) storage: &'a mut StorageProcessor<'c>, -} - -impl GpuProverQueueDal<'_, '_> { - pub async fn lock_available_prover( - &mut self, - processing_timeout: Duration, - specialized_prover_group_id: u8, - region: String, - zone: String, - ) -> Option { - { - let processing_timeout = pg_interval_from_duration(processing_timeout); - let result: Option = sqlx::query!( - r#" - UPDATE gpu_prover_queue - SET - instance_status = 'reserved', - updated_at = NOW(), - processing_started_at = NOW() - WHERE - id IN ( - SELECT - id - FROM - gpu_prover_queue - WHERE - specialized_prover_group_id = $2 - AND region = $3 - AND zone = $4 - AND ( - instance_status = 'available' - OR ( - instance_status = 'reserved' - AND processing_started_at < NOW() - $1::INTERVAL - ) - ) - ORDER BY - updated_at ASC - LIMIT - 1 - FOR UPDATE - SKIP LOCKED - ) - RETURNING - gpu_prover_queue.* - "#, - &processing_timeout, - specialized_prover_group_id as i16, - region, - zone - ) - .fetch_optional(self.storage.conn()) - .await - .unwrap() - .map(|row| SocketAddress { - host: row.instance_host.network(), - port: row.instance_port as u16, - }); - - result - } - } - - pub async fn insert_prover_instance( - &mut self, - address: SocketAddress, - queue_capacity: usize, - specialized_prover_group_id: u8, - region: String, - zone: String, - num_gpu: u8, - ) { - { - sqlx::query!( - r#" - INSERT INTO - gpu_prover_queue ( - instance_host, - instance_port, - queue_capacity, - queue_free_slots, - instance_status, - specialized_prover_group_id, - region, - zone, - num_gpu, - created_at, - updated_at - ) - VALUES - (CAST($1::TEXT AS inet), $2, $3, $3, 'available', $4, $5, $6, $7, NOW(), NOW()) - ON CONFLICT (instance_host, instance_port, region, zone) DO - UPDATE - SET - instance_status = 'available', - queue_capacity = $3, - queue_free_slots = $3, - specialized_prover_group_id = $4, - region = $5, - zone = $6, - num_gpu = $7, - updated_at = NOW() - "#, - format!("{}",address.host), - address.port as i32, - queue_capacity as i32, - specialized_prover_group_id as i16, - region, - zone, - num_gpu as i16) - .execute(self.storage.conn()) - .await - .unwrap(); - } - } - - pub async fn update_prover_instance_status( - &mut self, - address: SocketAddress, - status: GpuProverInstanceStatus, - queue_free_slots: usize, - region: String, - zone: String, - ) { - { - sqlx::query!( - r#" - UPDATE gpu_prover_queue - SET - instance_status = $1, - updated_at = NOW(), - queue_free_slots = $4 - WHERE - instance_host = $2::TEXT::inet - AND instance_port = $3 - AND region = $5 - AND zone = $6 - "#, - format!("{:?}", status).to_lowercase(), - format!("{}", address.host), - address.port as i32, - queue_free_slots as i32, - region, - zone - ) - .execute(self.storage.conn()) - .await - .unwrap(); - } - } - - pub async fn update_prover_instance_from_full_to_available( - &mut self, - address: SocketAddress, - queue_free_slots: usize, - region: String, - zone: String, - ) { - { - sqlx::query!( - r#" - UPDATE gpu_prover_queue - SET - instance_status = 'available', - updated_at = NOW(), - queue_free_slots = $3 - WHERE - instance_host = $1::TEXT::inet - AND instance_port = $2 - AND instance_status = 'full' - AND region = $4 - AND zone = $5 - "#, - format!("{}", address.host), - address.port as i32, - queue_free_slots as i32, - region, - zone - ) - .execute(self.storage.conn()) - .await - .unwrap(); - } - } - - pub async fn get_prover_gpu_count_per_region_zone(&mut self) -> HashMap<(String, String), u64> { - { - sqlx::query!( - r#" - SELECT - region, - zone, - SUM(num_gpu) AS total_gpus - FROM - gpu_prover_queue - GROUP BY - region, - zone - "#, - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| ((row.region, row.zone), row.total_gpus.unwrap() as u64)) - .collect() - } - } -} diff --git a/core/lib/dal/src/lib.rs b/core/lib/dal/src/lib.rs index d2e37230a373..3a5691a1c93e 100644 --- a/core/lib/dal/src/lib.rs +++ b/core/lib/dal/src/lib.rs @@ -13,9 +13,9 @@ use crate::{ fri_proof_compressor_dal::FriProofCompressorDal, fri_protocol_versions_dal::FriProtocolVersionsDal, fri_prover_dal::FriProverDal, fri_scheduler_dependency_tracker_dal::FriSchedulerDependencyTrackerDal, - fri_witness_generator_dal::FriWitnessGeneratorDal, gpu_prover_queue_dal::GpuProverQueueDal, - proof_generation_dal::ProofGenerationDal, protocol_versions_dal::ProtocolVersionsDal, - protocol_versions_web3_dal::ProtocolVersionsWeb3Dal, prover_dal::ProverDal, + fri_witness_generator_dal::FriWitnessGeneratorDal, proof_generation_dal::ProofGenerationDal, + protocol_versions_dal::ProtocolVersionsDal, + protocol_versions_web3_dal::ProtocolVersionsWeb3Dal, snapshot_recovery_dal::SnapshotRecoveryDal, snapshots_creator_dal::SnapshotsCreatorDal, snapshots_dal::SnapshotsDal, storage_dal::StorageDal, storage_logs_dal::StorageLogsDal, storage_logs_dedup_dal::StorageLogsDedupDal, storage_web3_dal::StorageWeb3Dal, @@ -42,7 +42,6 @@ pub mod fri_protocol_versions_dal; pub mod fri_prover_dal; pub mod fri_scheduler_dependency_tracker_dal; pub mod fri_witness_generator_dal; -pub mod gpu_prover_queue_dal; pub mod healthcheck; mod instrument; mod metrics; @@ -50,7 +49,6 @@ mod models; pub mod proof_generation_dal; pub mod protocol_versions_dal; pub mod protocol_versions_web3_dal; -pub mod prover_dal; pub mod snapshot_recovery_dal; pub mod snapshots_creator_dal; pub mod snapshots_dal; @@ -187,18 +185,10 @@ impl<'a> StorageProcessor<'a> { TokensWeb3Dal { storage: self } } - pub fn prover_dal(&mut self) -> ProverDal<'_, 'a> { - ProverDal { storage: self } - } - pub fn contract_verification_dal(&mut self) -> ContractVerificationDal<'_, 'a> { ContractVerificationDal { storage: self } } - pub fn gpu_prover_queue_dal(&mut self) -> GpuProverQueueDal<'_, 'a> { - GpuProverQueueDal { storage: self } - } - pub fn protocol_versions_dal(&mut self) -> ProtocolVersionsDal<'_, 'a> { ProtocolVersionsDal { storage: self } } diff --git a/core/lib/dal/src/protocol_versions_dal.rs b/core/lib/dal/src/protocol_versions_dal.rs index 0f4818864dfe..8aad040221c4 100644 --- a/core/lib/dal/src/protocol_versions_dal.rs +++ b/core/lib/dal/src/protocol_versions_dal.rs @@ -1,4 +1,4 @@ -use std::convert::{TryFrom, TryInto}; +use std::convert::TryInto; use zksync_contracts::{BaseSystemContracts, BaseSystemContractsHashes}; use zksync_types::{ @@ -98,51 +98,6 @@ impl ProtocolVersionsDal<'_, '_> { db_transaction.commit().await.unwrap(); } - pub async fn save_prover_protocol_version(&mut self, version: ProtocolVersion) { - sqlx::query!( - r#" - INSERT INTO - prover_protocol_versions ( - id, - timestamp, - recursion_scheduler_level_vk_hash, - recursion_node_level_vk_hash, - recursion_leaf_level_vk_hash, - recursion_circuits_set_vks_hash, - verifier_address, - created_at - ) - VALUES - ($1, $2, $3, $4, $5, $6, $7, NOW()) - "#, - version.id as i32, - version.timestamp as i64, - version - .l1_verifier_config - .recursion_scheduler_level_vk_hash - .as_bytes(), - version - .l1_verifier_config - .params - .recursion_node_level_vk_hash - .as_bytes(), - version - .l1_verifier_config - .params - .recursion_leaf_level_vk_hash - .as_bytes(), - version - .l1_verifier_config - .params - .recursion_circuits_set_vks_hash - .as_bytes(), - version.verifier_address.as_bytes(), - ) - .execute(self.storage.conn()) - .await - .unwrap(); - } - pub async fn base_system_contracts_by_timestamp( &mut self, current_timestamp: u64, @@ -392,61 +347,4 @@ impl ProtocolVersionsDal<'_, '_> { None } } - - pub async fn protocol_version_for( - &mut self, - vk_commitments: &L1VerifierConfig, - ) -> Vec { - sqlx::query!( - r#" - SELECT - id - FROM - prover_protocol_versions - WHERE - recursion_circuits_set_vks_hash = $1 - AND recursion_leaf_level_vk_hash = $2 - AND recursion_node_level_vk_hash = $3 - AND recursion_scheduler_level_vk_hash = $4 - "#, - vk_commitments - .params - .recursion_circuits_set_vks_hash - .as_bytes(), - vk_commitments - .params - .recursion_leaf_level_vk_hash - .as_bytes(), - vk_commitments - .params - .recursion_node_level_vk_hash - .as_bytes(), - vk_commitments.recursion_scheduler_level_vk_hash.as_bytes(), - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| ProtocolVersionId::try_from(row.id as u16).unwrap()) - .collect() - } - - pub async fn prover_protocol_version_exists(&mut self, id: ProtocolVersionId) -> bool { - sqlx::query!( - r#" - SELECT - COUNT(*) AS "count!" - FROM - prover_protocol_versions - WHERE - id = $1 - "#, - id as i32 - ) - .fetch_one(self.storage.conn()) - .await - .unwrap() - .count - > 0 - } } diff --git a/core/lib/dal/src/prover_dal.rs b/core/lib/dal/src/prover_dal.rs deleted file mode 100644 index c0a767531af3..000000000000 --- a/core/lib/dal/src/prover_dal.rs +++ /dev/null @@ -1,724 +0,0 @@ -use std::{ - collections::HashMap, - convert::{TryFrom, TryInto}, - ops::Range, - time::Duration, -}; - -use sqlx::Error; -use zksync_types::{ - proofs::{ - AggregationRound, JobCountStatistics, JobExtendedStatistics, ProverJobInfo, - ProverJobMetadata, - }, - L1BatchNumber, ProtocolVersionId, -}; - -use crate::{ - instrument::InstrumentExt, - models::storage_prover_job_info::StorageProverJobInfo, - time_utils::{duration_to_naive_time, pg_interval_from_duration}, - StorageProcessor, -}; - -#[derive(Debug)] -pub struct ProverDal<'a, 'c> { - pub(crate) storage: &'a mut StorageProcessor<'c>, -} - -impl ProverDal<'_, '_> { - pub async fn get_next_prover_job( - &mut self, - protocol_versions: &[ProtocolVersionId], - ) -> Option { - let protocol_versions: Vec = protocol_versions.iter().map(|&id| id as i32).collect(); - let result: Option = sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = 'in_progress', - attempts = attempts + 1, - updated_at = NOW(), - processing_started_at = NOW() - WHERE - id = ( - SELECT - id - FROM - prover_jobs - WHERE - status = 'queued' - AND protocol_version = ANY ($1) - ORDER BY - aggregation_round DESC, - l1_batch_number ASC, - id ASC - LIMIT - 1 - FOR UPDATE - SKIP LOCKED - ) - RETURNING - prover_jobs.* - "#, - &protocol_versions[..] - ) - .fetch_optional(self.storage.conn()) - .await - .unwrap() - .map(|row| ProverJobMetadata { - id: row.id as u32, - block_number: L1BatchNumber(row.l1_batch_number as u32), - circuit_type: row.circuit_type, - aggregation_round: AggregationRound::try_from(row.aggregation_round).unwrap(), - sequence_number: row.sequence_number as usize, - }); - result - } - - pub async fn get_proven_l1_batches(&mut self) -> Vec<(L1BatchNumber, AggregationRound)> { - { - sqlx::query!( - r#" - SELECT - MAX(l1_batch_number) AS "l1_batch_number!", - aggregation_round - FROM - prover_jobs - WHERE - status = 'successful' - GROUP BY - aggregation_round - "# - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|record| { - ( - L1BatchNumber(record.l1_batch_number as u32), - record.aggregation_round.try_into().unwrap(), - ) - }) - .collect() - } - } - - pub async fn get_next_prover_job_by_circuit_types( - &mut self, - circuit_types: Vec, - protocol_versions: &[ProtocolVersionId], - ) -> Option { - { - let protocol_versions: Vec = - protocol_versions.iter().map(|&id| id as i32).collect(); - let result: Option = sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = 'in_progress', - attempts = attempts + 1, - updated_at = NOW(), - processing_started_at = NOW() - WHERE - id = ( - SELECT - id - FROM - prover_jobs - WHERE - circuit_type = ANY ($1) - AND status = 'queued' - AND protocol_version = ANY ($2) - ORDER BY - aggregation_round DESC, - l1_batch_number ASC, - id ASC - LIMIT - 1 - FOR UPDATE - SKIP LOCKED - ) - RETURNING - prover_jobs.* - "#, - &circuit_types[..], - &protocol_versions[..] - ) - .fetch_optional(self.storage.conn()) - .await - .unwrap() - .map(|row| ProverJobMetadata { - id: row.id as u32, - block_number: L1BatchNumber(row.l1_batch_number as u32), - circuit_type: row.circuit_type, - aggregation_round: AggregationRound::try_from(row.aggregation_round).unwrap(), - sequence_number: row.sequence_number as usize, - }); - - result - } - } - - // If making changes to this method, consider moving the serialization logic to the DAL layer. - pub async fn insert_prover_jobs( - &mut self, - l1_batch_number: L1BatchNumber, - circuit_types_and_urls: Vec<(&'static str, String)>, - aggregation_round: AggregationRound, - protocol_version: i32, - ) { - { - let it = circuit_types_and_urls.into_iter().enumerate(); - for (sequence_number, (circuit, circuit_input_blob_url)) in it { - sqlx::query!( - r#" - INSERT INTO - prover_jobs ( - l1_batch_number, - circuit_type, - sequence_number, - prover_input, - aggregation_round, - circuit_input_blob_url, - protocol_version, - status, - created_at, - updated_at - ) - VALUES - ($1, $2, $3, $4, $5, $6, $7, 'queued', NOW(), NOW()) - ON CONFLICT (l1_batch_number, aggregation_round, sequence_number) DO NOTHING - "#, - l1_batch_number.0 as i64, - circuit, - sequence_number as i64, - &[] as &[u8], - aggregation_round as i64, - circuit_input_blob_url, - protocol_version - ) - .instrument("save_witness") - .report_latency() - .with_arg("l1_batch_number", &l1_batch_number) - .with_arg("circuit", &circuit) - .with_arg("circuit_input_blob_url", &circuit_input_blob_url) - .execute(self.storage.conn()) - .await - .unwrap(); - } - } - } - - pub async fn save_proof( - &mut self, - id: u32, - time_taken: Duration, - proof: Vec, - proccesed_by: &str, - ) -> Result<(), Error> { - { - sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = 'successful', - updated_at = NOW(), - time_taken = $1, - result = $2, - proccesed_by = $3 - WHERE - id = $4 - "#, - duration_to_naive_time(time_taken), - &proof, - proccesed_by, - id as i64, - ) - .instrument("save_proof") - .report_latency() - .with_arg("id", &id) - .with_arg("proof.len", &proof.len()) - .execute(self.storage.conn()) - .await?; - } - Ok(()) - } - - pub async fn save_proof_error( - &mut self, - id: u32, - error: String, - max_attempts: u32, - ) -> Result<(), Error> { - { - let mut transaction = self.storage.start_transaction().await.unwrap(); - - let row = sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = 'failed', - error = $1, - updated_at = NOW() - WHERE - id = $2 - RETURNING - l1_batch_number, - attempts - "#, - error, - id as i64, - ) - .fetch_one(transaction.conn()) - .await?; - - if row.attempts as u32 >= max_attempts { - transaction - .blocks_dal() - .set_skip_proof_for_l1_batch(L1BatchNumber(row.l1_batch_number as u32)) - .await - .unwrap(); - } - - transaction.commit().await.unwrap(); - Ok(()) - } - } - - pub async fn requeue_stuck_jobs( - &mut self, - processing_timeout: Duration, - max_attempts: u32, - ) -> Vec { - let processing_timeout = pg_interval_from_duration(processing_timeout); - { - sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = 'queued', - updated_at = NOW(), - processing_started_at = NOW() - WHERE - ( - status = 'in_progress' - AND processing_started_at <= NOW() - $1::INTERVAL - AND attempts < $2 - ) - OR ( - status = 'in_gpu_proof' - AND processing_started_at <= NOW() - $1::INTERVAL - AND attempts < $2 - ) - OR ( - status = 'failed' - AND attempts < $2 - ) - RETURNING - id, - status, - attempts - "#, - &processing_timeout, - max_attempts as i32, - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| StuckProverJobs { - id: row.id as u64, - status: row.status, - attempts: row.attempts as u64, - }) - .collect() - } - } - - pub async fn get_prover_jobs_stats_per_circuit( - &mut self, - ) -> HashMap { - { - sqlx::query!( - r#" - SELECT - COUNT(*) AS "count!", - circuit_type AS "circuit_type!", - status AS "status!" - FROM - prover_jobs - WHERE - status <> 'skipped' - AND status <> 'successful' - GROUP BY - circuit_type, - status - "# - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| (row.circuit_type, row.status, row.count as usize)) - .fold(HashMap::new(), |mut acc, (circuit_type, status, value)| { - let stats = acc.entry(circuit_type).or_insert(JobCountStatistics { - queued: 0, - in_progress: 0, - failed: 0, - successful: 0, - }); - match status.as_ref() { - "queued" => stats.queued = value, - "in_progress" => stats.in_progress = value, - "failed" => stats.failed = value, - "successful" => stats.successful = value, - _ => (), - } - acc - }) - } - } - - pub async fn get_prover_jobs_stats(&mut self) -> JobCountStatistics { - { - let mut results: HashMap = sqlx::query!( - r#" - SELECT - COUNT(*) AS "count!", - status AS "status!" - FROM - prover_jobs - GROUP BY - status - "# - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| (row.status, row.count as usize)) - .collect::>(); - JobCountStatistics { - queued: results.remove("queued").unwrap_or(0usize), - in_progress: results.remove("in_progress").unwrap_or(0usize), - failed: results.remove("failed").unwrap_or(0usize), - successful: results.remove("successful").unwrap_or(0usize), - } - } - } - - pub async fn min_unproved_l1_batch_number(&mut self) -> Option { - { - sqlx::query!( - r#" - SELECT - MIN(l1_batch_number) AS "l1_batch_number?" - FROM - ( - SELECT - MIN(l1_batch_number) AS "l1_batch_number" - FROM - prover_jobs - WHERE - status = 'successful' - OR aggregation_round < 3 - GROUP BY - l1_batch_number - HAVING - MAX(aggregation_round) < 3 - ) AS inn - "# - ) - .fetch_one(self.storage.conn()) - .await - .unwrap() - .l1_batch_number - .map(|n| L1BatchNumber(n as u32)) - } - } - - pub async fn min_unproved_l1_batch_number_by_basic_circuit_type( - &mut self, - ) -> Vec<(String, L1BatchNumber)> { - { - sqlx::query!( - r#" - SELECT - MIN(l1_batch_number) AS "l1_batch_number!", - circuit_type - FROM - prover_jobs - WHERE - aggregation_round = 0 - AND ( - status = 'queued' - OR status = 'in_progress' - OR status = 'in_gpu_proof' - OR status = 'failed' - ) - GROUP BY - circuit_type - "# - ) - .fetch_all(self.storage.conn()) - .await - .unwrap() - .into_iter() - .map(|row| (row.circuit_type, L1BatchNumber(row.l1_batch_number as u32))) - .collect() - } - } - - pub async fn get_extended_stats(&mut self) -> anyhow::Result { - { - let limits = sqlx::query!( - r#" - SELECT - ( - SELECT - l1_batch_number - FROM - prover_jobs - WHERE - status NOT IN ('successful', 'skipped') - ORDER BY - l1_batch_number - LIMIT - 1 - ) AS "successful_limit!", - ( - SELECT - l1_batch_number - FROM - prover_jobs - WHERE - status <> 'queued' - ORDER BY - l1_batch_number DESC - LIMIT - 1 - ) AS "queued_limit!", - ( - SELECT - MAX(l1_batch_number) AS "max!" - FROM - prover_jobs - ) AS "max_block!" - "# - ) - .fetch_one(self.storage.conn()) - .await?; - - let active_area = self - .get_jobs(GetProverJobsParams::blocks( - L1BatchNumber(limits.successful_limit as u32) - ..L1BatchNumber(limits.queued_limit as u32), - )) - .await?; - - Ok(JobExtendedStatistics { - successful_padding: L1BatchNumber(limits.successful_limit as u32 - 1), - queued_padding: L1BatchNumber(limits.queued_limit as u32 + 1), - queued_padding_len: (limits.max_block - limits.queued_limit) as u32, - active_area, - }) - } - } - - pub async fn get_jobs( - &mut self, - opts: GetProverJobsParams, - ) -> Result, sqlx::Error> { - let statuses = opts - .statuses - .map(|ss| { - { - // Until statuses are enums - let whitelist = ["queued", "in_progress", "successful", "failed"]; - if !ss.iter().all(|x| whitelist.contains(&x.as_str())) { - panic!("Forbidden value in statuses list.") - } - } - - format!( - "AND status IN ({})", - ss.iter() - .map(|x| format!("'{}'", x)) - .collect::>() - .join(",") - ) - }) - .unwrap_or_default(); - - let block_range = opts - .blocks - .as_ref() - .map(|range| { - format!( - "AND l1_batch_number >= {} - AND l1_batch_number <= {}", - range.start.0, range.end.0 - ) - }) - .unwrap_or_default(); - - let round = opts - .round - .map(|round| format!("AND aggregation_round = {}", round as u32)) - .unwrap_or_default(); - - let order = match opts.desc { - true => "DESC", - false => "ASC", - }; - - let limit = opts - .limit - .map(|limit| format!("LIMIT {}", limit)) - .unwrap_or_default(); - - let sql = format!( - r#" - SELECT - id, - circuit_type, - l1_batch_number, - status, - aggregation_round, - sequence_number, - length(prover_input) as input_length, - attempts, - created_at, - updated_at, - processing_started_at, - time_taken, - error - FROM prover_jobs - WHERE 1 = 1 -- Where clause can't be empty - {statuses} - {block_range} - {round} - ORDER BY "id" {order} - {limit} - "# - ); - - let query = sqlx::query_as(&sql); - - Ok(query - .fetch_all(self.storage.conn()) - .await? - .into_iter() - .map(|x: StorageProverJobInfo| x.into()) - .collect::>()) - } - - pub async fn get_prover_job_by_id( - &mut self, - job_id: u32, - ) -> Result, Error> { - { - let row = sqlx::query!( - r#" - SELECT - * - FROM - prover_jobs - WHERE - id = $1 - "#, - job_id as i64 - ) - .fetch_optional(self.storage.conn()) - .await?; - - Ok(row.map(|row| ProverJobMetadata { - id: row.id as u32, - block_number: L1BatchNumber(row.l1_batch_number as u32), - circuit_type: row.circuit_type, - aggregation_round: AggregationRound::try_from(row.aggregation_round).unwrap(), - sequence_number: row.sequence_number as usize, - })) - } - } - - pub async fn get_circuit_input_blob_urls_to_be_cleaned( - &mut self, - limit: u8, - ) -> Vec<(i64, String)> { - { - let job_ids = sqlx::query!( - r#" - SELECT - id, - circuit_input_blob_url - FROM - prover_jobs - WHERE - status = 'successful' - AND circuit_input_blob_url IS NOT NULL - AND updated_at < NOW() - INTERVAL '30 days' - LIMIT - $1; - "#, - limit as i32 - ) - .fetch_all(self.storage.conn()) - .await - .unwrap(); - job_ids - .into_iter() - .map(|row| (row.id, row.circuit_input_blob_url.unwrap())) - .collect() - } - } - - pub async fn update_status(&mut self, id: u32, status: &str) { - { - sqlx::query!( - r#" - UPDATE prover_jobs - SET - status = $1, - updated_at = NOW() - WHERE - id = $2 - "#, - status, - id as i64, - ) - .execute(self.storage.conn()) - .await - .unwrap(); - } - } -} - -pub struct GetProverJobsParams { - pub statuses: Option>, - pub blocks: Option>, - pub limit: Option, - pub desc: bool, - pub round: Option, -} - -impl GetProverJobsParams { - pub fn blocks(range: Range) -> GetProverJobsParams { - GetProverJobsParams { - blocks: Some(range), - statuses: None, - limit: None, - desc: false, - round: None, - } - } -} - -#[derive(Debug)] -pub struct StuckProverJobs { - pub id: u64, - pub status: String, - pub attempts: u64, -} diff --git a/core/lib/dal/src/tests/mod.rs b/core/lib/dal/src/tests/mod.rs index 6b8ae6090f9c..7a4a85b5edb3 100644 --- a/core/lib/dal/src/tests/mod.rs +++ b/core/lib/dal/src/tests/mod.rs @@ -2,22 +2,20 @@ use std::time::Duration; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{ - block::{L1BatchHeader, MiniblockHasher, MiniblockHeader}, + block::{MiniblockHasher, MiniblockHeader}, fee::{Fee, TransactionExecutionMetrics}, helpers::unix_timestamp_ms, l1::{L1Tx, OpProcessingType, PriorityQueueType}, l2::L2Tx, - proofs::AggregationRound, tx::{tx_execution_info::TxExecutionStatus, ExecutionMetrics, TransactionExecutionResult}, - Address, Execute, L1BatchNumber, L1BlockNumber, L1TxCommonData, L2ChainId, MiniblockNumber, - PriorityOpId, ProtocolVersion, ProtocolVersionId, H160, H256, MAX_GAS_PER_PUBDATA_BYTE, U256, + Address, Execute, L1BlockNumber, L1TxCommonData, L2ChainId, MiniblockNumber, PriorityOpId, + ProtocolVersionId, H160, H256, MAX_GAS_PER_PUBDATA_BYTE, U256, }; use crate::{ blocks_dal::BlocksDal, connection::ConnectionPool, protocol_versions_dal::ProtocolVersionsDal, - prover_dal::{GetProverJobsParams, ProverDal}, transactions_dal::{L2TxSubmissionResult, TransactionsDal}, transactions_web3_dal::TransactionsWeb3Dal, }; @@ -256,144 +254,3 @@ async fn remove_stuck_txs() { .unwrap() .unwrap(); } - -fn create_circuits() -> Vec<(&'static str, String)> { - vec![ - ("Main VM", "1_0_Main VM_BasicCircuits.bin".to_owned()), - ("SHA256", "1_1_SHA256_BasicCircuits.bin".to_owned()), - ( - "Code decommitter", - "1_2_Code decommitter_BasicCircuits.bin".to_owned(), - ), - ( - "Log demuxer", - "1_3_Log demuxer_BasicCircuits.bin".to_owned(), - ), - ] -} - -#[tokio::test] -async fn test_duplicate_insert_prover_jobs() { - let connection_pool = ConnectionPool::test_pool().await; - let storage = &mut connection_pool.access_storage().await.unwrap(); - storage - .protocol_versions_dal() - .save_protocol_version_with_tx(Default::default()) - .await; - storage - .protocol_versions_dal() - .save_prover_protocol_version(Default::default()) - .await; - let block_number = 1; - let header = L1BatchHeader::new( - L1BatchNumber(block_number), - 0, - Default::default(), - Default::default(), - Default::default(), - ); - storage - .blocks_dal() - .insert_l1_batch(&header, &[], Default::default(), &[], &[]) - .await - .unwrap(); - - let mut prover_dal = ProverDal { storage }; - let circuits = create_circuits(); - let l1_batch_number = L1BatchNumber(block_number); - prover_dal - .insert_prover_jobs( - l1_batch_number, - circuits.clone(), - AggregationRound::BasicCircuits, - ProtocolVersionId::latest() as i32, - ) - .await; - - // try inserting the same jobs again to ensure it does not panic - prover_dal - .insert_prover_jobs( - l1_batch_number, - circuits.clone(), - AggregationRound::BasicCircuits, - ProtocolVersionId::latest() as i32, - ) - .await; - - let prover_jobs_params = GetProverJobsParams { - statuses: None, - blocks: Some(std::ops::Range { - start: l1_batch_number, - end: l1_batch_number + 1, - }), - limit: None, - desc: false, - round: None, - }; - let jobs = prover_dal.get_jobs(prover_jobs_params).await.unwrap(); - assert_eq!(circuits.len(), jobs.len()); -} - -#[tokio::test] -async fn test_requeue_prover_jobs() { - let connection_pool = ConnectionPool::test_pool().await; - let storage = &mut connection_pool.access_storage().await.unwrap(); - let protocol_version = ProtocolVersion::default(); - storage - .protocol_versions_dal() - .save_protocol_version_with_tx(protocol_version) - .await; - storage - .protocol_versions_dal() - .save_prover_protocol_version(Default::default()) - .await; - let block_number = 1; - let header = L1BatchHeader::new( - L1BatchNumber(block_number), - 0, - Default::default(), - Default::default(), - ProtocolVersionId::latest(), - ); - storage - .blocks_dal() - .insert_l1_batch(&header, &[], Default::default(), &[], &[]) - .await - .unwrap(); - - let mut prover_dal = ProverDal { storage }; - let circuits = create_circuits(); - let l1_batch_number = L1BatchNumber(block_number); - prover_dal - .insert_prover_jobs( - l1_batch_number, - circuits, - AggregationRound::BasicCircuits, - ProtocolVersionId::latest() as i32, - ) - .await; - - // take all jobs from prover_job table - for _ in 1..=4 { - let job = prover_dal - .get_next_prover_job(&[ProtocolVersionId::latest()]) - .await; - assert!(job.is_some()); - } - let job = prover_dal - .get_next_prover_job(&[ProtocolVersionId::latest()]) - .await; - assert!(job.is_none()); - // re-queue jobs - let stuck_jobs = prover_dal - .requeue_stuck_jobs(Duration::from_secs(0), 10) - .await; - assert_eq!(4, stuck_jobs.len()); - // re-check that all jobs can be taken again - for _ in 1..=4 { - let job = prover_dal - .get_next_prover_job(&[ProtocolVersionId::latest()]) - .await; - assert!(job.is_some()); - } -} diff --git a/core/lib/env_config/src/fri_prover.rs b/core/lib/env_config/src/fri_prover.rs index 200f22d89b7e..7b97df50374b 100644 --- a/core/lib/env_config/src/fri_prover.rs +++ b/core/lib/env_config/src/fri_prover.rs @@ -30,6 +30,8 @@ mod tests { witness_vector_generator_thread_count: Some(5), queue_capacity: 10, witness_vector_receiver_port: 3316, + zone_read_url: "http://metadata.google.internal/computeMetadata/v1/instance/zone" + .to_string(), shall_save_to_public_bucket: true, } } @@ -49,6 +51,7 @@ mod tests { FRI_PROVER_WITNESS_VECTOR_GENERATOR_THREAD_COUNT="5" FRI_PROVER_QUEUE_CAPACITY="10" FRI_PROVER_WITNESS_VECTOR_RECEIVER_PORT="3316" + FRI_PROVER_ZONE_READ_URL="http://metadata.google.internal/computeMetadata/v1/instance/zone" FRI_PROVER_SHALL_SAVE_TO_PUBLIC_BUCKET=true "#; lock.set_env(config); diff --git a/core/lib/env_config/src/lib.rs b/core/lib/env_config/src/lib.rs index 48f55e51e7c6..fa2bb2371917 100644 --- a/core/lib/env_config/src/lib.rs +++ b/core/lib/env_config/src/lib.rs @@ -19,8 +19,6 @@ mod fri_witness_vector_generator; mod house_keeper; pub mod object_store; mod proof_data_handler; -mod prover; -mod prover_group; mod snapshots_creator; mod utils; mod witness_generator; diff --git a/core/lib/env_config/src/prover.rs b/core/lib/env_config/src/prover.rs deleted file mode 100644 index 700f0fffb96a..000000000000 --- a/core/lib/env_config/src/prover.rs +++ /dev/null @@ -1,197 +0,0 @@ -use zksync_config::ProverConfigs; - -use crate::{envy_load, FromEnv}; - -impl FromEnv for ProverConfigs { - fn from_env() -> anyhow::Result { - Ok(Self { - non_gpu: envy_load("non_gpu", "PROVER_NON_GPU_")?, - two_gpu_forty_gb_mem: envy_load( - "two_gpu_forty_gb_mem", - "PROVER_TWO_GPU_FORTY_GB_MEM_", - )?, - one_gpu_eighty_gb_mem: envy_load( - "one_gpu_eighty_gb_mem", - "PROVER_ONE_GPU_EIGHTY_GB_MEM_", - )?, - two_gpu_eighty_gb_mem: envy_load( - "two_gpu_eighty_gb_mem", - "PROVER_TWO_GPU_EIGHTY_GB_MEM_", - )?, - four_gpu_eighty_gb_mem: envy_load( - "four_gpu_eighty_gb_mem", - "PROVER_FOUR_GPU_EIGHTY_GB_MEM_", - )?, - }) - } -} - -#[cfg(test)] -mod tests { - use zksync_config::ProverConfig; - - use super::*; - use crate::test_utils::EnvMutex; - - static MUTEX: EnvMutex = EnvMutex::new(); - - fn expected_config() -> ProverConfigs { - ProverConfigs { - non_gpu: ProverConfig { - prometheus_port: 3313, - initial_setup_key_path: "key".to_owned(), - key_download_url: "value".to_owned(), - generation_timeout_in_secs: 2700u16, - number_of_threads: 2, - max_attempts: 4, - polling_duration_in_millis: 5, - setup_keys_path: "/usr/src/setup-keys".to_string(), - specialized_prover_group_id: 0, - number_of_setup_slots: 2, - assembly_receiver_port: 17791, - assembly_receiver_poll_time_in_millis: 250, - assembly_queue_capacity: 5, - }, - two_gpu_forty_gb_mem: ProverConfig { - prometheus_port: 3313, - initial_setup_key_path: "key".to_owned(), - key_download_url: "value".to_owned(), - generation_timeout_in_secs: 2700u16, - number_of_threads: 2, - max_attempts: 4, - polling_duration_in_millis: 5, - setup_keys_path: "/usr/src/setup-keys".to_string(), - specialized_prover_group_id: 1, - number_of_setup_slots: 5, - assembly_receiver_port: 17791, - assembly_receiver_poll_time_in_millis: 250, - assembly_queue_capacity: 5, - }, - one_gpu_eighty_gb_mem: ProverConfig { - prometheus_port: 3313, - initial_setup_key_path: "key".to_owned(), - key_download_url: "value".to_owned(), - generation_timeout_in_secs: 2700u16, - number_of_threads: 4, - max_attempts: 4, - polling_duration_in_millis: 5, - setup_keys_path: "/usr/src/setup-keys".to_string(), - specialized_prover_group_id: 2, - number_of_setup_slots: 5, - assembly_receiver_port: 17791, - assembly_receiver_poll_time_in_millis: 250, - assembly_queue_capacity: 5, - }, - two_gpu_eighty_gb_mem: ProverConfig { - prometheus_port: 3313, - initial_setup_key_path: "key".to_owned(), - key_download_url: "value".to_owned(), - generation_timeout_in_secs: 2700u16, - number_of_threads: 9, - max_attempts: 4, - polling_duration_in_millis: 5, - setup_keys_path: "/usr/src/setup-keys".to_string(), - specialized_prover_group_id: 3, - number_of_setup_slots: 9, - assembly_receiver_port: 17791, - assembly_receiver_poll_time_in_millis: 250, - assembly_queue_capacity: 5, - }, - four_gpu_eighty_gb_mem: ProverConfig { - prometheus_port: 3313, - initial_setup_key_path: "key".to_owned(), - key_download_url: "value".to_owned(), - generation_timeout_in_secs: 2700u16, - number_of_threads: 18, - max_attempts: 4, - polling_duration_in_millis: 5, - setup_keys_path: "/usr/src/setup-keys".to_string(), - specialized_prover_group_id: 4, - number_of_setup_slots: 18, - assembly_receiver_port: 17791, - assembly_receiver_poll_time_in_millis: 250, - assembly_queue_capacity: 5, - }, - } - } - - const CONFIG: &str = r#" - PROVER_NON_GPU_PROMETHEUS_PORT="3313" - PROVER_NON_GPU_INITIAL_SETUP_KEY_PATH="key" - PROVER_NON_GPU_KEY_DOWNLOAD_URL="value" - PROVER_NON_GPU_GENERATION_TIMEOUT_IN_SECS=2700 - PROVER_NON_GPU_NUMBER_OF_THREADS="2" - PROVER_NON_GPU_MAX_ATTEMPTS="4" - PROVER_NON_GPU_POLLING_DURATION_IN_MILLIS=5 - PROVER_NON_GPU_SETUP_KEYS_PATH="/usr/src/setup-keys" - PROVER_NON_GPU_NUMBER_OF_SETUP_SLOTS=2 - PROVER_NON_GPU_ASSEMBLY_RECEIVER_PORT=17791 - PROVER_NON_GPU_ASSEMBLY_RECEIVER_POLL_TIME_IN_MILLIS=250 - PROVER_NON_GPU_ASSEMBLY_QUEUE_CAPACITY=5 - PROVER_NON_GPU_SPECIALIZED_PROVER_GROUP_ID=0 - - PROVER_TWO_GPU_FORTY_GB_MEM_PROMETHEUS_PORT="3313" - PROVER_TWO_GPU_FORTY_GB_MEM_INITIAL_SETUP_KEY_PATH="key" - PROVER_TWO_GPU_FORTY_GB_MEM_KEY_DOWNLOAD_URL="value" - PROVER_TWO_GPU_FORTY_GB_MEM_GENERATION_TIMEOUT_IN_SECS=2700 - PROVER_TWO_GPU_FORTY_GB_MEM_NUMBER_OF_THREADS="2" - PROVER_TWO_GPU_FORTY_GB_MEM_MAX_ATTEMPTS="4" - PROVER_TWO_GPU_FORTY_GB_MEM_POLLING_DURATION_IN_MILLIS=5 - PROVER_TWO_GPU_FORTY_GB_MEM_SETUP_KEYS_PATH="/usr/src/setup-keys" - PROVER_TWO_GPU_FORTY_GB_MEM_NUMBER_OF_SETUP_SLOTS=5 - PROVER_TWO_GPU_FORTY_GB_MEM_ASSEMBLY_RECEIVER_PORT=17791 - PROVER_TWO_GPU_FORTY_GB_MEM_ASSEMBLY_RECEIVER_POLL_TIME_IN_MILLIS=250 - PROVER_TWO_GPU_FORTY_GB_MEM_ASSEMBLY_QUEUE_CAPACITY=5 - PROVER_TWO_GPU_FORTY_GB_MEM_SPECIALIZED_PROVER_GROUP_ID=1 - - PROVER_ONE_GPU_EIGHTY_GB_MEM_PROMETHEUS_PORT="3313" - PROVER_ONE_GPU_EIGHTY_GB_MEM_INITIAL_SETUP_KEY_PATH="key" - PROVER_ONE_GPU_EIGHTY_GB_MEM_KEY_DOWNLOAD_URL="value" - PROVER_ONE_GPU_EIGHTY_GB_MEM_GENERATION_TIMEOUT_IN_SECS=2700 - PROVER_ONE_GPU_EIGHTY_GB_MEM_NUMBER_OF_THREADS="4" - PROVER_ONE_GPU_EIGHTY_GB_MEM_MAX_ATTEMPTS="4" - PROVER_ONE_GPU_EIGHTY_GB_MEM_POLLING_DURATION_IN_MILLIS=5 - PROVER_ONE_GPU_EIGHTY_GB_MEM_SETUP_KEYS_PATH="/usr/src/setup-keys" - PROVER_ONE_GPU_EIGHTY_GB_MEM_NUMBER_OF_SETUP_SLOTS=5 - PROVER_ONE_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_PORT=17791 - PROVER_ONE_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_POLL_TIME_IN_MILLIS=250 - PROVER_ONE_GPU_EIGHTY_GB_MEM_ASSEMBLY_QUEUE_CAPACITY=5 - PROVER_ONE_GPU_EIGHTY_GB_MEM_SPECIALIZED_PROVER_GROUP_ID=2 - - PROVER_TWO_GPU_EIGHTY_GB_MEM_PROMETHEUS_PORT="3313" - PROVER_TWO_GPU_EIGHTY_GB_MEM_INITIAL_SETUP_KEY_PATH="key" - PROVER_TWO_GPU_EIGHTY_GB_MEM_KEY_DOWNLOAD_URL="value" - PROVER_TWO_GPU_EIGHTY_GB_MEM_GENERATION_TIMEOUT_IN_SECS=2700 - PROVER_TWO_GPU_EIGHTY_GB_MEM_NUMBER_OF_THREADS="9" - PROVER_TWO_GPU_EIGHTY_GB_MEM_MAX_ATTEMPTS="4" - PROVER_TWO_GPU_EIGHTY_GB_MEM_POLLING_DURATION_IN_MILLIS=5 - PROVER_TWO_GPU_EIGHTY_GB_MEM_SETUP_KEYS_PATH="/usr/src/setup-keys" - PROVER_TWO_GPU_EIGHTY_GB_MEM_NUMBER_OF_SETUP_SLOTS=9 - PROVER_TWO_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_PORT=17791 - PROVER_TWO_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_POLL_TIME_IN_MILLIS=250 - PROVER_TWO_GPU_EIGHTY_GB_MEM_ASSEMBLY_QUEUE_CAPACITY=5 - PROVER_TWO_GPU_EIGHTY_GB_MEM_SPECIALIZED_PROVER_GROUP_ID=3 - - PROVER_FOUR_GPU_EIGHTY_GB_MEM_PROMETHEUS_PORT="3313" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_INITIAL_SETUP_KEY_PATH="key" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_KEY_DOWNLOAD_URL="value" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_GENERATION_TIMEOUT_IN_SECS=2700 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_NUMBER_OF_THREADS="18" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_MAX_ATTEMPTS="4" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_POLLING_DURATION_IN_MILLIS=5 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_SETUP_KEYS_PATH="/usr/src/setup-keys" - PROVER_FOUR_GPU_EIGHTY_GB_MEM_NUMBER_OF_SETUP_SLOTS=18 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_PORT=17791 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_ASSEMBLY_RECEIVER_POLL_TIME_IN_MILLIS=250 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_ASSEMBLY_QUEUE_CAPACITY=5 - PROVER_FOUR_GPU_EIGHTY_GB_MEM_SPECIALIZED_PROVER_GROUP_ID=4 - "#; - - #[test] - fn from_env() { - let mut lock = MUTEX.lock(); - lock.set_env(CONFIG); - let actual = ProverConfigs::from_env().unwrap(); - assert_eq!(actual, expected_config()); - } -} diff --git a/core/lib/env_config/src/prover_group.rs b/core/lib/env_config/src/prover_group.rs deleted file mode 100644 index bdac82cbb9cc..000000000000 --- a/core/lib/env_config/src/prover_group.rs +++ /dev/null @@ -1,149 +0,0 @@ -use zksync_config::configs::ProverGroupConfig; - -use crate::{envy_load, FromEnv}; - -impl FromEnv for ProverGroupConfig { - fn from_env() -> anyhow::Result { - envy_load("prover_group", "PROVER_GROUP_") - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::test_utils::EnvMutex; - - static MUTEX: EnvMutex = EnvMutex::new(); - - fn expected_config() -> ProverGroupConfig { - ProverGroupConfig { - group_0_circuit_ids: vec![0, 18], - group_1_circuit_ids: vec![1, 4], - group_2_circuit_ids: vec![2, 5], - group_3_circuit_ids: vec![6, 7], - group_4_circuit_ids: vec![8, 9], - group_5_circuit_ids: vec![10, 11], - group_6_circuit_ids: vec![12, 13], - group_7_circuit_ids: vec![14, 15], - group_8_circuit_ids: vec![16, 17], - group_9_circuit_ids: vec![3], - region_read_url: "http://metadata.google.internal/computeMetadata/v1/instance/attributes/cluster-location".to_string(), - region_override: Some("us-central-1".to_string()), - zone_read_url: "http://metadata.google.internal/computeMetadata/v1/instance/zone".to_string(), - zone_override: Some("us-central-1-b".to_string()), - synthesizer_per_gpu: 10, - } - } - - const CONFIG: &str = r#" - PROVER_GROUP_GROUP_0_CIRCUIT_IDS="0,18" - PROVER_GROUP_GROUP_1_CIRCUIT_IDS="1,4" - PROVER_GROUP_GROUP_2_CIRCUIT_IDS="2,5" - PROVER_GROUP_GROUP_3_CIRCUIT_IDS="6,7" - PROVER_GROUP_GROUP_4_CIRCUIT_IDS="8,9" - PROVER_GROUP_GROUP_5_CIRCUIT_IDS="10,11" - PROVER_GROUP_GROUP_6_CIRCUIT_IDS="12,13" - PROVER_GROUP_GROUP_7_CIRCUIT_IDS="14,15" - PROVER_GROUP_GROUP_8_CIRCUIT_IDS="16,17" - PROVER_GROUP_GROUP_9_CIRCUIT_IDS="3" - PROVER_GROUP_REGION_READ_URL="http://metadata.google.internal/computeMetadata/v1/instance/attributes/cluster-location" - PROVER_GROUP_REGION_OVERRIDE="us-central-1" - PROVER_GROUP_ZONE_READ_URL="http://metadata.google.internal/computeMetadata/v1/instance/zone" - PROVER_GROUP_ZONE_OVERRIDE="us-central-1-b" - PROVER_GROUP_SYNTHESIZER_PER_GPU="10" - "#; - - #[test] - fn from_env() { - let mut lock = MUTEX.lock(); - lock.set_env(CONFIG); - let actual = ProverGroupConfig::from_env().unwrap(); - assert_eq!(actual, expected_config()); - } - - #[test] - fn get_group_id_for_circuit_id() { - let prover_group_config = expected_config(); - - assert_eq!(Some(0), prover_group_config.get_group_id_for_circuit_id(0)); - assert_eq!(Some(0), prover_group_config.get_group_id_for_circuit_id(18)); - - assert_eq!(Some(1), prover_group_config.get_group_id_for_circuit_id(1)); - assert_eq!(Some(1), prover_group_config.get_group_id_for_circuit_id(4)); - - assert_eq!(Some(2), prover_group_config.get_group_id_for_circuit_id(2)); - assert_eq!(Some(2), prover_group_config.get_group_id_for_circuit_id(5)); - - assert_eq!(Some(3), prover_group_config.get_group_id_for_circuit_id(6)); - assert_eq!(Some(3), prover_group_config.get_group_id_for_circuit_id(7)); - - assert_eq!(Some(4), prover_group_config.get_group_id_for_circuit_id(8)); - assert_eq!(Some(4), prover_group_config.get_group_id_for_circuit_id(9)); - - assert_eq!(Some(5), prover_group_config.get_group_id_for_circuit_id(10)); - assert_eq!(Some(5), prover_group_config.get_group_id_for_circuit_id(11)); - - assert_eq!(Some(6), prover_group_config.get_group_id_for_circuit_id(12)); - assert_eq!(Some(6), prover_group_config.get_group_id_for_circuit_id(13)); - - assert_eq!(Some(7), prover_group_config.get_group_id_for_circuit_id(14)); - assert_eq!(Some(7), prover_group_config.get_group_id_for_circuit_id(15)); - - assert_eq!(Some(8), prover_group_config.get_group_id_for_circuit_id(16)); - assert_eq!(Some(8), prover_group_config.get_group_id_for_circuit_id(17)); - - assert_eq!(Some(9), prover_group_config.get_group_id_for_circuit_id(3)); - assert!(prover_group_config - .get_group_id_for_circuit_id(19) - .is_none()); - } - - #[test] - fn get_circuit_ids_for_group_id() { - let prover_group_config = expected_config(); - - assert_eq!( - Some(vec![0, 18]), - prover_group_config.get_circuit_ids_for_group_id(0) - ); - assert_eq!( - Some(vec![1, 4]), - prover_group_config.get_circuit_ids_for_group_id(1) - ); - assert_eq!( - Some(vec![2, 5]), - prover_group_config.get_circuit_ids_for_group_id(2) - ); - assert_eq!( - Some(vec![6, 7]), - prover_group_config.get_circuit_ids_for_group_id(3) - ); - assert_eq!( - Some(vec![8, 9]), - prover_group_config.get_circuit_ids_for_group_id(4) - ); - assert_eq!( - Some(vec![10, 11]), - prover_group_config.get_circuit_ids_for_group_id(5) - ); - assert_eq!( - Some(vec![12, 13]), - prover_group_config.get_circuit_ids_for_group_id(6) - ); - assert_eq!( - Some(vec![14, 15]), - prover_group_config.get_circuit_ids_for_group_id(7) - ); - assert_eq!( - Some(vec![16, 17]), - prover_group_config.get_circuit_ids_for_group_id(8) - ); - assert_eq!( - Some(vec![3]), - prover_group_config.get_circuit_ids_for_group_id(9) - ); - assert!(prover_group_config - .get_circuit_ids_for_group_id(10) - .is_none()); - } -} diff --git a/core/lib/prover_utils/src/region_fetcher.rs b/core/lib/prover_utils/src/region_fetcher.rs index d2c49dd068dc..fceedd287245 100644 --- a/core/lib/prover_utils/src/region_fetcher.rs +++ b/core/lib/prover_utils/src/region_fetcher.rs @@ -4,23 +4,10 @@ use reqwest::{ header::{HeaderMap, HeaderValue}, Method, }; -use zksync_config::configs::ProverGroupConfig; use zksync_utils::http_with_retries::send_request_with_retries; -pub async fn get_region(prover_group_config: &ProverGroupConfig) -> anyhow::Result { - if let Some(region) = &prover_group_config.region_override { - return Ok(region.clone()); - } - let url = &prover_group_config.region_read_url; - fetch_from_url(url).await.context("fetch_from_url()") -} - -pub async fn get_zone(prover_group_config: &ProverGroupConfig) -> anyhow::Result { - if let Some(zone) = &prover_group_config.zone_override { - return Ok(zone.clone()); - } - let url = &prover_group_config.zone_read_url; - let data = fetch_from_url(url).await.context("fetch_from_url()")?; +pub async fn get_zone(zone_url: &str) -> anyhow::Result { + let data = fetch_from_url(zone_url).await.context("fetch_from_url()")?; parse_zone(&data).context("parse_zone") } @@ -47,9 +34,7 @@ fn parse_zone(data: &str) -> anyhow::Result { #[cfg(test)] mod tests { - use zksync_config::configs::ProverGroupConfig; - - use crate::region_fetcher::{get_region, get_zone, parse_zone}; + use crate::region_fetcher::parse_zone; #[test] fn test_parse_zone() { @@ -63,49 +48,4 @@ mod tests { let data = "invalid data"; assert!(parse_zone(data).is_err()); } - - #[tokio::test] - async fn test_get_region_with_override() { - let config = ProverGroupConfig { - group_0_circuit_ids: vec![], - group_1_circuit_ids: vec![], - group_2_circuit_ids: vec![], - group_3_circuit_ids: vec![], - group_4_circuit_ids: vec![], - group_5_circuit_ids: vec![], - group_6_circuit_ids: vec![], - group_7_circuit_ids: vec![], - group_8_circuit_ids: vec![], - group_9_circuit_ids: vec![], - region_override: Some("us-central-1".to_string()), - region_read_url: "".to_string(), - zone_override: Some("us-central-1-b".to_string()), - zone_read_url: "".to_string(), - synthesizer_per_gpu: 0, - }; - - assert_eq!("us-central-1", get_region(&config).await.unwrap()); - } - - #[tokio::test] - async fn test_get_zone_with_override() { - let config = ProverGroupConfig { - group_0_circuit_ids: vec![], - group_1_circuit_ids: vec![], - group_2_circuit_ids: vec![], - group_3_circuit_ids: vec![], - group_4_circuit_ids: vec![], - group_5_circuit_ids: vec![], - group_6_circuit_ids: vec![], - group_7_circuit_ids: vec![], - group_8_circuit_ids: vec![], - group_9_circuit_ids: vec![], - region_override: Some("us-central-1".to_string()), - region_read_url: "".to_string(), - zone_override: Some("us-central-1-b".to_string()), - zone_read_url: "".to_string(), - synthesizer_per_gpu: 0, - }; - assert_eq!("us-central-1-b", get_zone(&config).await.unwrap()); - } } diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index ce89e099397a..64c279ae0cf5 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -28,7 +28,6 @@ zksync_circuit_breaker = { path = "../circuit_breaker" } zksync_storage = { path = "../storage" } zksync_merkle_tree = { path = "../merkle_tree" } zksync_mini_merkle_tree = { path = "../mini_merkle_tree" } -zksync_verification_key_generator_and_server = { path = "../../bin/verification_key_generator_and_server" } prometheus_exporter = { path = "../prometheus_exporter" } zksync_web3_decl = { path = "../web3_decl", default-features = false, features = [ "server", diff --git a/core/lib/zksync_core/src/house_keeper/gpu_prover_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/gpu_prover_queue_monitor.rs deleted file mode 100644 index ab96b52bedc9..000000000000 --- a/core/lib/zksync_core/src/house_keeper/gpu_prover_queue_monitor.rs +++ /dev/null @@ -1,66 +0,0 @@ -use async_trait::async_trait; -use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; - -#[derive(Debug)] -pub struct GpuProverQueueMonitor { - synthesizer_per_gpu: u16, - reporting_interval_ms: u64, - prover_connection_pool: ConnectionPool, -} - -impl GpuProverQueueMonitor { - pub fn new( - synthesizer_per_gpu: u16, - reporting_interval_ms: u64, - prover_connection_pool: ConnectionPool, - ) -> Self { - Self { - synthesizer_per_gpu, - reporting_interval_ms, - prover_connection_pool, - } - } -} - -/// Invoked periodically to push prover job statistics to Prometheus -/// Note: these values will be used for auto-scaling circuit-synthesizer -#[async_trait] -impl PeriodicJob for GpuProverQueueMonitor { - const SERVICE_NAME: &'static str = "GpuProverQueueMonitor"; - - async fn run_routine_task(&mut self) -> anyhow::Result<()> { - let prover_gpu_count_per_region_zone = self - .prover_connection_pool - .access_storage() - .await - .unwrap() - .gpu_prover_queue_dal() - .get_prover_gpu_count_per_region_zone() - .await; - - for ((region, zone), num_gpu) in prover_gpu_count_per_region_zone { - let synthesizers = self.synthesizer_per_gpu as u64 * num_gpu; - if synthesizers > 0 { - tracing::info!( - "Would be spawning {} circuit synthesizers in region {} zone {}", - synthesizers, - region, - zone - ); - } - metrics::gauge!( - "server.circuit_synthesizer.jobs", - synthesizers as f64, - "region" => region, - "zone" => zone, - "type" => "queued" - ); - } - Ok(()) - } - - fn polling_interval_ms(&self) -> u64 { - self.reporting_interval_ms - } -} diff --git a/core/lib/zksync_core/src/house_keeper/mod.rs b/core/lib/zksync_core/src/house_keeper/mod.rs index b4a5a3047321..fbf59642d356 100644 --- a/core/lib/zksync_core/src/house_keeper/mod.rs +++ b/core/lib/zksync_core/src/house_keeper/mod.rs @@ -6,7 +6,4 @@ pub mod fri_prover_queue_monitor; pub mod fri_scheduler_circuit_queuer; pub mod fri_witness_generator_jobs_retry_manager; pub mod fri_witness_generator_queue_monitor; -pub mod gpu_prover_queue_monitor; -pub mod prover_job_retry_manager; -pub mod prover_queue_monitor; pub mod waiting_to_queued_fri_witness_job_mover; diff --git a/core/lib/zksync_core/src/house_keeper/prover_job_retry_manager.rs b/core/lib/zksync_core/src/house_keeper/prover_job_retry_manager.rs deleted file mode 100644 index f7b630475ea0..000000000000 --- a/core/lib/zksync_core/src/house_keeper/prover_job_retry_manager.rs +++ /dev/null @@ -1,56 +0,0 @@ -use std::time::Duration; - -use async_trait::async_trait; -use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; - -#[derive(Debug)] -pub struct ProverJobRetryManager { - max_attempts: u32, - processing_timeout: Duration, - retry_interval_ms: u64, - prover_connection_pool: ConnectionPool, -} - -impl ProverJobRetryManager { - pub fn new( - max_attempts: u32, - processing_timeout: Duration, - retry_interval_ms: u64, - prover_connection_pool: ConnectionPool, - ) -> Self { - Self { - max_attempts, - processing_timeout, - retry_interval_ms, - prover_connection_pool, - } - } -} - -/// Invoked periodically to re-queue stuck prover jobs. -#[async_trait] -impl PeriodicJob for ProverJobRetryManager { - const SERVICE_NAME: &'static str = "ProverJobRetryManager"; - - async fn run_routine_task(&mut self) -> anyhow::Result<()> { - let stuck_jobs = self - .prover_connection_pool - .access_storage() - .await - .unwrap() - .prover_dal() - .requeue_stuck_jobs(self.processing_timeout, self.max_attempts) - .await; - let job_len = stuck_jobs.len(); - for stuck_job in stuck_jobs { - tracing::info!("re-queuing prover job {:?}", stuck_job); - } - metrics::counter!("server.prover.requeued_jobs", job_len as u64); - Ok(()) - } - - fn polling_interval_ms(&self) -> u64 { - self.retry_interval_ms - } -} diff --git a/core/lib/zksync_core/src/house_keeper/prover_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/prover_queue_monitor.rs deleted file mode 100644 index e0f598d5a590..000000000000 --- a/core/lib/zksync_core/src/house_keeper/prover_queue_monitor.rs +++ /dev/null @@ -1,82 +0,0 @@ -use async_trait::async_trait; -use zksync_config::configs::ProverGroupConfig; -use zksync_dal::ConnectionPool; -use zksync_prover_utils::{circuit_name_to_numeric_index, periodic_job::PeriodicJob}; - -#[derive(Debug)] -pub struct ProverStatsReporter { - reporting_interval_ms: u64, - prover_connection_pool: ConnectionPool, - config: ProverGroupConfig, -} - -impl ProverStatsReporter { - pub fn new( - reporting_interval_ms: u64, - prover_connection_pool: ConnectionPool, - config: ProverGroupConfig, - ) -> Self { - Self { - reporting_interval_ms, - prover_connection_pool, - config, - } - } -} - -/// Invoked periodically to push job statistics to Prometheus -/// Note: these values will be used for manually scaling provers. -#[async_trait] -impl PeriodicJob for ProverStatsReporter { - const SERVICE_NAME: &'static str = "ProverStatsReporter"; - - async fn run_routine_task(&mut self) -> anyhow::Result<()> { - let mut conn = self.prover_connection_pool.access_storage().await.unwrap(); - let stats = conn.prover_dal().get_prover_jobs_stats_per_circuit().await; - - for (circuit_name, stats) in stats.into_iter() { - let group_id = self - .config - .get_group_id_for_circuit_id(circuit_name_to_numeric_index(&circuit_name).unwrap()) - .unwrap(); - - metrics::gauge!( - "server.prover.jobs", - stats.queued as f64, - "type" => "queued", - "prover_group_id" => group_id.to_string(), - "circuit_name" => circuit_name.clone(), - "circuit_type" => circuit_name_to_numeric_index(&circuit_name).unwrap().to_string() - ); - - metrics::gauge!( - "server.prover.jobs", - stats.in_progress as f64, - "type" => "in_progress", - "prover_group_id" => group_id.to_string(), - "circuit_name" => circuit_name.clone(), - "circuit_type" => circuit_name_to_numeric_index(&circuit_name).unwrap().to_string() - ); - } - - if let Some(min_unproved_l1_batch_number) = - conn.prover_dal().min_unproved_l1_batch_number().await - { - metrics::gauge!("server.block_number", min_unproved_l1_batch_number.0 as f64, "stage" => "circuit_aggregation") - } - - let lag_by_circuit_type = conn - .prover_dal() - .min_unproved_l1_batch_number_by_basic_circuit_type() - .await; - - for (circuit_type, l1_batch_number) in lag_by_circuit_type { - metrics::gauge!("server.block_number", l1_batch_number.0 as f64, "stage" => format!("circuit_{}", circuit_type)); - } - Ok(()) - } - - fn polling_interval_ms(&self) -> u64 { - self.reporting_interval_ms - } -} diff --git a/core/lib/zksync_core/src/house_keeper/waiting_to_queued_witness_job_mover.rs b/core/lib/zksync_core/src/house_keeper/waiting_to_queued_witness_job_mover.rs deleted file mode 100644 index 4521b4bfc47b..000000000000 --- a/core/lib/zksync_core/src/house_keeper/waiting_to_queued_witness_job_mover.rs +++ /dev/null @@ -1,95 +0,0 @@ -use async_trait::async_trait; -use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; - -#[derive(Debug)] -pub struct WaitingToQueuedWitnessJobMover { - job_moving_interval_ms: u64, - prover_connection_pool: ConnectionPool, -} - -impl WaitingToQueuedWitnessJobMover { - pub fn new(job_mover_interval_ms: u64, prover_connection_pool: ConnectionPool) -> Self { - Self { - job_moving_interval_ms: job_mover_interval_ms, - prover_connection_pool, - } - } - - async fn move_jobs(&mut self) { - self.move_leaf_aggregation_jobs().await; - self.move_node_aggregation_jobs().await; - self.move_scheduler_jobs().await; - } - - async fn move_leaf_aggregation_jobs(&mut self) { - let mut conn = self.prover_connection_pool.access_storage().await.unwrap(); - let l1_batch_numbers = conn - .witness_generator_dal() - .move_leaf_aggregation_jobs_from_waiting_to_queued() - .await; - let len = l1_batch_numbers.len(); - for l1_batch_number in l1_batch_numbers { - tracing::info!( - "Marked leaf aggregation job for l1_batch {} as queued", - l1_batch_number - ); - } - metrics::counter!( - "server.leaf_witness_generator.waiting_to_queued_jobs_transitions", - len as u64 - ); - } - - async fn move_node_aggregation_jobs(&mut self) { - let mut conn = self.prover_connection_pool.access_storage().await.unwrap(); - let l1_batch_numbers = conn - .witness_generator_dal() - .move_node_aggregation_jobs_from_waiting_to_queued() - .await; - let len = l1_batch_numbers.len(); - for l1_batch_number in l1_batch_numbers { - tracing::info!( - "Marking node aggregation job for l1_batch {} as queued", - l1_batch_number - ); - } - metrics::counter!( - "server.node_witness_generator.waiting_to_queued_jobs_transitions", - len as u64 - ); - } - - async fn move_scheduler_jobs(&mut self) { - let mut conn = self.prover_connection_pool.access_storage().await.unwrap(); - let l1_batch_numbers = conn - .witness_generator_dal() - .move_scheduler_jobs_from_waiting_to_queued() - .await; - let len = l1_batch_numbers.len(); - for l1_batch_number in l1_batch_numbers { - tracing::info!( - "Marking scheduler aggregation job for l1_batch {} as queued", - l1_batch_number - ); - } - metrics::counter!( - "server.scheduler_witness_generator.waiting_to_queued_jobs_transitions", - len as u64 - ); - } -} - -#[async_trait] -impl PeriodicJob for WaitingToQueuedWitnessJobMover { - const SERVICE_NAME: &'static str = "WaitingToQueuedWitnessJobMover"; - - async fn run_routine_task(&mut self) -> anyhow::Result<()> { - self.move_jobs().await; - Ok(()) - } - - fn polling_interval_ms(&self) -> u64 { - self.job_moving_interval_ms - } -} diff --git a/core/lib/zksync_core/src/house_keeper/witness_generator_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/witness_generator_queue_monitor.rs deleted file mode 100644 index da816da3c668..000000000000 --- a/core/lib/zksync_core/src/house_keeper/witness_generator_queue_monitor.rs +++ /dev/null @@ -1,121 +0,0 @@ -use std::collections::HashMap; - -use async_trait::async_trait; -use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; -use zksync_types::proofs::{AggregationRound, JobCountStatistics}; - -const WITNESS_GENERATOR_SERVICE_NAME: &str = "witness_generator"; - -#[derive(Debug)] -pub struct WitnessGeneratorStatsReporter { - reporting_interval_ms: u64, - prover_connection_pool: ConnectionPool, -} - -impl WitnessGeneratorStatsReporter { - pub fn new(reporting_interval_ms: u64, prover_connection_pool: ConnectionPool) -> Self { - Self { - reporting_interval_ms, - prover_connection_pool, - } - } - - async fn get_job_statistics( - prover_connection_pool: &ConnectionPool, - ) -> HashMap { - let mut conn = prover_connection_pool.access_storage().await.unwrap(); - HashMap::from([ - ( - AggregationRound::BasicCircuits, - conn.witness_generator_dal() - .get_witness_jobs_stats(AggregationRound::BasicCircuits) - .await, - ), - ( - AggregationRound::LeafAggregation, - conn.witness_generator_dal() - .get_witness_jobs_stats(AggregationRound::LeafAggregation) - .await, - ), - ( - AggregationRound::NodeAggregation, - conn.witness_generator_dal() - .get_witness_jobs_stats(AggregationRound::NodeAggregation) - .await, - ), - ( - AggregationRound::Scheduler, - conn.witness_generator_dal() - .get_witness_jobs_stats(AggregationRound::Scheduler) - .await, - ), - ]) - } -} - -fn emit_metrics_for_round(round: AggregationRound, stats: JobCountStatistics) { - if stats.queued > 0 || stats.in_progress > 0 { - tracing::trace!( - "Found {} free and {} in progress {:?} witness generators jobs", - stats.queued, - stats.in_progress, - round - ); - } - - metrics::gauge!( - format!("server.{}.jobs", WITNESS_GENERATOR_SERVICE_NAME), - stats.queued as f64, - "type" => "queued", - "round" => format!("{:?}", round) - ); - - metrics::gauge!( - format!("server.{}.jobs", WITNESS_GENERATOR_SERVICE_NAME), - stats.in_progress as f64, - "type" => "in_progress", - "round" => format!("{:?}", round) - ); -} - -/// Invoked periodically to push job statistics to Prometheus -/// Note: these values will be used for auto-scaling job processors -#[async_trait] -impl PeriodicJob for WitnessGeneratorStatsReporter { - const SERVICE_NAME: &'static str = "WitnessGeneratorStatsReporter"; - - async fn run_routine_task(&mut self) -> anyhow::Result<()> { - let stats_for_all_rounds = Self::get_job_statistics(&self.prover_connection_pool).await; - let mut aggregated = JobCountStatistics::default(); - for (round, stats) in stats_for_all_rounds { - emit_metrics_for_round(round, stats); - aggregated = aggregated + stats; - } - - if aggregated.queued > 0 { - tracing::trace!( - "Found {} free {} in progress witness generators jobs", - aggregated.queued, - aggregated.in_progress - ); - } - - metrics::gauge!( - format!("server.{}.jobs", WITNESS_GENERATOR_SERVICE_NAME), - aggregated.queued as f64, - "type" => "queued" - ); - - metrics::gauge!( - format!("server.{}.jobs", WITNESS_GENERATOR_SERVICE_NAME), - aggregated.in_progress as f64, - "type" => "in_progress" - ); - Ok(()) - } - - fn polling_interval_ms(&self) -> u64 { - self.reporting_interval_ms - } -} diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 8269793a40b1..2132ec1cf11a 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -61,8 +61,6 @@ use crate::{ fri_scheduler_circuit_queuer::SchedulerCircuitQueuer, fri_witness_generator_jobs_retry_manager::FriWitnessGeneratorJobRetryManager, fri_witness_generator_queue_monitor::FriWitnessGeneratorStatsReporter, - gpu_prover_queue_monitor::GpuProverQueueMonitor, - prover_job_retry_manager::ProverJobRetryManager, prover_queue_monitor::ProverStatsReporter, waiting_to_queued_fri_witness_job_mover::WaitingToQueuedFriWitnessJobMover, }, l1_gas_price::{GasAdjusterSingleton, L1GasPriceProvider}, @@ -897,32 +895,7 @@ async fn add_house_keeper_to_task_futures( .build() .await .context("failed to build a prover_connection_pool")?; - let prover_group_config = configs - .prover_group_config - .clone() - .context("prover_group_config")?; - let prover_configs = configs.prover_configs.clone().context("prover_configs")?; - let gpu_prover_queue = GpuProverQueueMonitor::new( - prover_group_config.synthesizer_per_gpu, - house_keeper_config.gpu_prover_queue_reporting_interval_ms, - prover_connection_pool.clone(), - ); - let config = prover_configs.non_gpu.clone(); - let prover_job_retry_manager = ProverJobRetryManager::new( - config.max_attempts, - config.proof_generation_timeout(), - house_keeper_config.prover_job_retrying_interval_ms, - prover_connection_pool.clone(), - ); - let prover_stats_reporter = ProverStatsReporter::new( - house_keeper_config.prover_stats_reporting_interval_ms, - prover_connection_pool.clone(), - prover_group_config.clone(), - ); - task_futures.push(tokio::spawn(gpu_prover_queue.run())); task_futures.push(tokio::spawn(l1_batch_metrics_reporter.run())); - task_futures.push(tokio::spawn(prover_stats_reporter.run())); - task_futures.push(tokio::spawn(prover_job_retry_manager.run())); // All FRI Prover related components are configured below. let fri_prover_config = configs diff --git a/core/lib/zksync_core/src/temp_config_store.rs b/core/lib/zksync_core/src/temp_config_store.rs index 0c47f5325ac1..393cd014be82 100644 --- a/core/lib/zksync_core/src/temp_config_store.rs +++ b/core/lib/zksync_core/src/temp_config_store.rs @@ -8,10 +8,10 @@ use zksync_config::{ fri_prover_group::FriProverGroupConfig, house_keeper::HouseKeeperConfig, FriProofCompressorConfig, FriProverConfig, FriWitnessGeneratorConfig, PrometheusConfig, - ProofDataHandlerConfig, ProverGroupConfig, WitnessGeneratorConfig, + ProofDataHandlerConfig, WitnessGeneratorConfig, }, ApiConfig, ContractsConfig, DBConfig, ETHClientConfig, ETHSenderConfig, ETHWatchConfig, - GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, ProverConfigs, + GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, }; // TODO (QIT-22): This structure is going to be removed when components will be respnsible for their own configs. @@ -35,7 +35,6 @@ pub struct TempConfigStore { pub fri_witness_generator_config: Option, pub prometheus_config: Option, pub proof_data_handler_config: Option, - pub prover_group_config: Option, pub witness_generator_config: Option, pub api_config: Option, pub contracts_config: Option, @@ -44,6 +43,5 @@ pub struct TempConfigStore { pub eth_sender_config: Option, pub eth_watch_config: Option, pub gas_adjuster_config: Option, - pub prover_configs: Option, pub object_store_config: Option, } diff --git a/docker/prover-gar/Dockerfile b/docker/prover-gar/Dockerfile deleted file mode 100644 index 1e96e23bbbac..000000000000 --- a/docker/prover-gar/Dockerfile +++ /dev/null @@ -1,21 +0,0 @@ -# Will work locally only after prior universal key download and Docker login to the private registry - -ARG PROVER_IMAGE=latest -FROM us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/prover-v2:2.0-$PROVER_IMAGE as prover - -FROM nvidia/cuda:11.8.0-runtime-ubuntu22.04 as app - -# HACK copying to root is the only way to make Docker layer caching work for these files for some reason -COPY *.bin / -COPY setup_2\^26.key /setup_2\^26.key - -RUN apt-get update && apt-get install -y libpq5 ca-certificates openssl && rm -rf /var/lib/apt/lists/* - -COPY --from=prover contracts/system-contracts/bootloader/build/artifacts/ /contracts/system-contracts/bootloader/build/artifacts/ -COPY --from=prover contracts/system-contracts/artifacts-zk /contracts/system-contracts/artifacts-zk -COPY --from=prover contracts/l1-contracts/artifacts/ /contracts/l1-contracts/artifacts/ -COPY --from=prover contracts/l2-contracts/artifacts-zk/ /contracts/l2-contracts/artifacts-zk/ -COPY --from=prover core/bin/verification_key_generator_and_server/data/ /core/bin/verification_key_generator_and_server/data/ -COPY --from=prover /usr/bin/zksync_prover /usr/bin/ - -ENTRYPOINT ["zksync_prover"] diff --git a/docker/prover/Dockerfile b/docker/prover/Dockerfile deleted file mode 100644 index d6dc38b29873..000000000000 --- a/docker/prover/Dockerfile +++ /dev/null @@ -1,63 +0,0 @@ -# Will work locally only after prior contracts build and universal setup key download - -FROM nvidia/cuda:11.8.0-devel-ubuntu22.04 as builder - -ARG DEBIAN_FRONTEND=noninteractive - -RUN apt-get update && apt-get install -y curl jq clang openssl libssl-dev gcc g++ \ - pkg-config build-essential libclang-dev && \ - rm -rf /var/lib/apt/lists/* - -ENV RUSTUP_HOME=/usr/local/rustup \ - CARGO_HOME=/usr/local/cargo \ - PATH=/usr/local/cargo/bin:$PATH - -RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ - rustup install nightly-2023-08-21 && \ - rustup default nightly-2023-08-21 - -WORKDIR /usr/src/zksync - -ARG ERA_BELLMAN_CUDA_RELEASE -ENV ERA_BELLMAN_CUDA_RELEASE=$ERA_BELLMAN_CUDA_RELEASE -ENV GITHUB_OWNER=matter-labs -ENV GITHUB_REPO=era-bellman-cuda - -RUN set -e; \ - if [ -z "$ERA_BELLMAN_CUDA_RELEASE" ]; then \ - ERA_BELLMAN_CUDA_RELEASE="latest"; \ - fi; \ - if [ "$ERA_BELLMAN_CUDA_RELEASE" = "latest" ]; then \ - ERA_BELLMAN_CUDA_RELEASE=$(curl --silent "https://api.github.com/repos/${GITHUB_OWNER}/${GITHUB_REPO}/releases" | jq -r '.[0].tag_name'); \ - fi; \ - source_url="https://github.com/${GITHUB_OWNER}/${GITHUB_REPO}/archive/refs/tags/${ERA_BELLMAN_CUDA_RELEASE}.tar.gz"; \ - binary_url="https://github.com/${GITHUB_OWNER}/${GITHUB_REPO}/releases/download/${ERA_BELLMAN_CUDA_RELEASE}/bellman-cuda.tar.gz"; \ - curl --silent --location "$source_url" --output bellman-cuda-source.tar.gz; \ - curl --silent --location "$binary_url" --output bellman-cuda.tar.gz; \ - mkdir -p bellman-cuda; \ - tar xvfz bellman-cuda.tar.gz -C ./bellman-cuda; \ - tar xvfz bellman-cuda-source.tar.gz -C ./bellman-cuda --strip-components=1 - -ENV BELLMAN_CUDA_DIR=/usr/src/zksync/bellman-cuda - -COPY . . - -RUN cargo build --release --features gpu - -FROM nvidia/cuda:11.8.0-runtime-ubuntu22.04 as runner - -ARG DEBIAN_FRONTEND=noninteractive - -RUN apt-get update && apt-get install -y libpq5 ca-certificates openssl && rm -rf /var/lib/apt/lists/* - -COPY contracts/system-contracts/bootloader/build/artifacts/ /contracts/system-contracts/bootloader/build/artifacts/ -COPY contracts/system-contracts/artifacts-zk /contracts/system-contracts/artifacts-zk -COPY contracts/l1-contracts/artifacts/ /contracts/l1-contracts/artifacts/ -COPY contracts/l2-contracts/artifacts-zk/ /contracts/l2-contracts/artifacts-zk/ -COPY setup_2\^26.key /etc/ - -COPY core/bin/verification_key_generator_and_server/data/ /core/bin/verification_key_generator_and_server/data/ - -COPY --from=builder /usr/src/zksync/target/release/zksync_prover /usr/bin/ - -ENTRYPOINT ["zksync_prover"] diff --git a/docker/server-v2/Dockerfile b/docker/server-v2/Dockerfile index 90b3375b609c..a7d8fc7487fd 100644 --- a/docker/server-v2/Dockerfile +++ b/docker/server-v2/Dockerfile @@ -40,6 +40,5 @@ COPY contracts/l2-contracts/artifacts-zk/ /contracts/l2-contracts/artifacts-zk/ COPY etc/tokens/ /etc/tokens/ COPY etc/ERC20/ /etc/ERC20/ COPY etc/multivm_bootloaders/ /etc/multivm_bootloaders/ -COPY core/bin/verification_key_generator_and_server/data/ /core/bin/verification_key_generator_and_server/data/ ENTRYPOINT ["zksync_server"] diff --git a/docs/guides/launch.md b/docs/guides/launch.md index b463655719c6..6402bedc3fb7 100644 --- a/docs/guides/launch.md +++ b/docs/guides/launch.md @@ -111,29 +111,6 @@ cargo run --release --bin zksync_verification_key_generator ``` -## Running the setup key generator on machine with GPU - -- uncomment `"core/bin/setup_key_generator_and_server",` from root `Cargo.toml` file. -- ensure that the setup_2^26.key in the current directory, the file can be downloaded from - - -```shell -export BELLMAN_CUDA_DIR=$PWD -# To generate setup key for specific circuit type[0 - 17], 2 below corresponds to circuit type 2. -cargo +nightly run --features gpu --release --bin zksync_setup_key_generator -- --numeric-circuit 2 -``` - -## Running the setup key generator on machine without GPU - -- uncomment `"core/bin/setup_key_generator_and_server",` from root `Cargo.toml` file. -- ensure that the setup_2^26.key in the current directory, the file can be downloaded from - - -```shell -# To generate setup key for specific circuit type[0 - 17], 2 below corresponds to circuit type 2. -cargo +nightly run --release --bin zksync_setup_key_generator -- --numeric-circuit 2 -``` - ## Generating binary verification keys for existing json verification keys ```shell diff --git a/etc/env/base/fri_prover.toml b/etc/env/base/fri_prover.toml index e714f66ed99c..ec21e35e1f00 100644 --- a/etc/env/base/fri_prover.toml +++ b/etc/env/base/fri_prover.toml @@ -10,4 +10,5 @@ specialized_group_id=100 witness_vector_generator_thread_count=5 queue_capacity=10 witness_vector_receiver_port=4000 +zone_read_url="http://metadata.google.internal/computeMetadata/v1/instance/zone" shall_save_to_public_bucket=true diff --git a/etc/env/base/prover.toml b/etc/env/base/prover.toml deleted file mode 100644 index 3504b41343f2..000000000000 --- a/etc/env/base/prover.toml +++ /dev/null @@ -1,74 +0,0 @@ -[prover.non_gpu] -prometheus_port=3313 -initial_setup_key_path="./../../../keys/setup/setup_2^22.key" -key_download_url="https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^22.key" -generation_timeout_in_secs=2700 -number_of_threads=22 -max_attempts=1 -polling_duration_in_millis=750 -setup_keys_path="/usr/src/setup-keys" -number_of_setup_slots=2 -assembly_receiver_port=17791 -assembly_receiver_poll_time_in_millis=250 -assembly_queue_capacity=1 -specialized_prover_group_id=0 - -[prover.two_gpu_forty_gb_mem] -prometheus_port=3313 -initial_setup_key_path="./../../../keys/setup/setup_2^26.key" -key_download_url="https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^26.key" -generation_timeout_in_secs=2700 -number_of_threads=5 -max_attempts=1 -polling_duration_in_millis=750 -setup_keys_path="/usr/src/setup-keys" -number_of_setup_slots=5 -assembly_receiver_port=17791 -assembly_receiver_poll_time_in_millis=250 -assembly_queue_capacity=3 -specialized_prover_group_id=1 - -[prover.one_gpu_eighty_gb_mem] -prometheus_port=3313 -initial_setup_key_path="./../../../keys/setup/setup_2^26.key" -key_download_url="https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^26.key" -generation_timeout_in_secs=2700 -number_of_threads=5 -max_attempts=1 -polling_duration_in_millis=750 -setup_keys_path="/usr/src/setup-keys" -number_of_setup_slots=5 -assembly_receiver_port=17791 -assembly_receiver_poll_time_in_millis=250 -assembly_queue_capacity=3 -specialized_prover_group_id=2 - -[prover.two_gpu_eighty_gb_mem] -prometheus_port=3313 -initial_setup_key_path="./../../../keys/setup/setup_2^26.key" -key_download_url="https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^26.key" -generation_timeout_in_secs=2700 -number_of_threads=9 -max_attempts=1 -polling_duration_in_millis=750 -setup_keys_path="/usr/src/setup-keys" -number_of_setup_slots=11 -assembly_receiver_port=17791 -assembly_receiver_poll_time_in_millis=250 -assembly_queue_capacity=4 -specialized_prover_group_id=3 - -[prover.four_gpu_eighty_gb_mem] -prometheus_port=3313 -initial_setup_key_path="./../../../keys/setup/setup_2^26.key" -key_download_url="https://storage.googleapis.com/matterlabs-setup-keys-us/setup-keys/setup_2^26.key" -generation_timeout_in_secs=2700 -number_of_threads=18 -max_attempts=1 -polling_duration_in_millis=750 -setup_keys_path="/usr/src/setup-keys" -number_of_setup_slots=18 -assembly_receiver_port=17791 -assembly_receiver_poll_time_in_millis=250 -assembly_queue_capacity=20 -specialized_prover_group_id=4 diff --git a/etc/env/base/prover_group.toml b/etc/env/base/prover_group.toml deleted file mode 100644 index 7372a407f3b8..000000000000 --- a/etc/env/base/prover_group.toml +++ /dev/null @@ -1,17 +0,0 @@ -[prover_group] -group_0_circuit_ids="0,18" -group_1_circuit_ids="1,4" -group_2_circuit_ids="2,5" -group_3_circuit_ids="6,7" -group_4_circuit_ids="8,9" -group_5_circuit_ids="10,11" -group_6_circuit_ids="12,13" -group_7_circuit_ids="14,15" -group_8_circuit_ids="16,17" -group_9_circuit_ids="3" -group_100_circuit_ids="" -region_read_url="http://metadata.google.internal/computeMetadata/v1/instance/attributes/cluster-location" -region_override="us-central-1" -zone_read_url="http://metadata.google.internal/computeMetadata/v1/instance/zone" -zone_override="us-central-1-b" -synthesizer_per_gpu="10" diff --git a/etc/env/base/rust.toml b/etc/env/base/rust.toml index 093ba04da738..0374867630d4 100644 --- a/etc/env/base/rust.toml +++ b/etc/env/base/rust.toml @@ -7,7 +7,6 @@ RUST_LOG="""\ zksync_core=debug,\ zksync_server=debug,\ -zksync_prover=debug,\ zksync_contract_verifier=debug,\ zksync_dal=info,\ zksync_eth_client=info,\ @@ -22,9 +21,7 @@ zksync_mempool=debug,\ loadnext=info,\ vm=info,\ block_sizes_test=info,\ -zksync_verification_key_generator_and_server=info,\ zksync_object_store=info,\ -setup_key_generator_and_server=info,\ en_playground=info,\ zksync_external_node=info,\ cross_nodes_checker=debug,\ diff --git a/infrastructure/zk/src/config.ts b/infrastructure/zk/src/config.ts index dbbbc5e81fc6..6d2b722c0d25 100644 --- a/infrastructure/zk/src/config.ts +++ b/infrastructure/zk/src/config.ts @@ -16,11 +16,9 @@ const CONFIG_FILES = [ 'misc.toml', 'object_store.toml', 'nfs.toml', - 'prover.toml', 'rust.toml', 'private.toml', 'witness_generator.toml', - 'prover_group.toml', 'house_keeper.toml', 'fri_prover.toml', 'fri_witness_generator.toml', diff --git a/prover/Cargo.lock b/prover/Cargo.lock index 8e4c1fd01d8b..c5c7d1bc7be9 100644 --- a/prover/Cargo.lock +++ b/prover/Cargo.lock @@ -145,18 +145,6 @@ version = "1.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" -[[package]] -name = "api" -version = "0.1.0" -source = "git+https://github.com/matter-labs/era-heavy-ops-service.git?branch=v1.3.3#ac6a3af6415dc12c9ae2932fa5ad906939023d82" -dependencies = [ - "bellman_ce 0.3.2 (git+https://github.com/matter-labs/bellman?branch=dev)", - "cfg-if 1.0.0", - "gpu-prover", - "num_cpus", - "serde", -] - [[package]] name = "arr_macro" version = "0.1.3" @@ -273,18 +261,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" -[[package]] -name = "backon" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c1a6197b2120bb2185a267f6515038558b019e92b832bb0320e96d66268dcf9" -dependencies = [ - "fastrand 1.9.0", - "futures-core", - "pin-project", - "tokio", -] - [[package]] name = "backtrace" version = "0.3.69" @@ -397,29 +373,6 @@ dependencies = [ "serde", ] -[[package]] -name = "bindgen" -version = "0.59.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bd2a9a458e8f4304c52c43ebb0cfbd520289f8379a52e329a38afda99bf8eb8" -dependencies = [ - "bitflags 1.3.2", - "cexpr", - "clang-sys", - "clap 2.34.0", - "env_logger 0.9.3", - "lazy_static", - "lazycell", - "log", - "peeking_take_while", - "proc-macro2 1.0.69", - "quote 1.0.33", - "regex", - "rustc-hash", - "shlex", - "which", -] - [[package]] name = "bindgen" version = "0.65.1" @@ -1780,15 +1733,6 @@ version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" -[[package]] -name = "fastrand" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] - [[package]] name = "fastrand" version = "2.0.1" @@ -2079,17 +2023,6 @@ version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" -[[package]] -name = "futures-locks" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45ec6fe3675af967e67c5536c0b9d44e34e6c52f86bedc4ea49c5317b8e94d06" -dependencies = [ - "futures-channel", - "futures-task", - "tokio", -] - [[package]] name = "futures-macro" version = "0.3.29" @@ -2255,35 +2188,6 @@ dependencies = [ "async-trait", ] -[[package]] -name = "gpu-ffi" -version = "0.1.0" -source = "git+https://github.com/matter-labs/era-heavy-ops-service.git?branch=v1.3.3#ac6a3af6415dc12c9ae2932fa5ad906939023d82" -dependencies = [ - "bindgen 0.59.2", - "crossbeam 0.8.2", - "derivative", - "futures 0.3.29", - "futures-locks", - "num_cpus", -] - -[[package]] -name = "gpu-prover" -version = "0.1.0" -source = "git+https://github.com/matter-labs/era-heavy-ops-service.git?branch=v1.3.3#ac6a3af6415dc12c9ae2932fa5ad906939023d82" -dependencies = [ - "bit-vec", - "cfg-if 1.0.0", - "crossbeam 0.8.2", - "franklin-crypto 0.0.5 (git+https://github.com/matter-labs/franklin-crypto?branch=dev)", - "gpu-ffi", - "itertools 0.11.0", - "num_cpus", - "rand 0.4.6", - "serde", -] - [[package]] name = "group" version = "0.12.1" @@ -4204,22 +4108,6 @@ dependencies = [ "thiserror", ] -[[package]] -name = "prover-service" -version = "0.1.0" -source = "git+https://github.com/matter-labs/era-heavy-ops-service.git?branch=v1.3.3#ac6a3af6415dc12c9ae2932fa5ad906939023d82" -dependencies = [ - "api", - "bincode", - "crossbeam-utils 0.8.16", - "log", - "num_cpus", - "rand 0.4.6", - "serde", - "serde_json", - "zkevm_test_harness 1.3.3", -] - [[package]] name = "pulldown-cmark" version = "0.9.3" @@ -5192,24 +5080,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "setup_key_generator_and_server" -version = "0.1.0" -dependencies = [ - "anyhow", - "api", - "circuit_testing", - "itertools 0.10.5", - "prover-service", - "structopt", - "tracing", - "vlog", - "zkevm_test_harness 1.3.3", - "zksync_config", - "zksync_env_config", - "zksync_types", -] - [[package]] name = "sha-1" version = "0.10.1" @@ -5769,7 +5639,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ef1adac450ad7f4b3c28589471ade84f25f731a7a0fe30d71dfa9f60fd808e5" dependencies = [ "cfg-if 1.0.0", - "fastrand 2.0.1", + "fastrand", "redox_syscall 0.4.1", "rustix", "windows-sys", @@ -6933,28 +6803,6 @@ dependencies = [ "web3", ] -[[package]] -name = "zksync_circuit_breaker" -version = "0.1.0" -dependencies = [ - "anyhow", - "async-trait", - "backon", - "convert_case 0.6.0", - "futures 0.3.29", - "hex", - "metrics", - "serde_json", - "thiserror", - "tokio", - "tracing", - "zksync_config", - "zksync_contracts", - "zksync_dal", - "zksync_eth_client", - "zksync_types", -] - [[package]] name = "zksync_concurrency" version = "0.1.0" @@ -7117,43 +6965,6 @@ dependencies = [ "zksync_config", ] -[[package]] -name = "zksync_eth_client" -version = "0.1.0" -dependencies = [ - "anyhow", - "async-trait", - "hex", - "jsonrpc-core", - "serde", - "thiserror", - "tokio", - "tracing", - "vise", - "zksync_config", - "zksync_contracts", - "zksync_eth_signer", - "zksync_types", -] - -[[package]] -name = "zksync_eth_signer" -version = "0.1.0" -dependencies = [ - "async-trait", - "hex", - "jsonrpc-core", - "parity-crypto", - "reqwest", - "rlp", - "secp256k1 0.27.0", - "serde", - "serde_derive", - "serde_json", - "thiserror", - "zksync_types", -] - [[package]] name = "zksync_health_check" version = "0.1.0" @@ -7258,44 +7069,6 @@ dependencies = [ "syn 2.0.39", ] -[[package]] -name = "zksync_prover" -version = "0.1.0" -dependencies = [ - "anyhow", - "api", - "bincode", - "chrono", - "ctrlc", - "ethabi", - "futures 0.3.29", - "hex", - "local-ip-address", - "prometheus_exporter", - "prover-service", - "queues", - "reqwest", - "serde", - "serde_json", - "setup_key_generator_and_server", - "thiserror", - "tokio", - "tracing", - "vise", - "vlog", - "zkevm_test_harness 1.3.3", - "zksync_circuit_breaker", - "zksync_config", - "zksync_dal", - "zksync_env_config", - "zksync_eth_client", - "zksync_object_store", - "zksync_prover_utils", - "zksync_types", - "zksync_utils", - "zksync_verification_key_generator_and_server", -] - [[package]] name = "zksync_prover_fri" version = "0.1.0" @@ -7505,25 +7278,6 @@ dependencies = [ "zksync_basic_types", ] -[[package]] -name = "zksync_verification_key_generator_and_server" -version = "0.1.0" -dependencies = [ - "anyhow", - "bincode", - "circuit_testing", - "ff_ce", - "hex", - "itertools 0.10.5", - "once_cell", - "serde_json", - "structopt", - "tracing", - "vlog", - "zksync_prover_utils", - "zksync_types", -] - [[package]] name = "zksync_witness_generator" version = "0.1.0" diff --git a/prover/Cargo.toml b/prover/Cargo.toml index 4bd8af4bbddc..e450fce09c08 100644 --- a/prover/Cargo.toml +++ b/prover/Cargo.toml @@ -5,8 +5,6 @@ members = [ "prover_fri_types", # binaries - "prover", - "setup_key_generator_and_server", "witness_generator", "vk_setup_data_generator_server_fri", "prover_fri", diff --git a/prover/README.md b/prover/README.md index 01f1c3785f1e..5e537bf8bc0b 100644 --- a/prover/README.md +++ b/prover/README.md @@ -35,15 +35,3 @@ witness_vector_generators, that can 'share' as single gpu based prover_fri). ### proof_fri_compressor Used as a 'last step' to compress/wrap the final FRI proof into a SNARK (to make L1 verification cheaper). - -## Old proof system - -Some of the components here belong to the old proof system: - -- prover -- setup_key_generator_and_server - -Moreover old proof system is also using components from 'core' directory, like: - -- core/bin/verification_key_generator_and_server -- core/lib/zksycn_core/src/witness_generator. diff --git a/prover/prover/Cargo.toml b/prover/prover/Cargo.toml deleted file mode 100644 index aa91013f0ff6..000000000000 --- a/prover/prover/Cargo.toml +++ /dev/null @@ -1,57 +0,0 @@ -[package] -name = "zksync_prover" -version = "0.1.0" -edition = "2018" -authors = ["The Matter Labs Team "] -homepage = "https://zksync.io/" -repository = "https://github.com/matter-labs/zksync-era" -license = "MIT OR Apache-2.0" -keywords = ["blockchain", "zksync"] -categories = ["cryptography"] -publish = false # We don't want to publish our binaries. - -[dependencies] -vise = { git = "https://github.com/matter-labs/vise.git", version = "0.1.0", rev = "1c9cc500e92cf9ea052b230e114a6f9cce4fb2c1" } - -zksync_dal = { path = "../../core/lib/dal" } -zksync_config = { path = "../../core/lib/config" } -zksync_env_config = { path = "../../core/lib/env_config" } -zksync_utils = { path = "../../core/lib/utils" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } -zksync_circuit_breaker = { path = "../../core/lib/circuit_breaker" } -zksync_eth_client = { path = "../../core/lib/eth_client" } -zksync_types = { path = "../../core/lib/types" } -prometheus_exporter = { path = "../../core/lib/prometheus_exporter" } -vlog = { path = "../../core/lib/vlog" } -zksync_verification_key_generator_and_server = { path = "../../core/bin/verification_key_generator_and_server" } -zksync_object_store = { path = "../../core/lib/object_store" } - -setup_key_generator_and_server = { path = "../setup_key_generator_and_server" } -api = { git = "https://github.com/matter-labs/era-heavy-ops-service.git", branch = "v1.3.3", features = [ - "gpu", -], optional = true, default-features = false } -prover-service = { git = "https://github.com/matter-labs/era-heavy-ops-service.git", branch = "v1.3.3", features = [ - "gpu", -], optional = true, default-features = false } - -zkevm_test_harness = { git = "https://github.com/matter-labs/era-zkevm_test_harness.git", branch = "v1.3.3" } - -anyhow = "1.0" -tracing = "0.1" -tokio = { version = "1", features = ["time"] } -futures = { version = "0.3", features = ["compat"] } -ctrlc = { version = "3.1", features = ["termination"] } -thiserror = "1.0" -chrono = "0.4" -serde_json = "1.0" -ethabi = "18.0.0" -hex = "0.4" -serde = { version = "1.0", features = ["derive"] } -bincode = "1.3.2" -reqwest = { version = "0.11", features = ["blocking"] } -queues = "1.1.0" -local-ip-address = "0.5.0" - -[features] -default = [] -gpu = ["api", "prover-service", "setup_key_generator_and_server/gpu"] diff --git a/prover/prover/README.md b/prover/prover/README.md deleted file mode 100644 index 6ccb690a33f3..000000000000 --- a/prover/prover/README.md +++ /dev/null @@ -1,8 +0,0 @@ -# OLD PROVER - OBSOLETE - -For compiling locally (no cuda) set `features=["legacy"], default-features=false` for: - -- `./Cargo.toml`: `heavy-ops-service` dependency. -- `../setup_key_generator_and_server/Cargo.toml`: `api` and `prover-service` dependencies. - -**! Don't push those changes !** diff --git a/prover/prover/src/artifact_provider.rs b/prover/prover/src/artifact_provider.rs deleted file mode 100644 index 9af365d95f40..000000000000 --- a/prover/prover/src/artifact_provider.rs +++ /dev/null @@ -1,25 +0,0 @@ -use std::io::Read; - -use anyhow::Context as _; -use prover_service::ArtifactProvider; -use zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncVerificationKey, pairing::bn256::Bn256, -}; -use zksync_setup_key_server::get_setup_for_circuit_type; -use zksync_verification_key_server::get_vk_for_circuit_type; - -#[derive(Debug)] -pub struct ProverArtifactProvider; - -impl ArtifactProvider for ProverArtifactProvider { - type ArtifactError = anyhow::Error; - - fn get_setup(&self, circuit_id: u8) -> Result, Self::ArtifactError> { - get_setup_for_circuit_type(circuit_id).context("get_setup_for_circuit_type()") - } - - fn get_vk(&self, circuit_id: u8) -> Result, Self::ArtifactError> { - let vk = get_vk_for_circuit_type(circuit_id); - Ok(ZkSyncVerificationKey::from_verification_key_and_numeric_type(circuit_id, vk)) - } -} diff --git a/prover/prover/src/main.rs b/prover/prover/src/main.rs deleted file mode 100644 index 56ac77336c24..000000000000 --- a/prover/prover/src/main.rs +++ /dev/null @@ -1,26 +0,0 @@ -#![cfg_attr(not(feature = "gpu"), allow(unused_imports))] - -#[cfg(feature = "gpu")] -mod artifact_provider; -mod metrics; -#[cfg(feature = "gpu")] -mod prover; -#[cfg(feature = "gpu")] -mod prover_params; -#[cfg(feature = "gpu")] -mod run; -#[cfg(feature = "gpu")] -mod socket_listener; -#[cfg(feature = "gpu")] -mod synthesized_circuit_provider; - -#[cfg(not(feature = "gpu"))] -fn main() { - unimplemented!("This binary is only available with `gpu` feature enabled"); -} - -#[cfg(feature = "gpu")] -#[tokio::main] -async fn main() -> anyhow::Result<()> { - run::run().await -} diff --git a/prover/prover/src/metrics.rs b/prover/prover/src/metrics.rs deleted file mode 100644 index ab18c59bcf7c..000000000000 --- a/prover/prover/src/metrics.rs +++ /dev/null @@ -1,41 +0,0 @@ -use std::time::Duration; - -use vise::{Buckets, Counter, Histogram, LabeledFamily, Metrics}; - -const PROVER_LATENCY_BUCKETS: Buckets = Buckets::values(&[ - 1.0, 10.0, 20.0, 40.0, 60.0, 120.0, 240.0, 360.0, 600.0, 1800.0, 3600.0, -]); - -#[derive(Debug, Metrics)] -#[metrics(prefix = "prover")] -pub(crate) struct ProverMetrics { - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub proof_generation_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub circuit_synthesis_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub assembly_finalize_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub assembly_encoding_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub assembly_decoding_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub assembly_transferring_time: LabeledFamily>, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["circuit_type"])] - pub setup_load_time: LabeledFamily>, - #[metrics(labels = ["circuit_type"])] - pub setup_loading_cache_miss: LabeledFamily, - #[metrics(buckets = PROVER_LATENCY_BUCKETS)] - pub prover_wait_idle_time: Histogram, - #[metrics(buckets = PROVER_LATENCY_BUCKETS)] - pub setup_load_wait_idle_time: Histogram, - #[metrics(buckets = PROVER_LATENCY_BUCKETS)] - pub scheduler_wait_idle_time: Histogram, - #[metrics(buckets = PROVER_LATENCY_BUCKETS)] - pub download_time: Histogram, - #[metrics(buckets = PROVER_LATENCY_BUCKETS, labels = ["queue_capacity"])] - pub queue_free_slots: LabeledFamily>, -} - -#[vise::register] -pub(crate) static METRICS: vise::Global = vise::Global::new(); diff --git a/prover/prover/src/prover.rs b/prover/prover/src/prover.rs deleted file mode 100644 index efb570050b57..000000000000 --- a/prover/prover/src/prover.rs +++ /dev/null @@ -1,286 +0,0 @@ -use std::{env, time::Duration}; - -use anyhow::Context as _; -use prover_service::{ - JobReporter, - JobResult::{self, Failure, ProofGenerated}, -}; -use tokio::runtime::Handle; -use zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncProof, pairing::bn256::Bn256, -}; -use zksync_config::{PostgresConfig, ProverConfig}; -use zksync_dal::{ConnectionPool, StorageProcessor}; -use zksync_object_store::{Bucket, ObjectStore, ObjectStoreFactory}; -use zksync_types::proofs::ProverJobMetadata; - -use crate::metrics::METRICS; - -#[derive(Debug)] -pub struct ProverReporter { - rt_handle: Handle, - pool: ConnectionPool, - config: ProverConfig, - processed_by: String, - object_store: Box, -} - -fn assembly_debug_blob_url(job_id: usize, circuit_id: u8) -> String { - format!("assembly_debugging_{}_{}.bin", job_id, circuit_id) -} - -impl ProverReporter { - pub(crate) fn new( - postgres_config: PostgresConfig, - config: ProverConfig, - store_factory: &ObjectStoreFactory, - rt_handle: Handle, - ) -> anyhow::Result { - let pool = rt_handle - .block_on(ConnectionPool::singleton(postgres_config.prover_url()?).build()) - .context("failed to build a connection pool")?; - Ok(Self { - pool, - config, - processed_by: env::var("POD_NAME").unwrap_or("Unknown".to_string()), - object_store: rt_handle.block_on(store_factory.create_store()), - rt_handle, - }) - } - - fn handle_successful_proof_generation( - &self, - job_id: usize, - proof: ZkSyncProof, - duration: Duration, - index: usize, - ) { - let circuit_type = self.get_circuit_type(job_id); - let serialized = bincode::serialize(&proof).expect("Failed to serialize proof"); - tracing::info!( - "Successfully generated proof with id {:?} and type: {} for index: {}. Size: {:?}KB took: {:?}", - job_id, - circuit_type, - index, - serialized.len() >> 10, - duration, - ); - - METRICS.proof_generation_time[&circuit_type].observe(duration); - - let job_id = job_id as u32; - self.rt_handle.block_on(async { - let mut connection = self.pool.access_storage().await.unwrap(); - let mut transaction = connection.start_transaction().await.unwrap(); - - // BEWARE, HERE BE DRAGONS. - // `send_report` method is called in an operating system thread, - // which is in charge of saving proof output (ok, errored, etc.). - // The code that calls it is in a thread that does not check it's status. - // If the thread panics, proofs will be generated, but their status won't be saved. - // So a prover will work like this: - // Pick task, execute task, prepare task to be saved, be restarted as nothing happens. - // The error prevents the "fake" work by killing the prover, which causes it to restart. - // A proper fix would be to have the thread signal it was dead or be watched from outside. - // Given we want to deprecate old prover, this is the quick and dirty hack I'm not proud of. - let result = transaction - .prover_dal() - .save_proof(job_id, duration, serialized, &self.processed_by) - .await; - if let Err(e) = result { - tracing::warn!("panicked inside heavy-ops thread: {e:?}; exiting..."); - std::process::exit(-1); - } - self.get_prover_job_metadata_by_id_and_exit_if_error(&mut transaction, job_id) - .await; - transaction.commit().await.unwrap(); - }); - } - - fn get_circuit_type(&self, job_id: usize) -> String { - let prover_job_metadata = self.rt_handle.block_on(async { - let mut connection = self.pool.access_storage().await.unwrap(); - self.get_prover_job_metadata_by_id_and_exit_if_error(&mut connection, job_id as u32) - .await - }); - prover_job_metadata.circuit_type - } - - async fn get_prover_job_metadata_by_id_and_exit_if_error( - &self, - connection: &mut StorageProcessor<'_>, - job_id: u32, - ) -> ProverJobMetadata { - // BEWARE, HERE BE DRAGONS. - // `send_report` method is called in an operating system thread, - // which is in charge of saving proof output (ok, errored, etc.). - // The code that calls it is in a thread that does not check it's status. - // If the thread panics, proofs will be generated, but their status won't be saved. - // So a prover will work like this: - // Pick task, execute task, prepare task to be saved, be restarted as nothing happens. - // The error prevents the "fake" work by killing the prover, which causes it to restart. - // A proper fix would be to have the thread signal it was dead or be watched from outside. - // Given we want to deprecate old prover, this is the quick and dirty hack I'm not proud of. - let result = connection.prover_dal().get_prover_job_by_id(job_id).await; - let prover_job_metadata = match result { - Ok(option) => option, - Err(e) => { - tracing::warn!("panicked inside heavy-ops thread: {e:?}; exiting..."); - std::process::exit(-1); - } - }; - match prover_job_metadata { - Some(val) => val, - None => { - tracing::error!("No job with id: {} exist; exiting...", job_id); - std::process::exit(-1); - } - } - } -} - -impl JobReporter for ProverReporter { - fn send_report(&mut self, report: JobResult) { - match report { - Failure(job_id, error) => { - tracing::error!( - "Failed to generate proof for id {:?}. error reason; {}", - job_id, - error - ); - self.rt_handle.block_on(async { - let result = self - .pool - .access_storage() - .await - .unwrap() - .prover_dal() - .save_proof_error(job_id as u32, error, self.config.max_attempts) - .await; - // BEWARE, HERE BE DRAGONS. - // `send_report` method is called in an operating system thread, - // which is in charge of saving proof output (ok, errored, etc.). - // The code that calls it is in a thread that does not check it's status. - // If the thread panics, proofs will be generated, but their status won't be saved. - // So a prover will work like this: - // Pick task, execute task, prepare task to be saved, be restarted as nothing happens. - // The error prevents the "fake" work by killing the prover, which causes it to restart. - // A proper fix would be to have the thread signal it was dead or be watched from outside. - // Given we want to deprecate old prover, this is the quick and dirty hack I'm not proud of. - if let Err(e) = result { - tracing::warn!("panicked inside heavy-ops thread: {e:?}; exiting..."); - std::process::exit(-1); - } - }); - } - - ProofGenerated(job_id, duration, proof, index) => { - self.handle_successful_proof_generation(job_id, proof, duration, index); - } - - JobResult::Synthesized(job_id, duration) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully synthesized circuit with id {:?} and type: {}. took: {:?}", - job_id, - circuit_type, - duration, - ); - METRICS.circuit_synthesis_time[&circuit_type].observe(duration); - } - - JobResult::AssemblyFinalized(job_id, duration) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully finalized assembly with id {:?} and type: {}. took: {:?}", - job_id, - circuit_type, - duration, - ); - METRICS.assembly_finalize_time[&circuit_type].observe(duration); - } - - JobResult::SetupLoaded(job_id, duration, cache_miss) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully setup loaded with id {:?} and type: {}. \ - took: {:?} and had cache_miss: {}", - job_id, - circuit_type, - duration, - cache_miss - ); - METRICS.setup_load_time[&circuit_type].observe(duration); - METRICS.setup_loading_cache_miss[&circuit_type].inc(); - } - - JobResult::AssemblyEncoded(job_id, duration) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully encoded assembly with id {:?} and type: {}. took: {:?}", - job_id, - circuit_type, - duration, - ); - METRICS.assembly_encoding_time[&circuit_type].observe(duration); - } - - JobResult::AssemblyDecoded(job_id, duration) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully decoded assembly with id {:?} and type: {}. took: {:?}", - job_id, - circuit_type, - duration, - ); - METRICS.assembly_decoding_time[&circuit_type].observe(duration); - } - - JobResult::FailureWithDebugging(job_id, circuit_id, assembly, error) => { - tracing::trace!( - "Failed assembly decoding for job-id {} and circuit-type: {}. error: {}", - job_id, - circuit_id, - error, - ); - let blob_url = assembly_debug_blob_url(job_id, circuit_id); - let put_task = self - .object_store - .put_raw(Bucket::ProverJobs, &blob_url, assembly); - self.rt_handle - .block_on(put_task) - .expect("Failed saving debug assembly to GCS"); - } - - JobResult::AssemblyTransferred(job_id, duration) => { - let circuit_type = self.get_circuit_type(job_id); - tracing::trace!( - "Successfully transferred assembly with id {:?} and type: {}. took: {:?}", - job_id, - circuit_type, - duration, - ); - METRICS.assembly_transferring_time[&circuit_type].observe(duration); - } - - JobResult::ProverWaitedIdle(prover_id, duration) => { - tracing::trace!( - "Prover wait idle time: {:?} for prover-id: {:?}", - duration, - prover_id - ); - METRICS.prover_wait_idle_time.observe(duration); - } - - JobResult::SetupLoaderWaitedIdle(duration) => { - tracing::trace!("Setup load wait idle time: {:?}", duration); - METRICS.setup_load_wait_idle_time.observe(duration); - } - - JobResult::SchedulerWaitedIdle(duration) => { - tracing::trace!("Scheduler wait idle time: {:?}", duration); - METRICS.scheduler_wait_idle_time.observe(duration); - } - } - } -} diff --git a/prover/prover/src/prover_params.rs b/prover/prover/src/prover_params.rs deleted file mode 100644 index fc59b88ddf77..000000000000 --- a/prover/prover/src/prover_params.rs +++ /dev/null @@ -1,35 +0,0 @@ -use std::time::Duration; - -use prover_service::Params; -use zksync_config::ProverConfig; - -#[derive(Debug)] -pub struct ProverParams { - number_of_threads: u8, - polling_duration: Duration, - number_of_setup_slots: u8, -} - -impl ProverParams { - pub(crate) fn new(config: &ProverConfig) -> Self { - Self { - number_of_threads: config.number_of_threads as u8, - polling_duration: Duration::from_millis(config.polling_duration_in_millis), - number_of_setup_slots: config.number_of_setup_slots, - } - } -} - -impl Params for ProverParams { - fn number_of_parallel_synthesis(&self) -> u8 { - self.number_of_threads - } - - fn number_of_setup_slots(&self) -> u8 { - self.number_of_setup_slots - } - - fn polling_duration(&self) -> Duration { - self.polling_duration - } -} diff --git a/prover/prover/src/run.rs b/prover/prover/src/run.rs deleted file mode 100644 index 9784b2f1b666..000000000000 --- a/prover/prover/src/run.rs +++ /dev/null @@ -1,255 +0,0 @@ -use std::{env, future::Future, sync::Arc, time::Instant}; - -use anyhow::Context as _; -use local_ip_address::local_ip; -use prometheus_exporter::PrometheusExporterConfig; -use queues::Buffer; -use tokio::sync::{oneshot, Mutex}; -use zksync_config::{ - configs::{ - api::PrometheusConfig, prover_group::ProverGroupConfig, AlertsConfig, ObjectStoreConfig, - }, - ApiConfig, PostgresConfig, ProverConfig, ProverConfigs, -}; -use zksync_dal::ConnectionPool; -use zksync_env_config::FromEnv; -use zksync_object_store::ObjectStoreFactory; -use zksync_prover_utils::region_fetcher::{get_region, get_zone}; -use zksync_types::proofs::{GpuProverInstanceStatus, SocketAddress}; -use zksync_utils::wait_for_tasks::wait_for_tasks; - -use crate::{ - artifact_provider::ProverArtifactProvider, metrics::METRICS, prover::ProverReporter, - prover_params::ProverParams, socket_listener::incoming_socket_listener, - synthesized_circuit_provider::SynthesizedCircuitProvider, -}; - -async fn graceful_shutdown() -> anyhow::Result> { - let postgres_config = PostgresConfig::from_env().context("PostgresConfig::from_env()")?; - let pool = ConnectionPool::singleton(postgres_config.prover_url()?) - .build() - .await - .context("failed to build a connection pool")?; - let host = local_ip().context("Failed obtaining local IP address")?; - let port = ProverConfigs::from_env() - .context("ProverConfigs")? - .non_gpu - .assembly_receiver_port; - let prover_group_config = - ProverGroupConfig::from_env().context("ProverGroupConfig::from_env()")?; - let region = get_region(&prover_group_config) - .await - .context("get_region()")?; - let zone = get_zone(&prover_group_config).await.context("get_zone()")?; - let address = SocketAddress { host, port }; - Ok(async move { - pool.access_storage() - .await - .unwrap() - .gpu_prover_queue_dal() - .update_prover_instance_status(address, GpuProverInstanceStatus::Dead, 0, region, zone) - .await - }) -} - -fn get_ram_per_gpu() -> anyhow::Result { - use api::gpu_prover::cuda_bindings; - - let device_info = - cuda_bindings::device_info(0).map_err(|err| anyhow::anyhow!("device_info(): {err:?}"))?; - let ram_in_gb: u64 = device_info.total / (1024 * 1024 * 1024); - tracing::info!("Detected RAM per GPU: {:?} GB", ram_in_gb); - Ok(ram_in_gb) -} - -fn get_prover_config_for_machine_type() -> anyhow::Result<(ProverConfig, u8)> { - use api::gpu_prover::cuda_bindings; - - let prover_configs = ProverConfigs::from_env().context("ProverConfigs::from_env()")?; - let actual_num_gpus = match cuda_bindings::devices() { - Ok(gpus) => gpus as u8, - Err(err) => { - tracing::error!("unable to get number of GPUs: {err:?}"); - anyhow::bail!("unable to get number of GPUs: {:?}", err); - } - }; - tracing::info!("detected number of gpus: {}", actual_num_gpus); - let ram_in_gb = get_ram_per_gpu().context("get_ram_per_gpu()")?; - - Ok(match actual_num_gpus { - 1 => { - tracing::info!("Detected machine type with 1 GPU and 80GB RAM"); - (prover_configs.one_gpu_eighty_gb_mem, actual_num_gpus) - } - 2 => { - if ram_in_gb > 39 { - tracing::info!("Detected machine type with 2 GPU and 80GB RAM"); - (prover_configs.two_gpu_eighty_gb_mem, actual_num_gpus) - } else { - tracing::info!("Detected machine type with 2 GPU and 40GB RAM"); - (prover_configs.two_gpu_forty_gb_mem, actual_num_gpus) - } - } - 4 => { - tracing::info!("Detected machine type with 4 GPU and 80GB RAM"); - (prover_configs.four_gpu_eighty_gb_mem, actual_num_gpus) - } - _ => anyhow::bail!("actual_num_gpus: {} not supported yet", actual_num_gpus), - }) -} - -pub async fn run() -> anyhow::Result<()> { - #[allow(deprecated)] // TODO (QIT-21): Use centralized configuration approach. - let log_format = vlog::log_format_from_env(); - #[allow(deprecated)] // TODO (QIT-21): Use centralized configuration approach. - let sentry_url = vlog::sentry_url_from_env(); - #[allow(deprecated)] // TODO (QIT-21): Use centralized configuration approach. - let environment = vlog::environment_from_env(); - - let mut builder = vlog::ObservabilityBuilder::new().with_log_format(log_format); - if let Some(sentry_url) = sentry_url { - builder = builder - .with_sentry_url(&sentry_url) - .context("Invalid Sentry URL")? - .with_sentry_environment(environment); - } - let _guard = builder.build(); - - tracing::trace!("starting prover"); - let (prover_config, num_gpu) = - get_prover_config_for_machine_type().context("get_prover_config_for_machine_type()")?; - - let prometheus_config = PrometheusConfig { - listener_port: prover_config.prometheus_port, - ..ApiConfig::from_env() - .context("ApiConfig::from_env()")? - .prometheus - }; - - let prover_group_config = - ProverGroupConfig::from_env().context("ProverGroupConfig::from_env()")?; - let region = get_region(&prover_group_config) - .await - .context("get_region()")?; - let zone = get_zone(&prover_group_config).await.context("get_zone()")?; - - let (stop_signal_sender, stop_signal_receiver) = oneshot::channel(); - let mut stop_signal_sender = Some(stop_signal_sender); - ctrlc::set_handler(move || { - if let Some(sender) = stop_signal_sender.take() { - sender.send(()).ok(); - } - }) - .expect("Error setting Ctrl+C handler"); - - let started_at = Instant::now(); - zksync_prover_utils::ensure_initial_setup_keys_present( - &prover_config.initial_setup_key_path, - &prover_config.key_download_url, - ); - METRICS.download_time.observe(started_at.elapsed()); - env::set_var("CRS_FILE", prover_config.initial_setup_key_path.clone()); - // We don't have a graceful shutdown process for the prover, so `_stop_sender` is unused. - // Though we still need to create a channel because circuit breaker expects `stop_receiver`. - let (_stop_sender, stop_receiver) = tokio::sync::watch::channel(false); - - let circuit_ids = - prover_group_config.get_circuit_ids_for_group_id(prover_config.specialized_prover_group_id); - - tracing::info!( - "Starting proof generation for circuits: {circuit_ids:?} \ - in region: {region} and zone: {zone} with group-id: {}", - prover_config.specialized_prover_group_id - ); - let mut tasks = vec![]; - - let exporter_config = PrometheusExporterConfig::pull(prometheus_config.listener_port); - tasks.push(tokio::spawn(exporter_config.run(stop_receiver.clone()))); - - let assembly_queue = Buffer::new(prover_config.assembly_queue_capacity); - let shared_assembly_queue = Arc::new(Mutex::new(assembly_queue)); - let producer = shared_assembly_queue.clone(); - let consumer = shared_assembly_queue.clone(); - let local_ip = local_ip().context("Failed obtaining local IP address")?; - let address = SocketAddress { - host: local_ip, - port: prover_config.assembly_receiver_port, - }; - tracing::info!("local IP address is: {:?}", local_ip); - - let postgres_config = PostgresConfig::from_env().context("PostgresConfig::from_env()")?; - tasks.push(tokio::task::spawn(incoming_socket_listener( - local_ip, - prover_config.assembly_receiver_port, - producer, - ConnectionPool::singleton(postgres_config.prover_url()?) - .build() - .await - .context("failed to build a connection pool")?, - prover_config.specialized_prover_group_id, - region.clone(), - zone.clone(), - num_gpu, - ))); - - let params = ProverParams::new(&prover_config); - let object_store_config = - ObjectStoreConfig::from_env().context("ObjectStoreConfig::from_env()")?; - let store_factory = ObjectStoreFactory::new(object_store_config); - - let circuit_provider_pool = ConnectionPool::singleton(postgres_config.prover_url()?) - .build() - .await - .context("failed to build circuit_provider_pool")?; - tasks.push(tokio::task::spawn_blocking(move || { - let rt_handle = tokio::runtime::Handle::current(); - let synthesized_circuit_provider = SynthesizedCircuitProvider::new( - consumer, - circuit_provider_pool, - address, - region, - zone, - rt_handle.clone(), - ); - let prover_job_reporter = - ProverReporter::new(postgres_config, prover_config, &store_factory, rt_handle) - .context("ProverReporter::new()")?; - prover_service::run_prover::run_prover_with_remote_synthesizer( - synthesized_circuit_provider, - ProverArtifactProvider, - prover_job_reporter, - circuit_ids, - params, - ); - Ok(()) - })); - - let particular_crypto_alerts = Some( - AlertsConfig::from_env() - .context("AlertsConfig::from_env()")? - .sporadic_crypto_errors_substrs, - ); - let graceful_shutdown = Some( - graceful_shutdown() - .await - .context("failed to prepare graceful shutdown future")?, - ); - let tasks_allowed_to_finish = false; - tokio::select! { - _ = wait_for_tasks(tasks, particular_crypto_alerts, graceful_shutdown, tasks_allowed_to_finish) => {}, - _ = stop_signal_receiver => { - tracing::info!("Stop signal received, shutting down"); - - // BEWARE, HERE BE DRAGONS. - // This is necessary because of blocking prover. See end of functions for more details. - std::process::exit(0); - }, - }; - - // BEWARE, HERE BE DRAGONS. - // The process hangs here if we panic outside `run_prover_with_remote_synthesizer`. - // Given the task is spawned as blocking, it's in a different thread that can't be cancelled on demand. - // See: https://docs.rs/tokio/latest/tokio/task/fn.spawn_blocking.html for more information - // Follow [PR](https://github.com/matter-labs/zksync-2-dev/pull/2129) for logic behind it - std::process::exit(-1); -} diff --git a/prover/prover/src/socket_listener.rs b/prover/prover/src/socket_listener.rs deleted file mode 100644 index 95a369e70786..000000000000 --- a/prover/prover/src/socket_listener.rs +++ /dev/null @@ -1,122 +0,0 @@ -use std::{ - net::{IpAddr, SocketAddr}, - time::Instant, -}; - -use anyhow::Context as _; -use queues::IsQueue; -use tokio::{ - io::copy, - net::{TcpListener, TcpStream}, -}; -use zksync_dal::ConnectionPool; -use zksync_types::proofs::{GpuProverInstanceStatus, SocketAddress}; - -use crate::synthesized_circuit_provider::SharedAssemblyQueue; - -#[allow(clippy::too_many_arguments)] -pub async fn incoming_socket_listener( - host: IpAddr, - port: u16, - queue: SharedAssemblyQueue, - pool: ConnectionPool, - specialized_prover_group_id: u8, - region: String, - zone: String, - num_gpu: u8, -) -> anyhow::Result<()> { - let listening_address = SocketAddr::new(host, port); - tracing::info!( - "Starting assembly receiver at host: {}, port: {}", - host, - port - ); - let listener = TcpListener::bind(listening_address) - .await - .with_context(|| format!("Failed binding address: {listening_address:?}"))?; - let address = SocketAddress { host, port }; - - let queue_capacity = queue.lock().await.capacity(); - pool.access_storage() - .await - .unwrap() - .gpu_prover_queue_dal() - .insert_prover_instance( - address.clone(), - queue_capacity, - specialized_prover_group_id, - region.clone(), - zone.clone(), - num_gpu, - ) - .await; - - let mut now = Instant::now(); - - loop { - let stream = listener - .accept() - .await - .context("could not accept connection")? - .0; - tracing::trace!( - "Received new assembly send connection, waited for {}ms.", - now.elapsed().as_millis() - ); - - handle_incoming_file( - stream, - queue.clone(), - pool.clone(), - address.clone(), - region.clone(), - zone.clone(), - ) - .await; - - now = Instant::now(); - } -} - -async fn handle_incoming_file( - mut stream: TcpStream, - queue: SharedAssemblyQueue, - pool: ConnectionPool, - address: SocketAddress, - region: String, - zone: String, -) { - let mut assembly: Vec = vec![]; - let started_at = Instant::now(); - copy(&mut stream, &mut assembly) - .await - .expect("Failed reading from stream"); - let file_size_in_gb = assembly.len() / (1024 * 1024 * 1024); - tracing::trace!( - "Read file of size: {}GB from stream took: {} seconds", - file_size_in_gb, - started_at.elapsed().as_secs() - ); - // acquiring lock from queue and updating db must be done atomically otherwise it results in TOCTTOU - // Time-of-Check to Time-of-Use - let mut assembly_queue = queue.lock().await; - let (queue_free_slots, status) = { - assembly_queue - .add(assembly) - .expect("Failed saving assembly to queue"); - let status = if assembly_queue.capacity() == assembly_queue.size() { - GpuProverInstanceStatus::Full - } else { - GpuProverInstanceStatus::Available - }; - let queue_free_slots = assembly_queue.capacity() - assembly_queue.size(); - (queue_free_slots, status) - }; - - pool.access_storage() - .await - .unwrap() - .gpu_prover_queue_dal() - .update_prover_instance_status(address, status, queue_free_slots, region, zone) - .await; -} diff --git a/prover/prover/src/synthesized_circuit_provider.rs b/prover/prover/src/synthesized_circuit_provider.rs deleted file mode 100644 index e1cec64162b0..000000000000 --- a/prover/prover/src/synthesized_circuit_provider.rs +++ /dev/null @@ -1,81 +0,0 @@ -use std::{ - io::{Cursor, Read}, - sync::Arc, -}; - -use prover_service::RemoteSynthesizer; -use queues::{Buffer, IsQueue}; -use tokio::{runtime::Handle, sync::Mutex}; -use zksync_dal::ConnectionPool; -use zksync_types::proofs::SocketAddress; - -use crate::metrics::METRICS; - -pub type SharedAssemblyQueue = Arc>>>; - -pub struct SynthesizedCircuitProvider { - rt_handle: Handle, - queue: SharedAssemblyQueue, - pool: ConnectionPool, - address: SocketAddress, - region: String, - zone: String, -} - -impl SynthesizedCircuitProvider { - pub fn new( - queue: SharedAssemblyQueue, - pool: ConnectionPool, - address: SocketAddress, - region: String, - zone: String, - rt_handle: Handle, - ) -> Self { - Self { - rt_handle, - queue, - pool, - address, - region, - zone, - } - } -} - -impl RemoteSynthesizer for SynthesizedCircuitProvider { - fn try_next(&mut self) -> Option> { - let mut assembly_queue = self.rt_handle.block_on(async { self.queue.lock().await }); - let is_full = assembly_queue.capacity() == assembly_queue.size(); - return match assembly_queue.remove() { - Ok(blob) => { - let queue_free_slots = assembly_queue.capacity() - assembly_queue.size(); - if is_full { - self.rt_handle.block_on(async { - self.pool - .access_storage() - .await - .unwrap() - .gpu_prover_queue_dal() - .update_prover_instance_from_full_to_available( - self.address.clone(), - queue_free_slots, - self.region.clone(), - self.zone.clone(), - ) - .await - }); - } - tracing::trace!( - "Queue free slot {} for capacity {}", - queue_free_slots, - assembly_queue.capacity() - ); - METRICS.queue_free_slots[&assembly_queue.capacity().to_string()] - .observe(queue_free_slots); - - Some(Box::new(Cursor::new(blob))) - } - Err(_) => None, - }; - } -} diff --git a/prover/prover_fri/src/main.rs b/prover/prover_fri/src/main.rs index a1f1ae0088d1..980a499f5b48 100644 --- a/prover/prover_fri/src/main.rs +++ b/prover/prover_fri/src/main.rs @@ -9,7 +9,7 @@ use tokio::{ task::JoinHandle, }; use zksync_config::configs::{ - fri_prover_group::FriProverGroupConfig, FriProverConfig, PostgresConfig, ProverGroupConfig, + fri_prover_group::FriProverGroupConfig, FriProverConfig, PostgresConfig, }; use zksync_dal::ConnectionPool; use zksync_env_config::{ @@ -39,9 +39,10 @@ async fn graceful_shutdown(port: u16) -> anyhow::Result .await .context("failed to build a connection pool")?; let host = local_ip().context("Failed obtaining local IP address")?; - let prover_group_config = - ProverGroupConfig::from_env().context("ProverGroupConfig::from_env()")?; - let zone = get_zone(&prover_group_config).await.context("get_zone()")?; + let zone_url = &FriProverConfig::from_env() + .context("FriProverConfig::from_env()")? + .zone_read_url; + let zone = get_zone(zone_url).await.context("get_zone()")?; let address = SocketAddress { host, port }; Ok(async move { pool.access_storage() @@ -221,9 +222,9 @@ async fn get_prover_tasks( let shared_witness_vector_queue = Arc::new(Mutex::new(witness_vector_queue)); let consumer = shared_witness_vector_queue.clone(); - let prover_group_config = - ProverGroupConfig::from_env().context("ProverGroupConfig::from_env()")?; - let zone = get_zone(&prover_group_config).await.context("get_zone()")?; + let zone = get_zone(&prover_config.zone_read_url) + .await + .context("get_zone()")?; let local_ip = local_ip().context("Failed obtaining local IP address")?; let address = SocketAddress { host: local_ip, diff --git a/prover/setup_key_generator_and_server/Cargo.toml b/prover/setup_key_generator_and_server/Cargo.toml deleted file mode 100644 index 22b1cef97bd6..000000000000 --- a/prover/setup_key_generator_and_server/Cargo.toml +++ /dev/null @@ -1,42 +0,0 @@ -[package] -name = "setup_key_generator_and_server" -version = "0.1.0" -edition = "2018" -authors = ["The Matter Labs Team "] -homepage = "https://zksync.io/" -repository = "https://github.com/matter-labs/zksync-era" -license = "MIT OR Apache-2.0" -keywords = ["blockchain", "zksync"] -categories = ["cryptography"] - -[lib] -name = "zksync_setup_key_server" -path = "src/lib.rs" - -[[bin]] -name = "zksync_setup_key_generator" -path = "src/main.rs" - -[dependencies] -zksync_types = { path = "../../core/lib/types" } -vlog = { path = "../../core/lib/vlog" } -zksync_config = { path = "../../core/lib/config" } -zksync_env_config = { path = "../../core/lib/env_config" } - -circuit_testing = { git = "https://github.com/matter-labs/era-circuit_testing.git", branch = "main" } -api = { git = "https://github.com/matter-labs/era-heavy-ops-service.git", branch = "v1.3.3", features = [ - "gpu", -], optional = true, default-features = false } -prover-service = { git = "https://github.com/matter-labs/era-heavy-ops-service.git", branch = "v1.3.3", features = [ - "gpu", -], optional = true, default-features = false } -zkevm_test_harness = { git = "https://github.com/matter-labs/era-zkevm_test_harness.git", branch = "v1.3.3" } - -anyhow = "1.0" -tracing = "0.1" -structopt = "0.3.26" -itertools = "0.10.5" - -[features] -default = [] -gpu = ["api", "prover-service"] diff --git a/prover/setup_key_generator_and_server/data/.gitkeep b/prover/setup_key_generator_and_server/data/.gitkeep deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/prover/setup_key_generator_and_server/src/lib.rs b/prover/setup_key_generator_and_server/src/lib.rs deleted file mode 100644 index 34b4896cefe1..000000000000 --- a/prover/setup_key_generator_and_server/src/lib.rs +++ /dev/null @@ -1,63 +0,0 @@ -use std::{fs::File, io::Read, path::Path}; - -use anyhow::Context as _; -use zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncCircuit, - bellman::bn256::Bn256, - witness::{ - oracle::VmWitnessOracle, recursive_aggregation::padding_aggregations, - vk_set_generator::circuits_for_vk_generation, - }, -}; -use zksync_config::ProverConfigs; -use zksync_env_config::FromEnv; -use zksync_types::circuit::{ - GEOMETRY_CONFIG, LEAF_SPLITTING_FACTOR, NODE_SPLITTING_FACTOR, SCHEDULER_UPPER_BOUND, -}; - -pub fn get_setup_for_circuit_type(circuit_type: u8) -> anyhow::Result> { - let filepath = get_setup_key_file_path(circuit_type).context("get_setup_key_file_path()")?; - tracing::info!("Fetching setup key from path: {}", filepath); - let file = File::open(filepath.clone()) - .with_context(|| format!("Failed reading setup key from path: {filepath}"))?; - Ok(Box::new(file)) -} - -pub fn get_circuits_for_vk() -> anyhow::Result>>> { - ensure_setup_key_exist().context("ensure_setup_key_exists()")?; - let padding_aggregations = padding_aggregations(NODE_SPLITTING_FACTOR); - Ok(circuits_for_vk_generation( - GEOMETRY_CONFIG, - LEAF_SPLITTING_FACTOR, - NODE_SPLITTING_FACTOR, - SCHEDULER_UPPER_BOUND, - padding_aggregations, - )) -} - -fn ensure_setup_key_exist() -> anyhow::Result<()> { - if !Path::new("setup_2^26.key").exists() { - anyhow::bail!("File setup_2^26.key is required to be present in current directory."); - } - Ok(()) -} - -pub fn get_setup_key_write_file_path(circuit_type: u8) -> String { - let zksync_home = std::env::var("ZKSYNC_HOME").unwrap_or_else(|_| "/".into()); - format!("{}/{}", zksync_home, get_setup_key_filename(circuit_type)) -} - -fn get_setup_key_file_path(circuit_type: u8) -> anyhow::Result { - let prover_config = ProverConfigs::from_env() - .context("ProverConfigs::from_env()")? - .non_gpu; - Ok(format!( - "{}/{}", - prover_config.setup_keys_path, - get_setup_key_filename(circuit_type) - )) -} - -fn get_setup_key_filename(circuit_type: u8) -> String { - format!("setup_{}_key.bin", circuit_type) -} diff --git a/prover/setup_key_generator_and_server/src/main.rs b/prover/setup_key_generator_and_server/src/main.rs deleted file mode 100644 index 9eee0aa5c09e..000000000000 --- a/prover/setup_key_generator_and_server/src/main.rs +++ /dev/null @@ -1,68 +0,0 @@ -#![cfg_attr(not(feature = "gpu"), allow(unused_imports))] - -use std::{env, fs::File}; - -use anyhow::Context as _; -use structopt::StructOpt; -use zkevm_test_harness::{ - abstract_zksync_circuit::concrete_circuits::ZkSyncCircuit, bellman::bn256::Bn256, - witness::oracle::VmWitnessOracle, -}; -use zksync_setup_key_server::{get_circuits_for_vk, get_setup_key_write_file_path}; - -#[cfg(feature = "gpu")] -#[derive(Debug, StructOpt)] -#[structopt( - name = "Generate setup keys for individual circuit", - about = "Tool for generating setup key for individual circuit" -)] -struct Opt { - /// Numeric circuit type valid value from [0-17]. - #[structopt(long)] - numeric_circuit: u8, -} - -#[cfg(not(feature = "gpu"))] -fn main() { - unimplemented!("This binary is only available with `gpu` feature enabled"); -} - -#[cfg(feature = "gpu")] -fn main() -> anyhow::Result<()> { - let opt = Opt::from_args(); - env::set_var("CRS_FILE", "setup_2^26.key"); - tracing::info!("Starting setup key generation!"); - get_circuits_for_vk() - .context("get_circuits_for_vk()")? - .into_iter() - .filter(|c| c.numeric_circuit_type() == opt.numeric_circuit) - .for_each(generate_setup_key_for_circuit); - Ok(()) -} - -#[cfg(feature = "gpu")] -fn generate_setup_key_for_circuit(circuit: ZkSyncCircuit>) { - use prover_service::utils::generate_setup_for_circuit; - - let mut prover = api::Prover::new(); - let setup = generate_setup_for_circuit(&mut prover, &circuit); - save_setup_for_circuit_type(circuit.numeric_circuit_type(), setup); - tracing::info!( - "Finished setup key generation for circuit {:?} (id {:?})", - circuit.short_description(), - circuit.numeric_circuit_type() - ); -} - -#[cfg(feature = "gpu")] -fn save_setup_for_circuit_type(circuit_type: u8, setup: prover_service::Setup) { - let filepath = get_setup_key_write_file_path(circuit_type); - tracing::info!("saving setup key to: {}", filepath); - let setup_file = File::create(&filepath).unwrap(); - setup - .write(setup_file) - .expect("Failed saving setup key to file."); - let setup_file = File::open(filepath).expect("Unable to open file"); - let size = setup_file.metadata().unwrap().len() as f64 / (1024.0 * 1024.0); - println!("Saved file size: {:?}MB", size); -} diff --git a/prover/witness_vector_generator/src/main.rs b/prover/witness_vector_generator/src/main.rs index 79faa17c39cc..552644782c0c 100644 --- a/prover/witness_vector_generator/src/main.rs +++ b/prover/witness_vector_generator/src/main.rs @@ -6,7 +6,7 @@ use structopt::StructOpt; use tokio::sync::{oneshot, watch}; use zksync_config::configs::{ fri_prover_group::FriProverGroupConfig, FriProverConfig, FriWitnessVectorGeneratorConfig, - PostgresConfig, ProverGroupConfig, + PostgresConfig, }; use zksync_dal::ConnectionPool; use zksync_env_config::{object_store::ProverObjectStoreConfig, FromEnv}; @@ -73,11 +73,10 @@ async fn main() -> anyhow::Result<()> { .unwrap_or_default(); let circuit_ids_for_round_to_be_proven = get_all_circuit_id_round_tuples_for(circuit_ids_for_round_to_be_proven); - let prover_group_config = - ProverGroupConfig::from_env().context("ProverGroupConfig::from_env()")?; - let zone = get_zone(&prover_group_config).await.context("get_zone()")?; - let vk_commitments = get_cached_commitments(); let fri_prover_config = FriProverConfig::from_env().context("FriProverConfig::from_env()")?; + let zone_url = &fri_prover_config.zone_read_url; + let zone = get_zone(zone_url).await.context("get_zone()")?; + let vk_commitments = get_cached_commitments(); let witness_vector_generator = WitnessVectorGenerator::new( blob_store, pool, From 76948d75f3d42dac88b380514dcf3848884e121a Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Wed, 3 Jan 2024 16:10:50 +0200 Subject: [PATCH 04/49] refactor(eth-sender): Make `EthInterface` object-safe (#807) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Makes `EthInterface` trait object-safe. - Extends its mock implementation (`MockEthereum`) so that it returns real values for the `get_tx()` method. ## Why ❔ Required to use `EthInterface` in consistency checker. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- Cargo.lock | 3 +- core/lib/dal/src/eth_sender_dal.rs | 2 +- core/lib/eth_client/Cargo.toml | 9 +- core/lib/eth_client/src/clients/generic.rs | 170 +++++++ core/lib/eth_client/src/clients/http/query.rs | 46 +- .../eth_client/src/clients/http/signing.rs | 61 +-- core/lib/eth_client/src/clients/mock.rs | 481 +++++++----------- core/lib/eth_client/src/clients/mod.rs | 12 +- core/lib/eth_client/src/lib.rs | 65 +-- core/lib/eth_client/src/types.rs | 114 ++++- core/lib/zksync_core/src/eth_sender/error.rs | 3 +- .../src/eth_sender/eth_tx_aggregator.rs | 67 +-- .../src/eth_sender/eth_tx_manager.rs | 11 +- core/lib/zksync_core/src/eth_sender/tests.rs | 62 ++- core/lib/zksync_core/src/eth_watch/client.rs | 45 +- .../src/l1_gas_price/gas_adjuster/mod.rs | 2 +- .../src/l1_gas_price/gas_adjuster/tests.rs | 2 +- .../zksync_core/src/l1_gas_price/singleton.rs | 2 +- core/lib/zksync_core/src/lib.rs | 23 +- .../src/state_keeper/io/tests/tester.rs | 2 +- sdk/zksync-rs/src/ethereum/mod.rs | 52 +- 21 files changed, 649 insertions(+), 585 deletions(-) create mode 100644 core/lib/eth_client/src/clients/generic.rs diff --git a/Cargo.lock b/Cargo.lock index d6f5176bf778..9322e0e2a7b4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8575,11 +8575,10 @@ dependencies = [ name = "zksync_eth_client" version = "0.1.0" dependencies = [ - "anyhow", "async-trait", - "hex", "jsonrpc-core", "serde", + "static_assertions", "thiserror", "tokio", "tracing", diff --git a/core/lib/dal/src/eth_sender_dal.rs b/core/lib/dal/src/eth_sender_dal.rs index 797928740602..a524ae5694c7 100644 --- a/core/lib/dal/src/eth_sender_dal.rs +++ b/core/lib/dal/src/eth_sender_dal.rs @@ -211,7 +211,7 @@ impl EthSenderDal<'_, '_> { base_fee_per_gas: u64, priority_fee_per_gas: u64, tx_hash: H256, - raw_signed_tx: Vec, + raw_signed_tx: &[u8], ) -> anyhow::Result> { let priority_fee_per_gas = i64::try_from(priority_fee_per_gas).context("Can't convert u64 to i64")?; diff --git a/core/lib/eth_client/Cargo.toml b/core/lib/eth_client/Cargo.toml index 2eb37a0c9b24..ff3e56ef7311 100644 --- a/core/lib/eth_client/Cargo.toml +++ b/core/lib/eth_client/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "zksync_eth_client" version = "0.1.0" -edition = "2018" +edition = "2021" authors = ["The Matter Labs Team "] homepage = "https://zksync.io/" repository = "https://github.com/matter-labs/zksync-era" @@ -18,9 +18,10 @@ zksync_contracts = { path = "../contracts" } jsonrpc-core = "18" serde = "1.0.90" -hex = "0.4" -anyhow = "1.0" thiserror = "1" -tokio = { version = "1", features = ["full"] } async-trait = "0.1" tracing = "0.1" + +[dev-dependencies] +static_assertions = "1.1.0" +tokio = { version = "1", features = ["full"] } diff --git a/core/lib/eth_client/src/clients/generic.rs b/core/lib/eth_client/src/clients/generic.rs new file mode 100644 index 000000000000..fa9950d524a9 --- /dev/null +++ b/core/lib/eth_client/src/clients/generic.rs @@ -0,0 +1,170 @@ +use std::sync::Arc; + +use async_trait::async_trait; +use zksync_types::{ + web3::{ + contract::Options, + ethabi, + types::{ + Address, Block, BlockId, BlockNumber, Filter, Log, Transaction, TransactionReceipt, + H160, H256, U256, U64, + }, + }, + L1ChainId, +}; + +use crate::{ + BoundEthInterface, ContractCall, Error, EthInterface, ExecutedTxStatus, FailureInfo, + RawTransactionBytes, SignedCallResult, +}; + +#[async_trait] +impl EthInterface for Arc { + async fn nonce_at_for_account( + &self, + account: Address, + block: BlockNumber, + component: &'static str, + ) -> Result { + self.as_ref() + .nonce_at_for_account(account, block, component) + .await + } + + async fn base_fee_history( + &self, + from_block: usize, + block_count: usize, + component: &'static str, + ) -> Result, Error> { + self.as_ref() + .base_fee_history(from_block, block_count, component) + .await + } + + async fn get_pending_block_base_fee_per_gas( + &self, + component: &'static str, + ) -> Result { + self.as_ref() + .get_pending_block_base_fee_per_gas(component) + .await + } + + async fn get_gas_price(&self, component: &'static str) -> Result { + self.as_ref().get_gas_price(component).await + } + + async fn block_number(&self, component: &'static str) -> Result { + self.as_ref().block_number(component).await + } + + async fn send_raw_tx(&self, tx: RawTransactionBytes) -> Result { + self.as_ref().send_raw_tx(tx).await + } + + async fn get_tx_status( + &self, + hash: H256, + component: &'static str, + ) -> Result, Error> { + self.as_ref().get_tx_status(hash, component).await + } + + async fn failure_reason(&self, tx_hash: H256) -> Result, Error> { + self.as_ref().failure_reason(tx_hash).await + } + + async fn get_tx( + &self, + hash: H256, + component: &'static str, + ) -> Result, Error> { + self.as_ref().get_tx(hash, component).await + } + + async fn tx_receipt( + &self, + tx_hash: H256, + component: &'static str, + ) -> Result, Error> { + self.as_ref().tx_receipt(tx_hash, component).await + } + + async fn eth_balance(&self, address: Address, component: &'static str) -> Result { + self.as_ref().eth_balance(address, component).await + } + + async fn call_contract_function( + &self, + call: ContractCall, + ) -> Result, Error> { + self.as_ref().call_contract_function(call).await + } + + async fn logs(&self, filter: Filter, component: &'static str) -> Result, Error> { + self.as_ref().logs(filter, component).await + } + + async fn block( + &self, + block_id: BlockId, + component: &'static str, + ) -> Result>, Error> { + self.as_ref().block(block_id, component).await + } +} + +#[async_trait::async_trait] +impl BoundEthInterface for Arc { + fn contract(&self) -> ðabi::Contract { + self.as_ref().contract() + } + + fn contract_addr(&self) -> H160 { + self.as_ref().contract_addr() + } + + fn chain_id(&self) -> L1ChainId { + self.as_ref().chain_id() + } + + fn sender_account(&self) -> Address { + self.as_ref().sender_account() + } + + async fn allowance_on_account( + &self, + token_address: Address, + contract_address: Address, + erc20_abi: ethabi::Contract, + ) -> Result { + self.as_ref() + .allowance_on_account(token_address, contract_address, erc20_abi) + .await + } + + async fn sign_prepared_tx_for_addr( + &self, + data: Vec, + contract_addr: H160, + options: Options, + component: &'static str, + ) -> Result { + self.as_ref() + .sign_prepared_tx_for_addr(data, contract_addr, options, component) + .await + } + + async fn nonce_at(&self, block: BlockNumber, component: &'static str) -> Result { + self.as_ref().nonce_at(block, component).await + } + + async fn current_nonce(&self, _: &'static str) -> Result { + self.as_ref().current_nonce("").await + } + + async fn pending_nonce(&self, _: &'static str) -> Result { + self.as_ref().pending_nonce("").await + } +} diff --git a/core/lib/eth_client/src/clients/http/query.rs b/core/lib/eth_client/src/clients/http/query.rs index 3e88944ca0e4..de8a51a731eb 100644 --- a/core/lib/eth_client/src/clients/http/query.rs +++ b/core/lib/eth_client/src/clients/http/query.rs @@ -3,10 +3,7 @@ use std::sync::Arc; use async_trait::async_trait; use zksync_types::web3::{ self, - contract::{ - tokens::{Detokenize, Tokenize}, - Contract, Options, - }, + contract::Contract, ethabi, transports::Http, types::{ @@ -18,8 +15,8 @@ use zksync_types::web3::{ use crate::{ clients::http::{Method, COUNTERS, LATENCIES}, - types::{Error, ExecutedTxStatus, FailureInfo}, - EthInterface, + types::{Error, ExecutedTxStatus, FailureInfo, RawTokens}, + ContractCall, EthInterface, RawTransactionBytes, }; /// An "anonymous" Ethereum client that can invoke read-only methods that aren't @@ -40,7 +37,7 @@ impl From for QueryClient { impl QueryClient { /// Creates a new HTTP client. pub fn new(node_url: &str) -> Result { - let transport = web3::transports::Http::new(node_url)?; + let transport = Http::new(node_url)?; Ok(transport.into()) } } @@ -80,9 +77,9 @@ impl EthInterface for QueryClient { Ok(network_gas_price) } - async fn send_raw_tx(&self, tx: Vec) -> Result { + async fn send_raw_tx(&self, tx: RawTransactionBytes) -> Result { let latency = LATENCIES.direct[&Method::SendRawTx].start(); - let tx = self.web3.eth().send_raw_transaction(Bytes(tx)).await?; + let tx = self.web3.eth().send_raw_transaction(Bytes(tx.0)).await?; latency.observe(); Ok(tx) } @@ -233,26 +230,21 @@ impl EthInterface for QueryClient { Ok(tx) } - #[allow(clippy::too_many_arguments)] - async fn call_contract_function( + async fn call_contract_function( &self, - func: &str, - params: P, - from: A, - options: Options, - block: B, - contract_address: Address, - contract_abi: ethabi::Contract, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - B: Into> + Send, - P: Tokenize + Send, - { + call: ContractCall, + ) -> Result, Error> { let latency = LATENCIES.direct[&Method::CallContractFunction].start(); - let contract = Contract::new(self.web3.eth(), contract_address, contract_abi); - let res = contract.query(func, params, from, options, block).await?; + let contract = Contract::new(self.web3.eth(), call.contract_address, call.contract_abi); + let RawTokens(res) = contract + .query( + &call.inner.name, + call.inner.params, + call.inner.from, + call.inner.options, + call.inner.block, + ) + .await?; latency.observe(); Ok(res) } diff --git a/core/lib/eth_client/src/clients/http/signing.rs b/core/lib/eth_client/src/clients/http/signing.rs index 8b56dc1cfbdf..269d773a2588 100644 --- a/core/lib/eth_client/src/clients/http/signing.rs +++ b/core/lib/eth_client/src/clients/http/signing.rs @@ -7,10 +7,7 @@ use zksync_eth_signer::{raw_ethereum_tx::TransactionParameters, EthereumSigner, use zksync_types::{ web3::{ self, - contract::{ - tokens::{Detokenize, Tokenize}, - Options, - }, + contract::{tokens::Detokenize, Options}, ethabi, transports::Http, types::{ @@ -24,7 +21,7 @@ use zksync_types::{ use super::{query::QueryClient, Method, LATENCIES}; use crate::{ types::{Error, ExecutedTxStatus, FailureInfo, SignedCallResult}, - BoundEthInterface, EthInterface, + BoundEthInterface, CallFunctionArgs, ContractCall, EthInterface, RawTransactionBytes, }; /// HTTP-based Ethereum client, backed by a private key to sign transactions. @@ -47,8 +44,7 @@ impl PKSigningClient { let default_priority_fee_per_gas = eth_sender.gas_adjuster.default_priority_fee_per_gas; let l1_chain_id = eth_client.chain_id; - let transport = - web3::transports::Http::new(main_node_url).expect("Failed to create transport"); + let transport = Http::new(main_node_url).expect("Failed to create transport"); let operator_address = PackedEthSignature::address_from_private_key(&operator_private_key) .expect("Failed to get address from private key"); @@ -122,7 +118,7 @@ impl EthInterface for SigningClient { self.query_client.get_gas_price(component).await } - async fn send_raw_tx(&self, tx: Vec) -> Result { + async fn send_raw_tx(&self, tx: RawTransactionBytes) -> Result { self.query_client.send_raw_tx(tx).await } @@ -166,34 +162,11 @@ impl EthInterface for SigningClient { self.query_client.get_tx(hash, component).await } - #[allow(clippy::too_many_arguments)] - async fn call_contract_function( + async fn call_contract_function( &self, - func: &str, - params: P, - from: A, - options: Options, - block: B, - contract_address: Address, - contract_abi: ethabi::Contract, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - B: Into> + Send, - P: Tokenize + Send, - { - self.query_client - .call_contract_function( - func, - params, - from, - options, - block, - contract_address, - contract_abi, - ) - .await + call: ContractCall, + ) -> Result, Error> { + self.query_client.call_contract_function(call).await } async fn tx_receipt( @@ -303,7 +276,7 @@ impl BoundEthInterface for SigningClient { let hash = web3::signing::keccak256(&signed_tx).into(); latency.observe(); Ok(SignedCallResult { - raw_tx: signed_tx, + raw_tx: RawTransactionBytes(signed_tx), max_priority_fee_per_gas, max_fee_per_gas, nonce, @@ -318,19 +291,11 @@ impl BoundEthInterface for SigningClient { erc20_abi: ethabi::Contract, ) -> Result { let latency = LATENCIES.direct[&Method::Allowance].start(); - let res = self - .call_contract_function( - "allowance", - (self.inner.sender_account, address), - None, - Options::default(), - None, - token_address, - erc20_abi, - ) - .await?; + let args = CallFunctionArgs::new("allowance", (self.inner.sender_account, address)) + .for_contract(token_address, erc20_abi); + let res = self.call_contract_function(args).await?; latency.observe(); - Ok(res) + Ok(U256::from_tokens(res)?) } } diff --git a/core/lib/eth_client/src/clients/mock.rs b/core/lib/eth_client/src/clients/mock.rs index 0528066eefd2..ae223bb47101 100644 --- a/core/lib/eth_client/src/clients/mock.rs +++ b/core/lib/eth_client/src/clients/mock.rs @@ -1,21 +1,14 @@ use std::{ collections::{BTreeMap, HashMap}, - fmt, - sync::{ - atomic::{AtomicU64, Ordering}, - RwLock, - }, + sync::RwLock, }; use async_trait::async_trait; use jsonrpc_core::types::error::Error as RpcError; use zksync_types::{ web3::{ - contract::{ - tokens::{Detokenize, Tokenize}, - Options, - }, - ethabi::{self, Token}, + contract::{tokens::Tokenize, Options}, + ethabi, types::{Block, BlockId, BlockNumber, Filter, Log, Transaction, TransactionReceipt, U64}, Error as Web3Error, }, @@ -24,55 +17,112 @@ use zksync_types::{ use crate::{ types::{Error, ExecutedTxStatus, FailureInfo, SignedCallResult}, - BoundEthInterface, EthInterface, + BoundEthInterface, ContractCall, EthInterface, RawTransactionBytes, }; -#[derive(Debug, Clone, Default, Copy)] -pub struct MockTx { - pub hash: H256, - pub nonce: u64, - pub base_fee: U256, +#[derive(Debug, Clone)] +struct MockTx { + input: Vec, + hash: H256, + nonce: u64, + max_fee_per_gas: U256, + max_priority_fee_per_gas: U256, } impl From> for MockTx { fn from(tx: Vec) -> Self { - use std::convert::TryFrom; - let len = tx.len(); - let total_gas_price = U256::try_from(&tx[len - 96..len - 64]).unwrap(); - let priority_fee = U256::try_from(&tx[len - 64..len - 32]).unwrap(); - let base_fee = total_gas_price - priority_fee; + let max_fee_per_gas = U256::try_from(&tx[len - 96..len - 64]).unwrap(); + let max_priority_fee_per_gas = U256::try_from(&tx[len - 64..len - 32]).unwrap(); let nonce = U256::try_from(&tx[len - 32..]).unwrap().as_u64(); let hash = { - let mut buffer: [u8; 32] = Default::default(); + let mut buffer = [0_u8; 32]; buffer.copy_from_slice(&tx[..32]); buffer.into() }; Self { + input: tx[32..len - 96].to_vec(), nonce, hash, - base_fee, + max_fee_per_gas, + max_priority_fee_per_gas, + } + } +} + +impl From for Transaction { + fn from(tx: MockTx) -> Self { + Self { + input: tx.input.into(), + hash: tx.hash, + nonce: tx.nonce.into(), + max_fee_per_gas: Some(tx.max_fee_per_gas), + max_priority_fee_per_gas: Some(tx.max_priority_fee_per_gas), + ..Self::default() } } } +/// Mutable part of [`MockEthereum`] that needs to be synchronized via an `RwLock`. +#[derive(Debug, Default)] +struct MockEthereumInner { + block_number: u64, + tx_statuses: HashMap, + sent_txs: HashMap, + current_nonce: u64, + pending_nonce: u64, + nonces: BTreeMap, +} + +impl MockEthereumInner { + fn execute_tx( + &mut self, + tx_hash: H256, + success: bool, + confirmations: u64, + non_ordering_confirmations: bool, + ) { + let block_number = self.block_number; + self.block_number += confirmations; + let nonce = self.current_nonce; + self.current_nonce += 1; + let tx_nonce = self.sent_txs[&tx_hash].nonce; + + if non_ordering_confirmations { + if tx_nonce >= nonce { + self.current_nonce = tx_nonce; + } + } else { + assert_eq!(tx_nonce, nonce, "nonce mismatch"); + } + self.nonces.insert(block_number, nonce + 1); + + let status = ExecutedTxStatus { + tx_hash, + success, + receipt: TransactionReceipt { + gas_used: Some(21000u32.into()), + block_number: Some(block_number.into()), + transaction_hash: tx_hash, + ..TransactionReceipt::default() + }, + }; + self.tx_statuses.insert(tx_hash, status); + } +} + /// Mock Ethereum client is capable of recording all the incoming requests for the further analysis. #[derive(Debug)] pub struct MockEthereum { - pub block_number: AtomicU64, - pub max_fee_per_gas: U256, - pub base_fee_history: RwLock>, - pub max_priority_fee_per_gas: U256, - pub tx_statuses: RwLock>, - pub sent_txs: RwLock>, - pub current_nonce: AtomicU64, - pub pending_nonce: AtomicU64, - pub nonces: RwLock>, + max_fee_per_gas: U256, + max_priority_fee_per_gas: U256, + base_fee_history: Vec, /// If true, the mock will not check the ordering nonces of the transactions. /// This is useful for testing the cases when the transactions are executed out of order. - pub non_ordering_confirmations: bool, - pub multicall_address: Address, + non_ordering_confirmations: bool, + multicall_address: Address, + inner: RwLock, } impl Default for MockEthereum { @@ -80,15 +130,10 @@ impl Default for MockEthereum { Self { max_fee_per_gas: 100.into(), max_priority_fee_per_gas: 10.into(), - block_number: Default::default(), - base_fee_history: Default::default(), - tx_statuses: Default::default(), - sent_txs: Default::default(), - current_nonce: Default::default(), - pending_nonce: Default::default(), - nonces: RwLock::new([(0, 0)].into()), + base_fee_history: vec![], non_ordering_confirmations: false, multicall_address: Address::default(), + inner: RwLock::default(), } } } @@ -96,53 +141,29 @@ impl Default for MockEthereum { impl MockEthereum { /// A fake `sha256` hasher, which calculates an `std::hash` instead. /// This is done for simplicity and it's also much faster. - pub fn fake_sha256(data: &[u8]) -> H256 { + fn fake_sha256(data: &[u8]) -> H256 { use std::{collections::hash_map::DefaultHasher, hash::Hasher}; let mut hasher = DefaultHasher::new(); hasher.write(data); - let result = hasher.finish(); - H256::from_low_u64_ne(result) } + /// Returns the number of transactions sent via this client. + pub fn sent_tx_count(&self) -> usize { + self.inner.read().unwrap().sent_txs.len() + } + /// Increments the blocks by a provided `confirmations` and marks the sent transaction /// as a success. - pub fn execute_tx( - &self, - tx_hash: H256, - success: bool, - confirmations: u64, - ) -> anyhow::Result<()> { - let block_number = self.block_number.fetch_add(confirmations, Ordering::SeqCst); - let nonce = self.current_nonce.fetch_add(1, Ordering::SeqCst); - let tx_nonce = self.sent_txs.read().unwrap()[&tx_hash].nonce; - - if self.non_ordering_confirmations { - if tx_nonce >= nonce { - self.current_nonce.store(tx_nonce, Ordering::SeqCst); - } - } else { - anyhow::ensure!(tx_nonce == nonce, "nonce mismatch"); - } - - self.nonces.write().unwrap().insert(block_number, nonce + 1); - - let status = ExecutedTxStatus { + pub fn execute_tx(&self, tx_hash: H256, success: bool, confirmations: u64) { + self.inner.write().unwrap().execute_tx( tx_hash, success, - receipt: TransactionReceipt { - gas_used: Some(21000u32.into()), - block_number: Some(block_number.into()), - transaction_hash: tx_hash, - ..Default::default() - }, - }; - - self.tx_statuses.write().unwrap().insert(tx_hash, status); - - Ok(()) + confirmations, + self.non_ordering_confirmations, + ); } pub fn sign_prepared_tx( @@ -167,7 +188,7 @@ impl MockEthereum { let mut new_raw_tx = hash.as_bytes().to_vec(); new_raw_tx.extend(raw_tx); Ok(SignedCallResult { - raw_tx: new_raw_tx, + raw_tx: RawTransactionBytes(new_raw_tx), max_priority_fee_per_gas, max_fee_per_gas, nonce, @@ -176,12 +197,14 @@ impl MockEthereum { } pub fn advance_block_number(&self, val: u64) -> u64 { - self.block_number.fetch_add(val, Ordering::SeqCst) + val + let mut inner = self.inner.write().unwrap(); + inner.block_number += val; + inner.block_number } pub fn with_fee_history(self, history: Vec) -> Self { Self { - base_fee_history: RwLock::new(history), + base_fee_history: history, ..self } } @@ -208,17 +231,19 @@ impl EthInterface for MockEthereum { hash: H256, _: &'static str, ) -> Result, Error> { - Ok(self.tx_statuses.read().unwrap().get(&hash).cloned()) + Ok(self.inner.read().unwrap().tx_statuses.get(&hash).cloned()) } async fn block_number(&self, _: &'static str) -> Result { - Ok(self.block_number.load(Ordering::SeqCst).into()) + Ok(self.inner.read().unwrap().block_number.into()) } - async fn send_raw_tx(&self, tx: Vec) -> Result { - let mock_tx = MockTx::from(tx); + async fn send_raw_tx(&self, tx: RawTransactionBytes) -> Result { + let mock_tx = MockTx::from(tx.0); + let mock_tx_hash = mock_tx.hash; + let mut inner = self.inner.write().unwrap(); - if mock_tx.nonce < self.current_nonce.load(Ordering::SeqCst) { + if mock_tx.nonce < inner.current_nonce { return Err(Error::EthereumGateway(Web3Error::Rpc(RpcError { message: "transaction with the same nonce already processed".to_string(), code: 101.into(), @@ -226,13 +251,11 @@ impl EthInterface for MockEthereum { }))); } - if mock_tx.nonce == self.pending_nonce.load(Ordering::SeqCst) { - self.pending_nonce.fetch_add(1, Ordering::SeqCst); + if mock_tx.nonce == inner.pending_nonce { + inner.pending_nonce += 1; } - - self.sent_txs.write().unwrap().insert(mock_tx.hash, mock_tx); - - Ok(mock_tx.hash) + inner.sent_txs.insert(mock_tx_hash, mock_tx); + Ok(mock_tx_hash) } async fn nonce_at_for_account( @@ -254,18 +277,15 @@ impl EthInterface for MockEthereum { block_count: usize, _component: &'static str, ) -> Result, Error> { - Ok(self.base_fee_history.read().unwrap() - [from_block.saturating_sub(block_count - 1)..=from_block] - .to_vec()) + let start_block = from_block.saturating_sub(block_count - 1); + Ok(self.base_fee_history[start_block..=from_block].to_vec()) } async fn get_pending_block_base_fee_per_gas( &self, _component: &'static str, ) -> Result { - Ok(U256::from( - *self.base_fee_history.read().unwrap().last().unwrap(), - )) + Ok(U256::from(*self.base_fee_history.last().unwrap())) } async fn failure_reason(&self, tx_hash: H256) -> Result, Error> { @@ -279,24 +299,13 @@ impl EthInterface for MockEthereum { })) } - #[allow(clippy::too_many_arguments)] - async fn call_contract_function( + async fn call_contract_function( &self, - _func: &str, - _params: P, - _from: A, - _options: Options, - _block: B, - contract_address: Address, - _contract_abi: ethabi::Contract, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - B: Into> + Send, - P: Tokenize + Send, - { - if contract_address == self.multicall_address { + call: ContractCall, + ) -> Result, Error> { + use ethabi::Token; + + if call.contract_address == self.multicall_address { let token = Token::Array(vec![ Token::Tuple(vec![Token::Bool(true), Token::Bytes(vec![1u8; 32])]), Token::Tuple(vec![Token::Bool(true), Token::Bytes(vec![2u8; 32])]), @@ -311,17 +320,21 @@ impl EthInterface for MockEthereum { ), ]), ]); - return Ok(R::from_tokens(vec![token]).unwrap()); + return Ok(vec![token]); } - Ok(R::from_tokens(vec![]).unwrap()) + Ok(vec![]) } async fn get_tx( &self, - _hash: H256, + hash: H256, _component: &'static str, ) -> Result, Error> { - unimplemented!("Not needed right now") + let txs = &self.inner.read().unwrap().sent_txs; + let Some(tx) = txs.get(&hash) else { + return Ok(None); + }; + Ok(Some(tx.clone().into())) } async fn tx_receipt( @@ -392,199 +405,79 @@ impl BoundEthInterface for MockEthereum { async fn nonce_at(&self, block: BlockNumber, _component: &'static str) -> Result { if let BlockNumber::Number(block_number) = block { - Ok((*self - .nonces - .read() - .unwrap() - .range(..=block_number.as_u64()) - .next_back() - .unwrap() - .1) - .into()) + let inner = self.inner.read().unwrap(); + let mut nonce_range = inner.nonces.range(..=block_number.as_u64()); + let (_, &nonce) = nonce_range.next_back().unwrap_or((&0, &0)); + Ok(nonce.into()) } else { panic!("MockEthereum::nonce_at called with non-number block tag"); } } async fn pending_nonce(&self, _: &'static str) -> Result { - Ok(self.pending_nonce.load(Ordering::SeqCst).into()) + Ok(self.inner.read().unwrap().pending_nonce.into()) } async fn current_nonce(&self, _: &'static str) -> Result { - Ok(self.current_nonce.load(Ordering::SeqCst).into()) + Ok(self.inner.read().unwrap().current_nonce.into()) } } -#[async_trait] -impl + Send + Sync + fmt::Debug> EthInterface for T { - async fn nonce_at_for_account( - &self, - account: Address, - block: BlockNumber, - component: &'static str, - ) -> Result { - self.as_ref() - .nonce_at_for_account(account, block, component) - .await - } - - async fn base_fee_history( - &self, - from_block: usize, - block_count: usize, - component: &'static str, - ) -> Result, Error> { - self.as_ref() - .base_fee_history(from_block, block_count, component) - .await - } - - async fn get_pending_block_base_fee_per_gas( - &self, - component: &'static str, - ) -> Result { - self.as_ref() - .get_pending_block_base_fee_per_gas(component) - .await - } - - async fn get_gas_price(&self, component: &'static str) -> Result { - self.as_ref().get_gas_price(component).await - } - - async fn block_number(&self, component: &'static str) -> Result { - self.as_ref().block_number(component).await - } - - async fn send_raw_tx(&self, tx: Vec) -> Result { - self.as_ref().send_raw_tx(tx).await - } - - async fn failure_reason(&self, tx_hash: H256) -> Result, Error> { - self.as_ref().failure_reason(tx_hash).await - } - - async fn get_tx_status( - &self, - hash: H256, - component: &'static str, - ) -> Result, Error> { - self.as_ref().get_tx_status(hash, component).await - } - - async fn get_tx( - &self, - hash: H256, - component: &'static str, - ) -> Result, Error> { - self.as_ref().get_tx(hash, component).await - } - - #[allow(clippy::too_many_arguments)] - async fn call_contract_function( - &self, - func: &str, - params: P, - from: A, - options: Options, - block: B, - contract_address: Address, - contract_abi: ethabi::Contract, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - B: Into> + Send, - P: Tokenize + Send, - { - self.as_ref() - .call_contract_function( - func, - params, - from, - options, - block, - contract_address, - contract_abi, +#[cfg(test)] +mod tests { + use super::*; + + #[tokio::test] + async fn managing_block_number() { + let client = MockEthereum::default(); + let block_number = client.block_number("test").await.unwrap(); + assert_eq!(block_number, 0.into()); + + client.advance_block_number(5); + let block_number = client.block_number("test").await.unwrap(); + assert_eq!(block_number, 5.into()); + } + + #[tokio::test] + async fn managing_transactions() { + let client = MockEthereum::default().with_non_ordering_confirmation(true); + client.advance_block_number(2); + + let signed_tx = client + .sign_prepared_tx( + b"test".to_vec(), + Options { + nonce: Some(1.into()), + ..Options::default() + }, ) - .await - } - - async fn tx_receipt( - &self, - tx_hash: H256, - component: &'static str, - ) -> Result, Error> { - self.as_ref().tx_receipt(tx_hash, component).await - } - - async fn eth_balance(&self, address: Address, component: &'static str) -> Result { - self.as_ref().eth_balance(address, component).await - } + .unwrap(); + assert_eq!(signed_tx.nonce, 1.into()); + assert!(signed_tx.max_priority_fee_per_gas > 0.into()); + assert!(signed_tx.max_fee_per_gas > 0.into()); - async fn logs(&self, filter: Filter, component: &'static str) -> Result, Error> { - self.as_ref().logs(filter, component).await - } - - async fn block( - &self, - block_id: BlockId, - component: &'static str, - ) -> Result>, Error> { - self.as_ref().block(block_id, component).await - } -} - -#[async_trait::async_trait] -impl + Send + Sync + fmt::Debug> BoundEthInterface for T { - fn contract(&self) -> ðabi::Contract { - self.as_ref().contract() - } - - fn contract_addr(&self) -> H160 { - self.as_ref().contract_addr() - } - - fn chain_id(&self) -> L1ChainId { - self.as_ref().chain_id() - } - - fn sender_account(&self) -> Address { - self.as_ref().sender_account() - } + let tx_hash = client.send_raw_tx(signed_tx.raw_tx).await.unwrap(); + assert_eq!(tx_hash, signed_tx.hash); - async fn sign_prepared_tx_for_addr( - &self, - data: Vec, - contract_addr: H160, - options: Options, - component: &'static str, - ) -> Result { - self.as_ref() - .sign_prepared_tx_for_addr(data, contract_addr, options, component) + client.execute_tx(tx_hash, true, 3); + let returned_tx = client + .get_tx(tx_hash, "test") .await - } - - async fn allowance_on_account( - &self, - token_address: Address, - contract_address: Address, - erc20_abi: ethabi::Contract, - ) -> Result { - self.as_ref() - .allowance_on_account(token_address, contract_address, erc20_abi) + .unwrap() + .expect("no transaction"); + assert_eq!(returned_tx.hash, tx_hash); + assert_eq!(returned_tx.input.0, b"test"); + assert_eq!(returned_tx.nonce, 1.into()); + assert!(returned_tx.max_priority_fee_per_gas.is_some()); + assert!(returned_tx.max_fee_per_gas.is_some()); + + let tx_status = client + .get_tx_status(tx_hash, "test") .await - } - - async fn nonce_at(&self, block: BlockNumber, component: &'static str) -> Result { - self.as_ref().nonce_at(block, component).await - } - - async fn pending_nonce(&self, _: &'static str) -> Result { - self.as_ref().pending_nonce("").await - } - - async fn current_nonce(&self, _: &'static str) -> Result { - self.as_ref().current_nonce("").await + .unwrap() + .expect("no transaction status"); + assert!(tx_status.success); + assert_eq!(tx_status.tx_hash, tx_hash); + assert_eq!(tx_status.receipt.block_number, Some(2.into())); } } diff --git a/core/lib/eth_client/src/clients/mod.rs b/core/lib/eth_client/src/clients/mod.rs index e992fac2eaf6..aa77974c4945 100644 --- a/core/lib/eth_client/src/clients/mod.rs +++ b/core/lib/eth_client/src/clients/mod.rs @@ -1,2 +1,10 @@ -pub mod http; -pub mod mock; +//! Various Ethereum client implementations. + +mod generic; +mod http; +mod mock; + +pub use self::{ + http::{PKSigningClient, QueryClient, SigningClient}, + mock::MockEthereum, +}; diff --git a/core/lib/eth_client/src/lib.rs b/core/lib/eth_client/src/lib.rs index ff7ce7ed8c60..2eb1ea51db81 100644 --- a/core/lib/eth_client/src/lib.rs +++ b/core/lib/eth_client/src/lib.rs @@ -1,14 +1,9 @@ -#![allow(clippy::upper_case_acronyms, clippy::derive_partial_eq_without_eq)] - use std::fmt; use async_trait::async_trait; use zksync_types::{ web3::{ - contract::{ - tokens::{Detokenize, Tokenize}, - Options, - }, + contract::{tokens::Tokenize, Options}, ethabi, types::{ Address, Block, BlockId, BlockNumber, Filter, Log, Transaction, TransactionReceipt, @@ -18,10 +13,13 @@ use zksync_types::{ L1ChainId, }; -use crate::types::{Error, ExecutedTxStatus, FailureInfo, SignedCallResult}; +pub use crate::types::{ + CallFunctionArgs, ContractCall, Error, ExecutedTxStatus, FailureInfo, RawTransactionBytes, + SignedCallResult, +}; pub mod clients; -pub mod types; +mod types; /// Common Web3 interface, as seen by the core applications. /// Encapsulates the raw Web3 interaction, providing a high-level interface. @@ -73,7 +71,7 @@ pub trait EthInterface: 'static + Sync + Send + fmt::Debug { async fn block_number(&self, component: &'static str) -> Result; /// Sends a transaction to the Ethereum network. - async fn send_raw_tx(&self, tx: Vec) -> Result; + async fn send_raw_tx(&self, tx: RawTransactionBytes) -> Result; /// Fetches the transaction status for a specified transaction hash. /// @@ -111,22 +109,8 @@ pub trait EthInterface: 'static + Sync + Send + fmt::Debug { async fn eth_balance(&self, address: Address, component: &'static str) -> Result; /// Invokes a function on a contract specified by `contract_address` / `contract_abi` using `eth_call`. - #[allow(clippy::too_many_arguments)] - async fn call_contract_function( - &self, - func: &str, - params: P, - from: A, - options: Options, - block: B, - contract_address: Address, - contract_abi: ethabi::Contract, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - B: Into> + Send, - P: Tokenize + Send; + async fn call_contract_function(&self, call: ContractCall) + -> Result, Error>; /// Returns the logs for the specified filter. async fn logs(&self, filter: Filter, component: &'static str) -> Result, Error>; @@ -139,6 +123,9 @@ pub trait EthInterface: 'static + Sync + Send + fmt::Debug { ) -> Result>, Error>; } +#[cfg(test)] +static_assertions::assert_obj_safe!(EthInterface); + /// An extension of `EthInterface` trait, which is used to perform queries that are bound to /// a certain contract and account. /// @@ -228,30 +215,12 @@ pub trait BoundEthInterface: EthInterface { } /// Invokes a function on a contract specified by `Self::contract()` / `Self::contract_addr()`. - async fn call_main_contract_function( + async fn call_main_contract_function( &self, - func: &str, - params: P, - from: A, - options: Options, - block: B, - ) -> Result - where - R: Detokenize + Unpin, - A: Into> + Send, - P: Tokenize + Send, - B: Into> + Send, - { - self.call_contract_function( - func, - params, - from, - options, - block, - self.contract_addr(), - self.contract().clone(), - ) - .await + args: CallFunctionArgs, + ) -> Result, Error> { + let args = args.for_contract(self.contract_addr(), self.contract().clone()); + self.call_contract_function(args).await } /// Encodes a function using the `Self::contract()` ABI. diff --git a/core/lib/eth_client/src/types.rs b/core/lib/eth_client/src/types.rs index bef3b63b69a9..71d9473b661a 100644 --- a/core/lib/eth_client/src/types.rs +++ b/core/lib/eth_client/src/types.rs @@ -1,9 +1,81 @@ -// External uses use zksync_types::web3::{ + contract::{ + tokens::{Detokenize, Tokenize}, + Error as ContractError, Options, + }, ethabi, - types::{TransactionReceipt, H256, U256}, + types::{Address, BlockId, TransactionReceipt, H256, U256}, }; +/// Wrapper for `Vec` that doesn't wrap them in an additional array in `Tokenize` implementation. +#[derive(Debug)] +pub(crate) struct RawTokens(pub Vec); + +impl Tokenize for RawTokens { + fn into_tokens(self) -> Vec { + self.0 + } +} + +impl Detokenize for RawTokens { + fn from_tokens(tokens: Vec) -> Result { + Ok(Self(tokens)) + } +} + +/// Arguments for calling a function in an unspecified Ethereum smart contract. +#[derive(Debug)] +pub struct CallFunctionArgs { + pub(crate) name: String, + pub(crate) from: Option
, + pub(crate) options: Options, + pub(crate) block: Option, + pub(crate) params: RawTokens, +} + +impl CallFunctionArgs { + pub fn new(name: &str, params: impl Tokenize) -> Self { + Self { + name: name.to_owned(), + from: None, + options: Options::default(), + block: None, + params: RawTokens(params.into_tokens()), + } + } + + pub fn with_sender(mut self, from: Address) -> Self { + self.from = Some(from); + self + } + + pub fn with_block(mut self, block: BlockId) -> Self { + self.block = Some(block); + self + } + + pub fn for_contract( + self, + contract_address: Address, + contract_abi: ethabi::Contract, + ) -> ContractCall { + ContractCall { + contract_address, + contract_abi, + inner: self, + } + } +} + +/// Information sufficient for calling a function in a specific Ethereum smart contract. Instantiated +/// using [`CallFunctionArgs::for_contract()`]. +#[derive(Debug)] +pub struct ContractCall { + pub(crate) contract_address: Address, + pub(crate) contract_abi: ethabi::Contract, + pub(crate) inner: CallFunctionArgs, +} + /// Common error type exposed by the crate, #[derive(Debug, thiserror::Error)] pub enum Error { @@ -24,11 +96,29 @@ pub enum Error { WrongFeeProvided(U256, U256), } +/// Raw transaction bytes. +#[derive(Debug, Clone, PartialEq)] +pub struct RawTransactionBytes(pub(crate) Vec); + +impl RawTransactionBytes { + /// Converts raw transaction bytes. It is caller's responsibility to ensure that these bytes + /// were actually obtained by signing a transaction. + pub fn new_unchecked(bytes: Vec) -> Self { + Self(bytes) + } +} + +impl AsRef<[u8]> for RawTransactionBytes { + fn as_ref(&self) -> &[u8] { + &self.0 + } +} + /// Representation of a signed transaction. #[derive(Debug, Clone, PartialEq)] pub struct SignedCallResult { /// Raw transaction bytes. - pub raw_tx: Vec, + pub raw_tx: RawTransactionBytes, /// `max_priority_fee_per_gas` field of transaction (EIP1559). pub max_priority_fee_per_gas: U256, /// `max_fee_per_gas` field of transaction (EIP1559). @@ -69,3 +159,21 @@ pub struct FailureInfo { /// Gas limit of the transaction. pub gas_limit: U256, } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn raw_tokens_are_compatible_with_actual_call() { + let vk_contract = zksync_contracts::verifier_contract(); + let args = CallFunctionArgs::new("verificationKeyHash", ()); + let func = vk_contract.function(&args.name).unwrap(); + func.encode_input(&args.params.into_tokens()).unwrap(); + + let output_tokens = vec![ethabi::Token::FixedBytes(vec![1; 32])]; + let RawTokens(output_tokens) = RawTokens::from_tokens(output_tokens).unwrap(); + let hash = H256::from_tokens(output_tokens).unwrap(); + assert_eq!(hash, H256::repeat_byte(1)); + } +} diff --git a/core/lib/zksync_core/src/eth_sender/error.rs b/core/lib/zksync_core/src/eth_sender/error.rs index 080e252c92c2..206bbf2d583a 100644 --- a/core/lib/zksync_core/src/eth_sender/error.rs +++ b/core/lib/zksync_core/src/eth_sender/error.rs @@ -1,10 +1,9 @@ -use zksync_eth_client::types; use zksync_types::web3::contract; #[derive(Debug, thiserror::Error)] pub enum ETHSenderError { #[error("Ethereum gateway Error {0}")] - EthereumGateWayError(#[from] types::Error), + EthereumGateWayError(#[from] zksync_eth_client::Error), #[error("Token parsing Error: {0}")] ParseError(#[from] contract::Error), } diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs index 5e07c3604b6c..e8f527c2abeb 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs @@ -4,7 +4,7 @@ use tokio::sync::watch; use zksync_config::configs::eth_sender::SenderConfig; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{ConnectionPool, StorageProcessor}; -use zksync_eth_client::BoundEthInterface; +use zksync_eth_client::{BoundEthInterface, CallFunctionArgs}; use zksync_types::{ aggregated_operations::AggregatedOperation, contracts::{Multicall3Call, Multicall3Result}, @@ -12,7 +12,10 @@ use zksync_types::{ ethabi::{Contract, Token}, protocol_version::{L1VerifierConfig, VerifierParams}, vk_transform::l1_vk_commitment, - web3::contract::{tokens::Tokenizable, Error, Options}, + web3::contract::{ + tokens::{Detokenize, Tokenizable}, + Error, + }, Address, ProtocolVersionId, H256, U256, }; @@ -99,20 +102,12 @@ impl EthTxAggregator { pub(super) async fn get_multicall_data(&mut self) -> Result { let calldata = self.generate_calldata_for_multicall(); - let aggregate3_result = self - .eth_client - .call_contract_function( - &self.functions.aggregate3.name, - calldata, - None, - Options::default(), - None, - self.l1_multicall3_address, - self.functions.multicall_contract.clone(), - ) - .await?; - - self.parse_multicall_data(aggregate3_result) + let args = CallFunctionArgs::new(&self.functions.aggregate3.name, calldata).for_contract( + self.l1_multicall3_address, + self.functions.multicall_contract.clone(), + ); + let aggregate3_result = self.eth_client.call_contract_function(args).await?; + self.parse_multicall_data(Token::from_tokens(aggregate3_result)?) } // Multicall's aggregate function accepts 1 argument - arrays of different contract calls. @@ -304,43 +299,25 @@ impl EthTxAggregator { tracing::debug!("Calling get_verification_key"); if contracts_are_pre_boojum { let abi = Contract { - functions: vec![( + functions: [( self.functions.get_verification_key.name.clone(), vec![self.functions.get_verification_key.clone()], )] - .into_iter() - .collect(), + .into(), ..Default::default() }; - let vk = self - .eth_client - .call_contract_function( - &self.functions.get_verification_key.name, - (), - None, - Default::default(), - None, - verifier_address, - abi, - ) - .await?; - Ok(l1_vk_commitment(vk)) + let args = CallFunctionArgs::new(&self.functions.get_verification_key.name, ()) + .for_contract(verifier_address, abi); + + let vk = self.eth_client.call_contract_function(args).await?; + Ok(l1_vk_commitment(Token::from_tokens(vk)?)) } else { let get_vk_hash = self.functions.verification_key_hash.as_ref(); tracing::debug!("Calling verificationKeyHash"); - let vk_hash = self - .eth_client - .call_contract_function( - &get_vk_hash.unwrap().name, - (), - None, - Default::default(), - None, - verifier_address, - self.functions.verifier_contract.clone(), - ) - .await?; - Ok(vk_hash) + let args = CallFunctionArgs::new(&get_vk_hash.unwrap().name, ()) + .for_contract(verifier_address, self.functions.verifier_contract.clone()); + let vk_hash = self.eth_client.call_contract_function(args).await?; + Ok(H256::from_tokens(vk_hash)?) } } diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs index 0d219b38da43..6000b841f3b8 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs @@ -5,8 +5,7 @@ use tokio::sync::watch; use zksync_config::configs::eth_sender::SenderConfig; use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_eth_client::{ - types::{Error, ExecutedTxStatus, SignedCallResult}, - BoundEthInterface, + BoundEthInterface, Error, ExecutedTxStatus, RawTransactionBytes, SignedCallResult, }; use zksync_types::{ eth_sender::EthTx, @@ -211,7 +210,7 @@ where base_fee_per_gas, priority_fee_per_gas, signed_tx.hash, - signed_tx.raw_tx.clone(), + signed_tx.raw_tx.as_ref(), ) .await .unwrap() @@ -236,7 +235,7 @@ where &self, storage: &mut StorageProcessor<'_>, tx_history_id: u32, - raw_tx: Vec, + raw_tx: RawTransactionBytes, current_block: L1BlockNumber, ) -> Result { match self.ethereum_gateway.send_raw_tx(raw_tx).await { @@ -439,12 +438,12 @@ where .send_raw_transaction( storage, tx.id, - tx.signed_raw_tx.clone(), + RawTransactionBytes::new_unchecked(tx.signed_raw_tx.clone()), l1_block_numbers.latest, ) .await { - tracing::warn!("Error {:?} in sending tx {:?}", error, &tx); + tracing::warn!("Error sending transaction {tx:?}: {error}"); } } } diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index b563cf0443e7..e20f56fb250b 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -1,4 +1,4 @@ -use std::sync::{atomic::Ordering, Arc}; +use std::sync::Arc; use assert_matches::assert_matches; use once_cell::sync::Lazy; @@ -8,7 +8,7 @@ use zksync_config::{ }; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{ConnectionPool, StorageProcessor}; -use zksync_eth_client::{clients::mock::MockEthereum, EthInterface}; +use zksync_eth_client::{clients::MockEthereum, EthInterface}; use zksync_object_store::ObjectStoreFactory; use zksync_types::{ aggregated_operations::{ @@ -84,9 +84,7 @@ impl EthSenderTester { .with_non_ordering_confirmation(non_ordering_confirmations) .with_multicall_address(contracts_config.l1_multicall3_addr), ); - gateway - .block_number - .fetch_add(Self::WAIT_CONFIRMATIONS, Ordering::Relaxed); + gateway.advance_block_number(Self::WAIT_CONFIRMATIONS); let gas_adjuster = Arc::new( GasAdjuster::new( @@ -176,7 +174,7 @@ async fn confirm_many() -> anyhow::Result<()> { } // check that we sent something - assert_eq!(tester.gateway.sent_txs.read().unwrap().len(), 5); + assert_eq!(tester.gateway.sent_tx_count(), 5); assert_eq!( tester .storage() @@ -192,7 +190,7 @@ async fn confirm_many() -> anyhow::Result<()> { for hash in hashes { tester .gateway - .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS)?; + .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS); } let to_resend = tester @@ -253,7 +251,7 @@ async fn resend_each_block() -> anyhow::Result<()> { .await?; // check that we sent something and stored it in the db - assert_eq!(tester.gateway.sent_txs.read().unwrap().len(), 1); + assert_eq!(tester.gateway.sent_tx_count(), 1); assert_eq!( tester .storage() @@ -266,10 +264,18 @@ async fn resend_each_block() -> anyhow::Result<()> { 1 ); - let sent_tx = tester.gateway.sent_txs.read().unwrap()[&hash]; + let sent_tx = tester + .gateway + .get_tx(hash, "") + .await + .unwrap() + .expect("no transaction"); assert_eq!(sent_tx.hash, hash); - assert_eq!(sent_tx.nonce, 0); - assert_eq!(sent_tx.base_fee.as_usize(), 18); // 6 * 3 * 2^0 + assert_eq!(sent_tx.nonce, 0.into()); + assert_eq!( + sent_tx.max_fee_per_gas.unwrap() - sent_tx.max_priority_fee_per_gas.unwrap(), + 18.into() // 6 * 3 * 2^0 + ); // now, median is 5 tester.gateway.advance_block_number(2); @@ -296,7 +302,7 @@ async fn resend_each_block() -> anyhow::Result<()> { .await?; // check that transaction has been resent - assert_eq!(tester.gateway.sent_txs.read().unwrap().len(), 2); + assert_eq!(tester.gateway.sent_tx_count(), 2); assert_eq!( tester .storage() @@ -309,9 +315,17 @@ async fn resend_each_block() -> anyhow::Result<()> { 1 ); - let resent_tx = tester.gateway.sent_txs.read().unwrap()[&resent_hash]; - assert_eq!(resent_tx.nonce, 0); - assert_eq!(resent_tx.base_fee.as_usize(), 30); // 5 * 3 * 2^1 + let resent_tx = tester + .gateway + .get_tx(resent_hash, "") + .await + .unwrap() + .expect("no transaction"); + assert_eq!(resent_tx.nonce, 0.into()); + assert_eq!( + resent_tx.max_fee_per_gas.unwrap() - resent_tx.max_priority_fee_per_gas.unwrap(), + 30.into() // 5 * 3 * 2^1 + ); Ok(()) } @@ -344,7 +358,7 @@ async fn dont_resend_already_mined() -> anyhow::Result<()> { .unwrap(); // check that we sent something and stored it in the db - assert_eq!(tester.gateway.sent_txs.read().unwrap().len(), 1); + assert_eq!(tester.gateway.sent_tx_count(), 1); assert_eq!( tester .storage() @@ -360,7 +374,7 @@ async fn dont_resend_already_mined() -> anyhow::Result<()> { // mine the transaction but don't have enough confirmations yet tester .gateway - .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS - 1)?; + .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS - 1); let to_resend = tester .manager @@ -421,17 +435,16 @@ async fn three_scenarios() -> anyhow::Result<()> { } // check that we sent something - assert_eq!(tester.gateway.sent_txs.read().unwrap().len(), 3); + assert_eq!(tester.gateway.sent_tx_count(), 3); // mined & confirmed tester .gateway - .execute_tx(hashes[0], true, EthSenderTester::WAIT_CONFIRMATIONS)?; - + .execute_tx(hashes[0], true, EthSenderTester::WAIT_CONFIRMATIONS); // mined but not confirmed tester .gateway - .execute_tx(hashes[1], true, EthSenderTester::WAIT_CONFIRMATIONS - 1)?; + .execute_tx(hashes[1], true, EthSenderTester::WAIT_CONFIRMATIONS - 1); let (to_resend, _) = tester .manager @@ -491,8 +504,7 @@ async fn failed_eth_tx() { // fail this tx tester .gateway - .execute_tx(hash, false, EthSenderTester::WAIT_CONFIRMATIONS) - .unwrap(); + .execute_tx(hash, false, EthSenderTester::WAIT_CONFIRMATIONS); tester .manager .monitor_inflight_transactions( @@ -980,9 +992,7 @@ async fn send_operation( async fn confirm_tx(tester: &mut EthSenderTester, hash: H256) { tester .gateway - .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS) - .unwrap(); - + .execute_tx(hash, true, EthSenderTester::WAIT_CONFIRMATIONS); tester .manager .monitor_inflight_transactions( diff --git a/core/lib/zksync_core/src/eth_watch/client.rs b/core/lib/zksync_core/src/eth_watch/client.rs index cbd3785640e2..b00d3e1d79de 100644 --- a/core/lib/zksync_core/src/eth_watch/client.rs +++ b/core/lib/zksync_core/src/eth_watch/client.rs @@ -1,10 +1,11 @@ use zksync_contracts::verifier_contract; -use zksync_eth_client::{types::Error as EthClientError, EthInterface}; +use zksync_eth_client::{CallFunctionArgs, Error as EthClientError, EthInterface}; use zksync_types::{ ethabi::{Contract, Token}, vk_transform::l1_vk_commitment, web3::{ self, + contract::tokens::Detokenize, types::{BlockId, BlockNumber, FilterBuilder, Log}, }, Address, H256, @@ -22,6 +23,12 @@ pub enum Error { InfiniteRecursion, } +impl From for Error { + fn from(err: web3::contract::Error) -> Self { + Self::EthClient(err.into()) + } +} + #[async_trait::async_trait] pub trait EthClient { /// Returns events in a given block range. @@ -107,35 +114,17 @@ impl EthClient for EthHttpQueryClient EthereumProvider { address: Address, token_address: Address, ) -> Result { + let args = CallFunctionArgs::new("balanceOf", address) + .for_contract(token_address, self.erc20_abi.clone()); let res = self .eth_client - .call_contract_function( - "balanceOf", - address, - None, - Options::default(), - None, - token_address, - self.erc20_abi.clone(), - ) + .call_contract_function(args) .await .map_err(|err| ClientError::NetworkError(err.to_string()))?; - Ok(res) + U256::from_tokens(res).map_err(|err| ClientError::MalformedResponse(err.to_string())) } /// Returns the pending nonce for the Ethereum account. @@ -186,20 +183,14 @@ impl EthereumProvider { bridge: Option
, ) -> Result { let bridge = bridge.unwrap_or(self.default_bridges.l1_erc20_default_bridge); - let l2_token_address = self + let args = CallFunctionArgs::new("l2TokenAddress", l1_token_address) + .for_contract(bridge, self.l1_bridge_abi.clone()); + let res = self .eth_client - .call_contract_function( - "l2TokenAddress", - l1_token_address, - None, - Options::default(), - None, - bridge, - self.l1_bridge_abi.clone(), - ) + .call_contract_function(args) .await .map_err(|err| ClientError::NetworkError(err.to_string()))?; - Ok(l2_token_address) + Address::from_tokens(res).map_err(|err| ClientError::MalformedResponse(err.to_string())) } /// Checks whether ERC20 of a certain token deposit with limit is approved for account. @@ -372,15 +363,12 @@ impl EthereumProvider { } else { self.eth_client.get_gas_price("zksync-rs").await? }; - self.eth_client - .call_main_contract_function( - "l2TransactionBaseCost", - (gas_price, gas_limit, gas_per_pubdata_byte), - None, - Default::default(), - None, - ) - .await + let args = CallFunctionArgs::new( + "l2TransactionBaseCost", + (gas_price, gas_limit, gas_per_pubdata_byte), + ); + let res = self.eth_client.call_main_contract_function(args).await?; + Ok(U256::from_tokens(res)?) } #[allow(clippy::too_many_arguments)] From 9d0aefc1ef4992e19d7b15ec1ce34697e61a3464 Mon Sep 17 00:00:00 2001 From: EmilLuta Date: Wed, 3 Jan 2024 15:30:45 +0100 Subject: [PATCH 05/49] fix(prover): Remove old prover subsystems tables (#812) These tables are unused and should be removed. Note that reverting back is impossible for a production system, but may still be desired for troubleshooting reasons. The changes are part of the migration to boojum from old system. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- ...5908_remove_old_prover_subsystems.down.sql | 52 +++++++++++++++++++ ...125908_remove_old_prover_subsystems.up.sql | 5 ++ 2 files changed, 57 insertions(+) create mode 100644 core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.down.sql create mode 100644 core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.up.sql diff --git a/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.down.sql b/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.down.sql new file mode 100644 index 000000000000..5b5531589af3 --- /dev/null +++ b/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.down.sql @@ -0,0 +1,52 @@ +-- Note that era can't revert to this point in time. +-- These tables are added only if engineers want to revert from a future codebase to a previous codebase. +-- This migration will enable backwards development (i.e. bisecting some error). + +CREATE TABLE IF NOT EXISTS gpu_prover_queue ( + id bigint NOT NULL PRIMARY KEY, + instance_host inet NOT NULL, + instance_port integer NOT NULL, + instance_status text NOT NULL, + created_at timestamp without time zone NOT NULL, + updated_at timestamp without time zone NOT NULL, + processing_started_at timestamp without time zone, + queue_free_slots integer, + queue_capacity integer, + specialized_prover_group_id smallint, + region text NOT NULL, + zone text NOT NULL, + num_gpu smallint, + CONSTRAINT valid_port CHECK (((instance_port >= 0) AND (instance_port <= 65535))) +); + +CREATE TABLE IF NOT EXISTS prover_jobs ( + id bigint NOT NULL PRIMARY KEY, + l1_batch_number bigint NOT NULL, + circuit_type text NOT NULL, + prover_input bytea NOT NULL, + status text NOT NULL, + error text, + processing_started_at timestamp without time zone, + created_at timestamp without time zone NOT NULL, + updated_at timestamp without time zone NOT NULL, + time_taken time without time zone DEFAULT '00:00:00'::time without time zone NOT NULL, + aggregation_round integer DEFAULT 0 NOT NULL, + result bytea, + sequence_number integer DEFAULT 0 NOT NULL, + attempts integer DEFAULT 0 NOT NULL, + circuit_input_blob_url text, + proccesed_by text, + is_blob_cleaned boolean DEFAULT false NOT NULL, + protocol_version integer +); + +CREATE TABLE IF NOT EXISTS prover_protocol_versions ( + id integer NOT NULL, + "timestamp" bigint NOT NULL, + recursion_scheduler_level_vk_hash bytea NOT NULL, + recursion_node_level_vk_hash bytea NOT NULL, + recursion_leaf_level_vk_hash bytea NOT NULL, + recursion_circuits_set_vks_hash bytea NOT NULL, + verifier_address bytea NOT NULL, + created_at timestamp without time zone NOT NULL +); diff --git a/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.up.sql b/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.up.sql new file mode 100644 index 000000000000..473706875fb4 --- /dev/null +++ b/core/lib/dal/migrations/20240103125908_remove_old_prover_subsystems.up.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS gpu_prover_queue; + +DROP TABLE IF EXISTS prover_jobs; + +DROP TABLE IF EXISTS prover_protocol_versions; From 4071e90578e0fc8c027a4d2a30d09d96db942b4f Mon Sep 17 00:00:00 2001 From: Stanislav Bezkorovainyi Date: Wed, 3 Jan 2024 16:33:27 +0100 Subject: [PATCH 06/49] feat(vm): Add boojum integration folder (#805) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ In preparation for the 1.4.1 we firsly copy the boojum integration folder (without adding it to the workspace), so that the next PR where the new VM is actually integrated is easier to review ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .../versions/vm_boojum_integration/README.md | 44 + .../bootloader_state/l2_block.rs | 87 ++ .../bootloader_state/mod.rs | 8 + .../bootloader_state/snapshot.rs | 25 + .../bootloader_state/state.rs | 295 +++++++ .../bootloader_state/tx.rs | 49 ++ .../bootloader_state/utils.rs | 177 ++++ .../vm_boojum_integration/constants.rs | 127 +++ .../implementation/bytecode.rs | 58 ++ .../implementation/execution.rs | 136 +++ .../implementation/gas.rs | 43 + .../implementation/logs.rs | 72 ++ .../implementation/mod.rs | 7 + .../implementation/snapshots.rs | 91 ++ .../implementation/statistics.rs | 70 ++ .../implementation/tx.rs | 68 ++ .../src/versions/vm_boojum_integration/mod.rs | 36 + .../old_vm/event_sink.rs | 263 ++++++ .../vm_boojum_integration/old_vm/events.rs | 146 ++++ .../old_vm/history_recorder.rs | 811 ++++++++++++++++++ .../vm_boojum_integration/old_vm/memory.rs | 327 +++++++ .../vm_boojum_integration/old_vm/mod.rs | 8 + .../old_vm/oracles/decommitter.rs | 236 +++++ .../old_vm/oracles/mod.rs | 8 + .../old_vm/oracles/precompile.rs | 72 ++ .../old_vm/oracles/storage.rs | 337 ++++++++ .../vm_boojum_integration/old_vm/utils.rs | 221 +++++ .../vm_boojum_integration/oracles/mod.rs | 1 + .../vm_boojum_integration/oracles/storage.rs | 489 +++++++++++ .../vm_boojum_integration/tests/bootloader.rs | 56 ++ .../tests/bytecode_publishing.rs | 43 + .../tests/call_tracer.rs | 92 ++ .../vm_boojum_integration/tests/default_aa.rs | 76 ++ .../vm_boojum_integration/tests/gas_limit.rs | 47 + .../tests/get_used_contracts.rs | 109 +++ .../tests/invalid_bytecode.rs | 120 +++ .../tests/is_write_initial.rs | 48 ++ .../tests/l1_tx_execution.rs | 139 +++ .../vm_boojum_integration/tests/l2_blocks.rs | 437 ++++++++++ .../vm_boojum_integration/tests/mod.rs | 20 + .../tests/nonce_holder.rs | 188 ++++ .../vm_boojum_integration/tests/refunds.rs | 167 ++++ .../tests/require_eip712.rs | 165 ++++ .../vm_boojum_integration/tests/rollbacks.rs | 263 ++++++ .../tests/simple_execution.rs | 81 ++ .../tests/tester/inner_state.rs | 130 +++ .../vm_boojum_integration/tests/tester/mod.rs | 7 + .../tests/tester/transaction_test_info.rs | 217 +++++ .../tests/tester/vm_tester.rs | 295 +++++++ .../tests/tracing_execution_error.rs | 54 ++ .../vm_boojum_integration/tests/upgrade.rs | 362 ++++++++ .../vm_boojum_integration/tests/utils.rs | 105 +++ .../tracers/default_tracers.rs | 298 +++++++ .../tracers/dispatcher.rs | 126 +++ .../vm_boojum_integration/tracers/mod.rs | 13 + .../tracers/pubdata_tracer.rs | 212 +++++ .../vm_boojum_integration/tracers/refunds.rs | 350 ++++++++ .../tracers/result_tracer.rs | 246 ++++++ .../vm_boojum_integration/tracers/traits.rs | 47 + .../vm_boojum_integration/tracers/utils.rs | 225 +++++ .../types/internals/mod.rs | 9 + .../types/internals/pubdata.rs | 124 +++ .../types/internals/snapshot.rs | 11 + .../types/internals/transaction_data.rs | 345 ++++++++ .../types/internals/vm_state.rs | 183 ++++ .../vm_boojum_integration/types/l1_batch.rs | 38 + .../vm_boojum_integration/types/mod.rs | 2 + .../vm_boojum_integration/utils/fee.rs | 29 + .../vm_boojum_integration/utils/l2_blocks.rs | 95 ++ .../vm_boojum_integration/utils/logs.rs | 25 + .../vm_boojum_integration/utils/mod.rs | 6 + .../vm_boojum_integration/utils/overhead.rs | 349 ++++++++ .../utils/transaction_encoding.rs | 16 + .../src/versions/vm_boojum_integration/vm.rs | 184 ++++ 74 files changed, 10466 insertions(+) create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/README.md create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/snapshot.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/constants.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/events.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/oracles/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/internals/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/types/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/mod.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/vm.rs diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/README.md b/core/lib/multivm/src/versions/vm_boojum_integration/README.md new file mode 100644 index 000000000000..d515df0dfc60 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/README.md @@ -0,0 +1,44 @@ +# VM Crate + +This crate contains code that interacts with the VM (Virtual Machine). The VM itself is in a separate repository +[era-zk_evm][zk_evm_repo_ext]. + +## VM Dependencies + +The VM relies on several subcomponents or traits, such as Memory and Storage. These traits are defined in the `zk_evm` +repository, while their implementations can be found in this crate, such as the storage implementation in +`oracles/storage.rs` and the Memory implementation in `memory.rs`. + +Many of these implementations also support easy rollbacks and history, which is useful when creating a block with +multiple transactions and needing to return the VM to a previous state if a transaction doesn't fit. + +## Running the VM + +To interact with the VM, you must initialize it with `L1BatchEnv`, which represents the initial parameters of the batch, +`SystemEnv`, that represents the system parameters, and a reference to the Storage. To execute a transaction, you have +to push the transaction into the bootloader memory and call the `execute_next_transaction` method. + +### Tracers + +The VM implementation allows for the addition of `Tracers`, which are activated before and after each instruction. This +provides a more in-depth look into the VM, collecting detailed debugging information and logs. More details can be found +in the `tracer/` directory. + +This VM also supports custom tracers. You can call the `inspect_next_transaction` method with a custom tracer and +receive the result of the execution. + +### Bootloader + +In the context of zkEVM, we usually think about transactions. However, from the VM's perspective, it runs a single +program called the bootloader, which internally processes multiple transactions. + +### Rollbacks + +The `VMInstance` in `vm.rs` allows for easy rollbacks. You can save the current state at any moment by calling +`make_snapshot()` and return to that state using `rollback_to_the_latest_snapshot()`. + +This rollback affects all subcomponents, such as memory, storage, and events, and is mainly used if a transaction +doesn't fit in a block. + +[zk_evm_repo]: https://github.com/matter-labs/zk_evm 'internal zk EVM repo' +[zk_evm_repo_ext]: https://github.com/matter-labs/era-zk_evm 'external zk EVM repo' diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs new file mode 100644 index 000000000000..146e8713c69e --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs @@ -0,0 +1,87 @@ +use std::cmp::Ordering; + +use zksync_types::{MiniblockNumber, H256}; +use zksync_utils::concat_and_hash; + +use crate::{ + interface::{L2Block, L2BlockEnv}, + vm_latest::{ + bootloader_state::{snapshot::L2BlockSnapshot, tx::BootloaderTx}, + utils::l2_blocks::l2_block_hash, + }, +}; + +const EMPTY_TXS_ROLLING_HASH: H256 = H256::zero(); + +#[derive(Debug, Clone)] +pub(crate) struct BootloaderL2Block { + pub(crate) number: u32, + pub(crate) timestamp: u64, + pub(crate) txs_rolling_hash: H256, // The rolling hash of all the transactions in the miniblock + pub(crate) prev_block_hash: H256, + // Number of the first l2 block tx in l1 batch + pub(crate) first_tx_index: usize, + pub(crate) max_virtual_blocks_to_create: u32, + pub(super) txs: Vec, +} + +impl BootloaderL2Block { + pub(crate) fn new(l2_block: L2BlockEnv, first_tx_place: usize) -> Self { + Self { + number: l2_block.number, + timestamp: l2_block.timestamp, + txs_rolling_hash: EMPTY_TXS_ROLLING_HASH, + prev_block_hash: l2_block.prev_block_hash, + first_tx_index: first_tx_place, + max_virtual_blocks_to_create: l2_block.max_virtual_blocks_to_create, + txs: vec![], + } + } + + pub(super) fn push_tx(&mut self, tx: BootloaderTx) { + self.update_rolling_hash(tx.hash); + self.txs.push(tx) + } + + pub(crate) fn get_hash(&self) -> H256 { + l2_block_hash( + MiniblockNumber(self.number), + self.timestamp, + self.prev_block_hash, + self.txs_rolling_hash, + ) + } + + fn update_rolling_hash(&mut self, tx_hash: H256) { + self.txs_rolling_hash = concat_and_hash(self.txs_rolling_hash, tx_hash) + } + + pub(crate) fn interim_version(&self) -> BootloaderL2Block { + let mut interim = self.clone(); + interim.max_virtual_blocks_to_create = 0; + interim + } + + pub(crate) fn make_snapshot(&self) -> L2BlockSnapshot { + L2BlockSnapshot { + txs_rolling_hash: self.txs_rolling_hash, + txs_len: self.txs.len(), + } + } + + pub(crate) fn apply_snapshot(&mut self, snapshot: L2BlockSnapshot) { + self.txs_rolling_hash = snapshot.txs_rolling_hash; + match self.txs.len().cmp(&snapshot.txs_len) { + Ordering::Greater => self.txs.truncate(snapshot.txs_len), + Ordering::Less => panic!("Applying snapshot from future is not supported"), + Ordering::Equal => {} + } + } + pub(crate) fn l2_block(&self) -> L2Block { + L2Block { + number: self.number, + timestamp: self.timestamp, + hash: self.get_hash(), + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/mod.rs new file mode 100644 index 000000000000..73830de2759b --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/mod.rs @@ -0,0 +1,8 @@ +mod l2_block; +mod snapshot; +mod state; +mod tx; + +pub(crate) mod utils; +pub(crate) use snapshot::BootloaderStateSnapshot; +pub use state::BootloaderState; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/snapshot.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/snapshot.rs new file mode 100644 index 000000000000..8f1cec3cb7f1 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/snapshot.rs @@ -0,0 +1,25 @@ +use zksync_types::H256; + +#[derive(Debug, Clone)] +pub(crate) struct BootloaderStateSnapshot { + /// ID of the next transaction to be executed. + pub(crate) tx_to_execute: usize, + /// Stored L2 blocks in bootloader memory + pub(crate) l2_blocks_len: usize, + /// Snapshot of the last L2 block. Only this block could be changed during the rollback + pub(crate) last_l2_block: L2BlockSnapshot, + /// The number of 32-byte words spent on the already included compressed bytecodes. + pub(crate) compressed_bytecodes_encoding: usize, + /// Current offset of the free space in the bootloader memory. + pub(crate) free_tx_offset: usize, + /// Whether the pubdata information has been provided already + pub(crate) is_pubdata_information_provided: bool, +} + +#[derive(Debug, Clone)] +pub(crate) struct L2BlockSnapshot { + /// The rolling hash of all the transactions in the miniblock + pub(crate) txs_rolling_hash: H256, + /// The number of transactions in the last L2 block + pub(crate) txs_len: usize, +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs new file mode 100644 index 000000000000..d914aacab178 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs @@ -0,0 +1,295 @@ +use std::cmp::Ordering; + +use once_cell::sync::OnceCell; +use zksync_types::{L2ChainId, U256}; +use zksync_utils::bytecode::CompressedBytecodeInfo; + +use super::{tx::BootloaderTx, utils::apply_pubdata_to_memory}; +use crate::{ + interface::{BootloaderMemory, L2BlockEnv, TxExecutionMode}, + vm_latest::{ + bootloader_state::{ + l2_block::BootloaderL2Block, + snapshot::BootloaderStateSnapshot, + utils::{apply_l2_block, apply_tx_to_memory}, + }, + constants::TX_DESCRIPTION_OFFSET, + types::internals::{PubdataInput, TransactionData}, + utils::l2_blocks::assert_next_block, + }, +}; + +/// Intermediate bootloader-related VM state. +/// +/// Required to process transactions one by one (since we intercept the VM execution to execute +/// transactions and add new ones to the memory on the fly). +/// Keeps tracking everything related to the bootloader memory and can restore the whole memory. +/// +/// +/// Serves two purposes: +/// - Tracks where next tx should be pushed to in the bootloader memory. +/// - Tracks which transaction should be executed next. +#[derive(Debug, Clone)] +pub struct BootloaderState { + /// ID of the next transaction to be executed. + /// See the structure doc-comment for a better explanation of purpose. + tx_to_execute: usize, + /// Stored txs in bootloader memory + l2_blocks: Vec, + /// The number of 32-byte words spent on the already included compressed bytecodes. + compressed_bytecodes_encoding: usize, + /// Initial memory of bootloader + initial_memory: BootloaderMemory, + /// Mode of txs for execution, it can be changed once per vm lunch + execution_mode: TxExecutionMode, + /// Current offset of the free space in the bootloader memory. + free_tx_offset: usize, + /// Information about the the pubdata that will be needed to supply to the L1Messenger + pubdata_information: OnceCell, +} + +impl BootloaderState { + pub(crate) fn new( + execution_mode: TxExecutionMode, + initial_memory: BootloaderMemory, + first_l2_block: L2BlockEnv, + ) -> Self { + let l2_block = BootloaderL2Block::new(first_l2_block, 0); + Self { + tx_to_execute: 0, + compressed_bytecodes_encoding: 0, + l2_blocks: vec![l2_block], + initial_memory, + execution_mode, + free_tx_offset: 0, + pubdata_information: Default::default(), + } + } + + pub(crate) fn set_refund_for_current_tx(&mut self, refund: u32) { + let current_tx = self.current_tx(); + // We can't set the refund for the latest tx or using the latest l2_block for fining tx + // Because we can fill the whole batch first and then execute txs one by one + let tx = self.find_tx_mut(current_tx); + tx.refund = refund; + } + + pub(crate) fn set_pubdata_input(&mut self, info: PubdataInput) { + self.pubdata_information + .set(info) + .expect("Pubdata information is already set"); + } + + pub(crate) fn start_new_l2_block(&mut self, l2_block: L2BlockEnv) { + let last_block = self.last_l2_block(); + assert!( + !last_block.txs.is_empty(), + "Can not create new miniblocks on top of empty ones" + ); + assert_next_block(&last_block.l2_block(), &l2_block); + self.push_l2_block(l2_block); + } + + /// This method bypass sanity checks and should be used carefully. + pub(crate) fn push_l2_block(&mut self, l2_block: L2BlockEnv) { + self.l2_blocks + .push(BootloaderL2Block::new(l2_block, self.free_tx_index())) + } + + pub(crate) fn push_tx( + &mut self, + tx: TransactionData, + predefined_overhead: u32, + predefined_refund: u32, + compressed_bytecodes: Vec, + trusted_ergs_limit: U256, + chain_id: L2ChainId, + ) -> BootloaderMemory { + let tx_offset = self.free_tx_offset(); + let bootloader_tx = BootloaderTx::new( + tx, + predefined_refund, + predefined_overhead, + trusted_ergs_limit, + compressed_bytecodes, + tx_offset, + chain_id, + ); + + let mut memory = vec![]; + let compressed_bytecode_size = apply_tx_to_memory( + &mut memory, + &bootloader_tx, + self.last_l2_block(), + self.free_tx_index(), + self.free_tx_offset(), + self.compressed_bytecodes_encoding, + self.execution_mode, + self.last_l2_block().txs.is_empty(), + ); + self.compressed_bytecodes_encoding += compressed_bytecode_size; + self.free_tx_offset = tx_offset + bootloader_tx.encoded_len(); + self.last_mut_l2_block().push_tx(bootloader_tx); + memory + } + + pub(crate) fn last_l2_block(&self) -> &BootloaderL2Block { + self.l2_blocks.last().unwrap() + } + pub(crate) fn get_pubdata_information(&self) -> &PubdataInput { + self.pubdata_information + .get() + .expect("Pubdata information is not set") + } + + fn last_mut_l2_block(&mut self) -> &mut BootloaderL2Block { + self.l2_blocks.last_mut().unwrap() + } + + /// Apply all bootloader transaction to the initial memory + pub(crate) fn bootloader_memory(&self) -> BootloaderMemory { + let mut initial_memory = self.initial_memory.clone(); + let mut offset = 0; + let mut compressed_bytecodes_offset = 0; + let mut tx_index = 0; + for l2_block in &self.l2_blocks { + for (num, tx) in l2_block.txs.iter().enumerate() { + let compressed_bytecodes_size = apply_tx_to_memory( + &mut initial_memory, + tx, + l2_block, + tx_index, + offset, + compressed_bytecodes_offset, + self.execution_mode, + num == 0, + ); + offset += tx.encoded_len(); + compressed_bytecodes_offset += compressed_bytecodes_size; + tx_index += 1; + } + if l2_block.txs.is_empty() { + apply_l2_block(&mut initial_memory, l2_block, tx_index) + } + } + + let pubdata_information = self + .pubdata_information + .clone() + .into_inner() + .expect("Empty pubdata information"); + + apply_pubdata_to_memory(&mut initial_memory, pubdata_information); + initial_memory + } + + fn free_tx_offset(&self) -> usize { + self.free_tx_offset + } + + pub(crate) fn free_tx_index(&self) -> usize { + let l2_block = self.last_l2_block(); + l2_block.first_tx_index + l2_block.txs.len() + } + + pub(crate) fn get_last_tx_compressed_bytecodes(&self) -> Vec { + if let Some(tx) = self.last_l2_block().txs.last() { + tx.compressed_bytecodes.clone() + } else { + vec![] + } + } + + /// Returns the id of current tx + pub(crate) fn current_tx(&self) -> usize { + self.tx_to_execute + .checked_sub(1) + .expect("There are no current tx to execute") + } + + /// Returns the ID of the next transaction to be executed and increments the local transaction counter. + pub(crate) fn move_tx_to_execute_pointer(&mut self) -> usize { + assert!( + self.tx_to_execute < self.free_tx_index(), + "Attempt to execute tx that was not pushed to memory. Tx ID: {}, txs in bootloader: {}", + self.tx_to_execute, + self.free_tx_index() + ); + + let old = self.tx_to_execute; + self.tx_to_execute += 1; + old + } + + /// Get offset of tx description + pub(crate) fn get_tx_description_offset(&self, tx_index: usize) -> usize { + TX_DESCRIPTION_OFFSET + self.find_tx(tx_index).offset + } + + pub(crate) fn insert_fictive_l2_block(&mut self) -> &BootloaderL2Block { + let block = self.last_l2_block(); + if !block.txs.is_empty() { + self.start_new_l2_block(L2BlockEnv { + timestamp: block.timestamp + 1, + number: block.number + 1, + prev_block_hash: block.get_hash(), + max_virtual_blocks_to_create: 1, + }); + } + self.last_l2_block() + } + + fn find_tx(&self, tx_index: usize) -> &BootloaderTx { + for block in self.l2_blocks.iter().rev() { + if tx_index >= block.first_tx_index { + return &block.txs[tx_index - block.first_tx_index]; + } + } + panic!("The tx with index {} must exist", tx_index) + } + + fn find_tx_mut(&mut self, tx_index: usize) -> &mut BootloaderTx { + for block in self.l2_blocks.iter_mut().rev() { + if tx_index >= block.first_tx_index { + return &mut block.txs[tx_index - block.first_tx_index]; + } + } + panic!("The tx with index {} must exist", tx_index) + } + + pub(crate) fn get_snapshot(&self) -> BootloaderStateSnapshot { + BootloaderStateSnapshot { + tx_to_execute: self.tx_to_execute, + l2_blocks_len: self.l2_blocks.len(), + last_l2_block: self.last_l2_block().make_snapshot(), + compressed_bytecodes_encoding: self.compressed_bytecodes_encoding, + free_tx_offset: self.free_tx_offset, + is_pubdata_information_provided: self.pubdata_information.get().is_some(), + } + } + + pub(crate) fn apply_snapshot(&mut self, snapshot: BootloaderStateSnapshot) { + self.tx_to_execute = snapshot.tx_to_execute; + self.compressed_bytecodes_encoding = snapshot.compressed_bytecodes_encoding; + self.free_tx_offset = snapshot.free_tx_offset; + match self.l2_blocks.len().cmp(&snapshot.l2_blocks_len) { + Ordering::Greater => self.l2_blocks.truncate(snapshot.l2_blocks_len), + Ordering::Less => panic!("Applying snapshot from future is not supported"), + Ordering::Equal => {} + } + self.last_mut_l2_block() + .apply_snapshot(snapshot.last_l2_block); + + if !snapshot.is_pubdata_information_provided { + self.pubdata_information = Default::default(); + } else { + // Under the correct usage of the snapshots of the bootloader state, + // this assertion should never fail, i.e. since the pubdata information + // can be set only once. However, we have this assertion just in case. + assert!( + self.pubdata_information.get().is_some(), + "Snapshot with no pubdata can not rollback to snapshot with one" + ); + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs new file mode 100644 index 000000000000..21aee75b38b5 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs @@ -0,0 +1,49 @@ +use zksync_types::{L2ChainId, H256, U256}; +use zksync_utils::bytecode::CompressedBytecodeInfo; + +use crate::vm_latest::types::internals::TransactionData; + +/// Information about tx necessary for execution in bootloader. +#[derive(Debug, Clone)] +pub(super) struct BootloaderTx { + pub(super) hash: H256, + /// Encoded transaction + pub(super) encoded: Vec, + /// Compressed bytecodes, which has been published during this transaction + pub(super) compressed_bytecodes: Vec, + /// Refunds for this transaction + pub(super) refund: u32, + /// Gas overhead + pub(super) gas_overhead: u32, + /// Gas Limit for this transaction. It can be different from the gas limit inside the transaction + pub(super) trusted_gas_limit: U256, + /// Offset of the tx in bootloader memory + pub(super) offset: usize, +} + +impl BootloaderTx { + pub(super) fn new( + tx: TransactionData, + predefined_refund: u32, + predefined_overhead: u32, + trusted_gas_limit: U256, + compressed_bytecodes: Vec, + offset: usize, + chain_id: L2ChainId, + ) -> Self { + let hash = tx.tx_hash(chain_id); + Self { + hash, + encoded: tx.into_tokens(), + compressed_bytecodes, + refund: predefined_refund, + gas_overhead: predefined_overhead, + trusted_gas_limit, + offset, + } + } + + pub(super) fn encoded_len(&self) -> usize { + self.encoded.len() + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs new file mode 100644 index 000000000000..93ec255e7740 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs @@ -0,0 +1,177 @@ +use zksync_types::{ethabi, U256}; +use zksync_utils::{bytecode::CompressedBytecodeInfo, bytes_to_be_words, h256_to_u256}; + +use super::tx::BootloaderTx; +use crate::{ + interface::{BootloaderMemory, TxExecutionMode}, + vm_latest::{ + bootloader_state::l2_block::BootloaderL2Block, + constants::{ + BOOTLOADER_TX_DESCRIPTION_OFFSET, BOOTLOADER_TX_DESCRIPTION_SIZE, + COMPRESSED_BYTECODES_OFFSET, OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET, + OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS, OPERATOR_REFUNDS_OFFSET, + TX_DESCRIPTION_OFFSET, TX_OPERATOR_L2_BLOCK_INFO_OFFSET, + TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO, TX_OVERHEAD_OFFSET, TX_TRUSTED_GAS_LIMIT_OFFSET, + }, + types::internals::PubdataInput, + }, +}; + +pub(super) fn get_memory_for_compressed_bytecodes( + compressed_bytecodes: &[CompressedBytecodeInfo], +) -> Vec { + let memory_addition: Vec<_> = compressed_bytecodes + .iter() + .flat_map(|x| x.encode_call()) + .collect(); + + bytes_to_be_words(memory_addition) +} + +#[allow(clippy::too_many_arguments)] +pub(super) fn apply_tx_to_memory( + memory: &mut BootloaderMemory, + bootloader_tx: &BootloaderTx, + bootloader_l2_block: &BootloaderL2Block, + tx_index: usize, + tx_offset: usize, + compressed_bytecodes_size: usize, + execution_mode: TxExecutionMode, + start_new_l2_block: bool, +) -> usize { + let bootloader_description_offset = + BOOTLOADER_TX_DESCRIPTION_OFFSET + BOOTLOADER_TX_DESCRIPTION_SIZE * tx_index; + let tx_description_offset = TX_DESCRIPTION_OFFSET + tx_offset; + + memory.push(( + bootloader_description_offset, + assemble_tx_meta(execution_mode, true), + )); + + memory.push(( + bootloader_description_offset + 1, + U256::from_big_endian(&(32 * tx_description_offset).to_be_bytes()), + )); + + let refund_offset = OPERATOR_REFUNDS_OFFSET + tx_index; + memory.push((refund_offset, bootloader_tx.refund.into())); + + let overhead_offset = TX_OVERHEAD_OFFSET + tx_index; + memory.push((overhead_offset, bootloader_tx.gas_overhead.into())); + + let trusted_gas_limit_offset = TX_TRUSTED_GAS_LIMIT_OFFSET + tx_index; + memory.push((trusted_gas_limit_offset, bootloader_tx.trusted_gas_limit)); + + memory.extend( + (tx_description_offset..tx_description_offset + bootloader_tx.encoded_len()) + .zip(bootloader_tx.encoded.clone()), + ); + + let bootloader_l2_block = if start_new_l2_block { + bootloader_l2_block.clone() + } else { + bootloader_l2_block.interim_version() + }; + apply_l2_block(memory, &bootloader_l2_block, tx_index); + + // Note, +1 is moving for pointer + let compressed_bytecodes_offset = COMPRESSED_BYTECODES_OFFSET + 1 + compressed_bytecodes_size; + + let encoded_compressed_bytecodes = + get_memory_for_compressed_bytecodes(&bootloader_tx.compressed_bytecodes); + let compressed_bytecodes_encoding = encoded_compressed_bytecodes.len(); + + memory.extend( + (compressed_bytecodes_offset + ..compressed_bytecodes_offset + encoded_compressed_bytecodes.len()) + .zip(encoded_compressed_bytecodes), + ); + compressed_bytecodes_encoding +} + +pub(crate) fn apply_l2_block( + memory: &mut BootloaderMemory, + bootloader_l2_block: &BootloaderL2Block, + txs_index: usize, +) { + // Since L2 block infos start from the TX_OPERATOR_L2_BLOCK_INFO_OFFSET and each + // L2 block info takes TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots, the position where the L2 block info + // for this transaction needs to be written is: + + let block_position = + TX_OPERATOR_L2_BLOCK_INFO_OFFSET + txs_index * TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO; + + memory.extend(vec![ + (block_position, bootloader_l2_block.number.into()), + (block_position + 1, bootloader_l2_block.timestamp.into()), + ( + block_position + 2, + h256_to_u256(bootloader_l2_block.prev_block_hash), + ), + ( + block_position + 3, + bootloader_l2_block.max_virtual_blocks_to_create.into(), + ), + ]) +} + +pub(crate) fn apply_pubdata_to_memory( + memory: &mut BootloaderMemory, + pubdata_information: PubdataInput, +) { + // Skipping two slots as they will be filled by the bootloader itself: + // - One slot is for the selector of the call to the L1Messenger. + // - The other slot is for the 0x20 offset for the calldata. + let l1_messenger_pubdata_start_slot = OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET + 2; + + // Need to skip first word as it represents array offset + // while bootloader expects only [len || data] + let pubdata = ethabi::encode(&[ethabi::Token::Bytes( + pubdata_information.build_pubdata(true), + )])[32..] + .to_vec(); + + assert!( + pubdata.len() / 32 <= OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS - 2, + "The encoded pubdata is too big" + ); + + pubdata + .chunks(32) + .enumerate() + .for_each(|(slot_offset, value)| { + memory.push(( + l1_messenger_pubdata_start_slot + slot_offset, + U256::from(value), + )) + }); +} + +/// Forms a word that contains meta information for the transaction execution. +/// +/// # Current layout +/// +/// - 0 byte (MSB): server-side tx execution mode +/// In the server, we may want to execute different parts of the transaction in the different context +/// For example, when checking validity, we don't want to actually execute transaction and have side effects. +/// +/// Possible values: +/// - 0x00: validate & execute (normal mode) +/// - 0x02: execute but DO NOT validate +/// +/// - 31 byte (LSB): whether to execute transaction or not (at all). +pub(super) fn assemble_tx_meta(execution_mode: TxExecutionMode, execute_tx: bool) -> U256 { + let mut output = [0u8; 32]; + + // Set 0 byte (execution mode) + output[0] = match execution_mode { + TxExecutionMode::VerifyExecute => 0x00, + TxExecutionMode::EstimateFee { .. } => 0x00, + TxExecutionMode::EthCall { .. } => 0x02, + }; + + // Set 31 byte (marker for tx execution) + output[31] = u8::from(execute_tx); + + U256::from_big_endian(&output) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs new file mode 100644 index 000000000000..44266344be61 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs @@ -0,0 +1,127 @@ +use zk_evm_1_4_0::aux_structures::MemoryPage; +pub use zk_evm_1_4_0::zkevm_opcode_defs::system_params::{ + ERGS_PER_CIRCUIT, INITIAL_STORAGE_WRITE_PUBDATA_BYTES, MAX_PUBDATA_PER_BLOCK, +}; +use zksync_system_constants::{ + L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + USED_BOOTLOADER_MEMORY_WORDS, +}; + +use crate::vm_latest::old_vm::utils::heap_page_from_base; + +/// Max cycles for a single transaction. +pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; + +/// The first 32 slots are reserved for debugging purposes +pub(crate) const DEBUG_SLOTS_OFFSET: usize = 8; +pub(crate) const DEBUG_FIRST_SLOTS: usize = 32; +/// The next 33 slots are reserved for dealing with the paymaster context (1 slot for storing length + 32 slots for storing the actual context). +pub(crate) const PAYMASTER_CONTEXT_SLOTS: usize = 32 + 1; +/// The next PAYMASTER_CONTEXT_SLOTS + 7 slots free slots are needed before each tx, so that the +/// postOp operation could be encoded correctly. +pub(crate) const MAX_POSTOP_SLOTS: usize = PAYMASTER_CONTEXT_SLOTS + 7; + +/// Slots used to store the current L2 transaction's hash and the hash recommended +/// to be used for signing the transaction's content. +const CURRENT_L2_TX_HASHES_SLOTS: usize = 2; + +/// Slots used to store the calldata for the KnownCodesStorage to mark new factory +/// dependencies as known ones. Besides the slots for the new factory dependencies themselves +/// another 4 slots are needed for: selector, marker of whether the user should pay for the pubdata, +/// the offset for the encoding of the array as well as the length of the array. +const NEW_FACTORY_DEPS_RESERVED_SLOTS: usize = MAX_NEW_FACTORY_DEPS + 4; + +/// The operator can provide for each transaction the proposed minimal refund +pub(crate) const OPERATOR_REFUNDS_SLOTS: usize = MAX_TXS_IN_BLOCK; + +pub(crate) const OPERATOR_REFUNDS_OFFSET: usize = DEBUG_SLOTS_OFFSET + + DEBUG_FIRST_SLOTS + + PAYMASTER_CONTEXT_SLOTS + + CURRENT_L2_TX_HASHES_SLOTS + + NEW_FACTORY_DEPS_RESERVED_SLOTS; + +pub(crate) const TX_OVERHEAD_OFFSET: usize = OPERATOR_REFUNDS_OFFSET + OPERATOR_REFUNDS_SLOTS; +pub(crate) const TX_OVERHEAD_SLOTS: usize = MAX_TXS_IN_BLOCK; + +pub(crate) const TX_TRUSTED_GAS_LIMIT_OFFSET: usize = TX_OVERHEAD_OFFSET + TX_OVERHEAD_SLOTS; +pub(crate) const TX_TRUSTED_GAS_LIMIT_SLOTS: usize = MAX_TXS_IN_BLOCK; + +pub(crate) const COMPRESSED_BYTECODES_SLOTS: usize = 32768; + +pub(crate) const PRIORITY_TXS_L1_DATA_OFFSET: usize = + COMPRESSED_BYTECODES_OFFSET + COMPRESSED_BYTECODES_SLOTS; +pub(crate) const PRIORITY_TXS_L1_DATA_SLOTS: usize = 2; + +pub const OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET: usize = + PRIORITY_TXS_L1_DATA_OFFSET + PRIORITY_TXS_L1_DATA_SLOTS; + +/// One of "worst case" scenarios for the number of state diffs in a batch is when 120kb of pubdata is spent +/// on repeated writes, that are all zeroed out. In this case, the number of diffs is 120k / 5 = 24k. This means that they will have +/// accommodate 6528000 bytes of calldata for the uncompressed state diffs. Adding 120k on top leaves us with +/// roughly 6650000 bytes needed for calldata. 207813 slots are needed to accommodate this amount of data. +/// We round up to 208000 slots just in case. +/// +/// In theory though much more calldata could be used (if for instance 1 byte is used for enum index). It is the responsibility of the +/// operator to ensure that it can form the correct calldata for the L1Messenger. +pub(crate) const OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS: usize = 208000; + +pub(crate) const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = + OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET + OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS; + +/// The size of the bootloader memory dedicated to the encodings of transactions +pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = + (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; + +// Size of the bootloader tx description in words +pub(crate) const BOOTLOADER_TX_DESCRIPTION_SIZE: usize = 2; + +/// The actual descriptions of transactions should start after the minor descriptions and a MAX_POSTOP_SLOTS +/// free slots to allow postOp encoding. +pub(crate) const TX_DESCRIPTION_OFFSET: usize = BOOTLOADER_TX_DESCRIPTION_OFFSET + + BOOTLOADER_TX_DESCRIPTION_SIZE * MAX_TXS_IN_BLOCK + + MAX_POSTOP_SLOTS; + +pub(crate) const TX_GAS_LIMIT_OFFSET: usize = 4; + +const INITIAL_BASE_PAGE: u32 = 8; +pub const BOOTLOADER_HEAP_PAGE: u32 = heap_page_from_base(MemoryPage(INITIAL_BASE_PAGE)).0; +pub const BLOCK_OVERHEAD_GAS: u32 = 1200000; +pub const BLOCK_OVERHEAD_L1_GAS: u32 = 1000000; +pub const BLOCK_OVERHEAD_PUBDATA: u32 = BLOCK_OVERHEAD_L1_GAS / L1_GAS_PER_PUBDATA_BYTE; + +/// VM Hooks are used for communication between bootloader and tracers. +/// The 'type' / 'opcode' is put into VM_HOOK_POSITION slot, +/// and VM_HOOKS_PARAMS_COUNT parameters (each 32 bytes) are put in the slots before. +/// So the layout looks like this: +/// `[param 0][param 1][vmhook opcode]` +pub const VM_HOOK_POSITION: u32 = RESULT_SUCCESS_FIRST_SLOT - 1; +pub const VM_HOOK_PARAMS_COUNT: u32 = 2; +pub const VM_HOOK_PARAMS_START_POSITION: u32 = VM_HOOK_POSITION - VM_HOOK_PARAMS_COUNT; + +pub(crate) const MAX_MEM_SIZE_BYTES: u32 = 16777216; // 2^24 + +/// Arbitrary space in memory closer to the end of the page +pub const RESULT_SUCCESS_FIRST_SLOT: u32 = + (MAX_MEM_SIZE_BYTES - (MAX_TXS_IN_BLOCK as u32) * 32) / 32; + +/// How many gas bootloader is allowed to spend within one block. +/// Note that this value doesn't correspond to the gas limit of any particular transaction +/// (except for the fact that, of course, gas limit for each transaction should be <= `BLOCK_GAS_LIMIT`). +pub const BLOCK_GAS_LIMIT: u32 = + zk_evm_1_4_0::zkevm_opcode_defs::system_params::VM_INITIAL_FRAME_ERGS; + +/// How many gas is allowed to spend on a single transaction in eth_call method +pub const ETH_CALL_GAS_LIMIT: u32 = MAX_L2_TX_GAS_LIMIT as u32; + +/// ID of the transaction from L1 +pub const L1_TX_TYPE: u8 = 255; + +pub(crate) const TX_OPERATOR_L2_BLOCK_INFO_OFFSET: usize = + TX_TRUSTED_GAS_LIMIT_OFFSET + TX_TRUSTED_GAS_LIMIT_SLOTS; + +pub(crate) const TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO: usize = 4; +pub(crate) const TX_OPERATOR_L2_BLOCK_INFO_SLOTS: usize = + (MAX_TXS_IN_BLOCK + 1) * TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO; + +pub(crate) const COMPRESSED_BYTECODES_OFFSET: usize = + TX_OPERATOR_L2_BLOCK_INFO_OFFSET + TX_OPERATOR_L2_BLOCK_INFO_SLOTS; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs new file mode 100644 index 000000000000..bda1803067fb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs @@ -0,0 +1,58 @@ +use itertools::Itertools; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::U256; +use zksync_utils::{ + bytecode::{compress_bytecode, hash_bytecode, CompressedBytecodeInfo}, + bytes_to_be_words, +}; + +use crate::{interface::VmInterface, vm_latest::Vm, HistoryMode}; + +impl Vm { + /// Checks the last transaction has successfully published compressed bytecodes and returns `true` if there is at least one is still unknown. + pub(crate) fn has_unpublished_bytecodes(&mut self) -> bool { + self.get_last_tx_compressed_bytecodes().iter().any(|info| { + !self + .state + .storage + .storage + .get_ptr() + .borrow_mut() + .is_bytecode_known(&hash_bytecode(&info.original)) + }) + } +} + +/// Converts bytecode to tokens and hashes it. +pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { + let bytecode_hash = hash_bytecode(&bytecode); + let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + + let bytecode_words = bytes_to_be_words(bytecode); + + (bytecode_hash, bytecode_words) +} + +pub(crate) fn compress_bytecodes( + bytecodes: &[Vec], + storage: StoragePtr, +) -> Vec { + bytecodes + .iter() + .enumerate() + .sorted_by_key(|(_idx, dep)| *dep) + .dedup_by(|x, y| x.1 == y.1) + .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .sorted_by_key(|(idx, _dep)| *idx) + .filter_map(|(_idx, dep)| { + let compressed_bytecode = compress_bytecode(dep); + + compressed_bytecode + .ok() + .map(|compressed| CompressedBytecodeInfo { + original: dep.clone(), + compressed, + }) + }) + .collect() +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs new file mode 100644 index 000000000000..a913ea3ed463 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs @@ -0,0 +1,136 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; + +use crate::{ + interface::{ + types::tracer::{TracerExecutionStatus, VmExecutionStopReason}, + VmExecutionMode, VmExecutionResultAndLogs, + }, + vm_latest::{ + old_vm::utils::{vm_may_have_ended_inner, VmExecutionResult}, + tracers::{ + dispatcher::TracerDispatcher, DefaultExecutionTracer, PubdataTracer, RefundsTracer, + }, + vm::Vm, + }, + HistoryMode, +}; + +impl Vm { + pub(crate) fn inspect_inner( + &mut self, + dispatcher: TracerDispatcher, + execution_mode: VmExecutionMode, + ) -> VmExecutionResultAndLogs { + let mut enable_refund_tracer = false; + if let VmExecutionMode::OneTx = execution_mode { + // Move the pointer to the next transaction + self.bootloader_state.move_tx_to_execute_pointer(); + enable_refund_tracer = true; + } + + let (_, result) = + self.inspect_and_collect_results(dispatcher, execution_mode, enable_refund_tracer); + result + } + + /// Execute VM with given traces until the stop reason is reached. + /// Collect the result from the default tracers. + fn inspect_and_collect_results( + &mut self, + dispatcher: TracerDispatcher, + execution_mode: VmExecutionMode, + with_refund_tracer: bool, + ) -> (VmExecutionStopReason, VmExecutionResultAndLogs) { + let refund_tracers = + with_refund_tracer.then_some(RefundsTracer::new(self.batch_env.clone())); + let mut tx_tracer: DefaultExecutionTracer = + DefaultExecutionTracer::new( + self.system_env.default_validation_computational_gas_limit, + execution_mode, + dispatcher, + self.storage.clone(), + refund_tracers, + Some(PubdataTracer::new(self.batch_env.clone(), execution_mode)), + ); + + let timestamp_initial = Timestamp(self.state.local_state.timestamp); + let cycles_initial = self.state.local_state.monotonic_cycle_counter; + let gas_remaining_before = self.gas_remaining(); + let spent_pubdata_counter_before = self.state.local_state.spent_pubdata_counter; + + let stop_reason = self.execute_with_default_tracer(&mut tx_tracer); + + let gas_remaining_after = self.gas_remaining(); + + let logs = self.collect_execution_logs_after_timestamp(timestamp_initial); + + let (refunds, pubdata_published) = tx_tracer + .refund_tracer + .as_ref() + .map(|x| (x.get_refunds(), x.pubdata_published())) + .unwrap_or_default(); + + let statistics = self.get_statistics( + timestamp_initial, + cycles_initial, + &tx_tracer, + gas_remaining_before, + gas_remaining_after, + spent_pubdata_counter_before, + pubdata_published, + logs.total_log_queries_count, + ); + let result = tx_tracer.result_tracer.into_result(); + + let result = VmExecutionResultAndLogs { + result, + logs, + statistics, + refunds, + }; + + (stop_reason, result) + } + + /// Execute vm with given tracers until the stop reason is reached. + fn execute_with_default_tracer( + &mut self, + tracer: &mut DefaultExecutionTracer, + ) -> VmExecutionStopReason { + tracer.initialize_tracer(&mut self.state); + let result = loop { + // Sanity check: we should never reach the maximum value, because then we won't be able to process the next cycle. + assert_ne!( + self.state.local_state.monotonic_cycle_counter, + u32::MAX, + "VM reached maximum possible amount of cycles. Vm state: {:?}", + self.state + ); + + self.state + .cycle(tracer) + .expect("Failed execution VM cycle."); + + if let TracerExecutionStatus::Stop(reason) = + tracer.finish_cycle(&mut self.state, &mut self.bootloader_state) + { + break VmExecutionStopReason::TracerRequestedStop(reason); + } + if self.has_ended() { + break VmExecutionStopReason::VmFinished; + } + }; + tracer.after_vm_execution(&mut self.state, &self.bootloader_state, result.clone()); + result + } + + fn has_ended(&self) -> bool { + match vm_may_have_ended_inner(&self.state) { + None | Some(VmExecutionResult::MostLikelyDidNotFinish(_, _)) => false, + Some( + VmExecutionResult::Ok(_) | VmExecutionResult::Revert(_) | VmExecutionResult::Panic, + ) => true, + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs new file mode 100644 index 000000000000..526eab76f07f --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs @@ -0,0 +1,43 @@ +use zksync_state::WriteStorage; + +use crate::{ + vm_latest::{tracers::DefaultExecutionTracer, vm::Vm}, + HistoryMode, +}; + +impl Vm { + /// Returns the amount of gas remaining to the VM. + /// Note that this *does not* correspond to the gas limit of a transaction. + /// To calculate the amount of gas spent by transaction, you should call this method before and after + /// the execution, and subtract these values. + /// + /// Note: this method should only be called when either transaction is fully completed or VM completed + /// its execution. Remaining gas value is read from the current stack frame, so if you'll attempt to + /// read it during the transaction execution, you may receive invalid value. + pub(crate) fn gas_remaining(&self) -> u32 { + self.state.local_state.callstack.current.ergs_remaining + } + + pub(crate) fn calculate_computational_gas_used( + &self, + tracer: &DefaultExecutionTracer, + gas_remaining_before: u32, + spent_pubdata_counter_before: u32, + ) -> u32 { + let total_gas_used = gas_remaining_before + .checked_sub(self.gas_remaining()) + .expect("underflow"); + let gas_used_on_pubdata = + tracer.gas_spent_on_pubdata(&self.state.local_state) - spent_pubdata_counter_before; + total_gas_used + .checked_sub(gas_used_on_pubdata) + .unwrap_or_else(|| { + tracing::error!( + "Gas used on pubdata is greater than total gas used. On pubdata: {}, total: {}", + gas_used_on_pubdata, + total_gas_used + ); + 0 + }) + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs new file mode 100644 index 000000000000..9e0817aa9394 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs @@ -0,0 +1,72 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; +use zksync_types::{ + event::extract_l2tol1logs_from_l1_messenger, + l2_to_l1_log::{L2ToL1Log, SystemL2ToL1Log, UserL2ToL1Log}, + VmEvent, +}; + +use crate::{ + interface::types::outputs::VmExecutionLogs, + vm_latest::{old_vm::utils::precompile_calls_count_after_timestamp, utils::logs, vm::Vm}, + HistoryMode, +}; + +impl Vm { + pub(crate) fn collect_execution_logs_after_timestamp( + &self, + from_timestamp: Timestamp, + ) -> VmExecutionLogs { + let storage_logs: Vec<_> = self + .state + .storage + .storage_log_queries_after_timestamp(from_timestamp) + .iter() + .map(|log| **log) + .collect(); + let storage_logs_count = storage_logs.len(); + + let (events, system_l2_to_l1_logs) = + self.collect_events_and_l1_system_logs_after_timestamp(from_timestamp); + + let log_queries = self + .state + .event_sink + .log_queries_after_timestamp(from_timestamp); + + let precompile_calls_count = precompile_calls_count_after_timestamp( + self.state.precompiles_processor.timestamp_history.inner(), + from_timestamp, + ); + + let user_logs = extract_l2tol1logs_from_l1_messenger(&events); + + let total_log_queries_count = + storage_logs_count + log_queries.len() + precompile_calls_count; + + VmExecutionLogs { + storage_logs, + events, + user_l2_to_l1_logs: user_logs + .into_iter() + .map(|log| UserL2ToL1Log(log.into())) + .collect(), + system_l2_to_l1_logs: system_l2_to_l1_logs + .into_iter() + .map(SystemL2ToL1Log) + .collect(), + total_log_queries_count, + } + } + + pub(crate) fn collect_events_and_l1_system_logs_after_timestamp( + &self, + from_timestamp: Timestamp, + ) -> (Vec, Vec) { + logs::collect_events_and_l1_system_logs_after_timestamp( + &self.state, + &self.batch_env, + from_timestamp, + ) + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/mod.rs new file mode 100644 index 000000000000..161732cf0348 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/mod.rs @@ -0,0 +1,7 @@ +mod bytecode; +mod execution; +mod gas; +mod logs; +mod snapshots; +mod statistics; +mod tx; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs new file mode 100644 index 000000000000..b6b452834388 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs @@ -0,0 +1,91 @@ +use std::time::Duration; + +use vise::{Buckets, EncodeLabelSet, EncodeLabelValue, Family, Histogram, Metrics}; +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; + +use crate::vm_latest::{ + old_vm::{history_recorder::HistoryEnabled, oracles::OracleWithHistory}, + types::internals::VmSnapshot, + vm::Vm, +}; + +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelSet, EncodeLabelValue)] +#[metrics(label = "stage", rename_all = "snake_case")] +enum RollbackStage { + DecommitmentProcessorRollback, + EventSinkRollback, + StorageRollback, + MemoryRollback, + PrecompilesProcessorRollback, + ApplyBootloaderSnapshot, +} + +#[derive(Debug, Metrics)] +#[metrics(prefix = "server_vm")] +struct VmMetrics { + #[metrics(buckets = Buckets::LATENCIES)] + rollback_time: Family>, +} + +#[vise::register] +static METRICS: vise::Global = vise::Global::new(); + +/// Implementation of VM related to rollbacks inside virtual machine +impl Vm { + pub(crate) fn make_snapshot_inner(&mut self) { + self.snapshots.push(VmSnapshot { + // Vm local state contains O(1) various parameters (registers/etc). + // The only "expensive" copying here is copying of the callstack. + // It will take O(callstack_depth) to copy it. + // So it is generally recommended to get snapshots of the bootloader frame, + // where the depth is 1. + local_state: self.state.local_state.clone(), + bootloader_state: self.bootloader_state.get_snapshot(), + }); + } + + pub(crate) fn rollback_to_snapshot(&mut self, snapshot: VmSnapshot) { + let VmSnapshot { + local_state, + bootloader_state, + } = snapshot; + + let stage_latency = + METRICS.rollback_time[&RollbackStage::DecommitmentProcessorRollback].start(); + let timestamp = Timestamp(local_state.timestamp); + tracing::trace!("Rolling back decomitter"); + self.state + .decommittment_processor + .rollback_to_timestamp(timestamp); + stage_latency.observe(); + + let stage_latency = METRICS.rollback_time[&RollbackStage::EventSinkRollback].start(); + tracing::trace!("Rolling back event_sink"); + self.state.event_sink.rollback_to_timestamp(timestamp); + stage_latency.observe(); + + let stage_latency = METRICS.rollback_time[&RollbackStage::StorageRollback].start(); + tracing::trace!("Rolling back storage"); + self.state.storage.rollback_to_timestamp(timestamp); + stage_latency.observe(); + + let stage_latency = METRICS.rollback_time[&RollbackStage::MemoryRollback].start(); + tracing::trace!("Rolling back memory"); + self.state.memory.rollback_to_timestamp(timestamp); + stage_latency.observe(); + + let stage_latency = + METRICS.rollback_time[&RollbackStage::PrecompilesProcessorRollback].start(); + tracing::trace!("Rolling back precompiles_processor"); + self.state + .precompiles_processor + .rollback_to_timestamp(timestamp); + stage_latency.observe(); + + self.state.local_state = local_state; + let stage_latency = METRICS.rollback_time[&RollbackStage::ApplyBootloaderSnapshot].start(); + self.bootloader_state.apply_snapshot(bootloader_state); + stage_latency.observe(); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs new file mode 100644 index 000000000000..6af9ad041feb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs @@ -0,0 +1,70 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; +use zksync_types::U256; + +use crate::{ + interface::{VmExecutionStatistics, VmMemoryMetrics}, + vm_latest::{tracers::DefaultExecutionTracer, vm::Vm}, + HistoryMode, +}; + +/// Module responsible for observing the VM behavior, i.e. calculating the statistics of the VM runs +/// or reporting the VM memory usage. + +impl Vm { + /// Get statistics about TX execution. + #[allow(clippy::too_many_arguments)] + pub(crate) fn get_statistics( + &self, + timestamp_initial: Timestamp, + cycles_initial: u32, + tracer: &DefaultExecutionTracer, + gas_remaining_before: u32, + gas_remaining_after: u32, + spent_pubdata_counter_before: u32, + pubdata_published: u32, + total_log_queries_count: usize, + ) -> VmExecutionStatistics { + let computational_gas_used = self.calculate_computational_gas_used( + tracer, + gas_remaining_before, + spent_pubdata_counter_before, + ); + VmExecutionStatistics { + contracts_used: self + .state + .decommittment_processor + .get_decommitted_bytecodes_after_timestamp(timestamp_initial), + cycles_used: self.state.local_state.monotonic_cycle_counter - cycles_initial, + gas_used: gas_remaining_before - gas_remaining_after, + computational_gas_used, + total_log_queries: total_log_queries_count, + pubdata_published, + } + } + + /// Returns the hashes the bytecodes that have been decommitted by the decommitment processor. + pub(crate) fn get_used_contracts(&self) -> Vec { + self.state + .decommittment_processor + .decommitted_code_hashes + .inner() + .keys() + .cloned() + .collect() + } + + /// Returns the info about all oracles' sizes. + pub(crate) fn record_vm_memory_metrics_inner(&self) -> VmMemoryMetrics { + VmMemoryMetrics { + event_sink_inner: self.state.event_sink.get_size(), + event_sink_history: self.state.event_sink.get_history_size(), + memory_inner: self.state.memory.get_size(), + memory_history: self.state.memory.get_history_size(), + decommittment_processor_inner: self.state.decommittment_processor.get_size(), + decommittment_processor_history: self.state.decommittment_processor.get_history_size(), + storage_inner: self.state.storage.get_size(), + storage_history: self.state.storage.get_history_size(), + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs new file mode 100644 index 000000000000..326be41c5ee7 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs @@ -0,0 +1,68 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; +use zksync_types::{l1::is_l1_tx_type, Transaction}; + +use crate::{ + vm_latest::{ + constants::BOOTLOADER_HEAP_PAGE, + implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, + types::internals::TransactionData, + vm::Vm, + }, + HistoryMode, +}; + +impl Vm { + pub(crate) fn push_raw_transaction( + &mut self, + tx: TransactionData, + predefined_overhead: u32, + predefined_refund: u32, + with_compression: bool, + ) { + let timestamp = Timestamp(self.state.local_state.timestamp); + let codes_for_decommiter = tx + .factory_deps + .iter() + .map(|dep| bytecode_to_factory_dep(dep.clone())) + .collect(); + + let compressed_bytecodes = if is_l1_tx_type(tx.tx_type) || !with_compression { + // L1 transactions do not need compression + vec![] + } else { + compress_bytecodes(&tx.factory_deps, self.state.storage.storage.get_ptr()) + }; + + self.state + .decommittment_processor + .populate(codes_for_decommiter, timestamp); + + let trusted_ergs_limit = + tx.trusted_ergs_limit(self.batch_env.block_gas_price_per_pubdata()); + + let memory = self.bootloader_state.push_tx( + tx, + predefined_overhead, + predefined_refund, + compressed_bytecodes, + trusted_ergs_limit, + self.system_env.chain_id, + ); + + self.state + .memory + .populate_page(BOOTLOADER_HEAP_PAGE as usize, memory, timestamp); + } + + pub(crate) fn push_transaction_with_compression( + &mut self, + tx: Transaction, + with_compression: bool, + ) { + let tx: TransactionData = tx.into(); + let block_gas_per_pubdata_byte = self.batch_env.block_gas_price_per_pubdata(); + let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); + self.push_raw_transaction(tx, overhead, 0, with_compression); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs new file mode 100644 index 000000000000..c3df28f6c31c --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs @@ -0,0 +1,36 @@ +pub use self::{ + bootloader_state::BootloaderState, + old_vm::{ + history_recorder::{ + AppDataFrameManagerWithHistory, HistoryDisabled, HistoryEnabled, HistoryMode, + }, + memory::SimpleMemory, + }, + oracles::storage::StorageOracle, + tracers::{ + dispatcher::TracerDispatcher, + traits::{ToTracerPointer, TracerPointer, VmTracer}, + }, + types::internals::ZkSyncVmState, + utils::transaction_encoding::TransactionVmExt, + vm::Vm, +}; +pub use crate::interface::types::{ + inputs::{L1BatchEnv, L2BlockEnv, SystemEnv, TxExecutionMode, VmExecutionMode}, + outputs::{ + BootloaderMemory, CurrentExecutionState, ExecutionResult, FinishedL1Batch, L2Block, + Refunds, VmExecutionLogs, VmExecutionResultAndLogs, VmExecutionStatistics, VmMemoryMetrics, + }, +}; + +mod bootloader_state; +pub mod constants; +mod implementation; +mod old_vm; +mod oracles; +#[cfg(test)] +mod tests; +pub(crate) mod tracers; +mod types; +pub mod utils; +mod vm; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs new file mode 100644 index 000000000000..8e7f4d447b4d --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs @@ -0,0 +1,263 @@ +use std::collections::HashMap; + +use itertools::Itertools; +use zk_evm_1_4_0::{ + abstractions::EventSink, + aux_structures::{LogQuery, Timestamp}, + reference_impls::event_sink::EventMessage, + zkevm_opcode_defs::system_params::{ + BOOTLOADER_FORMAL_ADDRESS, EVENT_AUX_BYTE, L1_MESSAGE_AUX_BYTE, + }, +}; +use zksync_types::U256; + +use crate::vm_latest::old_vm::{ + history_recorder::{AppDataFrameManagerWithHistory, HistoryEnabled, HistoryMode}, + oracles::OracleWithHistory, +}; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct InMemoryEventSink { + frames_stack: AppDataFrameManagerWithHistory, H>, +} + +impl OracleWithHistory for InMemoryEventSink { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.frames_stack.rollback_to_timestamp(timestamp); + } +} + +// as usual, if we rollback the current frame then we apply changes to storage immediately, +// otherwise we carry rollbacks to the parent's frames + +impl InMemoryEventSink { + pub fn flatten(&self) -> (Vec, Vec, Vec) { + assert_eq!( + self.frames_stack.len(), + 1, + "there must exist an initial keeper frame" + ); + // we forget rollbacks as we have finished the execution and can just apply them + let history = self.frames_stack.forward().current_frame(); + + let (events, l1_messages) = Self::events_and_l1_messages_from_history(history); + let events_logs = Self::events_logs_from_history(history); + + (events_logs, events, l1_messages) + } + + pub fn get_log_queries(&self) -> usize { + self.frames_stack.forward().current_frame().len() + } + + /// Returns the log queries in the current frame where `log_query.timestamp >= from_timestamp`. + pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { + let events = self.frames_stack.forward().current_frame(); + + // Select all of the last elements where e.timestamp >= from_timestamp. + // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. + events + .rsplit(|e| e.timestamp < from_timestamp) + .next() + .unwrap_or(&[]) + } + + pub fn get_events_and_l2_l1_logs_after_timestamp( + &self, + from_timestamp: Timestamp, + ) -> (Vec, Vec) { + Self::events_and_l1_messages_from_history(self.log_queries_after_timestamp(from_timestamp)) + } + + fn events_logs_from_history(history: &[Box]) -> Vec { + // Filter out all the L2->L1 logs and leave only events + let mut events = history + .iter() + .filter_map(|log_query| (log_query.aux_byte == EVENT_AUX_BYTE).then_some(**log_query)) + .collect_vec(); + + // Sort the events by timestamp and rollback flag, basically ensuring that + // if an event has been rolled back, the original event and its rollback will be put together + events.sort_by_key(|log| (log.timestamp, log.rollback)); + + let mut stack = Vec::::new(); + let mut net_history = vec![]; + for el in events.iter() { + assert_eq!(el.shard_id, 0, "only rollup shard is supported"); + if stack.is_empty() { + assert!(!el.rollback); + stack.push(*el); + } else { + // we can always pop as it's either one to add to queue, or discard + let previous = stack.pop().unwrap(); + if previous.timestamp == el.timestamp { + // Only rollback can have the same timestamp, so here we do nothing and simply + // double check the invariants + assert!(!previous.rollback); + assert!(el.rollback); + assert!(previous.rw_flag); + assert!(el.rw_flag); + assert_eq!(previous.tx_number_in_block, el.tx_number_in_block); + assert_eq!(previous.shard_id, el.shard_id); + assert_eq!(previous.address, el.address); + assert_eq!(previous.key, el.key); + assert_eq!(previous.written_value, el.written_value); + assert_eq!(previous.is_service, el.is_service); + continue; + } else { + // The event on the stack has not been rolled back. It must be a different event, + // with a different timestamp. + assert!(!el.rollback); + stack.push(*el); + + // cleanup some fields + // flags are conventions + let sorted_log_query = LogQuery { + timestamp: Timestamp(0), + tx_number_in_block: previous.tx_number_in_block, + aux_byte: 0, + shard_id: previous.shard_id, + address: previous.address, + key: previous.key, + read_value: U256::zero(), + written_value: previous.written_value, + rw_flag: false, + rollback: false, + is_service: previous.is_service, + }; + + net_history.push(sorted_log_query); + } + } + } + + // In case the stack is non-empty, then the last element of it has not been rolled back. + if let Some(previous) = stack.pop() { + // cleanup some fields + // flags are conventions + let sorted_log_query = LogQuery { + timestamp: Timestamp(0), + tx_number_in_block: previous.tx_number_in_block, + aux_byte: 0, + shard_id: previous.shard_id, + address: previous.address, + key: previous.key, + read_value: U256::zero(), + written_value: previous.written_value, + rw_flag: false, + rollback: false, + is_service: previous.is_service, + }; + + net_history.push(sorted_log_query); + } + + net_history + } + + fn events_and_l1_messages_from_history( + history: &[Box], + ) -> (Vec, Vec) { + let mut tmp = HashMap::::with_capacity(history.len()); + + // note that we only use "forward" part and discard the rollbacks at the end, + // since if rollbacks of parents were not appended anywhere we just still keep them + for el in history { + // we are time ordered here in terms of rollbacks + if tmp.get(&el.timestamp.0).is_some() { + assert!(el.rollback); + tmp.remove(&el.timestamp.0); + } else { + assert!(!el.rollback); + tmp.insert(el.timestamp.0, **el); + } + } + + // naturally sorted by timestamp + let mut keys: Vec<_> = tmp.keys().cloned().collect(); + keys.sort_unstable(); + + let mut events = vec![]; + let mut l1_messages = vec![]; + + for k in keys.into_iter() { + let el = tmp.remove(&k).unwrap(); + let LogQuery { + shard_id, + is_service, + tx_number_in_block, + address, + key, + written_value, + aux_byte, + .. + } = el; + + let event = EventMessage { + shard_id, + is_first: is_service, + tx_number_in_block, + address, + key, + value: written_value, + }; + + if aux_byte == EVENT_AUX_BYTE { + events.push(event); + } else { + l1_messages.push(event); + } + } + + (events, l1_messages) + } + + pub(crate) fn get_size(&self) -> usize { + self.frames_stack.get_size() + } + + pub fn get_history_size(&self) -> usize { + self.frames_stack.get_history_size() + } + + pub fn delete_history(&mut self) { + self.frames_stack.delete_history(); + } +} + +impl EventSink for InMemoryEventSink { + // when we enter a new frame we should remember all our current applications and rollbacks + // when we exit the current frame then if we did panic we should concatenate all current + // forward and rollback cases + + fn add_partial_query(&mut self, _monotonic_cycle_counter: u32, mut query: LogQuery) { + assert!(query.rw_flag); + assert!(query.aux_byte == EVENT_AUX_BYTE || query.aux_byte == L1_MESSAGE_AUX_BYTE); + assert!(!query.rollback); + + // just append to rollbacks and a full history + + self.frames_stack + .push_forward(Box::new(query), query.timestamp); + // we do not need it explicitly here, but let's be consistent with circuit counterpart + query.rollback = true; + self.frames_stack + .push_rollback(Box::new(query), query.timestamp); + } + + fn start_frame(&mut self, timestamp: Timestamp) { + self.frames_stack.push_frame(timestamp) + } + + fn finish_frame(&mut self, panicked: bool, timestamp: Timestamp) { + // if we panic then we append forward and rollbacks to the forward of parent, + // otherwise we place rollbacks of child before rollbacks of the parent + if panicked { + self.frames_stack.move_rollback_to_forward( + |q| q.address != *BOOTLOADER_FORMAL_ADDRESS || q.aux_byte != EVENT_AUX_BYTE, + timestamp, + ); + } + self.frames_stack.merge_frame(timestamp); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/events.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/events.rs new file mode 100644 index 000000000000..eed8fee4ac86 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/events.rs @@ -0,0 +1,146 @@ +use zk_evm_1_4_0::{ethereum_types::Address, reference_impls::event_sink::EventMessage}; +use zksync_types::{L1BatchNumber, VmEvent, EVENT_WRITER_ADDRESS, H256}; +use zksync_utils::{be_chunks_to_h256_words, h256_to_account_address}; + +#[derive(Clone)] +pub(crate) struct SolidityLikeEvent { + pub(crate) shard_id: u8, + pub(crate) tx_number_in_block: u16, + pub(crate) address: Address, + pub(crate) topics: Vec<[u8; 32]>, + pub(crate) data: Vec, +} + +impl SolidityLikeEvent { + pub(crate) fn into_vm_event(self, block_number: L1BatchNumber) -> VmEvent { + VmEvent { + location: (block_number, self.tx_number_in_block as u32), + address: self.address, + indexed_topics: be_chunks_to_h256_words(self.topics), + value: self.data, + } + } +} + +fn merge_events_inner(events: Vec) -> Vec { + let mut result = vec![]; + let mut current: Option<(usize, u32, SolidityLikeEvent)> = None; + + for message in events.into_iter() { + if !message.is_first { + let EventMessage { + shard_id, + is_first: _, + tx_number_in_block, + address, + key, + value, + } = message; + + if let Some((mut remaining_data_length, mut remaining_topics, mut event)) = + current.take() + { + if event.address != address + || event.shard_id != shard_id + || event.tx_number_in_block != tx_number_in_block + { + continue; + } + let mut data_0 = [0u8; 32]; + let mut data_1 = [0u8; 32]; + key.to_big_endian(&mut data_0); + value.to_big_endian(&mut data_1); + for el in [data_0, data_1].iter() { + if remaining_topics != 0 { + event.topics.push(*el); + remaining_topics -= 1; + } else if remaining_data_length != 0 { + if remaining_data_length >= 32 { + event.data.extend_from_slice(el); + remaining_data_length -= 32; + } else { + event.data.extend_from_slice(&el[..remaining_data_length]); + remaining_data_length = 0; + } + } + } + + if remaining_data_length != 0 || remaining_topics != 0 { + current = Some((remaining_data_length, remaining_topics, event)) + } else { + result.push(event); + } + } + } else { + // start new one. First take the old one only if it's well formed + if let Some((remaining_data_length, remaining_topics, event)) = current.take() { + if remaining_data_length == 0 && remaining_topics == 0 { + result.push(event); + } + } + + let EventMessage { + shard_id, + is_first: _, + tx_number_in_block, + address, + key, + value, + } = message; + // split key as our internal marker. Ignore higher bits + let mut num_topics = key.0[0] as u32; + let mut data_length = (key.0[0] >> 32) as usize; + let mut buffer = [0u8; 32]; + value.to_big_endian(&mut buffer); + + let (topics, data) = if num_topics == 0 && data_length == 0 { + (vec![], vec![]) + } else if num_topics == 0 { + data_length -= 32; + (vec![], buffer.to_vec()) + } else { + num_topics -= 1; + (vec![buffer], vec![]) + }; + + let new_event = SolidityLikeEvent { + shard_id, + tx_number_in_block, + address, + topics, + data, + }; + + current = Some((data_length, num_topics, new_event)) + } + } + + // add the last one + if let Some((remaining_data_length, remaining_topics, event)) = current.take() { + if remaining_data_length == 0 && remaining_topics == 0 { + result.push(event); + } + } + + result +} + +pub(crate) fn merge_events(events: Vec) -> Vec { + let raw_events = merge_events_inner(events); + + raw_events + .into_iter() + .filter(|e| e.address == EVENT_WRITER_ADDRESS) + .map(|event| { + // The events writer events where the first topic is the actual address of the event and the rest of the topics are real topics + let address = h256_to_account_address(&H256(event.topics[0])); + let topics = event.topics.into_iter().skip(1).collect(); + + SolidityLikeEvent { + topics, + address, + ..event + } + }) + .collect() +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs new file mode 100644 index 000000000000..83a22f35b4a1 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs @@ -0,0 +1,811 @@ +use std::{collections::HashMap, fmt::Debug, hash::Hash}; + +use zk_evm_1_4_0::{ + aux_structures::Timestamp, + vm_state::PrimitiveValue, + zkevm_opcode_defs::{self}, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::{StorageKey, U256}; +use zksync_utils::{h256_to_u256, u256_to_h256}; + +pub(crate) type MemoryWithHistory = HistoryRecorder; +pub(crate) type IntFrameManagerWithHistory = HistoryRecorder, H>; + +// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// can be used. This can sometimes violate monotonicity of the timestamp within the +// same cycle, so it should be normalized. +#[inline] +fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { + let timestamp = timestamp.0; + + // Making sure it is divisible by TIME_DELTA_PER_CYCLE + Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) +} + +/// Accepts history item as its parameter and applies it. +pub trait WithHistory { + type HistoryRecord; + type ReturnValue; + + // Applies an action and returns the action that would + // rollback its effect as well as some returned value + fn apply_historic_record( + &mut self, + item: Self::HistoryRecord, + ) -> (Self::HistoryRecord, Self::ReturnValue); +} + +type EventList = Vec<(Timestamp, ::HistoryRecord)>; + +/// Controls if rolling back is possible or not. +/// Either [HistoryEnabled] or [HistoryDisabled]. +pub trait HistoryMode: private::Sealed + Debug + Clone + Default { + type History: Default; + + fn clone_history(history: &Self::History) -> Self::History + where + T::HistoryRecord: Clone; + fn mutate_history)>( + recorder: &mut HistoryRecorder, + f: F, + ); + fn borrow_history) -> R, R>( + recorder: &HistoryRecorder, + f: F, + default: R, + ) -> R; +} + +mod private { + pub trait Sealed {} + impl Sealed for super::HistoryEnabled {} + impl Sealed for super::HistoryDisabled {} +} + +// derives require that all type parameters implement the trait, which is why +// HistoryEnabled/Disabled derive so many traits even though they mostly don't +// exist at runtime. + +/// A data structure with this parameter can be rolled back. +/// See also: [HistoryDisabled] +#[derive(Debug, Clone, Default, PartialEq)] +pub struct HistoryEnabled; + +/// A data structure with this parameter cannot be rolled back. +/// It won't even have rollback methods. +/// See also: [HistoryEnabled] +#[derive(Debug, Clone, Default)] +pub struct HistoryDisabled; + +impl HistoryMode for HistoryEnabled { + type History = EventList; + + fn clone_history(history: &Self::History) -> Self::History + where + T::HistoryRecord: Clone, + { + history.clone() + } + fn mutate_history)>( + recorder: &mut HistoryRecorder, + f: F, + ) { + f(&mut recorder.inner, &mut recorder.history) + } + fn borrow_history) -> R, R>( + recorder: &HistoryRecorder, + f: F, + _: R, + ) -> R { + f(&recorder.history) + } +} + +impl HistoryMode for HistoryDisabled { + type History = (); + + fn clone_history(_: &Self::History) -> Self::History {} + fn mutate_history)>( + _: &mut HistoryRecorder, + _: F, + ) { + } + fn borrow_history) -> R, R>( + _: &HistoryRecorder, + _: F, + default: R, + ) -> R { + default + } +} + +/// A struct responsible for tracking history for +/// a component that is passed as a generic parameter to it (`inner`). +#[derive(Default)] +pub struct HistoryRecorder { + inner: T, + history: H::History, +} + +impl PartialEq for HistoryRecorder +where + T::HistoryRecord: PartialEq, +{ + fn eq(&self, other: &Self) -> bool { + self.inner == other.inner + && self.borrow_history(|h1| other.borrow_history(|h2| h1 == h2, true), true) + } +} + +impl Debug for HistoryRecorder +where + T::HistoryRecord: Debug, +{ + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let mut debug_struct = f.debug_struct("HistoryRecorder"); + debug_struct.field("inner", &self.inner); + self.borrow_history( + |h| { + debug_struct.field("history", h); + }, + (), + ); + debug_struct.finish() + } +} + +impl Clone for HistoryRecorder +where + T::HistoryRecord: Clone, + H: HistoryMode, +{ + fn clone(&self) -> Self { + Self { + inner: self.inner.clone(), + history: H::clone_history(&self.history), + } + } +} + +impl HistoryRecorder { + pub fn from_inner(inner: T) -> Self { + Self { + inner, + history: Default::default(), + } + } + + pub fn inner(&self) -> &T { + &self.inner + } + + /// If history exists, modify it using `f`. + pub fn mutate_history)>(&mut self, f: F) { + H::mutate_history(self, f); + } + + /// If history exists, feed it into `f`. Otherwise return `default`. + pub fn borrow_history) -> R, R>(&self, f: F, default: R) -> R { + H::borrow_history(self, f, default) + } + + pub fn apply_historic_record( + &mut self, + item: T::HistoryRecord, + timestamp: Timestamp, + ) -> T::ReturnValue { + let (reversed_item, return_value) = self.inner.apply_historic_record(item); + + self.mutate_history(|_, history| { + let last_recorded_timestamp = history.last().map(|(t, _)| *t).unwrap_or(Timestamp(0)); + let timestamp = normalize_timestamp(timestamp); + assert!( + last_recorded_timestamp <= timestamp, + "Timestamps are not monotonic" + ); + history.push((timestamp, reversed_item)); + }); + + return_value + } + + /// Deletes all the history for its component, making + /// its current state irreversible + pub fn delete_history(&mut self) { + self.mutate_history(|_, h| h.clear()) + } +} + +impl HistoryRecorder { + pub fn history(&self) -> &Vec<(Timestamp, T::HistoryRecord)> { + &self.history + } + + pub(crate) fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + loop { + let should_undo = self + .history + .last() + .map(|(item_timestamp, _)| *item_timestamp >= timestamp) + .unwrap_or(false); + if !should_undo { + break; + } + + let (_, item_to_apply) = self.history.pop().unwrap(); + self.inner.apply_historic_record(item_to_apply); + } + } +} + +#[derive(Debug, Clone, PartialEq)] +pub enum VectorHistoryEvent { + Push(X), + Pop, +} + +impl WithHistory for Vec { + type HistoryRecord = VectorHistoryEvent; + type ReturnValue = Option; + fn apply_historic_record( + &mut self, + item: VectorHistoryEvent, + ) -> (Self::HistoryRecord, Self::ReturnValue) { + match item { + VectorHistoryEvent::Pop => { + // Note, that here we assume that the users + // will check themselves whether this vector is empty + // prior to popping from it. + let poped_item = self.pop().unwrap(); + + (VectorHistoryEvent::Push(poped_item), Some(poped_item)) + } + VectorHistoryEvent::Push(x) => { + self.push(x); + + (VectorHistoryEvent::Pop, None) + } + } + } +} + +impl HistoryRecorder, H> { + pub fn push(&mut self, elem: T, timestamp: Timestamp) { + self.apply_historic_record(VectorHistoryEvent::Push(elem), timestamp); + } + + pub fn pop(&mut self, timestamp: Timestamp) -> T { + self.apply_historic_record(VectorHistoryEvent::Pop, timestamp) + .unwrap() + } + + pub fn len(&self) -> usize { + self.inner.len() + } + + pub fn is_empty(&self) -> bool { + self.len() == 0 + } +} + +#[derive(Debug, Clone, PartialEq)] +pub struct HashMapHistoryEvent { + pub key: K, + pub value: Option, +} + +impl WithHistory for HashMap { + type HistoryRecord = HashMapHistoryEvent; + type ReturnValue = Option; + fn apply_historic_record( + &mut self, + item: Self::HistoryRecord, + ) -> (Self::HistoryRecord, Self::ReturnValue) { + let HashMapHistoryEvent { key, value } = item; + + let prev_value = match value { + Some(x) => self.insert(key, x), + None => self.remove(&key), + }; + + ( + HashMapHistoryEvent { + key, + value: prev_value.clone(), + }, + prev_value, + ) + } +} + +impl HistoryRecorder, H> { + pub fn insert(&mut self, key: K, value: V, timestamp: Timestamp) -> Option { + self.apply_historic_record( + HashMapHistoryEvent { + key, + value: Some(value), + }, + timestamp, + ) + } + + pub(crate) fn remove(&mut self, key: K, timestamp: Timestamp) -> Option { + self.apply_historic_record(HashMapHistoryEvent { key, value: None }, timestamp) + } +} + +/// A stack of stacks. The inner stacks are called frames. +/// +/// Does not support popping from the outer stack. Instead, the outer stack can +/// push its topmost frame's contents onto the previous frame. +#[derive(Debug, Clone, PartialEq)] +pub struct FramedStack { + data: Vec, + frame_start_indices: Vec, +} + +impl Default for FramedStack { + fn default() -> Self { + // We typically require at least the first frame to be there + // since the last user-provided frame might be reverted + Self { + data: vec![], + frame_start_indices: vec![0], + } + } +} + +#[derive(Debug, Clone, PartialEq)] +pub enum FramedStackEvent { + Push(T), + Pop, + PushFrame(usize), + MergeFrame, +} + +impl WithHistory for FramedStack { + type HistoryRecord = FramedStackEvent; + type ReturnValue = (); + + fn apply_historic_record( + &mut self, + item: Self::HistoryRecord, + ) -> (Self::HistoryRecord, Self::ReturnValue) { + use FramedStackEvent::*; + match item { + Push(x) => { + self.data.push(x); + (Pop, ()) + } + Pop => { + let x = self.data.pop().unwrap(); + (Push(x), ()) + } + PushFrame(i) => { + self.frame_start_indices.push(i); + (MergeFrame, ()) + } + MergeFrame => { + let pos = self.frame_start_indices.pop().unwrap(); + (PushFrame(pos), ()) + } + } + } +} + +impl FramedStack { + fn push_frame(&self) -> FramedStackEvent { + FramedStackEvent::PushFrame(self.data.len()) + } + + pub fn current_frame(&self) -> &[T] { + &self.data[*self.frame_start_indices.last().unwrap()..self.data.len()] + } + + fn len(&self) -> usize { + self.frame_start_indices.len() + } + + /// Returns the amount of memory taken up by the stored items + pub fn get_size(&self) -> usize { + self.data.len() * std::mem::size_of::() + } +} + +impl HistoryRecorder, H> { + pub fn push_to_frame(&mut self, x: T, timestamp: Timestamp) { + self.apply_historic_record(FramedStackEvent::Push(x), timestamp); + } + pub fn clear_frame(&mut self, timestamp: Timestamp) { + let start = *self.inner.frame_start_indices.last().unwrap(); + while self.inner.data.len() > start { + self.apply_historic_record(FramedStackEvent::Pop, timestamp); + } + } + pub fn extend_frame(&mut self, items: impl IntoIterator, timestamp: Timestamp) { + for x in items { + self.push_to_frame(x, timestamp); + } + } + pub fn push_frame(&mut self, timestamp: Timestamp) { + self.apply_historic_record(self.inner.push_frame(), timestamp); + } + pub fn merge_frame(&mut self, timestamp: Timestamp) { + self.apply_historic_record(FramedStackEvent::MergeFrame, timestamp); + } +} + +#[derive(Debug, Clone, PartialEq)] +pub struct AppDataFrameManagerWithHistory { + forward: HistoryRecorder, H>, + rollback: HistoryRecorder, H>, +} + +impl Default for AppDataFrameManagerWithHistory { + fn default() -> Self { + Self { + forward: Default::default(), + rollback: Default::default(), + } + } +} + +impl AppDataFrameManagerWithHistory { + pub(crate) fn delete_history(&mut self) { + self.forward.delete_history(); + self.rollback.delete_history(); + } + + pub(crate) fn push_forward(&mut self, item: T, timestamp: Timestamp) { + self.forward.push_to_frame(item, timestamp); + } + pub(crate) fn push_rollback(&mut self, item: T, timestamp: Timestamp) { + self.rollback.push_to_frame(item, timestamp); + } + pub(crate) fn push_frame(&mut self, timestamp: Timestamp) { + self.forward.push_frame(timestamp); + self.rollback.push_frame(timestamp); + } + pub(crate) fn merge_frame(&mut self, timestamp: Timestamp) { + self.forward.merge_frame(timestamp); + self.rollback.merge_frame(timestamp); + } + + pub(crate) fn len(&self) -> usize { + self.forward.inner.len() + } + pub(crate) fn forward(&self) -> &FramedStack { + &self.forward.inner + } + pub(crate) fn rollback(&self) -> &FramedStack { + &self.rollback.inner + } + + /// Returns the amount of memory taken up by the stored items + pub(crate) fn get_size(&self) -> usize { + self.forward().get_size() + self.rollback().get_size() + } + + pub(crate) fn get_history_size(&self) -> usize { + (self.forward.borrow_history(|h| h.len(), 0) + self.rollback.borrow_history(|h| h.len(), 0)) + * std::mem::size_of::< as WithHistory>::HistoryRecord>() + } +} + +impl AppDataFrameManagerWithHistory { + pub(crate) fn move_rollback_to_forward bool>( + &mut self, + filter: F, + timestamp: Timestamp, + ) { + for x in self.rollback.inner.current_frame().iter().rev() { + if filter(x) { + self.forward.push_to_frame(x.clone(), timestamp); + } + } + self.rollback.clear_frame(timestamp); + } +} + +impl AppDataFrameManagerWithHistory { + pub(crate) fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.forward.rollback_to_timestamp(timestamp); + self.rollback.rollback_to_timestamp(timestamp); + } +} + +const PRIMITIVE_VALUE_EMPTY: PrimitiveValue = PrimitiveValue::empty(); +const PAGE_SUBDIVISION_LEN: usize = 64; + +#[derive(Debug, Default, Clone)] +struct MemoryPage { + root: Vec>>, +} + +impl MemoryPage { + fn get(&self, slot: usize) -> &PrimitiveValue { + self.root + .get(slot / PAGE_SUBDIVISION_LEN) + .and_then(|inner| inner.as_ref()) + .map(|leaf| &leaf[slot % PAGE_SUBDIVISION_LEN]) + .unwrap_or(&PRIMITIVE_VALUE_EMPTY) + } + fn set(&mut self, slot: usize, value: PrimitiveValue) -> PrimitiveValue { + let root_index = slot / PAGE_SUBDIVISION_LEN; + let leaf_index = slot % PAGE_SUBDIVISION_LEN; + + if self.root.len() <= root_index { + self.root.resize_with(root_index + 1, || None); + } + let node = &mut self.root[root_index]; + + if let Some(leaf) = node { + let old = leaf[leaf_index]; + leaf[leaf_index] = value; + old + } else { + let mut leaf = [PrimitiveValue::empty(); PAGE_SUBDIVISION_LEN]; + leaf[leaf_index] = value; + self.root[root_index] = Some(Box::new(leaf)); + PrimitiveValue::empty() + } + } + + fn get_size(&self) -> usize { + self.root.iter().filter_map(|x| x.as_ref()).count() + * PAGE_SUBDIVISION_LEN + * std::mem::size_of::() + } +} + +impl PartialEq for MemoryPage { + fn eq(&self, other: &Self) -> bool { + for slot in 0..self.root.len().max(other.root.len()) * PAGE_SUBDIVISION_LEN { + if self.get(slot) != other.get(slot) { + return false; + } + } + true + } +} + +#[derive(Debug, Default, Clone)] +pub struct MemoryWrapper { + memory: Vec, +} + +impl PartialEq for MemoryWrapper { + fn eq(&self, other: &Self) -> bool { + let empty_page = MemoryPage::default(); + let empty_pages = std::iter::repeat(&empty_page); + self.memory + .iter() + .chain(empty_pages.clone()) + .zip(other.memory.iter().chain(empty_pages)) + .take(self.memory.len().max(other.memory.len())) + .all(|(a, b)| a == b) + } +} + +#[derive(Debug, Clone, PartialEq)] +pub struct MemoryHistoryRecord { + pub page: usize, + pub slot: usize, + pub set_value: PrimitiveValue, +} + +impl MemoryWrapper { + pub fn ensure_page_exists(&mut self, page: usize) { + if self.memory.len() <= page { + // We don't need to record such events in history + // because all these vectors will be empty + self.memory.resize_with(page + 1, MemoryPage::default); + } + } + + pub fn dump_page_content_as_u256_words( + &self, + page_number: u32, + range: std::ops::Range, + ) -> Vec { + if let Some(page) = self.memory.get(page_number as usize) { + let mut result = vec![]; + for i in range { + result.push(*page.get(i as usize)); + } + result + } else { + vec![PrimitiveValue::empty(); range.len()] + } + } + + pub fn read_slot(&self, page: usize, slot: usize) -> &PrimitiveValue { + self.memory + .get(page) + .map(|page| page.get(slot)) + .unwrap_or(&PRIMITIVE_VALUE_EMPTY) + } + + pub fn get_size(&self) -> usize { + self.memory.iter().map(|page| page.get_size()).sum() + } +} + +impl WithHistory for MemoryWrapper { + type HistoryRecord = MemoryHistoryRecord; + type ReturnValue = PrimitiveValue; + + fn apply_historic_record( + &mut self, + item: MemoryHistoryRecord, + ) -> (Self::HistoryRecord, Self::ReturnValue) { + let MemoryHistoryRecord { + page, + slot, + set_value, + } = item; + + self.ensure_page_exists(page); + let page_handle = self.memory.get_mut(page).unwrap(); + let prev_value = page_handle.set(slot, set_value); + + let undo = MemoryHistoryRecord { + page, + slot, + set_value: prev_value, + }; + + (undo, prev_value) + } +} + +impl HistoryRecorder { + pub fn write_to_memory( + &mut self, + page: usize, + slot: usize, + value: PrimitiveValue, + timestamp: Timestamp, + ) -> PrimitiveValue { + self.apply_historic_record( + MemoryHistoryRecord { + page, + slot, + set_value: value, + }, + timestamp, + ) + } + + pub fn clear_page(&mut self, page: usize, timestamp: Timestamp) { + self.mutate_history(|inner, history| { + if let Some(page_handle) = inner.memory.get(page) { + for (i, x) in page_handle.root.iter().enumerate() { + if let Some(slots) = x { + for (j, value) in slots.iter().enumerate() { + if *value != PrimitiveValue::empty() { + history.push(( + timestamp, + MemoryHistoryRecord { + page, + slot: PAGE_SUBDIVISION_LEN * i + j, + set_value: *value, + }, + )) + } + } + } + } + inner.memory[page] = MemoryPage::default(); + } + }); + } +} + +#[derive(Debug)] +pub struct StorageWrapper { + storage_ptr: StoragePtr, +} + +impl StorageWrapper { + pub fn new(storage_ptr: StoragePtr) -> Self { + Self { storage_ptr } + } + + pub fn get_ptr(&self) -> StoragePtr { + self.storage_ptr.clone() + } + + pub fn read_from_storage(&self, key: &StorageKey) -> U256 { + h256_to_u256(self.storage_ptr.borrow_mut().read_value(key)) + } +} + +#[derive(Debug, Clone)] +pub struct StorageHistoryRecord { + pub key: StorageKey, + pub value: U256, +} + +impl WithHistory for StorageWrapper { + type HistoryRecord = StorageHistoryRecord; + type ReturnValue = U256; + + fn apply_historic_record( + &mut self, + item: Self::HistoryRecord, + ) -> (Self::HistoryRecord, Self::ReturnValue) { + let prev_value = h256_to_u256( + self.storage_ptr + .borrow_mut() + .set_value(item.key, u256_to_h256(item.value)), + ); + + let reverse_item = StorageHistoryRecord { + key: item.key, + value: prev_value, + }; + + (reverse_item, prev_value) + } +} + +impl HistoryRecorder, H> { + pub fn read_from_storage(&self, key: &StorageKey) -> U256 { + self.inner.read_from_storage(key) + } + + pub fn write_to_storage(&mut self, key: StorageKey, value: U256, timestamp: Timestamp) -> U256 { + self.apply_historic_record(StorageHistoryRecord { key, value }, timestamp) + } + + /// Returns a pointer to the storage. + /// Note, that any changes done to the storage via this pointer + /// will NOT be recorded as its history. + pub fn get_ptr(&self) -> StoragePtr { + self.inner.get_ptr() + } +} + +#[cfg(test)] +mod tests { + use zk_evm_1_4_0::{aux_structures::Timestamp, vm_state::PrimitiveValue}; + use zksync_types::U256; + + use crate::vm_latest::{ + old_vm::history_recorder::{HistoryRecorder, MemoryWrapper}, + HistoryDisabled, + }; + + #[test] + fn memory_equality() { + let mut a: HistoryRecorder = Default::default(); + let mut b = a.clone(); + let nonzero = U256::from_dec_str("123").unwrap(); + let different_value = U256::from_dec_str("1234").unwrap(); + + let write = |memory: &mut HistoryRecorder, value| { + memory.write_to_memory( + 17, + 34, + PrimitiveValue { + value, + is_pointer: false, + }, + Timestamp::empty(), + ); + }; + + assert_eq!(a, b); + + write(&mut b, nonzero); + assert_ne!(a, b); + + write(&mut a, different_value); + assert_ne!(a, b); + + write(&mut a, nonzero); + assert_eq!(a, b); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs new file mode 100644 index 000000000000..5a7592ce9654 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs @@ -0,0 +1,327 @@ +use zk_evm_1_4_0::{ + abstractions::{Memory, MemoryType}, + aux_structures::{MemoryPage, MemoryQuery, Timestamp}, + vm_state::PrimitiveValue, + zkevm_opcode_defs::FatPointer, +}; +use zksync_types::U256; + +use crate::vm_latest::old_vm::{ + history_recorder::{ + FramedStack, HistoryEnabled, HistoryMode, IntFrameManagerWithHistory, MemoryWithHistory, + MemoryWrapper, WithHistory, + }, + oracles::OracleWithHistory, + utils::{aux_heap_page_from_base, heap_page_from_base, stack_page_from_base}, +}; + +#[derive(Debug, Clone, PartialEq)] +pub struct SimpleMemory { + memory: MemoryWithHistory, + observable_pages: IntFrameManagerWithHistory, +} + +impl Default for SimpleMemory { + fn default() -> Self { + let mut memory: MemoryWithHistory = Default::default(); + memory.mutate_history(|_, h| h.reserve(607)); + Self { + memory, + observable_pages: Default::default(), + } + } +} + +impl OracleWithHistory for SimpleMemory { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.memory.rollback_to_timestamp(timestamp); + self.observable_pages.rollback_to_timestamp(timestamp); + } +} + +impl SimpleMemory { + pub fn populate(&mut self, elements: Vec<(u32, Vec)>, timestamp: Timestamp) { + for (page, values) in elements.into_iter() { + for (i, value) in values.into_iter().enumerate() { + let value = PrimitiveValue { + value, + is_pointer: false, + }; + self.memory + .write_to_memory(page as usize, i, value, timestamp); + } + } + } + + pub fn populate_page( + &mut self, + page: usize, + elements: Vec<(usize, U256)>, + timestamp: Timestamp, + ) { + elements.into_iter().for_each(|(offset, value)| { + let value = PrimitiveValue { + value, + is_pointer: false, + }; + + self.memory.write_to_memory(page, offset, value, timestamp); + }); + } + + pub fn dump_page_content_as_u256_words( + &self, + page: u32, + range: std::ops::Range, + ) -> Vec { + self.memory + .inner() + .dump_page_content_as_u256_words(page, range) + .into_iter() + .map(|v| v.value) + .collect() + } + + pub fn read_slot(&self, page: usize, slot: usize) -> &PrimitiveValue { + self.memory.inner().read_slot(page, slot) + } + + // This method should be used with relatively small lengths, since + // we don't heavily optimize here for cases with long lengths + pub fn read_unaligned_bytes(&self, page: usize, start: usize, length: usize) -> Vec { + if length == 0 { + return vec![]; + } + + let end = start + length - 1; + + let mut current_word = start / 32; + let mut result = vec![]; + while current_word * 32 <= end { + let word_value = self.read_slot(page, current_word).value; + let word_value = { + let mut bytes: Vec = vec![0u8; 32]; + word_value.to_big_endian(&mut bytes); + bytes + }; + + result.extend(extract_needed_bytes_from_word( + word_value, + current_word, + start, + end, + )); + + current_word += 1; + } + + assert_eq!(result.len(), length); + + result + } + + pub(crate) fn get_size(&self) -> usize { + // Hashmap memory overhead is neglected. + let memory_size = self.memory.inner().get_size(); + let observable_pages_size = self.observable_pages.inner().get_size(); + + memory_size + observable_pages_size + } + + pub fn get_history_size(&self) -> usize { + let memory_size = self.memory.borrow_history(|h| h.len(), 0) + * std::mem::size_of::<::HistoryRecord>(); + let observable_pages_size = self.observable_pages.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + + memory_size + observable_pages_size + } + + pub fn delete_history(&mut self) { + self.memory.delete_history(); + self.observable_pages.delete_history(); + } +} + +impl Memory for SimpleMemory { + fn execute_partial_query( + &mut self, + _monotonic_cycle_counter: u32, + mut query: MemoryQuery, + ) -> MemoryQuery { + match query.location.memory_type { + MemoryType::Stack => {} + MemoryType::Heap | MemoryType::AuxHeap => { + // The following assertion works fine even when doing a read + // from heap through pointer, since `value_is_pointer` can only be set to + // `true` during memory writes. + assert!( + !query.value_is_pointer, + "Pointers can only be stored on stack" + ); + } + MemoryType::FatPointer => { + assert!(!query.rw_flag); + assert!( + !query.value_is_pointer, + "Pointers can only be stored on stack" + ); + } + MemoryType::Code => { + unreachable!("code should be through specialized query"); + } + } + + let page = query.location.page.0 as usize; + let slot = query.location.index.0 as usize; + + if query.rw_flag { + self.memory.write_to_memory( + page, + slot, + PrimitiveValue { + value: query.value, + is_pointer: query.value_is_pointer, + }, + query.timestamp, + ); + } else { + let current_value = self.read_slot(page, slot); + query.value = current_value.value; + query.value_is_pointer = current_value.is_pointer; + } + + query + } + + fn specialized_code_query( + &mut self, + _monotonic_cycle_counter: u32, + mut query: MemoryQuery, + ) -> MemoryQuery { + assert_eq!(query.location.memory_type, MemoryType::Code); + assert!( + !query.value_is_pointer, + "Pointers are not used for decommmits" + ); + + let page = query.location.page.0 as usize; + let slot = query.location.index.0 as usize; + + if query.rw_flag { + self.memory.write_to_memory( + page, + slot, + PrimitiveValue { + value: query.value, + is_pointer: query.value_is_pointer, + }, + query.timestamp, + ); + } else { + let current_value = self.read_slot(page, slot); + query.value = current_value.value; + query.value_is_pointer = current_value.is_pointer; + } + + query + } + + fn read_code_query( + &self, + _monotonic_cycle_counter: u32, + mut query: MemoryQuery, + ) -> MemoryQuery { + assert_eq!(query.location.memory_type, MemoryType::Code); + assert!( + !query.value_is_pointer, + "Pointers are not used for decommmits" + ); + assert!(!query.rw_flag, "Only read queries can be processed"); + + let page = query.location.page.0 as usize; + let slot = query.location.index.0 as usize; + + let current_value = self.read_slot(page, slot); + query.value = current_value.value; + query.value_is_pointer = current_value.is_pointer; + + query + } + + fn start_global_frame( + &mut self, + _current_base_page: MemoryPage, + new_base_page: MemoryPage, + calldata_fat_pointer: FatPointer, + timestamp: Timestamp, + ) { + // Besides the calldata page, we also formally include the current stack + // page, heap page and aux heap page. + // The code page will be always left observable, so we don't include it here. + self.observable_pages.push_frame(timestamp); + self.observable_pages.extend_frame( + vec![ + calldata_fat_pointer.memory_page, + stack_page_from_base(new_base_page).0, + heap_page_from_base(new_base_page).0, + aux_heap_page_from_base(new_base_page).0, + ], + timestamp, + ); + } + + fn finish_global_frame( + &mut self, + base_page: MemoryPage, + returndata_fat_pointer: FatPointer, + timestamp: Timestamp, + ) { + // Safe to unwrap here, since `finish_global_frame` is never called with empty stack + let current_observable_pages = self.observable_pages.inner().current_frame(); + let returndata_page = returndata_fat_pointer.memory_page; + + for &page in current_observable_pages { + // If the page's number is greater than or equal to the base_page, + // it means that it was created by the internal calls of this contract. + // We need to add this check as the calldata pointer is also part of the + // observable pages. + if page >= base_page.0 && page != returndata_page { + self.memory.clear_page(page as usize, timestamp); + } + } + + self.observable_pages.clear_frame(timestamp); + self.observable_pages.merge_frame(timestamp); + + self.observable_pages + .push_to_frame(returndata_page, timestamp); + } +} + +// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +fn extract_needed_bytes_from_word( + word_value: Vec, + word_number: usize, + start: usize, + end: usize, +) -> Vec { + let word_start = word_number * 32; + let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + + let intersection_left = std::cmp::max(word_start, start); + let intersection_right = std::cmp::min(word_end, end); + + if intersection_right < intersection_left { + vec![] + } else { + let start_bytes = intersection_left - word_start; + let to_take = intersection_right - intersection_left + 1; + + word_value + .into_iter() + .skip(start_bytes) + .take(to_take) + .collect() + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/mod.rs new file mode 100644 index 000000000000..afade1984614 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/mod.rs @@ -0,0 +1,8 @@ +/// This module contains the parts from old VM implementation, which were not changed during the vm implementation. +/// It should be refactored and removed in the future. +pub(crate) mod event_sink; +pub(crate) mod events; +pub(crate) mod history_recorder; +pub(crate) mod memory; +pub(crate) mod oracles; +pub(crate) mod utils; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs new file mode 100644 index 000000000000..4a718917a21d --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs @@ -0,0 +1,236 @@ +use std::{collections::HashMap, fmt::Debug}; + +use zk_evm_1_4_0::{ + abstractions::{DecommittmentProcessor, Memory, MemoryType}, + aux_structures::{ + DecommittmentQuery, MemoryIndex, MemoryLocation, MemoryPage, MemoryQuery, Timestamp, + }, +}; +use zksync_state::{ReadStorage, StoragePtr}; +use zksync_types::U256; +use zksync_utils::{bytecode::bytecode_len_in_words, bytes_to_be_words, u256_to_h256}; + +use super::OracleWithHistory; +use crate::vm_latest::old_vm::history_recorder::{ + HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, +}; + +/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// used by the VM to 'load' bytecodes into memory. +#[derive(Debug)] +pub struct DecommitterOracle { + /// Pointer that enables to read contract bytecodes from the database. + storage: StoragePtr, + /// The cache of bytecodes that the bootloader "knows", but that are not necessarily in the database. + /// And it is also used as a database cache. + pub known_bytecodes: HistoryRecorder>, H>, + /// Stores pages of memory where certain code hashes have already been decommitted. + /// It is expected that they all are present in the DB. + // `decommitted_code_hashes` history is necessary + pub decommitted_code_hashes: HistoryRecorder, HistoryEnabled>, + /// Stores history of decommitment requests. + decommitment_requests: HistoryRecorder, H>, +} + +impl DecommitterOracle { + pub fn new(storage: StoragePtr) -> Self { + Self { + storage, + known_bytecodes: HistoryRecorder::default(), + decommitted_code_hashes: HistoryRecorder::default(), + decommitment_requests: HistoryRecorder::default(), + } + } + + /// Gets the bytecode for a given hash (either from storage, or from 'known_bytecodes' that were populated by `populate` method). + /// Panics if bytecode doesn't exist. + pub fn get_bytecode(&mut self, hash: U256, timestamp: Timestamp) -> Vec { + let entry = self.known_bytecodes.inner().get(&hash); + + match entry { + Some(x) => x.clone(), + None => { + // It is ok to panic here, since the decommitter is never called directly by + // the users and always called by the VM. VM will never let decommit the + // code hash which we didn't previously claim to know the preimage of. + let value = self + .storage + .borrow_mut() + .load_factory_dep(u256_to_h256(hash)) + .expect("Trying to decode unexisting hash"); + + let value = bytes_to_be_words(value); + self.known_bytecodes.insert(hash, value.clone(), timestamp); + value + } + } + } + + /// Adds additional bytecodes. They will take precedent over the bytecodes from storage. + pub fn populate(&mut self, bytecodes: Vec<(U256, Vec)>, timestamp: Timestamp) { + for (hash, bytecode) in bytecodes { + self.known_bytecodes.insert(hash, bytecode, timestamp); + } + } + + pub fn get_used_bytecode_hashes(&self) -> Vec { + self.decommitted_code_hashes + .inner() + .iter() + .map(|item| *item.0) + .collect() + } + + pub fn get_decommitted_bytecodes_after_timestamp(&self, timestamp: Timestamp) -> usize { + // Note, that here we rely on the fact that for each used bytecode + // there is one and only one corresponding event in the history of it. + self.decommitted_code_hashes + .history() + .iter() + .rev() + .take_while(|(t, _)| *t >= timestamp) + .count() + } + + pub fn get_decommitted_code_hashes_with_history( + &self, + ) -> &HistoryRecorder, HistoryEnabled> { + &self.decommitted_code_hashes + } + + /// Returns the storage handle. Used only in tests. + pub fn get_storage(&self) -> StoragePtr { + self.storage.clone() + } + + /// Measures the amount of memory used by this Oracle (used for metrics only). + pub(crate) fn get_size(&self) -> usize { + // Hashmap memory overhead is neglected. + let known_bytecodes_size = self + .known_bytecodes + .inner() + .iter() + .map(|(_, value)| value.len() * std::mem::size_of::()) + .sum::(); + let decommitted_code_hashes_size = + self.decommitted_code_hashes.inner().len() * std::mem::size_of::<(U256, u32)>(); + + known_bytecodes_size + decommitted_code_hashes_size + } + + pub(crate) fn get_history_size(&self) -> usize { + let known_bytecodes_stack_size = self.known_bytecodes.borrow_history(|h| h.len(), 0) + * std::mem::size_of::<> as WithHistory>::HistoryRecord>(); + let known_bytecodes_heap_size = self.known_bytecodes.borrow_history( + |h| { + h.iter() + .map(|(_, event)| { + if let Some(bytecode) = event.value.as_ref() { + bytecode.len() * std::mem::size_of::() + } else { + 0 + } + }) + .sum::() + }, + 0, + ); + let decommitted_code_hashes_size = + self.decommitted_code_hashes.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + + known_bytecodes_stack_size + known_bytecodes_heap_size + decommitted_code_hashes_size + } + + pub fn delete_history(&mut self) { + self.decommitted_code_hashes.delete_history(); + self.known_bytecodes.delete_history(); + self.decommitment_requests.delete_history(); + } +} + +impl OracleWithHistory for DecommitterOracle { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.decommitted_code_hashes + .rollback_to_timestamp(timestamp); + self.known_bytecodes.rollback_to_timestamp(timestamp); + self.decommitment_requests.rollback_to_timestamp(timestamp); + } +} + +impl DecommittmentProcessor + for DecommitterOracle +{ + /// Loads a given bytecode hash into memory (see trait description for more details). + fn decommit_into_memory( + &mut self, + monotonic_cycle_counter: u32, + mut partial_query: DecommittmentQuery, + memory: &mut M, + ) -> Result< + ( + zk_evm_1_4_0::aux_structures::DecommittmentQuery, + Option>, + ), + anyhow::Error, + > { + self.decommitment_requests.push((), partial_query.timestamp); + // First - check if we didn't fetch this bytecode in the past. + // If we did - we can just return the page that we used before (as the memory is readonly). + if let Some(memory_page) = self + .decommitted_code_hashes + .inner() + .get(&partial_query.hash) + .copied() + { + partial_query.is_fresh = false; + partial_query.memory_page = MemoryPage(memory_page); + partial_query.decommitted_length = + bytecode_len_in_words(&u256_to_h256(partial_query.hash)); + + Ok((partial_query, None)) + } else { + // We are fetching a fresh bytecode that we didn't read before. + let values = self.get_bytecode(partial_query.hash, partial_query.timestamp); + let page_to_use = partial_query.memory_page; + let timestamp = partial_query.timestamp; + partial_query.decommitted_length = values.len() as u16; + partial_query.is_fresh = true; + + // Create a template query, that we'll use for writing into memory. + // value & index are set to 0 - as they will be updated in the inner loop below. + let mut tmp_q = MemoryQuery { + timestamp, + location: MemoryLocation { + memory_type: MemoryType::Code, + page: page_to_use, + index: MemoryIndex(0), + }, + value: U256::zero(), + value_is_pointer: false, + rw_flag: true, + }; + self.decommitted_code_hashes + .insert(partial_query.hash, page_to_use.0, timestamp); + + // Copy the bytecode (that is stored in 'values' Vec) into the memory page. + if B { + for (i, value) in values.iter().enumerate() { + tmp_q.location.index = MemoryIndex(i as u32); + tmp_q.value = *value; + memory.specialized_code_query(monotonic_cycle_counter, tmp_q); + } + // If we're in the witness mode - we also have to return the values. + Ok((partial_query, Some(values))) + } else { + for (i, value) in values.into_iter().enumerate() { + tmp_q.location.index = MemoryIndex(i as u32); + tmp_q.value = value; + memory.specialized_code_query(monotonic_cycle_counter, tmp_q); + } + + Ok((partial_query, None)) + } + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/mod.rs new file mode 100644 index 000000000000..3f8d2d0f1383 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/mod.rs @@ -0,0 +1,8 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; + +pub(crate) mod decommitter; +pub(crate) mod precompile; + +pub(crate) trait OracleWithHistory { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs new file mode 100644 index 000000000000..92b88e40fc95 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs @@ -0,0 +1,72 @@ +use zk_evm_1_4_0::{ + abstractions::{Memory, PrecompileCyclesWitness, PrecompilesProcessor}, + aux_structures::{LogQuery, MemoryQuery, Timestamp}, + zk_evm_abstractions::precompiles::DefaultPrecompilesProcessor, +}; + +use super::OracleWithHistory; +use crate::vm_latest::old_vm::history_recorder::{HistoryEnabled, HistoryMode, HistoryRecorder}; + +/// Wrap of DefaultPrecompilesProcessor that store queue +/// of timestamp when precompiles are called to be executed. +/// Number of precompiles per block is strictly limited, +/// saving timestamps allows us to check the exact number +/// of log queries, that were used during the tx execution. +#[derive(Debug, Clone)] +pub struct PrecompilesProcessorWithHistory { + pub timestamp_history: HistoryRecorder, H>, + pub default_precompiles_processor: DefaultPrecompilesProcessor, +} + +impl Default for PrecompilesProcessorWithHistory { + fn default() -> Self { + Self { + timestamp_history: Default::default(), + default_precompiles_processor: DefaultPrecompilesProcessor, + } + } +} + +impl OracleWithHistory for PrecompilesProcessorWithHistory { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.timestamp_history.rollback_to_timestamp(timestamp); + } +} + +impl PrecompilesProcessorWithHistory { + pub fn get_timestamp_history(&self) -> &Vec { + self.timestamp_history.inner() + } + + pub fn delete_history(&mut self) { + self.timestamp_history.delete_history(); + } +} + +impl PrecompilesProcessor for PrecompilesProcessorWithHistory { + fn start_frame(&mut self) { + self.default_precompiles_processor.start_frame(); + } + fn execute_precompile( + &mut self, + monotonic_cycle_counter: u32, + query: LogQuery, + memory: &mut M, + ) -> Option<(Vec, Vec, PrecompileCyclesWitness)> { + // In the next line we same `query.timestamp` as both + // an operation in the history of precompiles processor and + // the time when this operation occurred. + // While slightly weird, it is done for consistency with other oracles + // where operations and timestamp have different types. + self.timestamp_history + .push(query.timestamp, query.timestamp); + self.default_precompiles_processor.execute_precompile( + monotonic_cycle_counter, + query, + memory, + ) + } + fn finish_frame(&mut self, _panicked: bool) { + self.default_precompiles_processor.finish_frame(_panicked); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs new file mode 100644 index 000000000000..b2c471832e46 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs @@ -0,0 +1,337 @@ +use std::collections::HashMap; + +use crate::vm_latest::old_vm::history_recorder::{ + AppDataFrameManagerWithHistory, HashMapHistoryEvent, HistoryEnabled, HistoryMode, + HistoryRecorder, StorageWrapper, WithHistory, +}; + +use zk_evm_1_4_0::abstractions::RefundedAmounts; +use zk_evm_1_4_0::zkevm_opcode_defs::system_params::INITIAL_STORAGE_WRITE_PUBDATA_BYTES; +use zk_evm_1_4_0::{ + abstractions::{RefundType, Storage as VmStorageOracle}, + aux_structures::{LogQuery, Timestamp}, +}; + +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::utils::storage_key_for_eth_balance; +use zksync_types::{ + AccountTreeId, Address, StorageKey, StorageLogQuery, StorageLogQueryType, BOOTLOADER_ADDRESS, + U256, +}; +use zksync_utils::u256_to_h256; + +use super::OracleWithHistory; + +// While the storage does not support different shards, it was decided to write the +// code of the StorageOracle with the shard parameters in mind. +pub(crate) fn triplet_to_storage_key(_shard_id: u8, address: Address, key: U256) -> StorageKey { + StorageKey::new(AccountTreeId::new(address), u256_to_h256(key)) +} + +pub(crate) fn storage_key_of_log(query: &LogQuery) -> StorageKey { + triplet_to_storage_key(query.shard_id, query.address, query.key) +} + +#[derive(Debug)] +pub struct StorageOracle { + // Access to the persistent storage. Please note that it + // is used only for read access. All the actual writes happen + // after the execution ended. + pub(crate) storage: HistoryRecorder, H>, + + pub(crate) frames_stack: AppDataFrameManagerWithHistory, H>, + + // The changes that have been paid for in previous transactions. + // It is a mapping from storage key to the number of *bytes* that was paid by the user + // to cover this slot. + // `paid_changes` history is necessary + pub(crate) paid_changes: HistoryRecorder, HistoryEnabled>, +} + +impl OracleWithHistory for StorageOracle { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.frames_stack.rollback_to_timestamp(timestamp); + self.storage.rollback_to_timestamp(timestamp); + self.paid_changes.rollback_to_timestamp(timestamp); + } +} + +impl StorageOracle { + pub fn new(storage: StoragePtr) -> Self { + Self { + storage: HistoryRecorder::from_inner(StorageWrapper::new(storage)), + frames_stack: Default::default(), + paid_changes: Default::default(), + } + } + + pub fn delete_history(&mut self) { + self.frames_stack.delete_history(); + self.storage.delete_history(); + self.paid_changes.delete_history(); + } + + fn is_storage_key_free(&self, key: &StorageKey) -> bool { + key.address() == &zksync_system_constants::SYSTEM_CONTEXT_ADDRESS + || *key == storage_key_for_eth_balance(&BOOTLOADER_ADDRESS) + } + + pub fn read_value(&mut self, mut query: LogQuery) -> LogQuery { + let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + let current_value = self.storage.read_from_storage(&key); + + query.read_value = current_value; + + self.frames_stack.push_forward( + Box::new(StorageLogQuery { + log_query: query, + log_type: StorageLogQueryType::Read, + }), + query.timestamp, + ); + + query + } + + pub fn write_value(&mut self, mut query: LogQuery) -> LogQuery { + let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + let current_value = + self.storage + .write_to_storage(key, query.written_value, query.timestamp); + + let is_initial_write = self.storage.get_ptr().borrow_mut().is_write_initial(&key); + let log_query_type = if is_initial_write { + StorageLogQueryType::InitialWrite + } else { + StorageLogQueryType::RepeatedWrite + }; + + query.read_value = current_value; + + let mut storage_log_query = StorageLogQuery { + log_query: query, + log_type: log_query_type, + }; + self.frames_stack + .push_forward(Box::new(storage_log_query), query.timestamp); + storage_log_query.log_query.rollback = true; + self.frames_stack + .push_rollback(Box::new(storage_log_query), query.timestamp); + + query + } + + // Returns the amount of funds that has been already paid for writes into the storage slot + fn prepaid_for_write(&self, storage_key: &StorageKey) -> u32 { + self.paid_changes + .inner() + .get(storage_key) + .copied() + .unwrap_or_default() + } + + pub(crate) fn base_price_for_write(&self, query: &LogQuery) -> u32 { + let storage_key = storage_key_of_log(query); + + if self.is_storage_key_free(&storage_key) { + return 0; + } + + let is_initial_write = self + .storage + .get_ptr() + .borrow_mut() + .is_write_initial(&storage_key); + + get_pubdata_price_bytes(query, is_initial_write) + } + + // Returns the price of the update in terms of pubdata bytes. + // TODO (SMA-1701): update VM to accept gas instead of pubdata. + fn value_update_price(&self, query: &LogQuery) -> u32 { + let storage_key = storage_key_of_log(query); + + let base_cost = self.base_price_for_write(query); + + let already_paid = self.prepaid_for_write(&storage_key); + + if base_cost <= already_paid { + // Some other transaction has already paid for this slot, no need to pay anything + 0u32 + } else { + base_cost - already_paid + } + } + + /// Returns storage log queries from current frame where `log.log_query.timestamp >= from_timestamp`. + pub(crate) fn storage_log_queries_after_timestamp( + &self, + from_timestamp: Timestamp, + ) -> &[Box] { + let logs = self.frames_stack.forward().current_frame(); + + // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. + logs.rsplit(|l| l.log_query.timestamp < from_timestamp) + .next() + .unwrap_or(&[]) + } + + pub(crate) fn get_final_log_queries(&self) -> Vec { + assert_eq!( + self.frames_stack.len(), + 1, + "VM finished execution in unexpected state" + ); + + self.frames_stack + .forward() + .current_frame() + .iter() + .map(|x| **x) + .collect() + } + + pub(crate) fn get_size(&self) -> usize { + let frames_stack_size = self.frames_stack.get_size(); + let paid_changes_size = + self.paid_changes.inner().len() * std::mem::size_of::<(StorageKey, u32)>(); + + frames_stack_size + paid_changes_size + } + + pub(crate) fn get_history_size(&self) -> usize { + let storage_size = self.storage.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + let frames_stack_size = self.frames_stack.get_history_size(); + let paid_changes_size = self.paid_changes.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + storage_size + frames_stack_size + paid_changes_size + } +} + +impl VmStorageOracle for StorageOracle { + // Perform a storage read/write access by taking an partially filled query + // and returning filled query and cold/warm marker for pricing purposes + fn execute_partial_query( + &mut self, + _monotonic_cycle_counter: u32, + query: LogQuery, + ) -> LogQuery { + // tracing::trace!( + // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", + // _monotonic_cycle_counter, + // query.address, + // query.key, + // query.rw_flag, + // query.written_value, + // query.tx_number_in_block + // ); + assert!(!query.rollback); + if query.rw_flag { + // The number of bytes that have been compensated by the user to perform this write + let storage_key = storage_key_of_log(&query); + + // It is considered that the user has paid for the whole base price for the writes + let to_pay_by_user = self.base_price_for_write(&query); + let prepaid = self.prepaid_for_write(&storage_key); + + if to_pay_by_user > prepaid { + self.paid_changes.apply_historic_record( + HashMapHistoryEvent { + key: storage_key, + value: Some(to_pay_by_user), + }, + query.timestamp, + ); + } + self.write_value(query) + } else { + self.read_value(query) + } + } + + // We can return the size of the refund before each storage query. + // Note, that while the `RefundType` allows to provide refunds both in + // `ergs` and `pubdata`, only refunds in pubdata will be compensated for the users + fn estimate_refunds_for_write( + &mut self, // to avoid any hacks inside, like prefetch + _monotonic_cycle_counter: u32, + partial_query: &LogQuery, + ) -> RefundType { + let price_to_pay = self.value_update_price(partial_query); + + RefundType::RepeatedWrite(RefundedAmounts { + ergs: 0, + // `INITIAL_STORAGE_WRITE_PUBDATA_BYTES` is the default amount of pubdata bytes the user pays for. + pubdata_bytes: (INITIAL_STORAGE_WRITE_PUBDATA_BYTES as u32) - price_to_pay, + }) + } + + // Indicate a start of execution frame for rollback purposes + fn start_frame(&mut self, timestamp: Timestamp) { + self.frames_stack.push_frame(timestamp); + } + + // Indicate that execution frame went out from the scope, so we can + // log the history and either rollback immediately or keep records to rollback later + fn finish_frame(&mut self, timestamp: Timestamp, panicked: bool) { + // If we panic then we append forward and rollbacks to the forward of parent, + // otherwise we place rollbacks of child before rollbacks of the parent + if panicked { + // perform actual rollback + for query in self.frames_stack.rollback().current_frame().iter().rev() { + let read_value = match query.log_type { + StorageLogQueryType::Read => { + // Having Read logs in rollback is not possible + tracing::warn!("Read log in rollback queue {:?}", query); + continue; + } + StorageLogQueryType::InitialWrite | StorageLogQueryType::RepeatedWrite => { + query.log_query.read_value + } + }; + + let LogQuery { written_value, .. } = query.log_query; + let key = triplet_to_storage_key( + query.log_query.shard_id, + query.log_query.address, + query.log_query.key, + ); + let current_value = self.storage.write_to_storage( + key, + // NOTE, that since it is a rollback query, + // the `read_value` is being set + read_value, timestamp, + ); + + // Additional validation that the current value was correct + // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // It is impossible to set leaf value to `None` + assert_eq!(current_value, written_value); + } + + self.frames_stack + .move_rollback_to_forward(|_| true, timestamp); + } + self.frames_stack.merge_frame(timestamp); + } +} + +/// Returns the number of bytes needed to publish a slot. +// Since we need to publish the state diffs onchain, for each of the updated storage slot +// we basically need to publish the following pair: (). +// While new_value is always 32 bytes long, for key we use the following optimization: +// - The first time we publish it, we use 32 bytes. +// Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. +// - The second time we publish it, we will use this 8-byte instead of the 32 bytes of the entire key. +// So the total size of the publish pubdata is 40 bytes. We call this kind of write the repeated one +fn get_pubdata_price_bytes(_query: &LogQuery, is_initial: bool) -> u32 { + // TODO (SMA-1702): take into account the content of the log query, i.e. values that contain mostly zeroes + // should cost less. + if is_initial { + zk_evm_1_4_0::zkevm_opcode_defs::system_params::INITIAL_STORAGE_WRITE_PUBDATA_BYTES as u32 + } else { + zk_evm_1_4_0::zkevm_opcode_defs::system_params::REPEATED_STORAGE_WRITE_PUBDATA_BYTES as u32 + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs new file mode 100644 index 000000000000..1dbe82a81d4f --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs @@ -0,0 +1,221 @@ +use zk_evm_1_4_0::{ + aux_structures::{MemoryPage, Timestamp}, + vm_state::PrimitiveValue, + zkevm_opcode_defs::{ + decoding::{AllowedPcOrImm, EncodingModeProduction, VmEncodingMode}, + FatPointer, RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER, + }, +}; +use zksync_state::WriteStorage; +use zksync_system_constants::L1_GAS_PER_PUBDATA_BYTE; +use zksync_types::{Address, U256}; + +use crate::vm_latest::{ + old_vm::memory::SimpleMemory, types::internals::ZkSyncVmState, HistoryMode, +}; + +#[derive(Debug, Clone)] +pub(crate) enum VmExecutionResult { + Ok(Vec), + Revert(Vec), + Panic, + MostLikelyDidNotFinish(Address, u16), +} + +pub(crate) const fn stack_page_from_base(base: MemoryPage) -> MemoryPage { + MemoryPage(base.0 + 1) +} + +pub(crate) const fn heap_page_from_base(base: MemoryPage) -> MemoryPage { + MemoryPage(base.0 + 2) +} + +pub(crate) const fn aux_heap_page_from_base(base: MemoryPage) -> MemoryPage { + MemoryPage(base.0 + 3) +} + +pub(crate) trait FixedLengthIterator<'a, I: 'a, const N: usize>: Iterator +where + Self: 'a, +{ + fn next(&mut self) -> Option<::Item> { + ::next(self) + } +} + +pub(crate) trait IntoFixedLengthByteIterator { + type IntoIter: FixedLengthIterator<'static, u8, N>; + fn into_le_iter(self) -> Self::IntoIter; + fn into_be_iter(self) -> Self::IntoIter; +} + +pub(crate) struct FixedBufferValueIterator { + iter: std::array::IntoIter, +} + +impl Iterator for FixedBufferValueIterator { + type Item = T; + fn next(&mut self) -> Option { + self.iter.next() + } +} + +impl FixedLengthIterator<'static, T, N> + for FixedBufferValueIterator +{ +} + +impl IntoFixedLengthByteIterator<32> for U256 { + type IntoIter = FixedBufferValueIterator; + fn into_le_iter(self) -> Self::IntoIter { + let mut buffer = [0u8; 32]; + self.to_little_endian(&mut buffer); + + FixedBufferValueIterator { + iter: IntoIterator::into_iter(buffer), + } + } + + fn into_be_iter(self) -> Self::IntoIter { + let mut buffer = [0u8; 32]; + self.to_big_endian(&mut buffer); + + FixedBufferValueIterator { + iter: IntoIterator::into_iter(buffer), + } + } +} + +/// Receives sorted slice of timestamps. +/// Returns count of timestamps that are greater than or equal to `from_timestamp`. +/// Works in O(log(sorted_timestamps.len())). +pub(crate) fn precompile_calls_count_after_timestamp( + sorted_timestamps: &[Timestamp], + from_timestamp: Timestamp, +) -> usize { + sorted_timestamps.len() - sorted_timestamps.partition_point(|t| *t < from_timestamp) +} + +pub(crate) fn eth_price_per_pubdata_byte(l1_gas_price: u64) -> u64 { + // This value will typically be a lot less than u64 + // unless the gas price on L1 goes beyond tens of millions of gwei + l1_gas_price * (L1_GAS_PER_PUBDATA_BYTE as u64) +} + +pub(crate) fn vm_may_have_ended_inner( + vm: &ZkSyncVmState, +) -> Option { + let execution_has_ended = vm.execution_has_ended(); + + let r1 = vm.local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; + let current_address = vm.local_state.callstack.get_current_stack().this_address; + + let outer_eh_location = >::PcOrImm::MAX.as_u64(); + match ( + execution_has_ended, + vm.local_state.callstack.get_current_stack().pc.as_u64(), + ) { + (true, 0) => { + let returndata = dump_memory_page_using_primitive_value(&vm.memory, r1); + + Some(VmExecutionResult::Ok(returndata)) + } + (false, _) => None, + (true, l) if l == outer_eh_location => { + // check r1,r2,r3 + if vm.local_state.flags.overflow_or_less_than_flag { + Some(VmExecutionResult::Panic) + } else { + let returndata = dump_memory_page_using_primitive_value(&vm.memory, r1); + Some(VmExecutionResult::Revert(returndata)) + } + } + (_, a) => Some(VmExecutionResult::MostLikelyDidNotFinish( + current_address, + a as u16, + )), + } +} + +pub(crate) fn dump_memory_page_using_primitive_value( + memory: &SimpleMemory, + ptr: PrimitiveValue, +) -> Vec { + if !ptr.is_pointer { + return vec![]; + } + let fat_ptr = FatPointer::from_u256(ptr.value); + dump_memory_page_using_fat_pointer(memory, fat_ptr) +} + +pub(crate) fn dump_memory_page_using_fat_pointer( + memory: &SimpleMemory, + fat_ptr: FatPointer, +) -> Vec { + dump_memory_page_by_offset_and_length( + memory, + fat_ptr.memory_page, + (fat_ptr.start + fat_ptr.offset) as usize, + (fat_ptr.length - fat_ptr.offset) as usize, + ) +} + +pub(crate) fn dump_memory_page_by_offset_and_length( + memory: &SimpleMemory, + page: u32, + offset: usize, + length: usize, +) -> Vec { + assert!(offset < (1u32 << 24) as usize); + assert!(length < (1u32 << 24) as usize); + let mut dump = Vec::with_capacity(length); + if length == 0 { + return dump; + } + + let first_word = offset / 32; + let end_byte = offset + length; + let mut last_word = end_byte / 32; + if end_byte % 32 != 0 { + last_word += 1; + } + + let unalignment = offset % 32; + + let page_part = + memory.dump_page_content_as_u256_words(page, (first_word as u32)..(last_word as u32)); + + let mut is_first = true; + let mut remaining = length; + for word in page_part.into_iter() { + let it = word.into_be_iter(); + if is_first { + is_first = false; + let it = it.skip(unalignment); + for next in it { + if remaining > 0 { + dump.push(next); + remaining -= 1; + } + } + } else { + for next in it { + if remaining > 0 { + dump.push(next); + remaining -= 1; + } + } + } + } + + assert_eq!( + dump.len(), + length, + "tried to dump with offset {}, length {}, got a bytestring of length {}", + offset, + length, + dump.len() + ); + + dump +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/mod.rs new file mode 100644 index 000000000000..b21c842572fe --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/mod.rs @@ -0,0 +1 @@ +pub(crate) mod storage; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs new file mode 100644 index 000000000000..2b6b5988e06e --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs @@ -0,0 +1,489 @@ +use std::collections::HashMap; + +use zk_evm_1_4_0::{ + abstractions::{RefundType, RefundedAmounts, Storage as VmStorageOracle}, + aux_structures::{LogQuery, Timestamp}, + zkevm_opcode_defs::system_params::INITIAL_STORAGE_WRITE_PUBDATA_BYTES, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::{ + utils::storage_key_for_eth_balance, + writes::{ + compression::compress_with_best_strategy, BYTES_PER_DERIVED_KEY, + BYTES_PER_ENUMERATION_INDEX, + }, + AccountTreeId, Address, StorageKey, StorageLogQuery, StorageLogQueryType, BOOTLOADER_ADDRESS, + U256, +}; +use zksync_utils::u256_to_h256; + +use crate::vm_latest::old_vm::{ + history_recorder::{ + AppDataFrameManagerWithHistory, HashMapHistoryEvent, HistoryEnabled, HistoryMode, + HistoryRecorder, StorageWrapper, VectorHistoryEvent, WithHistory, + }, + oracles::OracleWithHistory, +}; + +// While the storage does not support different shards, it was decided to write the +// code of the StorageOracle with the shard parameters in mind. +pub(crate) fn triplet_to_storage_key(_shard_id: u8, address: Address, key: U256) -> StorageKey { + StorageKey::new(AccountTreeId::new(address), u256_to_h256(key)) +} + +pub(crate) fn storage_key_of_log(query: &LogQuery) -> StorageKey { + triplet_to_storage_key(query.shard_id, query.address, query.key) +} + +#[derive(Debug)] +pub struct StorageOracle { + // Access to the persistent storage. Please note that it + // is used only for read access. All the actual writes happen + // after the execution ended. + pub(crate) storage: HistoryRecorder, H>, + + pub(crate) frames_stack: AppDataFrameManagerWithHistory, H>, + + // The changes that have been paid for in previous transactions. + // It is a mapping from storage key to the number of *bytes* that was paid by the user + // to cover this slot. + pub(crate) pre_paid_changes: HistoryRecorder, H>, + + // The changes that have been paid for in the current transaction + pub(crate) paid_changes: HistoryRecorder, H>, + + // The map that contains all the first values read from storage for each slot. + // While formally it does not have to be rollbackable, we still do it to avoid memory bloat + // for unused slots. + pub(crate) initial_values: HistoryRecorder, H>, + + // Storage refunds that oracle has returned in `estimate_refunds_for_write`. + pub(crate) returned_refunds: HistoryRecorder, H>, +} + +impl OracleWithHistory for StorageOracle { + fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { + self.storage.rollback_to_timestamp(timestamp); + self.frames_stack.rollback_to_timestamp(timestamp); + self.pre_paid_changes.rollback_to_timestamp(timestamp); + self.paid_changes.rollback_to_timestamp(timestamp); + self.initial_values.rollback_to_timestamp(timestamp); + self.returned_refunds.rollback_to_timestamp(timestamp); + } +} + +impl StorageOracle { + pub fn new(storage: StoragePtr) -> Self { + Self { + storage: HistoryRecorder::from_inner(StorageWrapper::new(storage)), + frames_stack: Default::default(), + pre_paid_changes: Default::default(), + paid_changes: Default::default(), + initial_values: Default::default(), + returned_refunds: Default::default(), + } + } + + pub fn delete_history(&mut self) { + self.storage.delete_history(); + self.frames_stack.delete_history(); + self.pre_paid_changes.delete_history(); + self.paid_changes.delete_history(); + self.initial_values.delete_history(); + self.returned_refunds.delete_history(); + } + + fn is_storage_key_free(&self, key: &StorageKey) -> bool { + key.address() == &zksync_system_constants::SYSTEM_CONTEXT_ADDRESS + || *key == storage_key_for_eth_balance(&BOOTLOADER_ADDRESS) + } + + fn get_initial_value(&self, storage_key: &StorageKey) -> Option { + self.initial_values.inner().get(storage_key).copied() + } + + fn set_initial_value(&mut self, storage_key: &StorageKey, value: U256, timestamp: Timestamp) { + if !self.initial_values.inner().contains_key(storage_key) { + self.initial_values.insert(*storage_key, value, timestamp); + } + } + + pub fn read_value(&mut self, mut query: LogQuery) -> LogQuery { + let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + let current_value = self.storage.read_from_storage(&key); + + query.read_value = current_value; + + self.set_initial_value(&key, current_value, query.timestamp); + + self.frames_stack.push_forward( + Box::new(StorageLogQuery { + log_query: query, + log_type: StorageLogQueryType::Read, + }), + query.timestamp, + ); + + query + } + + pub fn write_value(&mut self, query: LogQuery) -> LogQuery { + let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + let current_value = + self.storage + .write_to_storage(key, query.written_value, query.timestamp); + + let is_initial_write = self.storage.get_ptr().borrow_mut().is_write_initial(&key); + let log_query_type = if is_initial_write { + StorageLogQueryType::InitialWrite + } else { + StorageLogQueryType::RepeatedWrite + }; + + self.set_initial_value(&key, current_value, query.timestamp); + + let mut storage_log_query = StorageLogQuery { + log_query: query, + log_type: log_query_type, + }; + self.frames_stack + .push_forward(Box::new(storage_log_query), query.timestamp); + storage_log_query.log_query.rollback = true; + self.frames_stack + .push_rollback(Box::new(storage_log_query), query.timestamp); + storage_log_query.log_query.rollback = false; + + query + } + + // Returns the amount of funds that has been already paid for writes into the storage slot + fn prepaid_for_write(&self, storage_key: &StorageKey) -> u32 { + self.paid_changes + .inner() + .get(storage_key) + .copied() + .unwrap_or_else(|| { + self.pre_paid_changes + .inner() + .get(storage_key) + .copied() + .unwrap_or(0) + }) + } + + // Remembers the changes that have been paid for in the current transaction. + // It also returns how much pubdata did the user pay for and how much was actually published. + pub(crate) fn save_paid_changes(&mut self, timestamp: Timestamp) -> u32 { + let mut published = 0; + + let modified_keys = self + .paid_changes + .inner() + .iter() + .map(|(k, v)| (*k, *v)) + .collect::>(); + + for (key, _) in modified_keys { + // It is expected that for each slot for which we have paid changes, there is some + // first slot value already read. + let first_slot_value = self.initial_values.inner().get(&key).copied().unwrap(); + + // This is the value has been written to the storage slot at the end. + let current_slot_value = self.storage.read_from_storage(&key); + + let required_pubdata = + self.base_price_for_write(&key, first_slot_value, current_slot_value); + + // We assume that "prepaid_for_slot" represents both the number of pubdata published and the number of bytes paid by the previous transactions + // as they should be identical. + let prepaid_for_slot = self + .pre_paid_changes + .inner() + .get(&key) + .copied() + .unwrap_or_default(); + + published += required_pubdata.saturating_sub(prepaid_for_slot); + + // We remove the slot from the paid changes and move to the pre-paid changes as + // the transaction ends. + self.paid_changes.remove(key, timestamp); + self.pre_paid_changes + .insert(key, prepaid_for_slot.max(required_pubdata), timestamp); + } + + published + } + + fn base_price_for_write_query(&self, query: &LogQuery) -> u32 { + let storage_key = storage_key_of_log(query); + + let initial_value = self + .get_initial_value(&storage_key) + .unwrap_or(self.storage.read_from_storage(&storage_key)); + + self.base_price_for_write(&storage_key, initial_value, query.written_value) + } + + pub(crate) fn base_price_for_write( + &self, + storage_key: &StorageKey, + prev_value: U256, + new_value: U256, + ) -> u32 { + if self.is_storage_key_free(storage_key) || prev_value == new_value { + return 0; + } + + let is_initial_write = self + .storage + .get_ptr() + .borrow_mut() + .is_write_initial(storage_key); + + get_pubdata_price_bytes(prev_value, new_value, is_initial_write) + } + + // Returns the price of the update in terms of pubdata bytes. + // TODO (SMA-1701): update VM to accept gas instead of pubdata. + fn value_update_price(&mut self, query: &LogQuery) -> u32 { + let storage_key = storage_key_of_log(query); + + let base_cost = self.base_price_for_write_query(query); + + let initial_value = self + .get_initial_value(&storage_key) + .unwrap_or(self.storage.read_from_storage(&storage_key)); + + self.set_initial_value(&storage_key, initial_value, query.timestamp); + + let already_paid = self.prepaid_for_write(&storage_key); + + if base_cost <= already_paid { + // Some other transaction has already paid for this slot, no need to pay anything + 0u32 + } else { + base_cost - already_paid + } + } + + /// Returns storage log queries from current frame where `log.log_query.timestamp >= from_timestamp`. + pub(crate) fn storage_log_queries_after_timestamp( + &self, + from_timestamp: Timestamp, + ) -> &[Box] { + let logs = self.frames_stack.forward().current_frame(); + + // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. + logs.rsplit(|l| l.log_query.timestamp < from_timestamp) + .next() + .unwrap_or(&[]) + } + + pub(crate) fn get_final_log_queries(&self) -> Vec { + assert_eq!( + self.frames_stack.len(), + 1, + "VM finished execution in unexpected state" + ); + + self.frames_stack + .forward() + .current_frame() + .iter() + .map(|x| **x) + .collect() + } + + pub(crate) fn get_size(&self) -> usize { + let frames_stack_size = self.frames_stack.get_size(); + let paid_changes_size = + self.paid_changes.inner().len() * std::mem::size_of::<(StorageKey, u32)>(); + + frames_stack_size + paid_changes_size + } + + pub(crate) fn get_history_size(&self) -> usize { + let storage_size = self.storage.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + let frames_stack_size = self.frames_stack.get_history_size(); + let paid_changes_size = self.paid_changes.borrow_history(|h| h.len(), 0) + * std::mem::size_of::< as WithHistory>::HistoryRecord>(); + storage_size + frames_stack_size + paid_changes_size + } +} + +impl VmStorageOracle for StorageOracle { + // Perform a storage read/write access by taking an partially filled query + // and returning filled query and cold/warm marker for pricing purposes + fn execute_partial_query( + &mut self, + _monotonic_cycle_counter: u32, + mut query: LogQuery, + ) -> LogQuery { + // tracing::trace!( + // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", + // _monotonic_cycle_counter, + // query.address, + // query.key, + // query.rw_flag, + // query.written_value, + // query.tx_number_in_block + // ); + assert!(!query.rollback); + if query.rw_flag { + // The number of bytes that have been compensated by the user to perform this write + let storage_key = storage_key_of_log(&query); + let read_value = self.storage.read_from_storage(&storage_key); + query.read_value = read_value; + + // It is considered that the user has paid for the whole base price for the writes + let to_pay_by_user = self.base_price_for_write_query(&query); + let prepaid = self.prepaid_for_write(&storage_key); + + if to_pay_by_user > prepaid { + self.paid_changes.apply_historic_record( + HashMapHistoryEvent { + key: storage_key, + value: Some(to_pay_by_user), + }, + query.timestamp, + ); + } + self.write_value(query) + } else { + self.read_value(query) + } + } + + // We can return the size of the refund before each storage query. + // Note, that while the `RefundType` allows to provide refunds both in + // `ergs` and `pubdata`, only refunds in pubdata will be compensated for the users + fn estimate_refunds_for_write( + &mut self, // to avoid any hacks inside, like prefetch + _monotonic_cycle_counter: u32, + partial_query: &LogQuery, + ) -> RefundType { + let storage_key = storage_key_of_log(partial_query); + let mut partial_query = *partial_query; + let read_value = self.storage.read_from_storage(&storage_key); + partial_query.read_value = read_value; + + let price_to_pay = self + .value_update_price(&partial_query) + .min(INITIAL_STORAGE_WRITE_PUBDATA_BYTES as u32); + + let refund = RefundType::RepeatedWrite(RefundedAmounts { + ergs: 0, + // `INITIAL_STORAGE_WRITE_PUBDATA_BYTES` is the default amount of pubdata bytes the user pays for. + pubdata_bytes: (INITIAL_STORAGE_WRITE_PUBDATA_BYTES as u32) - price_to_pay, + }); + self.returned_refunds.apply_historic_record( + VectorHistoryEvent::Push(refund.pubdata_refund()), + partial_query.timestamp, + ); + + refund + } + + // Indicate a start of execution frame for rollback purposes + fn start_frame(&mut self, timestamp: Timestamp) { + self.frames_stack.push_frame(timestamp); + } + + // Indicate that execution frame went out from the scope, so we can + // log the history and either rollback immediately or keep records to rollback later + fn finish_frame(&mut self, timestamp: Timestamp, panicked: bool) { + // If we panic then we append forward and rollbacks to the forward of parent, + // otherwise we place rollbacks of child before rollbacks of the parent + if panicked { + // perform actual rollback + for query in self.frames_stack.rollback().current_frame().iter().rev() { + let read_value = match query.log_type { + StorageLogQueryType::Read => { + // Having Read logs in rollback is not possible + tracing::warn!("Read log in rollback queue {:?}", query); + continue; + } + StorageLogQueryType::InitialWrite | StorageLogQueryType::RepeatedWrite => { + query.log_query.read_value + } + }; + + let LogQuery { written_value, .. } = query.log_query; + let key = triplet_to_storage_key( + query.log_query.shard_id, + query.log_query.address, + query.log_query.key, + ); + let current_value = self.storage.write_to_storage( + key, + // NOTE, that since it is a rollback query, + // the `read_value` is being set + read_value, timestamp, + ); + + // Additional validation that the current value was correct + // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // It is impossible to set leaf value to `None` + assert_eq!(current_value, written_value); + } + + self.frames_stack + .move_rollback_to_forward(|_| true, timestamp); + } + self.frames_stack.merge_frame(timestamp); + } +} + +/// Returns the number of bytes needed to publish a slot. +// Since we need to publish the state diffs onchain, for each of the updated storage slot +// we basically need to publish the following pair: (). +// For key we use the following optimization: +// - The first time we publish it, we use 32 bytes. +// Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. +// - The second time we publish it, we will use the 4/5 byte representation of this 8-byte instead of the 32 +// bytes of the entire key. +// For value compression, we use a metadata byte which holds the length of the value and the operation from the +// previous state to the new state, and the compressed value. The maximum for this is 33 bytes. +// Total bytes for initial writes then becomes 65 bytes and repeated writes becomes 38 bytes. +fn get_pubdata_price_bytes(initial_value: U256, final_value: U256, is_initial: bool) -> u32 { + // TODO (SMA-1702): take into account the content of the log query, i.e. values that contain mostly zeroes + // should cost less. + + let compressed_value_size = + compress_with_best_strategy(initial_value, final_value).len() as u32; + + if is_initial { + (BYTES_PER_DERIVED_KEY as u32) + compressed_value_size + } else { + (BYTES_PER_ENUMERATION_INDEX as u32) + compressed_value_size + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_get_pubdata_price_bytes() { + let initial_value = U256::default(); + let final_value = U256::from(92122); + let is_initial = true; + + let compression_len = 4; + + let initial_bytes_price = get_pubdata_price_bytes(initial_value, final_value, is_initial); + let repeated_bytes_price = get_pubdata_price_bytes(initial_value, final_value, !is_initial); + + assert_eq!( + initial_bytes_price, + (compression_len + BYTES_PER_DERIVED_KEY as usize) as u32 + ); + assert_eq!( + repeated_bytes_price, + (compression_len + BYTES_PER_ENUMERATION_INDEX as usize) as u32 + ); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs new file mode 100644 index 000000000000..78fb964f7221 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs @@ -0,0 +1,56 @@ +use zksync_types::U256; + +use crate::{ + interface::{ExecutionResult, Halt, TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + constants::BOOTLOADER_HEAP_PAGE, + tests::{ + tester::VmTesterBuilder, + utils::{get_bootloader, verify_required_memory, BASE_SYSTEM_CONTRACTS}, + }, + HistoryEnabled, + }, +}; + +#[test] +fn test_dummy_bootloader() { + let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); + base_system_contracts.bootloader = get_bootloader("dummy"); + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_base_system_smart_contracts(base_system_contracts) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let result = vm.vm.execute(VmExecutionMode::Batch); + assert!(!result.result.is_failed()); + + let correct_first_cell = U256::from_str_radix("123123123", 16).unwrap(); + verify_required_memory( + &vm.vm.state, + vec![(correct_first_cell, BOOTLOADER_HEAP_PAGE, 0)], + ); +} + +#[test] +fn test_bootloader_out_of_gas() { + let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); + base_system_contracts.bootloader = get_bootloader("dummy"); + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_base_system_smart_contracts(base_system_contracts) + .with_gas_limit(10) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let res = vm.vm.execute(VmExecutionMode::Batch); + + assert!(matches!( + res.result, + ExecutionResult::Halt { + reason: Halt::BootloaderOutOfGas + } + )); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs new file mode 100644 index 000000000000..a0c10addff93 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs @@ -0,0 +1,43 @@ +use zksync_types::event::extract_long_l2_to_l1_messages; +use zksync_utils::bytecode::compress_bytecode; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{DeployContractsTx, TxType, VmTesterBuilder}, + utils::read_test_contract, + }, + HistoryEnabled, + }, +}; + +#[test] +fn test_bytecode_publishing() { + // In this test, we aim to ensure that the contents of the compressed bytecodes + // are included as part of the L2->L1 long messages + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let counter = read_test_contract(); + let account = &mut vm.rich_accounts[0]; + + let compressed_bytecode = compress_bytecode(&counter).unwrap(); + + let DeployContractsTx { tx, .. } = account.get_deploy_tx(&counter, None, TxType::L2); + vm.vm.push_transaction(tx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed(), "Transaction wasn't successful"); + + vm.vm.execute(VmExecutionMode::Batch); + + let state = vm.vm.get_current_execution_state(); + let long_messages = extract_long_l2_to_l1_messages(&state.events); + assert!( + long_messages.contains(&compressed_bytecode), + "Bytecode not published" + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs new file mode 100644 index 000000000000..2f8f37e081bd --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs @@ -0,0 +1,92 @@ +use std::sync::Arc; + +use once_cell::sync::OnceCell; +use zksync_types::{Address, Execute}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + tracers::CallTracer, + vm_latest::{ + constants::BLOCK_GAS_LIMIT, + tests::{ + tester::VmTesterBuilder, + utils::{read_max_depth_contract, read_test_contract}, + }, + HistoryEnabled, ToTracerPointer, + }, +}; + +// This test is ultra slow, so it's ignored by default. +#[test] +#[ignore] +fn test_max_depth() { + let contarct = read_max_depth_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contarct, address, true)]) + .build(); + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: vec![], + value: Default::default(), + factory_deps: None, + }, + None, + ); + + let result = Arc::new(OnceCell::new()); + let call_tracer = CallTracer::new(result.clone()).into_tracer_pointer(); + vm.vm.push_transaction(tx); + let res = vm.vm.inspect(call_tracer.into(), VmExecutionMode::OneTx); + assert!(result.get().is_some()); + assert!(res.result.is_failed()); +} + +#[test] +fn test_basic_behavior() { + let contarct = read_test_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contarct, address, true)]) + .build(); + + let increment_by_6_calldata = + "7cf5dab00000000000000000000000000000000000000000000000000000000000000006"; + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: hex::decode(increment_by_6_calldata).unwrap(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + + let result = Arc::new(OnceCell::new()); + let call_tracer = CallTracer::new(result.clone()).into_tracer_pointer(); + vm.vm.push_transaction(tx); + let res = vm.vm.inspect(call_tracer.into(), VmExecutionMode::OneTx); + + let call_tracer_result = result.get().unwrap(); + + assert_eq!(call_tracer_result.len(), 1); + // Expect that there are a plenty of subcalls underneath. + let subcall = &call_tracer_result[0].calls; + assert!(subcall.len() > 10); + assert!(!res.result.is_failed()); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs new file mode 100644 index 000000000000..7c951e313210 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs @@ -0,0 +1,76 @@ +use zksync_system_constants::L2_ETH_TOKEN_ADDRESS; +use zksync_types::{ + get_code_key, get_known_code_key, get_nonce_key, + system_contracts::{DEPLOYMENT_NONCE_INCREMENT, TX_NONCE_INCREMENT}, + AccountTreeId, U256, +}; +use zksync_utils::u256_to_h256; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{DeployContractsTx, TxType, VmTesterBuilder}, + utils::{get_balance, read_test_contract, verify_required_storage}, + }, + HistoryEnabled, + }, +}; + +#[test] +fn test_default_aa_interaction() { + // In this test, we aim to test whether a simple account interaction (without any fee logic) + // will work. The account will try to deploy a simple contract from integration tests. + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let counter = read_test_contract(); + let account = &mut vm.rich_accounts[0]; + let DeployContractsTx { + tx, + bytecode_hash, + address, + } = account.get_deploy_tx(&counter, None, TxType::L2); + let maximal_fee = tx.gas_limit() * vm.vm.batch_env.base_fee(); + + vm.vm.push_transaction(tx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed(), "Transaction wasn't successful"); + + vm.vm.execute(VmExecutionMode::Batch); + vm.vm.get_current_execution_state(); + + // Both deployment and ordinary nonce should be incremented by one. + let account_nonce_key = get_nonce_key(&account.address); + let expected_nonce = TX_NONCE_INCREMENT + DEPLOYMENT_NONCE_INCREMENT; + + // The code hash of the deployed contract should be marked as republished. + let known_codes_key = get_known_code_key(&bytecode_hash); + + // The contract should be deployed successfully. + let account_code_key = get_code_key(&address); + + let expected_slots = vec![ + (u256_to_h256(expected_nonce), account_nonce_key), + (u256_to_h256(U256::from(1u32)), known_codes_key), + (bytecode_hash, account_code_key), + ]; + + verify_required_storage(&vm.vm.state, expected_slots); + + let expected_fee = maximal_fee + - U256::from(result.refunds.gas_refunded) * U256::from(vm.vm.batch_env.base_fee()); + let operator_balance = get_balance( + AccountTreeId::new(L2_ETH_TOKEN_ADDRESS), + &vm.fee_account, + vm.vm.state.storage.storage.get_ptr(), + ); + + assert_eq!( + operator_balance, expected_fee, + "Operator did not receive his fee" + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs new file mode 100644 index 000000000000..533d9ec660eb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs @@ -0,0 +1,47 @@ +use zksync_types::{fee::Fee, Execute}; + +use crate::{ + interface::{TxExecutionMode, VmInterface}, + vm_latest::{ + constants::{BOOTLOADER_HEAP_PAGE, TX_DESCRIPTION_OFFSET, TX_GAS_LIMIT_OFFSET}, + tests::tester::VmTesterBuilder, + HistoryDisabled, + }, +}; + +/// Checks that `TX_GAS_LIMIT_OFFSET` constant is correct. +#[test] +fn test_tx_gas_limit_offset() { + let mut vm = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let gas_limit = 9999.into(); + let tx = vm.rich_accounts[0].get_l2_tx_for_execute( + Execute { + contract_address: Default::default(), + calldata: vec![], + value: Default::default(), + factory_deps: None, + }, + Some(Fee { + gas_limit, + ..Default::default() + }), + ); + + vm.vm.push_transaction(tx); + + let gas_limit_from_memory = vm + .vm + .state + .memory + .read_slot( + BOOTLOADER_HEAP_PAGE as usize, + TX_DESCRIPTION_OFFSET + TX_GAS_LIMIT_OFFSET, + ) + .value; + assert_eq!(gas_limit_from_memory, gas_limit); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs new file mode 100644 index 000000000000..b82057bef8b7 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs @@ -0,0 +1,109 @@ +use std::collections::{HashMap, HashSet}; + +use itertools::Itertools; +use zksync_state::WriteStorage; +use zksync_system_constants::CONTRACT_DEPLOYER_ADDRESS; +use zksync_test_account::Account; +use zksync_types::{Execute, U256}; +use zksync_utils::{bytecode::hash_bytecode, h256_to_u256}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{TxType, VmTesterBuilder}, + utils::{read_test_contract, BASE_SYSTEM_CONTRACTS}, + }, + HistoryDisabled, Vm, + }, + HistoryMode, +}; + +#[test] +fn test_get_used_contracts() { + let mut vm = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + assert!(known_bytecodes_without_aa_code(&vm.vm).is_empty()); + + // create and push and execute some not-empty factory deps transaction with success status + // to check that get_used_contracts() updates + let contract_code = read_test_contract(); + let mut account = Account::random(); + let tx = account.get_deploy_tx(&contract_code, None, TxType::L1 { serial_id: 0 }); + vm.vm.push_transaction(tx.tx.clone()); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed()); + + assert!(vm + .vm + .get_used_contracts() + .contains(&h256_to_u256(tx.bytecode_hash))); + + // Note: Default_AA will be in the list of used contracts if l2 tx is used + assert_eq!( + vm.vm + .get_used_contracts() + .into_iter() + .collect::>(), + known_bytecodes_without_aa_code(&vm.vm) + .keys() + .cloned() + .collect::>() + ); + + // create push and execute some non-empty factory deps transaction that fails + // (known_bytecodes will be updated but we expect get_used_contracts() to not be updated) + + let calldata = [1, 2, 3]; + let big_calldata: Vec = calldata + .iter() + .cycle() + .take(calldata.len() * 1024) + .cloned() + .collect(); + let account2 = Account::random(); + let tx2 = account2.get_l1_tx( + Execute { + contract_address: CONTRACT_DEPLOYER_ADDRESS, + calldata: big_calldata, + value: Default::default(), + factory_deps: Some(vec![vec![1; 32]]), + }, + 1, + ); + + vm.vm.push_transaction(tx2.clone()); + + let res2 = vm.vm.execute(VmExecutionMode::OneTx); + + assert!(res2.result.is_failed()); + + for factory_dep in tx2.execute.factory_deps.unwrap() { + let hash = hash_bytecode(&factory_dep); + let hash_to_u256 = h256_to_u256(hash); + assert!(known_bytecodes_without_aa_code(&vm.vm) + .keys() + .contains(&hash_to_u256)); + assert!(!vm.vm.get_used_contracts().contains(&hash_to_u256)); + } +} + +fn known_bytecodes_without_aa_code( + vm: &Vm, +) -> HashMap> { + let mut known_bytecodes_without_aa_code = vm + .state + .decommittment_processor + .known_bytecodes + .inner() + .clone(); + + known_bytecodes_without_aa_code + .remove(&h256_to_u256(BASE_SYSTEM_CONTRACTS.default_aa.hash)) + .unwrap(); + + known_bytecodes_without_aa_code +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs new file mode 100644 index 000000000000..dde83d8a9f36 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs @@ -0,0 +1,120 @@ +use zksync_types::H256; +use zksync_utils::h256_to_u256; + +use crate::vm_latest::tests::tester::VmTesterBuilder; +use crate::vm_latest::types::inputs::system_env::TxExecutionMode; +use crate::vm_latest::{HistoryEnabled, TxRevertReason}; + +// TODO this test requires a lot of hacks for bypassing the bytecode checks in the VM. +// Port it later, it's not significant. for now + +#[test] +fn test_invalid_bytecode() { + let mut vm_builder = VmTesterBuilder::new(HistoryEnabled) + .with_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1); + let mut storage = vm_builder.take_storage(); + let mut vm = vm_builder.build(&mut storage); + + let block_gas_per_pubdata = vm_test_env + .block_context + .context + .block_gas_price_per_pubdata(); + + let mut test_vm_with_custom_bytecode_hash = + |bytecode_hash: H256, expected_revert_reason: Option| { + let mut oracle_tools = + OracleTools::new(vm_test_env.storage_ptr.as_mut(), HistoryEnabled); + + let (encoded_tx, predefined_overhead) = get_l1_tx_with_custom_bytecode_hash( + h256_to_u256(bytecode_hash), + block_gas_per_pubdata as u32, + ); + + run_vm_with_custom_factory_deps( + &mut oracle_tools, + vm_test_env.block_context.context, + &vm_test_env.block_properties, + encoded_tx, + predefined_overhead, + expected_revert_reason, + ); + }; + + let failed_to_mark_factory_deps = |msg: &str, data: Vec| { + TxRevertReason::FailedToMarkFactoryDependencies(VmRevertReason::General { + msg: msg.to_string(), + data, + }) + }; + + // Here we provide the correctly-formatted bytecode hash of + // odd length, so it should work. + test_vm_with_custom_bytecode_hash( + H256([ + 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, + ]), + None, + ); + + // Here we provide correctly formatted bytecode of even length, so + // it should fail. + test_vm_with_custom_bytecode_hash( + H256([ + 1, 0, 2, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, + ]), + Some(failed_to_mark_factory_deps( + "Code length in words must be odd", + vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 32, 67, 111, 100, 101, 32, 108, 101, 110, + 103, 116, 104, 32, 105, 110, 32, 119, 111, 114, 100, 115, 32, 109, 117, 115, 116, + 32, 98, 101, 32, 111, 100, 100, + ], + )), + ); + + // Here we provide incorrectly formatted bytecode of odd length, so + // it should fail. + test_vm_with_custom_bytecode_hash( + H256([ + 1, 1, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, + ]), + Some(failed_to_mark_factory_deps( + "Incorrectly formatted bytecodeHash", + vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 34, 73, 110, 99, 111, 114, 114, 101, 99, + 116, 108, 121, 32, 102, 111, 114, 109, 97, 116, 116, 101, 100, 32, 98, 121, 116, + 101, 99, 111, 100, 101, 72, 97, 115, 104, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ], + )), + ); + + // Here we provide incorrectly formatted bytecode of odd length, so + // it should fail. + test_vm_with_custom_bytecode_hash( + H256([ + 2, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, + ]), + Some(failed_to_mark_factory_deps( + "Incorrectly formatted bytecodeHash", + vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 34, 73, 110, 99, 111, 114, 114, 101, 99, + 116, 108, 121, 32, 102, 111, 114, 109, 97, 116, 116, 101, 100, 32, 98, 121, 116, + 101, 99, 111, 100, 101, 72, 97, 115, 104, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ], + )), + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs new file mode 100644 index 000000000000..d5a6679502b5 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs @@ -0,0 +1,48 @@ +use zksync_state::ReadStorage; +use zksync_types::get_nonce_key; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{Account, TxType, VmTesterBuilder}, + utils::read_test_contract, + }, + HistoryDisabled, + }, +}; + +#[test] +fn test_is_write_initial_behaviour() { + // In this test, we check result of `is_write_initial` at different stages. + // The main idea is to check that `is_write_initial` storage uses the correct cache for initial_writes and doesn't + // messed up it with the repeated writes during the one batch execution. + + let mut account = Account::random(); + let mut vm = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_rich_accounts(vec![account.clone()]) + .build(); + + let nonce_key = get_nonce_key(&account.address); + // Check that the next write to the nonce key will be initial. + assert!(vm + .storage + .as_ref() + .borrow_mut() + .is_write_initial(&nonce_key)); + + let contract_code = read_test_contract(); + let tx = account.get_deploy_tx(&contract_code, None, TxType::L2).tx; + + vm.vm.push_transaction(tx); + vm.vm.execute(VmExecutionMode::OneTx); + + // Check that `is_write_initial` still returns true for the nonce key. + assert!(vm + .storage + .as_ref() + .borrow_mut() + .is_write_initial(&nonce_key)); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs new file mode 100644 index 000000000000..4f61dd90fad7 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs @@ -0,0 +1,139 @@ +use zksync_system_constants::BOOTLOADER_ADDRESS; +use zksync_types::{ + get_code_key, get_known_code_key, + l2_to_l1_log::{L2ToL1Log, UserL2ToL1Log}, + storage_writes_deduplicator::StorageWritesDeduplicator, + U256, +}; +use zksync_utils::u256_to_h256; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{TxType, VmTesterBuilder}, + utils::{read_test_contract, verify_required_storage, BASE_SYSTEM_CONTRACTS}, + }, + types::internals::TransactionData, + HistoryEnabled, + }, +}; + +#[test] +fn test_l1_tx_execution() { + // In this test, we try to execute a contract deployment from L1 + // Here instead of marking code hash via the bootloader means, we will be + // using L1->L2 communication, the same it would likely be done during the priority mode. + + // There are always at least 7 initial writes here, because we pay fees from l1: + // - totalSupply of ETH token + // - balance of the refund recipient + // - balance of the bootloader + // - tx_rolling hash + // - rolling hash of L2->L1 logs + // - transaction number in block counter + // - L2->L1 log counter in L1Messenger + + // TODO(PLA-537): right now we are using 4 slots instead of 7 due to 0 fee for transaction. + let basic_initial_writes = 4; + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_base_system_smart_contracts(BASE_SYSTEM_CONTRACTS.clone()) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let contract_code = read_test_contract(); + let account = &mut vm.rich_accounts[0]; + let deploy_tx = account.get_deploy_tx(&contract_code, None, TxType::L1 { serial_id: 1 }); + let tx_data: TransactionData = deploy_tx.tx.clone().into(); + + let required_l2_to_l1_logs: Vec<_> = vec![L2ToL1Log { + shard_id: 0, + is_service: true, + tx_number_in_block: 0, + sender: BOOTLOADER_ADDRESS, + key: tx_data.tx_hash(0.into()), + value: u256_to_h256(U256::from(1u32)), + }] + .into_iter() + .map(UserL2ToL1Log) + .collect(); + + vm.vm.push_transaction(deploy_tx.tx.clone()); + + let res = vm.vm.execute(VmExecutionMode::OneTx); + + // The code hash of the deployed contract should be marked as republished. + let known_codes_key = get_known_code_key(&deploy_tx.bytecode_hash); + + // The contract should be deployed successfully. + let account_code_key = get_code_key(&deploy_tx.address); + + let expected_slots = vec![ + (u256_to_h256(U256::from(1u32)), known_codes_key), + (deploy_tx.bytecode_hash, account_code_key), + ]; + assert!(!res.result.is_failed()); + + verify_required_storage(&vm.vm.state, expected_slots); + + assert_eq!(res.logs.user_l2_to_l1_logs, required_l2_to_l1_logs); + + let tx = account.get_test_contract_transaction( + deploy_tx.address, + true, + None, + false, + TxType::L1 { serial_id: 0 }, + ); + vm.vm.push_transaction(tx); + let res = vm.vm.execute(VmExecutionMode::OneTx); + let storage_logs = res.logs.storage_logs; + let res = StorageWritesDeduplicator::apply_on_empty_state(&storage_logs); + + // Tx panicked + assert_eq!(res.initial_storage_writes - basic_initial_writes, 0); + + let tx = account.get_test_contract_transaction( + deploy_tx.address, + false, + None, + false, + TxType::L1 { serial_id: 0 }, + ); + vm.vm.push_transaction(tx.clone()); + let res = vm.vm.execute(VmExecutionMode::OneTx); + let storage_logs = res.logs.storage_logs; + let res = StorageWritesDeduplicator::apply_on_empty_state(&storage_logs); + // We changed one slot inside contract + assert_eq!(res.initial_storage_writes - basic_initial_writes, 1); + + // No repeated writes + let repeated_writes = res.repeated_storage_writes; + assert_eq!(res.repeated_storage_writes, 0); + + vm.vm.push_transaction(tx); + let storage_logs = vm.vm.execute(VmExecutionMode::OneTx).logs.storage_logs; + let res = StorageWritesDeduplicator::apply_on_empty_state(&storage_logs); + // We do the same storage write, it will be deduplicated, so still 4 initial write and 0 repeated + assert_eq!(res.initial_storage_writes - basic_initial_writes, 1); + assert_eq!(res.repeated_storage_writes, repeated_writes); + + let tx = account.get_test_contract_transaction( + deploy_tx.address, + false, + Some(10.into()), + false, + TxType::L1 { serial_id: 1 }, + ); + vm.vm.push_transaction(tx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + // Method is not payable tx should fail + assert!(result.result.is_failed(), "The transaction should fail"); + + let res = StorageWritesDeduplicator::apply_on_empty_state(&result.logs.storage_logs); + // There are only basic initial writes + assert_eq!(res.initial_storage_writes - basic_initial_writes, 2); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs new file mode 100644 index 000000000000..1faeba9652f9 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs @@ -0,0 +1,437 @@ +//! +//! Tests for the bootloader +//! The description for each of the tests can be found in the corresponding `.yul` file. +//! + +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_state::WriteStorage; +use zksync_system_constants::REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE; +use zksync_types::{ + block::{pack_block_info, MiniblockHasher}, + AccountTreeId, Execute, ExecuteTransactionCommon, L1BatchNumber, L1TxCommonData, + MiniblockNumber, ProtocolVersionId, StorageKey, Transaction, H160, H256, + SYSTEM_CONTEXT_ADDRESS, SYSTEM_CONTEXT_BLOCK_INFO_POSITION, + SYSTEM_CONTEXT_CURRENT_L2_BLOCK_INFO_POSITION, SYSTEM_CONTEXT_CURRENT_TX_ROLLING_HASH_POSITION, + U256, +}; +use zksync_utils::{h256_to_u256, u256_to_h256}; + +use crate::{ + interface::{ExecutionResult, Halt, L2BlockEnv, TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + constants::{ + BOOTLOADER_HEAP_PAGE, TX_OPERATOR_L2_BLOCK_INFO_OFFSET, + TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO, + }, + tests::tester::{default_l1_batch, VmTesterBuilder}, + utils::l2_blocks::get_l2_block_hash_key, + HistoryEnabled, Vm, + }, + HistoryMode, +}; + +fn get_l1_noop() -> Transaction { + Transaction { + common_data: ExecuteTransactionCommon::L1(L1TxCommonData { + sender: H160::random(), + gas_limit: U256::from(2000000u32), + gas_per_pubdata_limit: REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE.into(), + ..Default::default() + }), + execute: Execute { + contract_address: H160::zero(), + calldata: vec![], + value: U256::zero(), + factory_deps: None, + }, + received_timestamp_ms: 0, + raw_bytes: None, + } +} + +#[test] +fn test_l2_block_initialization_timestamp() { + // This test checks that the L2 block initialization works correctly. + // Here we check that that the first block must have timestamp that is greater or equal to the timestamp + // of the current batch. + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + // Override the timestamp of the current miniblock to be 0. + vm.vm.bootloader_state.push_l2_block(L2BlockEnv { + number: 1, + timestamp: 0, + prev_block_hash: MiniblockHasher::legacy_hash(MiniblockNumber(0)), + max_virtual_blocks_to_create: 1, + }); + let l1_tx = get_l1_noop(); + + vm.vm.push_transaction(l1_tx); + let res = vm.vm.execute(VmExecutionMode::OneTx); + + assert_eq!( + res.result, + ExecutionResult::Halt {reason: Halt::FailedToSetL2Block("The timestamp of the L2 block must be greater than or equal to the timestamp of the current batch".to_string())} + ); +} + +#[test] +fn test_l2_block_initialization_number_non_zero() { + // This test checks that the L2 block initialization works correctly. + // Here we check that the first miniblock number can not be zero. + + let l1_batch = default_l1_batch(L1BatchNumber(1)); + let first_l2_block = L2BlockEnv { + number: 0, + timestamp: l1_batch.timestamp, + prev_block_hash: MiniblockHasher::legacy_hash(MiniblockNumber(0)), + max_virtual_blocks_to_create: 1, + }; + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_l1_batch_env(l1_batch) + .with_random_rich_accounts(1) + .build(); + + let l1_tx = get_l1_noop(); + + vm.vm.push_transaction(l1_tx); + + let timestamp = Timestamp(vm.vm.state.local_state.timestamp); + set_manual_l2_block_info(&mut vm.vm, 0, first_l2_block, timestamp); + + let res = vm.vm.execute(VmExecutionMode::OneTx); + + assert_eq!( + res.result, + ExecutionResult::Halt { + reason: Halt::FailedToSetL2Block( + "L2 block number is never expected to be zero".to_string() + ) + } + ); +} + +fn test_same_l2_block( + expected_error: Option, + override_timestamp: Option, + override_prev_block_hash: Option, +) { + let mut l1_batch = default_l1_batch(L1BatchNumber(1)); + l1_batch.timestamp = 1; + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_l1_batch_env(l1_batch) + .with_random_rich_accounts(1) + .build(); + + let l1_tx = get_l1_noop(); + vm.vm.push_transaction(l1_tx.clone()); + let res = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!res.result.is_failed()); + + let mut current_l2_block = vm.vm.batch_env.first_l2_block; + + if let Some(timestamp) = override_timestamp { + current_l2_block.timestamp = timestamp; + } + if let Some(prev_block_hash) = override_prev_block_hash { + current_l2_block.prev_block_hash = prev_block_hash; + } + + if (None, None) == (override_timestamp, override_prev_block_hash) { + current_l2_block.max_virtual_blocks_to_create = 0; + } + + vm.vm.push_transaction(l1_tx); + let timestamp = Timestamp(vm.vm.state.local_state.timestamp); + set_manual_l2_block_info(&mut vm.vm, 1, current_l2_block, timestamp); + + let result = vm.vm.execute(VmExecutionMode::OneTx); + + if let Some(err) = expected_error { + assert_eq!(result.result, ExecutionResult::Halt { reason: err }); + } else { + assert_eq!(result.result, ExecutionResult::Success { output: vec![] }); + } +} + +#[test] +fn test_l2_block_same_l2_block() { + // This test aims to test the case when there are multiple transactions inside the same L2 block. + + // Case 1: Incorrect timestamp + test_same_l2_block( + Some(Halt::FailedToSetL2Block( + "The timestamp of the same L2 block must be same".to_string(), + )), + Some(0), + None, + ); + + // Case 2: Incorrect previous block hash + test_same_l2_block( + Some(Halt::FailedToSetL2Block( + "The previous hash of the same L2 block must be same".to_string(), + )), + None, + Some(H256::zero()), + ); + + // Case 3: Correct continuation of the same L2 block + test_same_l2_block(None, None, None); +} + +fn test_new_l2_block( + first_l2_block: L2BlockEnv, + overriden_second_block_number: Option, + overriden_second_block_timestamp: Option, + overriden_second_block_prev_block_hash: Option, + expected_error: Option, +) { + let mut l1_batch = default_l1_batch(L1BatchNumber(1)); + l1_batch.timestamp = 1; + l1_batch.first_l2_block = first_l2_block; + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_l1_batch_env(l1_batch) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let l1_tx = get_l1_noop(); + + // Firstly we execute the first transaction + vm.vm.push_transaction(l1_tx.clone()); + vm.vm.execute(VmExecutionMode::OneTx); + + let mut second_l2_block = vm.vm.batch_env.first_l2_block; + second_l2_block.number += 1; + second_l2_block.timestamp += 1; + second_l2_block.prev_block_hash = vm.vm.bootloader_state.last_l2_block().get_hash(); + + if let Some(block_number) = overriden_second_block_number { + second_l2_block.number = block_number; + } + if let Some(timestamp) = overriden_second_block_timestamp { + second_l2_block.timestamp = timestamp; + } + if let Some(prev_block_hash) = overriden_second_block_prev_block_hash { + second_l2_block.prev_block_hash = prev_block_hash; + } + + vm.vm.bootloader_state.push_l2_block(second_l2_block); + + vm.vm.push_transaction(l1_tx); + + let result = vm.vm.execute(VmExecutionMode::OneTx); + if let Some(err) = expected_error { + assert_eq!(result.result, ExecutionResult::Halt { reason: err }); + } else { + assert_eq!(result.result, ExecutionResult::Success { output: vec![] }); + } +} + +#[test] +fn test_l2_block_new_l2_block() { + // This test is aimed to cover potential issue + + let correct_first_block = L2BlockEnv { + number: 1, + timestamp: 1, + prev_block_hash: MiniblockHasher::legacy_hash(MiniblockNumber(0)), + max_virtual_blocks_to_create: 1, + }; + + // Case 1: Block number increasing by more than 1 + test_new_l2_block( + correct_first_block, + Some(3), + None, + None, + Some(Halt::FailedToSetL2Block( + "Invalid new L2 block number".to_string(), + )), + ); + + // Case 2: Timestamp not increasing + test_new_l2_block( + correct_first_block, + None, + Some(1), + None, + Some(Halt::FailedToSetL2Block("The timestamp of the new L2 block must be greater than the timestamp of the previous L2 block".to_string())), + ); + + // Case 3: Incorrect previous block hash + test_new_l2_block( + correct_first_block, + None, + None, + Some(H256::zero()), + Some(Halt::FailedToSetL2Block( + "The current L2 block hash is incorrect".to_string(), + )), + ); + + // Case 4: Correct new block + test_new_l2_block(correct_first_block, None, None, None, None); +} + +#[allow(clippy::too_many_arguments)] +fn test_first_in_batch( + miniblock_timestamp: u64, + miniblock_number: u32, + pending_txs_hash: H256, + batch_timestamp: u64, + new_batch_timestamp: u64, + batch_number: u32, + proposed_block: L2BlockEnv, + expected_error: Option, +) { + let mut l1_batch = default_l1_batch(L1BatchNumber(1)); + l1_batch.number += 1; + l1_batch.timestamp = new_batch_timestamp; + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_l1_batch_env(l1_batch) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + let l1_tx = get_l1_noop(); + + // Setting the values provided. + let storage_ptr = vm.vm.state.storage.storage.get_ptr(); + let miniblock_info_slot = StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + SYSTEM_CONTEXT_CURRENT_L2_BLOCK_INFO_POSITION, + ); + let pending_txs_hash_slot = StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + SYSTEM_CONTEXT_CURRENT_TX_ROLLING_HASH_POSITION, + ); + let batch_info_slot = StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + SYSTEM_CONTEXT_BLOCK_INFO_POSITION, + ); + let prev_block_hash_position = get_l2_block_hash_key(miniblock_number - 1); + + storage_ptr.borrow_mut().set_value( + miniblock_info_slot, + u256_to_h256(pack_block_info( + miniblock_number as u64, + miniblock_timestamp, + )), + ); + storage_ptr + .borrow_mut() + .set_value(pending_txs_hash_slot, pending_txs_hash); + storage_ptr.borrow_mut().set_value( + batch_info_slot, + u256_to_h256(pack_block_info(batch_number as u64, batch_timestamp)), + ); + storage_ptr.borrow_mut().set_value( + prev_block_hash_position, + MiniblockHasher::legacy_hash(MiniblockNumber(miniblock_number - 1)), + ); + + // In order to skip checks from the Rust side of the VM, we firstly use some definitely correct L2 block info. + // And then override it with the user-provided value + + let last_l2_block = vm.vm.bootloader_state.last_l2_block(); + let new_l2_block = L2BlockEnv { + number: last_l2_block.number + 1, + timestamp: last_l2_block.timestamp + 1, + prev_block_hash: last_l2_block.get_hash(), + max_virtual_blocks_to_create: last_l2_block.max_virtual_blocks_to_create, + }; + + vm.vm.bootloader_state.push_l2_block(new_l2_block); + vm.vm.push_transaction(l1_tx); + let timestamp = Timestamp(vm.vm.state.local_state.timestamp); + set_manual_l2_block_info(&mut vm.vm, 0, proposed_block, timestamp); + + let result = vm.vm.execute(VmExecutionMode::OneTx); + if let Some(err) = expected_error { + assert_eq!(result.result, ExecutionResult::Halt { reason: err }); + } else { + assert_eq!(result.result, ExecutionResult::Success { output: vec![] }); + } +} + +#[test] +fn test_l2_block_first_in_batch() { + let prev_block_hash = MiniblockHasher::legacy_hash(MiniblockNumber(0)); + let prev_block_hash = MiniblockHasher::new(MiniblockNumber(1), 1, prev_block_hash) + .finalize(ProtocolVersionId::latest()); + test_first_in_batch( + 1, + 1, + H256::zero(), + 1, + 2, + 1, + L2BlockEnv { + number: 2, + timestamp: 2, + prev_block_hash, + max_virtual_blocks_to_create: 1, + }, + None, + ); + + let prev_block_hash = MiniblockHasher::legacy_hash(MiniblockNumber(0)); + let prev_block_hash = MiniblockHasher::new(MiniblockNumber(1), 8, prev_block_hash) + .finalize(ProtocolVersionId::latest()); + test_first_in_batch( + 8, + 1, + H256::zero(), + 5, + 12, + 1, + L2BlockEnv { + number: 2, + timestamp: 9, + prev_block_hash, + max_virtual_blocks_to_create: 1, + }, + Some(Halt::FailedToSetL2Block("The timestamp of the L2 block must be greater than or equal to the timestamp of the current batch".to_string())), + ); +} + +fn set_manual_l2_block_info( + vm: &mut Vm, + tx_number: usize, + block_info: L2BlockEnv, + timestamp: Timestamp, +) { + let fictive_miniblock_position = + TX_OPERATOR_L2_BLOCK_INFO_OFFSET + TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO * tx_number; + + vm.state.memory.populate_page( + BOOTLOADER_HEAP_PAGE as usize, + vec![ + (fictive_miniblock_position, block_info.number.into()), + (fictive_miniblock_position + 1, block_info.timestamp.into()), + ( + fictive_miniblock_position + 2, + h256_to_u256(block_info.prev_block_hash), + ), + ( + fictive_miniblock_position + 3, + block_info.max_virtual_blocks_to_create.into(), + ), + ], + timestamp, + ) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs new file mode 100644 index 000000000000..ffb38dd3725a --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs @@ -0,0 +1,20 @@ +mod bootloader; +mod default_aa; +// TODO - fix this test +// mod invalid_bytecode; +mod bytecode_publishing; +mod call_tracer; +mod gas_limit; +mod get_used_contracts; +mod is_write_initial; +mod l1_tx_execution; +mod l2_blocks; +mod nonce_holder; +mod refunds; +mod require_eip712; +mod rollbacks; +mod simple_execution; +mod tester; +mod tracing_execution_error; +mod upgrade; +mod utils; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs new file mode 100644 index 000000000000..2de5e23bdd23 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs @@ -0,0 +1,188 @@ +use zksync_types::{Execute, Nonce}; + +use crate::{ + interface::{ + ExecutionResult, Halt, TxExecutionMode, TxRevertReason, VmExecutionMode, VmInterface, + VmRevertReason, + }, + vm_latest::{ + tests::{ + tester::{Account, VmTesterBuilder}, + utils::read_nonce_holder_tester, + }, + types::internals::TransactionData, + HistoryEnabled, + }, +}; + +pub enum NonceHolderTestMode { + SetValueUnderNonce, + IncreaseMinNonceBy5, + IncreaseMinNonceTooMuch, + LeaveNonceUnused, + IncreaseMinNonceBy1, + SwitchToArbitraryOrdering, +} + +impl From for u8 { + fn from(mode: NonceHolderTestMode) -> u8 { + match mode { + NonceHolderTestMode::SetValueUnderNonce => 0, + NonceHolderTestMode::IncreaseMinNonceBy5 => 1, + NonceHolderTestMode::IncreaseMinNonceTooMuch => 2, + NonceHolderTestMode::LeaveNonceUnused => 3, + NonceHolderTestMode::IncreaseMinNonceBy1 => 4, + NonceHolderTestMode::SwitchToArbitraryOrdering => 5, + } + } +} + +#[test] +fn test_nonce_holder() { + let mut account = Account::random(); + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_deployer() + .with_custom_contracts(vec![( + read_nonce_holder_tester().to_vec(), + account.address, + true, + )]) + .with_rich_accounts(vec![account.clone()]) + .build(); + + let mut run_nonce_test = |nonce: u32, + test_mode: NonceHolderTestMode, + error_message: Option, + comment: &'static str| { + // In this test we have to reset VM state after each test case. Because once bootloader failed during the validation of the transaction, + // it will fail again and again. At the same time we have to keep the same storage, because we want to keep the nonce holder contract state. + // The easiest way in terms of lifetimes is to reuse vm_builder to achieve it. + vm.reset_state(true); + let mut transaction_data: TransactionData = account + .get_l2_tx_for_execute_with_nonce( + Execute { + contract_address: account.address, + calldata: vec![12], + value: Default::default(), + factory_deps: None, + }, + None, + Nonce(nonce), + ) + .into(); + + transaction_data.signature = vec![test_mode.into()]; + vm.vm.push_raw_transaction(transaction_data, 0, 0, true); + let result = vm.vm.execute(VmExecutionMode::OneTx); + + if let Some(msg) = error_message { + let expected_error = + TxRevertReason::Halt(Halt::ValidationFailed(VmRevertReason::General { + msg, + data: vec![], + })); + let ExecutionResult::Halt { reason } = result.result else { + panic!("Expected revert, got {:?}", result.result); + }; + assert_eq!( + reason.to_string(), + expected_error.to_string(), + "{}", + comment + ); + } else { + assert!(!result.result.is_failed(), "{}", comment); + } + }; + // Test 1: trying to set value under non sequential nonce value. + run_nonce_test( + 1u32, + NonceHolderTestMode::SetValueUnderNonce, + Some("Previous nonce has not been used".to_string()), + "Allowed to set value under non sequential value", + ); + + // Test 2: increase min nonce by 1 with sequential nonce ordering: + run_nonce_test( + 0u32, + NonceHolderTestMode::IncreaseMinNonceBy1, + None, + "Failed to increment nonce by 1 for sequential account", + ); + + // Test 3: correctly set value under nonce with sequential nonce ordering: + run_nonce_test( + 1u32, + NonceHolderTestMode::SetValueUnderNonce, + None, + "Failed to set value under nonce sequential value", + ); + + // Test 5: migrate to the arbitrary nonce ordering: + run_nonce_test( + 2u32, + NonceHolderTestMode::SwitchToArbitraryOrdering, + None, + "Failed to switch to arbitrary ordering", + ); + + // Test 6: increase min nonce by 5 + run_nonce_test( + 6u32, + NonceHolderTestMode::IncreaseMinNonceBy5, + None, + "Failed to increase min nonce by 5", + ); + + // Test 7: since the nonces in range [6,10] are no longer allowed, the + // tx with nonce 10 should not be allowed + run_nonce_test( + 10u32, + NonceHolderTestMode::IncreaseMinNonceBy5, + Some("Reusing the same nonce twice".to_string()), + "Allowed to reuse nonce below the minimal one", + ); + + // Test 8: we should be able to use nonce 13 + run_nonce_test( + 13u32, + NonceHolderTestMode::SetValueUnderNonce, + None, + "Did not allow to use unused nonce 10", + ); + + // Test 9: we should not be able to reuse nonce 13 + run_nonce_test( + 13u32, + NonceHolderTestMode::IncreaseMinNonceBy5, + Some("Reusing the same nonce twice".to_string()), + "Allowed to reuse the same nonce twice", + ); + + // Test 10: we should be able to simply use nonce 14, while bumping the minimal nonce by 5 + run_nonce_test( + 14u32, + NonceHolderTestMode::IncreaseMinNonceBy5, + None, + "Did not allow to use a bumped nonce", + ); + + // Test 11: Do not allow bumping nonce by too much + run_nonce_test( + 16u32, + NonceHolderTestMode::IncreaseMinNonceTooMuch, + Some("The value for incrementing the nonce is too high".to_string()), + "Allowed for incrementing min nonce too much", + ); + + // Test 12: Do not allow not setting a nonce as used + run_nonce_test( + 16u32, + NonceHolderTestMode::LeaveNonceUnused, + Some("The nonce was not set as used".to_string()), + "Allowed to leave nonce as unused", + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs new file mode 100644 index 000000000000..dc1f4fe55bca --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs @@ -0,0 +1,167 @@ +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{DeployContractsTx, TxType, VmTesterBuilder}, + utils::read_test_contract, + }, + types::internals::TransactionData, + HistoryEnabled, + }, +}; + +#[test] +fn test_predetermined_refunded_gas() { + // In this test, we compare the execution of the bootloader with the predefined + // refunded gas and without them + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + let l1_batch = vm.vm.batch_env.clone(); + + let counter = read_test_contract(); + let account = &mut vm.rich_accounts[0]; + + let DeployContractsTx { + tx, + bytecode_hash: _, + address: _, + } = account.get_deploy_tx(&counter, None, TxType::L2); + vm.vm.push_transaction(tx.clone()); + let result = vm.vm.execute(VmExecutionMode::OneTx); + + assert!(!result.result.is_failed()); + + // If the refund provided by the operator or the final refund are the 0 + // there is no impact of the operator's refund at all and so this test does not + // make much sense. + assert!( + result.refunds.operator_suggested_refund > 0, + "The operator's refund is 0" + ); + assert!(result.refunds.gas_refunded > 0, "The final refund is 0"); + + let result_without_predefined_refunds = vm.vm.execute(VmExecutionMode::Batch); + let mut current_state_without_predefined_refunds = vm.vm.get_current_execution_state(); + assert!(!result_without_predefined_refunds.result.is_failed(),); + + // Here we want to provide the same refund from the operator and check that it's the correct one. + // We execute the whole block without refund tracer, because refund tracer will eventually override the provided refund. + // But the overall result should be the same + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_l1_batch_env(l1_batch.clone()) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_rich_accounts(vec![account.clone()]) + .build(); + + let tx: TransactionData = tx.into(); + let block_gas_per_pubdata_byte = vm.vm.batch_env.block_gas_price_per_pubdata(); + // Overhead + let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); + vm.vm + .push_raw_transaction(tx.clone(), overhead, result.refunds.gas_refunded, true); + + let result_with_predefined_refunds = vm.vm.execute(VmExecutionMode::Batch); + let mut current_state_with_predefined_refunds = vm.vm.get_current_execution_state(); + + assert!(!result_with_predefined_refunds.result.is_failed()); + + // We need to sort these lists as those are flattened from HashMaps + current_state_with_predefined_refunds + .used_contract_hashes + .sort(); + current_state_without_predefined_refunds + .used_contract_hashes + .sort(); + + assert_eq!( + current_state_with_predefined_refunds.events, + current_state_without_predefined_refunds.events + ); + + assert_eq!( + current_state_with_predefined_refunds.user_l2_to_l1_logs, + current_state_without_predefined_refunds.user_l2_to_l1_logs + ); + + assert_eq!( + current_state_with_predefined_refunds.system_logs, + current_state_without_predefined_refunds.system_logs + ); + + assert_eq!( + current_state_with_predefined_refunds.storage_log_queries, + current_state_without_predefined_refunds.storage_log_queries + ); + assert_eq!( + current_state_with_predefined_refunds.used_contract_hashes, + current_state_without_predefined_refunds.used_contract_hashes + ); + + // In this test we put the different refund from the operator. + // We still can't use the refund tracer, because it will override the refund. + // But we can check that the logs and events have changed. + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_l1_batch_env(l1_batch) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_rich_accounts(vec![account.clone()]) + .build(); + + let changed_operator_suggested_refund = result.refunds.gas_refunded + 1000; + vm.vm + .push_raw_transaction(tx, overhead, changed_operator_suggested_refund, true); + let result = vm.vm.execute(VmExecutionMode::Batch); + let mut current_state_with_changed_predefined_refunds = vm.vm.get_current_execution_state(); + + assert!(!result.result.is_failed()); + current_state_with_changed_predefined_refunds + .used_contract_hashes + .sort(); + current_state_without_predefined_refunds + .used_contract_hashes + .sort(); + + assert_eq!( + current_state_with_changed_predefined_refunds.events.len(), + current_state_without_predefined_refunds.events.len() + ); + + assert_ne!( + current_state_with_changed_predefined_refunds.events, + current_state_without_predefined_refunds.events + ); + + assert_eq!( + current_state_with_changed_predefined_refunds.user_l2_to_l1_logs, + current_state_without_predefined_refunds.user_l2_to_l1_logs + ); + + assert_ne!( + current_state_with_changed_predefined_refunds.system_logs, + current_state_without_predefined_refunds.system_logs + ); + + assert_eq!( + current_state_with_changed_predefined_refunds + .storage_log_queries + .len(), + current_state_without_predefined_refunds + .storage_log_queries + .len() + ); + + assert_ne!( + current_state_with_changed_predefined_refunds.storage_log_queries, + current_state_without_predefined_refunds.storage_log_queries + ); + assert_eq!( + current_state_with_changed_predefined_refunds.used_contract_hashes, + current_state_without_predefined_refunds.used_contract_hashes + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs new file mode 100644 index 000000000000..c03e5fe64212 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs @@ -0,0 +1,165 @@ +use std::convert::TryInto; + +use ethabi::Token; +use zksync_eth_signer::{raw_ethereum_tx::TransactionParameters, EthereumSigner}; +use zksync_system_constants::L2_ETH_TOKEN_ADDRESS; +use zksync_types::{ + fee::Fee, l2::L2Tx, transaction_request::TransactionRequest, + utils::storage_key_for_standard_token_balance, AccountTreeId, Address, Eip712Domain, Execute, + L2ChainId, Nonce, Transaction, U256, +}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::{ + tester::{Account, VmTester, VmTesterBuilder}, + utils::read_many_owners_custom_account_contract, + }, + HistoryDisabled, + }, +}; + +impl VmTester { + pub(crate) fn get_eth_balance(&mut self, address: Address) -> U256 { + let key = storage_key_for_standard_token_balance( + AccountTreeId::new(L2_ETH_TOKEN_ADDRESS), + &address, + ); + self.vm.state.storage.storage.read_from_storage(&key) + } +} + +// TODO refactor this test it use too much internal details of the VM +#[tokio::test] +/// This test deploys 'buggy' account abstraction code, and then tries accessing it both with legacy +/// and EIP712 transactions. +/// Currently we support both, but in the future, we should allow only EIP712 transactions to access the AA accounts. +async fn test_require_eip712() { + // Use 3 accounts: + // - private_address - EOA account, where we have the key + // - account_address - AA account, where the contract is deployed + // - beneficiary - an EOA account, where we'll try to transfer the tokens. + let account_abstraction = Account::random(); + let mut private_account = Account::random(); + let beneficiary = Account::random(); + + let (bytecode, contract) = read_many_owners_custom_account_contract(); + let mut vm = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_custom_contracts(vec![(bytecode, account_abstraction.address, true)]) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_rich_accounts(vec![account_abstraction.clone(), private_account.clone()]) + .build(); + + assert_eq!(vm.get_eth_balance(beneficiary.address), U256::from(0)); + + let chain_id: u32 = 270; + + // First, let's set the owners of the AA account to the private_address. + // (so that messages signed by private_address, are authorized to act on behalf of the AA account). + let set_owners_function = contract.function("setOwners").unwrap(); + let encoded_input = set_owners_function + .encode_input(&[Token::Array(vec![Token::Address(private_account.address)])]) + .unwrap(); + + let tx = private_account.get_l2_tx_for_execute( + Execute { + contract_address: account_abstraction.address, + calldata: encoded_input, + value: Default::default(), + factory_deps: None, + }, + None, + ); + + vm.vm.push_transaction(tx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed()); + + let private_account_balance = vm.get_eth_balance(private_account.address); + + // And now let's do the transfer from the 'account abstraction' to 'beneficiary' (using 'legacy' transaction). + // Normally this would not work - unless the operator is malicious. + let aa_raw_tx = TransactionParameters { + nonce: U256::from(0), + to: Some(beneficiary.address), + gas: U256::from(100000000), + gas_price: Some(U256::from(10000000)), + value: U256::from(888000088), + data: vec![], + chain_id: 270, + transaction_type: None, + access_list: None, + max_fee_per_gas: U256::from(1000000000), + max_priority_fee_per_gas: U256::from(1000000000), + }; + + let aa_tx = private_account.sign_legacy_tx(aa_raw_tx).await; + let (tx_request, hash) = TransactionRequest::from_bytes(&aa_tx, L2ChainId::from(270)).unwrap(); + + let mut l2_tx: L2Tx = L2Tx::from_request(tx_request, 10000).unwrap(); + l2_tx.set_input(aa_tx, hash); + // Pretend that operator is malicious and sets the initiator to the AA account. + l2_tx.common_data.initiator_address = account_abstraction.address; + let transaction: Transaction = l2_tx.try_into().unwrap(); + + vm.vm.push_transaction(transaction); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed()); + assert_eq!( + vm.get_eth_balance(beneficiary.address), + U256::from(888000088) + ); + // Make sure that the tokens were transferred from the AA account. + assert_eq!( + private_account_balance, + vm.get_eth_balance(private_account.address) + ); + + // // Now send the 'classic' EIP712 transaction + let tx_712 = L2Tx::new( + beneficiary.address, + vec![], + Nonce(1), + Fee { + gas_limit: U256::from(1000000000), + max_fee_per_gas: U256::from(1000000000), + max_priority_fee_per_gas: U256::from(1000000000), + gas_per_pubdata_limit: U256::from(1000000000), + }, + account_abstraction.address, + U256::from(28374938), + None, + Default::default(), + ); + + let transaction_request: TransactionRequest = tx_712.into(); + + let domain = Eip712Domain::new(L2ChainId::from(chain_id)); + let signature = private_account + .get_pk_signer() + .sign_typed_data(&domain, &transaction_request) + .await + .unwrap(); + let encoded_tx = transaction_request.get_signed_bytes(&signature, L2ChainId::from(chain_id)); + + let (aa_txn_request, aa_hash) = + TransactionRequest::from_bytes(&encoded_tx, L2ChainId::from(chain_id)).unwrap(); + + let mut l2_tx = L2Tx::from_request(aa_txn_request, 100000).unwrap(); + l2_tx.set_input(encoded_tx, aa_hash); + + let transaction: Transaction = l2_tx.try_into().unwrap(); + vm.vm.push_transaction(transaction); + vm.vm.execute(VmExecutionMode::OneTx); + + assert_eq!( + vm.get_eth_balance(beneficiary.address), + U256::from(916375026) + ); + assert_eq!( + private_account_balance, + vm.get_eth_balance(private_account.address) + ); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs new file mode 100644 index 000000000000..23c1ab49ad98 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs @@ -0,0 +1,263 @@ +use ethabi::Token; +use zksync_contracts::{get_loadnext_contract, test_contracts::LoadnextContractExecutionParams}; +use zksync_state::WriteStorage; +use zksync_types::{get_nonce_key, Execute, U256}; + +use crate::{ + interface::{ + dyn_tracers::vm_1_4_0::DynTracer, + tracer::{TracerExecutionStatus, TracerExecutionStopReason}, + TxExecutionMode, VmExecutionMode, VmInterface, VmInterfaceHistoryEnabled, + }, + vm_latest::{ + tests::{ + tester::{DeployContractsTx, TransactionTestInfo, TxModifier, TxType, VmTesterBuilder}, + utils::read_test_contract, + }, + types::internals::ZkSyncVmState, + BootloaderState, HistoryEnabled, HistoryMode, SimpleMemory, ToTracerPointer, VmTracer, + }, +}; + +#[test] +fn test_vm_rollbacks() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let mut account = vm.rich_accounts[0].clone(); + let counter = read_test_contract(); + let tx_0 = account.get_deploy_tx(&counter, None, TxType::L2).tx; + let tx_1 = account.get_deploy_tx(&counter, None, TxType::L2).tx; + let tx_2 = account.get_deploy_tx(&counter, None, TxType::L2).tx; + + let result_without_rollbacks = vm.execute_and_verify_txs(&vec![ + TransactionTestInfo::new_processed(tx_0.clone(), false), + TransactionTestInfo::new_processed(tx_1.clone(), false), + TransactionTestInfo::new_processed(tx_2.clone(), false), + ]); + + // reset vm + vm.reset_with_empty_storage(); + + let result_with_rollbacks = vm.execute_and_verify_txs(&vec![ + TransactionTestInfo::new_rejected(tx_0.clone(), TxModifier::WrongSignatureLength.into()), + TransactionTestInfo::new_rejected(tx_0.clone(), TxModifier::WrongMagicValue.into()), + TransactionTestInfo::new_rejected(tx_0.clone(), TxModifier::WrongSignature.into()), + // The correct nonce is 0, this tx will fail + TransactionTestInfo::new_rejected(tx_2.clone(), TxModifier::WrongNonce.into()), + // This tx will succeed + TransactionTestInfo::new_processed(tx_0.clone(), false), + // The correct nonce is 1, this tx will fail + TransactionTestInfo::new_rejected(tx_0.clone(), TxModifier::NonceReused.into()), + // The correct nonce is 1, this tx will fail + TransactionTestInfo::new_rejected(tx_2.clone(), TxModifier::WrongNonce.into()), + // This tx will succeed + TransactionTestInfo::new_processed(tx_1, false), + // The correct nonce is 2, this tx will fail + TransactionTestInfo::new_rejected(tx_0.clone(), TxModifier::NonceReused.into()), + // This tx will succeed + TransactionTestInfo::new_processed(tx_2.clone(), false), + // This tx will fail + TransactionTestInfo::new_rejected(tx_2, TxModifier::NonceReused.into()), + TransactionTestInfo::new_rejected(tx_0, TxModifier::NonceReused.into()), + ]); + + assert_eq!(result_without_rollbacks, result_with_rollbacks); +} + +#[test] +fn test_vm_loadnext_rollbacks() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + let mut account = vm.rich_accounts[0].clone(); + + let loadnext_contract = get_loadnext_contract(); + let loadnext_constructor_data = &[Token::Uint(U256::from(100))]; + let DeployContractsTx { + tx: loadnext_deploy_tx, + address, + .. + } = account.get_deploy_tx_with_factory_deps( + &loadnext_contract.bytecode, + Some(loadnext_constructor_data), + loadnext_contract.factory_deps.clone(), + TxType::L2, + ); + + let loadnext_tx_1 = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: LoadnextContractExecutionParams { + reads: 100, + writes: 100, + events: 100, + hashes: 500, + recursive_calls: 10, + deploys: 60, + } + .to_bytes(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + + let loadnext_tx_2 = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: LoadnextContractExecutionParams { + reads: 100, + writes: 100, + events: 100, + hashes: 500, + recursive_calls: 10, + deploys: 60, + } + .to_bytes(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + + let result_without_rollbacks = vm.execute_and_verify_txs(&vec![ + TransactionTestInfo::new_processed(loadnext_deploy_tx.clone(), false), + TransactionTestInfo::new_processed(loadnext_tx_1.clone(), false), + TransactionTestInfo::new_processed(loadnext_tx_2.clone(), false), + ]); + + // reset vm + vm.reset_with_empty_storage(); + + let result_with_rollbacks = vm.execute_and_verify_txs(&vec![ + TransactionTestInfo::new_processed(loadnext_deploy_tx.clone(), false), + TransactionTestInfo::new_processed(loadnext_tx_1.clone(), true), + TransactionTestInfo::new_rejected( + loadnext_deploy_tx.clone(), + TxModifier::NonceReused.into(), + ), + TransactionTestInfo::new_processed(loadnext_tx_1, false), + TransactionTestInfo::new_processed(loadnext_tx_2.clone(), true), + TransactionTestInfo::new_processed(loadnext_tx_2.clone(), true), + TransactionTestInfo::new_rejected(loadnext_deploy_tx, TxModifier::NonceReused.into()), + TransactionTestInfo::new_processed(loadnext_tx_2, false), + ]); + + assert_eq!(result_without_rollbacks, result_with_rollbacks); +} + +// Testing tracer that does not allow the recursion to go deeper than a certain limit +struct MaxRecursionTracer { + max_recursion_depth: usize, +} + +/// Tracer responsible for calculating the number of storage invocations and +/// stopping the VM execution if the limit is reached. +impl DynTracer> for MaxRecursionTracer {} + +impl VmTracer for MaxRecursionTracer { + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + let current_depth = state.local_state.callstack.depth(); + + if current_depth > self.max_recursion_depth { + TracerExecutionStatus::Stop(TracerExecutionStopReason::Finish) + } else { + TracerExecutionStatus::Continue + } + } +} + +#[test] +fn test_layered_rollback() { + // This test checks that the layered rollbacks work correctly, i.e. + // the rollback by the operator will always revert all the changes + + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let account = &mut vm.rich_accounts[0]; + let loadnext_contract = get_loadnext_contract().bytecode; + + let DeployContractsTx { + tx: deploy_tx, + address, + .. + } = account.get_deploy_tx( + &loadnext_contract, + Some(&[Token::Uint(0.into())]), + TxType::L2, + ); + vm.vm.push_transaction(deploy_tx); + let deployment_res = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!deployment_res.result.is_failed(), "transaction failed"); + + let loadnext_transaction = account.get_loadnext_transaction( + address, + LoadnextContractExecutionParams { + writes: 1, + recursive_calls: 20, + ..LoadnextContractExecutionParams::empty() + }, + TxType::L2, + ); + + let nonce_val = vm + .vm + .state + .storage + .storage + .read_from_storage(&get_nonce_key(&account.address)); + + vm.vm.make_snapshot(); + + vm.vm.push_transaction(loadnext_transaction.clone()); + vm.vm.inspect( + MaxRecursionTracer { + max_recursion_depth: 15, + } + .into_tracer_pointer() + .into(), + VmExecutionMode::OneTx, + ); + + let nonce_val2 = vm + .vm + .state + .storage + .storage + .read_from_storage(&get_nonce_key(&account.address)); + + // The tracer stopped after the validation has passed, so nonce has already been increased + assert_eq!(nonce_val + U256::one(), nonce_val2, "nonce did not change"); + + vm.vm.rollback_to_the_latest_snapshot(); + + let nonce_val_after_rollback = vm + .vm + .state + .storage + .storage + .read_from_storage(&get_nonce_key(&account.address)); + + assert_eq!( + nonce_val, nonce_val_after_rollback, + "nonce changed after rollback" + ); + + vm.vm.push_transaction(loadnext_transaction); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed(), "transaction must not fail"); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs new file mode 100644 index 000000000000..a864538524a2 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs @@ -0,0 +1,81 @@ +use crate::{ + interface::{ExecutionResult, VmExecutionMode, VmInterface}, + vm_latest::{ + tests::tester::{TxType, VmTesterBuilder}, + HistoryDisabled, + }, +}; + +#[test] +fn estimate_fee() { + let mut vm_tester = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_deployer() + .with_random_rich_accounts(1) + .build(); + + vm_tester.deploy_test_contract(); + let account = &mut vm_tester.rich_accounts[0]; + + let tx = account.get_test_contract_transaction( + vm_tester.test_contract.unwrap(), + false, + Default::default(), + false, + TxType::L2, + ); + + vm_tester.vm.push_transaction(tx); + + let result = vm_tester.vm.execute(VmExecutionMode::OneTx); + assert!(matches!(result.result, ExecutionResult::Success { .. })); +} + +#[test] +fn simple_execute() { + let mut vm_tester = VmTesterBuilder::new(HistoryDisabled) + .with_empty_in_memory_storage() + .with_deployer() + .with_random_rich_accounts(1) + .build(); + + vm_tester.deploy_test_contract(); + + let account = &mut vm_tester.rich_accounts[0]; + + let tx1 = account.get_test_contract_transaction( + vm_tester.test_contract.unwrap(), + false, + Default::default(), + false, + TxType::L1 { serial_id: 1 }, + ); + + let tx2 = account.get_test_contract_transaction( + vm_tester.test_contract.unwrap(), + true, + Default::default(), + false, + TxType::L1 { serial_id: 1 }, + ); + + let tx3 = account.get_test_contract_transaction( + vm_tester.test_contract.unwrap(), + false, + Default::default(), + false, + TxType::L1 { serial_id: 1 }, + ); + let vm = &mut vm_tester.vm; + vm.push_transaction(tx1); + vm.push_transaction(tx2); + vm.push_transaction(tx3); + let tx = vm.execute(VmExecutionMode::OneTx); + assert!(matches!(tx.result, ExecutionResult::Success { .. })); + let tx = vm.execute(VmExecutionMode::OneTx); + assert!(matches!(tx.result, ExecutionResult::Revert { .. })); + let tx = vm.execute(VmExecutionMode::OneTx); + assert!(matches!(tx.result, ExecutionResult::Success { .. })); + let block_tip = vm.execute(VmExecutionMode::Batch); + assert!(matches!(block_tip.result, ExecutionResult::Success { .. })); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs new file mode 100644 index 000000000000..b82e995c2db3 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs @@ -0,0 +1,130 @@ +use std::collections::HashMap; + +use zk_evm_1_4_0::{aux_structures::Timestamp, vm_state::VmLocalState}; +use zksync_state::WriteStorage; +use zksync_types::{StorageKey, StorageLogQuery, StorageValue, U256}; + +use crate::{ + vm_latest::{ + old_vm::{ + event_sink::InMemoryEventSink, + history_recorder::{AppDataFrameManagerWithHistory, HistoryRecorder}, + }, + HistoryEnabled, HistoryMode, SimpleMemory, Vm, + }, + HistoryMode as CommonHistoryMode, +}; + +#[derive(Clone, Debug)] +pub(crate) struct ModifiedKeysMap(HashMap); + +// We consider hashmaps to be equal even if there is a key +// that is not present in one but has zero value in another. +impl PartialEq for ModifiedKeysMap { + fn eq(&self, other: &Self) -> bool { + for (key, value) in self.0.iter() { + if *value != other.0.get(key).cloned().unwrap_or_default() { + return false; + } + } + for (key, value) in other.0.iter() { + if *value != self.0.get(key).cloned().unwrap_or_default() { + return false; + } + } + true + } +} + +#[derive(Clone, PartialEq, Debug)] +pub(crate) struct DecommitterTestInnerState { + /// There is no way to "truly" compare the storage pointer, + /// so we just compare the modified keys. This is reasonable enough. + pub(crate) modified_storage_keys: ModifiedKeysMap, + pub(crate) known_bytecodes: HistoryRecorder>, H>, + pub(crate) decommitted_code_hashes: HistoryRecorder, HistoryEnabled>, +} + +#[derive(Clone, PartialEq, Debug)] +pub(crate) struct StorageOracleInnerState { + /// There is no way to "truly" compare the storage pointer, + /// so we just compare the modified keys. This is reasonable enough. + pub(crate) modified_storage_keys: ModifiedKeysMap, + + pub(crate) frames_stack: AppDataFrameManagerWithHistory, H>, + + pub(crate) pre_paid_changes: HistoryRecorder, H>, + pub(crate) paid_changes: HistoryRecorder, H>, + pub(crate) initial_values: HistoryRecorder, H>, + pub(crate) returned_refunds: HistoryRecorder, H>, +} + +#[derive(Clone, PartialEq, Debug)] +pub(crate) struct PrecompileProcessorTestInnerState { + pub(crate) timestamp_history: HistoryRecorder, H>, +} + +/// A struct that encapsulates the state of the VM's oracles +/// The state is to be used in tests. +#[derive(Clone, PartialEq, Debug)] +pub(crate) struct VmInstanceInnerState { + event_sink: InMemoryEventSink, + precompile_processor_state: PrecompileProcessorTestInnerState, + memory: SimpleMemory, + decommitter_state: DecommitterTestInnerState, + storage_oracle_state: StorageOracleInnerState, + local_state: VmLocalState, +} + +impl Vm { + // Dump inner state of the VM. + pub(crate) fn dump_inner_state(&self) -> VmInstanceInnerState { + let event_sink = self.state.event_sink.clone(); + let precompile_processor_state = PrecompileProcessorTestInnerState { + timestamp_history: self.state.precompiles_processor.timestamp_history.clone(), + }; + let memory = self.state.memory.clone(); + let decommitter_state = DecommitterTestInnerState { + modified_storage_keys: ModifiedKeysMap( + self.state + .decommittment_processor + .get_storage() + .borrow() + .modified_storage_keys() + .clone(), + ), + known_bytecodes: self.state.decommittment_processor.known_bytecodes.clone(), + decommitted_code_hashes: self + .state + .decommittment_processor + .get_decommitted_code_hashes_with_history() + .clone(), + }; + let storage_oracle_state = StorageOracleInnerState { + modified_storage_keys: ModifiedKeysMap( + self.state + .storage + .storage + .get_ptr() + .borrow() + .modified_storage_keys() + .clone(), + ), + frames_stack: self.state.storage.frames_stack.clone(), + pre_paid_changes: self.state.storage.pre_paid_changes.clone(), + paid_changes: self.state.storage.paid_changes.clone(), + initial_values: self.state.storage.initial_values.clone(), + returned_refunds: self.state.storage.returned_refunds.clone(), + }; + let local_state = self.state.local_state.clone(); + + VmInstanceInnerState { + event_sink, + precompile_processor_state, + memory, + decommitter_state, + storage_oracle_state, + local_state, + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/mod.rs new file mode 100644 index 000000000000..dfe8905a7e08 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/mod.rs @@ -0,0 +1,7 @@ +pub(crate) use transaction_test_info::{ExpectedError, TransactionTestInfo, TxModifier}; +pub(crate) use vm_tester::{default_l1_batch, InMemoryStorageView, VmTester, VmTesterBuilder}; +pub(crate) use zksync_test_account::{Account, DeployContractsTx, TxType}; + +mod inner_state; +mod transaction_test_info; +mod vm_tester; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs new file mode 100644 index 000000000000..114f80d1a217 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs @@ -0,0 +1,217 @@ +use zksync_types::{ExecuteTransactionCommon, Transaction}; + +use crate::{ + interface::{ + CurrentExecutionState, ExecutionResult, Halt, TxRevertReason, VmExecutionMode, + VmExecutionResultAndLogs, VmInterface, VmInterfaceHistoryEnabled, VmRevertReason, + }, + vm_latest::{tests::tester::vm_tester::VmTester, HistoryEnabled}, +}; + +#[derive(Debug, Clone)] +pub(crate) enum TxModifier { + WrongSignatureLength, + WrongSignature, + WrongMagicValue, + WrongNonce, + NonceReused, +} + +#[derive(Debug, Clone)] +pub(crate) enum TxExpectedResult { + Rejected { error: ExpectedError }, + Processed { rollback: bool }, +} + +#[derive(Debug, Clone)] +pub(crate) struct TransactionTestInfo { + tx: Transaction, + result: TxExpectedResult, +} + +#[derive(Debug, Clone)] +pub(crate) struct ExpectedError { + pub(crate) revert_reason: TxRevertReason, + pub(crate) modifier: Option, +} + +impl From for ExpectedError { + fn from(value: TxModifier) -> Self { + let revert_reason = match value { + TxModifier::WrongSignatureLength => { + Halt::ValidationFailed(VmRevertReason::General { + msg: "Signature length is incorrect".to_string(), + data: vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 29, 83, 105, 103, 110, 97, 116, 117, 114, 101, 32, + 108, 101, 110, 103, 116, 104, 32, 105, 115, 32, 105, 110, 99, 111, 114, 114, 101, 99, + 116, 0, 0, 0, + ], + }) + } + TxModifier::WrongSignature => { + Halt::ValidationFailed(VmRevertReason::General { + msg: "Account validation returned invalid magic value. Most often this means that the signature is incorrect".to_string(), + data: vec![], + }) + } + TxModifier::WrongMagicValue => { + Halt::ValidationFailed(VmRevertReason::General { + msg: "v is neither 27 nor 28".to_string(), + data: vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 22, 118, 32, 105, 115, 32, 110, 101, 105, 116, 104, + 101, 114, 32, 50, 55, 32, 110, 111, 114, 32, 50, 56, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ], + }) + + } + TxModifier::WrongNonce => { + Halt::ValidationFailed(VmRevertReason::General { + msg: "Incorrect nonce".to_string(), + data: vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 15, 73, 110, 99, 111, 114, 114, 101, 99, 116, 32, 110, + 111, 110, 99, 101, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + ], + }) + } + TxModifier::NonceReused => { + Halt::ValidationFailed(VmRevertReason::General { + msg: "Reusing the same nonce twice".to_string(), + data: vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 28, 82, 101, 117, 115, 105, 110, 103, 32, 116, 104, + 101, 32, 115, 97, 109, 101, 32, 110, 111, 110, 99, 101, 32, 116, 119, 105, 99, 101, 0, + 0, 0, 0, + ], + }) + } + }; + + ExpectedError { + revert_reason: TxRevertReason::Halt(revert_reason), + modifier: Some(value), + } + } +} + +impl TransactionTestInfo { + pub(crate) fn new_rejected( + mut transaction: Transaction, + expected_error: ExpectedError, + ) -> Self { + transaction.common_data = match transaction.common_data { + ExecuteTransactionCommon::L2(mut data) => { + if let Some(modifier) = &expected_error.modifier { + match modifier { + TxModifier::WrongSignatureLength => { + data.signature = data.signature[..data.signature.len() - 20].to_vec() + } + TxModifier::WrongSignature => data.signature = vec![27u8; 65], + TxModifier::WrongMagicValue => data.signature = vec![1u8; 65], + TxModifier::WrongNonce => { + // Do not need to modify signature for nonce error + } + TxModifier::NonceReused => { + // Do not need to modify signature for nonce error + } + } + } + ExecuteTransactionCommon::L2(data) + } + _ => panic!("L1 transactions are not supported"), + }; + + Self { + tx: transaction, + result: TxExpectedResult::Rejected { + error: expected_error, + }, + } + } + + pub(crate) fn new_processed(transaction: Transaction, should_be_rollbacked: bool) -> Self { + Self { + tx: transaction, + result: TxExpectedResult::Processed { + rollback: should_be_rollbacked, + }, + } + } + + fn verify_result(&self, result: &VmExecutionResultAndLogs) { + match &self.result { + TxExpectedResult::Rejected { error } => match &result.result { + ExecutionResult::Success { .. } => { + panic!("Transaction should be reverted {:?}", self.tx.nonce()) + } + ExecutionResult::Revert { output } => match &error.revert_reason { + TxRevertReason::TxReverted(expected) => { + assert_eq!(output, expected) + } + _ => { + panic!("Error types mismatch"); + } + }, + ExecutionResult::Halt { reason } => match &error.revert_reason { + TxRevertReason::Halt(expected) => { + assert_eq!(reason, expected) + } + _ => { + panic!("Error types mismatch"); + } + }, + }, + TxExpectedResult::Processed { .. } => { + assert!(!result.result.is_failed()); + } + } + } + + fn should_rollback(&self) -> bool { + match &self.result { + TxExpectedResult::Rejected { .. } => true, + TxExpectedResult::Processed { rollback } => *rollback, + } + } +} + +impl VmTester { + pub(crate) fn execute_and_verify_txs( + &mut self, + txs: &[TransactionTestInfo], + ) -> CurrentExecutionState { + for tx_test_info in txs { + self.execute_tx_and_verify(tx_test_info.clone()); + } + self.vm.execute(VmExecutionMode::Batch); + let mut state = self.vm.get_current_execution_state(); + state.used_contract_hashes.sort(); + state + } + + pub(crate) fn execute_tx_and_verify( + &mut self, + tx_test_info: TransactionTestInfo, + ) -> VmExecutionResultAndLogs { + let inner_state_before = self.vm.dump_inner_state(); + self.vm.make_snapshot(); + self.vm.push_transaction(tx_test_info.tx.clone()); + let result = self.vm.execute(VmExecutionMode::OneTx); + tx_test_info.verify_result(&result); + if tx_test_info.should_rollback() { + self.vm.rollback_to_the_latest_snapshot(); + let inner_state_after = self.vm.dump_inner_state(); + assert_eq!( + inner_state_before, inner_state_after, + "Inner state before and after rollback should be equal" + ); + } + result + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs new file mode 100644 index 000000000000..25f1361f14d0 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs @@ -0,0 +1,295 @@ +use std::marker::PhantomData; + +use zksync_contracts::BaseSystemContracts; +use zksync_state::{InMemoryStorage, StoragePtr, StorageView, WriteStorage}; +use zksync_types::{ + block::MiniblockHasher, + get_code_key, get_is_account_key, + helpers::unix_timestamp_ms, + utils::{deployed_address_create, storage_key_for_eth_balance}, + Address, L1BatchNumber, L2ChainId, MiniblockNumber, Nonce, ProtocolVersionId, U256, +}; +use zksync_utils::{bytecode::hash_bytecode, u256_to_h256}; + +use crate::{ + interface::{ + L1BatchEnv, L2Block, L2BlockEnv, SystemEnv, TxExecutionMode, VmExecutionMode, VmInterface, + }, + vm_latest::{ + constants::BLOCK_GAS_LIMIT, + tests::{ + tester::{Account, TxType}, + utils::read_test_contract, + }, + utils::l2_blocks::load_last_l2_block, + Vm, + }, + HistoryMode, +}; + +pub(crate) type InMemoryStorageView = StorageView; + +pub(crate) struct VmTester { + pub(crate) vm: Vm, + pub(crate) storage: StoragePtr, + pub(crate) fee_account: Address, + pub(crate) deployer: Option, + pub(crate) test_contract: Option
, + pub(crate) rich_accounts: Vec, + pub(crate) custom_contracts: Vec, + _phantom: std::marker::PhantomData, +} + +impl VmTester { + pub(crate) fn deploy_test_contract(&mut self) { + let contract = read_test_contract(); + let tx = self + .deployer + .as_mut() + .expect("You have to initialize builder with deployer") + .get_deploy_tx(&contract, None, TxType::L2) + .tx; + let nonce = tx.nonce().unwrap().0.into(); + self.vm.push_transaction(tx); + self.vm.execute(VmExecutionMode::OneTx); + let deployed_address = + deployed_address_create(self.deployer.as_ref().unwrap().address, nonce); + self.test_contract = Some(deployed_address); + } + + pub(crate) fn reset_with_empty_storage(&mut self) { + self.storage = StorageView::new(get_empty_storage()).to_rc_ptr(); + self.reset_state(false); + } + + /// Reset the state of the VM to the initial state. + /// If `use_latest_l2_block` is true, then the VM will use the latest L2 block from storage, + /// otherwise it will use the first L2 block of l1 batch env + pub(crate) fn reset_state(&mut self, use_latest_l2_block: bool) { + for account in self.rich_accounts.iter_mut() { + account.nonce = Nonce(0); + make_account_rich(self.storage.clone(), account); + } + if let Some(deployer) = &self.deployer { + make_account_rich(self.storage.clone(), deployer); + } + + if !self.custom_contracts.is_empty() { + println!("Inserting custom contracts is not yet supported") + // insert_contracts(&mut self.storage, &self.custom_contracts); + } + + let mut l1_batch = self.vm.batch_env.clone(); + if use_latest_l2_block { + let last_l2_block = load_last_l2_block(self.storage.clone()).unwrap_or(L2Block { + number: 0, + timestamp: 0, + hash: MiniblockHasher::legacy_hash(MiniblockNumber(0)), + }); + l1_batch.first_l2_block = L2BlockEnv { + number: last_l2_block.number + 1, + timestamp: std::cmp::max(last_l2_block.timestamp + 1, l1_batch.timestamp), + prev_block_hash: last_l2_block.hash, + max_virtual_blocks_to_create: 1, + }; + } + + let vm = Vm::new(l1_batch, self.vm.system_env.clone(), self.storage.clone()); + + if self.test_contract.is_some() { + self.deploy_test_contract(); + } + + self.vm = vm; + } +} + +pub(crate) type ContractsToDeploy = (Vec, Address, bool); + +pub(crate) struct VmTesterBuilder { + storage: Option, + l1_batch_env: Option, + system_env: SystemEnv, + deployer: Option, + rich_accounts: Vec, + custom_contracts: Vec, + _phantom: PhantomData, +} + +impl Clone for VmTesterBuilder { + fn clone(&self) -> Self { + Self { + storage: None, + l1_batch_env: self.l1_batch_env.clone(), + system_env: self.system_env.clone(), + deployer: self.deployer.clone(), + rich_accounts: self.rich_accounts.clone(), + custom_contracts: self.custom_contracts.clone(), + _phantom: PhantomData, + } + } +} + +#[allow(dead_code)] +impl VmTesterBuilder { + pub(crate) fn new(_: H) -> Self { + Self { + storage: None, + l1_batch_env: None, + system_env: SystemEnv { + zk_porter_available: false, + version: ProtocolVersionId::latest(), + base_system_smart_contracts: BaseSystemContracts::playground(), + gas_limit: BLOCK_GAS_LIMIT, + execution_mode: TxExecutionMode::VerifyExecute, + default_validation_computational_gas_limit: BLOCK_GAS_LIMIT, + chain_id: L2ChainId::from(270), + }, + deployer: None, + rich_accounts: vec![], + custom_contracts: vec![], + _phantom: PhantomData, + } + } + + pub(crate) fn with_l1_batch_env(mut self, l1_batch_env: L1BatchEnv) -> Self { + self.l1_batch_env = Some(l1_batch_env); + self + } + + pub(crate) fn with_system_env(mut self, system_env: SystemEnv) -> Self { + self.system_env = system_env; + self + } + + pub(crate) fn with_storage(mut self, storage: InMemoryStorage) -> Self { + self.storage = Some(storage); + self + } + + pub(crate) fn with_base_system_smart_contracts( + mut self, + base_system_smart_contracts: BaseSystemContracts, + ) -> Self { + self.system_env.base_system_smart_contracts = base_system_smart_contracts; + self + } + + pub(crate) fn with_gas_limit(mut self, gas_limit: u32) -> Self { + self.system_env.gas_limit = gas_limit; + self + } + + pub(crate) fn with_execution_mode(mut self, execution_mode: TxExecutionMode) -> Self { + self.system_env.execution_mode = execution_mode; + self + } + + pub(crate) fn with_empty_in_memory_storage(mut self) -> Self { + self.storage = Some(get_empty_storage()); + self + } + + pub(crate) fn with_random_rich_accounts(mut self, number: u32) -> Self { + for _ in 0..number { + let account = Account::random(); + self.rich_accounts.push(account); + } + self + } + + pub(crate) fn with_rich_accounts(mut self, accounts: Vec) -> Self { + self.rich_accounts.extend(accounts); + self + } + + pub(crate) fn with_deployer(mut self) -> Self { + let deployer = Account::random(); + self.deployer = Some(deployer); + self + } + + pub(crate) fn with_custom_contracts(mut self, contracts: Vec) -> Self { + self.custom_contracts = contracts; + self + } + + pub(crate) fn build(self) -> VmTester { + let l1_batch_env = self + .l1_batch_env + .unwrap_or_else(|| default_l1_batch(L1BatchNumber(1))); + + let mut raw_storage = self.storage.unwrap_or_else(get_empty_storage); + insert_contracts(&mut raw_storage, &self.custom_contracts); + let storage_ptr = StorageView::new(raw_storage).to_rc_ptr(); + for account in self.rich_accounts.iter() { + make_account_rich(storage_ptr.clone(), account); + } + if let Some(deployer) = &self.deployer { + make_account_rich(storage_ptr.clone(), deployer); + } + let fee_account = l1_batch_env.fee_account; + + let vm = Vm::new(l1_batch_env, self.system_env, storage_ptr.clone()); + + VmTester { + vm, + storage: storage_ptr, + fee_account, + deployer: self.deployer, + test_contract: None, + rich_accounts: self.rich_accounts.clone(), + custom_contracts: self.custom_contracts.clone(), + _phantom: PhantomData, + } + } +} + +pub(crate) fn default_l1_batch(number: L1BatchNumber) -> L1BatchEnv { + let timestamp = unix_timestamp_ms(); + L1BatchEnv { + previous_batch_hash: None, + number, + timestamp, + l1_gas_price: 50_000_000_000, // 50 gwei + fair_l2_gas_price: 250_000_000, // 0.25 gwei + fee_account: Address::random(), + enforced_base_fee: None, + first_l2_block: L2BlockEnv { + number: 1, + timestamp, + prev_block_hash: MiniblockHasher::legacy_hash(MiniblockNumber(0)), + max_virtual_blocks_to_create: 100, + }, + } +} + +pub(crate) fn make_account_rich(storage: StoragePtr, account: &Account) { + let key = storage_key_for_eth_balance(&account.address); + storage + .as_ref() + .borrow_mut() + .set_value(key, u256_to_h256(U256::from(10u64.pow(19)))); +} + +pub(crate) fn get_empty_storage() -> InMemoryStorage { + InMemoryStorage::with_system_contracts(hash_bytecode) +} + +// Inserts the contracts into the test environment, bypassing the +// deployer system contract. Besides the reference to storage +// it accepts a `contracts` tuple of information about the contract +// and whether or not it is an account. +fn insert_contracts(raw_storage: &mut InMemoryStorage, contracts: &[ContractsToDeploy]) { + for (contract, address, is_account) in contracts { + let deployer_code_key = get_code_key(address); + raw_storage.set_value(deployer_code_key, hash_bytecode(contract)); + + if *is_account { + let is_account_key = get_is_account_key(address); + raw_storage.set_value(is_account_key, u256_to_h256(1_u32.into())); + } + + raw_storage.store_factory_dep(hash_bytecode(contract), contract.clone()); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs new file mode 100644 index 000000000000..f02de899b03e --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs @@ -0,0 +1,54 @@ +use zksync_types::{Execute, H160}; + +use crate::{ + interface::{TxExecutionMode, TxRevertReason, VmRevertReason}, + vm_latest::{ + tests::{ + tester::{ExpectedError, TransactionTestInfo, VmTesterBuilder}, + utils::{get_execute_error_calldata, read_error_contract, BASE_SYSTEM_CONTRACTS}, + }, + HistoryEnabled, + }, +}; + +#[test] +fn test_tracing_of_execution_errors() { + let contract_address = H160::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_base_system_smart_contracts(BASE_SYSTEM_CONTRACTS.clone()) + .with_custom_contracts(vec![(read_error_contract(), contract_address, false)]) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_deployer() + .with_random_rich_accounts(1) + .build(); + + let account = &mut vm.rich_accounts[0]; + + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address, + calldata: get_execute_error_calldata(), + value: Default::default(), + factory_deps: Some(vec![]), + }, + None, + ); + + vm.execute_tx_and_verify(TransactionTestInfo::new_rejected( + tx, + ExpectedError { + revert_reason: TxRevertReason::TxReverted(VmRevertReason::General { + msg: "short".to_string(), + data: vec![ + 8, 195, 121, 160, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 115, 104, 111, 114, 116, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, + ], + }), + modifier: None, + }, + )); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs new file mode 100644 index 000000000000..b5c493ca7075 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs @@ -0,0 +1,362 @@ +use zk_evm_1_4_0::aux_structures::Timestamp; +use zksync_contracts::{deployer_contract, load_contract, load_sys_contract, read_bytecode}; +use zksync_state::WriteStorage; +use zksync_test_account::TxType; +use zksync_types::{ + ethabi::{Contract, Token}, + get_code_key, get_known_code_key, + protocol_version::ProtocolUpgradeTxCommonData, + Address, Execute, ExecuteTransactionCommon, Transaction, COMPLEX_UPGRADER_ADDRESS, + CONTRACT_DEPLOYER_ADDRESS, CONTRACT_FORCE_DEPLOYER_ADDRESS, H160, H256, + REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE, U256, +}; +use zksync_utils::{bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}; + +use super::utils::read_test_contract; +use crate::{ + interface::{ + ExecutionResult, Halt, TxExecutionMode, VmExecutionMode, VmInterface, + VmInterfaceHistoryEnabled, + }, + vm_latest::{ + tests::{tester::VmTesterBuilder, utils::verify_required_storage}, + HistoryEnabled, + }, +}; + +/// In this test we ensure that the requirements for protocol upgrade transactions are enforced by the bootloader: +/// - This transaction must be the only one in block +/// - If present, this transaction must be the first one in block +#[test] +fn test_protocol_upgrade_is_first() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let bytecode_hash = hash_bytecode(&read_test_contract()); + vm.vm + .storage + .borrow_mut() + .set_value(get_known_code_key(&bytecode_hash), u256_to_h256(1.into())); + + // Here we just use some random transaction of protocol upgrade type: + let protocol_upgrade_transaction = get_forced_deploy_tx(&[ForceDeployment { + // The bytecode hash to put on an address + bytecode_hash, + // The address on which to deploy the bytecodehash to + address: H160::random(), + // Whether to run the constructor on the force deployment + call_constructor: false, + // The value with which to initialize a contract + value: U256::zero(), + // The constructor calldata + input: vec![], + }]); + + // Another random upgrade transaction + let another_protocol_upgrade_transaction = get_forced_deploy_tx(&[ForceDeployment { + // The bytecode hash to put on an address + bytecode_hash, + // The address on which to deploy the bytecodehash to + address: H160::random(), + // Whether to run the constructor on the force deployment + call_constructor: false, + // The value with which to initialize a contract + value: U256::zero(), + // The constructor calldata + input: vec![], + }]); + + let normal_l1_transaction = vm.rich_accounts[0] + .get_deploy_tx(&read_test_contract(), None, TxType::L1 { serial_id: 0 }) + .tx; + + let expected_error = + Halt::UnexpectedVMBehavior("Assertion error: Protocol upgrade tx not first".to_string()); + + vm.vm.make_snapshot(); + // Test 1: there must be only one system transaction in block + vm.vm.push_transaction(protocol_upgrade_transaction.clone()); + vm.vm.push_transaction(normal_l1_transaction.clone()); + vm.vm.push_transaction(another_protocol_upgrade_transaction); + + vm.vm.execute(VmExecutionMode::OneTx); + vm.vm.execute(VmExecutionMode::OneTx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert_eq!( + result.result, + ExecutionResult::Halt { + reason: expected_error.clone() + } + ); + + // Test 2: the protocol upgrade tx must be the first one in block + vm.vm.rollback_to_the_latest_snapshot(); + vm.vm.make_snapshot(); + vm.vm.push_transaction(normal_l1_transaction.clone()); + vm.vm.push_transaction(protocol_upgrade_transaction.clone()); + + vm.vm.execute(VmExecutionMode::OneTx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert_eq!( + result.result, + ExecutionResult::Halt { + reason: expected_error + } + ); + + vm.vm.rollback_to_the_latest_snapshot(); + vm.vm.make_snapshot(); + vm.vm.push_transaction(protocol_upgrade_transaction); + vm.vm.push_transaction(normal_l1_transaction); + + vm.vm.execute(VmExecutionMode::OneTx); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!(!result.result.is_failed()); +} + +/// In this test we try to test how force deployments could be done via protocol upgrade transactions. +#[test] +fn test_force_deploy_upgrade() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let storage_view = vm.storage.clone(); + let bytecode_hash = hash_bytecode(&read_test_contract()); + + let known_code_key = get_known_code_key(&bytecode_hash); + // It is generally expected that all the keys will be set as known prior to the protocol upgrade. + storage_view + .borrow_mut() + .set_value(known_code_key, u256_to_h256(1.into())); + drop(storage_view); + + let address_to_deploy = H160::random(); + // Here we just use some random transaction of protocol upgrade type: + let transaction = get_forced_deploy_tx(&[ForceDeployment { + // The bytecode hash to put on an address + bytecode_hash, + // The address on which to deploy the bytecodehash to + address: address_to_deploy, + // Whether to run the constructor on the force deployment + call_constructor: false, + // The value with which to initialize a contract + value: U256::zero(), + // The constructor calldata + input: vec![], + }]); + + vm.vm.push_transaction(transaction); + + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!( + !result.result.is_failed(), + "The force upgrade was not successful" + ); + + let expected_slots = vec![(bytecode_hash, get_code_key(&address_to_deploy))]; + + // Verify that the bytecode has been set correctly + verify_required_storage(&vm.vm.state, expected_slots); +} + +/// Here we show how the work with the complex upgrader could be done +#[test] +fn test_complex_upgrader() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_random_rich_accounts(1) + .build(); + + let storage_view = vm.storage.clone(); + + let bytecode_hash = hash_bytecode(&read_complex_upgrade()); + let msg_sender_test_hash = hash_bytecode(&read_msg_sender_test()); + + // Let's assume that the bytecode for the implementation of the complex upgrade + // is already deployed in some address in userspace + let upgrade_impl = H160::random(); + let account_code_key = get_code_key(&upgrade_impl); + + storage_view + .borrow_mut() + .set_value(get_known_code_key(&bytecode_hash), u256_to_h256(1.into())); + storage_view.borrow_mut().set_value( + get_known_code_key(&msg_sender_test_hash), + u256_to_h256(1.into()), + ); + storage_view + .borrow_mut() + .set_value(account_code_key, bytecode_hash); + drop(storage_view); + + vm.vm.state.decommittment_processor.populate( + vec![ + ( + h256_to_u256(bytecode_hash), + bytes_to_be_words(read_complex_upgrade()), + ), + ( + h256_to_u256(msg_sender_test_hash), + bytes_to_be_words(read_msg_sender_test()), + ), + ], + Timestamp(0), + ); + + let address_to_deploy1 = H160::random(); + let address_to_deploy2 = H160::random(); + + let transaction = get_complex_upgrade_tx( + upgrade_impl, + address_to_deploy1, + address_to_deploy2, + bytecode_hash, + ); + + vm.vm.push_transaction(transaction); + let result = vm.vm.execute(VmExecutionMode::OneTx); + assert!( + !result.result.is_failed(), + "The force upgrade was not successful" + ); + + let expected_slots = vec![ + (bytecode_hash, get_code_key(&address_to_deploy1)), + (bytecode_hash, get_code_key(&address_to_deploy2)), + ]; + + // Verify that the bytecode has been set correctly + verify_required_storage(&vm.vm.state, expected_slots); +} + +#[derive(Debug, Clone)] +struct ForceDeployment { + // The bytecode hash to put on an address + bytecode_hash: H256, + // The address on which to deploy the bytecodehash to + address: Address, + // Whether to run the constructor on the force deployment + call_constructor: bool, + // The value with which to initialize a contract + value: U256, + // The constructor calldata + input: Vec, +} + +fn get_forced_deploy_tx(deployment: &[ForceDeployment]) -> Transaction { + let deployer = deployer_contract(); + let contract_function = deployer.function("forceDeployOnAddresses").unwrap(); + + let encoded_deployments: Vec<_> = deployment + .iter() + .map(|deployment| { + Token::Tuple(vec![ + Token::FixedBytes(deployment.bytecode_hash.as_bytes().to_vec()), + Token::Address(deployment.address), + Token::Bool(deployment.call_constructor), + Token::Uint(deployment.value), + Token::Bytes(deployment.input.clone()), + ]) + }) + .collect(); + + let params = [Token::Array(encoded_deployments)]; + + let calldata = contract_function + .encode_input(¶ms) + .expect("failed to encode parameters"); + + let execute = Execute { + contract_address: CONTRACT_DEPLOYER_ADDRESS, + calldata, + factory_deps: None, + value: U256::zero(), + }; + + Transaction { + common_data: ExecuteTransactionCommon::ProtocolUpgrade(ProtocolUpgradeTxCommonData { + sender: CONTRACT_FORCE_DEPLOYER_ADDRESS, + gas_limit: U256::from(200_000_000u32), + gas_per_pubdata_limit: REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE.into(), + ..Default::default() + }), + execute, + received_timestamp_ms: 0, + raw_bytes: None, + } +} + +// Returns the transaction that performs a complex protocol upgrade. +// The first param is the address of the implementation of the complex upgrade +// in user-space, while the next 3 params are params of the implenentaiton itself +// For the explanatation for the parameters, please refer to: +// etc/contracts-test-data/complex-upgrade/complex-upgrade.sol +fn get_complex_upgrade_tx( + implementation_address: Address, + address1: Address, + address2: Address, + bytecode_hash: H256, +) -> Transaction { + let impl_contract = get_complex_upgrade_abi(); + let impl_function = impl_contract.function("someComplexUpgrade").unwrap(); + let impl_calldata = impl_function + .encode_input(&[ + Token::Address(address1), + Token::Address(address2), + Token::FixedBytes(bytecode_hash.as_bytes().to_vec()), + ]) + .unwrap(); + + let complex_upgrader = get_complex_upgrader_abi(); + let upgrade_function = complex_upgrader.function("upgrade").unwrap(); + let complex_upgrader_calldata = upgrade_function + .encode_input(&[ + Token::Address(implementation_address), + Token::Bytes(impl_calldata), + ]) + .unwrap(); + + let execute = Execute { + contract_address: COMPLEX_UPGRADER_ADDRESS, + calldata: complex_upgrader_calldata, + factory_deps: None, + value: U256::zero(), + }; + + Transaction { + common_data: ExecuteTransactionCommon::ProtocolUpgrade(ProtocolUpgradeTxCommonData { + sender: CONTRACT_FORCE_DEPLOYER_ADDRESS, + gas_limit: U256::from(200_000_000u32), + gas_per_pubdata_limit: REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE.into(), + ..Default::default() + }), + execute, + received_timestamp_ms: 0, + raw_bytes: None, + } +} + +fn read_complex_upgrade() -> Vec { + read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/complex-upgrade.sol/ComplexUpgrade.json") +} + +fn read_msg_sender_test() -> Vec { + read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/msg-sender.sol/MsgSenderTest.json") +} + +fn get_complex_upgrade_abi() -> Contract { + load_contract( + "etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/complex-upgrade.sol/ComplexUpgrade.json" + ) +} + +fn get_complex_upgrader_abi() -> Contract { + load_sys_contract("ComplexUpgrader") +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs new file mode 100644 index 000000000000..c126b50cb574 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs @@ -0,0 +1,105 @@ +use ethabi::Contract; +use once_cell::sync::Lazy; +use zksync_contracts::{ + load_contract, read_bytecode, read_zbin_bytecode, BaseSystemContracts, SystemContractCode, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::{ + utils::storage_key_for_standard_token_balance, AccountTreeId, Address, StorageKey, H256, U256, +}; +use zksync_utils::{bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}; + +use crate::vm_latest::{ + tests::tester::InMemoryStorageView, types::internals::ZkSyncVmState, HistoryMode, +}; + +pub(crate) static BASE_SYSTEM_CONTRACTS: Lazy = + Lazy::new(BaseSystemContracts::load_from_disk); + +// Probably make it a part of vm tester +pub(crate) fn verify_required_storage( + state: &ZkSyncVmState, + required_values: Vec<(H256, StorageKey)>, +) { + for (required_value, key) in required_values { + let current_value = state.storage.storage.read_from_storage(&key); + + assert_eq!( + u256_to_h256(current_value), + required_value, + "Invalid value at key {key:?}" + ); + } +} + +pub(crate) fn verify_required_memory( + state: &ZkSyncVmState, + required_values: Vec<(U256, u32, u32)>, +) { + for (required_value, memory_page, cell) in required_values { + let current_value = state + .memory + .read_slot(memory_page as usize, cell as usize) + .value; + assert_eq!(current_value, required_value); + } +} + +pub(crate) fn get_balance( + token_id: AccountTreeId, + account: &Address, + main_storage: StoragePtr, +) -> U256 { + let key = storage_key_for_standard_token_balance(token_id, account); + h256_to_u256(main_storage.borrow_mut().read_value(&key)) +} + +pub(crate) fn read_test_contract() -> Vec { + read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json") +} + +pub(crate) fn get_bootloader(test: &str) -> SystemContractCode { + let bootloader_code = read_zbin_bytecode(format!( + "contracts/system-contracts/bootloader/tests/artifacts/{}.yul.zbin", + test + )); + + let bootloader_hash = hash_bytecode(&bootloader_code); + SystemContractCode { + code: bytes_to_be_words(bootloader_code), + hash: bootloader_hash, + } +} + +pub(crate) fn read_nonce_holder_tester() -> Vec { + read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/custom-account/nonce-holder-test.sol/NonceHolderTest.json") +} + +pub(crate) fn read_error_contract() -> Vec { + read_bytecode( + "etc/contracts-test-data/artifacts-zk/contracts/error/error.sol/SimpleRequire.json", + ) +} + +pub(crate) fn get_execute_error_calldata() -> Vec { + let test_contract = load_contract( + "etc/contracts-test-data/artifacts-zk/contracts/error/error.sol/SimpleRequire.json", + ); + + let function = test_contract.function("require_short").unwrap(); + + function + .encode_input(&[]) + .expect("failed to encode parameters") +} + +pub(crate) fn read_many_owners_custom_account_contract() -> (Vec, Contract) { + let path = "etc/contracts-test-data/artifacts-zk/contracts/custom-account/many-owners-custom-account.sol/ManyOwnersCustomAccount.json"; + (read_bytecode(path), load_contract(path)) +} + +pub(crate) fn read_max_depth_contract() -> Vec { + read_zbin_bytecode( + "core/tests/ts-integration/contracts/zkasm/artifacts/deep_stak.zkasm/deep_stak.zkasm.zbin", + ) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs new file mode 100644 index 000000000000..0e18d989af62 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs @@ -0,0 +1,298 @@ +use std::{ + fmt::{Debug, Formatter}, + marker::PhantomData, +}; + +use zk_evm_1_4_0::{ + tracing::{ + AfterDecodingData, AfterExecutionData, BeforeExecutionData, Tracer, VmLocalStateData, + }, + vm_state::VmLocalState, + witness_trace::DummyTracer, + zkevm_opcode_defs::{decoding::EncodingModeProduction, Opcode, RetOpcode}, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::Timestamp; + +use super::PubdataTracer; +use crate::{ + interface::{ + tracer::{TracerExecutionStopReason, VmExecutionStopReason}, + traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, + types::tracer::TracerExecutionStatus, + Halt, VmExecutionMode, + }, + vm_latest::{ + bootloader_state::{utils::apply_l2_block, BootloaderState}, + constants::BOOTLOADER_HEAP_PAGE, + old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, + tracers::{ + dispatcher::TracerDispatcher, + utils::{ + computational_gas_price, gas_spent_on_bytecodes_and_long_messages_this_opcode, + print_debug_if_needed, VmHook, + }, + RefundsTracer, ResultTracer, + }, + types::internals::ZkSyncVmState, + VmTracer, + }, +}; + +/// Default tracer for the VM. It manages the other tracers execution and stop the vm when needed. +pub(crate) struct DefaultExecutionTracer { + tx_has_been_processed: bool, + execution_mode: VmExecutionMode, + + pub(crate) gas_spent_on_bytecodes_and_long_messages: u32, + // Amount of gas used during account validation. + pub(crate) computational_gas_used: u32, + // Maximum number of gas that we're allowed to use during account validation. + tx_validation_gas_limit: u32, + in_account_validation: bool, + final_batch_info_requested: bool, + pub(crate) result_tracer: ResultTracer, + // This tracer is designed specifically for calculating refunds. Its separation from the custom tracer + // ensures static dispatch, enhancing performance by avoiding dynamic dispatch overhead. + // Additionally, being an internal tracer, it saves the results directly to VmResultAndLogs. + pub(crate) refund_tracer: Option>, + // The pubdata tracer is responsible for inserting the pubdata packing information into the bootloader + // memory at the end of the batch. Its separation from the custom tracer + // ensures static dispatch, enhancing performance by avoiding dynamic dispatch overhead. + pub(crate) pubdata_tracer: Option>, + pub(crate) dispatcher: TracerDispatcher, + ret_from_the_bootloader: Option, + storage: StoragePtr, + _phantom: PhantomData, +} + +impl DefaultExecutionTracer { + pub(crate) fn new( + computational_gas_limit: u32, + execution_mode: VmExecutionMode, + dispatcher: TracerDispatcher, + storage: StoragePtr, + refund_tracer: Option>, + pubdata_tracer: Option>, + ) -> Self { + Self { + tx_has_been_processed: false, + execution_mode, + gas_spent_on_bytecodes_and_long_messages: 0, + computational_gas_used: 0, + tx_validation_gas_limit: computational_gas_limit, + in_account_validation: false, + final_batch_info_requested: false, + result_tracer: ResultTracer::new(execution_mode), + refund_tracer, + dispatcher, + pubdata_tracer, + ret_from_the_bootloader: None, + storage, + _phantom: PhantomData, + } + } + + pub(crate) fn tx_has_been_processed(&self) -> bool { + self.tx_has_been_processed + } + + pub(crate) fn validation_run_out_of_gas(&self) -> bool { + self.computational_gas_used > self.tx_validation_gas_limit + } + + pub(crate) fn gas_spent_on_pubdata(&self, vm_local_state: &VmLocalState) -> u32 { + self.gas_spent_on_bytecodes_and_long_messages + vm_local_state.spent_pubdata_counter + } + + fn set_fictive_l2_block( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &mut BootloaderState, + ) { + let current_timestamp = Timestamp(state.local_state.timestamp); + let txs_index = bootloader_state.free_tx_index(); + let l2_block = bootloader_state.insert_fictive_l2_block(); + let mut memory = vec![]; + apply_l2_block(&mut memory, l2_block, txs_index); + state + .memory + .populate_page(BOOTLOADER_HEAP_PAGE as usize, memory, current_timestamp); + self.final_batch_info_requested = false; + } + + fn should_stop_execution(&self) -> TracerExecutionStatus { + match self.execution_mode { + VmExecutionMode::OneTx if self.tx_has_been_processed() => { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Finish); + } + VmExecutionMode::Bootloader if self.ret_from_the_bootloader == Some(RetOpcode::Ok) => { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Finish); + } + _ => {} + }; + if self.validation_run_out_of_gas() { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Abort( + Halt::ValidationOutOfGas, + )); + } + TracerExecutionStatus::Continue + } +} + +impl Debug for DefaultExecutionTracer { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("DefaultExecutionTracer").finish() + } +} + +/// The default tracer for the VM manages all other tracers. For the sake of optimization, these tracers are statically dispatched. +/// At the same time, the boilerplate for calling these tracers for all tracer calls is quite extensive. +/// This macro is used to reduce the boilerplate. +/// +/// Usage: +/// ``` +/// dispatch_tracers!( +/// self.after_decoding(state, data, memory) +/// ); +/// ``` +/// Whenever a new tracer is added, it should be added to the macro call. +/// +/// The macro passes the function call to all tracers. +macro_rules! dispatch_tracers { + ($self:ident.$function:ident($( $params:expr ),*)) => { + $self.result_tracer.$function($( $params ),*); + $self.dispatcher.$function($( $params ),*); + if let Some(tracer) = &mut $self.refund_tracer { + tracer.$function($( $params ),*); + } + if let Some(tracer) = &mut $self.pubdata_tracer { + tracer.$function($( $params ),*); + } + }; +} + +impl Tracer for DefaultExecutionTracer { + const CALL_BEFORE_DECODING: bool = false; + const CALL_AFTER_DECODING: bool = true; + const CALL_BEFORE_EXECUTION: bool = true; + const CALL_AFTER_EXECUTION: bool = true; + type SupportedMemory = SimpleMemory; + + fn before_decoding( + &mut self, + _state: VmLocalStateData<'_, 8, EncodingModeProduction>, + _memory: &Self::SupportedMemory, + ) { + } + + fn after_decoding( + &mut self, + state: VmLocalStateData<'_>, + data: AfterDecodingData, + memory: &Self::SupportedMemory, + ) { + dispatch_tracers!(self.after_decoding(state, data, memory)); + } + + fn before_execution( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + memory: &Self::SupportedMemory, + ) { + if self.in_account_validation { + self.computational_gas_used = self + .computational_gas_used + .saturating_add(computational_gas_price(state, &data)); + } + + let hook = VmHook::from_opcode_memory(&state, &data); + print_debug_if_needed(&hook, &state, memory); + + match hook { + VmHook::TxHasEnded => self.tx_has_been_processed = true, + VmHook::NoValidationEntered => self.in_account_validation = false, + VmHook::AccountValidationEntered => self.in_account_validation = true, + VmHook::FinalBatchInfo => self.final_batch_info_requested = true, + _ => {} + } + + self.gas_spent_on_bytecodes_and_long_messages += + gas_spent_on_bytecodes_and_long_messages_this_opcode(&state, &data); + + dispatch_tracers!(self.before_execution(state, data, memory, self.storage.clone())); + } + + fn after_execution( + &mut self, + state: VmLocalStateData<'_>, + data: AfterExecutionData, + memory: &Self::SupportedMemory, + ) { + if let VmExecutionMode::Bootloader = self.execution_mode { + let (next_opcode, _, _) = zk_evm_1_4_0::vm_state::read_and_decode( + state.vm_local_state, + memory, + &mut DummyTracer, + self, + ); + if current_frame_is_bootloader(state.vm_local_state) { + if let Opcode::Ret(ret) = next_opcode.inner.variant.opcode { + self.ret_from_the_bootloader = Some(ret); + } + } + } + + dispatch_tracers!(self.after_execution(state, data, memory, self.storage.clone())); + } +} + +impl DefaultExecutionTracer { + pub(crate) fn initialize_tracer(&mut self, state: &mut ZkSyncVmState) { + dispatch_tracers!(self.initialize_tracer(state)); + } + + pub(crate) fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + if self.final_batch_info_requested { + self.set_fictive_l2_block(state, bootloader_state) + } + + let mut result = self.result_tracer.finish_cycle(state, bootloader_state); + if let Some(refund_tracer) = &mut self.refund_tracer { + result = refund_tracer + .finish_cycle(state, bootloader_state) + .stricter(&result); + } + result = self + .dispatcher + .finish_cycle(state, bootloader_state) + .stricter(&result); + if let Some(pubdata_tracer) = &mut self.pubdata_tracer { + result = pubdata_tracer + .finish_cycle(state, bootloader_state) + .stricter(&result); + } + result.stricter(&self.should_stop_execution()) + } + + pub(crate) fn after_vm_execution( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &BootloaderState, + stop_reason: VmExecutionStopReason, + ) { + dispatch_tracers!(self.after_vm_execution(state, bootloader_state, stop_reason.clone())); + } +} + +fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { + // The current frame is bootloader if the callstack depth is 1. + // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior + // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. + local_state.callstack.inner.len() == 1 +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs new file mode 100644 index 000000000000..5ee5c8ab0c19 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs @@ -0,0 +1,126 @@ +use zk_evm_1_4_0::tracing::{ + AfterDecodingData, AfterExecutionData, BeforeExecutionData, VmLocalStateData, +}; +use zksync_state::{StoragePtr, WriteStorage}; + +use crate::{ + interface::{ + dyn_tracers::vm_1_4_0::DynTracer, + tracer::{TracerExecutionStatus, VmExecutionStopReason}, + }, + vm_latest::{ + BootloaderState, HistoryMode, SimpleMemory, TracerPointer, VmTracer, ZkSyncVmState, + }, +}; + +/// Tracer dispatcher is a tracer that can dispatch calls to multiple tracers. +pub struct TracerDispatcher { + tracers: Vec>, +} + +impl TracerDispatcher { + pub fn new(tracers: Vec>) -> Self { + Self { tracers } + } +} + +impl From> for TracerDispatcher { + fn from(value: TracerPointer) -> Self { + Self { + tracers: vec![value], + } + } +} + +impl From>> for TracerDispatcher { + fn from(value: Vec>) -> Self { + Self { tracers: value } + } +} + +impl Default for TracerDispatcher { + fn default() -> Self { + Self { tracers: vec![] } + } +} + +impl DynTracer> for TracerDispatcher { + #[inline(always)] + fn before_decoding(&mut self, _state: VmLocalStateData<'_>, _memory: &SimpleMemory) { + for tracer in self.tracers.iter_mut() { + tracer.before_decoding(_state, _memory); + } + } + + #[inline(always)] + fn after_decoding( + &mut self, + _state: VmLocalStateData<'_>, + _data: AfterDecodingData, + _memory: &SimpleMemory, + ) { + for tracer in self.tracers.iter_mut() { + tracer.after_decoding(_state, _data, _memory); + } + } + + #[inline(always)] + fn before_execution( + &mut self, + _state: VmLocalStateData<'_>, + _data: BeforeExecutionData, + _memory: &SimpleMemory, + _storage: StoragePtr, + ) { + for tracer in self.tracers.iter_mut() { + tracer.before_execution(_state, _data, _memory, _storage.clone()); + } + } + + #[inline(always)] + fn after_execution( + &mut self, + _state: VmLocalStateData<'_>, + _data: AfterExecutionData, + _memory: &SimpleMemory, + _storage: StoragePtr, + ) { + for tracer in self.tracers.iter_mut() { + tracer.after_execution(_state, _data, _memory, _storage.clone()); + } + } +} + +impl VmTracer for TracerDispatcher { + fn initialize_tracer(&mut self, _state: &mut ZkSyncVmState) { + for tracer in self.tracers.iter_mut() { + tracer.initialize_tracer(_state); + } + } + + /// Run after each vm execution cycle + #[inline(always)] + fn finish_cycle( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + let mut result = TracerExecutionStatus::Continue; + for tracer in self.tracers.iter_mut() { + result = result.stricter(&tracer.finish_cycle(_state, _bootloader_state)); + } + result + } + + /// Run after the vm execution + fn after_vm_execution( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &BootloaderState, + _stop_reason: VmExecutionStopReason, + ) { + for tracer in self.tracers.iter_mut() { + tracer.after_vm_execution(_state, _bootloader_state, _stop_reason.clone()); + } + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs new file mode 100644 index 000000000000..33d043de6eb1 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs @@ -0,0 +1,13 @@ +pub(crate) use default_tracers::DefaultExecutionTracer; +pub(crate) use pubdata_tracer::PubdataTracer; +pub(crate) use refunds::RefundsTracer; +pub(crate) use result_tracer::ResultTracer; + +pub(crate) mod default_tracers; +pub(crate) mod pubdata_tracer; +pub(crate) mod refunds; +pub(crate) mod result_tracer; + +pub mod dispatcher; +pub(crate) mod traits; +pub(crate) mod utils; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs new file mode 100644 index 000000000000..5773e3797e2b --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs @@ -0,0 +1,212 @@ +use std::marker::PhantomData; + +use zk_evm_1_4_0::{ + aux_structures::Timestamp, + tracing::{BeforeExecutionData, VmLocalStateData}, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::{ + event::{ + extract_bytecode_publication_requests_from_l1_messenger, + extract_l2tol1logs_from_l1_messenger, extract_long_l2_to_l1_messages, L1MessengerL2ToL1Log, + }, + writes::StateDiffRecord, + zkevm_test_harness::witness::sort_storage_access::sort_storage_access_queries, + AccountTreeId, StorageKey, L1_MESSENGER_ADDRESS, +}; +use zksync_utils::{h256_to_u256, u256_to_bytes_be, u256_to_h256}; + +use crate::{ + interface::{ + dyn_tracers::vm_1_4_0::DynTracer, + tracer::{TracerExecutionStatus, TracerExecutionStopReason}, + types::inputs::L1BatchEnv, + VmExecutionMode, + }, + vm_latest::{ + bootloader_state::{utils::apply_pubdata_to_memory, BootloaderState}, + constants::BOOTLOADER_HEAP_PAGE, + old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, + tracers::{traits::VmTracer, utils::VmHook}, + types::internals::{PubdataInput, ZkSyncVmState}, + utils::logs::collect_events_and_l1_system_logs_after_timestamp, + StorageOracle, + }, +}; + +/// Tracer responsible for collecting information about refunds. +#[derive(Debug, Clone)] +pub(crate) struct PubdataTracer { + l1_batch_env: L1BatchEnv, + pubdata_info_requested: bool, + execution_mode: VmExecutionMode, + _phantom_data: PhantomData, +} + +impl PubdataTracer { + pub(crate) fn new(l1_batch_env: L1BatchEnv, execution_mode: VmExecutionMode) -> Self { + Self { + l1_batch_env, + pubdata_info_requested: false, + execution_mode, + _phantom_data: Default::default(), + } + } +} + +impl PubdataTracer { + // Packs part of L1 Messenger total pubdata that corresponds to + // L2toL1Logs sent in the block + fn get_total_user_logs( + &self, + state: &ZkSyncVmState, + ) -> Vec { + let (all_generated_events, _) = collect_events_and_l1_system_logs_after_timestamp( + state, + &self.l1_batch_env, + Timestamp(0), + ); + extract_l2tol1logs_from_l1_messenger(&all_generated_events) + } + + // Packs part of L1 Messenger total pubdata that corresponds to + // Messages sent in the block + fn get_total_l1_messenger_messages( + &self, + state: &ZkSyncVmState, + ) -> Vec> { + let (all_generated_events, _) = collect_events_and_l1_system_logs_after_timestamp( + state, + &self.l1_batch_env, + Timestamp(0), + ); + + extract_long_l2_to_l1_messages(&all_generated_events) + } + + // Packs part of L1 Messenger total pubdata that corresponds to + // Bytecodes needed to be published on L1 + fn get_total_published_bytecodes( + &self, + state: &ZkSyncVmState, + ) -> Vec> { + let (all_generated_events, _) = collect_events_and_l1_system_logs_after_timestamp( + state, + &self.l1_batch_env, + Timestamp(0), + ); + + let bytecode_publication_requests = + extract_bytecode_publication_requests_from_l1_messenger(&all_generated_events); + + bytecode_publication_requests + .iter() + .map(|bytecode_publication_request| { + state + .decommittment_processor + .known_bytecodes + .inner() + .get(&h256_to_u256(bytecode_publication_request.bytecode_hash)) + .unwrap() + .iter() + .flat_map(u256_to_bytes_be) + .collect() + }) + .collect() + } + + // Packs part of L1Messenger total pubdata that corresponds to + // State diffs needed to be published on L1 + fn get_state_diffs(storage: &StorageOracle) -> Vec { + sort_storage_access_queries( + storage + .storage_log_queries_after_timestamp(Timestamp(0)) + .iter() + .map(|log| &log.log_query), + ) + .1 + .into_iter() + .filter(|log| log.rw_flag) + .filter(|log| log.read_value != log.written_value) + .filter(|log| log.address != L1_MESSENGER_ADDRESS) + .map(|log| StateDiffRecord { + address: log.address, + key: log.key, + derived_key: log.derive_final_address(), + enumeration_index: storage + .storage + .get_ptr() + .borrow_mut() + .get_enumeration_index(&StorageKey::new( + AccountTreeId::new(log.address), + u256_to_h256(log.key), + )) + .unwrap_or_default(), + initial_value: log.read_value, + final_value: log.written_value, + }) + .collect() + } + + fn build_pubdata_input(&self, state: &ZkSyncVmState) -> PubdataInput { + PubdataInput { + user_logs: self.get_total_user_logs(state), + l2_to_l1_messages: self.get_total_l1_messenger_messages(state), + published_bytecodes: self.get_total_published_bytecodes(state), + state_diffs: Self::get_state_diffs(&state.storage), + } + } +} + +impl DynTracer> for PubdataTracer { + fn before_execution( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + _memory: &SimpleMemory, + _storage: StoragePtr, + ) { + let hook = VmHook::from_opcode_memory(&state, &data); + if let VmHook::PubdataRequested = hook { + self.pubdata_info_requested = true; + } + } +} + +impl VmTracer for PubdataTracer { + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + if !matches!(self.execution_mode, VmExecutionMode::Batch) { + // We do not provide the pubdata when executing the block tip or a single transaction + if self.pubdata_info_requested { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Finish); + } else { + return TracerExecutionStatus::Continue; + } + } + + if self.pubdata_info_requested { + let pubdata_input = self.build_pubdata_input(state); + + // Save the pubdata for the future initial bootloader memory building + bootloader_state.set_pubdata_input(pubdata_input.clone()); + + // Apply the pubdata to the current memory + let mut memory_to_apply = vec![]; + + apply_pubdata_to_memory(&mut memory_to_apply, pubdata_input); + state.memory.populate_page( + BOOTLOADER_HEAP_PAGE as usize, + memory_to_apply, + Timestamp(state.local_state.timestamp), + ); + + self.pubdata_info_requested = false; + } + + TracerExecutionStatus::Continue + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs new file mode 100644 index 000000000000..e852fba1dac8 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs @@ -0,0 +1,350 @@ +use std::marker::PhantomData; + +use vise::{Buckets, EncodeLabelSet, EncodeLabelValue, Family, Histogram, Metrics}; +use zk_evm_1_4_0::{ + aux_structures::Timestamp, + tracing::{BeforeExecutionData, VmLocalStateData}, + vm_state::VmLocalState, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; +use zksync_types::{ + event::{extract_long_l2_to_l1_messages, extract_published_bytecodes}, + l2_to_l1_log::L2ToL1Log, + L1BatchNumber, U256, +}; +use zksync_utils::{bytecode::bytecode_len_in_bytes, ceil_div_u256, u256_to_h256}; + +use crate::{ + interface::{ + traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, types::tracer::TracerExecutionStatus, + L1BatchEnv, Refunds, + }, + vm_latest::{ + bootloader_state::BootloaderState, + constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, + old_vm::{ + events::merge_events, history_recorder::HistoryMode, memory::SimpleMemory, + utils::eth_price_per_pubdata_byte, + }, + tracers::{ + traits::VmTracer, + utils::{ + gas_spent_on_bytecodes_and_long_messages_this_opcode, get_vm_hook_params, VmHook, + }, + }, + types::internals::ZkSyncVmState, + }, +}; + +/// Tracer responsible for collecting information about refunds. +#[derive(Debug, Clone)] +pub(crate) struct RefundsTracer { + // Some(x) means that the bootloader has asked the operator + // to provide the refund the user, where `x` is the refund proposed + // by the bootloader itself. + pending_operator_refund: Option, + refund_gas: u32, + operator_refund: Option, + timestamp_initial: Timestamp, + timestamp_before_cycle: Timestamp, + gas_remaining_before: u32, + spent_pubdata_counter_before: u32, + gas_spent_on_bytecodes_and_long_messages: u32, + l1_batch: L1BatchEnv, + pubdata_published: u32, + _phantom: PhantomData, +} + +impl RefundsTracer { + pub(crate) fn new(l1_batch: L1BatchEnv) -> Self { + Self { + pending_operator_refund: None, + refund_gas: 0, + operator_refund: None, + timestamp_initial: Timestamp(0), + timestamp_before_cycle: Timestamp(0), + gas_remaining_before: 0, + spent_pubdata_counter_before: 0, + gas_spent_on_bytecodes_and_long_messages: 0, + l1_batch, + pubdata_published: 0, + _phantom: PhantomData, + } + } +} + +impl RefundsTracer { + fn requested_refund(&self) -> Option { + self.pending_operator_refund + } + + fn set_refund_as_done(&mut self) { + self.pending_operator_refund = None; + } + + fn block_overhead_refund(&mut self) -> u32 { + 0 + } + + pub(crate) fn get_refunds(&self) -> Refunds { + Refunds { + gas_refunded: self.refund_gas, + operator_suggested_refund: self.operator_refund.unwrap_or_default(), + } + } + + pub(crate) fn tx_body_refund( + &self, + bootloader_refund: u32, + gas_spent_on_pubdata: u32, + tx_gas_limit: u32, + current_ergs_per_pubdata_byte: u32, + pubdata_published: u32, + ) -> u32 { + let total_gas_spent = tx_gas_limit - bootloader_refund; + + let gas_spent_on_computation = total_gas_spent + .checked_sub(gas_spent_on_pubdata) + .unwrap_or_else(|| { + tracing::error!( + "Gas spent on pubdata is greater than total gas spent. On pubdata: {}, total: {}", + gas_spent_on_pubdata, + total_gas_spent + ); + 0 + }); + + // For now, bootloader charges only for base fee. + let effective_gas_price = self.l1_batch.base_fee(); + + let bootloader_eth_price_per_pubdata_byte = + U256::from(effective_gas_price) * U256::from(current_ergs_per_pubdata_byte); + + let fair_eth_price_per_pubdata_byte = + U256::from(eth_price_per_pubdata_byte(self.l1_batch.l1_gas_price)); + + // For now, L1 originated transactions are allowed to pay less than fair fee per pubdata, + // so we should take it into account. + let eth_price_per_pubdata_byte_for_calculation = std::cmp::min( + bootloader_eth_price_per_pubdata_byte, + fair_eth_price_per_pubdata_byte, + ); + + let fair_fee_eth = U256::from(gas_spent_on_computation) + * U256::from(self.l1_batch.fair_l2_gas_price) + + U256::from(pubdata_published) * eth_price_per_pubdata_byte_for_calculation; + let pre_paid_eth = U256::from(tx_gas_limit) * U256::from(effective_gas_price); + let refund_eth = pre_paid_eth.checked_sub(fair_fee_eth).unwrap_or_else(|| { + tracing::error!( + "Fair fee is greater than pre paid. Fair fee: {} wei, pre paid: {} wei", + fair_fee_eth, + pre_paid_eth + ); + U256::zero() + }); + + ceil_div_u256(refund_eth, effective_gas_price.into()).as_u32() + } + + pub(crate) fn gas_spent_on_pubdata(&self, vm_local_state: &VmLocalState) -> u32 { + self.gas_spent_on_bytecodes_and_long_messages + vm_local_state.spent_pubdata_counter + } + + pub(crate) fn pubdata_published(&self) -> u32 { + self.pubdata_published + } +} + +impl DynTracer> for RefundsTracer { + fn before_execution( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + memory: &SimpleMemory, + _storage: StoragePtr, + ) { + self.timestamp_before_cycle = Timestamp(state.vm_local_state.timestamp); + let hook = VmHook::from_opcode_memory(&state, &data); + match hook { + VmHook::NotifyAboutRefund => self.refund_gas = get_vm_hook_params(memory)[0].as_u32(), + VmHook::AskOperatorForRefund => { + self.pending_operator_refund = Some(get_vm_hook_params(memory)[0].as_u32()) + } + _ => {} + } + + self.gas_spent_on_bytecodes_and_long_messages += + gas_spent_on_bytecodes_and_long_messages_this_opcode(&state, &data); + } +} + +impl VmTracer for RefundsTracer { + fn initialize_tracer(&mut self, state: &mut ZkSyncVmState) { + self.timestamp_initial = Timestamp(state.local_state.timestamp); + self.gas_remaining_before = state.local_state.callstack.current.ergs_remaining; + self.spent_pubdata_counter_before = state.local_state.spent_pubdata_counter; + } + + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelValue, EncodeLabelSet)] + #[metrics(label = "type", rename_all = "snake_case")] + enum RefundType { + Bootloader, + Operator, + } + + const PERCENT_BUCKETS: Buckets = Buckets::values(&[ + 5.0, 10.0, 20.0, 30.0, 40.0, 50.0, 60.0, 70.0, 80.0, 90.0, 100.0, 120.0, + ]); + + #[derive(Debug, Metrics)] + #[metrics(prefix = "vm")] + struct RefundMetrics { + #[metrics(buckets = PERCENT_BUCKETS)] + refund: Family>, + #[metrics(buckets = PERCENT_BUCKETS)] + refund_diff: Histogram, + } + + #[vise::register] + static METRICS: vise::Global = vise::Global::new(); + + // This means that the bootloader has informed the system (usually via VMHooks) - that some gas + // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + if let Some(bootloader_refund) = self.requested_refund() { + assert!( + self.operator_refund.is_none(), + "Operator was asked for refund two times" + ); + let gas_spent_on_pubdata = + self.gas_spent_on_pubdata(&state.local_state) - self.spent_pubdata_counter_before; + + let current_tx_index = bootloader_state.current_tx(); + let tx_description_offset = + bootloader_state.get_tx_description_offset(current_tx_index); + let tx_gas_limit = state + .memory + .read_slot( + BOOTLOADER_HEAP_PAGE as usize, + tx_description_offset + TX_GAS_LIMIT_OFFSET, + ) + .value + .as_u32(); + + let used_published_storage_slots = state + .storage + .save_paid_changes(Timestamp(state.local_state.timestamp)); + + let pubdata_published = pubdata_published( + state, + used_published_storage_slots, + self.timestamp_initial, + self.l1_batch.number, + ); + + self.pubdata_published = pubdata_published; + let current_ergs_per_pubdata_byte = state.local_state.current_ergs_per_pubdata_byte; + let tx_body_refund = self.tx_body_refund( + bootloader_refund, + gas_spent_on_pubdata, + tx_gas_limit, + current_ergs_per_pubdata_byte, + pubdata_published, + ); + + if tx_body_refund < bootloader_refund { + tracing::error!( + "Suggested tx body refund is less than bootloader refund. Tx body refund: {tx_body_refund}, \ + bootloader refund: {bootloader_refund}" + ); + } + + let refund_to_propose = tx_body_refund + self.block_overhead_refund(); + + let refund_slot = OPERATOR_REFUNDS_OFFSET + current_tx_index; + + // Writing the refund into memory + state.memory.populate_page( + BOOTLOADER_HEAP_PAGE as usize, + vec![(refund_slot, refund_to_propose.into())], + self.timestamp_before_cycle, + ); + + bootloader_state.set_refund_for_current_tx(refund_to_propose); + self.operator_refund = Some(refund_to_propose); + self.set_refund_as_done(); + + if tx_gas_limit < bootloader_refund { + tracing::error!( + "Tx gas limit is less than bootloader refund. Tx gas limit: {tx_gas_limit}, \ + bootloader refund: {bootloader_refund}" + ); + } + if tx_gas_limit < refund_to_propose { + tracing::error!( + "Tx gas limit is less than operator refund. Tx gas limit: {tx_gas_limit}, \ + operator refund: {refund_to_propose}" + ); + } + + METRICS.refund[&RefundType::Bootloader] + .observe(bootloader_refund as f64 / tx_gas_limit as f64 * 100.0); + METRICS.refund[&RefundType::Operator] + .observe(refund_to_propose as f64 / tx_gas_limit as f64 * 100.0); + let refund_diff = + (refund_to_propose as f64 - bootloader_refund as f64) / tx_gas_limit as f64 * 100.0; + METRICS.refund_diff.observe(refund_diff); + } + TracerExecutionStatus::Continue + } +} + +/// Returns the given transactions' gas limit - by reading it directly from the VM memory. +pub(crate) fn pubdata_published( + state: &ZkSyncVmState, + storage_writes_pubdata_published: u32, + from_timestamp: Timestamp, + batch_number: L1BatchNumber, +) -> u32 { + let (raw_events, l1_messages) = state + .event_sink + .get_events_and_l2_l1_logs_after_timestamp(from_timestamp); + let events: Vec<_> = merge_events(raw_events) + .into_iter() + .map(|e| e.into_vm_event(batch_number)) + .collect(); + // For the first transaction in L1 batch there may be (it depends on the execution mode) an L2->L1 log + // that is sent by `SystemContext` in `setNewBlock`. It's a part of the L1 batch pubdata overhead and not the transaction itself. + let l2_l1_logs_bytes = (l1_messages + .into_iter() + .map(|log| L2ToL1Log { + shard_id: log.shard_id, + is_service: log.is_first, + tx_number_in_block: log.tx_number_in_block, + sender: log.address, + key: u256_to_h256(log.key), + value: u256_to_h256(log.value), + }) + .filter(|log| log.sender != SYSTEM_CONTEXT_ADDRESS) + .count() as u32) + * zk_evm_1_4_0::zkevm_opcode_defs::system_params::L1_MESSAGE_PUBDATA_BYTES; + let l2_l1_long_messages_bytes: u32 = extract_long_l2_to_l1_messages(&events) + .iter() + .map(|event| event.len() as u32) + .sum(); + + let published_bytecode_bytes: u32 = extract_published_bytecodes(&events) + .iter() + .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .sum(); + + storage_writes_pubdata_published + + l2_l1_logs_bytes + + l2_l1_long_messages_bytes + + published_bytecode_bytes +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs new file mode 100644 index 000000000000..b3412587725c --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs @@ -0,0 +1,246 @@ +use std::marker::PhantomData; + +use zk_evm_1_4_0::{ + tracing::{AfterDecodingData, BeforeExecutionData, VmLocalStateData}, + vm_state::{ErrorFlags, VmLocalState}, + zkevm_opcode_defs::FatPointer, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::U256; + +use crate::{ + interface::{ + tracer::VmExecutionStopReason, traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, + types::tracer::TracerExecutionStopReason, ExecutionResult, Halt, TxRevertReason, + VmExecutionMode, VmRevertReason, + }, + vm_latest::{ + constants::{BOOTLOADER_HEAP_PAGE, RESULT_SUCCESS_FIRST_SLOT}, + old_vm::utils::{vm_may_have_ended_inner, VmExecutionResult}, + tracers::{ + traits::VmTracer, + utils::{get_vm_hook_params, read_pointer, VmHook}, + }, + types::internals::ZkSyncVmState, + BootloaderState, HistoryMode, SimpleMemory, + }, +}; + +#[derive(Debug, Clone)] +enum Result { + Error { error_reason: VmRevertReason }, + Success { return_data: Vec }, + Halt { reason: Halt }, +} + +/// Tracer responsible for handling the VM execution result. +#[derive(Debug, Clone)] +pub(crate) struct ResultTracer { + result: Option, + bootloader_out_of_gas: bool, + execution_mode: VmExecutionMode, + _phantom: PhantomData, +} + +impl ResultTracer { + pub(crate) fn new(execution_mode: VmExecutionMode) -> Self { + Self { + result: None, + bootloader_out_of_gas: false, + execution_mode, + _phantom: PhantomData, + } + } +} + +fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { + // The current frame is bootloader if the callstack depth is 1. + // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior + // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. + local_state.callstack.inner.len() == 1 +} + +impl DynTracer> for ResultTracer { + fn after_decoding( + &mut self, + state: VmLocalStateData<'_>, + data: AfterDecodingData, + _memory: &SimpleMemory, + ) { + // We should check not only for the `NOT_ENOUGH_ERGS` flag but if the current frame is bootloader too. + if current_frame_is_bootloader(state.vm_local_state) + && data + .error_flags_accumulated + .contains(ErrorFlags::NOT_ENOUGH_ERGS) + { + self.bootloader_out_of_gas = true; + } + } + + fn before_execution( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + memory: &SimpleMemory, + _storage: StoragePtr, + ) { + let hook = VmHook::from_opcode_memory(&state, &data); + if let VmHook::ExecutionResult = hook { + let vm_hook_params = get_vm_hook_params(memory); + let success = vm_hook_params[0]; + let returndata_ptr = FatPointer::from_u256(vm_hook_params[1]); + let returndata = read_pointer(memory, returndata_ptr); + if success == U256::zero() { + self.result = Some(Result::Error { + // Tx has reverted, without bootloader error, we can simply parse the revert reason + error_reason: (VmRevertReason::from(returndata.as_slice())), + }); + } else { + self.result = Some(Result::Success { + return_data: returndata, + }); + } + } + } +} + +impl VmTracer for ResultTracer { + fn after_vm_execution( + &mut self, + state: &mut ZkSyncVmState, + bootloader_state: &BootloaderState, + stop_reason: VmExecutionStopReason, + ) { + match stop_reason { + // Vm has finished execution, we need to check the result of it + VmExecutionStopReason::VmFinished => { + self.vm_finished_execution(state); + } + // One of the tracers above has requested to stop the execution. + // If it was the correct stop we already have the result, + // otherwise it can be out of gas error + VmExecutionStopReason::TracerRequestedStop(reason) => { + match self.execution_mode { + VmExecutionMode::OneTx => { + self.vm_stopped_execution(state, bootloader_state, reason) + } + VmExecutionMode::Batch => self.vm_finished_execution(state), + VmExecutionMode::Bootloader => self.vm_finished_execution(state), + }; + } + } + } +} + +impl ResultTracer { + fn vm_finished_execution(&mut self, state: &ZkSyncVmState) { + let Some(result) = vm_may_have_ended_inner(state) else { + // The VM has finished execution, but the result is not yet available. + self.result = Some(Result::Success { + return_data: vec![], + }); + return; + }; + + // Check it's not inside tx + match result { + VmExecutionResult::Ok(output) => { + self.result = Some(Result::Success { + return_data: output, + }); + } + VmExecutionResult::Revert(output) => { + // Unlike VmHook::ExecutionResult, vm has completely finished and returned not only the revert reason, + // but with bytecode, which represents the type of error from the bootloader side + let revert_reason = TxRevertReason::parse_error(&output); + + match revert_reason { + TxRevertReason::TxReverted(reason) => { + self.result = Some(Result::Error { + error_reason: reason, + }); + } + TxRevertReason::Halt(halt) => { + self.result = Some(Result::Halt { reason: halt }); + } + }; + } + VmExecutionResult::Panic => { + if self.bootloader_out_of_gas { + self.result = Some(Result::Halt { + reason: Halt::BootloaderOutOfGas, + }); + } else { + self.result = Some(Result::Halt { + reason: Halt::VMPanic, + }); + } + } + VmExecutionResult::MostLikelyDidNotFinish(_, _) => { + unreachable!() + } + } + } + + fn vm_stopped_execution( + &mut self, + state: &ZkSyncVmState, + bootloader_state: &BootloaderState, + reason: TracerExecutionStopReason, + ) { + if let TracerExecutionStopReason::Abort(halt) = reason { + self.result = Some(Result::Halt { reason: halt }); + return; + } + + if self.bootloader_out_of_gas { + self.result = Some(Result::Halt { + reason: Halt::BootloaderOutOfGas, + }); + } else { + if self.result.is_some() { + return; + } + + let has_failed = tx_has_failed(state, bootloader_state.current_tx() as u32); + if has_failed { + self.result = Some(Result::Error { + error_reason: VmRevertReason::General { + msg: "Transaction reverted with empty reason. Possibly out of gas" + .to_string(), + data: vec![], + }, + }); + } else { + self.result = Some(self.result.clone().unwrap_or(Result::Success { + return_data: vec![], + })); + } + } + } + + pub(crate) fn into_result(self) -> ExecutionResult { + match self.result.unwrap() { + Result::Error { error_reason } => ExecutionResult::Revert { + output: error_reason, + }, + Result::Success { return_data } => ExecutionResult::Success { + output: return_data, + }, + Result::Halt { reason } => ExecutionResult::Halt { reason }, + } + } +} + +pub(crate) fn tx_has_failed( + state: &ZkSyncVmState, + tx_id: u32, +) -> bool { + let mem_slot = RESULT_SUCCESS_FIRST_SLOT + tx_id; + let mem_value = state + .memory + .read_slot(BOOTLOADER_HEAP_PAGE as usize, mem_slot as usize) + .value; + + mem_value == U256::zero() +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs new file mode 100644 index 000000000000..68307b3f2867 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs @@ -0,0 +1,47 @@ +use zksync_state::WriteStorage; + +use crate::{ + interface::{ + dyn_tracers::vm_1_4_0::DynTracer, + tracer::{TracerExecutionStatus, VmExecutionStopReason}, + }, + vm_latest::{ + bootloader_state::BootloaderState, + old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, + types::internals::ZkSyncVmState, + }, +}; + +pub type TracerPointer = Box>; + +/// Run tracer for collecting data during the vm execution cycles +pub trait VmTracer: DynTracer> { + /// Initialize the tracer before the vm execution + fn initialize_tracer(&mut self, _state: &mut ZkSyncVmState) {} + /// Run after each vm execution cycle + fn finish_cycle( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + TracerExecutionStatus::Continue + } + /// Run after the vm execution + fn after_vm_execution( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &BootloaderState, + _stop_reason: VmExecutionStopReason, + ) { + } +} + +pub trait ToTracerPointer { + fn into_tracer_pointer(self) -> TracerPointer; +} + +impl + 'static> ToTracerPointer for T { + fn into_tracer_pointer(self) -> TracerPointer { + Box::new(self) + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs new file mode 100644 index 000000000000..93710586fdac --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs @@ -0,0 +1,225 @@ +use zk_evm_1_4_0::{ + aux_structures::MemoryPage, + tracing::{BeforeExecutionData, VmLocalStateData}, + zkevm_opcode_defs::{ + FarCallABI, FarCallForwardPageType, FatPointer, LogOpcode, Opcode, UMAOpcode, + }, +}; +use zksync_system_constants::{ + ECRECOVER_PRECOMPILE_ADDRESS, KECCAK256_PRECOMPILE_ADDRESS, KNOWN_CODES_STORAGE_ADDRESS, + L1_MESSENGER_ADDRESS, SHA256_PRECOMPILE_ADDRESS, +}; +use zksync_types::U256; +use zksync_utils::u256_to_h256; + +use crate::vm_latest::{ + constants::{ + BOOTLOADER_HEAP_PAGE, VM_HOOK_PARAMS_COUNT, VM_HOOK_PARAMS_START_POSITION, VM_HOOK_POSITION, + }, + old_vm::{ + history_recorder::HistoryMode, + memory::SimpleMemory, + utils::{aux_heap_page_from_base, heap_page_from_base}, + }, +}; + +#[derive(Clone, Debug, Copy)] +pub(crate) enum VmHook { + AccountValidationEntered, + PaymasterValidationEntered, + NoValidationEntered, + ValidationStepEndeded, + TxHasEnded, + DebugLog, + DebugReturnData, + NoHook, + NearCallCatch, + AskOperatorForRefund, + NotifyAboutRefund, + ExecutionResult, + FinalBatchInfo, + // Hook used to signal that the final pubdata for a batch is requested. + PubdataRequested, +} + +impl VmHook { + pub(crate) fn from_opcode_memory( + state: &VmLocalStateData<'_>, + data: &BeforeExecutionData, + ) -> Self { + let opcode_variant = data.opcode.variant; + let heap_page = + heap_page_from_base(state.vm_local_state.callstack.current.base_memory_page).0; + + let src0_value = data.src0_value.value; + + let fat_ptr = FatPointer::from_u256(src0_value); + + let value = data.src1_value.value; + + // Only UMA opcodes in the bootloader serve for vm hooks + if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) + || heap_page != BOOTLOADER_HEAP_PAGE + || fat_ptr.offset != VM_HOOK_POSITION * 32 + { + return Self::NoHook; + } + + match value.as_u32() { + 0 => Self::AccountValidationEntered, + 1 => Self::PaymasterValidationEntered, + 2 => Self::NoValidationEntered, + 3 => Self::ValidationStepEndeded, + 4 => Self::TxHasEnded, + 5 => Self::DebugLog, + 6 => Self::DebugReturnData, + 7 => Self::NearCallCatch, + 8 => Self::AskOperatorForRefund, + 9 => Self::NotifyAboutRefund, + 10 => Self::ExecutionResult, + 11 => Self::FinalBatchInfo, + 12 => Self::PubdataRequested, + _ => panic!("Unknown hook: {}", value.as_u32()), + } + } +} + +pub(crate) fn get_debug_log( + state: &VmLocalStateData<'_>, + memory: &SimpleMemory, +) -> String { + let vm_hook_params: Vec<_> = get_vm_hook_params(memory) + .into_iter() + .map(u256_to_h256) + .collect(); + let msg = vm_hook_params[0].as_bytes().to_vec(); + let data = vm_hook_params[1].as_bytes().to_vec(); + + let msg = String::from_utf8(msg).expect("Invalid debug message"); + let data = U256::from_big_endian(&data); + + // For long data, it is better to use hex-encoding for greater readability + let data_str = if data > U256::from(u64::max_value()) { + let mut bytes = [0u8; 32]; + data.to_big_endian(&mut bytes); + format!("0x{}", hex::encode(bytes)) + } else { + data.to_string() + }; + + let tx_id = state.vm_local_state.tx_number_in_block; + + format!("Bootloader transaction {}: {} {}", tx_id, msg, data_str) +} + +/// Reads the memory slice represented by the fat pointer. +/// Note, that the fat pointer must point to the accessible memory (i.e. not cleared up yet). +pub(crate) fn read_pointer( + memory: &SimpleMemory, + pointer: FatPointer, +) -> Vec { + let FatPointer { + offset, + length, + start, + memory_page, + } = pointer; + + // The actual bounds of the returndata ptr is [start+offset..start+length] + let mem_region_start = start + offset; + let mem_region_length = length - offset; + + memory.read_unaligned_bytes( + memory_page as usize, + mem_region_start as usize, + mem_region_length as usize, + ) +} + +/// Outputs the returndata for the latest call. +/// This is usually used to output the revert reason. +pub(crate) fn get_debug_returndata(memory: &SimpleMemory) -> String { + let vm_hook_params: Vec<_> = get_vm_hook_params(memory); + let returndata_ptr = FatPointer::from_u256(vm_hook_params[0]); + let returndata = read_pointer(memory, returndata_ptr); + + format!("0x{}", hex::encode(returndata)) +} + +/// Accepts a vm hook and, if it requires to output some debug log, outputs it. +pub(crate) fn print_debug_if_needed( + hook: &VmHook, + state: &VmLocalStateData<'_>, + memory: &SimpleMemory, +) { + let log = match hook { + VmHook::DebugLog => get_debug_log(state, memory), + VmHook::DebugReturnData => get_debug_returndata(memory), + _ => return, + }; + + tracing::trace!("{}", log); +} + +pub(crate) fn computational_gas_price( + state: VmLocalStateData<'_>, + data: &BeforeExecutionData, +) -> u32 { + // We calculate computational gas used as a raw price for opcode plus cost for precompiles. + // This calculation is incomplete as it misses decommitment and memory growth costs. + // To calculate decommitment cost we need an access to decommitter oracle which is missing in tracer now. + // Memory growth calculation is complex and it will require different logic for different opcodes (`FarCall`, `Ret`, `UMA`). + let base_price = data.opcode.inner.variant.ergs_price(); + let precompile_price = match data.opcode.variant.opcode { + Opcode::Log(LogOpcode::PrecompileCall) => { + let address = state.vm_local_state.callstack.current.this_address; + + if address == KECCAK256_PRECOMPILE_ADDRESS + || address == SHA256_PRECOMPILE_ADDRESS + || address == ECRECOVER_PRECOMPILE_ADDRESS + { + data.src1_value.value.low_u32() + } else { + 0 + } + } + _ => 0, + }; + base_price + precompile_price +} + +pub(crate) fn gas_spent_on_bytecodes_and_long_messages_this_opcode( + state: &VmLocalStateData<'_>, + data: &BeforeExecutionData, +) -> u32 { + if data.opcode.variant.opcode == Opcode::Log(LogOpcode::PrecompileCall) { + let current_stack = state.vm_local_state.callstack.get_current_stack(); + // Trace for precompile calls from `KNOWN_CODES_STORAGE_ADDRESS` and `L1_MESSENGER_ADDRESS` that burn some gas. + // Note, that if there is less gas left than requested to burn it will be burnt anyway. + if current_stack.this_address == KNOWN_CODES_STORAGE_ADDRESS + || current_stack.this_address == L1_MESSENGER_ADDRESS + { + std::cmp::min(data.src1_value.value.as_u32(), current_stack.ergs_remaining) + } else { + 0 + } + } else { + 0 + } +} + +pub(crate) fn get_calldata_page_via_abi(far_call_abi: &FarCallABI, base_page: MemoryPage) -> u32 { + match far_call_abi.forwarding_mode { + FarCallForwardPageType::ForwardFatPointer => { + far_call_abi.memory_quasi_fat_pointer.memory_page + } + FarCallForwardPageType::UseAuxHeap => aux_heap_page_from_base(base_page).0, + FarCallForwardPageType::UseHeap => heap_page_from_base(base_page).0, + } +} +pub(crate) fn get_vm_hook_params(memory: &SimpleMemory) -> Vec { + memory.dump_page_content_as_u256_words( + BOOTLOADER_HEAP_PAGE, + VM_HOOK_PARAMS_START_POSITION..VM_HOOK_PARAMS_START_POSITION + VM_HOOK_PARAMS_COUNT, + ) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/mod.rs new file mode 100644 index 000000000000..7dc60ec5b0fb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/mod.rs @@ -0,0 +1,9 @@ +pub(crate) use pubdata::PubdataInput; +pub(crate) use snapshot::VmSnapshot; +pub(crate) use transaction_data::TransactionData; +pub(crate) use vm_state::new_vm_state; +pub use vm_state::ZkSyncVmState; +mod pubdata; +mod snapshot; +mod transaction_data; +mod vm_state; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs new file mode 100644 index 000000000000..aa9e8a7eda70 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs @@ -0,0 +1,124 @@ +use zksync_types::{ + event::L1MessengerL2ToL1Log, + writes::{compress_state_diffs, StateDiffRecord}, +}; + +/// Struct based on which the pubdata blob is formed +#[derive(Debug, Clone, Default)] +pub(crate) struct PubdataInput { + pub(crate) user_logs: Vec, + pub(crate) l2_to_l1_messages: Vec>, + pub(crate) published_bytecodes: Vec>, + pub(crate) state_diffs: Vec, +} + +impl PubdataInput { + pub(crate) fn build_pubdata(self, with_uncompressed_state_diffs: bool) -> Vec { + let mut l1_messenger_pubdata = vec![]; + + let PubdataInput { + user_logs, + l2_to_l1_messages, + published_bytecodes, + state_diffs, + } = self; + + // Encoding user L2->L1 logs. + // Format: [(numberOfL2ToL1Logs as u32) || l2tol1logs[1] || ... || l2tol1logs[n]] + l1_messenger_pubdata.extend((user_logs.len() as u32).to_be_bytes()); + for l2tol1log in user_logs { + l1_messenger_pubdata.extend(l2tol1log.packed_encoding()); + } + + // Encoding L2->L1 messages + // Format: [(numberOfMessages as u32) || (messages[1].len() as u32) || messages[1] || ... || (messages[n].len() as u32) || messages[n]] + l1_messenger_pubdata.extend((l2_to_l1_messages.len() as u32).to_be_bytes()); + for message in l2_to_l1_messages { + l1_messenger_pubdata.extend((message.len() as u32).to_be_bytes()); + l1_messenger_pubdata.extend(message); + } + + // Encoding bytecodes + // Format: [(numberOfBytecodes as u32) || (bytecodes[1].len() as u32) || bytecodes[1] || ... || (bytecodes[n].len() as u32) || bytecodes[n]] + l1_messenger_pubdata.extend((published_bytecodes.len() as u32).to_be_bytes()); + for bytecode in published_bytecodes { + l1_messenger_pubdata.extend((bytecode.len() as u32).to_be_bytes()); + l1_messenger_pubdata.extend(bytecode); + } + + // Encoding state diffs + // Format: [size of compressed state diffs u32 || compressed state diffs || (# state diffs: intial + repeated) as u32 || sorted state diffs by ] + let state_diffs_compressed = compress_state_diffs(state_diffs.clone()); + l1_messenger_pubdata.extend(state_diffs_compressed); + + if with_uncompressed_state_diffs { + l1_messenger_pubdata.extend((state_diffs.len() as u32).to_be_bytes()); + for state_diff in state_diffs { + l1_messenger_pubdata.extend(state_diff.encode_padded()); + } + } + + l1_messenger_pubdata + } +} + +#[cfg(test)] +mod tests { + use zksync_system_constants::{ACCOUNT_CODE_STORAGE_ADDRESS, BOOTLOADER_ADDRESS}; + use zksync_utils::u256_to_h256; + + use super::*; + + #[test] + fn test_basic_pubdata_building() { + // Just using some constant addresses for tests + let addr1 = BOOTLOADER_ADDRESS; + let addr2 = ACCOUNT_CODE_STORAGE_ADDRESS; + + let user_logs = vec![L1MessengerL2ToL1Log { + l2_shard_id: 0, + is_service: false, + tx_number_in_block: 0, + sender: addr1, + key: 1.into(), + value: 128.into(), + }]; + + let l2_to_l1_messages = vec![hex::decode("deadbeef").unwrap()]; + + let published_bytecodes = vec![hex::decode("aaaabbbb").unwrap()]; + + // For covering more cases, we have two state diffs: + // One with enumeration index present (and so it is a repeated write) and the one without it. + let state_diffs = vec![ + StateDiffRecord { + address: addr2, + key: 155.into(), + derived_key: u256_to_h256(125.into()).0, + enumeration_index: 12, + initial_value: 11.into(), + final_value: 12.into(), + }, + StateDiffRecord { + address: addr2, + key: 156.into(), + derived_key: u256_to_h256(126.into()).0, + enumeration_index: 0, + initial_value: 0.into(), + final_value: 14.into(), + }, + ]; + + let input = PubdataInput { + user_logs, + l2_to_l1_messages, + published_bytecodes, + state_diffs, + }; + + let pubdata = + ethabi::encode(&[ethabi::Token::Bytes(input.build_pubdata(true))])[32..].to_vec(); + + assert_eq!(hex::encode(pubdata), "00000000000000000000000000000000000000000000000000000000000002c700000001000000000000000000000000000000000000000000008001000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000000000000000800000000100000004deadbeef0000000100000004aaaabbbb0100002a040001000000000000000000000000000000000000000000000000000000000000007e090e0000000c0901000000020000000000000000000000000000000000008002000000000000000000000000000000000000000000000000000000000000009b000000000000000000000000000000000000000000000000000000000000007d000000000000000c000000000000000000000000000000000000000000000000000000000000000b000000000000000000000000000000000000000000000000000000000000000c00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000008002000000000000000000000000000000000000000000000000000000000000009c000000000000000000000000000000000000000000000000000000000000007e00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000e000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs new file mode 100644 index 000000000000..2a9368c37a39 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs @@ -0,0 +1,11 @@ +use zk_evm_1_4_0::vm_state::VmLocalState; + +use crate::vm_latest::bootloader_state::BootloaderStateSnapshot; + +/// A snapshot of the VM that holds enough information to +/// rollback the VM to some historical state. +#[derive(Debug, Clone)] +pub(crate) struct VmSnapshot { + pub(crate) local_state: VmLocalState, + pub(crate) bootloader_state: BootloaderStateSnapshot, +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs new file mode 100644 index 000000000000..f5b97ca9793e --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs @@ -0,0 +1,345 @@ +use std::convert::TryInto; + +use zksync_types::{ + ethabi::{encode, Address, Token}, + fee::{encoding_len, Fee}, + l1::is_l1_tx_type, + l2::{L2Tx, TransactionType}, + transaction_request::{PaymasterParams, TransactionRequest}, + Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, + U256, +}; +use zksync_utils::{address_to_h256, bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256}; + +use crate::vm_latest::utils::overhead::{get_amortized_overhead, OverheadCoefficients}; + +/// This structure represents the data that is used by +/// the Bootloader to describe the transaction. +#[derive(Debug, Default, Clone)] +pub(crate) struct TransactionData { + pub(crate) tx_type: u8, + pub(crate) from: Address, + pub(crate) to: Address, + pub(crate) gas_limit: U256, + pub(crate) pubdata_price_limit: U256, + pub(crate) max_fee_per_gas: U256, + pub(crate) max_priority_fee_per_gas: U256, + pub(crate) paymaster: Address, + pub(crate) nonce: U256, + pub(crate) value: U256, + // The reserved fields that are unique for different types of transactions. + // E.g. nonce is currently used in all transaction, but it should not be mandatory + // in the long run. + pub(crate) reserved: [U256; 4], + pub(crate) data: Vec, + pub(crate) signature: Vec, + // The factory deps provided with the transaction. + // Note that *only hashes* of these bytecodes are signed by the user + // and they are used in the ABI encoding of the struct. + // TODO: include this into the tx signature as part of SMA-1010 + pub(crate) factory_deps: Vec>, + pub(crate) paymaster_input: Vec, + pub(crate) reserved_dynamic: Vec, + pub(crate) raw_bytes: Option>, +} + +impl From for TransactionData { + fn from(execute_tx: Transaction) -> Self { + match execute_tx.common_data { + ExecuteTransactionCommon::L2(common_data) => { + let nonce = U256::from_big_endian(&common_data.nonce.to_be_bytes()); + + let should_check_chain_id = if matches!( + common_data.transaction_type, + TransactionType::LegacyTransaction + ) && common_data.extract_chain_id().is_some() + { + U256([1, 0, 0, 0]) + } else { + U256::zero() + }; + + TransactionData { + tx_type: (common_data.transaction_type as u32) as u8, + from: common_data.initiator_address, + to: execute_tx.execute.contract_address, + gas_limit: common_data.fee.gas_limit, + pubdata_price_limit: common_data.fee.gas_per_pubdata_limit, + max_fee_per_gas: common_data.fee.max_fee_per_gas, + max_priority_fee_per_gas: common_data.fee.max_priority_fee_per_gas, + paymaster: common_data.paymaster_params.paymaster, + nonce, + value: execute_tx.execute.value, + reserved: [ + should_check_chain_id, + U256::zero(), + U256::zero(), + U256::zero(), + ], + data: execute_tx.execute.calldata, + signature: common_data.signature, + factory_deps: execute_tx.execute.factory_deps.unwrap_or_default(), + paymaster_input: common_data.paymaster_params.paymaster_input, + reserved_dynamic: vec![], + raw_bytes: execute_tx.raw_bytes.map(|a| a.0), + } + } + ExecuteTransactionCommon::L1(common_data) => { + let refund_recipient = h256_to_u256(address_to_h256(&common_data.refund_recipient)); + TransactionData { + tx_type: common_data.tx_format() as u8, + from: common_data.sender, + to: execute_tx.execute.contract_address, + gas_limit: common_data.gas_limit, + pubdata_price_limit: common_data.gas_per_pubdata_limit, + // It doesn't matter what we put here, since + // the bootloader does not charge anything + max_fee_per_gas: common_data.max_fee_per_gas, + max_priority_fee_per_gas: U256::zero(), + paymaster: Address::default(), + nonce: U256::from(common_data.serial_id.0), // priority op ID + value: execute_tx.execute.value, + reserved: [ + common_data.to_mint, + refund_recipient, + U256::zero(), + U256::zero(), + ], + data: execute_tx.execute.calldata, + // The signature isn't checked for L1 transactions so we don't care + signature: vec![], + factory_deps: execute_tx.execute.factory_deps.unwrap_or_default(), + paymaster_input: vec![], + reserved_dynamic: vec![], + raw_bytes: None, + } + } + ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { + let refund_recipient = h256_to_u256(address_to_h256(&common_data.refund_recipient)); + TransactionData { + tx_type: common_data.tx_format() as u8, + from: common_data.sender, + to: execute_tx.execute.contract_address, + gas_limit: common_data.gas_limit, + pubdata_price_limit: common_data.gas_per_pubdata_limit, + // It doesn't matter what we put here, since + // the bootloader does not charge anything + max_fee_per_gas: common_data.max_fee_per_gas, + max_priority_fee_per_gas: U256::zero(), + paymaster: Address::default(), + nonce: U256::from(common_data.upgrade_id as u16), + value: execute_tx.execute.value, + reserved: [ + common_data.to_mint, + refund_recipient, + U256::zero(), + U256::zero(), + ], + data: execute_tx.execute.calldata, + // The signature isn't checked for L1 transactions so we don't care + signature: vec![], + factory_deps: execute_tx.execute.factory_deps.unwrap_or_default(), + paymaster_input: vec![], + reserved_dynamic: vec![], + raw_bytes: None, + } + } + } + } +} + +impl TransactionData { + pub(crate) fn abi_encode_with_custom_factory_deps( + self, + factory_deps_hashes: Vec, + ) -> Vec { + encode(&[Token::Tuple(vec![ + Token::Uint(U256::from_big_endian(&self.tx_type.to_be_bytes())), + Token::Address(self.from), + Token::Address(self.to), + Token::Uint(self.gas_limit), + Token::Uint(self.pubdata_price_limit), + Token::Uint(self.max_fee_per_gas), + Token::Uint(self.max_priority_fee_per_gas), + Token::Address(self.paymaster), + Token::Uint(self.nonce), + Token::Uint(self.value), + Token::FixedArray(self.reserved.iter().copied().map(Token::Uint).collect()), + Token::Bytes(self.data), + Token::Bytes(self.signature), + Token::Array(factory_deps_hashes.into_iter().map(Token::Uint).collect()), + Token::Bytes(self.paymaster_input), + Token::Bytes(self.reserved_dynamic), + ])]) + } + + pub(crate) fn abi_encode(self) -> Vec { + let factory_deps_hashes = self + .factory_deps + .iter() + .map(|dep| h256_to_u256(hash_bytecode(dep))) + .collect(); + self.abi_encode_with_custom_factory_deps(factory_deps_hashes) + } + + pub(crate) fn into_tokens(self) -> Vec { + let bytes = self.abi_encode(); + assert!(bytes.len() % 32 == 0); + + bytes_to_be_words(bytes) + } + + pub(crate) fn effective_gas_price_per_pubdata(&self, block_gas_price_per_pubdata: u32) -> u32 { + // It is enforced by the protocol that the L1 transactions always pay the exact amount of gas per pubdata + // as was supplied in the transaction. + if is_l1_tx_type(self.tx_type) { + self.pubdata_price_limit.as_u32() + } else { + block_gas_price_per_pubdata + } + } + + pub(crate) fn overhead_gas(&self, block_gas_price_per_pubdata: u32) -> u32 { + let total_gas_limit = self.gas_limit.as_u32(); + let gas_price_per_pubdata = + self.effective_gas_price_per_pubdata(block_gas_price_per_pubdata); + + let encoded_len = encoding_len( + self.data.len() as u64, + self.signature.len() as u64, + self.factory_deps.len() as u64, + self.paymaster_input.len() as u64, + self.reserved_dynamic.len() as u64, + ); + + let coefficients = OverheadCoefficients::from_tx_type(self.tx_type); + get_amortized_overhead( + total_gas_limit, + gas_price_per_pubdata, + encoded_len, + coefficients, + ) + } + + pub(crate) fn trusted_ergs_limit(&self, _block_gas_price_per_pubdata: u64) -> U256 { + // TODO (EVM-66): correctly calculate the trusted gas limit for a transaction + self.gas_limit + } + + pub(crate) fn tx_hash(&self, chain_id: L2ChainId) -> H256 { + if is_l1_tx_type(self.tx_type) { + return self.canonical_l1_tx_hash().unwrap(); + } + + let l2_tx: L2Tx = self.clone().try_into().unwrap(); + let transaction_request: TransactionRequest = l2_tx.into(); + + // It is assumed that the TransactionData always has all the necessary components to recover the hash. + transaction_request + .get_tx_hash(chain_id) + .expect("Could not recover L2 transaction hash") + } + + fn canonical_l1_tx_hash(&self) -> Result { + use zksync_types::web3::signing::keccak256; + + if !is_l1_tx_type(self.tx_type) { + return Err(TxHashCalculationError::CannotCalculateL1HashForL2Tx); + } + + let encoded_bytes = self.clone().abi_encode(); + + Ok(H256(keccak256(&encoded_bytes))) + } +} + +#[derive(Debug, Clone, Copy)] +pub(crate) enum TxHashCalculationError { + CannotCalculateL1HashForL2Tx, + CannotCalculateL2HashForL1Tx, +} + +impl TryInto for TransactionData { + type Error = TxHashCalculationError; + + fn try_into(self) -> Result { + if is_l1_tx_type(self.tx_type) { + return Err(TxHashCalculationError::CannotCalculateL2HashForL1Tx); + } + + let common_data = L2TxCommonData { + transaction_type: (self.tx_type as u32).try_into().unwrap(), + nonce: Nonce(self.nonce.as_u32()), + fee: Fee { + max_fee_per_gas: self.max_fee_per_gas, + max_priority_fee_per_gas: self.max_priority_fee_per_gas, + gas_limit: self.gas_limit, + gas_per_pubdata_limit: self.pubdata_price_limit, + }, + signature: self.signature, + input: None, + initiator_address: self.from, + paymaster_params: PaymasterParams { + paymaster: self.paymaster, + paymaster_input: self.paymaster_input, + }, + }; + let factory_deps = (!self.factory_deps.is_empty()).then_some(self.factory_deps); + let execute = Execute { + contract_address: self.to, + value: self.value, + calldata: self.data, + factory_deps, + }; + + Ok(L2Tx { + execute, + common_data, + received_timestamp_ms: 0, + raw_bytes: self.raw_bytes.map(Bytes::from), + }) + } +} + +#[cfg(test)] +mod tests { + use zksync_types::fee::encoding_len; + + use super::*; + + #[test] + fn test_consistency_with_encoding_length() { + let transaction = TransactionData { + tx_type: 113, + from: Address::random(), + to: Address::random(), + gas_limit: U256::from(1u32), + pubdata_price_limit: U256::from(1u32), + max_fee_per_gas: U256::from(1u32), + max_priority_fee_per_gas: U256::from(1u32), + paymaster: Address::random(), + nonce: U256::zero(), + value: U256::zero(), + // The reserved fields that are unique for different types of transactions. + // E.g. nonce is currently used in all transaction, but it should not be mandatory + // in the long run. + reserved: [U256::zero(); 4], + data: vec![0u8; 65], + signature: vec![0u8; 75], + // The factory deps provided with the transaction. + // Note that *only hashes* of these bytecodes are signed by the user + // and they are used in the ABI encoding of the struct. + // TODO: include this into the tx signature as part of SMA-1010 + factory_deps: vec![vec![0u8; 32], vec![1u8; 32]], + paymaster_input: vec![0u8; 85], + reserved_dynamic: vec![0u8; 32], + raw_bytes: None, + }; + + let assumed_encoded_len = encoding_len(65, 75, 2, 85, 32); + + let true_encoding_len = transaction.into_tokens().len(); + + assert_eq!(assumed_encoded_len, true_encoding_len); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs new file mode 100644 index 000000000000..da9c81321440 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs @@ -0,0 +1,183 @@ +use zk_evm_1_4_0::{ + aux_structures::{MemoryPage, Timestamp}, + block_properties::BlockProperties, + vm_state::{CallStackEntry, PrimitiveValue, VmState}, + witness_trace::DummyTracer, + zkevm_opcode_defs::{ + system_params::{BOOTLOADER_MAX_MEMORY, INITIAL_FRAME_FORMAL_EH_LOCATION}, + FatPointer, BOOTLOADER_BASE_PAGE, BOOTLOADER_CALLDATA_PAGE, BOOTLOADER_CODE_PAGE, + STARTING_BASE_PAGE, STARTING_TIMESTAMP, + }, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_system_constants::BOOTLOADER_ADDRESS; +use zksync_types::{ + block::MiniblockHasher, zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, + MiniblockNumber, +}; +use zksync_utils::h256_to_u256; + +use crate::{ + interface::{L1BatchEnv, L2Block, SystemEnv}, + vm_latest::{ + bootloader_state::BootloaderState, + constants::BOOTLOADER_HEAP_PAGE, + old_vm::{ + event_sink::InMemoryEventSink, + history_recorder::HistoryMode, + memory::SimpleMemory, + oracles::{ + decommitter::DecommitterOracle, precompile::PrecompilesProcessorWithHistory, + }, + }, + oracles::storage::StorageOracle, + types::l1_batch::bootloader_initial_memory, + utils::l2_blocks::{assert_next_block, load_last_l2_block}, + }, +}; + +pub type ZkSyncVmState = VmState< + StorageOracle, + SimpleMemory, + InMemoryEventSink, + PrecompilesProcessorWithHistory, + DecommitterOracle, + DummyTracer, +>; + +fn formal_calldata_abi() -> PrimitiveValue { + let fat_pointer = FatPointer { + offset: 0, + memory_page: BOOTLOADER_CALLDATA_PAGE, + start: 0, + length: 0, + }; + + PrimitiveValue { + value: fat_pointer.to_u256(), + is_pointer: true, + } +} + +/// Initialize the vm state and all necessary oracles +pub(crate) fn new_vm_state( + storage: StoragePtr, + system_env: &SystemEnv, + l1_batch_env: &L1BatchEnv, +) -> (ZkSyncVmState, BootloaderState) { + let last_l2_block = if let Some(last_l2_block) = load_last_l2_block(storage.clone()) { + last_l2_block + } else { + // This is the scenario of either the first L2 block ever or + // the first block after the upgrade for support of L2 blocks. + L2Block { + number: l1_batch_env.first_l2_block.number.saturating_sub(1), + timestamp: 0, + hash: MiniblockHasher::legacy_hash( + MiniblockNumber(l1_batch_env.first_l2_block.number) - 1, + ), + } + }; + + assert_next_block(&last_l2_block, &l1_batch_env.first_l2_block); + let first_l2_block = l1_batch_env.first_l2_block; + let storage_oracle: StorageOracle = StorageOracle::new(storage.clone()); + let mut memory = SimpleMemory::default(); + let event_sink = InMemoryEventSink::default(); + let precompiles_processor = PrecompilesProcessorWithHistory::::default(); + let mut decommittment_processor: DecommitterOracle = + DecommitterOracle::new(storage); + + decommittment_processor.populate( + vec![( + h256_to_u256(system_env.base_system_smart_contracts.default_aa.hash), + system_env + .base_system_smart_contracts + .default_aa + .code + .clone(), + )], + Timestamp(0), + ); + + memory.populate( + vec![( + BOOTLOADER_CODE_PAGE, + system_env + .base_system_smart_contracts + .bootloader + .code + .clone(), + )], + Timestamp(0), + ); + + let bootloader_initial_memory = bootloader_initial_memory(l1_batch_env); + memory.populate_page( + BOOTLOADER_HEAP_PAGE as usize, + bootloader_initial_memory.clone(), + Timestamp(0), + ); + + let mut vm = VmState::empty_state( + storage_oracle, + memory, + event_sink, + precompiles_processor, + decommittment_processor, + DummyTracer, + BlockProperties { + default_aa_code_hash: h256_to_u256( + system_env.base_system_smart_contracts.default_aa.hash, + ), + zkporter_is_available: system_env.zk_porter_available, + }, + ); + + vm.local_state.callstack.current.ergs_remaining = system_env.gas_limit; + + let initial_context = CallStackEntry { + this_address: BOOTLOADER_ADDRESS, + msg_sender: Address::zero(), + code_address: BOOTLOADER_ADDRESS, + base_memory_page: MemoryPage(BOOTLOADER_BASE_PAGE), + code_page: MemoryPage(BOOTLOADER_CODE_PAGE), + sp: 0, + pc: 0, + // Note, that since the results are written at the end of the memory + // it is needed to have the entire heap available from the beginning + heap_bound: BOOTLOADER_MAX_MEMORY, + aux_heap_bound: BOOTLOADER_MAX_MEMORY, + exception_handler_location: INITIAL_FRAME_FORMAL_EH_LOCATION, + ergs_remaining: system_env.gas_limit, + this_shard_id: 0, + caller_shard_id: 0, + code_shard_id: 0, + is_static: false, + is_local_frame: false, + context_u128_value: 0, + }; + + // We consider the contract that is being run as a bootloader + vm.push_bootloader_context(INITIAL_MONOTONIC_CYCLE_COUNTER - 1, initial_context); + vm.local_state.timestamp = STARTING_TIMESTAMP; + vm.local_state.memory_page_counter = STARTING_BASE_PAGE; + vm.local_state.monotonic_cycle_counter = INITIAL_MONOTONIC_CYCLE_COUNTER; + vm.local_state.current_ergs_per_pubdata_byte = 0; + vm.local_state.registers[0] = formal_calldata_abi(); + + // Deleting all the historical records brought by the initial + // initialization of the VM to make them permanent. + vm.decommittment_processor.delete_history(); + vm.event_sink.delete_history(); + vm.storage.delete_history(); + vm.memory.delete_history(); + vm.precompiles_processor.delete_history(); + let bootloader_state = BootloaderState::new( + system_env.execution_mode, + bootloader_initial_memory, + first_l2_block, + ); + + (vm, bootloader_state) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs new file mode 100644 index 000000000000..6f16e95f8d77 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs @@ -0,0 +1,38 @@ +use zksync_types::U256; +use zksync_utils::{address_to_u256, h256_to_u256}; + +use crate::interface::L1BatchEnv; + +const OPERATOR_ADDRESS_SLOT: usize = 0; +const PREV_BLOCK_HASH_SLOT: usize = 1; +const NEW_BLOCK_TIMESTAMP_SLOT: usize = 2; +const NEW_BLOCK_NUMBER_SLOT: usize = 3; +const L1_GAS_PRICE_SLOT: usize = 4; +const FAIR_L2_GAS_PRICE_SLOT: usize = 5; +const EXPECTED_BASE_FEE_SLOT: usize = 6; +const SHOULD_SET_NEW_BLOCK_SLOT: usize = 7; + +/// Returns the initial memory for the bootloader based on the current batch environment. +pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U256)> { + let (prev_block_hash, should_set_new_block) = l1_batch + .previous_batch_hash + .map(|prev_block_hash| (h256_to_u256(prev_block_hash), U256::one())) + .unwrap_or_default(); + + vec![ + ( + OPERATOR_ADDRESS_SLOT, + address_to_u256(&l1_batch.fee_account), + ), + (PREV_BLOCK_HASH_SLOT, prev_block_hash), + (NEW_BLOCK_TIMESTAMP_SLOT, U256::from(l1_batch.timestamp)), + (NEW_BLOCK_NUMBER_SLOT, U256::from(l1_batch.number.0)), + (L1_GAS_PRICE_SLOT, U256::from(l1_batch.l1_gas_price)), + ( + FAIR_L2_GAS_PRICE_SLOT, + U256::from(l1_batch.fair_l2_gas_price), + ), + (EXPECTED_BASE_FEE_SLOT, U256::from(l1_batch.base_fee())), + (SHOULD_SET_NEW_BLOCK_SLOT, should_set_new_block), + ] +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/mod.rs new file mode 100644 index 000000000000..a12005734abb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/mod.rs @@ -0,0 +1,2 @@ +pub(crate) mod internals; +mod l1_batch; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs new file mode 100644 index 000000000000..23b744a348f7 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs @@ -0,0 +1,29 @@ +//! Utility functions for vm +use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; +use zksync_utils::ceil_div; + +use crate::vm_latest::old_vm::utils::eth_price_per_pubdata_byte; + +/// Calculates the amount of gas required to publish one byte of pubdata +pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); + + ceil_div(eth_price_per_pubdata_byte, base_fee) +} + +/// Calculates the base fee and gas per pubdata for the given L1 gas price. +pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); + + // The baseFee is set in such a way that it is always possible for a transaction to + // publish enough public data while compensating us for it. + let base_fee = std::cmp::max( + fair_gas_price, + ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), + ); + + ( + base_fee, + base_fee_to_gas_per_pubdata(l1_gas_price, base_fee), + ) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs new file mode 100644 index 000000000000..5dd26c4c0277 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs @@ -0,0 +1,95 @@ +use zksync_state::{ReadStorage, StoragePtr}; +use zksync_system_constants::{ + SYSTEM_CONTEXT_ADDRESS, SYSTEM_CONTEXT_CURRENT_L2_BLOCK_HASHES_POSITION, + SYSTEM_CONTEXT_CURRENT_L2_BLOCK_INFO_POSITION, SYSTEM_CONTEXT_CURRENT_TX_ROLLING_HASH_POSITION, + SYSTEM_CONTEXT_STORED_L2_BLOCK_HASHES, +}; +use zksync_types::{ + block::unpack_block_info, web3::signing::keccak256, AccountTreeId, MiniblockNumber, StorageKey, + H256, U256, +}; +use zksync_utils::{h256_to_u256, u256_to_h256}; + +use crate::interface::{L2Block, L2BlockEnv}; + +pub(crate) fn get_l2_block_hash_key(block_number: u32) -> StorageKey { + let position = h256_to_u256(SYSTEM_CONTEXT_CURRENT_L2_BLOCK_HASHES_POSITION) + + U256::from(block_number % SYSTEM_CONTEXT_STORED_L2_BLOCK_HASHES); + StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + u256_to_h256(position), + ) +} + +pub(crate) fn assert_next_block(prev_block: &L2Block, next_block: &L2BlockEnv) { + if prev_block.number == 0 { + // Special case for the first block it can have the same timestamp as the previous block. + assert!(prev_block.timestamp <= next_block.timestamp); + } else { + assert_eq!(prev_block.number + 1, next_block.number); + assert!(prev_block.timestamp < next_block.timestamp); + } + assert_eq!(prev_block.hash, next_block.prev_block_hash); +} + +/// Returns the hash of the l2_block. +/// `txs_rolling_hash` of the l2_block is calculated the following way: +/// If the l2_block has 0 transactions, then `txs_rolling_hash` is equal to `H256::zero()`. +/// If the l2_block has i transactions, then `txs_rolling_hash` is equal to `H(H_{i-1}, H(tx_i))`, where +/// `H_{i-1}` is the `txs_rolling_hash` of the first i-1 transactions. +pub(crate) fn l2_block_hash( + l2_block_number: MiniblockNumber, + l2_block_timestamp: u64, + prev_l2_block_hash: H256, + txs_rolling_hash: H256, +) -> H256 { + let mut digest: [u8; 128] = [0u8; 128]; + U256::from(l2_block_number.0).to_big_endian(&mut digest[0..32]); + U256::from(l2_block_timestamp).to_big_endian(&mut digest[32..64]); + digest[64..96].copy_from_slice(prev_l2_block_hash.as_bytes()); + digest[96..128].copy_from_slice(txs_rolling_hash.as_bytes()); + + H256(keccak256(&digest)) +} + +/// Get last saved block from storage +pub fn load_last_l2_block(storage: StoragePtr) -> Option { + // Get block number and timestamp + let current_l2_block_info_key = StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + SYSTEM_CONTEXT_CURRENT_L2_BLOCK_INFO_POSITION, + ); + let mut storage_ptr = storage.borrow_mut(); + let current_l2_block_info = storage_ptr.read_value(¤t_l2_block_info_key); + let (block_number, block_timestamp) = unpack_block_info(h256_to_u256(current_l2_block_info)); + let block_number = block_number as u32; + if block_number == 0 { + // The block does not exist yet + return None; + } + + // Get prev block hash + let position = get_l2_block_hash_key(block_number - 1); + let prev_block_hash = storage_ptr.read_value(&position); + + // Get current tx rolling hash + let position = StorageKey::new( + AccountTreeId::new(SYSTEM_CONTEXT_ADDRESS), + SYSTEM_CONTEXT_CURRENT_TX_ROLLING_HASH_POSITION, + ); + let current_tx_rolling_hash = storage_ptr.read_value(&position); + + // Calculate current hash + let current_block_hash = l2_block_hash( + MiniblockNumber(block_number), + block_timestamp, + prev_block_hash, + current_tx_rolling_hash, + ); + + Some(L2Block { + number: block_number, + timestamp: block_timestamp, + hash: current_block_hash, + }) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs new file mode 100644 index 000000000000..b7fa07956a96 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs @@ -0,0 +1,25 @@ +use zksync_state::WriteStorage; +use zksync_types::{l2_to_l1_log::L2ToL1Log, Timestamp, VmEvent}; + +use crate::{ + interface::L1BatchEnv, + vm_latest::{ + old_vm::{events::merge_events, history_recorder::HistoryMode}, + types::internals::ZkSyncVmState, + }, +}; + +pub(crate) fn collect_events_and_l1_system_logs_after_timestamp( + vm_state: &ZkSyncVmState, + batch_env: &L1BatchEnv, + from_timestamp: Timestamp, +) -> (Vec, Vec) { + let (raw_events, l1_messages) = vm_state + .event_sink + .get_events_and_l2_l1_logs_after_timestamp(from_timestamp); + let events = merge_events(raw_events) + .into_iter() + .map(|e| e.into_vm_event(batch_env.number)) + .collect(); + (events, l1_messages.into_iter().map(Into::into).collect()) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/mod.rs new file mode 100644 index 000000000000..0fb803de5d4e --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/mod.rs @@ -0,0 +1,6 @@ +/// Utility functions for the VM. +pub mod fee; +pub mod l2_blocks; +pub(crate) mod logs; +pub mod overhead; +pub mod transaction_encoding; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs new file mode 100644 index 000000000000..38aaede8d4bb --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs @@ -0,0 +1,349 @@ +use zk_evm_1_4_0::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; +use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; +use zksync_types::{l1::is_l1_tx_type, U256}; +use zksync_utils::ceil_div_u256; + +use crate::vm_latest::constants::{ + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, +}; + +/// Derives the overhead for processing transactions in a block. +pub fn derive_overhead( + gas_limit: u32, + gas_price_per_pubdata: u32, + encoded_len: usize, + coefficients: OverheadCoefficients, +) -> u32 { + // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value + let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); + + // Using large U256 type to avoid overflow + let max_block_overhead = U256::from(block_overhead_gas(gas_price_per_pubdata)); + let gas_limit = U256::from(gas_limit); + let encoded_len = U256::from(encoded_len); + + // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // circuits. + let overhead_for_single_instance_circuits = + ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); + + // The overhead for occupying the bootloader memory + let overhead_for_length = ceil_div_u256( + encoded_len * max_block_overhead, + BOOTLOADER_TX_ENCODING_SPACE.into(), + ); + + // The overhead for occupying a single tx slot + let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); + + // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) + // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + + // The maximal potential overhead from pubdata + // TODO (EVM-67): possibly use overhead for pubdata + // let pubdata_overhead = ceil_div_u256( + // max_pubdata_in_tx * max_block_overhead, + // MAX_PUBDATA_PER_BLOCK.into(), + // ); + + vec![ + (coefficients.ergs_limit_overhead_coeficient + * overhead_for_single_instance_circuits.as_u32() as f64) + .floor() as u32, + (coefficients.bootloader_memory_overhead_coeficient * overhead_for_length.as_u32() as f64) + .floor() as u32, + (coefficients.slot_overhead_coeficient * tx_slot_overhead.as_u32() as f64) as u32, + ] + .into_iter() + .max() + .unwrap() +} + +/// Contains the coefficients with which the overhead for transactions will be calculated. +/// All of the coefficients should be <= 1. There are here to provide a certain "discount" for normal transactions +/// at the risk of malicious transactions that may close the block prematurely. +/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coeficient` MUST +/// result in an integer number +#[derive(Debug, Clone, Copy)] +pub struct OverheadCoefficients { + slot_overhead_coeficient: f64, + bootloader_memory_overhead_coeficient: f64, + ergs_limit_overhead_coeficient: f64, +} + +impl OverheadCoefficients { + // This method ensures that the parameters keep the required invariants + fn new_checked( + slot_overhead_coeficient: f64, + bootloader_memory_overhead_coeficient: f64, + ergs_limit_overhead_coeficient: f64, + ) -> Self { + assert!( + (MAX_TX_ERGS_LIMIT as f64 / ergs_limit_overhead_coeficient).round() + == MAX_TX_ERGS_LIMIT as f64 / ergs_limit_overhead_coeficient, + "MAX_TX_ERGS_LIMIT / ergs_limit_overhead_coeficient must be an integer" + ); + + Self { + slot_overhead_coeficient, + bootloader_memory_overhead_coeficient, + ergs_limit_overhead_coeficient, + } + } + + // L1->L2 do not receive any discounts + fn new_l1() -> Self { + OverheadCoefficients::new_checked(1.0, 1.0, 1.0) + } + + fn new_l2() -> Self { + OverheadCoefficients::new_checked( + 1.0, 1.0, + // For L2 transactions we allow a certain default discount with regard to the number of ergs. + // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // to be paid by the users is 0.1. + 0.1, + ) + } + + /// Return the coefficients for the given transaction type + pub fn from_tx_type(tx_type: u8) -> Self { + if is_l1_tx_type(tx_type) { + Self::new_l1() + } else { + Self::new_l2() + } + } +} + +/// This method returns the overhead for processing the block +pub(crate) fn get_amortized_overhead( + total_gas_limit: u32, + gas_per_pubdata_byte_limit: u32, + encoded_len: usize, + coefficients: OverheadCoefficients, +) -> u32 { + // Using large U256 type to prevent overflows. + let overhead_for_block_gas = U256::from(block_overhead_gas(gas_per_pubdata_byte_limit)); + let total_gas_limit = U256::from(total_gas_limit); + let encoded_len = U256::from(encoded_len); + + // Derivation of overhead consists of 4 parts: + // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK + // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE + // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK + // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // + // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: + // + // max_overhead = max(O1, O2, O3, O4) + // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of + // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // + // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): + // + // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: + // 1. overhead_gas is maximal possible (the operator is paid fairly) + // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // The third part boils to the following 4 inequalities (at least one of these must hold): + // ceil(O1 * overhead_for_block_gas) >= overhead_gas + // ceil(O2 * overhead_for_block_gas) >= overhead_gas + // ceil(O3 * overhead_for_block_gas) >= overhead_gas + // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // + // Now, we need to solve each of these separately: + + // 1. The overhead for occupying a single tx slot is a constant: + let tx_slot_overhead = { + let tx_slot_overhead = + ceil_div_u256(overhead_for_block_gas, MAX_TXS_IN_BLOCK.into()).as_u32(); + (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 + }; + + // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + let overhead_for_length = { + let overhead_for_length = ceil_div_u256( + encoded_len * overhead_for_block_gas, + BOOTLOADER_TX_ENCODING_SPACE.into(), + ) + .as_u32(); + + (coefficients.bootloader_memory_overhead_coeficient * overhead_for_length as f64).floor() + as u32 + }; + + // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, + // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. + // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas + // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // overhead to the operator, provides substantially easier formula to work with. + // + // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE + // ceil(OB * (TL - OE) / (EP * MP)) >= OE + // + // OB * (TL - OE) / (MP * EP) > OE - 1 + // OB * (TL - OE) > (OE - 1) * EP * MP + // OB * TL + EP * MP > OE * EP * MP + OE * OB + // (OB * TL + EP * MP) / (EP * MP + OB) > OE + // OE = floor((OB * TL + EP * MP) / (EP * MP + OB)) with possible -1 if the division is without remainder + // let overhead_for_pubdata = { + // let numerator: U256 = overhead_for_block_gas * total_gas_limit + // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); + // let denominator = + // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; + + // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 + // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. + // if numerator.is_zero() { + // 0.into() + // } else { + // (numerator - 1) / denominator + // } + // }; + + // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount + // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: + // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) + // ceil(OB * (TL - OE) / MAX_TX_ERGS_LIMIT) >= (OE/K) + // OB * (TL - OE) / MAX_TX_ERGS_LIMIT > (OE/K) - 1 + // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT + // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) + // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + let overhead_for_gas = { + let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); + let denominator: U256 = U256::from( + (MAX_TX_ERGS_LIMIT as f64 / coefficients.ergs_limit_overhead_coeficient) as u64, + ) + overhead_for_block_gas; + + let overhead_for_gas = (numerator - 1) / denominator; + + overhead_for_gas.as_u32() + }; + + let overhead = vec![tx_slot_overhead, overhead_for_length, overhead_for_gas] + .into_iter() + .max() + // For the sake of consistency making sure that total_gas_limit >= max_overhead + .map(|max_overhead| std::cmp::min(max_overhead, total_gas_limit.as_u32())) + .unwrap(); + + let limit_after_deducting_overhead = total_gas_limit - overhead; + + // During double checking of the overhead, the bootloader will assume that the + // body of the transaction does not have any more than MAX_L2_TX_GAS_LIMIT ergs available to it. + if limit_after_deducting_overhead.as_u64() > MAX_L2_TX_GAS_LIMIT { + // We derive the same overhead that would exist for the MAX_L2_TX_GAS_LIMIT ergs + derive_overhead( + MAX_L2_TX_GAS_LIMIT as u32, + gas_per_pubdata_byte_limit, + encoded_len.as_usize(), + coefficients, + ) + } else { + overhead + } +} + +pub(crate) fn block_overhead_gas(gas_per_pubdata_byte: u32) -> u32 { + BLOCK_OVERHEAD_GAS + BLOCK_OVERHEAD_PUBDATA * gas_per_pubdata_byte +} + +#[cfg(test)] +mod tests { + + use super::*; + + // This method returns the maximum block overhead that can be charged from the user based on the binary search approach + pub(crate) fn get_maximal_allowed_overhead_bin_search( + total_gas_limit: u32, + gas_per_pubdata_byte_limit: u32, + encoded_len: usize, + coefficients: OverheadCoefficients, + ) -> u32 { + let mut left_bound = if MAX_TX_ERGS_LIMIT < total_gas_limit { + total_gas_limit - MAX_TX_ERGS_LIMIT + } else { + 0u32 + }; + // Safe cast: the gas_limit for a transaction can not be larger than 2^32 + let mut right_bound = total_gas_limit; + + // The closure returns whether a certain overhead would be accepted by the bootloader. + // It is accepted if the derived overhead (i.e. the actual overhead that the user has to pay) + // is >= than the overhead proposed by the operator. + let is_overhead_accepted = |suggested_overhead: u32| { + let derived_overhead = derive_overhead( + total_gas_limit - suggested_overhead, + gas_per_pubdata_byte_limit, + encoded_len, + coefficients, + ); + + derived_overhead >= suggested_overhead + }; + + // In order to find the maximal allowed overhead we are doing binary search + while left_bound + 1 < right_bound { + let mid = (left_bound + right_bound) / 2; + + if is_overhead_accepted(mid) { + left_bound = mid; + } else { + right_bound = mid; + } + } + + if is_overhead_accepted(right_bound) { + right_bound + } else { + left_bound + } + } + + #[test] + fn test_correctness_for_efficient_overhead() { + let test_params = |total_gas_limit: u32, + gas_per_pubdata: u32, + encoded_len: usize, + coefficients: OverheadCoefficients| { + let result_by_efficient_search = + get_amortized_overhead(total_gas_limit, gas_per_pubdata, encoded_len, coefficients); + + let result_by_binary_search = get_maximal_allowed_overhead_bin_search( + total_gas_limit, + gas_per_pubdata, + encoded_len, + coefficients, + ); + + assert_eq!(result_by_efficient_search, result_by_binary_search); + }; + + // Some arbitrary test + test_params(60_000_000, 800, 2900, OverheadCoefficients::new_l2()); + + // Very small parameters + test_params(0, 1, 12, OverheadCoefficients::new_l2()); + + // Relatively big parameters + let max_tx_overhead = derive_overhead( + MAX_TX_ERGS_LIMIT, + 5000, + 10000, + OverheadCoefficients::new_l2(), + ); + test_params( + MAX_TX_ERGS_LIMIT + max_tx_overhead, + 5000, + 10000, + OverheadCoefficients::new_l2(), + ); + + test_params(115432560, 800, 2900, OverheadCoefficients::new_l1()); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs new file mode 100644 index 000000000000..86c49a3eb15d --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs @@ -0,0 +1,16 @@ +use zksync_types::Transaction; + +use crate::vm_latest::types::internals::TransactionData; + +/// Extension for transactions, specific for VM. Required for bypassing the orphan rule +pub trait TransactionVmExt { + /// Get the size of the transaction in tokens. + fn bootloader_encoding_size(&self) -> usize; +} + +impl TransactionVmExt for Transaction { + fn bootloader_encoding_size(&self) -> usize { + let transaction_data: TransactionData = self.clone().into(); + transaction_data.into_tokens().len() + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs new file mode 100644 index 000000000000..73d551942bff --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs @@ -0,0 +1,184 @@ +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_types::{ + event::extract_l2tol1logs_from_l1_messenger, + l2_to_l1_log::{SystemL2ToL1Log, UserL2ToL1Log}, + Transaction, +}; +use zksync_utils::bytecode::CompressedBytecodeInfo; + +use crate::{ + interface::{ + BootloaderMemory, BytecodeCompressionError, CurrentExecutionState, FinishedL1Batch, + L1BatchEnv, L2BlockEnv, SystemEnv, VmExecutionMode, VmExecutionResultAndLogs, VmInterface, + VmInterfaceHistoryEnabled, VmMemoryMetrics, + }, + vm_latest::{ + bootloader_state::BootloaderState, + old_vm::{events::merge_events, history_recorder::HistoryEnabled}, + tracers::dispatcher::TracerDispatcher, + types::internals::{new_vm_state, VmSnapshot, ZkSyncVmState}, + }, + HistoryMode, +}; + +/// Main entry point for Virtual Machine integration. +/// The instance should process only one l1 batch +#[derive(Debug)] +pub struct Vm { + pub(crate) bootloader_state: BootloaderState, + // Current state and oracles of virtual machine + pub(crate) state: ZkSyncVmState, + pub(crate) storage: StoragePtr, + pub(crate) system_env: SystemEnv, + pub(crate) batch_env: L1BatchEnv, + // Snapshots for the current run + pub(crate) snapshots: Vec, + _phantom: std::marker::PhantomData, +} + +impl VmInterface for Vm { + type TracerDispatcher = TracerDispatcher; + + fn new(batch_env: L1BatchEnv, system_env: SystemEnv, storage: StoragePtr) -> Self { + let (state, bootloader_state) = new_vm_state(storage.clone(), &system_env, &batch_env); + Self { + bootloader_state, + state, + storage, + system_env, + batch_env, + snapshots: vec![], + _phantom: Default::default(), + } + } + + /// Push tx into memory for the future execution + fn push_transaction(&mut self, tx: Transaction) { + self.push_transaction_with_compression(tx, true); + } + + /// Execute VM with custom tracers. + fn inspect( + &mut self, + tracer: Self::TracerDispatcher, + execution_mode: VmExecutionMode, + ) -> VmExecutionResultAndLogs { + self.inspect_inner(tracer, execution_mode) + } + + /// Get current state of bootloader memory. + fn get_bootloader_memory(&self) -> BootloaderMemory { + self.bootloader_state.bootloader_memory() + } + + /// Get compressed bytecodes of the last executed transaction + fn get_last_tx_compressed_bytecodes(&self) -> Vec { + self.bootloader_state.get_last_tx_compressed_bytecodes() + } + + fn start_new_l2_block(&mut self, l2_block_env: L2BlockEnv) { + self.bootloader_state.start_new_l2_block(l2_block_env); + } + + /// Get current state of virtual machine. + /// This method should be used only after the batch execution. + /// Otherwise it can panic. + fn get_current_execution_state(&self) -> CurrentExecutionState { + let (deduplicated_events_logs, raw_events, l1_messages) = self.state.event_sink.flatten(); + let events: Vec<_> = merge_events(raw_events) + .into_iter() + .map(|e| e.into_vm_event(self.batch_env.number)) + .collect(); + + let user_l2_to_l1_logs = extract_l2tol1logs_from_l1_messenger(&events); + let system_logs = l1_messages + .into_iter() + .map(|log| SystemL2ToL1Log(log.into())) + .collect(); + let total_log_queries = self.state.event_sink.get_log_queries() + + self + .state + .precompiles_processor + .get_timestamp_history() + .len() + + self.state.storage.get_final_log_queries().len(); + + CurrentExecutionState { + events, + storage_log_queries: self.state.storage.get_final_log_queries(), + used_contract_hashes: self.get_used_contracts(), + user_l2_to_l1_logs: user_l2_to_l1_logs + .into_iter() + .map(|log| UserL2ToL1Log(log.into())) + .collect(), + system_logs, + total_log_queries, + cycles_used: self.state.local_state.monotonic_cycle_counter, + deduplicated_events_logs, + storage_refunds: self.state.storage.returned_refunds.inner().clone(), + } + } + + /// Execute transaction with optional bytecode compression. + + /// Inspect transaction with optional bytecode compression. + fn inspect_transaction_with_bytecode_compression( + &mut self, + tracer: Self::TracerDispatcher, + tx: Transaction, + with_compression: bool, + ) -> Result { + self.push_transaction_with_compression(tx, with_compression); + let result = self.inspect_inner(tracer, VmExecutionMode::OneTx); + if self.has_unpublished_bytecodes() { + Err(BytecodeCompressionError::BytecodeCompressionFailed) + } else { + Ok(result) + } + } + + fn record_vm_memory_metrics(&self) -> VmMemoryMetrics { + self.record_vm_memory_metrics_inner() + } + + fn finish_batch(&mut self) -> FinishedL1Batch { + let result = self.execute(VmExecutionMode::Batch); + let execution_state = self.get_current_execution_state(); + let bootloader_memory = self.get_bootloader_memory(); + FinishedL1Batch { + block_tip_execution_result: result, + final_execution_state: execution_state, + final_bootloader_memory: Some(bootloader_memory), + pubdata_input: Some( + self.bootloader_state + .get_pubdata_information() + .clone() + .build_pubdata(false), + ), + } + } +} + +/// Methods of vm, which required some history manipulations +impl VmInterfaceHistoryEnabled for Vm { + /// Create snapshot of current vm state and push it into the memory + fn make_snapshot(&mut self) { + self.make_snapshot_inner() + } + + /// Rollback vm state to the latest snapshot and destroy the snapshot + fn rollback_to_the_latest_snapshot(&mut self) { + let snapshot = self + .snapshots + .pop() + .expect("Snapshot should be created before rolling it back"); + self.rollback_to_snapshot(snapshot); + } + + /// Pop the latest snapshot from the memory and destroy it + fn pop_snapshot_no_rollback(&mut self) { + self.snapshots + .pop() + .expect("Snapshot should be created before rolling it back"); + } +} From 286c7d15a623604e01effa7119de3362f0fb4eb9 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Wed, 3 Jan 2024 20:56:48 +0200 Subject: [PATCH 07/49] feat(snapshot creator): Make snapshot creator fault-tolerant (#691) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Makes the snapshot creator tolerant to failures. The snapshot creator now checks whether there's a partially created snapshot on start. If there is, the creator continues producing this snapshot instead of starting from scratch. ## Why ❔ Since the snapshot creator can run for an extended amount of time, not having fault tolerance is unmaintainable in the long run. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/snapshots_creator/src/chunking.rs | 6 +- core/bin/snapshots_creator/src/creator.rs | 336 ++++++++++++++++++ core/bin/snapshots_creator/src/main.rs | 238 ++----------- core/bin/snapshots_creator/src/metrics.rs | 2 +- core/bin/snapshots_creator/src/tests.rs | 240 ++++++++++++- core/lib/dal/sqlx-data.json | 120 ++++--- core/lib/dal/src/snapshots_dal.rs | 193 +++++++--- core/lib/types/src/snapshots.rs | 28 +- .../api_server/web3/namespaces/snapshots.rs | 86 +++-- .../src/api_server/web3/tests/mod.rs | 38 +- .../src/api_server/web3/tests/snapshots.rs | 125 +++++++ .../src/api_server/web3/tests/ws.rs | 51 +-- 12 files changed, 1057 insertions(+), 406 deletions(-) create mode 100644 core/bin/snapshots_creator/src/creator.rs create mode 100644 core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs diff --git a/core/bin/snapshots_creator/src/chunking.rs b/core/bin/snapshots_creator/src/chunking.rs index 11768febd44f..047a6a23d24e 100644 --- a/core/bin/snapshots_creator/src/chunking.rs +++ b/core/bin/snapshots_creator/src/chunking.rs @@ -5,10 +5,10 @@ use zksync_utils::u256_to_h256; pub(crate) fn get_chunk_hashed_keys_range( chunk_id: u64, - chunks_count: u64, + chunk_count: u64, ) -> ops::RangeInclusive { - assert!(chunks_count > 0); - let mut stride = U256::MAX / chunks_count; + assert!(chunk_count > 0); + let mut stride = U256::MAX / chunk_count; let stride_minus_one = if stride < U256::MAX { stride += U256::one(); stride - 1 diff --git a/core/bin/snapshots_creator/src/creator.rs b/core/bin/snapshots_creator/src/creator.rs new file mode 100644 index 000000000000..da2ac930dbd4 --- /dev/null +++ b/core/bin/snapshots_creator/src/creator.rs @@ -0,0 +1,336 @@ +//! [`SnapshotCreator`] and tightly related types. + +use anyhow::Context as _; +use tokio::sync::Semaphore; +use zksync_config::SnapshotsCreatorConfig; +use zksync_dal::{ConnectionPool, StorageProcessor}; +use zksync_object_store::ObjectStore; +use zksync_types::{ + snapshots::{ + SnapshotFactoryDependencies, SnapshotMetadata, SnapshotStorageLogsChunk, + SnapshotStorageLogsStorageKey, + }, + L1BatchNumber, MiniblockNumber, +}; +use zksync_utils::ceil_div; + +#[cfg(test)] +use crate::tests::HandleEvent; +use crate::{ + chunking::get_chunk_hashed_keys_range, + metrics::{FactoryDepsStage, StorageChunkStage, METRICS}, +}; + +/// Encapsulates progress of creating a particular storage snapshot. +#[derive(Debug)] +struct SnapshotProgress { + l1_batch_number: L1BatchNumber, + /// `true` if the snapshot is new (i.e., its progress is not recovered from Postgres). + is_new_snapshot: bool, + chunk_count: u64, + remaining_chunk_ids: Vec, +} + +impl SnapshotProgress { + fn new(l1_batch_number: L1BatchNumber, chunk_count: u64) -> Self { + Self { + l1_batch_number, + is_new_snapshot: true, + chunk_count, + remaining_chunk_ids: (0..chunk_count).collect(), + } + } + + fn from_existing_snapshot(snapshot: &SnapshotMetadata) -> Self { + let remaining_chunk_ids = snapshot + .storage_logs_filepaths + .iter() + .enumerate() + .filter_map(|(chunk_id, path)| path.is_none().then_some(chunk_id as u64)) + .collect(); + + Self { + l1_batch_number: snapshot.l1_batch_number, + is_new_snapshot: false, + chunk_count: snapshot.storage_logs_filepaths.len() as u64, + remaining_chunk_ids, + } + } +} + +/// Creator of a single storage snapshot. +#[derive(Debug)] +pub(crate) struct SnapshotCreator { + pub blob_store: Box, + pub master_pool: ConnectionPool, + pub replica_pool: ConnectionPool, + #[cfg(test)] + pub event_listener: Box, +} + +impl SnapshotCreator { + async fn connect_to_replica(&self) -> anyhow::Result> { + self.replica_pool + .access_storage_tagged("snapshots_creator") + .await + } + + async fn process_storage_logs_single_chunk( + &self, + semaphore: &Semaphore, + miniblock_number: MiniblockNumber, + l1_batch_number: L1BatchNumber, + chunk_id: u64, + chunk_count: u64, + ) -> anyhow::Result<()> { + let _permit = semaphore.acquire().await?; + #[cfg(test)] + if self.event_listener.on_chunk_started().should_exit() { + return Ok(()); + } + + let hashed_keys_range = get_chunk_hashed_keys_range(chunk_id, chunk_count); + let mut conn = self.connect_to_replica().await?; + + let latency = + METRICS.storage_logs_processing_duration[&StorageChunkStage::LoadFromPostgres].start(); + let logs = conn + .snapshots_creator_dal() + .get_storage_logs_chunk(miniblock_number, hashed_keys_range) + .await + .context("Error fetching storage logs count")?; + drop(conn); + let latency = latency.observe(); + tracing::info!( + "Loaded chunk {chunk_id} ({} logs) from Postgres in {latency:?}", + logs.len() + ); + + let latency = + METRICS.storage_logs_processing_duration[&StorageChunkStage::SaveToGcs].start(); + let storage_logs_chunk = SnapshotStorageLogsChunk { storage_logs: logs }; + let key = SnapshotStorageLogsStorageKey { + l1_batch_number, + chunk_id, + }; + let filename = self + .blob_store + .put(key, &storage_logs_chunk) + .await + .context("Error storing storage logs chunk in blob store")?; + let output_filepath_prefix = self + .blob_store + .get_storage_prefix::(); + let output_filepath = format!("{output_filepath_prefix}/{filename}"); + let latency = latency.observe(); + + let mut master_conn = self + .master_pool + .access_storage_tagged("snapshots_creator") + .await?; + master_conn + .snapshots_dal() + .add_storage_logs_filepath_for_snapshot(l1_batch_number, chunk_id, &output_filepath) + .await?; + #[cfg(test)] + self.event_listener.on_chunk_saved(); + + let tasks_left = METRICS.storage_logs_chunks_left_to_process.dec_by(1) - 1; + tracing::info!( + "Saved chunk {chunk_id} (overall progress {}/{chunk_count}) in {latency:?} to location: {output_filepath}", + chunk_count - tasks_left as u64 + ); + Ok(()) + } + + async fn process_factory_deps( + &self, + miniblock_number: MiniblockNumber, + l1_batch_number: L1BatchNumber, + ) -> anyhow::Result { + let mut conn = self.connect_to_replica().await?; + + tracing::info!("Loading factory deps from Postgres..."); + let latency = + METRICS.factory_deps_processing_duration[&FactoryDepsStage::LoadFromPostgres].start(); + let factory_deps = conn + .snapshots_creator_dal() + .get_all_factory_deps(miniblock_number) + .await?; + drop(conn); + let latency = latency.observe(); + tracing::info!("Loaded {} factory deps in {latency:?}", factory_deps.len()); + + tracing::info!("Saving factory deps to GCS..."); + let latency = + METRICS.factory_deps_processing_duration[&FactoryDepsStage::SaveToGcs].start(); + let factory_deps = SnapshotFactoryDependencies { factory_deps }; + let filename = self + .blob_store + .put(l1_batch_number, &factory_deps) + .await + .context("Error storing factory deps in blob store")?; + let output_filepath_prefix = self + .blob_store + .get_storage_prefix::(); + let output_filepath = format!("{output_filepath_prefix}/{filename}"); + let latency = latency.observe(); + tracing::info!( + "Saved {} factory deps in {latency:?} to location: {output_filepath}", + factory_deps.factory_deps.len() + ); + + Ok(output_filepath) + } + + /// Returns `Ok(None)` if the created snapshot would coincide with `latest_snapshot`. + async fn initialize_snapshot_progress( + config: &SnapshotsCreatorConfig, + min_chunk_count: u64, + latest_snapshot: Option<&SnapshotMetadata>, + conn: &mut StorageProcessor<'_>, + ) -> anyhow::Result> { + // We subtract 1 so that after restore, EN node has at least one L1 batch to fetch + let sealed_l1_batch_number = conn.blocks_dal().get_sealed_l1_batch_number().await?; + let sealed_l1_batch_number = sealed_l1_batch_number.context("No L1 batches in Postgres")?; + anyhow::ensure!( + sealed_l1_batch_number != L1BatchNumber(0), + "Cannot create snapshot when only the genesis L1 batch is present in Postgres" + ); + let l1_batch_number = sealed_l1_batch_number - 1; + + let latest_snapshot_l1_batch_number = + latest_snapshot.map(|snapshot| snapshot.l1_batch_number); + if latest_snapshot_l1_batch_number == Some(l1_batch_number) { + tracing::info!( + "Snapshot at expected L1 batch #{l1_batch_number} is already created; exiting" + ); + return Ok(None); + } + + let distinct_storage_logs_keys_count = conn + .snapshots_creator_dal() + .get_distinct_storage_logs_keys_count(l1_batch_number) + .await?; + let chunk_size = config.storage_logs_chunk_size; + // We force the minimum number of chunks to avoid situations where only one chunk is created in tests. + let chunk_count = + ceil_div(distinct_storage_logs_keys_count, chunk_size).max(min_chunk_count); + + tracing::info!( + "Selected storage logs chunking for L1 batch {l1_batch_number}: \ + {chunk_count} chunks of expected size {chunk_size}" + ); + Ok(Some(SnapshotProgress::new(l1_batch_number, chunk_count))) + } + + /// Returns `Ok(None)` if a snapshot should not be created / resumed. + async fn load_or_initialize_snapshot_progress( + &self, + config: &SnapshotsCreatorConfig, + min_chunk_count: u64, + ) -> anyhow::Result> { + let mut master_conn = self + .master_pool + .access_storage_tagged("snapshots_creator") + .await?; + let latest_snapshot = master_conn + .snapshots_dal() + .get_newest_snapshot_metadata() + .await?; + drop(master_conn); + + let pending_snapshot = latest_snapshot + .as_ref() + .filter(|snapshot| !snapshot.is_complete()); + if let Some(snapshot) = pending_snapshot { + Ok(Some(SnapshotProgress::from_existing_snapshot(snapshot))) + } else { + Self::initialize_snapshot_progress( + config, + min_chunk_count, + latest_snapshot.as_ref(), + &mut self.connect_to_replica().await?, + ) + .await + } + } + + pub async fn run( + self, + config: SnapshotsCreatorConfig, + min_chunk_count: u64, + ) -> anyhow::Result<()> { + let latency = METRICS.snapshot_generation_duration.start(); + + let Some(progress) = self + .load_or_initialize_snapshot_progress(&config, min_chunk_count) + .await? + else { + // No snapshot creation is necessary; a snapshot for the current L1 batch is already created + return Ok(()); + }; + + let mut conn = self.connect_to_replica().await?; + let (_, last_miniblock_number_in_batch) = conn + .blocks_dal() + .get_miniblock_range_of_l1_batch(progress.l1_batch_number) + .await? + .context("Error fetching last miniblock number")?; + drop(conn); + + METRICS.storage_logs_chunks_count.set(progress.chunk_count); + tracing::info!( + "Creating snapshot for storage logs up to miniblock {last_miniblock_number_in_batch}, \ + L1 batch {}", + progress.l1_batch_number + ); + + if progress.is_new_snapshot { + let factory_deps_output_file = self + .process_factory_deps(last_miniblock_number_in_batch, progress.l1_batch_number) + .await?; + + let mut master_conn = self + .master_pool + .access_storage_tagged("snapshots_creator") + .await?; + master_conn + .snapshots_dal() + .add_snapshot( + progress.l1_batch_number, + progress.chunk_count, + &factory_deps_output_file, + ) + .await?; + } + + METRICS + .storage_logs_chunks_left_to_process + .set(progress.remaining_chunk_ids.len()); + let semaphore = Semaphore::new(config.concurrent_queries_count as usize); + let tasks = progress.remaining_chunk_ids.into_iter().map(|chunk_id| { + self.process_storage_logs_single_chunk( + &semaphore, + last_miniblock_number_in_batch, + progress.l1_batch_number, + chunk_id, + progress.chunk_count, + ) + }); + futures::future::try_join_all(tasks).await?; + + METRICS + .snapshot_l1_batch + .set(progress.l1_batch_number.0.into()); + + let elapsed = latency.observe(); + tracing::info!("snapshot_generation_duration: {elapsed:?}"); + tracing::info!("snapshot_l1_batch: {}", METRICS.snapshot_l1_batch.get()); + tracing::info!( + "storage_logs_chunks_count: {}", + METRICS.storage_logs_chunks_count.get() + ); + Ok(()) + } +} diff --git a/core/bin/snapshots_creator/src/main.rs b/core/bin/snapshots_creator/src/main.rs index e8f1fa3bf7fe..0571500615bb 100644 --- a/core/bin/snapshots_creator/src/main.rs +++ b/core/bin/snapshots_creator/src/main.rs @@ -1,29 +1,26 @@ //! Snapshot creator utility. Intended to run on a schedule, with each run creating a new snapshot. +//! +//! # Assumptions +//! +//! The snapshot creator is fault-tolerant; if it stops in the middle of creating a snapshot, +//! this snapshot will be continued from roughly the same point after the restart. If this is +//! undesired, remove the `snapshots` table record corresponding to the pending snapshot. +//! +//! It is assumed that the snapshot creator is run as a singleton process (no more than 1 instance +//! at a time). use anyhow::Context as _; use prometheus_exporter::PrometheusExporterConfig; -use tokio::{ - sync::{watch, Semaphore}, - task::JoinHandle, -}; +use tokio::{sync::watch, task::JoinHandle}; use zksync_config::{configs::PrometheusConfig, PostgresConfig, SnapshotsCreatorConfig}; use zksync_dal::ConnectionPool; use zksync_env_config::{object_store::SnapshotsObjectStoreConfig, FromEnv}; -use zksync_object_store::{ObjectStore, ObjectStoreFactory}; -use zksync_types::{ - snapshots::{ - SnapshotFactoryDependencies, SnapshotStorageLogsChunk, SnapshotStorageLogsStorageKey, - }, - L1BatchNumber, MiniblockNumber, -}; -use zksync_utils::ceil_div; +use zksync_object_store::ObjectStoreFactory; -use crate::{ - chunking::get_chunk_hashed_keys_range, - metrics::{FactoryDepsStage, StorageChunkStage, METRICS}, -}; +use crate::creator::SnapshotCreator; mod chunking; +mod creator; mod metrics; #[cfg(test)] mod tests; @@ -47,205 +44,6 @@ async fn maybe_enable_prometheus_metrics( } } -async fn process_storage_logs_single_chunk( - blob_store: &dyn ObjectStore, - pool: &ConnectionPool, - semaphore: &Semaphore, - miniblock_number: MiniblockNumber, - l1_batch_number: L1BatchNumber, - chunk_id: u64, - chunks_count: u64, -) -> anyhow::Result { - let _permit = semaphore.acquire().await?; - let hashed_keys_range = get_chunk_hashed_keys_range(chunk_id, chunks_count); - let mut conn = pool.access_storage_tagged("snapshots_creator").await?; - - let latency = - METRICS.storage_logs_processing_duration[&StorageChunkStage::LoadFromPostgres].start(); - let logs = conn - .snapshots_creator_dal() - .get_storage_logs_chunk(miniblock_number, hashed_keys_range) - .await - .context("Error fetching storage logs count")?; - drop(conn); - let latency = latency.observe(); - tracing::info!( - "Loaded chunk {chunk_id} ({} logs) from Postgres in {latency:?}", - logs.len() - ); - - let latency = METRICS.storage_logs_processing_duration[&StorageChunkStage::SaveToGcs].start(); - let storage_logs_chunk = SnapshotStorageLogsChunk { storage_logs: logs }; - let key = SnapshotStorageLogsStorageKey { - l1_batch_number, - chunk_id, - }; - let filename = blob_store - .put(key, &storage_logs_chunk) - .await - .context("Error storing storage logs chunk in blob store")?; - let output_filepath_prefix = blob_store.get_storage_prefix::(); - let output_filepath = format!("{output_filepath_prefix}/{filename}"); - let latency = latency.observe(); - - let tasks_left = METRICS.storage_logs_chunks_left_to_process.dec_by(1) - 1; - tracing::info!( - "Saved chunk {chunk_id} (overall progress {}/{chunks_count}) in {latency:?} to location: {output_filepath}", - chunks_count - tasks_left - ); - Ok(output_filepath) -} - -async fn process_factory_deps( - blob_store: &dyn ObjectStore, - pool: &ConnectionPool, - miniblock_number: MiniblockNumber, - l1_batch_number: L1BatchNumber, -) -> anyhow::Result { - let mut conn = pool.access_storage_tagged("snapshots_creator").await?; - - tracing::info!("Loading factory deps from Postgres..."); - let latency = - METRICS.factory_deps_processing_duration[&FactoryDepsStage::LoadFromPostgres].start(); - let factory_deps = conn - .snapshots_creator_dal() - .get_all_factory_deps(miniblock_number) - .await?; - drop(conn); - let latency = latency.observe(); - tracing::info!("Loaded {} factory deps in {latency:?}", factory_deps.len()); - - tracing::info!("Saving factory deps to GCS..."); - let latency = METRICS.factory_deps_processing_duration[&FactoryDepsStage::SaveToGcs].start(); - let factory_deps = SnapshotFactoryDependencies { factory_deps }; - let filename = blob_store - .put(l1_batch_number, &factory_deps) - .await - .context("Error storing factory deps in blob store")?; - let output_filepath_prefix = blob_store.get_storage_prefix::(); - let output_filepath = format!("{output_filepath_prefix}/{filename}"); - let latency = latency.observe(); - tracing::info!( - "Saved {} factory deps in {latency:?} to location: {output_filepath}", - factory_deps.factory_deps.len() - ); - - Ok(output_filepath) -} - -async fn run( - blob_store: Box, - replica_pool: ConnectionPool, - master_pool: ConnectionPool, - min_chunk_count: u64, -) -> anyhow::Result<()> { - let latency = METRICS.snapshot_generation_duration.start(); - let config = SnapshotsCreatorConfig::from_env().context("SnapshotsCreatorConfig::from_env")?; - - let mut conn = replica_pool - .access_storage_tagged("snapshots_creator") - .await?; - - // We subtract 1 so that after restore, EN node has at least one L1 batch to fetch - let sealed_l1_batch_number = conn.blocks_dal().get_sealed_l1_batch_number().await?; - let sealed_l1_batch_number = sealed_l1_batch_number.context("No L1 batches in Postgres")?; - anyhow::ensure!( - sealed_l1_batch_number != L1BatchNumber(0), - "Cannot create snapshot when only the genesis L1 batch is present in Postgres" - ); - let l1_batch_number = sealed_l1_batch_number - 1; - - let mut master_conn = master_pool - .access_storage_tagged("snapshots_creator") - .await?; - if master_conn - .snapshots_dal() - .get_snapshot_metadata(l1_batch_number) - .await? - .is_some() - { - tracing::info!("Snapshot for L1 batch number {l1_batch_number} already exists, exiting"); - return Ok(()); - } - drop(master_conn); - - let (_, last_miniblock_number_in_batch) = conn - .blocks_dal() - .get_miniblock_range_of_l1_batch(l1_batch_number) - .await? - .context("Error fetching last miniblock number")?; - let distinct_storage_logs_keys_count = conn - .snapshots_creator_dal() - .get_distinct_storage_logs_keys_count(l1_batch_number) - .await?; - drop(conn); - - let chunk_size = config.storage_logs_chunk_size; - // We force the minimum number of chunks to avoid situations where only one chunk is created in tests. - let chunks_count = ceil_div(distinct_storage_logs_keys_count, chunk_size).max(min_chunk_count); - - METRICS.storage_logs_chunks_count.set(chunks_count); - - tracing::info!( - "Creating snapshot for storage logs up to miniblock {last_miniblock_number_in_batch}, \ - L1 batch {l1_batch_number}" - ); - tracing::info!("Starting to generate {chunks_count} chunks of expected size {chunk_size}"); - - let factory_deps_output_file = process_factory_deps( - &*blob_store, - &replica_pool, - last_miniblock_number_in_batch, - l1_batch_number, - ) - .await?; - - METRICS - .storage_logs_chunks_left_to_process - .set(chunks_count); - - let semaphore = Semaphore::new(config.concurrent_queries_count as usize); - let tasks = (0..chunks_count).map(|chunk_id| { - process_storage_logs_single_chunk( - &*blob_store, - &replica_pool, - &semaphore, - last_miniblock_number_in_batch, - l1_batch_number, - chunk_id, - chunks_count, - ) - }); - let mut storage_logs_output_files = futures::future::try_join_all(tasks).await?; - // Sanity check: the number of files should equal the number of chunks. - assert_eq!(storage_logs_output_files.len(), chunks_count as usize); - storage_logs_output_files.sort(); - - tracing::info!("Finished generating snapshot, storing progress in Postgres"); - let mut master_conn = master_pool - .access_storage_tagged("snapshots_creator") - .await?; - master_conn - .snapshots_dal() - .add_snapshot( - l1_batch_number, - &storage_logs_output_files, - &factory_deps_output_file, - ) - .await?; - - METRICS.snapshot_l1_batch.set(l1_batch_number.0 as u64); - - let elapsed = latency.observe(); - tracing::info!("snapshot_generation_duration: {elapsed:?}"); - tracing::info!("snapshot_l1_batch: {}", METRICS.snapshot_l1_batch.get()); - tracing::info!( - "storage_logs_chunks_count: {}", - METRICS.storage_logs_chunks_count.get() - ); - Ok(()) -} - /// Minimum number of storage log chunks to produce. const MIN_CHUNK_COUNT: u64 = 10; @@ -292,7 +90,15 @@ async fn main() -> anyhow::Result<()> { .build() .await?; - run(blob_store, replica_pool, master_pool, MIN_CHUNK_COUNT).await?; + let creator = SnapshotCreator { + blob_store, + master_pool, + replica_pool, + #[cfg(test)] + event_listener: Box::new(()), + }; + creator.run(creator_config, MIN_CHUNK_COUNT).await?; + tracing::info!("Finished running snapshot creator!"); stop_sender.send(true).ok(); if let Some(prometheus_exporter_task) = prometheus_exporter_task { diff --git a/core/bin/snapshots_creator/src/metrics.rs b/core/bin/snapshots_creator/src/metrics.rs index 194ed8f1e680..5eb1984712e5 100644 --- a/core/bin/snapshots_creator/src/metrics.rs +++ b/core/bin/snapshots_creator/src/metrics.rs @@ -24,7 +24,7 @@ pub(crate) struct SnapshotsCreatorMetrics { /// Number of chunks in the most recently generated snapshot. Set when a snapshot generation starts. pub storage_logs_chunks_count: Gauge, /// Number of chunks left to process for the snapshot being currently generated. - pub storage_logs_chunks_left_to_process: Gauge, + pub storage_logs_chunks_left_to_process: Gauge, /// Total latency of snapshot generation. #[metrics(buckets = Buckets::LATENCIES, unit = Unit::Seconds)] pub snapshot_generation_duration: Histogram, diff --git a/core/bin/snapshots_creator/src/tests.rs b/core/bin/snapshots_creator/src/tests.rs index 33d7a225b7d6..c0e8dd0cbc28 100644 --- a/core/bin/snapshots_creator/src/tests.rs +++ b/core/bin/snapshots_creator/src/tests.rs @@ -1,17 +1,107 @@ //! Lower-level tests for the snapshot creator component. -use std::collections::{HashMap, HashSet}; +use std::{ + collections::{HashMap, HashSet}, + fmt, + sync::atomic::{AtomicUsize, Ordering}, +}; use rand::{thread_rng, Rng}; use zksync_dal::StorageProcessor; +use zksync_object_store::ObjectStore; use zksync_types::{ block::{BlockGasCount, L1BatchHeader, MiniblockHeader}, - snapshots::{SnapshotFactoryDependency, SnapshotStorageLog}, - AccountTreeId, Address, ProtocolVersion, StorageKey, StorageLog, H256, + snapshots::{ + SnapshotFactoryDependencies, SnapshotFactoryDependency, SnapshotStorageLog, + SnapshotStorageLogsChunk, SnapshotStorageLogsStorageKey, + }, + AccountTreeId, Address, L1BatchNumber, MiniblockNumber, ProtocolVersion, StorageKey, + StorageLog, H256, }; use super::*; +const TEST_CONFIG: SnapshotsCreatorConfig = SnapshotsCreatorConfig { + storage_logs_chunk_size: 1_000_000, + concurrent_queries_count: 10, +}; +const SEQUENTIAL_TEST_CONFIG: SnapshotsCreatorConfig = SnapshotsCreatorConfig { + storage_logs_chunk_size: 1_000_000, + concurrent_queries_count: 1, +}; + +#[derive(Debug)] +struct TestEventListener { + stop_after_chunk_count: usize, + processed_chunk_count: AtomicUsize, +} + +impl TestEventListener { + fn new(stop_after_chunk_count: usize) -> Self { + Self { + stop_after_chunk_count, + processed_chunk_count: AtomicUsize::new(0), + } + } +} + +impl HandleEvent for TestEventListener { + fn on_chunk_started(&self) -> TestBehavior { + let should_stop = + self.processed_chunk_count.load(Ordering::SeqCst) >= self.stop_after_chunk_count; + TestBehavior::new(should_stop) + } + + fn on_chunk_saved(&self) { + self.processed_chunk_count.fetch_add(1, Ordering::SeqCst); + } +} + +impl SnapshotCreator { + fn for_tests(blob_store: Box, pool: ConnectionPool) -> Self { + Self { + blob_store, + master_pool: pool.clone(), + replica_pool: pool, + event_listener: Box::new(()), + } + } + + fn stop_after_chunk_count(self, stop_after_chunk_count: usize) -> Self { + Self { + event_listener: Box::new(TestEventListener::new(stop_after_chunk_count)), + ..self + } + } +} + +#[derive(Debug)] +pub(crate) struct TestBehavior { + should_exit: bool, +} + +impl TestBehavior { + fn new(should_exit: bool) -> Self { + Self { should_exit } + } + + pub fn should_exit(&self) -> bool { + self.should_exit + } +} + +pub(crate) trait HandleEvent: fmt::Debug { + fn on_chunk_started(&self) -> TestBehavior { + TestBehavior::new(false) + } + + fn on_chunk_saved(&self) { + // Do nothing + } +} + +impl HandleEvent for () {} + fn gen_storage_logs(rng: &mut impl Rng, count: usize) -> Vec { (0..count) .map(|_| { @@ -159,12 +249,17 @@ async fn persisting_snapshot_metadata() { let mut conn = pool.access_storage().await.unwrap(); prepare_postgres(&mut rng, &mut conn, 10).await; - run(object_store, pool.clone(), pool.clone(), MIN_CHUNK_COUNT) + SnapshotCreator::for_tests(object_store, pool.clone()) + .run(TEST_CONFIG, MIN_CHUNK_COUNT) .await .unwrap(); // Check snapshot metadata in Postgres. - let snapshots = conn.snapshots_dal().get_all_snapshots().await.unwrap(); + let snapshots = conn + .snapshots_dal() + .get_all_complete_snapshots() + .await + .unwrap(); assert_eq!(snapshots.snapshots_l1_batch_numbers.len(), 1); let snapshot_l1_batch_number = snapshots.snapshots_l1_batch_numbers[0]; assert_eq!(snapshot_l1_batch_number, L1BatchNumber(8)); @@ -183,7 +278,11 @@ async fn persisting_snapshot_metadata() { MIN_CHUNK_COUNT as usize ); for path in &snapshot_metadata.storage_logs_filepaths { - let path = path.strip_prefix("storage_logs_snapshots/").unwrap(); + let path = path + .as_ref() + .unwrap() + .strip_prefix("storage_logs_snapshots/") + .unwrap(); assert!(path.ends_with(".proto.gzip")); } } @@ -194,12 +293,11 @@ async fn persisting_snapshot_factory_deps() { let mut rng = thread_rng(); let object_store_factory = ObjectStoreFactory::mock(); let object_store = object_store_factory.create_store().await; - - // Insert some data to Postgres. let mut conn = pool.access_storage().await.unwrap(); let expected_outputs = prepare_postgres(&mut rng, &mut conn, 10).await; - run(object_store, pool.clone(), pool.clone(), MIN_CHUNK_COUNT) + SnapshotCreator::for_tests(object_store, pool.clone()) + .run(TEST_CONFIG, MIN_CHUNK_COUNT) .await .unwrap(); let snapshot_l1_batch_number = L1BatchNumber(8); @@ -217,17 +315,24 @@ async fn persisting_snapshot_logs() { let mut rng = thread_rng(); let object_store_factory = ObjectStoreFactory::mock(); let object_store = object_store_factory.create_store().await; - - // Insert some data to Postgres. let mut conn = pool.access_storage().await.unwrap(); let expected_outputs = prepare_postgres(&mut rng, &mut conn, 10).await; - run(object_store, pool.clone(), pool.clone(), MIN_CHUNK_COUNT) + SnapshotCreator::for_tests(object_store, pool.clone()) + .run(TEST_CONFIG, MIN_CHUNK_COUNT) .await .unwrap(); let snapshot_l1_batch_number = L1BatchNumber(8); let object_store = object_store_factory.create_store().await; + assert_storage_logs(&*object_store, snapshot_l1_batch_number, &expected_outputs).await; +} + +async fn assert_storage_logs( + object_store: &dyn ObjectStore, + snapshot_l1_batch_number: L1BatchNumber, + expected_outputs: &ExpectedOutputs, +) { let mut actual_logs = HashSet::new(); for chunk_id in 0..MIN_CHUNK_COUNT { let key = SnapshotStorageLogsStorageKey { @@ -239,3 +344,114 @@ async fn persisting_snapshot_logs() { } assert_eq!(actual_logs, expected_outputs.storage_logs); } + +#[tokio::test] +async fn recovery_workflow() { + let pool = ConnectionPool::test_pool().await; + let mut rng = thread_rng(); + let object_store_factory = ObjectStoreFactory::mock(); + let object_store = object_store_factory.create_store().await; + let mut conn = pool.access_storage().await.unwrap(); + let expected_outputs = prepare_postgres(&mut rng, &mut conn, 10).await; + + SnapshotCreator::for_tests(object_store, pool.clone()) + .stop_after_chunk_count(0) + .run(SEQUENTIAL_TEST_CONFIG, MIN_CHUNK_COUNT) + .await + .unwrap(); + + let snapshot_l1_batch_number = L1BatchNumber(8); + let snapshot_metadata = conn + .snapshots_dal() + .get_snapshot_metadata(snapshot_l1_batch_number) + .await + .unwrap() + .expect("No snapshot metadata"); + assert!(snapshot_metadata + .storage_logs_filepaths + .iter() + .all(Option::is_none)); + + let object_store = object_store_factory.create_store().await; + let SnapshotFactoryDependencies { factory_deps } = + object_store.get(snapshot_l1_batch_number).await.unwrap(); + let actual_deps: HashSet<_> = factory_deps.into_iter().collect(); + assert_eq!(actual_deps, expected_outputs.deps); + + // Process 2 storage log chunks, then stop. + SnapshotCreator::for_tests(object_store, pool.clone()) + .stop_after_chunk_count(2) + .run(SEQUENTIAL_TEST_CONFIG, MIN_CHUNK_COUNT) + .await + .unwrap(); + + let snapshot_metadata = conn + .snapshots_dal() + .get_snapshot_metadata(snapshot_l1_batch_number) + .await + .unwrap() + .expect("No snapshot metadata"); + assert_eq!( + snapshot_metadata + .storage_logs_filepaths + .iter() + .flatten() + .count(), + 2 + ); + + // Process the remaining chunks. + let object_store = object_store_factory.create_store().await; + SnapshotCreator::for_tests(object_store, pool.clone()) + .run(SEQUENTIAL_TEST_CONFIG, MIN_CHUNK_COUNT) + .await + .unwrap(); + + let object_store = object_store_factory.create_store().await; + assert_storage_logs(&*object_store, snapshot_l1_batch_number, &expected_outputs).await; +} + +#[tokio::test] +async fn recovery_workflow_with_varying_chunk_size() { + let pool = ConnectionPool::test_pool().await; + let mut rng = thread_rng(); + let object_store_factory = ObjectStoreFactory::mock(); + let object_store = object_store_factory.create_store().await; + let mut conn = pool.access_storage().await.unwrap(); + let expected_outputs = prepare_postgres(&mut rng, &mut conn, 10).await; + + SnapshotCreator::for_tests(object_store, pool.clone()) + .stop_after_chunk_count(2) + .run(SEQUENTIAL_TEST_CONFIG, MIN_CHUNK_COUNT) + .await + .unwrap(); + + let snapshot_l1_batch_number = L1BatchNumber(8); + let snapshot_metadata = conn + .snapshots_dal() + .get_snapshot_metadata(snapshot_l1_batch_number) + .await + .unwrap() + .expect("No snapshot metadata"); + assert_eq!( + snapshot_metadata + .storage_logs_filepaths + .iter() + .flatten() + .count(), + 2 + ); + + let config_with_other_size = SnapshotsCreatorConfig { + storage_logs_chunk_size: 1, // << should be ignored + ..SEQUENTIAL_TEST_CONFIG + }; + let object_store = object_store_factory.create_store().await; + SnapshotCreator::for_tests(object_store, pool.clone()) + .run(config_with_other_size, MIN_CHUNK_COUNT) + .await + .unwrap(); + + let object_store = object_store_factory.create_store().await; + assert_storage_logs(&*object_store, snapshot_l1_batch_number, &expected_outputs).await; +} diff --git a/core/lib/dal/sqlx-data.json b/core/lib/dal/sqlx-data.json index 1caa9f54a1e8..183ab24fc458 100644 --- a/core/lib/dal/sqlx-data.json +++ b/core/lib/dal/sqlx-data.json @@ -339,6 +339,24 @@ }, "query": "\n SELECT\n storage_refunds\n FROM\n l1_batches\n WHERE\n number = $1\n " }, + "040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981": { + "describe": { + "columns": [ + { + "name": "l1_batch_number", + "ordinal": 0, + "type_info": "Int8" + } + ], + "nullable": [ + false + ], + "parameters": { + "Left": [] + } + }, + "query": "\n SELECT\n l1_batch_number\n FROM\n snapshots\n WHERE\n NOT (''::TEXT = ANY (storage_logs_filepaths))\n ORDER BY\n l1_batch_number DESC\n " + }, "04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3": { "describe": { "columns": [ @@ -3543,6 +3561,20 @@ }, "query": "\n UPDATE transactions\n SET\n in_mempool = FALSE\n FROM\n UNNEST($1::bytea[]) AS s (address)\n WHERE\n transactions.in_mempool = TRUE\n AND transactions.initiator_address = s.address\n " }, + "31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Text" + ] + } + }, + "query": "\n INSERT INTO\n snapshots (\n l1_batch_number,\n storage_logs_filepaths,\n factory_deps_filepath,\n created_at,\n updated_at\n )\n VALUES\n ($1, ARRAY_FILL(''::TEXT, ARRAY[$2::INTEGER]), $3, NOW(), NOW())\n " + }, "314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37": { "describe": { "columns": [], @@ -4575,36 +4607,6 @@ }, "query": "\n INSERT INTO\n prover_fri_protocol_versions (\n id,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW())\n ON CONFLICT (id) DO NOTHING\n " }, - "4e2cb66131a524d1bd628424d0c0735d7f9b0b5820ae3a07467d2e76cd6280f9": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "factory_deps_filepath", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "storage_logs_filepaths", - "ordinal": 2, - "type_info": "TextArray" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n " - }, "525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63": { "describe": { "columns": [ @@ -6724,20 +6726,6 @@ }, "query": "\n INSERT INTO\n storage (hashed_key, address, key, value, tx_hash, created_at, updated_at)\n SELECT\n u.hashed_key,\n u.address,\n u.key,\n u.value,\n u.tx_hash,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::bytea[], $3::bytea[], $4::bytea[], $5::bytea[]) AS u (hashed_key, address, key, value, tx_hash)\n ON CONFLICT (hashed_key) DO\n UPDATE\n SET\n tx_hash = excluded.tx_hash,\n value = excluded.value,\n updated_at = NOW()\n " }, - "83134807aee4b6154a1aee4f76dd989d5b4637a97f815b84ace70587acc95e7c": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "TextArray", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n snapshots (\n l1_batch_number,\n storage_logs_filepaths,\n factory_deps_filepath,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, NOW(), NOW())\n " - }, "83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8": { "describe": { "columns": [ @@ -7570,6 +7558,36 @@ }, "query": "\n SELECT\n factory_deps.bytecode,\n transactions.data AS \"data?\",\n transactions.contract_address AS \"contract_address?\"\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n ORDER BY\n miniblock_number DESC,\n operation_number DESC\n LIMIT\n 1\n ) storage_logs\n JOIN factory_deps ON factory_deps.bytecode_hash = storage_logs.value\n LEFT JOIN transactions ON transactions.hash = storage_logs.tx_hash\n WHERE\n storage_logs.value != $2\n " }, + "995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068": { + "describe": { + "columns": [ + { + "name": "l1_batch_number", + "ordinal": 0, + "type_info": "Int8" + }, + { + "name": "factory_deps_filepath", + "ordinal": 1, + "type_info": "Text" + }, + { + "name": "storage_logs_filepaths", + "ordinal": 2, + "type_info": "TextArray" + } + ], + "nullable": [ + false, + false, + false + ], + "parameters": { + "Left": [] + } + }, + "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n ORDER BY\n l1_batch_number DESC\n LIMIT\n 1\n " + }, "99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6": { "describe": { "columns": [ @@ -7779,6 +7797,20 @@ }, "query": "\n UPDATE contract_verification_requests\n SET\n status = 'successful',\n updated_at = NOW()\n WHERE\n id = $1\n " }, + "9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Text" + ] + } + }, + "query": "\n UPDATE snapshots\n SET\n storage_logs_filepaths[$2] = $3,\n updated_at = NOW()\n WHERE\n l1_batch_number = $1\n " + }, "9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03": { "describe": { "columns": [], diff --git a/core/lib/dal/src/snapshots_dal.rs b/core/lib/dal/src/snapshots_dal.rs index 9582b3a72094..3b2e62085bb6 100644 --- a/core/lib/dal/src/snapshots_dal.rs +++ b/core/lib/dal/src/snapshots_dal.rs @@ -5,6 +5,27 @@ use zksync_types::{ use crate::{instrument::InstrumentExt, StorageProcessor}; +#[derive(Debug, sqlx::FromRow)] +struct StorageSnapshotMetadata { + l1_batch_number: i64, + storage_logs_filepaths: Vec, + factory_deps_filepath: String, +} + +impl From for SnapshotMetadata { + fn from(row: StorageSnapshotMetadata) -> Self { + Self { + l1_batch_number: L1BatchNumber(row.l1_batch_number as u32), + storage_logs_filepaths: row + .storage_logs_filepaths + .into_iter() + .map(|path| (!path.is_empty()).then_some(path)) + .collect(), + factory_deps_filepath: row.factory_deps_filepath, + } + } +} + #[derive(Debug)] pub struct SnapshotsDal<'a, 'c> { pub(crate) storage: &'a mut StorageProcessor<'c>, @@ -14,9 +35,9 @@ impl SnapshotsDal<'_, '_> { pub async fn add_snapshot( &mut self, l1_batch_number: L1BatchNumber, - storage_logs_filepaths: &[String], + storage_logs_chunk_count: u64, factory_deps_filepaths: &str, - ) -> Result<(), sqlx::Error> { + ) -> sqlx::Result<()> { sqlx::query!( r#" INSERT INTO @@ -28,10 +49,10 @@ impl SnapshotsDal<'_, '_> { updated_at ) VALUES - ($1, $2, $3, NOW(), NOW()) + ($1, ARRAY_FILL(''::TEXT, ARRAY[$2::INTEGER]), $3, NOW(), NOW()) "#, l1_batch_number.0 as i32, - storage_logs_filepaths, + storage_logs_chunk_count as i32, factory_deps_filepaths, ) .instrument("add_snapshot") @@ -41,34 +62,89 @@ impl SnapshotsDal<'_, '_> { Ok(()) } - pub async fn get_all_snapshots(&mut self) -> Result { - let records: Vec = sqlx::query!( + pub async fn add_storage_logs_filepath_for_snapshot( + &mut self, + l1_batch_number: L1BatchNumber, + chunk_id: u64, + storage_logs_filepath: &str, + ) -> sqlx::Result<()> { + sqlx::query!( + r#" + UPDATE snapshots + SET + storage_logs_filepaths[$2] = $3, + updated_at = NOW() + WHERE + l1_batch_number = $1 + "#, + l1_batch_number.0 as i32, + chunk_id as i32 + 1, + storage_logs_filepath, + ) + .execute(self.storage.conn()) + .await?; + + Ok(()) + } + + pub async fn get_all_complete_snapshots(&mut self) -> sqlx::Result { + let rows = sqlx::query!( r#" SELECT - l1_batch_number, - factory_deps_filepath, - storage_logs_filepaths + l1_batch_number FROM snapshots + WHERE + NOT (''::TEXT = ANY (storage_logs_filepaths)) + ORDER BY + l1_batch_number DESC "# ) - .instrument("get_all_snapshots") + .instrument("get_all_complete_snapshots") .report_latency() .fetch_all(self.storage.conn()) - .await? - .into_iter() - .map(|r| L1BatchNumber(r.l1_batch_number as u32)) - .collect(); + .await?; + + let snapshots_l1_batch_numbers = rows + .into_iter() + .map(|row| L1BatchNumber(row.l1_batch_number as u32)) + .collect(); + Ok(AllSnapshots { - snapshots_l1_batch_numbers: records, + snapshots_l1_batch_numbers, }) } + pub async fn get_newest_snapshot_metadata(&mut self) -> sqlx::Result> { + let row = sqlx::query_as!( + StorageSnapshotMetadata, + r#" + SELECT + l1_batch_number, + factory_deps_filepath, + storage_logs_filepaths + FROM + snapshots + ORDER BY + l1_batch_number DESC + LIMIT + 1 + "# + ) + .instrument("get_newest_snapshot_metadata") + .report_latency() + .fetch_optional(self.storage.conn()) + .await?; + + Ok(row.map(Into::into)) + } + pub async fn get_snapshot_metadata( &mut self, l1_batch_number: L1BatchNumber, - ) -> Result, sqlx::Error> { - let record: Option = sqlx::query!( + ) -> sqlx::Result> { + let row = sqlx::query_as!( + StorageSnapshotMetadata, r#" SELECT l1_batch_number, @@ -84,13 +160,9 @@ impl SnapshotsDal<'_, '_> { .instrument("get_snapshot_metadata") .report_latency() .fetch_optional(self.storage.conn()) - .await? - .map(|r| SnapshotMetadata { - l1_batch_number: L1BatchNumber(r.l1_batch_number as u32), - factory_deps_filepath: r.factory_deps_filepath, - storage_logs_filepaths: r.storage_logs_filepaths, - }); - Ok(record) + .await?; + + Ok(row.map(Into::into)) } } @@ -106,29 +178,38 @@ mod tests { let mut conn = pool.access_storage().await.unwrap(); let mut dal = conn.snapshots_dal(); let l1_batch_number = L1BatchNumber(100); - dal.add_snapshot(l1_batch_number, &[], "gs:///bucket/factory_deps.bin") + dal.add_snapshot(l1_batch_number, 2, "gs:///bucket/factory_deps.bin") .await .expect("Failed to add snapshot"); let snapshots = dal - .get_all_snapshots() + .get_all_complete_snapshots() .await .expect("Failed to retrieve snapshots"); - assert_eq!(1, snapshots.snapshots_l1_batch_numbers.len()); - assert_eq!( - snapshots.snapshots_l1_batch_numbers[0], - l1_batch_number as L1BatchNumber - ); + assert_eq!(snapshots.snapshots_l1_batch_numbers, []); + + for i in 0..2 { + dal.add_storage_logs_filepath_for_snapshot( + l1_batch_number, + i, + "gs:///bucket/chunk.bin", + ) + .await + .unwrap(); + } + + let snapshots = dal + .get_all_complete_snapshots() + .await + .expect("Failed to retrieve snapshots"); + assert_eq!(snapshots.snapshots_l1_batch_numbers, [l1_batch_number]); let snapshot_metadata = dal .get_snapshot_metadata(l1_batch_number) .await .expect("Failed to retrieve snapshot") .unwrap(); - assert_eq!( - snapshot_metadata.l1_batch_number, - l1_batch_number as L1BatchNumber - ); + assert_eq!(snapshot_metadata.l1_batch_number, l1_batch_number); } #[tokio::test] @@ -137,16 +218,14 @@ mod tests { let mut conn = pool.access_storage().await.unwrap(); let mut dal = conn.snapshots_dal(); let l1_batch_number = L1BatchNumber(100); - dal.add_snapshot( - l1_batch_number, - &[ - "gs:///bucket/test_file1.bin".to_string(), - "gs:///bucket/test_file2.bin".to_string(), - ], - "gs:///bucket/factory_deps.bin", - ) - .await - .expect("Failed to add snapshot"); + dal.add_snapshot(l1_batch_number, 2, "gs:///bucket/factory_deps.bin") + .await + .expect("Failed to add snapshot"); + + let storage_log_filepaths = ["gs:///bucket/test_file1.bin", "gs:///bucket/test_file2.bin"]; + dal.add_storage_logs_filepath_for_snapshot(l1_batch_number, 1, storage_log_filepaths[1]) + .await + .unwrap(); let files = dal .get_snapshot_metadata(l1_batch_number) @@ -154,7 +233,27 @@ mod tests { .expect("Failed to retrieve snapshot") .unwrap() .storage_logs_filepaths; - assert!(files.contains(&"gs:///bucket/test_file1.bin".to_string())); - assert!(files.contains(&"gs:///bucket/test_file2.bin".to_string())); + assert_eq!( + files, + [None, Some("gs:///bucket/test_file2.bin".to_string())] + ); + + dal.add_storage_logs_filepath_for_snapshot(l1_batch_number, 0, storage_log_filepaths[0]) + .await + .unwrap(); + + let files = dal + .get_snapshot_metadata(l1_batch_number) + .await + .expect("Failed to retrieve snapshot") + .unwrap() + .storage_logs_filepaths; + assert_eq!( + files, + [ + Some("gs:///bucket/test_file1.bin".to_string()), + Some("gs:///bucket/test_file2.bin".to_string()) + ] + ); } } diff --git a/core/lib/types/src/snapshots.rs b/core/lib/types/src/snapshots.rs index c4804880c2ac..b71a8e34b4be 100644 --- a/core/lib/types/src/snapshots.rs +++ b/core/lib/types/src/snapshots.rs @@ -7,28 +7,42 @@ use zksync_protobuf::{required, ProtoFmt}; use crate::{commitment::L1BatchWithMetadata, Bytes, StorageKey, StorageValue}; +/// Information about all snapshots persisted by the node. #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct AllSnapshots { + /// L1 batch numbers for complete snapshots. Ordered by descending number (i.e., 0th element + /// corresponds to the newest snapshot). pub snapshots_l1_batch_numbers: Vec, } -// used in dal to fetch certain snapshot data -#[derive(Debug, Clone, Serialize, Deserialize)] -#[serde(rename_all = "camelCase")] +/// Storage snapshot metadata. Used in DAL to fetch certain snapshot data. +#[derive(Debug, Clone)] pub struct SnapshotMetadata { + /// L1 batch for the snapshot. The data in the snapshot captures node storage at the end of this batch. pub l1_batch_number: L1BatchNumber, + /// Path to the factory dependencies blob. pub factory_deps_filepath: String, - pub storage_logs_filepaths: Vec, + /// Paths to the storage log blobs. Ordered by the chunk ID. If a certain chunk is not produced yet, + /// the corresponding path is `None`. + pub storage_logs_filepaths: Vec>, +} + +impl SnapshotMetadata { + /// Checks whether a snapshot is complete (contains all information to restore from). + pub fn is_complete(&self) -> bool { + self.storage_logs_filepaths.iter().all(Option::is_some) + } } -//contains all data not contained in factory_deps/storage_logs files to perform restore process +/// Snapshot data returned by using JSON-RPC API. +/// Contains all data not contained in factory deps / storage logs files to perform restore process. #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct SnapshotHeader { pub l1_batch_number: L1BatchNumber, pub miniblock_number: MiniblockNumber, - //ordered by chunk ids + /// Ordered by chunk IDs. pub storage_logs_chunks: Vec, pub factory_deps_filepath: String, pub last_l1_batch_with_metadata: L1BatchWithMetadata, @@ -42,7 +56,7 @@ pub struct SnapshotStorageLogsChunkMetadata { pub filepath: String, } -#[derive(Debug, Clone, Copy, Serialize, Deserialize, PartialEq)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct SnapshotStorageLogsStorageKey { pub l1_batch_number: L1BatchNumber, diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs b/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs index bd192e872866..b45fe9a472ee 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs +++ b/core/lib/zksync_core/src/api_server/web3/namespaces/snapshots.rs @@ -29,7 +29,7 @@ impl SnapshotsNamespace { .map_err(|err| internal_error(method_name, err))?; let mut snapshots_dal = storage_processor.snapshots_dal(); let response = snapshots_dal - .get_all_snapshots() + .get_all_complete_snapshots() .await .map_err(|err| internal_error(method_name, err)); method_latency.observe(); @@ -48,45 +48,61 @@ impl SnapshotsNamespace { .access_storage_tagged("api") .await .map_err(|err| internal_error(method_name, err))?; - let mut snapshots_dal = storage_processor.snapshots_dal(); - let snapshot_metadata = snapshots_dal + let snapshot_metadata = storage_processor + .snapshots_dal() .get_snapshot_metadata(l1_batch_number) .await .map_err(|err| internal_error(method_name, err))?; - if let Some(snapshot_metadata) = snapshot_metadata { - let snapshot_files = snapshot_metadata.storage_logs_filepaths.clone(); - let chunks = snapshot_files - .iter() - .enumerate() - .map(|(chunk_id, filepath)| SnapshotStorageLogsChunkMetadata { - chunk_id: chunk_id as u64, - filepath: filepath.clone(), - }) - .collect(); - let l1_batch_with_metadata = storage_processor - .blocks_dal() - .get_l1_batch_metadata(l1_batch_number) - .await - .map_err(|err| internal_error(method_name, err))? - .unwrap(); - let miniblock_number = storage_processor - .blocks_dal() - .get_miniblock_range_of_l1_batch(l1_batch_number) - .await - .map_err(|err| internal_error(method_name, err))? - .unwrap() - .1; + + let Some(snapshot_metadata) = snapshot_metadata else { method_latency.observe(); - Ok(Some(SnapshotHeader { - l1_batch_number: snapshot_metadata.l1_batch_number, - miniblock_number, - last_l1_batch_with_metadata: l1_batch_with_metadata, - storage_logs_chunks: chunks, - factory_deps_filepath: snapshot_metadata.factory_deps_filepath, - })) - } else { + return Ok(None); + }; + + let snapshot_files = snapshot_metadata.storage_logs_filepaths; + let is_complete = snapshot_files.iter().all(Option::is_some); + if !is_complete { + // We don't return incomplete snapshots via API. method_latency.observe(); - Ok(None) + return Ok(None); } + + let chunks = snapshot_files + .into_iter() + .enumerate() + .filter_map(|(chunk_id, filepath)| { + Some(SnapshotStorageLogsChunkMetadata { + chunk_id: chunk_id as u64, + filepath: filepath?, + }) + }) + .collect(); + let l1_batch_with_metadata = storage_processor + .blocks_dal() + .get_l1_batch_metadata(l1_batch_number) + .await + .map_err(|err| internal_error(method_name, err))? + .ok_or_else(|| { + let err = format!("missing metadata for L1 batch #{l1_batch_number}"); + internal_error(method_name, err) + })?; + let (_, miniblock_number) = storage_processor + .blocks_dal() + .get_miniblock_range_of_l1_batch(l1_batch_number) + .await + .map_err(|err| internal_error(method_name, err))? + .ok_or_else(|| { + let err = format!("missing miniblocks for L1 batch #{l1_batch_number}"); + internal_error(method_name, err) + })?; + + method_latency.observe(); + Ok(Some(SnapshotHeader { + l1_batch_number: snapshot_metadata.l1_batch_number, + miniblock_number, + last_l1_batch_with_metadata: l1_batch_with_metadata, + storage_logs_chunks: chunks, + factory_deps_filepath: snapshot_metadata.factory_deps_filepath, + })) } } diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs index 7160975d8706..3333e72faf54 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs @@ -30,6 +30,7 @@ use crate::{ state_keeper::tests::create_l2_transaction, }; +mod snapshots; mod ws; const TEST_TIMEOUT: Duration = Duration::from_secs(10); @@ -138,6 +139,9 @@ async fn spawn_server( .await; let (pub_sub_events_sender, pub_sub_events_receiver) = mpsc::unbounded_channel(); + let mut namespaces = Namespace::DEFAULT.to_vec(); + namespaces.push(Namespace::Snapshots); + let server_builder = match transport { ApiTransportLabel::Http => ApiBuilder::jsonrpsee_backend(api_config, pool).http(0), ApiTransportLabel::Ws => { @@ -156,7 +160,7 @@ async fn spawn_server( .with_polling_interval(POLL_INTERVAL) .with_tx_sender(tx_sender, vm_barrier) .with_pub_sub_events(pub_sub_events_sender) - .enable_api_namespaces(Namespace::DEFAULT.to_vec()) + .enable_api_namespaces(namespaces) .build(stop_receiver) .await .expect("Failed spawning JSON-RPC server"); @@ -221,8 +225,9 @@ fn create_miniblock(number: u32) -> MiniblockHeader { } } -async fn store_block(pool: &ConnectionPool) -> anyhow::Result<(MiniblockHeader, H256)> { - let mut storage = pool.access_storage().await?; +async fn store_miniblock( + storage: &mut StorageProcessor<'_>, +) -> anyhow::Result<(MiniblockHeader, H256)> { let new_tx = create_l2_transaction(1, 2); let new_tx_hash = new_tx.hash(); let tx_submission_result = storage @@ -295,10 +300,10 @@ async fn store_events( } #[derive(Debug)] -struct HttpServerBasics; +struct HttpServerBasicsTest; #[async_trait] -impl HttpTest for HttpServerBasics { +impl HttpTest for HttpServerBasicsTest { async fn test(&self, client: &HttpClient, _pool: &ConnectionPool) -> anyhow::Result<()> { let block_number = client.get_block_number().await?; assert_eq!(block_number, U64::from(0)); @@ -317,19 +322,20 @@ impl HttpTest for HttpServerBasics { #[tokio::test] async fn http_server_basics() { - test_http_server(HttpServerBasics).await; + test_http_server(HttpServerBasicsTest).await; } #[derive(Debug)] -struct BasicFilterChanges; +struct BasicFilterChangesTest; #[async_trait] -impl HttpTest for BasicFilterChanges { +impl HttpTest for BasicFilterChangesTest { async fn test(&self, client: &HttpClient, pool: &ConnectionPool) -> anyhow::Result<()> { let block_filter_id = client.new_block_filter().await?; let tx_filter_id = client.new_pending_transaction_filter().await?; - let (new_miniblock, new_tx_hash) = store_block(pool).await?; + let (new_miniblock, new_tx_hash) = + store_miniblock(&mut pool.access_storage().await?).await?; let block_filter_changes = client.get_filter_changes(block_filter_id).await?; assert_matches!( @@ -364,14 +370,14 @@ impl HttpTest for BasicFilterChanges { #[tokio::test] async fn basic_filter_changes() { - test_http_server(BasicFilterChanges).await; + test_http_server(BasicFilterChangesTest).await; } #[derive(Debug)] -struct LogFilterChanges; +struct LogFilterChangesTest; #[async_trait] -impl HttpTest for LogFilterChanges { +impl HttpTest for LogFilterChangesTest { async fn test(&self, client: &HttpClient, pool: &ConnectionPool) -> anyhow::Result<()> { let all_logs_filter_id = client.new_filter(Filter::default()).await?; let address_filter = Filter { @@ -419,14 +425,14 @@ impl HttpTest for LogFilterChanges { #[tokio::test] async fn log_filter_changes() { - test_http_server(LogFilterChanges).await; + test_http_server(LogFilterChangesTest).await; } #[derive(Debug)] -struct LogFilterChangesWithBlockBoundaries; +struct LogFilterChangesWithBlockBoundariesTest; #[async_trait] -impl HttpTest for LogFilterChangesWithBlockBoundaries { +impl HttpTest for LogFilterChangesWithBlockBoundariesTest { async fn test(&self, client: &HttpClient, pool: &ConnectionPool) -> anyhow::Result<()> { let lower_bound_filter = Filter { from_block: Some(api::BlockNumber::Number(2.into())), @@ -516,5 +522,5 @@ impl HttpTest for LogFilterChangesWithBlockBoundaries { #[tokio::test] async fn log_filter_changes_with_block_boundaries() { - test_http_server(LogFilterChangesWithBlockBoundaries).await; + test_http_server(LogFilterChangesWithBlockBoundariesTest).await; } diff --git a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs new file mode 100644 index 000000000000..608f2845065f --- /dev/null +++ b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs @@ -0,0 +1,125 @@ +//! Tests for the `snapshots` Web3 namespace. + +use std::collections::HashSet; + +use zksync_types::block::{BlockGasCount, L1BatchHeader}; +use zksync_web3_decl::namespaces::SnapshotsNamespaceClient; + +use super::*; +use crate::state_keeper::tests::create_l1_batch_metadata; + +async fn seal_l1_batch( + storage: &mut StorageProcessor<'_>, + number: L1BatchNumber, +) -> anyhow::Result<()> { + let header = L1BatchHeader::new( + number, + number.0.into(), + Address::repeat_byte(1), + BaseSystemContractsHashes::default(), + ProtocolVersionId::latest(), + ); + storage + .blocks_dal() + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .await?; + storage + .blocks_dal() + .mark_miniblocks_as_executed_in_l1_batch(number) + .await?; + let metadata = create_l1_batch_metadata(number.0); + storage + .blocks_dal() + .save_l1_batch_metadata(number, &metadata, H256::zero(), false) + .await?; + Ok(()) +} + +#[derive(Debug)] +struct SnapshotBasicsTest { + chunk_ids: HashSet, +} + +impl SnapshotBasicsTest { + const CHUNK_COUNT: u64 = 5; + + fn new(chunk_ids: impl IntoIterator) -> Self { + let chunk_ids: HashSet<_> = chunk_ids.into_iter().collect(); + assert!(chunk_ids.iter().all(|&id| id < Self::CHUNK_COUNT)); + Self { chunk_ids } + } + + fn is_complete_snapshot(&self) -> bool { + self.chunk_ids == HashSet::from_iter(0..Self::CHUNK_COUNT) + } +} + +#[async_trait] +impl HttpTest for SnapshotBasicsTest { + async fn test(&self, client: &HttpClient, pool: &ConnectionPool) -> anyhow::Result<()> { + let mut storage = pool.access_storage().await.unwrap(); + store_miniblock(&mut storage).await?; + seal_l1_batch(&mut storage, L1BatchNumber(1)).await?; + storage + .snapshots_dal() + .add_snapshot(L1BatchNumber(1), Self::CHUNK_COUNT, "file:///factory_deps") + .await?; + + for &chunk_id in &self.chunk_ids { + let path = format!("file:///storage_logs/chunk{chunk_id}"); + storage + .snapshots_dal() + .add_storage_logs_filepath_for_snapshot(L1BatchNumber(1), chunk_id, &path) + .await?; + } + + let all_snapshots = client.get_all_snapshots().await?; + if self.is_complete_snapshot() { + assert_eq!(all_snapshots.snapshots_l1_batch_numbers, [L1BatchNumber(1)]); + } else { + assert_eq!(all_snapshots.snapshots_l1_batch_numbers, []); + } + + let snapshot_header = client + .get_snapshot_by_l1_batch_number(L1BatchNumber(1)) + .await?; + let snapshot_header = if self.is_complete_snapshot() { + snapshot_header.context("no snapshot for L1 batch #1")? + } else { + assert!(snapshot_header.is_none()); + return Ok(()); + }; + + assert_eq!(snapshot_header.l1_batch_number, L1BatchNumber(1)); + assert_eq!(snapshot_header.miniblock_number, MiniblockNumber(1)); + assert_eq!( + snapshot_header.factory_deps_filepath, + "file:///factory_deps" + ); + + assert_eq!( + snapshot_header.storage_logs_chunks.len(), + self.chunk_ids.len() + ); + for chunk in &snapshot_header.storage_logs_chunks { + assert!(self.chunk_ids.contains(&chunk.chunk_id)); + assert!(chunk.filepath.starts_with("file:///storage_logs/")); + } + Ok(()) + } +} + +#[tokio::test] +async fn snapshot_without_chunks() { + test_http_server(SnapshotBasicsTest::new([])).await; +} + +#[tokio::test] +async fn snapshot_with_some_chunks() { + test_http_server(SnapshotBasicsTest::new([0, 2, 4])).await; +} + +#[tokio::test] +async fn snapshot_with_all_chunks() { + test_http_server(SnapshotBasicsTest::new(0..SnapshotBasicsTest::CHUNK_COUNT)).await; +} diff --git a/core/lib/zksync_core/src/api_server/web3/tests/ws.rs b/core/lib/zksync_core/src/api_server/web3/tests/ws.rs index af062f8367b3..509b1e194e7f 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/ws.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/ws.rs @@ -114,10 +114,10 @@ async fn test_ws_server(test: impl WsTest) { } #[derive(Debug)] -struct WsServerCanStart; +struct WsServerCanStartTest; #[async_trait] -impl WsTest for WsServerCanStart { +impl WsTest for WsServerCanStartTest { async fn test( &self, client: &WsClient, @@ -141,14 +141,14 @@ impl WsTest for WsServerCanStart { #[tokio::test] async fn ws_server_can_start() { - test_ws_server(WsServerCanStart).await; + test_ws_server(WsServerCanStartTest).await; } #[derive(Debug)] -struct BasicSubscriptions; +struct BasicSubscriptionsTest; #[async_trait] -impl WsTest for BasicSubscriptions { +impl WsTest for BasicSubscriptionsTest { async fn test( &self, client: &WsClient, @@ -172,7 +172,8 @@ impl WsTest for BasicSubscriptions { .await?; wait_for_subscription(&mut pub_sub_events, SubscriptionType::Txs).await; - let (new_miniblock, new_tx_hash) = store_block(pool).await?; + let (new_miniblock, new_tx_hash) = + store_miniblock(&mut pool.access_storage().await?).await?; let received_tx_hash = tokio::time::timeout(TEST_TIMEOUT, txs_subscription.next()) .await @@ -193,11 +194,11 @@ impl WsTest for BasicSubscriptions { #[tokio::test] async fn basic_subscriptions() { - test_ws_server(BasicSubscriptions).await; + test_ws_server(BasicSubscriptionsTest).await; } #[derive(Debug)] -struct LogSubscriptions; +struct LogSubscriptionsTest; #[derive(Debug)] struct Subscriptions { @@ -248,7 +249,7 @@ impl Subscriptions { } #[async_trait] -impl WsTest for LogSubscriptions { +impl WsTest for LogSubscriptionsTest { async fn test( &self, client: &WsClient, @@ -314,14 +315,14 @@ async fn collect_logs( #[tokio::test] async fn log_subscriptions() { - test_ws_server(LogSubscriptions).await; + test_ws_server(LogSubscriptionsTest).await; } #[derive(Debug)] -struct LogSubscriptionsWithNewBlock; +struct LogSubscriptionsWithNewBlockTest; #[async_trait] -impl WsTest for LogSubscriptionsWithNewBlock { +impl WsTest for LogSubscriptionsWithNewBlockTest { async fn test( &self, client: &WsClient, @@ -362,14 +363,14 @@ impl WsTest for LogSubscriptionsWithNewBlock { #[tokio::test] async fn log_subscriptions_with_new_block() { - test_ws_server(LogSubscriptionsWithNewBlock).await; + test_ws_server(LogSubscriptionsWithNewBlockTest).await; } #[derive(Debug)] -struct LogSubscriptionsWithManyBlocks; +struct LogSubscriptionsWithManyBlocksTest; #[async_trait] -impl WsTest for LogSubscriptionsWithManyBlocks { +impl WsTest for LogSubscriptionsWithManyBlocksTest { async fn test( &self, client: &WsClient, @@ -408,14 +409,14 @@ impl WsTest for LogSubscriptionsWithManyBlocks { #[tokio::test] async fn log_subscriptions_with_many_new_blocks_at_once() { - test_ws_server(LogSubscriptionsWithManyBlocks).await; + test_ws_server(LogSubscriptionsWithManyBlocksTest).await; } #[derive(Debug)] -struct LogSubscriptionsWithDelay; +struct LogSubscriptionsWithDelayTest; #[async_trait] -impl WsTest for LogSubscriptionsWithDelay { +impl WsTest for LogSubscriptionsWithDelayTest { async fn test( &self, client: &WsClient, @@ -472,14 +473,14 @@ impl WsTest for LogSubscriptionsWithDelay { #[tokio::test] async fn log_subscriptions_with_delay() { - test_ws_server(LogSubscriptionsWithDelay).await; + test_ws_server(LogSubscriptionsWithDelayTest).await; } #[derive(Debug)] -struct RateLimiting; +struct RateLimitingTest; #[async_trait] -impl WsTest for RateLimiting { +impl WsTest for RateLimitingTest { async fn test( &self, client: &WsClient, @@ -509,14 +510,14 @@ impl WsTest for RateLimiting { #[tokio::test] async fn rate_limiting() { - test_ws_server(RateLimiting).await; + test_ws_server(RateLimitingTest).await; } #[derive(Debug)] -struct BatchGetsRateLimited; +struct BatchGetsRateLimitedTest; #[async_trait] -impl WsTest for BatchGetsRateLimited { +impl WsTest for BatchGetsRateLimitedTest { async fn test( &self, client: &WsClient, @@ -553,5 +554,5 @@ impl WsTest for BatchGetsRateLimited { #[tokio::test] async fn batch_rate_limiting() { - test_ws_server(BatchGetsRateLimited).await; + test_ws_server(BatchGetsRateLimitedTest).await; } From ee26dd929a64c8954ab8d0c2ad969f1470cfc593 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Thu, 4 Jan 2024 03:41:03 -0600 Subject: [PATCH 08/49] feat: adds spellchecker to zk tool (#748) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Adds `zk spellcheck` to zk tool for convenient spell checks ## Why ❔ - Adding the tool before including its usage in the CI workflow to ensure its available in published docker container for #681 ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov --- .github/workflows/nodejs-license.yaml | 3 +- infrastructure/zk/package.json | 3 +- infrastructure/zk/src/index.ts | 2 ++ infrastructure/zk/src/spellcheck.ts | 37 ++++++++++++++++++++++ spellcheck/cspell.json | 44 +++++++++++++++++++++++++++ 5 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 infrastructure/zk/src/spellcheck.ts create mode 100644 spellcheck/cspell.json diff --git a/.github/workflows/nodejs-license.yaml b/.github/workflows/nodejs-license.yaml index 5980eaae5449..80a2eb276b3f 100644 --- a/.github/workflows/nodejs-license.yaml +++ b/.github/workflows/nodejs-license.yaml @@ -19,8 +19,9 @@ env: Public Domain; WTFPL; Unlicense; + BlueOak-1.0.0; # It has to be one line, there must be no space between packages. - EXCLUDE_PACKAGES: testrpc@0.0.1;uuid@2.0.1; + EXCLUDE_PACKAGES: testrpc@0.0.1;uuid@2.0.1;@cspell/dict-en-common-misspellings@2.0.0; jobs: generate-matrix: diff --git a/infrastructure/zk/package.json b/infrastructure/zk/package.json index 7e2f70d82cba..c8ed6d63e50f 100644 --- a/infrastructure/zk/package.json +++ b/infrastructure/zk/package.json @@ -31,6 +31,7 @@ "@types/pg": "^8.10.3", "@types/tabtab": "^3.0.1", "hardhat": "=2.16.0", - "typescript": "^4.3.5" + "typescript": "^4.3.5", + "cspell": "^8.2.3" } } diff --git a/infrastructure/zk/src/index.ts b/infrastructure/zk/src/index.ts index 5d65df824d71..ea9ead744f1f 100644 --- a/infrastructure/zk/src/index.ts +++ b/infrastructure/zk/src/index.ts @@ -22,6 +22,7 @@ import { command as db } from './database'; import { command as verifyUpgrade } from './verify-upgrade'; import { proverCommand } from './prover_setup'; import { command as status } from './status'; +import { command as spellcheck } from './spellcheck'; import * as env from './env'; const COMMANDS = [ @@ -48,6 +49,7 @@ const COMMANDS = [ proverCommand, env.command, status, + spellcheck, completion(program as Command) ]; diff --git a/infrastructure/zk/src/spellcheck.ts b/infrastructure/zk/src/spellcheck.ts new file mode 100644 index 000000000000..dfa387921bcb --- /dev/null +++ b/infrastructure/zk/src/spellcheck.ts @@ -0,0 +1,37 @@ +import { Command } from 'commander'; +import * as utils from './utils'; + +export async function runSpellCheck(pattern: string, config: string, useCargo: boolean, useCSpell: boolean) { + const cSpellCommand = `cspell ${pattern} --config=./spellcheck/cspell.json`; + const cargoCommand = `cargo spellcheck --cfg=./spellcheck/era.cfg`; + + try { + if (!useCargo && !useCSpell) { + await Promise.all([utils.spawn(cSpellCommand), utils.spawn(cargoCommand)]); + } else { + // Run cspell if specified + // zk spellcheck --use-cspell + if (useCSpell) { + await utils.spawn(cSpellCommand); + } + + // Run cargo spellcheck if specified + // zk spellcheck --use-cargo + if (useCargo) { + await utils.spawn(cargoCommand); + } + } + } catch (error) { + console.error('Error occurred during spell checking:', error); + } +} + +export const command = new Command('spellcheck') + .option('--pattern ', 'Glob pattern for files to check', 'docs/**/*') + .option('--config ', 'Path to configuration file', './spellcheck/cspell.json') + .option('--use-cargo', 'Use cargo spellcheck') + .option('--use-cspell', 'Use cspell') + .description('Run spell check on specified files') + .action((cmd) => { + runSpellCheck(cmd.pattern, cmd.config, cmd.useCargo, cmd.useCSpell); + }); diff --git a/spellcheck/cspell.json b/spellcheck/cspell.json new file mode 100644 index 000000000000..216ec950fc87 --- /dev/null +++ b/spellcheck/cspell.json @@ -0,0 +1,44 @@ +{ + "language": "en", + "ignorePaths": [ + "node_modules/**", + ".github/**", + ".firebase/**", + ".yarn/**", + "dist/**" + ], + "dictionaries": [ + "typescript", + "cpp", + "npm", + "filetypes", + "cpp", + "en_GB", + "en_US", + "node", + "bash", + "fonts", + "npm", + "cryptocurrencies", + "companies", + "rust", + "html", + "css", + "entities", + "softwareTerms", + "misc", + "fullstack", + "softwareTerms", + "zksync", + "nuxt", + "viem" + ], + "dictionaryDefinitions": [ + { + "name": "zksync", + "addWords": true, + "path": "./era.dic" + } + ], + "allowCompoundWords": true + } \ No newline at end of file From d77e26fa30d36a6c14af2633b2f154c000136876 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Thu, 4 Jan 2024 14:10:09 +0400 Subject: [PATCH 09/49] refactor: Remove generics in eth_watch and eth_sender (#815) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Replaces generic parameters in `eth_watch` and `eth_sender` with `Arc` or `Box`. ## Why ❔ - Prerequisite for ZK Stack framework. ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/lib/eth_client/src/lib.rs | 11 +++++-- .../src/eth_sender/eth_tx_aggregator.rs | 10 +++---- .../src/eth_sender/eth_tx_manager.rs | 11 +++---- .../src/eth_sender/grafana_metrics.rs | 0 core/lib/zksync_core/src/eth_sender/tests.rs | 4 +-- core/lib/zksync_core/src/eth_watch/client.rs | 14 +++++---- .../event_processors/governance_upgrades.rs | 4 +-- .../src/eth_watch/event_processors/mod.rs | 6 ++-- .../event_processors/priority_ops.rs | 4 +-- .../eth_watch/event_processors/upgrades.rs | 4 +-- core/lib/zksync_core/src/eth_watch/mod.rs | 29 ++++++++++--------- core/lib/zksync_core/src/eth_watch/tests.rs | 17 ++++++----- core/lib/zksync_core/src/lib.rs | 6 ++-- sdk/zksync-rs/src/ethereum/mod.rs | 3 +- spellcheck/era.dic | 3 +- 15 files changed, 69 insertions(+), 57 deletions(-) delete mode 100644 core/lib/zksync_core/src/eth_sender/grafana_metrics.rs diff --git a/core/lib/eth_client/src/lib.rs b/core/lib/eth_client/src/lib.rs index 2eb1ea51db81..eeabcba47e0b 100644 --- a/core/lib/eth_client/src/lib.rs +++ b/core/lib/eth_client/src/lib.rs @@ -3,7 +3,7 @@ use std::fmt; use async_trait::async_trait; use zksync_types::{ web3::{ - contract::{tokens::Tokenize, Options}, + contract::Options, ethabi, types::{ Address, Block, BlockId, BlockNumber, Filter, Log, Transaction, TransactionReceipt, @@ -224,13 +224,18 @@ pub trait BoundEthInterface: EthInterface { } /// Encodes a function using the `Self::contract()` ABI. - fn encode_tx_data(&self, func: &str, params: P) -> Vec { + /// + /// `params` are tokenized parameters of the function. Most of the time, you can use + /// [`Tokenize`][tokenize] trait to convert the parameters into tokens. + /// + /// [tokenize]: https://docs.rs/web3/latest/web3/contract/tokens/trait.Tokenize.html + fn encode_tx_data(&self, func: &str, params: Vec) -> Vec { let f = self .contract() .function(func) .expect("failed to get function parameters"); - f.encode_input(¶ms.into_tokens()) + f.encode_input(¶ms) .expect("failed to encode parameters") } } diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs index e8f527c2abeb..b6413509641b 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs @@ -1,4 +1,4 @@ -use std::convert::TryInto; +use std::{convert::TryInto, sync::Arc}; use tokio::sync::watch; use zksync_config::configs::eth_sender::SenderConfig; @@ -42,9 +42,9 @@ pub struct MulticallData { /// Such as CommitBlocks, PublishProofBlocksOnchain and ExecuteBlock /// These eth_txs will be used as a queue for generating signed txs and send them later #[derive(Debug)] -pub struct EthTxAggregator { +pub struct EthTxAggregator { aggregator: Aggregator, - eth_client: E, + eth_client: Arc, config: SenderConfig, timelock_contract_address: Address, l1_multicall3_address: Address, @@ -53,11 +53,11 @@ pub struct EthTxAggregator { base_nonce: u64, } -impl EthTxAggregator { +impl EthTxAggregator { pub fn new( config: SenderConfig, aggregator: Aggregator, - eth_client: E, + eth_client: Arc, timelock_contract_address: Address, l1_multicall3_address: Address, main_zksync_contract_address: Address, diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs index 6000b841f3b8..ae5c8a128298 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs @@ -47,20 +47,17 @@ pub(super) struct L1BlockNumbers { /// Based on eth_tx_history queue the component can mark txs as stuck and create the new attempt /// with higher gas price #[derive(Debug)] -pub struct EthTxManager { - ethereum_gateway: E, +pub struct EthTxManager { + ethereum_gateway: Arc, config: SenderConfig, gas_adjuster: Arc, } -impl EthTxManager -where - E: BoundEthInterface + Sync, -{ +impl EthTxManager { pub fn new( config: SenderConfig, gas_adjuster: Arc, - ethereum_gateway: E, + ethereum_gateway: Arc, ) -> Self { Self { ethereum_gateway, diff --git a/core/lib/zksync_core/src/eth_sender/grafana_metrics.rs b/core/lib/zksync_core/src/eth_sender/grafana_metrics.rs deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index e20f56fb250b..2fae4cda755f 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -30,7 +30,7 @@ use crate::{ }; // Alias to conveniently call static methods of ETHSender. -type MockEthTxManager = EthTxManager>; +type MockEthTxManager = EthTxManager; static DUMMY_OPERATION: Lazy = Lazy::new(|| { AggregatedOperation::Execute(L1BatchExecuteOperation { @@ -53,7 +53,7 @@ struct EthSenderTester { conn: ConnectionPool, gateway: Arc, manager: MockEthTxManager, - aggregator: EthTxAggregator>, + aggregator: EthTxAggregator, gas_adjuster: Arc>>, } diff --git a/core/lib/zksync_core/src/eth_watch/client.rs b/core/lib/zksync_core/src/eth_watch/client.rs index b00d3e1d79de..08e62c3f4ea0 100644 --- a/core/lib/zksync_core/src/eth_watch/client.rs +++ b/core/lib/zksync_core/src/eth_watch/client.rs @@ -1,3 +1,5 @@ +use std::fmt; + use zksync_contracts::verifier_contract; use zksync_eth_client::{CallFunctionArgs, Error as EthClientError, EthInterface}; use zksync_types::{ @@ -30,7 +32,7 @@ impl From for Error { } #[async_trait::async_trait] -pub trait EthClient { +pub trait EthClient: 'static + fmt::Debug + Send + Sync { /// Returns events in a given block range. async fn get_events( &self, @@ -51,8 +53,8 @@ const TOO_MANY_RESULTS_INFURA: &str = "query returned more than"; const TOO_MANY_RESULTS_ALCHEMY: &str = "response size exceeded"; #[derive(Debug)] -pub struct EthHttpQueryClient { - client: E, +pub struct EthHttpQueryClient { + client: Box, topics: Vec, zksync_contract_addr: Address, /// Address of the `Governance` contract. It's optional because it is present only for post-boojum chains. @@ -62,9 +64,9 @@ pub struct EthHttpQueryClient { confirmations_for_eth_event: Option, } -impl EthHttpQueryClient { +impl EthHttpQueryClient { pub fn new( - client: E, + client: Box, zksync_contract_addr: Address, governance_address: Option
, confirmations_for_eth_event: Option, @@ -108,7 +110,7 @@ impl EthHttpQueryClient { } #[async_trait::async_trait] -impl EthClient for EthHttpQueryClient { +impl EthClient for EthHttpQueryClient { async fn scheduler_vk_hash(&self, verifier_address: Address) -> Result { // This is here for backward compatibility with the old verifier: // Legacy verifier returns the full verification key; diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs b/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs index 2f7e2e86b2c7..9bec83d26833 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs @@ -38,11 +38,11 @@ impl GovernanceUpgradesEventProcessor { } #[async_trait::async_trait] -impl EventProcessor for GovernanceUpgradesEventProcessor { +impl EventProcessor for GovernanceUpgradesEventProcessor { async fn process_events( &mut self, storage: &mut StorageProcessor<'_>, - client: &W, + client: &dyn EthClient, events: Vec, ) -> Result<(), Error> { let mut upgrades = Vec::new(); diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/mod.rs b/core/lib/zksync_core/src/eth_watch/event_processors/mod.rs index 202b7efb586a..0a068033f2bd 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/mod.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/mod.rs @@ -1,3 +1,5 @@ +use std::fmt; + use zksync_dal::StorageProcessor; use zksync_types::{web3::types::Log, H256}; @@ -8,12 +10,12 @@ pub mod priority_ops; pub mod upgrades; #[async_trait::async_trait] -pub trait EventProcessor: Send + std::fmt::Debug { +pub trait EventProcessor: 'static + fmt::Debug + Send + Sync { /// Processes given events async fn process_events( &mut self, storage: &mut StorageProcessor<'_>, - client: &W, + client: &dyn EthClient, events: Vec, ) -> Result<(), Error>; diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/priority_ops.rs b/core/lib/zksync_core/src/eth_watch/event_processors/priority_ops.rs index 9c2d34bf589e..ad24eba1791b 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/priority_ops.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/priority_ops.rs @@ -33,11 +33,11 @@ impl PriorityOpsEventProcessor { } #[async_trait::async_trait] -impl EventProcessor for PriorityOpsEventProcessor { +impl EventProcessor for PriorityOpsEventProcessor { async fn process_events( &mut self, storage: &mut StorageProcessor<'_>, - _client: &W, + _client: &dyn EthClient, events: Vec, ) -> Result<(), Error> { let mut priority_ops = Vec::new(); diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs b/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs index 497cb705ee1c..a46b3f21cb53 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs @@ -29,11 +29,11 @@ impl UpgradesEventProcessor { } #[async_trait::async_trait] -impl EventProcessor for UpgradesEventProcessor { +impl EventProcessor for UpgradesEventProcessor { async fn process_events( &mut self, storage: &mut StorageProcessor<'_>, - client: &W, + client: &dyn EthClient, events: Vec, ) -> Result<(), Error> { let mut upgrades = Vec::new(); diff --git a/core/lib/zksync_core/src/eth_watch/mod.rs b/core/lib/zksync_core/src/eth_watch/mod.rs index 69f447e7fba9..5aac8624d474 100644 --- a/core/lib/zksync_core/src/eth_watch/mod.rs +++ b/core/lib/zksync_core/src/eth_watch/mod.rs @@ -39,32 +39,32 @@ struct EthWatchState { } #[derive(Debug)] -pub struct EthWatch { - client: W, +pub struct EthWatch { + client: Box, poll_interval: Duration, - event_processors: Vec>>, + event_processors: Vec>, last_processed_ethereum_block: u64, } -impl EthWatch { +impl EthWatch { pub async fn new( diamond_proxy_address: Address, governance_contract: Option, - mut client: W, + mut client: Box, pool: &ConnectionPool, poll_interval: Duration, ) -> Self { let mut storage = pool.access_storage_tagged("eth_watch").await.unwrap(); - let state = Self::initialize_state(&client, &mut storage).await; + let state = Self::initialize_state(&*client, &mut storage).await; tracing::info!("initialized state: {:?}", state); let priority_ops_processor = PriorityOpsEventProcessor::new(state.next_expected_priority_id); let upgrades_processor = UpgradesEventProcessor::new(state.last_seen_version_id); - let mut event_processors: Vec>> = vec![ + let mut event_processors: Vec> = vec![ Box::new(priority_ops_processor), Box::new(upgrades_processor), ]; @@ -92,7 +92,10 @@ impl EthWatch { } } - async fn initialize_state(client: &W, storage: &mut StorageProcessor<'_>) -> EthWatchState { + async fn initialize_state( + client: &dyn EthClient, + storage: &mut StorageProcessor<'_>, + ) -> EthWatchState { let next_expected_priority_id: PriorityOpId = storage .transactions_dal() .last_priority_id() @@ -149,7 +152,7 @@ impl EthWatch { // thus entering priority mode, which is not desired. tracing::error!("Failed to process new blocks {}", error); self.last_processed_ethereum_block = - Self::initialize_state(&self.client, &mut storage) + Self::initialize_state(&*self.client, &mut storage) .await .last_processed_ethereum_block; } @@ -177,7 +180,7 @@ impl EthWatch { for processor in self.event_processors.iter_mut() { processor - .process_events(storage, &self.client, events.clone()) + .process_events(storage, &*self.client, events.clone()) .await?; } self.last_processed_ethereum_block = to_block; @@ -185,10 +188,10 @@ impl EthWatch { } } -pub async fn start_eth_watch( +pub async fn start_eth_watch( config: ETHWatchConfig, pool: ConnectionPool, - eth_gateway: E, + eth_gateway: Box, diamond_proxy_addr: Address, governance: (Contract, Address), stop_receiver: watch::Receiver, @@ -203,7 +206,7 @@ pub async fn start_eth_watch( let mut eth_watch = EthWatch::new( diamond_proxy_addr, Some(governance.0), - eth_client, + Box::new(eth_client), &pool, config.poll_interval(), ) diff --git a/core/lib/zksync_core/src/eth_watch/tests.rs b/core/lib/zksync_core/src/eth_watch/tests.rs index 31c467419294..d606a15107ca 100644 --- a/core/lib/zksync_core/src/eth_watch/tests.rs +++ b/core/lib/zksync_core/src/eth_watch/tests.rs @@ -17,6 +17,7 @@ use crate::eth_watch::{ client::EthClient, event_processors::upgrades::UPGRADE_PROPOSAL_SIGNATURE, EthWatch, }; +#[derive(Debug)] struct FakeEthClientData { transactions: HashMap>, diamond_upgrades: HashMap>, @@ -67,7 +68,7 @@ impl FakeEthClientData { } } -#[derive(Clone)] +#[derive(Debug, Clone)] struct FakeEthClient { inner: Arc>, } @@ -208,7 +209,7 @@ async fn test_normal_operation_l1_txs() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -256,7 +257,7 @@ async fn test_normal_operation_upgrades() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -317,7 +318,7 @@ async fn test_gap_in_upgrades() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -356,7 +357,7 @@ async fn test_normal_operation_governance_upgrades() { let mut watcher = EthWatch::new( Address::default(), Some(governance_contract()), - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -418,7 +419,7 @@ async fn test_gap_in_single_batch() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -448,7 +449,7 @@ async fn test_gap_between_batches() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) @@ -483,7 +484,7 @@ async fn test_overlapping_batches() { let mut watcher = EthWatch::new( Address::default(), None, - client.clone(), + Box::new(client.clone()), &connection_pool, std::time::Duration::from_nanos(1), ) diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index cdb564942430..851b281bf8e3 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -521,7 +521,7 @@ pub async fn initialize_components( start_eth_watch( eth_watch_config, eth_watch_pool, - query_client.clone(), + Box::new(query_client.clone()), main_zksync_contract_address, governance, stop_receiver.clone(), @@ -555,7 +555,7 @@ pub async fn initialize_components( eth_sender.sender.clone(), store_factory.create_store().await, ), - eth_client, + Arc::new(eth_client), contracts_config.validator_timelock_addr, contracts_config.l1_multicall3_addr, main_zksync_contract_address, @@ -588,7 +588,7 @@ pub async fn initialize_components( .get_or_init() .await .context("gas_adjuster.get_or_init()")?, - eth_client, + Arc::new(eth_client), ); task_futures.extend([tokio::spawn( eth_tx_manager_actor.run(eth_manager_pool, stop_receiver.clone()), diff --git a/sdk/zksync-rs/src/ethereum/mod.rs b/sdk/zksync-rs/src/ethereum/mod.rs index 74d866ee83ba..a21c7727cb1f 100644 --- a/sdk/zksync-rs/src/ethereum/mod.rs +++ b/sdk/zksync-rs/src/ethereum/mod.rs @@ -408,7 +408,8 @@ impl EthereumProvider { L1_TO_L2_GAS_PER_PUBDATA, factory_deps, refund_recipient, - ), + ) + .into_tokens(), ); let tx = self diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 9c0f6bb87401..a8ad97cc859d 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -608,4 +608,5 @@ codebase compactions M6 compiler_common -noop \ No newline at end of file +noop +tokenized \ No newline at end of file From 61712a636f69be70d75719c04f364d679ef624e0 Mon Sep 17 00:00:00 2001 From: Stanislav Bezkorovainyi Date: Thu, 4 Jan 2024 14:40:26 +0100 Subject: [PATCH 10/49] feat(vm): Separate boojum integration vm (#806) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Changes needed to make the boojum integration VM work as a separate VM version from "latest" ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/lib/multivm/src/glue/history_mode.rs | 22 +- core/lib/multivm/src/glue/tracers/mod.rs | 33 ++- core/lib/multivm/src/lib.rs | 3 +- .../multivm/src/tracers/call_tracer/mod.rs | 1 + .../call_tracer/vm_boojum_integration/mod.rs | 216 ++++++++++++++++++ .../multivm/src/tracers/multivm_dispatcher.rs | 16 +- .../src/tracers/storage_invocation/mod.rs | 1 + .../vm_boojum_integration/mod.rs | 35 +++ core/lib/multivm/src/tracers/validator/mod.rs | 1 + .../validator/vm_boojum_integration/mod.rs | 201 ++++++++++++++++ .../src/tracers/validator/vm_latest/mod.rs | 10 +- core/lib/multivm/src/versions/mod.rs | 1 + .../bootloader_state/l2_block.rs | 2 +- .../bootloader_state/state.rs | 2 +- .../bootloader_state/tx.rs | 2 +- .../bootloader_state/utils.rs | 2 +- .../vm_boojum_integration/constants.rs | 2 +- .../implementation/bytecode.rs | 2 +- .../implementation/execution.rs | 2 +- .../implementation/gas.rs | 2 +- .../implementation/logs.rs | 4 +- .../implementation/snapshots.rs | 10 +- .../implementation/statistics.rs | 2 +- .../implementation/tx.rs | 2 +- .../src/versions/vm_boojum_integration/mod.rs | 2 - .../old_vm/event_sink.rs | 2 +- .../old_vm/history_recorder.rs | 2 +- .../vm_boojum_integration/old_vm/memory.rs | 2 +- .../old_vm/oracles/decommitter.rs | 2 +- .../old_vm/oracles/precompile.rs | 4 +- .../old_vm/oracles/storage.rs | 2 +- .../vm_boojum_integration/old_vm/utils.rs | 2 +- .../vm_boojum_integration/oracles/storage.rs | 2 +- .../vm_boojum_integration/tests/bootloader.rs | 2 +- .../tests/bytecode_publishing.rs | 2 +- .../tests/call_tracer.rs | 2 +- .../vm_boojum_integration/tests/default_aa.rs | 2 +- .../vm_boojum_integration/tests/gas_limit.rs | 2 +- .../tests/get_used_contracts.rs | 2 +- .../tests/invalid_bytecode.rs | 6 +- .../tests/is_write_initial.rs | 2 +- .../tests/l1_tx_execution.rs | 2 +- .../vm_boojum_integration/tests/l2_blocks.rs | 2 +- .../tests/nonce_holder.rs | 2 +- .../vm_boojum_integration/tests/refunds.rs | 2 +- .../tests/require_eip712.rs | 2 +- .../vm_boojum_integration/tests/rollbacks.rs | 2 +- .../tests/simple_execution.rs | 2 +- .../tests/tester/inner_state.rs | 2 +- .../tests/tester/transaction_test_info.rs | 2 +- .../tests/tester/vm_tester.rs | 2 +- .../tests/tracing_execution_error.rs | 2 +- .../vm_boojum_integration/tests/upgrade.rs | 2 +- .../vm_boojum_integration/tests/utils.rs | 2 +- .../tracers/default_tracers.rs | 2 +- .../tracers/dispatcher.rs | 2 +- .../tracers/pubdata_tracer.rs | 2 +- .../vm_boojum_integration/tracers/refunds.rs | 4 +- .../tracers/result_tracer.rs | 2 +- .../vm_boojum_integration/tracers/traits.rs | 2 +- .../vm_boojum_integration/tracers/utils.rs | 2 +- .../types/internals/snapshot.rs | 2 +- .../types/internals/transaction_data.rs | 2 +- .../types/internals/vm_state.rs | 2 +- .../vm_boojum_integration/utils/fee.rs | 2 +- .../vm_boojum_integration/utils/logs.rs | 2 +- .../vm_boojum_integration/utils/overhead.rs | 2 +- .../utils/transaction_encoding.rs | 2 +- .../src/versions/vm_boojum_integration/vm.rs | 6 +- .../vm_latest/implementation/execution.rs | 23 +- .../versions/vm_latest/implementation/gas.rs | 2 +- .../vm_latest/implementation/statistics.rs | 2 +- .../vm_latest/tests/tester/inner_state.rs | 2 +- core/lib/multivm/src/versions/vm_latest/vm.rs | 4 +- core/lib/multivm/src/vm_instance.rs | 5 +- .../api_server/execution_sandbox/tracers.rs | 2 +- 76 files changed, 613 insertions(+), 103 deletions(-) create mode 100644 core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs create mode 100644 core/lib/multivm/src/tracers/storage_invocation/vm_boojum_integration/mod.rs create mode 100644 core/lib/multivm/src/tracers/validator/vm_boojum_integration/mod.rs diff --git a/core/lib/multivm/src/glue/history_mode.rs b/core/lib/multivm/src/glue/history_mode.rs index ca56836d8e82..40b33ec32bf6 100644 --- a/core/lib/multivm/src/glue/history_mode.rs +++ b/core/lib/multivm/src/glue/history_mode.rs @@ -7,12 +7,14 @@ pub trait HistoryMode: + GlueInto + GlueInto + GlueInto + + GlueInto { type VmM6Mode: crate::vm_m6::HistoryMode; type Vm1_3_2Mode: crate::vm_1_3_2::HistoryMode; type VmVirtualBlocksMode: crate::vm_virtual_blocks::HistoryMode; type VmVirtualBlocksRefundsEnhancement: crate::vm_refunds_enhancement::HistoryMode; - type VmBoojumIntegration: crate::vm_latest::HistoryMode; + type VmBoojumIntegration: crate::vm_boojum_integration::HistoryMode; + type VmLatest: crate::vm_latest::HistoryMode; } impl GlueFrom for crate::vm_m6::HistoryEnabled { @@ -39,6 +41,12 @@ impl GlueFrom for crate::vm_refunds_enhancemen } } +impl GlueFrom for crate::vm_boojum_integration::HistoryEnabled { + fn glue_from(_: crate::vm_latest::HistoryEnabled) -> Self { + Self + } +} + impl GlueFrom for crate::vm_m6::HistoryDisabled { fn glue_from(_: crate::vm_latest::HistoryDisabled) -> Self { Self @@ -65,12 +73,19 @@ impl GlueFrom } } +impl GlueFrom for crate::vm_boojum_integration::HistoryDisabled { + fn glue_from(_: crate::vm_latest::HistoryDisabled) -> Self { + Self + } +} + impl HistoryMode for crate::vm_latest::HistoryEnabled { type VmM6Mode = crate::vm_m6::HistoryEnabled; type Vm1_3_2Mode = crate::vm_1_3_2::HistoryEnabled; type VmVirtualBlocksMode = crate::vm_virtual_blocks::HistoryEnabled; type VmVirtualBlocksRefundsEnhancement = crate::vm_refunds_enhancement::HistoryEnabled; - type VmBoojumIntegration = crate::vm_latest::HistoryEnabled; + type VmBoojumIntegration = crate::vm_boojum_integration::HistoryEnabled; + type VmLatest = crate::vm_latest::HistoryEnabled; } impl HistoryMode for crate::vm_latest::HistoryDisabled { @@ -78,5 +93,6 @@ impl HistoryMode for crate::vm_latest::HistoryDisabled { type Vm1_3_2Mode = crate::vm_1_3_2::HistoryDisabled; type VmVirtualBlocksMode = crate::vm_virtual_blocks::HistoryDisabled; type VmVirtualBlocksRefundsEnhancement = crate::vm_refunds_enhancement::HistoryDisabled; - type VmBoojumIntegration = crate::vm_latest::HistoryDisabled; + type VmBoojumIntegration = crate::vm_boojum_integration::HistoryDisabled; + type VmLatest = crate::vm_latest::HistoryDisabled; } diff --git a/core/lib/multivm/src/glue/tracers/mod.rs b/core/lib/multivm/src/glue/tracers/mod.rs index c58e717a646a..860aaf6de330 100644 --- a/core/lib/multivm/src/glue/tracers/mod.rs +++ b/core/lib/multivm/src/glue/tracers/mod.rs @@ -37,7 +37,10 @@ use crate::HistoryMode; pub type MultiVmTracerPointer = Box>; pub trait MultiVMTracer: - IntoLatestTracer + IntoVmVirtualBlocksTracer + IntoVmRefundsEnhancementTracer + IntoLatestTracer + + IntoVmVirtualBlocksTracer + + IntoVmRefundsEnhancementTracer + + IntoVmBoojumIntegrationTracer { fn into_tracer_pointer(self) -> MultiVmTracerPointer where @@ -48,7 +51,7 @@ pub trait MultiVMTracer: } pub trait IntoLatestTracer { - fn latest(&self) -> crate::vm_latest::TracerPointer; + fn latest(&self) -> crate::vm_latest::TracerPointer; } pub trait IntoVmVirtualBlocksTracer { @@ -63,13 +66,19 @@ pub trait IntoVmRefundsEnhancementTracer { ) -> Box>; } +pub trait IntoVmBoojumIntegrationTracer { + fn vm_boojum_integration( + &self, + ) -> Box>; +} + impl IntoLatestTracer for T where S: WriteStorage, H: HistoryMode, - T: crate::vm_latest::VmTracer + Clone + 'static, + T: crate::vm_latest::VmTracer + Clone + 'static, { - fn latest(&self) -> crate::vm_latest::TracerPointer { + fn latest(&self) -> crate::vm_latest::TracerPointer { Box::new(self.clone()) } } @@ -103,12 +112,26 @@ where } } +impl IntoVmBoojumIntegrationTracer for T +where + S: WriteStorage, + H: HistoryMode, + T: crate::vm_boojum_integration::VmTracer + Clone + 'static, +{ + fn vm_boojum_integration( + &self, + ) -> Box> { + Box::new(self.clone()) + } +} + impl MultiVMTracer for T where S: WriteStorage, H: HistoryMode, T: IntoLatestTracer + IntoVmVirtualBlocksTracer - + IntoVmRefundsEnhancementTracer, + + IntoVmRefundsEnhancementTracer + + IntoVmBoojumIntegrationTracer, { } diff --git a/core/lib/multivm/src/lib.rs b/core/lib/multivm/src/lib.rs index 23ea80a68602..a13d7cb7d14a 100644 --- a/core/lib/multivm/src/lib.rs +++ b/core/lib/multivm/src/lib.rs @@ -9,7 +9,8 @@ pub use zk_evm_1_4_0; pub use zksync_types::vm_version::VmVersion; pub use self::versions::{ - vm_1_3_2, vm_latest, vm_m5, vm_m6, vm_refunds_enhancement, vm_virtual_blocks, + vm_1_3_2, vm_boojum_integration, vm_latest, vm_m5, vm_m6, vm_refunds_enhancement, + vm_virtual_blocks, }; pub use crate::{ glue::{ diff --git a/core/lib/multivm/src/tracers/call_tracer/mod.rs b/core/lib/multivm/src/tracers/call_tracer/mod.rs index 0c7e4d3c2800..6d0285fb97d2 100644 --- a/core/lib/multivm/src/tracers/call_tracer/mod.rs +++ b/core/lib/multivm/src/tracers/call_tracer/mod.rs @@ -6,6 +6,7 @@ use zksync_types::vm_trace::Call; use crate::tracers::call_tracer::metrics::CALL_METRICS; mod metrics; +pub mod vm_boojum_integration; pub mod vm_latest; pub mod vm_refunds_enhancement; pub mod vm_virtual_blocks; diff --git a/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs b/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs new file mode 100644 index 000000000000..8b6742da4619 --- /dev/null +++ b/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs @@ -0,0 +1,216 @@ +use zk_evm_1_4_0::{ + tracing::{AfterExecutionData, VmLocalStateData}, + zkevm_opcode_defs::{ + FarCallABI, FatPointer, Opcode, RetOpcode, CALL_IMPLICIT_CALLDATA_FAT_PTR_REGISTER, + RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER, + }, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_system_constants::CONTRACT_DEPLOYER_ADDRESS; +use zksync_types::{ + vm_trace::{Call, CallType}, + FarCallOpcode, U256, +}; + +use crate::{ + interface::{ + tracer::VmExecutionStopReason, traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, + VmRevertReason, + }, + tracers::call_tracer::CallTracer, + vm_boojum_integration::{BootloaderState, HistoryMode, SimpleMemory, VmTracer, ZkSyncVmState}, +}; + +impl DynTracer> for CallTracer { + fn after_execution( + &mut self, + state: VmLocalStateData<'_>, + data: AfterExecutionData, + memory: &SimpleMemory, + _storage: StoragePtr, + ) { + match data.opcode.variant.opcode { + Opcode::NearCall(_) => { + self.increase_near_call_count(); + } + Opcode::FarCall(far_call) => { + // We use parent gas for properly calculating gas used in the trace. + let current_ergs = state.vm_local_state.callstack.current.ergs_remaining; + let parent_gas = state + .vm_local_state + .callstack + .inner + .last() + .map(|call| call.ergs_remaining + current_ergs) + .unwrap_or(current_ergs); + + let mut current_call = Call { + r#type: CallType::Call(far_call), + gas: 0, + parent_gas, + ..Default::default() + }; + + self.handle_far_call_op_code_vm_boojum_integration( + state, + memory, + &mut current_call, + ); + self.push_call_and_update_stats(current_call, 0); + } + Opcode::Ret(ret_code) => { + self.handle_ret_op_code_vm_boojum_integration(state, memory, ret_code); + } + _ => {} + }; + } +} + +impl VmTracer for CallTracer { + fn after_vm_execution( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &BootloaderState, + _stop_reason: VmExecutionStopReason, + ) { + self.store_result() + } +} + +impl CallTracer { + fn handle_far_call_op_code_vm_boojum_integration( + &mut self, + state: VmLocalStateData<'_>, + memory: &SimpleMemory, + current_call: &mut Call, + ) { + let current = state.vm_local_state.callstack.current; + // All calls from the actual users are mimic calls, + // so we need to check that the previous call was to the deployer. + // Actually it's a call of the constructor. + // And at this stage caller is user and callee is deployed contract. + let call_type = if let CallType::Call(far_call) = current_call.r#type { + if matches!(far_call, FarCallOpcode::Mimic) { + let previous_caller = state + .vm_local_state + .callstack + .inner + .last() + .map(|call| call.this_address) + // Actually it's safe to just unwrap here, because we have at least one call in the stack + // But i want to be sure that we will not have any problems in the future + .unwrap_or(current.this_address); + if previous_caller == CONTRACT_DEPLOYER_ADDRESS { + CallType::Create + } else { + CallType::Call(far_call) + } + } else { + CallType::Call(far_call) + } + } else { + unreachable!() + }; + let calldata = if current.code_page.0 == 0 || current.ergs_remaining == 0 { + vec![] + } else { + let packed_abi = + state.vm_local_state.registers[CALL_IMPLICIT_CALLDATA_FAT_PTR_REGISTER as usize]; + assert!(packed_abi.is_pointer); + let far_call_abi = FarCallABI::from_u256(packed_abi.value); + memory.read_unaligned_bytes( + far_call_abi.memory_quasi_fat_pointer.memory_page as usize, + far_call_abi.memory_quasi_fat_pointer.start as usize, + far_call_abi.memory_quasi_fat_pointer.length as usize, + ) + }; + + current_call.input = calldata; + current_call.r#type = call_type; + current_call.from = current.msg_sender; + current_call.to = current.this_address; + current_call.value = U256::from(current.context_u128_value); + current_call.gas = current.ergs_remaining; + } + + fn save_output_vm_boojum_integration( + &mut self, + state: VmLocalStateData<'_>, + memory: &SimpleMemory, + ret_opcode: RetOpcode, + current_call: &mut Call, + ) { + let fat_data_pointer = + state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; + + // if fat_data_pointer is not a pointer then there is no output + let output = if fat_data_pointer.is_pointer { + let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); + if !fat_data_pointer.is_trivial() { + Some(memory.read_unaligned_bytes( + fat_data_pointer.memory_page as usize, + fat_data_pointer.start as usize, + fat_data_pointer.length as usize, + )) + } else { + None + } + } else { + None + }; + + match ret_opcode { + RetOpcode::Ok => { + current_call.output = output.unwrap_or_default(); + } + RetOpcode::Revert => { + if let Some(output) = output { + current_call.revert_reason = + Some(VmRevertReason::from(output.as_slice()).to_string()); + } else { + current_call.revert_reason = Some("Unknown revert reason".to_string()); + } + } + RetOpcode::Panic => { + current_call.error = Some("Panic".to_string()); + } + } + } + + fn handle_ret_op_code_vm_boojum_integration( + &mut self, + state: VmLocalStateData<'_>, + memory: &SimpleMemory, + ret_opcode: RetOpcode, + ) { + let Some(mut current_call) = self.stack.pop() else { + return; + }; + + if current_call.near_calls_after > 0 { + current_call.near_calls_after -= 1; + self.push_call_and_update_stats(current_call.farcall, current_call.near_calls_after); + return; + } + + current_call.farcall.gas_used = current_call + .farcall + .parent_gas + .saturating_sub(state.vm_local_state.callstack.current.ergs_remaining); + + self.save_output_vm_boojum_integration( + state, + memory, + ret_opcode, + &mut current_call.farcall, + ); + + // If there is a parent call, push the current call to it + // Otherwise, push the current call to the stack, because it's the top level call + if let Some(parent_call) = self.stack.last_mut() { + parent_call.farcall.calls.push(current_call.farcall); + } else { + self.push_call_and_update_stats(current_call.farcall, current_call.near_calls_after); + } + } +} diff --git a/core/lib/multivm/src/tracers/multivm_dispatcher.rs b/core/lib/multivm/src/tracers/multivm_dispatcher.rs index 8ee858a61108..c7cbbf7f19bc 100644 --- a/core/lib/multivm/src/tracers/multivm_dispatcher.rs +++ b/core/lib/multivm/src/tracers/multivm_dispatcher.rs @@ -30,13 +30,27 @@ impl Default for TracerDispatcher { } impl From> - for crate::vm_latest::TracerDispatcher + for crate::vm_latest::TracerDispatcher { fn from(value: TracerDispatcher) -> Self { Self::new(value.tracers.into_iter().map(|x| x.latest()).collect()) } } +impl From> + for crate::vm_boojum_integration::TracerDispatcher +{ + fn from(value: TracerDispatcher) -> Self { + Self::new( + value + .tracers + .into_iter() + .map(|x| x.vm_boojum_integration()) + .collect(), + ) + } +} + impl From> for crate::vm_refunds_enhancement::TracerDispatcher { diff --git a/core/lib/multivm/src/tracers/storage_invocation/mod.rs b/core/lib/multivm/src/tracers/storage_invocation/mod.rs index 3816d2a07a14..f48534709add 100644 --- a/core/lib/multivm/src/tracers/storage_invocation/mod.rs +++ b/core/lib/multivm/src/tracers/storage_invocation/mod.rs @@ -1,3 +1,4 @@ +pub mod vm_boojum_integration; pub mod vm_latest; pub mod vm_refunds_enhancement; pub mod vm_virtual_blocks; diff --git a/core/lib/multivm/src/tracers/storage_invocation/vm_boojum_integration/mod.rs b/core/lib/multivm/src/tracers/storage_invocation/vm_boojum_integration/mod.rs new file mode 100644 index 000000000000..05651485bd79 --- /dev/null +++ b/core/lib/multivm/src/tracers/storage_invocation/vm_boojum_integration/mod.rs @@ -0,0 +1,35 @@ +use zksync_state::WriteStorage; + +use crate::{ + interface::{ + tracer::{TracerExecutionStatus, TracerExecutionStopReason}, + traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, + Halt, + }, + tracers::storage_invocation::StorageInvocations, + vm_boojum_integration::{BootloaderState, HistoryMode, SimpleMemory, VmTracer, ZkSyncVmState}, +}; + +impl DynTracer> for StorageInvocations {} + +impl VmTracer for StorageInvocations { + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + let current = state + .storage + .storage + .get_ptr() + .borrow() + .missed_storage_invocations(); + + if current >= self.limit { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Abort( + Halt::TracerCustom("Storage invocations limit reached".to_string()), + )); + } + TracerExecutionStatus::Continue + } +} diff --git a/core/lib/multivm/src/tracers/validator/mod.rs b/core/lib/multivm/src/tracers/validator/mod.rs index 0f43f235ade4..36d767dac632 100644 --- a/core/lib/multivm/src/tracers/validator/mod.rs +++ b/core/lib/multivm/src/tracers/validator/mod.rs @@ -16,6 +16,7 @@ use crate::tracers::validator::types::{NewTrustedValidationItems, ValidationTrac pub use crate::tracers::validator::types::{ValidationError, ValidationTracerParams}; mod types; +mod vm_boojum_integration; mod vm_latest; mod vm_refunds_enhancement; mod vm_virtual_blocks; diff --git a/core/lib/multivm/src/tracers/validator/vm_boojum_integration/mod.rs b/core/lib/multivm/src/tracers/validator/vm_boojum_integration/mod.rs new file mode 100644 index 000000000000..2c9a708abcaa --- /dev/null +++ b/core/lib/multivm/src/tracers/validator/vm_boojum_integration/mod.rs @@ -0,0 +1,201 @@ +use zk_evm_1_4_0::{ + tracing::{BeforeExecutionData, VmLocalStateData}, + zkevm_opcode_defs::{ContextOpcode, FarCallABI, LogOpcode, Opcode}, +}; +use zksync_state::{StoragePtr, WriteStorage}; +use zksync_system_constants::KECCAK256_PRECOMPILE_ADDRESS; +use zksync_types::{ + get_code_key, vm_trace::ViolatedValidationRule, AccountTreeId, StorageKey, H256, +}; +use zksync_utils::{h256_to_account_address, u256_to_account_address, u256_to_h256}; + +use crate::{ + interface::{ + traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, + types::tracer::{TracerExecutionStatus, TracerExecutionStopReason}, + Halt, + }, + tracers::validator::{ + types::{NewTrustedValidationItems, ValidationTracerMode}, + ValidationRoundResult, ValidationTracer, + }, + vm_boojum_integration::{ + tracers::utils::{ + computational_gas_price, get_calldata_page_via_abi, print_debug_if_needed, VmHook, + }, + BootloaderState, SimpleMemory, VmTracer, ZkSyncVmState, + }, + HistoryMode, +}; + +impl ValidationTracer { + fn check_user_restrictions_vm_boojum_integration( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + memory: &SimpleMemory, + storage: StoragePtr, + ) -> ValidationRoundResult { + if self.computational_gas_used > self.computational_gas_limit { + return Err(ViolatedValidationRule::TookTooManyComputationalGas( + self.computational_gas_limit, + )); + } + + let opcode_variant = data.opcode.variant; + match opcode_variant.opcode { + Opcode::FarCall(_) => { + let packed_abi = data.src0_value.value; + let call_destination_value = data.src1_value.value; + + let called_address = u256_to_account_address(&call_destination_value); + let far_call_abi = FarCallABI::from_u256(packed_abi); + + if called_address == KECCAK256_PRECOMPILE_ADDRESS + && far_call_abi.memory_quasi_fat_pointer.length == 64 + { + let calldata_page = get_calldata_page_via_abi( + &far_call_abi, + state.vm_local_state.callstack.current.base_memory_page, + ); + let calldata = memory.read_unaligned_bytes( + calldata_page as usize, + far_call_abi.memory_quasi_fat_pointer.start as usize, + 64, + ); + + let slot_to_add = + self.slot_to_add_from_keccak_call(&calldata, self.user_address); + + if let Some(slot) = slot_to_add { + return Ok(NewTrustedValidationItems { + new_allowed_slots: vec![slot], + ..Default::default() + }); + } + } else if called_address != self.user_address { + let code_key = get_code_key(&called_address); + let code = storage.borrow_mut().read_value(&code_key); + + if code == H256::zero() { + // The users are not allowed to call contracts with no code + return Err(ViolatedValidationRule::CalledContractWithNoCode( + called_address, + )); + } + } + } + Opcode::Context(context) => { + match context { + ContextOpcode::Meta => { + return Err(ViolatedValidationRule::TouchedUnallowedContext); + } + ContextOpcode::ErgsLeft => { + // TODO (SMA-1168): implement the correct restrictions for the gas left opcode. + } + _ => {} + } + } + Opcode::Log(LogOpcode::StorageRead) => { + let key = data.src0_value.value; + let this_address = state.vm_local_state.callstack.current.this_address; + let msg_sender = state.vm_local_state.callstack.current.msg_sender; + + if !self.is_allowed_storage_read(storage.clone(), this_address, key, msg_sender) { + return Err(ViolatedValidationRule::TouchedUnallowedStorageSlots( + this_address, + key, + )); + } + + if self.trusted_address_slots.contains(&(this_address, key)) { + let storage_key = + StorageKey::new(AccountTreeId::new(this_address), u256_to_h256(key)); + + let value = storage.borrow_mut().read_value(&storage_key); + + return Ok(NewTrustedValidationItems { + new_trusted_addresses: vec![h256_to_account_address(&value)], + ..Default::default() + }); + } + } + _ => {} + } + + Ok(Default::default()) + } +} + +impl DynTracer> + for ValidationTracer +{ + fn before_execution( + &mut self, + state: VmLocalStateData<'_>, + data: BeforeExecutionData, + memory: &SimpleMemory, + storage: StoragePtr, + ) { + // For now, we support only validations for users. + if let ValidationTracerMode::UserTxValidation = self.validation_mode { + self.computational_gas_used = self + .computational_gas_used + .saturating_add(computational_gas_price(state, &data)); + + let validation_round_result = + self.check_user_restrictions_vm_boojum_integration(state, data, memory, storage); + self.process_validation_round_result(validation_round_result); + } + + let hook = VmHook::from_opcode_memory(&state, &data); + print_debug_if_needed(&hook, &state, memory); + + let current_mode = self.validation_mode; + match (current_mode, hook) { + (ValidationTracerMode::NoValidation, VmHook::AccountValidationEntered) => { + // Account validation can be entered when there is no prior validation (i.e. "nested" validations are not allowed) + self.validation_mode = ValidationTracerMode::UserTxValidation; + } + (ValidationTracerMode::NoValidation, VmHook::PaymasterValidationEntered) => { + // Paymaster validation can be entered when there is no prior validation (i.e. "nested" validations are not allowed) + self.validation_mode = ValidationTracerMode::PaymasterTxValidation; + } + (_, VmHook::AccountValidationEntered | VmHook::PaymasterValidationEntered) => { + panic!( + "Unallowed transition inside the validation tracer. Mode: {:#?}, hook: {:#?}", + self.validation_mode, hook + ); + } + (_, VmHook::NoValidationEntered) => { + // Validation can be always turned off + self.validation_mode = ValidationTracerMode::NoValidation; + } + (_, VmHook::ValidationStepEndeded) => { + // The validation step has ended. + self.should_stop_execution = true; + } + (_, _) => { + // The hook is not relevant to the validation tracer. Ignore. + } + } + } +} + +impl VmTracer for ValidationTracer { + fn finish_cycle( + &mut self, + _state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + if self.should_stop_execution { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Finish); + } + if let Some(result) = self.result.get() { + return TracerExecutionStatus::Stop(TracerExecutionStopReason::Abort( + Halt::TracerCustom(format!("Validation error: {:#?}", result)), + )); + } + TracerExecutionStatus::Continue + } +} diff --git a/core/lib/multivm/src/tracers/validator/vm_latest/mod.rs b/core/lib/multivm/src/tracers/validator/vm_latest/mod.rs index dbdc7253f75d..f77a70041c50 100644 --- a/core/lib/multivm/src/tracers/validator/vm_latest/mod.rs +++ b/core/lib/multivm/src/tracers/validator/vm_latest/mod.rs @@ -33,7 +33,7 @@ impl ValidationTracer { &mut self, state: VmLocalStateData<'_>, data: BeforeExecutionData, - memory: &SimpleMemory, + memory: &SimpleMemory, storage: StoragePtr, ) -> ValidationRoundResult { if self.computational_gas_used > self.computational_gas_limit { @@ -127,14 +127,14 @@ impl ValidationTracer { } } -impl DynTracer> +impl DynTracer> for ValidationTracer { fn before_execution( &mut self, state: VmLocalStateData<'_>, data: BeforeExecutionData, - memory: &SimpleMemory, + memory: &SimpleMemory, storage: StoragePtr, ) { // For now, we support only validations for users. @@ -182,10 +182,10 @@ impl DynTracer VmTracer for ValidationTracer { +impl VmTracer for ValidationTracer { fn finish_cycle( &mut self, - _state: &mut ZkSyncVmState, + _state: &mut ZkSyncVmState, _bootloader_state: &mut BootloaderState, ) -> TracerExecutionStatus { if self.should_stop_execution { diff --git a/core/lib/multivm/src/versions/mod.rs b/core/lib/multivm/src/versions/mod.rs index 71379f6df5c4..0fc9111aa9a2 100644 --- a/core/lib/multivm/src/versions/mod.rs +++ b/core/lib/multivm/src/versions/mod.rs @@ -1,4 +1,5 @@ pub mod vm_1_3_2; +pub mod vm_boojum_integration; pub mod vm_latest; pub mod vm_m5; pub mod vm_m6; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs index 146e8713c69e..a1bd6323b848 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs @@ -5,7 +5,7 @@ use zksync_utils::concat_and_hash; use crate::{ interface::{L2Block, L2BlockEnv}, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::{snapshot::L2BlockSnapshot, tx::BootloaderTx}, utils::l2_blocks::l2_block_hash, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs index d914aacab178..255dc6aaebb9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs @@ -7,7 +7,7 @@ use zksync_utils::bytecode::CompressedBytecodeInfo; use super::{tx::BootloaderTx, utils::apply_pubdata_to_memory}; use crate::{ interface::{BootloaderMemory, L2BlockEnv, TxExecutionMode}, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::{ l2_block::BootloaderL2Block, snapshot::BootloaderStateSnapshot, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs index 21aee75b38b5..3030427281bf 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/tx.rs @@ -1,7 +1,7 @@ use zksync_types::{L2ChainId, H256, U256}; use zksync_utils::bytecode::CompressedBytecodeInfo; -use crate::vm_latest::types::internals::TransactionData; +use crate::vm_boojum_integration::types::internals::TransactionData; /// Information about tx necessary for execution in bootloader. #[derive(Debug, Clone)] diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs index 93ec255e7740..091bca449f75 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs @@ -4,7 +4,7 @@ use zksync_utils::{bytecode::CompressedBytecodeInfo, bytes_to_be_words, h256_to_ use super::tx::BootloaderTx; use crate::{ interface::{BootloaderMemory, TxExecutionMode}, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::l2_block::BootloaderL2Block, constants::{ BOOTLOADER_TX_DESCRIPTION_OFFSET, BOOTLOADER_TX_DESCRIPTION_SIZE, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs index 44266344be61..705327468875 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs @@ -7,7 +7,7 @@ use zksync_system_constants::{ USED_BOOTLOADER_MEMORY_WORDS, }; -use crate::vm_latest::old_vm::utils::heap_page_from_base; +use crate::vm_boojum_integration::old_vm::utils::heap_page_from_base; /// Max cycles for a single transaction. pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs index bda1803067fb..2e3770a9c52e 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs @@ -6,7 +6,7 @@ use zksync_utils::{ bytes_to_be_words, }; -use crate::{interface::VmInterface, vm_latest::Vm, HistoryMode}; +use crate::{interface::VmInterface, vm_boojum_integration::Vm, HistoryMode}; impl Vm { /// Checks the last transaction has successfully published compressed bytecodes and returns `true` if there is at least one is still unknown. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs index a913ea3ed463..9623b21a1945 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs @@ -6,7 +6,7 @@ use crate::{ types::tracer::{TracerExecutionStatus, VmExecutionStopReason}, VmExecutionMode, VmExecutionResultAndLogs, }, - vm_latest::{ + vm_boojum_integration::{ old_vm::utils::{vm_may_have_ended_inner, VmExecutionResult}, tracers::{ dispatcher::TracerDispatcher, DefaultExecutionTracer, PubdataTracer, RefundsTracer, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs index 526eab76f07f..56f13de05e54 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/gas.rs @@ -1,7 +1,7 @@ use zksync_state::WriteStorage; use crate::{ - vm_latest::{tracers::DefaultExecutionTracer, vm::Vm}, + vm_boojum_integration::{tracers::DefaultExecutionTracer, vm::Vm}, HistoryMode, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs index 9e0817aa9394..af307af55e28 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/logs.rs @@ -8,7 +8,9 @@ use zksync_types::{ use crate::{ interface::types::outputs::VmExecutionLogs, - vm_latest::{old_vm::utils::precompile_calls_count_after_timestamp, utils::logs, vm::Vm}, + vm_boojum_integration::{ + old_vm::utils::precompile_calls_count_after_timestamp, utils::logs, vm::Vm, + }, HistoryMode, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs index b6b452834388..98c4baad99f8 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs @@ -4,10 +4,8 @@ use vise::{Buckets, EncodeLabelSet, EncodeLabelValue, Family, Histogram, Metrics use zk_evm_1_4_0::aux_structures::Timestamp; use zksync_state::WriteStorage; -use crate::vm_latest::{ - old_vm::{history_recorder::HistoryEnabled, oracles::OracleWithHistory}, - types::internals::VmSnapshot, - vm::Vm, +use crate::vm_boojum_integration::{ + old_vm::oracles::OracleWithHistory, types::internals::VmSnapshot, vm::Vm, }; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EncodeLabelSet, EncodeLabelValue)] @@ -22,7 +20,7 @@ enum RollbackStage { } #[derive(Debug, Metrics)] -#[metrics(prefix = "server_vm")] +#[metrics(prefix = "server_vm_boojum_integration")] struct VmMetrics { #[metrics(buckets = Buckets::LATENCIES)] rollback_time: Family>, @@ -32,7 +30,7 @@ struct VmMetrics { static METRICS: vise::Global = vise::Global::new(); /// Implementation of VM related to rollbacks inside virtual machine -impl Vm { +impl Vm { pub(crate) fn make_snapshot_inner(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs index 6af9ad041feb..c11165cf78d1 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs @@ -4,7 +4,7 @@ use zksync_types::U256; use crate::{ interface::{VmExecutionStatistics, VmMemoryMetrics}, - vm_latest::{tracers::DefaultExecutionTracer, vm::Vm}, + vm_boojum_integration::{tracers::DefaultExecutionTracer, vm::Vm}, HistoryMode, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs index 326be41c5ee7..e341f79d1d46 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs @@ -3,7 +3,7 @@ use zksync_state::WriteStorage; use zksync_types::{l1::is_l1_tx_type, Transaction}; use crate::{ - vm_latest::{ + vm_boojum_integration::{ constants::BOOTLOADER_HEAP_PAGE, implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, types::internals::TransactionData, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs index c3df28f6c31c..83693e4b24e9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/mod.rs @@ -28,8 +28,6 @@ pub mod constants; mod implementation; mod old_vm; mod oracles; -#[cfg(test)] -mod tests; pub(crate) mod tracers; mod types; pub mod utils; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs index 8e7f4d447b4d..0693ad35742a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs @@ -11,7 +11,7 @@ use zk_evm_1_4_0::{ }; use zksync_types::U256; -use crate::vm_latest::old_vm::{ +use crate::vm_boojum_integration::old_vm::{ history_recorder::{AppDataFrameManagerWithHistory, HistoryEnabled, HistoryMode}, oracles::OracleWithHistory, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs index 83a22f35b4a1..10e138885aa4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs @@ -773,7 +773,7 @@ mod tests { use zk_evm_1_4_0::{aux_structures::Timestamp, vm_state::PrimitiveValue}; use zksync_types::U256; - use crate::vm_latest::{ + use crate::vm_boojum_integration::{ old_vm::history_recorder::{HistoryRecorder, MemoryWrapper}, HistoryDisabled, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs index 5a7592ce9654..f151278ffa4b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs @@ -6,7 +6,7 @@ use zk_evm_1_4_0::{ }; use zksync_types::U256; -use crate::vm_latest::old_vm::{ +use crate::vm_boojum_integration::old_vm::{ history_recorder::{ FramedStack, HistoryEnabled, HistoryMode, IntFrameManagerWithHistory, MemoryWithHistory, MemoryWrapper, WithHistory, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs index 4a718917a21d..299f7b170b54 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs @@ -11,7 +11,7 @@ use zksync_types::U256; use zksync_utils::{bytecode::bytecode_len_in_words, bytes_to_be_words, u256_to_h256}; use super::OracleWithHistory; -use crate::vm_latest::old_vm::history_recorder::{ +use crate::vm_boojum_integration::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs index 92b88e40fc95..4c798a00a37b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs @@ -5,7 +5,9 @@ use zk_evm_1_4_0::{ }; use super::OracleWithHistory; -use crate::vm_latest::old_vm::history_recorder::{HistoryEnabled, HistoryMode, HistoryRecorder}; +use crate::vm_boojum_integration::old_vm::history_recorder::{ + HistoryEnabled, HistoryMode, HistoryRecorder, +}; /// Wrap of DefaultPrecompilesProcessor that store queue /// of timestamp when precompiles are called to be executed. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs index b2c471832e46..1c14706de87a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/storage.rs @@ -1,6 +1,6 @@ use std::collections::HashMap; -use crate::vm_latest::old_vm::history_recorder::{ +use crate::vm_boojum_integration::old_vm::history_recorder::{ AppDataFrameManagerWithHistory, HashMapHistoryEvent, HistoryEnabled, HistoryMode, HistoryRecorder, StorageWrapper, WithHistory, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs index 1dbe82a81d4f..3aa4beb8e8de 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs @@ -10,7 +10,7 @@ use zksync_state::WriteStorage; use zksync_system_constants::L1_GAS_PER_PUBDATA_BYTE; use zksync_types::{Address, U256}; -use crate::vm_latest::{ +use crate::vm_boojum_integration::{ old_vm::memory::SimpleMemory, types::internals::ZkSyncVmState, HistoryMode, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs index 2b6b5988e06e..2e051db47481 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs @@ -17,7 +17,7 @@ use zksync_types::{ }; use zksync_utils::u256_to_h256; -use crate::vm_latest::old_vm::{ +use crate::vm_boojum_integration::old_vm::{ history_recorder::{ AppDataFrameManagerWithHistory, HashMapHistoryEvent, HistoryEnabled, HistoryMode, HistoryRecorder, StorageWrapper, VectorHistoryEvent, WithHistory, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs index 78fb964f7221..0ee3b811b4ca 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bootloader.rs @@ -2,7 +2,7 @@ use zksync_types::U256; use crate::{ interface::{ExecutionResult, Halt, TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ constants::BOOTLOADER_HEAP_PAGE, tests::{ tester::VmTesterBuilder, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs index a0c10addff93..ad1b0f26036e 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/bytecode_publishing.rs @@ -3,7 +3,7 @@ use zksync_utils::bytecode::compress_bytecode; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{DeployContractsTx, TxType, VmTesterBuilder}, utils::read_test_contract, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs index 2f8f37e081bd..e9df4fa80ff0 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/call_tracer.rs @@ -6,7 +6,7 @@ use zksync_types::{Address, Execute}; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, tracers::CallTracer, - vm_latest::{ + vm_boojum_integration::{ constants::BLOCK_GAS_LIMIT, tests::{ tester::VmTesterBuilder, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs index 7c951e313210..a8c20cfebc1d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/default_aa.rs @@ -8,7 +8,7 @@ use zksync_utils::u256_to_h256; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{DeployContractsTx, TxType, VmTesterBuilder}, utils::{get_balance, read_test_contract, verify_required_storage}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs index 533d9ec660eb..30a65097111d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/gas_limit.rs @@ -2,7 +2,7 @@ use zksync_types::{fee::Fee, Execute}; use crate::{ interface::{TxExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ constants::{BOOTLOADER_HEAP_PAGE, TX_DESCRIPTION_OFFSET, TX_GAS_LIMIT_OFFSET}, tests::tester::VmTesterBuilder, HistoryDisabled, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs index b82057bef8b7..25aab0871f14 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/get_used_contracts.rs @@ -9,7 +9,7 @@ use zksync_utils::{bytecode::hash_bytecode, h256_to_u256}; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{TxType, VmTesterBuilder}, utils::{read_test_contract, BASE_SYSTEM_CONTRACTS}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs index dde83d8a9f36..079e6d61b6c2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/invalid_bytecode.rs @@ -1,9 +1,9 @@ use zksync_types::H256; use zksync_utils::h256_to_u256; -use crate::vm_latest::tests::tester::VmTesterBuilder; -use crate::vm_latest::types::inputs::system_env::TxExecutionMode; -use crate::vm_latest::{HistoryEnabled, TxRevertReason}; +use crate::vm_boojum_integration::tests::tester::VmTesterBuilder; +use crate::vm_boojum_integration::types::inputs::system_env::TxExecutionMode; +use crate::vm_boojum_integration::{HistoryEnabled, TxRevertReason}; // TODO this test requires a lot of hacks for bypassing the bytecode checks in the VM. // Port it later, it's not significant. for now diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs index d5a6679502b5..bf56aa2b816d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/is_write_initial.rs @@ -3,7 +3,7 @@ use zksync_types::get_nonce_key; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{Account, TxType, VmTesterBuilder}, utils::read_test_contract, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs index 4f61dd90fad7..b547f346d28c 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l1_tx_execution.rs @@ -9,7 +9,7 @@ use zksync_utils::u256_to_h256; use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{TxType, VmTesterBuilder}, utils::{read_test_contract, verify_required_storage, BASE_SYSTEM_CONTRACTS}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs index 1faeba9652f9..b26cc09e0577 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/l2_blocks.rs @@ -18,7 +18,7 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; use crate::{ interface::{ExecutionResult, Halt, L2BlockEnv, TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ constants::{ BOOTLOADER_HEAP_PAGE, TX_OPERATOR_L2_BLOCK_INFO_OFFSET, TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs index 2de5e23bdd23..44ba3e4e323a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/nonce_holder.rs @@ -5,7 +5,7 @@ use crate::{ ExecutionResult, Halt, TxExecutionMode, TxRevertReason, VmExecutionMode, VmInterface, VmRevertReason, }, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{Account, VmTesterBuilder}, utils::read_nonce_holder_tester, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs index dc1f4fe55bca..521bd81f2ef4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/refunds.rs @@ -1,6 +1,6 @@ use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{DeployContractsTx, TxType, VmTesterBuilder}, utils::read_test_contract, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs index c03e5fe64212..90c3206b24bf 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/require_eip712.rs @@ -11,7 +11,7 @@ use zksync_types::{ use crate::{ interface::{TxExecutionMode, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{Account, VmTester, VmTesterBuilder}, utils::read_many_owners_custom_account_contract, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs index 23c1ab49ad98..3d3127f8428b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/rollbacks.rs @@ -9,7 +9,7 @@ use crate::{ tracer::{TracerExecutionStatus, TracerExecutionStopReason}, TxExecutionMode, VmExecutionMode, VmInterface, VmInterfaceHistoryEnabled, }, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{DeployContractsTx, TransactionTestInfo, TxModifier, TxType, VmTesterBuilder}, utils::read_test_contract, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs index a864538524a2..fc94e2c71526 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/simple_execution.rs @@ -1,6 +1,6 @@ use crate::{ interface::{ExecutionResult, VmExecutionMode, VmInterface}, - vm_latest::{ + vm_boojum_integration::{ tests::tester::{TxType, VmTesterBuilder}, HistoryDisabled, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs index b82e995c2db3..24f31c5a9393 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/inner_state.rs @@ -5,7 +5,7 @@ use zksync_state::WriteStorage; use zksync_types::{StorageKey, StorageLogQuery, StorageValue, U256}; use crate::{ - vm_latest::{ + vm_boojum_integration::{ old_vm::{ event_sink::InMemoryEventSink, history_recorder::{AppDataFrameManagerWithHistory, HistoryRecorder}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs index 114f80d1a217..4d6572fe78a2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/transaction_test_info.rs @@ -5,7 +5,7 @@ use crate::{ CurrentExecutionState, ExecutionResult, Halt, TxRevertReason, VmExecutionMode, VmExecutionResultAndLogs, VmInterface, VmInterfaceHistoryEnabled, VmRevertReason, }, - vm_latest::{tests::tester::vm_tester::VmTester, HistoryEnabled}, + vm_boojum_integration::{tests::tester::vm_tester::VmTester, HistoryEnabled}, }; #[derive(Debug, Clone)] diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs index 25f1361f14d0..30bf9535eb8b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tester/vm_tester.rs @@ -15,7 +15,7 @@ use crate::{ interface::{ L1BatchEnv, L2Block, L2BlockEnv, SystemEnv, TxExecutionMode, VmExecutionMode, VmInterface, }, - vm_latest::{ + vm_boojum_integration::{ constants::BLOCK_GAS_LIMIT, tests::{ tester::{Account, TxType}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs index f02de899b03e..8c538dcf9bf2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/tracing_execution_error.rs @@ -2,7 +2,7 @@ use zksync_types::{Execute, H160}; use crate::{ interface::{TxExecutionMode, TxRevertReason, VmRevertReason}, - vm_latest::{ + vm_boojum_integration::{ tests::{ tester::{ExpectedError, TransactionTestInfo, VmTesterBuilder}, utils::{get_execute_error_calldata, read_error_contract, BASE_SYSTEM_CONTRACTS}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs index b5c493ca7075..4442d7c4082d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/upgrade.rs @@ -18,7 +18,7 @@ use crate::{ ExecutionResult, Halt, TxExecutionMode, VmExecutionMode, VmInterface, VmInterfaceHistoryEnabled, }, - vm_latest::{ + vm_boojum_integration::{ tests::{tester::VmTesterBuilder, utils::verify_required_storage}, HistoryEnabled, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs index c126b50cb574..53ae1c17e917 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs @@ -9,7 +9,7 @@ use zksync_types::{ }; use zksync_utils::{bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}; -use crate::vm_latest::{ +use crate::vm_boojum_integration::{ tests::tester::InMemoryStorageView, types::internals::ZkSyncVmState, HistoryMode, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs index 0e18d989af62..f0690d996b21 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs @@ -22,7 +22,7 @@ use crate::{ types::tracer::TracerExecutionStatus, Halt, VmExecutionMode, }, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::{utils::apply_l2_block, BootloaderState}, constants::BOOTLOADER_HEAP_PAGE, old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs index 5ee5c8ab0c19..11262c4d7665 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/dispatcher.rs @@ -8,7 +8,7 @@ use crate::{ dyn_tracers::vm_1_4_0::DynTracer, tracer::{TracerExecutionStatus, VmExecutionStopReason}, }, - vm_latest::{ + vm_boojum_integration::{ BootloaderState, HistoryMode, SimpleMemory, TracerPointer, VmTracer, ZkSyncVmState, }, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs index 5773e3797e2b..78c8160d1af9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs @@ -23,7 +23,7 @@ use crate::{ types::inputs::L1BatchEnv, VmExecutionMode, }, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::{utils::apply_pubdata_to_memory, BootloaderState}, constants::BOOTLOADER_HEAP_PAGE, old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs index e852fba1dac8..5605661a2ec2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs @@ -20,7 +20,7 @@ use crate::{ traits::tracers::dyn_tracers::vm_1_4_0::DynTracer, types::tracer::TracerExecutionStatus, L1BatchEnv, Refunds, }, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, old_vm::{ @@ -203,7 +203,7 @@ impl VmTracer for RefundsTracer { ]); #[derive(Debug, Metrics)] - #[metrics(prefix = "vm")] + #[metrics(prefix = "vm_boojum_integration")] struct RefundMetrics { #[metrics(buckets = PERCENT_BUCKETS)] refund: Family>, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs index b3412587725c..b8b00b265e03 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs @@ -14,7 +14,7 @@ use crate::{ types::tracer::TracerExecutionStopReason, ExecutionResult, Halt, TxRevertReason, VmExecutionMode, VmRevertReason, }, - vm_latest::{ + vm_boojum_integration::{ constants::{BOOTLOADER_HEAP_PAGE, RESULT_SUCCESS_FIRST_SLOT}, old_vm::utils::{vm_may_have_ended_inner, VmExecutionResult}, tracers::{ diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs index 68307b3f2867..767f45c6050a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/traits.rs @@ -5,7 +5,7 @@ use crate::{ dyn_tracers::vm_1_4_0::DynTracer, tracer::{TracerExecutionStatus, VmExecutionStopReason}, }, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::BootloaderState, old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, types::internals::ZkSyncVmState, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs index 93710586fdac..592429677109 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs @@ -12,7 +12,7 @@ use zksync_system_constants::{ use zksync_types::U256; use zksync_utils::u256_to_h256; -use crate::vm_latest::{ +use crate::vm_boojum_integration::{ constants::{ BOOTLOADER_HEAP_PAGE, VM_HOOK_PARAMS_COUNT, VM_HOOK_PARAMS_START_POSITION, VM_HOOK_POSITION, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs index 2a9368c37a39..f1aa1a333595 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/snapshot.rs @@ -1,6 +1,6 @@ use zk_evm_1_4_0::vm_state::VmLocalState; -use crate::vm_latest::bootloader_state::BootloaderStateSnapshot; +use crate::vm_boojum_integration::bootloader_state::BootloaderStateSnapshot; /// A snapshot of the VM that holds enough information to /// rollback the VM to some historical state. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs index f5b97ca9793e..a2a9f57d2742 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs @@ -11,7 +11,7 @@ use zksync_types::{ }; use zksync_utils::{address_to_h256, bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256}; -use crate::vm_latest::utils::overhead::{get_amortized_overhead, OverheadCoefficients}; +use crate::vm_boojum_integration::utils::overhead::{get_amortized_overhead, OverheadCoefficients}; /// This structure represents the data that is used by /// the Bootloader to describe the transaction. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs index da9c81321440..b775b4b63bca 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs @@ -19,7 +19,7 @@ use zksync_utils::h256_to_u256; use crate::{ interface::{L1BatchEnv, L2Block, SystemEnv}, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::BootloaderState, constants::BOOTLOADER_HEAP_PAGE, old_vm::{ diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs index 23b744a348f7..9a2e58c72993 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs @@ -2,7 +2,7 @@ use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; use zksync_utils::ceil_div; -use crate::vm_latest::old_vm::utils::eth_price_per_pubdata_byte; +use crate::vm_boojum_integration::old_vm::utils::eth_price_per_pubdata_byte; /// Calculates the amount of gas required to publish one byte of pubdata pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs index b7fa07956a96..0461b4a8887a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/logs.rs @@ -3,7 +3,7 @@ use zksync_types::{l2_to_l1_log::L2ToL1Log, Timestamp, VmEvent}; use crate::{ interface::L1BatchEnv, - vm_latest::{ + vm_boojum_integration::{ old_vm::{events::merge_events, history_recorder::HistoryMode}, types::internals::ZkSyncVmState, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs index 38aaede8d4bb..5f50f664c234 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs @@ -3,7 +3,7 @@ use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; use zksync_types::{l1::is_l1_tx_type, U256}; use zksync_utils::ceil_div_u256; -use crate::vm_latest::constants::{ +use crate::vm_boojum_integration::constants::{ BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, }; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs index 86c49a3eb15d..0a447ac31db4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/transaction_encoding.rs @@ -1,6 +1,6 @@ use zksync_types::Transaction; -use crate::vm_latest::types::internals::TransactionData; +use crate::vm_boojum_integration::types::internals::TransactionData; /// Extension for transactions, specific for VM. Required for bypassing the orphan rule pub trait TransactionVmExt { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs index 73d551942bff..a602ebbb9d71 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs @@ -12,9 +12,9 @@ use crate::{ L1BatchEnv, L2BlockEnv, SystemEnv, VmExecutionMode, VmExecutionResultAndLogs, VmInterface, VmInterfaceHistoryEnabled, VmMemoryMetrics, }, - vm_latest::{ + vm_boojum_integration::{ bootloader_state::BootloaderState, - old_vm::{events::merge_events, history_recorder::HistoryEnabled}, + old_vm::events::merge_events, tracers::dispatcher::TracerDispatcher, types::internals::{new_vm_state, VmSnapshot, ZkSyncVmState}, }, @@ -160,7 +160,7 @@ impl VmInterface for Vm { } /// Methods of vm, which required some history manipulations -impl VmInterfaceHistoryEnabled for Vm { +impl VmInterfaceHistoryEnabled for Vm { /// Create snapshot of current vm state and push it into the memory fn make_snapshot(&mut self) { self.make_snapshot_inner() diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs b/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs index a913ea3ed463..cc76872d29ec 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs @@ -19,7 +19,7 @@ use crate::{ impl Vm { pub(crate) fn inspect_inner( &mut self, - dispatcher: TracerDispatcher, + dispatcher: TracerDispatcher, execution_mode: VmExecutionMode, ) -> VmExecutionResultAndLogs { let mut enable_refund_tracer = false; @@ -38,21 +38,20 @@ impl Vm { /// Collect the result from the default tracers. fn inspect_and_collect_results( &mut self, - dispatcher: TracerDispatcher, + dispatcher: TracerDispatcher, execution_mode: VmExecutionMode, with_refund_tracer: bool, ) -> (VmExecutionStopReason, VmExecutionResultAndLogs) { let refund_tracers = with_refund_tracer.then_some(RefundsTracer::new(self.batch_env.clone())); - let mut tx_tracer: DefaultExecutionTracer = - DefaultExecutionTracer::new( - self.system_env.default_validation_computational_gas_limit, - execution_mode, - dispatcher, - self.storage.clone(), - refund_tracers, - Some(PubdataTracer::new(self.batch_env.clone(), execution_mode)), - ); + let mut tx_tracer: DefaultExecutionTracer = DefaultExecutionTracer::new( + self.system_env.default_validation_computational_gas_limit, + execution_mode, + dispatcher, + self.storage.clone(), + refund_tracers, + Some(PubdataTracer::new(self.batch_env.clone(), execution_mode)), + ); let timestamp_initial = Timestamp(self.state.local_state.timestamp); let cycles_initial = self.state.local_state.monotonic_cycle_counter; @@ -96,7 +95,7 @@ impl Vm { /// Execute vm with given tracers until the stop reason is reached. fn execute_with_default_tracer( &mut self, - tracer: &mut DefaultExecutionTracer, + tracer: &mut DefaultExecutionTracer, ) -> VmExecutionStopReason { tracer.initialize_tracer(&mut self.state); let result = loop { diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/gas.rs b/core/lib/multivm/src/versions/vm_latest/implementation/gas.rs index 526eab76f07f..41443298658d 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/gas.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/gas.rs @@ -20,7 +20,7 @@ impl Vm { pub(crate) fn calculate_computational_gas_used( &self, - tracer: &DefaultExecutionTracer, + tracer: &DefaultExecutionTracer, gas_remaining_before: u32, spent_pubdata_counter_before: u32, ) -> u32 { diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs index 6af9ad041feb..8bb3f65bea47 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs @@ -18,7 +18,7 @@ impl Vm { &self, timestamp_initial: Timestamp, cycles_initial: u32, - tracer: &DefaultExecutionTracer, + tracer: &DefaultExecutionTracer, gas_remaining_before: u32, gas_remaining_after: u32, spent_pubdata_counter_before: u32, diff --git a/core/lib/multivm/src/versions/vm_latest/tests/tester/inner_state.rs b/core/lib/multivm/src/versions/vm_latest/tests/tester/inner_state.rs index b82e995c2db3..39ecdf24cb84 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/tester/inner_state.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/tester/inner_state.rs @@ -78,7 +78,7 @@ pub(crate) struct VmInstanceInnerState { impl Vm { // Dump inner state of the VM. - pub(crate) fn dump_inner_state(&self) -> VmInstanceInnerState { + pub(crate) fn dump_inner_state(&self) -> VmInstanceInnerState { let event_sink = self.state.event_sink.clone(); let precompile_processor_state = PrecompileProcessorTestInnerState { timestamp_history: self.state.precompiles_processor.timestamp_history.clone(), diff --git a/core/lib/multivm/src/versions/vm_latest/vm.rs b/core/lib/multivm/src/versions/vm_latest/vm.rs index 73d551942bff..920c4fefaab6 100644 --- a/core/lib/multivm/src/versions/vm_latest/vm.rs +++ b/core/lib/multivm/src/versions/vm_latest/vm.rs @@ -27,7 +27,7 @@ use crate::{ pub struct Vm { pub(crate) bootloader_state: BootloaderState, // Current state and oracles of virtual machine - pub(crate) state: ZkSyncVmState, + pub(crate) state: ZkSyncVmState, pub(crate) storage: StoragePtr, pub(crate) system_env: SystemEnv, pub(crate) batch_env: L1BatchEnv, @@ -37,7 +37,7 @@ pub struct Vm { } impl VmInterface for Vm { - type TracerDispatcher = TracerDispatcher; + type TracerDispatcher = TracerDispatcher; fn new(batch_env: L1BatchEnv, system_env: SystemEnv, storage: StoragePtr) -> Self { let (state, bootloader_state) = new_vm_state(storage.clone(), &system_env, &batch_env); diff --git a/core/lib/multivm/src/vm_instance.rs b/core/lib/multivm/src/vm_instance.rs index 6716aeaf146f..88b6a5275122 100644 --- a/core/lib/multivm/src/vm_instance.rs +++ b/core/lib/multivm/src/vm_instance.rs @@ -19,7 +19,7 @@ pub enum VmInstance { Vm1_3_2(crate::vm_1_3_2::Vm), VmVirtualBlocks(crate::vm_virtual_blocks::Vm), VmVirtualBlocksRefundsEnhancement(crate::vm_refunds_enhancement::Vm), - VmBoojumIntegration(crate::vm_latest::Vm), + VmBoojumIntegration(crate::vm_boojum_integration::Vm), } macro_rules! dispatch_vm { @@ -188,7 +188,8 @@ impl VmInstance { VmInstance::VmVirtualBlocksRefundsEnhancement(vm) } VmVersion::VmBoojumIntegration => { - let vm = crate::vm_latest::Vm::new(l1_batch_env, system_env, storage_view); + let vm = + crate::vm_boojum_integration::Vm::new(l1_batch_env, system_env, storage_view); VmInstance::VmBoojumIntegration(vm) } } diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs index 719f6da0b4a9..443b35fb46e0 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/tracers.rs @@ -14,7 +14,7 @@ pub(crate) enum ApiTracer { impl ApiTracer { pub fn into_boxed< S: WriteStorage, - H: HistoryMode + multivm::HistoryMode + 'static, + H: HistoryMode + multivm::HistoryMode + 'static, >( self, ) -> MultiVmTracerPointer { From 5053b0af158d9202b4da976c2ddf4c0e657825df Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Thu, 4 Jan 2024 17:51:16 +0400 Subject: [PATCH 11/49] refactor(metadata_calculator): Move ObjectStore out from MetadataCalculatorConfig (#816) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ title ## Why ❔ - There was some clunky back-and-forth for `MetadataCalculatorModeConfig` <->`MerkleTreeMode` because of the old component confuguration. - ObjectStore may not be available at time when `MetadataCalculatorModeConfig ` is created (prerequisite for ZK Stack thing). - Less code! ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/main.rs | 13 +++-- core/lib/zksync_core/src/lib.rs | 29 +++++------- .../src/metadata_calculator/mod.rs | 41 +++------------- .../src/metadata_calculator/recovery/tests.rs | 5 +- .../src/metadata_calculator/tests.rs | 47 +++++++++---------- 5 files changed, 49 insertions(+), 86 deletions(-) diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index d815619934f1..01b3a1260802 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -7,6 +7,7 @@ use metrics::EN_METRICS; use prometheus_exporter::PrometheusExporterConfig; use tokio::{sync::watch, task, time::sleep}; use zksync_basic_types::{Address, L2ChainId}; +use zksync_config::configs::database::MerkleTreeMode; use zksync_core::{ api_server::{ execution_sandbox::VmConcurrencyLimiter, @@ -17,9 +18,7 @@ use zksync_core::{ block_reverter::{BlockReverter, BlockReverterFlags, L1ExecutedBatchesRevert}, consistency_checker::ConsistencyChecker, l1_gas_price::MainNodeGasPriceFetcher, - metadata_calculator::{ - MetadataCalculator, MetadataCalculatorConfig, MetadataCalculatorModeConfig, - }, + metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, reorg_detector::ReorgDetector, setup_sigint_handler, state_keeper::{ @@ -186,17 +185,17 @@ async fn init_tasks( stop_receiver.clone(), ); - let metadata_calculator = MetadataCalculator::new(MetadataCalculatorConfig { + let metadata_calculator_config = MetadataCalculatorConfig { db_path: config.required.merkle_tree_path.clone(), - mode: MetadataCalculatorModeConfig::Full { object_store: None }, + mode: MerkleTreeMode::Full, delay_interval: config.optional.metadata_calculator_delay(), max_l1_batches_per_iter: config.optional.max_l1_batches_per_tree_iter, multi_get_chunk_size: config.optional.merkle_tree_multi_get_chunk_size, block_cache_capacity: config.optional.merkle_tree_block_cache_size(), memtable_capacity: config.optional.merkle_tree_memtable_capacity(), stalled_writes_timeout: config.optional.merkle_tree_stalled_writes_timeout(), - }) - .await; + }; + let metadata_calculator = MetadataCalculator::new(metadata_calculator_config, None).await; healthchecks.push(Box::new(metadata_calculator.tree_health_check())); let consistency_checker = ConsistencyChecker::new( diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 851b281bf8e3..b3a3d2e580ee 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -19,7 +19,7 @@ use zksync_config::{ StateKeeperConfig, }, contracts::ProverAtGenesis, - database::MerkleTreeMode, + database::{MerkleTreeConfig, MerkleTreeMode}, }, ApiConfig, ContractsConfig, DBConfig, ETHSenderConfig, PostgresConfig, }; @@ -65,9 +65,7 @@ use crate::{ waiting_to_queued_fri_witness_job_mover::WaitingToQueuedFriWitnessJobMover, }, l1_gas_price::{GasAdjusterSingleton, L1GasPriceProvider}, - metadata_calculator::{ - MetadataCalculator, MetadataCalculatorConfig, MetadataCalculatorModeConfig, - }, + metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, metrics::{InitStage, APP_METRICS}, state_keeper::{ create_state_keeper, MempoolFetcher, MempoolGuard, MiniblockSealer, SequencerSealer, @@ -768,21 +766,19 @@ async fn add_trees_to_task_futures( .contains(&Component::TreeApi) .then_some(&api_config); - let mode = match db_config.merkle_tree.mode { - MerkleTreeMode::Lightweight => MetadataCalculatorModeConfig::Lightweight, - MerkleTreeMode::Full => MetadataCalculatorModeConfig::Full { - object_store: Some(store_factory.create_store().await), - }, + let object_store = match db_config.merkle_tree.mode { + MerkleTreeMode::Lightweight => None, + MerkleTreeMode::Full => Some(store_factory.create_store().await), }; run_tree( task_futures, healthchecks, &postgres_config, - &db_config, + &db_config.merkle_tree, api_config, &operation_config, - mode, + object_store, stop_receiver, ) .await @@ -794,23 +790,22 @@ async fn run_tree( task_futures: &mut Vec>>, healthchecks: &mut Vec>, postgres_config: &PostgresConfig, - db_config: &DBConfig, + merkle_tree_config: &MerkleTreeConfig, api_config: Option<&MerkleTreeApiConfig>, operation_manager: &OperationsManagerConfig, - mode: MetadataCalculatorModeConfig, + object_store: Option>, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { let started_at = Instant::now(); - let mode_str = if matches!(mode, MetadataCalculatorModeConfig::Full { .. }) { + let mode_str = if matches!(merkle_tree_config.mode, MerkleTreeMode::Full) { "full" } else { "lightweight" }; tracing::info!("Initializing Merkle tree in {mode_str} mode"); - let config = - MetadataCalculatorConfig::for_main_node(&db_config.merkle_tree, operation_manager, mode); - let metadata_calculator = MetadataCalculator::new(config).await; + let config = MetadataCalculatorConfig::for_main_node(merkle_tree_config, operation_manager); + let metadata_calculator = MetadataCalculator::new(config, object_store).await; if let Some(api_config) = api_config { let address = (Ipv4Addr::UNSPECIFIED, api_config.port).into(); let tree_reader = metadata_calculator.tree_reader(); diff --git a/core/lib/zksync_core/src/metadata_calculator/mod.rs b/core/lib/zksync_core/src/metadata_calculator/mod.rs index c0fb5142eb6d..0244094be665 100644 --- a/core/lib/zksync_core/src/metadata_calculator/mod.rs +++ b/core/lib/zksync_core/src/metadata_calculator/mod.rs @@ -36,36 +36,13 @@ mod recovery; pub(crate) mod tests; mod updater; -/// Part of [`MetadataCalculator`] related to the operation mode of the Merkle tree. -#[derive(Debug)] -pub enum MetadataCalculatorModeConfig { - /// In this mode, `MetadataCalculator` computes Merkle tree root hashes and some auxiliary information - /// for L1 batches, but not witness inputs. - Lightweight, - /// In this mode, `MetadataCalculator` will compute commitments and witness inputs for all storage operations - /// and optionally put witness inputs into the object store (e.g., GCS). - Full { - object_store: Option>, - }, -} - -impl MetadataCalculatorModeConfig { - fn to_mode(&self) -> MerkleTreeMode { - if matches!(self, Self::Full { .. }) { - MerkleTreeMode::Full - } else { - MerkleTreeMode::Lightweight - } - } -} - /// Configuration of [`MetadataCalculator`]. #[derive(Debug)] pub struct MetadataCalculatorConfig { /// Filesystem path to the RocksDB instance that stores the tree. pub db_path: String, /// Configuration of the Merkle tree mode. - pub mode: MetadataCalculatorModeConfig, + pub mode: MerkleTreeMode, /// Interval between polling Postgres for updates if no progress was made by the tree. pub delay_interval: Duration, /// Maximum number of L1 batches to get from Postgres on a single update iteration. @@ -86,11 +63,10 @@ impl MetadataCalculatorConfig { pub(crate) fn for_main_node( merkle_tree_config: &MerkleTreeConfig, operation_config: &OperationsManagerConfig, - mode: MetadataCalculatorModeConfig, ) -> Self { Self { db_path: merkle_tree_config.path.clone(), - mode, + mode: merkle_tree_config.mode, delay_interval: operation_config.delay_interval(), max_l1_batches_per_iter: merkle_tree_config.max_l1_batches_per_iter, multi_get_chunk_size: merkle_tree_config.multi_get_chunk_size, @@ -113,18 +89,15 @@ pub struct MetadataCalculator { impl MetadataCalculator { /// Creates a calculator with the specified `config`. - pub async fn new(config: MetadataCalculatorConfig) -> Self { + pub async fn new( + config: MetadataCalculatorConfig, + object_store: Option>, + ) -> Self { assert!( config.max_l1_batches_per_iter > 0, "Maximum L1 batches per iteration is misconfigured to be 0; please update it to positive value" ); - let mode = config.mode.to_mode(); - let object_store = match config.mode { - MetadataCalculatorModeConfig::Full { object_store } => object_store, - MetadataCalculatorModeConfig::Lightweight => None, - }; - let db = create_db( config.db_path.clone().into(), config.block_cache_capacity, @@ -133,7 +106,7 @@ impl MetadataCalculator { config.multi_get_chunk_size, ) .await; - let tree = GenericAsyncTree::new(db, mode).await; + let tree = GenericAsyncTree::new(db, config.mode).await; let (_, health_updater) = ReactiveHealthCheck::new("tree"); Self { diff --git a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs index 082f684bbc9c..ee2fc0bb8a76 100644 --- a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs @@ -27,7 +27,7 @@ use crate::{ extend_db_state, extend_db_state_from_l1_batch, gen_storage_logs, run_calculator, setup_calculator, }, - MetadataCalculator, MetadataCalculatorConfig, MetadataCalculatorModeConfig, + MetadataCalculator, MetadataCalculatorConfig, }, }; @@ -292,9 +292,8 @@ async fn entire_recovery_workflow(case: RecoveryWorkflowCase) { let calculator_config = MetadataCalculatorConfig::for_main_node( &merkle_tree_config, &OperationsManagerConfig { delay_interval: 50 }, - MetadataCalculatorModeConfig::Lightweight, ); - let mut calculator = MetadataCalculator::new(calculator_config).await; + let mut calculator = MetadataCalculator::new(calculator_config, None).await; let (delay_sx, mut delay_rx) = mpsc::unbounded_channel(); calculator.delayer.delay_notifier = delay_sx; diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index 1a5d2a6d7704..b982039dbc57 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -6,7 +6,10 @@ use assert_matches::assert_matches; use itertools::Itertools; use tempfile::TempDir; use tokio::sync::{mpsc, watch}; -use zksync_config::configs::{chain::OperationsManagerConfig, database::MerkleTreeConfig}; +use zksync_config::configs::{ + chain::OperationsManagerConfig, + database::{MerkleTreeConfig, MerkleTreeMode}, +}; use zksync_contracts::BaseSystemContracts; use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_health_check::{CheckHealth, HealthStatus}; @@ -20,10 +23,7 @@ use zksync_types::{ }; use zksync_utils::u32_to_h256; -use super::{ - GenericAsyncTree, L1BatchWithLogs, MetadataCalculator, MetadataCalculatorConfig, - MetadataCalculatorModeConfig, -}; +use super::{GenericAsyncTree, L1BatchWithLogs, MetadataCalculator, MetadataCalculatorConfig}; use crate::genesis::{ensure_genesis_state, GenesisParams}; const RUN_TIMEOUT: Duration = Duration::from_secs(30); @@ -237,16 +237,12 @@ async fn running_metadata_calculator_with_additional_blocks() { async fn shutting_down_calculator() { let pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); - let (merkle_tree_config, mut operation_config) = create_config(temp_dir.path()); + let (merkle_tree_config, mut operation_config) = + create_config(temp_dir.path(), MerkleTreeMode::Lightweight); operation_config.delay_interval = 30_000; // ms; chosen to be larger than `RUN_TIMEOUT` - let calculator = setup_calculator_with_options( - &merkle_tree_config, - &operation_config, - &pool, - MetadataCalculatorModeConfig::Lightweight, - ) - .await; + let calculator = + setup_calculator_with_options(&merkle_tree_config, &operation_config, &pool, None).await; reset_db_state(&pool, 5).await; @@ -365,24 +361,25 @@ pub(crate) async fn setup_calculator( ) -> (MetadataCalculator, Box) { let store_factory = ObjectStoreFactory::mock(); let store = store_factory.create_store().await; - let (merkle_tree_config, operation_manager) = create_config(db_path); - let mode = MetadataCalculatorModeConfig::Full { - object_store: Some(store), - }; + let (merkle_tree_config, operation_manager) = create_config(db_path, MerkleTreeMode::Full); let calculator = - setup_calculator_with_options(&merkle_tree_config, &operation_manager, pool, mode).await; + setup_calculator_with_options(&merkle_tree_config, &operation_manager, pool, Some(store)) + .await; (calculator, store_factory.create_store().await) } async fn setup_lightweight_calculator(db_path: &Path, pool: &ConnectionPool) -> MetadataCalculator { - let mode = MetadataCalculatorModeConfig::Lightweight; - let (db_config, operation_config) = create_config(db_path); - setup_calculator_with_options(&db_config, &operation_config, pool, mode).await + let (db_config, operation_config) = create_config(db_path, MerkleTreeMode::Lightweight); + setup_calculator_with_options(&db_config, &operation_config, pool, None).await } -fn create_config(db_path: &Path) -> (MerkleTreeConfig, OperationsManagerConfig) { +fn create_config( + db_path: &Path, + mode: MerkleTreeMode, +) -> (MerkleTreeConfig, OperationsManagerConfig) { let db_config = MerkleTreeConfig { path: path_to_string(&db_path.join("new")), + mode, ..MerkleTreeConfig::default() }; @@ -396,11 +393,11 @@ async fn setup_calculator_with_options( merkle_tree_config: &MerkleTreeConfig, operation_config: &OperationsManagerConfig, pool: &ConnectionPool, - mode: MetadataCalculatorModeConfig, + object_store: Option>, ) -> MetadataCalculator { let calculator_config = - MetadataCalculatorConfig::for_main_node(merkle_tree_config, operation_config, mode); - let metadata_calculator = MetadataCalculator::new(calculator_config).await; + MetadataCalculatorConfig::for_main_node(merkle_tree_config, operation_config); + let metadata_calculator = MetadataCalculator::new(calculator_config, object_store).await; let mut storage = pool.access_storage().await.unwrap(); if storage.blocks_dal().is_genesis_needed().await.unwrap() { From 3f663eca2f38f4373339ad024e6578099c693af6 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Thu, 4 Jan 2024 20:54:52 +0200 Subject: [PATCH 12/49] fix(db): Fix parsing statement timeout from env (#818) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Fixes parsing statement timeout from the environment for the main node. ## Why ❔ Without a statement timeout, we may have long-running DB queries for API servers. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/lib/env_config/src/database.rs | 28 ++++++++++++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/core/lib/env_config/src/database.rs b/core/lib/env_config/src/database.rs index e6269cd23768..74f665617ce5 100644 --- a/core/lib/env_config/src/database.rs +++ b/core/lib/env_config/src/database.rs @@ -27,11 +27,11 @@ impl FromEnv for PostgresConfig { .ok() .map(|val| val.parse().context("failed to parse DATABASE_POOL_SIZE")) .transpose()?; - let statement_timeout_sec = env::var("DATABASE_STATEMENT_TIMEOUT") + let statement_timeout_sec = env::var("DATABASE_STATEMENT_TIMEOUT_SEC") .ok() .map(|val| { val.parse() - .context("failed to parse DATABASE_STATEMENT_TIMEOUT") + .context("failed to parse DATABASE_STATEMENT_TIMEOUT_SEC") }) .transpose()?; @@ -47,6 +47,8 @@ impl FromEnv for PostgresConfig { #[cfg(test)] mod tests { + use std::time::Duration; + use zksync_config::configs::database::MerkleTreeMode; use super::*; @@ -120,4 +122,26 @@ mod tests { let db_config = DBConfig::from_env().unwrap(); assert_eq!(db_config.merkle_tree.max_l1_batches_per_iter, 50); } + + #[test] + fn postgres_from_env() { + let mut lock = MUTEX.lock(); + let config = r#" + DATABASE_URL=postgres://postgres@localhost/zksync_local + DATABASE_POOL_SIZE=50 + DATABASE_STATEMENT_TIMEOUT_SEC=300 + "#; + lock.set_env(config); + + let postgres_config = PostgresConfig::from_env().unwrap(); + assert_eq!( + postgres_config.master_url().unwrap(), + "postgres://postgres@localhost/zksync_local" + ); + assert_eq!(postgres_config.max_connections().unwrap(), 50); + assert_eq!( + postgres_config.statement_timeout(), + Some(Duration::from_secs(300)) + ); + } } From 39f2d5057a7bb5b489e858bafb72fb925ebc6c8f Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Fri, 5 Jan 2024 13:18:30 +0400 Subject: [PATCH 13/49] refactor(config): Remove ChainConfig structure (#821) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ It wasn't used anywhere. ## Why ❔ Dead code bad. ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/lib/config/src/configs/chain.rs | 14 --- core/lib/config/src/configs/mod.rs | 1 - core/lib/config/src/lib.rs | 5 +- core/lib/env_config/src/chain.rs | 169 ++++++++++++++++----------- 4 files changed, 106 insertions(+), 83 deletions(-) diff --git a/core/lib/config/src/configs/chain.rs b/core/lib/config/src/configs/chain.rs index eb77467183fe..47f43c9f3f06 100644 --- a/core/lib/config/src/configs/chain.rs +++ b/core/lib/config/src/configs/chain.rs @@ -3,20 +3,6 @@ use std::{str::FromStr, time::Duration}; use serde::Deserialize; use zksync_basic_types::{network::Network, Address, L2ChainId}; -#[derive(Debug, Deserialize, Clone, PartialEq)] -pub struct ChainConfig { - /// L1 parameters configuration. - pub network: NetworkConfig, - /// State keeper / block generating configuration. - pub state_keeper: StateKeeperConfig, - /// Operations manager / Metadata calculator. - pub operations_manager: OperationsManagerConfig, - /// mempool configuration - pub mempool: MempoolConfig, - /// circuit breaker configuration - pub circuit_breaker: CircuitBreakerConfig, -} - #[derive(Debug, Deserialize, Clone, PartialEq)] pub struct NetworkConfig { /// Name of the used Ethereum network, e.g. `localhost` or `rinkeby`. diff --git a/core/lib/config/src/configs/mod.rs b/core/lib/config/src/configs/mod.rs index 3066ee5d5c69..fc0e7eb6d4d2 100644 --- a/core/lib/config/src/configs/mod.rs +++ b/core/lib/config/src/configs/mod.rs @@ -2,7 +2,6 @@ pub use self::{ alerts::AlertsConfig, api::ApiConfig, - chain::ChainConfig, contract_verifier::ContractVerifierConfig, contracts::ContractsConfig, database::{DBConfig, PostgresConfig}, diff --git a/core/lib/config/src/lib.rs b/core/lib/config/src/lib.rs index f329fbf4f4b4..d139596b80bc 100644 --- a/core/lib/config/src/lib.rs +++ b/core/lib/config/src/lib.rs @@ -1,9 +1,8 @@ #![allow(clippy::upper_case_acronyms, clippy::derive_partial_eq_without_eq)] pub use crate::configs::{ - ApiConfig, ChainConfig, ContractVerifierConfig, ContractsConfig, DBConfig, ETHClientConfig, - ETHSenderConfig, ETHWatchConfig, GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, - SnapshotsCreatorConfig, + ApiConfig, ContractVerifierConfig, ContractsConfig, DBConfig, ETHClientConfig, ETHSenderConfig, + ETHWatchConfig, GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, SnapshotsCreatorConfig, }; pub mod configs; diff --git a/core/lib/env_config/src/chain.rs b/core/lib/env_config/src/chain.rs index 7c2aa7e59419..22a397d290fc 100644 --- a/core/lib/env_config/src/chain.rs +++ b/core/lib/env_config/src/chain.rs @@ -1,24 +1,9 @@ -use anyhow::Context as _; use zksync_config::configs::chain::{ - ChainConfig, CircuitBreakerConfig, MempoolConfig, NetworkConfig, OperationsManagerConfig, - StateKeeperConfig, + CircuitBreakerConfig, MempoolConfig, NetworkConfig, OperationsManagerConfig, StateKeeperConfig, }; use crate::{envy_load, FromEnv}; -impl FromEnv for ChainConfig { - fn from_env() -> anyhow::Result { - Ok(Self { - network: NetworkConfig::from_env().context("NetworkConfig")?, - state_keeper: StateKeeperConfig::from_env().context("StateKeeperConfig")?, - operations_manager: OperationsManagerConfig::from_env() - .context("OperationsManagerConfig")?, - mempool: MempoolConfig::from_env().context("MempoolConfig")?, - circuit_breaker: CircuitBreakerConfig::from_env().context("CircuitBreakerConfig")?, - }) - } -} - impl FromEnv for NetworkConfig { fn from_env() -> anyhow::Result { envy_load("network", "CHAIN_ETH_") @@ -58,62 +43,57 @@ mod tests { static MUTEX: EnvMutex = EnvMutex::new(); - fn expected_config() -> ChainConfig { - ChainConfig { - network: NetworkConfig { - network: "localhost".parse().unwrap(), - zksync_network: "localhost".to_string(), - zksync_network_id: L2ChainId::from(270), - }, - state_keeper: StateKeeperConfig { - transaction_slots: 50, - block_commit_deadline_ms: 2500, - miniblock_commit_deadline_ms: 1000, - miniblock_seal_queue_capacity: 10, - max_single_tx_gas: 1_000_000, - max_allowed_l2_tx_gas_limit: 2_000_000_000, - close_block_at_eth_params_percentage: 0.2, - close_block_at_gas_percentage: 0.8, - close_block_at_geometry_percentage: 0.5, - reject_tx_at_eth_params_percentage: 0.8, - reject_tx_at_geometry_percentage: 0.3, - fee_account_addr: addr("de03a0B5963f75f1C8485B355fF6D30f3093BDE7"), - reject_tx_at_gas_percentage: 0.5, - fair_l2_gas_price: 250000000, - validation_computational_gas_limit: 10_000_000, - save_call_traces: false, - virtual_blocks_interval: 1, - virtual_blocks_per_miniblock: 1, - upload_witness_inputs_to_gcs: false, - enum_index_migration_chunk_size: Some(2_000), - }, - operations_manager: OperationsManagerConfig { - delay_interval: 100, - }, - mempool: MempoolConfig { - sync_interval_ms: 10, - sync_batch_size: 1000, - capacity: 1_000_000, - stuck_tx_timeout: 10, - remove_stuck_txs: true, - delay_interval: 100, - }, - circuit_breaker: CircuitBreakerConfig { - sync_interval_ms: 1000, - http_req_max_retry_number: 5, - http_req_retry_interval_sec: 2, - replication_lag_limit_sec: Some(10), - }, + fn expected_network_config() -> NetworkConfig { + NetworkConfig { + network: "localhost".parse().unwrap(), + zksync_network: "localhost".to_string(), + zksync_network_id: L2ChainId::from(270), } } #[test] - fn from_env() { + fn network_from_env() { let mut lock = MUTEX.lock(); let config = r#" CHAIN_ETH_NETWORK="localhost" CHAIN_ETH_ZKSYNC_NETWORK="localhost" CHAIN_ETH_ZKSYNC_NETWORK_ID=270 + "#; + lock.set_env(config); + + let actual = NetworkConfig::from_env().unwrap(); + assert_eq!(actual, expected_network_config()); + } + + fn expected_state_keeper_config() -> StateKeeperConfig { + StateKeeperConfig { + transaction_slots: 50, + block_commit_deadline_ms: 2500, + miniblock_commit_deadline_ms: 1000, + miniblock_seal_queue_capacity: 10, + max_single_tx_gas: 1_000_000, + max_allowed_l2_tx_gas_limit: 2_000_000_000, + close_block_at_eth_params_percentage: 0.2, + close_block_at_gas_percentage: 0.8, + close_block_at_geometry_percentage: 0.5, + reject_tx_at_eth_params_percentage: 0.8, + reject_tx_at_geometry_percentage: 0.3, + fee_account_addr: addr("de03a0B5963f75f1C8485B355fF6D30f3093BDE7"), + reject_tx_at_gas_percentage: 0.5, + fair_l2_gas_price: 250000000, + validation_computational_gas_limit: 10_000_000, + save_call_traces: false, + virtual_blocks_interval: 1, + virtual_blocks_per_miniblock: 1, + upload_witness_inputs_to_gcs: false, + enum_index_migration_chunk_size: Some(2_000), + } + } + + #[test] + fn state_keeper_from_env() { + let mut lock = MUTEX.lock(); + let config = r#" CHAIN_STATE_KEEPER_TRANSACTION_SLOTS="50" CHAIN_STATE_KEEPER_FEE_ACCOUNT_ADDR="0xde03a0B5963f75f1C8485B355fF6D30f3093BDE7" CHAIN_STATE_KEEPER_MAX_SINGLE_TX_GAS="1000000" @@ -132,13 +112,72 @@ mod tests { CHAIN_STATE_KEEPER_SAVE_CALL_TRACES="false" CHAIN_STATE_KEEPER_UPLOAD_WITNESS_INPUTS_TO_GCS="false" CHAIN_STATE_KEEPER_ENUM_INDEX_MIGRATION_CHUNK_SIZE="2000" + "#; + lock.set_env(config); + + let actual = StateKeeperConfig::from_env().unwrap(); + assert_eq!(actual, expected_state_keeper_config()); + } + + fn expected_operations_manager_config() -> OperationsManagerConfig { + OperationsManagerConfig { + delay_interval: 100, + } + } + + #[test] + fn operations_manager_from_env() { + let mut lock = MUTEX.lock(); + let config = r#" CHAIN_OPERATIONS_MANAGER_DELAY_INTERVAL="100" + "#; + lock.set_env(config); + + let actual = OperationsManagerConfig::from_env().unwrap(); + assert_eq!(actual, expected_operations_manager_config()); + } + + fn expected_mempool_config() -> MempoolConfig { + MempoolConfig { + sync_interval_ms: 10, + sync_batch_size: 1000, + capacity: 1_000_000, + stuck_tx_timeout: 10, + remove_stuck_txs: true, + delay_interval: 100, + } + } + + #[test] + fn mempool_from_env() { + let mut lock = MUTEX.lock(); + let config = r#" CHAIN_MEMPOOL_SYNC_INTERVAL_MS="10" CHAIN_MEMPOOL_SYNC_BATCH_SIZE="1000" CHAIN_MEMPOOL_STUCK_TX_TIMEOUT="10" CHAIN_MEMPOOL_REMOVE_STUCK_TXS="true" CHAIN_MEMPOOL_DELAY_INTERVAL="100" CHAIN_MEMPOOL_CAPACITY="1000000" + "#; + lock.set_env(config); + + let actual = MempoolConfig::from_env().unwrap(); + assert_eq!(actual, expected_mempool_config()); + } + + fn expected_circuit_breaker_config() -> CircuitBreakerConfig { + CircuitBreakerConfig { + sync_interval_ms: 1000, + http_req_max_retry_number: 5, + http_req_retry_interval_sec: 2, + replication_lag_limit_sec: Some(10), + } + } + + #[test] + fn circuit_breaker_from_env() { + let mut lock = MUTEX.lock(); + let config = r#" CHAIN_CIRCUIT_BREAKER_SYNC_INTERVAL_MS="1000" CHAIN_CIRCUIT_BREAKER_HTTP_REQ_MAX_RETRY_NUMBER="5" CHAIN_CIRCUIT_BREAKER_HTTP_REQ_RETRY_INTERVAL_SEC="2" @@ -146,7 +185,7 @@ mod tests { "#; lock.set_env(config); - let actual = ChainConfig::from_env().unwrap(); - assert_eq!(actual, expected_config()); + let actual = CircuitBreakerConfig::from_env().unwrap(); + assert_eq!(actual, expected_circuit_breaker_config()); } } From e5fbcb5dfc2a7d2582f40a481c861fb2f4dd5fb0 Mon Sep 17 00:00:00 2001 From: Stanislav Bezkorovainyi Date: Fri, 5 Jan 2024 11:52:00 +0100 Subject: [PATCH 14/49] feat(vm): Make utils version-dependent (#809) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ In the new 1.4.1 version, the fee formulas as well as max txs per batch will change. We'll prepare the main branch for it first to reduce the diff ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .../src/intrinsic_costs.rs | 7 +- .../system-constants-generator/src/main.rs | 25 ++- core/lib/constants/src/crypto.rs | 2 - core/lib/multivm/src/lib.rs | 1 + core/lib/multivm/src/utils.rs | 164 ++++++++++++++++++ .../src/versions/vm_1_3_2/transaction_data.rs | 6 +- .../src/versions/vm_1_3_2/vm_instance.rs | 2 +- .../versions/vm_1_3_2/vm_with_bootloader.rs | 11 +- .../vm_boojum_integration/constants.rs | 7 +- .../vm_boojum_integration/utils/fee.rs | 5 +- .../vm_boojum_integration/utils/overhead.rs | 6 +- .../src/versions/vm_latest/constants.rs | 7 +- .../src/versions/vm_latest/utils/fee.rs | 22 ++- .../src/versions/vm_latest/utils/overhead.rs | 6 +- .../src/versions/vm_m5/transaction_data.rs | 10 +- .../multivm/src/versions/vm_m5/vm_instance.rs | 2 +- .../src/versions/vm_m5/vm_with_bootloader.rs | 11 +- .../src/versions/vm_m6/transaction_data.rs | 5 +- .../multivm/src/versions/vm_m6/vm_instance.rs | 2 +- .../src/versions/vm_m6/vm_with_bootloader.rs | 11 +- .../vm_refunds_enhancement/constants.rs | 7 +- .../vm_refunds_enhancement/utils/fee.rs | 5 +- .../vm_refunds_enhancement/utils/overhead.rs | 6 +- .../versions/vm_virtual_blocks/constants.rs | 7 +- .../versions/vm_virtual_blocks/utils/fee.rs | 5 +- .../vm_virtual_blocks/utils/overhead.rs | 6 +- core/lib/types/src/protocol_version.rs | 4 + .../src/api_server/execution_sandbox/apply.rs | 16 ++ .../api_server/execution_sandbox/execute.rs | 35 +--- .../src/api_server/execution_sandbox/mod.rs | 29 +--- .../api_server/execution_sandbox/validate.rs | 38 +--- .../src/api_server/tx_sender/mod.rs | 110 ++++++++---- .../web3/backend_jsonrpsee/namespaces/eth.rs | 2 +- .../src/api_server/web3/namespaces/eth.rs | 6 +- .../src/state_keeper/io/mempool.rs | 41 +++-- .../src/state_keeper/io/tests/mod.rs | 11 +- .../src/state_keeper/mempool_actor.rs | 24 ++- core/lib/zksync_core/src/state_keeper/mod.rs | 8 - .../seal_criteria/criteria/slots.rs | 10 +- .../criteria/tx_encoding_size.rs | 22 ++- 40 files changed, 482 insertions(+), 222 deletions(-) create mode 100644 core/lib/multivm/src/utils.rs diff --git a/core/bin/system-constants-generator/src/intrinsic_costs.rs b/core/bin/system-constants-generator/src/intrinsic_costs.rs index d09da8c5b252..697d539f0c20 100644 --- a/core/bin/system-constants-generator/src/intrinsic_costs.rs +++ b/core/bin/system-constants-generator/src/intrinsic_costs.rs @@ -4,8 +4,8 @@ //! as well as contracts/SystemConfig.json //! -use multivm::vm_latest::constants::BOOTLOADER_TX_ENCODING_SPACE; -use zksync_types::{ethabi::Address, IntrinsicSystemGasConstants, U256}; +use multivm::utils::get_bootloader_encoding_space; +use zksync_types::{ethabi::Address, IntrinsicSystemGasConstants, ProtocolVersionId, U256}; use crate::utils::{ execute_internal_transfer_test, execute_user_txs_in_test_gas_vm, get_l1_tx, get_l1_txs, @@ -129,7 +129,8 @@ pub(crate) fn l2_gas_constants() -> IntrinsicSystemGasConstants { tx_with_more_factory_deps_result.pubdata_published - empty_l1_tx_result.pubdata_published; // The number of the bootloader memory that can be filled up with transactions. - let bootloader_tx_memory_size_slots = BOOTLOADER_TX_ENCODING_SPACE; + let bootloader_tx_memory_size_slots = + get_bootloader_encoding_space(ProtocolVersionId::latest().into()); IntrinsicSystemGasConstants { l2_tx_intrinsic_gas, diff --git a/core/bin/system-constants-generator/src/main.rs b/core/bin/system-constants-generator/src/main.rs index e6009a7a393d..0557eccd9d6e 100644 --- a/core/bin/system-constants-generator/src/main.rs +++ b/core/bin/system-constants-generator/src/main.rs @@ -1,8 +1,9 @@ use std::fs; use codegen::{Block, Scope}; -use multivm::vm_latest::constants::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_L1_GAS, BOOTLOADER_TX_ENCODING_SPACE, MAX_PUBDATA_PER_BLOCK, +use multivm::{ + utils::{get_bootloader_encoding_space, get_bootloader_max_txs_in_batch}, + vm_latest::constants::{BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_L1_GAS, MAX_PUBDATA_PER_BLOCK}, }; use serde::{Deserialize, Serialize}; use zksync_types::{ @@ -13,8 +14,8 @@ use zksync_types::{ }, system_params::MAX_TX_ERGS_LIMIT, }, - IntrinsicSystemGasConstants, GUARANTEED_PUBDATA_IN_TX, L1_GAS_PER_PUBDATA_BYTE, - MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + IntrinsicSystemGasConstants, ProtocolVersionId, GUARANTEED_PUBDATA_IN_TX, + L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, }; mod intrinsic_costs; @@ -57,8 +58,12 @@ pub fn generate_l1_contracts_system_config(gas_constants: &IntrinsicSystemGasCon l1_gas_per_pubdata_byte: L1_GAS_PER_PUBDATA_BYTE, block_overhead_l2_gas: BLOCK_OVERHEAD_GAS, block_overhead_l1_gas: BLOCK_OVERHEAD_L1_GAS, - max_transactions_in_block: MAX_TXS_IN_BLOCK as u32, - bootloader_tx_encoding_space: BOOTLOADER_TX_ENCODING_SPACE, + max_transactions_in_block: get_bootloader_max_txs_in_batch( + ProtocolVersionId::latest().into(), + ) as u32, + bootloader_tx_encoding_space: get_bootloader_encoding_space( + ProtocolVersionId::latest().into(), + ), l1_tx_intrinsic_l2_gas: gas_constants.l1_tx_intrinsic_gas, l1_tx_intrinsic_pubdata: gas_constants.l1_tx_intrinsic_pubdata, @@ -98,7 +103,9 @@ pub fn generate_l2_contracts_system_config(gas_constants: &IntrinsicSystemGasCon let l2_contracts_config = L2SystemConfig { guaranteed_pubdata_bytes: GUARANTEED_PUBDATA_IN_TX, max_pubdata_per_block: MAX_PUBDATA_PER_BLOCK, - max_transactions_in_block: MAX_TXS_IN_BLOCK as u32, + max_transactions_in_block: get_bootloader_max_txs_in_batch( + ProtocolVersionId::latest().into(), + ) as u32, block_overhead_l2_gas: BLOCK_OVERHEAD_GAS, block_overhead_l1_gas: BLOCK_OVERHEAD_L1_GAS, l2_tx_intrinsic_gas: gas_constants.l2_tx_intrinsic_gas, @@ -106,7 +113,9 @@ pub fn generate_l2_contracts_system_config(gas_constants: &IntrinsicSystemGasCon l1_tx_intrinsic_l2_gas: gas_constants.l1_tx_intrinsic_gas, l1_tx_intrinsic_pubdata: gas_constants.l1_tx_intrinsic_pubdata, max_gas_per_transaction: MAX_TX_ERGS_LIMIT, - bootloader_memory_for_txs: BOOTLOADER_TX_ENCODING_SPACE, + bootloader_memory_for_txs: get_bootloader_encoding_space( + ProtocolVersionId::latest().into(), + ), refund_gas: gas_constants.l2_tx_gas_for_refund_transfer, keccak_round_cost_gas: KECCAK256_CIRCUIT_COST_IN_ERGS, sha256_round_cost_gas: SHA256_CIRCUIT_COST_IN_ERGS, diff --git a/core/lib/constants/src/crypto.rs b/core/lib/constants/src/crypto.rs index 53a5bb98b796..a2a7d8d24bf4 100644 --- a/core/lib/constants/src/crypto.rs +++ b/core/lib/constants/src/crypto.rs @@ -19,8 +19,6 @@ pub const MAX_BYTES_PER_PACKED_SLOT: u64 = 65; pub static GAS_PER_SLOT: Lazy = Lazy::new(|| BigUint::from(MAX_BYTES_PER_PACKED_SLOT) * BigUint::from(GAS_PER_PUBDATA_BYTE)); -pub const MAX_TXS_IN_BLOCK: usize = 1024; - pub const MAX_NEW_FACTORY_DEPS: usize = 32; pub const PAD_MSG_BEFORE_HASH_BITS_LEN: usize = 736; diff --git a/core/lib/multivm/src/lib.rs b/core/lib/multivm/src/lib.rs index a13d7cb7d14a..bed4586d428e 100644 --- a/core/lib/multivm/src/lib.rs +++ b/core/lib/multivm/src/lib.rs @@ -23,5 +23,6 @@ pub use crate::{ mod glue; pub mod interface; pub mod tracers; +pub mod utils; pub mod versions; mod vm_instance; diff --git a/core/lib/multivm/src/utils.rs b/core/lib/multivm/src/utils.rs new file mode 100644 index 000000000000..4bab29f14b56 --- /dev/null +++ b/core/lib/multivm/src/utils.rs @@ -0,0 +1,164 @@ +use zksync_types::{VmVersion, U256}; + +/// Calculates the base fee and gas per pubdata for the given L1 gas price. +pub fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, + vm_version: VmVersion, +) -> (u64, u64) { + match vm_version { + VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { + crate::vm_m5::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { + crate::vm_m6::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + VmVersion::Vm1_3_2 => { + crate::vm_1_3_2::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + VmVersion::VmVirtualBlocks => { + crate::vm_virtual_blocks::utils::fee::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + VmVersion::VmVirtualBlocksRefundsEnhancement => { + crate::vm_refunds_enhancement::utils::fee::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + VmVersion::VmBoojumIntegration => { + crate::vm_boojum_integration::utils::fee::derive_base_fee_and_gas_per_pubdata( + l1_gas_price, + fair_gas_price, + ) + } + } +} + +/// Changes the fee model output so that the expected gas per pubdata is smaller than or the `tx_gas_per_pubdata_limit`. +pub fn adjust_l1_gas_price_for_tx( + l1_gas_price: u64, + fair_l2_gas_price: u64, + tx_gas_per_pubdata_limit: U256, + vm_version: VmVersion, +) -> u64 { + if U256::from( + derive_base_fee_and_gas_per_pubdata(l1_gas_price, fair_l2_gas_price, vm_version).1, + ) <= tx_gas_per_pubdata_limit + { + return l1_gas_price; + } + + // The latest VM supports adjusting the pubdata price for all the types of the fee models. + crate::vm_latest::utils::fee::adjust_l1_gas_price_for_tx( + fair_l2_gas_price, + tx_gas_per_pubdata_limit, + ) +} + +pub fn derive_overhead( + gas_limit: u32, + gas_price_per_pubdata: u32, + encoded_len: usize, + tx_type: u8, + vm_version: VmVersion, +) -> u32 { + match vm_version { + VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { + crate::vm_m5::transaction_data::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + ) + } + VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { + crate::vm_m6::transaction_data::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + crate::vm_m6::transaction_data::OverheadCoefficients::from_tx_type(tx_type), + ) + } + VmVersion::Vm1_3_2 => crate::vm_1_3_2::transaction_data::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + crate::vm_1_3_2::transaction_data::OverheadCoefficients::from_tx_type(tx_type), + ), + VmVersion::VmVirtualBlocks => crate::vm_virtual_blocks::utils::overhead::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + crate::vm_virtual_blocks::utils::overhead::OverheadCoefficients::from_tx_type(tx_type), + ), + VmVersion::VmVirtualBlocksRefundsEnhancement => { + crate::vm_refunds_enhancement::utils::overhead::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + crate::vm_refunds_enhancement::utils::overhead::OverheadCoefficients::from_tx_type( + tx_type, + ), + ) + } + VmVersion::VmBoojumIntegration => { + crate::vm_boojum_integration::utils::overhead::derive_overhead( + gas_limit, + gas_price_per_pubdata, + encoded_len, + crate::vm_boojum_integration::utils::overhead::OverheadCoefficients::from_tx_type( + tx_type, + ), + ) + } + } +} + +pub fn get_bootloader_encoding_space(version: VmVersion) -> u32 { + match version { + VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { + crate::vm_m5::vm_with_bootloader::BOOTLOADER_TX_ENCODING_SPACE + } + VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { + crate::vm_m6::vm_with_bootloader::BOOTLOADER_TX_ENCODING_SPACE + } + VmVersion::Vm1_3_2 => crate::vm_1_3_2::vm_with_bootloader::BOOTLOADER_TX_ENCODING_SPACE, + VmVersion::VmVirtualBlocks => { + crate::vm_virtual_blocks::constants::BOOTLOADER_TX_ENCODING_SPACE + } + VmVersion::VmVirtualBlocksRefundsEnhancement => { + crate::vm_refunds_enhancement::constants::BOOTLOADER_TX_ENCODING_SPACE + } + VmVersion::VmBoojumIntegration => { + crate::vm_boojum_integration::constants::BOOTLOADER_TX_ENCODING_SPACE + } + } +} + +pub fn get_bootloader_max_txs_in_batch(version: VmVersion) -> usize { + match version { + VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { + crate::vm_m5::vm_with_bootloader::MAX_TXS_IN_BLOCK + } + VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { + crate::vm_m6::vm_with_bootloader::MAX_TXS_IN_BLOCK + } + VmVersion::Vm1_3_2 => crate::vm_1_3_2::vm_with_bootloader::MAX_TXS_IN_BLOCK, + VmVersion::VmVirtualBlocks => crate::vm_virtual_blocks::constants::MAX_TXS_IN_BLOCK, + VmVersion::VmVirtualBlocksRefundsEnhancement => { + crate::vm_refunds_enhancement::constants::MAX_TXS_IN_BLOCK + } + VmVersion::VmBoojumIntegration => crate::vm_boojum_integration::constants::MAX_TXS_IN_BLOCK, + } +} diff --git a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs index f2c8f278f56d..d1a22f1a40f2 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs @@ -4,14 +4,14 @@ use zksync_types::{ fee::encoding_len, l1::is_l1_tx_type, l2::TransactionType, - ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK, U256, + ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, U256, }; use zksync_utils::{ address_to_h256, bytecode::hash_bytecode, bytes_to_be_words, ceil_div_u256, h256_to_u256, }; use crate::vm_1_3_2::vm_with_bootloader::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; // This structure represents the data that is used by @@ -229,7 +229,7 @@ impl TransactionData { } } -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs index 2217b4f50d66..5f48a276ac79 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs @@ -10,7 +10,6 @@ use zk_evm_1_3_3::{ }, }; use zksync_state::WriteStorage; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ l2_to_l1_log::{L2ToL1Log, UserL2ToL1Log}, tx::tx_execution_info::TxExecutionStatus, @@ -162,6 +161,7 @@ pub enum VmExecutionStopReason { TracerRequestedStop, } +use super::vm_with_bootloader::MAX_TXS_IN_BLOCK; use crate::vm_1_3_2::utils::VmExecutionResult as NewVmExecutionResult; fn vm_may_have_ended_inner( diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs index 71c108cae326..b8db8a480da3 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs @@ -13,7 +13,6 @@ use zk_evm_1_3_3::{ }; use zksync_contracts::BaseSystemContracts; use zksync_state::WriteStorage; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ l1::is_l1_tx_type, zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, @@ -84,7 +83,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { ceil_div(eth_price_per_pubdata_byte, base_fee) } -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to @@ -109,6 +111,9 @@ impl From for DerivedBlockContext { } } +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + // The first 32 slots are reserved for debugging purposes pub const DEBUG_SLOTS_OFFSET: usize = 8; pub const DEBUG_FIRST_SLOTS: usize = 32; @@ -151,7 +156,7 @@ pub const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = COMPRESSED_BYTECODES_OFFSET + COMPRESSED_BYTECODES_SLOTS; // The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs index 705327468875..7fc196974dc4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/constants.rs @@ -3,12 +3,15 @@ pub use zk_evm_1_4_0::zkevm_opcode_defs::system_params::{ ERGS_PER_CIRCUIT, INITIAL_STORAGE_WRITE_PUBDATA_BYTES, MAX_PUBDATA_PER_BLOCK, }; use zksync_system_constants::{ - L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, USED_BOOTLOADER_MEMORY_WORDS, }; use crate::vm_boojum_integration::old_vm::utils::heap_page_from_base; +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + /// Max cycles for a single transaction. pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; @@ -69,7 +72,7 @@ pub(crate) const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET + OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS; /// The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs index 9a2e58c72993..c19b401e3fce 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs @@ -12,7 +12,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } /// Calculates the base fee and gas per pubdata for the given L1 gas price. -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs index 5f50f664c234..98b1c460c2ac 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/overhead.rs @@ -1,14 +1,14 @@ use zk_evm_1_4_0::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; -use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; +use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{l1::is_l1_tx_type, U256}; use zksync_utils::ceil_div_u256; use crate::vm_boojum_integration::constants::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; /// Derives the overhead for processing transactions in a block. -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/multivm/src/versions/vm_latest/constants.rs b/core/lib/multivm/src/versions/vm_latest/constants.rs index 44266344be61..944000178825 100644 --- a/core/lib/multivm/src/versions/vm_latest/constants.rs +++ b/core/lib/multivm/src/versions/vm_latest/constants.rs @@ -3,12 +3,15 @@ pub use zk_evm_1_4_0::zkevm_opcode_defs::system_params::{ ERGS_PER_CIRCUIT, INITIAL_STORAGE_WRITE_PUBDATA_BYTES, MAX_PUBDATA_PER_BLOCK, }; use zksync_system_constants::{ - L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, USED_BOOTLOADER_MEMORY_WORDS, }; use crate::vm_latest::old_vm::utils::heap_page_from_base; +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + /// Max cycles for a single transaction. pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; @@ -69,7 +72,7 @@ pub(crate) const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET + OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_SLOTS; /// The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs index 23b744a348f7..ef0b26eeea1a 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs @@ -1,5 +1,6 @@ //! Utility functions for vm use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; +use zksync_types::U256; use zksync_utils::ceil_div; use crate::vm_latest::old_vm::utils::eth_price_per_pubdata_byte; @@ -12,7 +13,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } /// Calculates the base fee and gas per pubdata for the given L1 gas price. -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to @@ -27,3 +31,19 @@ pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u6 base_fee_to_gas_per_pubdata(l1_gas_price, base_fee), ) } + +/// Changes the fee model output so that the expected gas per pubdata is smaller than or the `tx_gas_per_pubdata_limit`. +/// This function expects that the currently expected gas per pubdata is greater than the `tx_gas_per_pubdata_limit`. +pub(crate) fn adjust_l1_gas_price_for_tx( + fair_l2_gas_price: u64, + tx_gas_per_pubdata_limit: U256, +) -> u64 { + // gasPerPubdata = ceil(17 * l1gasprice / fair_l2_gas_price) + // gasPerPubdata <= 17 * l1gasprice / fair_l2_gas_price + 1 + // fair_l2_gas_price(gasPerPubdata - 1) / 17 <= l1gasprice + let l1_gas_price = U256::from(fair_l2_gas_price) + * (tx_gas_per_pubdata_limit - U256::from(1u32)) + / U256::from(17); + + l1_gas_price.as_u64() +} diff --git a/core/lib/multivm/src/versions/vm_latest/utils/overhead.rs b/core/lib/multivm/src/versions/vm_latest/utils/overhead.rs index 38aaede8d4bb..612070ec870a 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/overhead.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/overhead.rs @@ -1,14 +1,14 @@ use zk_evm_1_4_0::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; -use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; +use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{l1::is_l1_tx_type, U256}; use zksync_utils::ceil_div_u256; use crate::vm_latest::constants::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; /// Derives the overhead for processing transactions in a block. -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs index f150db2ebaa9..2069f190598b 100644 --- a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs @@ -3,14 +3,14 @@ use zksync_types::{ ethabi::{encode, Address, Token}, fee::encoding_len, l2::TransactionType, - ExecuteTransactionCommon, Transaction, MAX_TXS_IN_BLOCK, U256, + ExecuteTransactionCommon, Transaction, U256, }; use zksync_utils::{ address_to_h256, bytecode::hash_bytecode, bytes_to_be_words, ceil_div_u256, h256_to_u256, }; use crate::vm_m5::vm_with_bootloader::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; const L1_TX_TYPE: u8 = 255; @@ -216,7 +216,11 @@ impl TransactionData { // } } -pub fn derive_overhead(gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize) -> u32 { +pub(crate) fn derive_overhead( + gas_limit: u32, + gas_price_per_pubdata: u32, + encoded_len: usize, +) -> u32 { assert!( gas_limit <= MAX_TX_ERGS_LIMIT, "gas limit is larger than the maximal one" diff --git a/core/lib/multivm/src/versions/vm_m5/vm_instance.rs b/core/lib/multivm/src/versions/vm_m5/vm_instance.rs index 99a96ded4d47..eafac7356d3d 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_instance.rs @@ -9,7 +9,6 @@ use zk_evm_1_3_1::{ definitions::RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER, }, }; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ l2_to_l1_log::{L2ToL1Log, UserL2ToL1Log}, tx::tx_execution_info::TxExecutionStatus, @@ -176,6 +175,7 @@ pub enum VmExecutionStopReason { TracerRequestedStop, } +use super::vm_with_bootloader::MAX_TXS_IN_BLOCK; use crate::vm_m5::utils::VmExecutionResult as NewVmExecutionResult; fn vm_may_have_ended_inner( diff --git a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs index f9ba88fea143..8826db6171cd 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs @@ -11,7 +11,6 @@ use zk_evm_1_3_1::{ }, }; use zksync_contracts::BaseSystemContracts; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, @@ -73,7 +72,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { ceil_div(eth_price_per_pubdata_byte, base_fee) } -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible to a transaciton to @@ -98,6 +100,9 @@ impl From for DerivedBlockContext { } } +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + // The first 32 slots are reserved for debugging purposes pub const DEBUG_SLOTS_OFFSET: usize = 8; pub const DEBUG_FIRST_SLOTS: usize = 32; @@ -132,7 +137,7 @@ pub const TX_OVERHEAD_SLOTS: usize = MAX_TXS_IN_BLOCK; pub const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = TX_OVERHEAD_OFFSET + TX_OVERHEAD_SLOTS; // The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (MAX_HEAP_PAGE_SIZE_IN_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs index 136a6d7647ac..3536159f658e 100644 --- a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs @@ -4,12 +4,13 @@ use zksync_types::{ fee::encoding_len, l1::is_l1_tx_type, l2::TransactionType, - ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK, U256, + ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, U256, }; use zksync_utils::{ address_to_h256, bytecode::hash_bytecode, bytes_to_be_words, ceil_div_u256, h256_to_u256, }; +use super::vm_with_bootloader::MAX_TXS_IN_BLOCK; use crate::vm_m6::vm_with_bootloader::{ BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, }; @@ -231,7 +232,7 @@ impl TransactionData { } } -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/multivm/src/versions/vm_m6/vm_instance.rs b/core/lib/multivm/src/versions/vm_m6/vm_instance.rs index 379476d7664c..3c77c1159149 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_instance.rs @@ -9,7 +9,6 @@ use zk_evm_1_3_1::{ definitions::RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER, }, }; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ l2_to_l1_log::{L2ToL1Log, UserL2ToL1Log}, tx::tx_execution_info::TxExecutionStatus, @@ -180,6 +179,7 @@ pub enum VmExecutionStopReason { TracerRequestedStop, } +use super::vm_with_bootloader::MAX_TXS_IN_BLOCK; use crate::vm_m6::utils::VmExecutionResult as NewVmExecutionResult; fn vm_may_have_ended_inner( diff --git a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs index c7d4ee3d45e8..66b2f3d07d78 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs @@ -11,7 +11,6 @@ use zk_evm_1_3_1::{ }, }; use zksync_contracts::BaseSystemContracts; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use zksync_types::{ zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, @@ -82,7 +81,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { ceil_div(eth_price_per_pubdata_byte, base_fee) } -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to @@ -107,6 +109,9 @@ impl From for DerivedBlockContext { } } +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + // The first 32 slots are reserved for debugging purposes pub const DEBUG_SLOTS_OFFSET: usize = 8; pub const DEBUG_FIRST_SLOTS: usize = 32; @@ -149,7 +154,7 @@ pub const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = COMPRESSED_BYTECODES_OFFSET + COMPRESSED_BYTECODES_SLOTS; // The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (MAX_HEAP_PAGE_SIZE_IN_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/constants.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/constants.rs index 82ab754e4036..25e89bfb1809 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/constants.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/constants.rs @@ -3,12 +3,15 @@ pub use zk_evm_1_3_3::zkevm_opcode_defs::system_params::{ ERGS_PER_CIRCUIT, INITIAL_STORAGE_WRITE_PUBDATA_BYTES, MAX_PUBDATA_PER_BLOCK, }; use zksync_system_constants::{ - L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, USED_BOOTLOADER_MEMORY_WORDS, }; use crate::vm_refunds_enhancement::old_vm::utils::heap_page_from_base; +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + /// Max cycles for a single transaction. pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; @@ -52,7 +55,7 @@ pub(crate) const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = COMPRESSED_BYTECODES_OFFSET + COMPRESSED_BYTECODES_SLOTS; /// The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs index cc6081d7a229..6df858efbf7c 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs @@ -12,7 +12,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } /// Calculates the base fee and gas per pubdata for the given L1 gas price. -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/overhead.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/overhead.rs index ab5149a050fb..401999402009 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/overhead.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/overhead.rs @@ -1,14 +1,14 @@ use zk_evm_1_3_3::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; -use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; +use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{l1::is_l1_tx_type, U256}; use zksync_utils::ceil_div_u256; use crate::vm_refunds_enhancement::constants::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; /// Derives the overhead for processing transactions in a block. -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/constants.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/constants.rs index c03260f1b6de..1993474bb4ed 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/constants.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/constants.rs @@ -3,12 +3,15 @@ pub use zk_evm_1_3_3::zkevm_opcode_defs::system_params::{ ERGS_PER_CIRCUIT, INITIAL_STORAGE_WRITE_PUBDATA_BYTES, MAX_PUBDATA_PER_BLOCK, }; use zksync_system_constants::{ - L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, MAX_TXS_IN_BLOCK, + L1_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, USED_BOOTLOADER_MEMORY_WORDS, }; use crate::vm_virtual_blocks::old_vm::utils::heap_page_from_base; +// The maximal number of transactions in a single batch +pub(crate) const MAX_TXS_IN_BLOCK: usize = 1024; + /// Max cycles for a single transaction. pub const MAX_CYCLES_FOR_TX: u32 = u32::MAX; @@ -52,7 +55,7 @@ pub(crate) const BOOTLOADER_TX_DESCRIPTION_OFFSET: usize = COMPRESSED_BYTECODES_OFFSET + COMPRESSED_BYTECODES_SLOTS; /// The size of the bootloader memory dedicated to the encodings of transactions -pub const BOOTLOADER_TX_ENCODING_SPACE: u32 = +pub(crate) const BOOTLOADER_TX_ENCODING_SPACE: u32 = (USED_BOOTLOADER_MEMORY_WORDS - TX_DESCRIPTION_OFFSET - MAX_TXS_IN_BLOCK) as u32; // Size of the bootloader tx description in words diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs index 6753e8197813..2762eb447161 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs @@ -12,7 +12,10 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } /// Calculates the base fee and gas per pubdata for the given L1 gas price. -pub fn derive_base_fee_and_gas_per_pubdata(l1_gas_price: u64, fair_gas_price: u64) -> (u64, u64) { +pub(crate) fn derive_base_fee_and_gas_per_pubdata( + l1_gas_price: u64, + fair_gas_price: u64, +) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The baseFee is set in such a way that it is always possible for a transaction to diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/overhead.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/overhead.rs index 79c52ac373b6..862154b7ad62 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/overhead.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/overhead.rs @@ -1,14 +1,14 @@ use zk_evm_1_3_3::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; -use zksync_system_constants::{MAX_L2_TX_GAS_LIMIT, MAX_TXS_IN_BLOCK}; +use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{l1::is_l1_tx_type, U256}; use zksync_utils::ceil_div_u256; use crate::vm_virtual_blocks::constants::{ - BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, + BLOCK_OVERHEAD_GAS, BLOCK_OVERHEAD_PUBDATA, BOOTLOADER_TX_ENCODING_SPACE, MAX_TXS_IN_BLOCK, }; /// Derives the overhead for processing transactions in a block. -pub fn derive_overhead( +pub(crate) fn derive_overhead( gas_limit: u32, gas_price_per_pubdata: u32, encoded_len: usize, diff --git a/core/lib/types/src/protocol_version.rs b/core/lib/types/src/protocol_version.rs index bc13ccc6a893..dabb9d127545 100644 --- a/core/lib/types/src/protocol_version.rs +++ b/core/lib/types/src/protocol_version.rs @@ -84,6 +84,10 @@ impl ProtocolVersionId { pub fn is_pre_boojum(&self) -> bool { self < &ProtocolVersionId::Version18 } + + pub fn last_pre_boojum() -> Self { + ProtocolVersionId::Version17 + } } impl Default for ProtocolVersionId { diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs index 4360519d1e51..13b215d047f4 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs @@ -10,6 +10,7 @@ use std::time::{Duration, Instant}; use multivm::{ interface::{L1BatchEnv, L2BlockEnv, SystemEnv, VmInterface}, + utils::adjust_l1_gas_price_for_tx, vm_latest::{constants::BLOCK_GAS_LIMIT, HistoryDisabled}, VmInstance, }; @@ -38,6 +39,10 @@ use super::{ pub(super) fn apply_vm_in_sandbox( vm_permit: VmPermit, shared_args: TxSharedArgs, + // If `true`, then the batch's L1/pubdata gas price will be adjusted so that the transaction's gas per pubdata limit is <= + // to the one in the block. This is often helpful in case we want the transaction validation to work regardless of the + // current L1 prices for gas or pubdata. + adjust_pubdata_price: bool, execution_args: &TxExecutionArgs, connection_pool: &ConnectionPool, tx: Transaction, @@ -183,6 +188,17 @@ pub(super) fn apply_vm_in_sandbox( .. } = shared_args; + let l1_gas_price = if adjust_pubdata_price { + adjust_l1_gas_price_for_tx( + l1_gas_price, + fair_l2_gas_price, + tx.gas_per_pubdata_byte_limit(), + protocol_version.into(), + ) + } else { + l1_gas_price + }; + let system_env = SystemEnv { zk_porter_available: ZKPORTER_IS_AVAILABLE, version: protocol_version, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs index c900dd4e5a5a..b915866ebe64 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs @@ -97,6 +97,7 @@ pub(crate) async fn execute_tx_eth_call( let (vm_result, _) = execute_tx_in_sandbox( vm_permit, shared_args, + false, execution_args, connection_pool, tx.into(), @@ -108,40 +109,17 @@ pub(crate) async fn execute_tx_eth_call( vm_result } -#[tracing::instrument(skip_all)] -pub(crate) async fn execute_tx_with_pending_state( - vm_permit: VmPermit, - mut shared_args: TxSharedArgs, - execution_args: TxExecutionArgs, - connection_pool: ConnectionPool, - tx: Transaction, -) -> (VmExecutionResultAndLogs, TransactionExecutionMetrics) { - let mut connection = connection_pool.access_storage_tagged("api").await.unwrap(); - let block_args = BlockArgs::pending(&mut connection).await; - drop(connection); - // In order for execution to pass smoothlessly, we need to ensure that block's required gasPerPubdata will be - // <= to the one in the transaction itself. - shared_args.adjust_l1_gas_price(tx.gas_per_pubdata_byte_limit()); - - execute_tx_in_sandbox( - vm_permit, - shared_args, - execution_args, - connection_pool, - tx, - block_args, - vec![], - ) - .await -} - /// This method assumes that (block with number `resolved_block_number` is present in DB) /// or (`block_id` is `pending` and block with number `resolved_block_number - 1` is present in DB) #[allow(clippy::too_many_arguments)] #[tracing::instrument(skip_all)] -async fn execute_tx_in_sandbox( +pub(crate) async fn execute_tx_in_sandbox( vm_permit: VmPermit, shared_args: TxSharedArgs, + // If `true`, then the batch's L1/pubdata gas price will be adjusted so that the transaction's gas per pubdata limit is <= + // to the one in the block. This is often helpful in case we want the transaction validation to work regardless of the + // current L1 prices for gas or pubdata. + adjust_pubdata_price: bool, execution_args: TxExecutionArgs, connection_pool: ConnectionPool, tx: Transaction, @@ -159,6 +137,7 @@ async fn execute_tx_in_sandbox( let result = apply::apply_vm_in_sandbox( vm_permit, shared_args, + adjust_pubdata_price, &execution_args, &connection_pool, tx, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs index 461be71c089f..8c62bd3f498d 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs @@ -1,17 +1,16 @@ use std::{sync::Arc, time::Duration}; -use multivm::vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata; use tokio::runtime::Handle; use zksync_dal::{ConnectionPool, SqlxError, StorageProcessor}; use zksync_state::{PostgresStorage, PostgresStorageCaches, ReadStorage, StorageView}; use zksync_system_constants::PUBLISH_BYTECODE_OVERHEAD; -use zksync_types::{api, AccountTreeId, L2ChainId, MiniblockNumber, U256}; +use zksync_types::{api, AccountTreeId, L2ChainId, MiniblockNumber}; use zksync_utils::bytecode::{compress_bytecode, hash_bytecode}; use self::vm_metrics::SandboxStage; pub(super) use self::{ error::SandboxExecutionError, - execute::{execute_tx_eth_call, execute_tx_with_pending_state, TxExecutionArgs}, + execute::{execute_tx_eth_call, execute_tx_in_sandbox, TxExecutionArgs}, tracers::ApiTracer, vm_metrics::{SubmitTxStage, SANDBOX_METRICS}, }; @@ -141,28 +140,6 @@ impl VmConcurrencyLimiter { } } -pub(super) fn adjust_l1_gas_price_for_tx( - l1_gas_price: u64, - fair_l2_gas_price: u64, - tx_gas_per_pubdata_limit: U256, -) -> u64 { - let (_, current_pubdata_price) = - derive_base_fee_and_gas_per_pubdata(l1_gas_price, fair_l2_gas_price); - if U256::from(current_pubdata_price) <= tx_gas_per_pubdata_limit { - // The current pubdata price is small enough - l1_gas_price - } else { - // gasPerPubdata = ceil(17 * l1gasprice / fair_l2_gas_price) - // gasPerPubdata <= 17 * l1gasprice / fair_l2_gas_price + 1 - // fair_l2_gas_price(gasPerPubdata - 1) / 17 <= l1gasprice - let l1_gas_price = U256::from(fair_l2_gas_price) - * (tx_gas_per_pubdata_limit - U256::from(1u32)) - / U256::from(17); - - l1_gas_price.as_u64() - } -} - async fn get_pending_state( connection: &mut StorageProcessor<'_>, ) -> (api::BlockId, MiniblockNumber) { @@ -241,7 +218,7 @@ pub(crate) struct BlockArgs { } impl BlockArgs { - async fn pending(connection: &mut StorageProcessor<'_>) -> Self { + pub(crate) async fn pending(connection: &mut StorageProcessor<'_>) -> Self { let (block_id, resolved_block_number) = get_pending_state(connection).await; Self { block_id, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs index df70d02fe44b..f2ad317800b5 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/validate.rs @@ -10,47 +10,16 @@ use multivm::{ MultiVMTracer, }; use zksync_dal::{ConnectionPool, StorageProcessor}; -use zksync_types::{l2::L2Tx, Transaction, TRUSTED_ADDRESS_SLOTS, TRUSTED_TOKEN_SLOTS, U256}; +use zksync_types::{l2::L2Tx, Transaction, TRUSTED_ADDRESS_SLOTS, TRUSTED_TOKEN_SLOTS}; use super::{ - adjust_l1_gas_price_for_tx, apply, + apply, vm_metrics::{SandboxStage, EXECUTION_METRICS, SANDBOX_METRICS}, BlockArgs, TxExecutionArgs, TxSharedArgs, VmPermit, }; impl TxSharedArgs { - pub async fn validate_tx_with_pending_state( - mut self, - vm_permit: VmPermit, - connection_pool: ConnectionPool, - tx: L2Tx, - computational_gas_limit: u32, - ) -> Result<(), ValidationError> { - let mut connection = connection_pool.access_storage_tagged("api").await.unwrap(); - let block_args = BlockArgs::pending(&mut connection).await; - drop(connection); - self.adjust_l1_gas_price(tx.common_data.fee.gas_per_pubdata_limit); - self.validate_tx_in_sandbox( - connection_pool, - vm_permit, - tx, - block_args, - computational_gas_limit, - ) - .await - } - - // In order for validation to pass smoothlessly, we need to ensure that block's required gasPerPubdata will be - // <= to the one in the transaction itself. - pub fn adjust_l1_gas_price(&mut self, gas_per_pubdata_limit: U256) { - self.l1_gas_price = adjust_l1_gas_price_for_tx( - self.l1_gas_price, - self.fair_l2_gas_price, - gas_per_pubdata_limit, - ); - } - - async fn validate_tx_in_sandbox( + pub(crate) async fn validate_tx_in_sandbox( self, connection_pool: ConnectionPool, vm_permit: VmPermit, @@ -72,6 +41,7 @@ impl TxSharedArgs { let result = apply::apply_vm_in_sandbox( vm_permit, self, + true, &execution_args, &connection_pool, tx, 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 f99b3cba5ed7..d4d6d54dd9ee 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 @@ -4,13 +4,8 @@ use std::{cmp, sync::Arc, time::Instant}; use multivm::{ interface::VmExecutionResultAndLogs, - vm_latest::{ - constants::{BLOCK_GAS_LIMIT, MAX_PUBDATA_PER_BLOCK}, - utils::{ - fee::derive_base_fee_and_gas_per_pubdata, - overhead::{derive_overhead, OverheadCoefficients}, - }, - }, + utils::{adjust_l1_gas_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead}, + vm_latest::constants::{BLOCK_GAS_LIMIT, MAX_PUBDATA_PER_BLOCK}, }; use zksync_config::configs::{api::Web3JsonRpcConfig, chain::StateKeeperConfig}; use zksync_contracts::BaseSystemContracts; @@ -22,18 +17,18 @@ use zksync_types::{ l2::{error::TxCheckError::TxDuplication, L2Tx}, utils::storage_key_for_eth_balance, AccountTreeId, Address, ExecuteTransactionCommon, L2ChainId, Nonce, PackedEthSignature, - ProtocolVersionId, Transaction, H160, H256, MAX_GAS_PER_PUBDATA_BYTE, MAX_L2_TX_GAS_LIMIT, - MAX_NEW_FACTORY_DEPS, U256, + ProtocolVersionId, Transaction, VmVersion, H160, H256, MAX_GAS_PER_PUBDATA_BYTE, + MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, U256, }; use zksync_utils::h256_to_u256; pub(super) use self::{proxy::TxProxy, result::SubmitTxError}; +use super::execution_sandbox::execute_tx_in_sandbox; use crate::{ api_server::{ execution_sandbox::{ - adjust_l1_gas_price_for_tx, execute_tx_eth_call, execute_tx_with_pending_state, - get_pubdata_for_factory_deps, BlockArgs, SubmitTxStage, TxExecutionArgs, TxSharedArgs, - VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, + execute_tx_eth_call, get_pubdata_for_factory_deps, BlockArgs, SubmitTxStage, + TxExecutionArgs, TxSharedArgs, VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, }, tx_sender::result::ApiCallResult, }, @@ -279,13 +274,24 @@ impl TxSender { let shared_args = self.shared_args(); let vm_permit = self.0.vm_concurrency_limiter.acquire().await; let vm_permit = vm_permit.ok_or(SubmitTxError::ServerShuttingDown)?; + let mut connection = self + .0 + .replica_connection_pool + .access_storage_tagged("api") + .await + .unwrap(); + let block_args = BlockArgs::pending(&mut connection).await; + drop(connection); - let (_, tx_metrics) = execute_tx_with_pending_state( + let (_, tx_metrics) = execute_tx_in_sandbox( vm_permit.clone(), shared_args.clone(), + true, TxExecutionArgs::for_validation(&tx), self.0.replica_connection_pool.clone(), tx.clone().into(), + block_args, + vec![], ) .await; @@ -299,10 +305,11 @@ impl TxSender { let stage_latency = SANDBOX_METRICS.submit_tx[&SubmitTxStage::VerifyExecute].start(); let computational_gas_limit = self.0.sender_config.validation_computational_gas_limit; let validation_result = shared_args - .validate_tx_with_pending_state( - vm_permit, + .validate_tx_in_sandbox( self.0.replica_connection_pool.clone(), + vm_permit, tx.clone(), + block_args, computational_gas_limit, ) .await; @@ -423,19 +430,12 @@ impl TxSender { )); } - let l1_gas_price = self.0.l1_gas_price_source.estimate_effective_gas_price(); - let (_, gas_per_pubdata_byte) = derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - self.0.sender_config.fair_l2_gas_price, - ); - let effective_gas_per_pubdata = cmp::min( - tx.common_data.fee.gas_per_pubdata_limit, - gas_per_pubdata_byte.into(), - ); - let intrinsic_consts = get_intrinsic_constants(); - let min_gas_limit = U256::from(intrinsic_consts.l2_tx_intrinsic_gas) - + U256::from(intrinsic_consts.l2_tx_intrinsic_pubdata) * effective_gas_per_pubdata; + assert!( + intrinsic_consts.l2_tx_intrinsic_pubdata == 0, + "Currently we assume that the L2 transactions do not have any intrinsic pubdata" + ); + let min_gas_limit = U256::from(intrinsic_consts.l2_tx_intrinsic_gas); if tx.common_data.fee.gas_limit < min_gas_limit { return Err(SubmitTxError::IntrinsicGas); } @@ -551,14 +551,17 @@ impl TxSender { gas_per_pubdata_byte: u64, tx_gas_limit: u32, l1_gas_price: u64, + block_args: BlockArgs, base_fee: u64, + vm_version: VmVersion, ) -> (VmExecutionResultAndLogs, TransactionExecutionMetrics) { let gas_limit_with_overhead = tx_gas_limit + derive_overhead( tx_gas_limit, gas_per_pubdata_byte as u32, tx.encoding_len(), - OverheadCoefficients::from_tx_type(tx.tx_format() as u8), + tx.tx_format() as u8, + vm_version, ); match &mut tx.common_data { @@ -585,12 +588,15 @@ impl TxSender { let vm_execution_cache_misses_limit = self.0.sender_config.vm_execution_cache_misses_limit; let execution_args = TxExecutionArgs::for_gas_estimate(vm_execution_cache_misses_limit, &tx, base_fee); - let (exec_result, tx_metrics) = execute_tx_with_pending_state( + let (exec_result, tx_metrics) = execute_tx_in_sandbox( vm_permit, shared_args, + true, execution_args, self.0.replica_connection_pool.clone(), tx.clone(), + block_args, + vec![], ) .await; @@ -618,6 +624,23 @@ impl TxSender { acceptable_overestimation: u32, ) -> Result { let estimation_started_at = Instant::now(); + + let mut connection = self + .0 + .replica_connection_pool + .access_storage_tagged("api") + .await + .unwrap(); + let block_args = BlockArgs::pending(&mut connection).await; + // If protocol version is not present, we'll use the pre-boojum one + let protocol_version = connection + .blocks_dal() + .get_miniblock_protocol_version_id(block_args.resolved_block_number()) + .await + .unwrap() + .unwrap_or(ProtocolVersionId::last_pre_boojum()); + drop(connection); + let l1_gas_price = { let effective_gas_price = self.0.l1_gas_price_source.estimate_effective_gas_price(); let current_l1_gas_price = @@ -629,12 +652,14 @@ impl TxSender { current_l1_gas_price, self.0.sender_config.fair_l2_gas_price, tx.gas_per_pubdata_byte_limit(), + protocol_version.into(), ) }; let (base_fee, gas_per_pubdata_byte) = derive_base_fee_and_gas_per_pubdata( l1_gas_price, self.0.sender_config.fair_l2_gas_price, + protocol_version.into(), ); match &mut tx.common_data { ExecuteTransactionCommon::L2(common_data) => { @@ -742,7 +767,9 @@ impl TxSender { gas_per_pubdata_byte, try_gas_limit, l1_gas_price, + block_args, base_fee, + protocol_version.into(), ) .await; @@ -779,7 +806,9 @@ impl TxSender { gas_per_pubdata_byte, suggested_gas_limit, l1_gas_price, + block_args, base_fee, + protocol_version.into(), ) .await; @@ -790,7 +819,8 @@ impl TxSender { suggested_gas_limit, gas_per_pubdata_byte as u32, tx.encoding_len(), - OverheadCoefficients::from_tx_type(tx.tx_format() as u8), + tx.tx_format() as u8, + protocol_version.into(), ); let full_gas_limit = @@ -834,12 +864,30 @@ impl TxSender { .into_api_call_result() } - pub fn gas_price(&self) -> u64 { + pub async fn gas_price(&self) -> u64 { let gas_price = self.0.l1_gas_price_source.estimate_effective_gas_price(); let l1_gas_price = (gas_price as f64 * self.0.sender_config.gas_price_scale_factor).round(); + + let mut connection = self + .0 + .replica_connection_pool + .access_storage_tagged("api") + .await + .unwrap(); + let block_args = BlockArgs::pending(&mut connection).await; + // If protocol version is not present, we'll use the pre-boojum one + let protocol_version = connection + .blocks_dal() + .get_miniblock_protocol_version_id(block_args.resolved_block_number()) + .await + .unwrap() + .unwrap_or(ProtocolVersionId::last_pre_boojum()); + drop(connection); + let (base_fee, _) = derive_base_fee_and_gas_per_pubdata( l1_gas_price as u64, self.0.sender_config.fair_l2_gas_price, + protocol_version.into(), ); base_fee } diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs b/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs index 079cb8bef164..5f3dfcd34171 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs +++ b/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/eth.rs @@ -38,7 +38,7 @@ impl EthNamespaceServer for EthNamespace { } async fn gas_price(&self) -> RpcResult { - self.gas_price_impl().map_err(into_jsrpc_error) + self.gas_price_impl().await.map_err(into_jsrpc_error) } async fn new_filter(&self, filter: Filter) -> RpcResult { diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs b/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs index 810dd57facb1..a2760b869edb 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs +++ b/core/lib/zksync_core/src/api_server/web3/namespaces/eth.rs @@ -136,7 +136,7 @@ impl EthNamespace { // When we're estimating fee, we are trying to deduce values related to fee, so we should // not consider provided ones. - tx.common_data.fee.max_fee_per_gas = self.state.tx_sender.gas_price().into(); + tx.common_data.fee.max_fee_per_gas = self.state.tx_sender.gas_price().await.into(); tx.common_data.fee.max_priority_fee_per_gas = tx.common_data.fee.max_fee_per_gas; // Modify the l1 gas price with the scale factor @@ -156,11 +156,11 @@ impl EthNamespace { } #[tracing::instrument(skip(self))] - pub fn gas_price_impl(&self) -> Result { + pub async fn gas_price_impl(&self) -> Result { const METHOD_NAME: &str = "gas_price"; let method_latency = API_METRICS.start_call(METHOD_NAME); - let price = self.state.tx_sender.gas_price(); + let price = self.state.tx_sender.gas_price().await; method_latency.observe(); Ok(price.into()) } diff --git a/core/lib/zksync_core/src/state_keeper/io/mempool.rs b/core/lib/zksync_core/src/state_keeper/io/mempool.rs index 3f84b2d43b79..78a71a6a7130 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mempool.rs +++ b/core/lib/zksync_core/src/state_keeper/io/mempool.rs @@ -8,7 +8,7 @@ use std::{ use async_trait::async_trait; use multivm::{ interface::{FinishedL1Batch, L1BatchEnv, SystemEnv}, - vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata, + utils::derive_base_fee_and_gas_per_pubdata, }; use zksync_config::configs::chain::StateKeeperConfig; use zksync_dal::ConnectionPool; @@ -110,6 +110,7 @@ impl StateKeeperIO for MempoolIO { let (base_fee, gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata( l1_batch_env.l1_gas_price, l1_batch_env.fair_l2_gas_price, + system_env.version.into(), ); self.filter = L2TxFilter { l1_gas_price: l1_batch_env.l1_gas_price, @@ -130,26 +131,17 @@ impl StateKeeperIO for MempoolIO { ) -> Option<(SystemEnv, L1BatchEnv)> { let deadline = Instant::now() + max_wait; + let prev_l1_batch_hash = self.load_previous_l1_batch_hash().await; + + let MiniblockHeader { + timestamp: prev_miniblock_timestamp, + hash: prev_miniblock_hash, + .. + } = self.load_previous_miniblock_header().await; + // Block until at least one transaction in the mempool can match the filter (or timeout happens). // This is needed to ensure that block timestamp is not too old. for _ in 0..poll_iters(self.delay_interval, max_wait) { - // We create a new filter each time, since parameters may change and a previously - // ignored transaction in the mempool may be scheduled for the execution. - self.filter = l2_tx_filter(self.l1_gas_price_provider.as_ref(), self.fair_l2_gas_price); - // We only need to get the root hash when we're certain that we have a new transaction. - if !self.mempool.has_next(&self.filter) { - tokio::time::sleep(self.delay_interval).await; - continue; - } - - let prev_l1_batch_hash = self.load_previous_l1_batch_hash().await; - - let MiniblockHeader { - timestamp: prev_miniblock_timestamp, - hash: prev_miniblock_hash, - .. - } = self.load_previous_miniblock_header().await; - // We cannot create two L1 batches or miniblocks with the same timestamp (forbidden by the bootloader). // Hence, we wait until the current timestamp is larger than the timestamp of the previous miniblock. // We can use `timeout_at` since `sleep_past` is cancel-safe; it only uses `sleep()` async calls. @@ -171,6 +163,19 @@ impl StateKeeperIO for MempoolIO { .base_system_contracts_by_timestamp(current_timestamp) .await; + // We create a new filter each time, since parameters may change and a previously + // ignored transaction in the mempool may be scheduled for the execution. + self.filter = l2_tx_filter( + self.l1_gas_price_provider.as_ref(), + self.fair_l2_gas_price, + protocol_version.into(), + ); + // We only need to get the root hash when we're certain that we have a new transaction. + if !self.mempool.has_next(&self.filter) { + tokio::time::sleep(self.delay_interval).await; + continue; + } + return Some(l1_batch_params( self.current_l1_batch_number, self.fee_account, diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index 70466ae48430..ce3a7714b2be 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -1,7 +1,7 @@ use std::time::Duration; use futures::FutureExt; -use multivm::vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata; +use multivm::utils::derive_base_fee_and_gas_per_pubdata; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::ConnectionPool; use zksync_mempool::L2TxFilter; @@ -72,8 +72,11 @@ async fn test_filter_with_pending_batch() { assert_eq!(mempool.filter(), &L2TxFilter::default()); mempool.load_pending_batch().await; - let (want_base_fee, want_gas_per_pubdata) = - derive_base_fee_and_gas_per_pubdata(give_l1_gas_price, give_fair_l2_gas_price); + let (want_base_fee, want_gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata( + give_l1_gas_price, + give_fair_l2_gas_price, + ProtocolVersionId::latest().into(), + ); let want_filter = L2TxFilter { l1_gas_price: give_l1_gas_price, fee_per_gas: want_base_fee, @@ -100,6 +103,7 @@ async fn test_filter_with_no_pending_batch() { let want_filter = l2_tx_filter( &tester.create_gas_adjuster().await, tester.fair_l2_gas_price(), + ProtocolVersionId::latest().into(), ); // Create a mempool without pending batch and ensure that filter is not initialized just yet. @@ -144,6 +148,7 @@ async fn test_timestamps_are_distinct( let tx_filter = l2_tx_filter( &tester.create_gas_adjuster().await, tester.fair_l2_gas_price(), + ProtocolVersionId::latest().into(), ); tester.insert_tx(&mut guard, tx_filter.fee_per_gas, tx_filter.gas_per_pubdata); diff --git a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs b/core/lib/zksync_core/src/state_keeper/mempool_actor.rs index 235b0ea7e88e..5ad962b112e9 100644 --- a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs +++ b/core/lib/zksync_core/src/state_keeper/mempool_actor.rs @@ -1,13 +1,14 @@ use std::{sync::Arc, time::Duration}; -use multivm::vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata; +use multivm::utils::derive_base_fee_and_gas_per_pubdata; use tokio::sync::watch; use zksync_config::configs::chain::MempoolConfig; use zksync_dal::ConnectionPool; use zksync_mempool::L2TxFilter; +use zksync_types::{ProtocolVersionId, VmVersion}; use super::{metrics::KEEPER_METRICS, types::MempoolGuard}; -use crate::l1_gas_price::L1GasPriceProvider; +use crate::{api_server::execution_sandbox::BlockArgs, l1_gas_price::L1GasPriceProvider}; /// 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 @@ -15,11 +16,12 @@ use crate::l1_gas_price::L1GasPriceProvider; pub fn l2_tx_filter( gas_price_provider: &dyn L1GasPriceProvider, fair_l2_gas_price: u64, + vm_version: VmVersion, ) -> L2TxFilter { let effective_gas_price = gas_price_provider.estimate_effective_gas_price(); let (base_fee, gas_per_pubdata) = - derive_base_fee_and_gas_per_pubdata(effective_gas_price, fair_l2_gas_price); + derive_base_fee_and_gas_per_pubdata(effective_gas_price, fair_l2_gas_price, vm_version); L2TxFilter { l1_gas_price: effective_gas_price, fee_per_gas: base_fee, @@ -77,7 +79,21 @@ impl MempoolFetcher { let latency = KEEPER_METRICS.mempool_sync.start(); let mut storage = pool.access_storage_tagged("state_keeper").await.unwrap(); let mempool_info = self.mempool.get_mempool_info(); - let l2_tx_filter = l2_tx_filter(self.l1_gas_price_provider.as_ref(), fair_l2_gas_price); + + let latest_miniblock = BlockArgs::pending(&mut storage).await; + + let protocol_version = storage + .blocks_dal() + .get_miniblock_protocol_version_id(latest_miniblock.resolved_block_number()) + .await + .unwrap() + .unwrap_or_else(ProtocolVersionId::latest); + + let l2_tx_filter = l2_tx_filter( + self.l1_gas_price_provider.as_ref(), + fair_l2_gas_price, + protocol_version.into(), + ); let (transactions, nonces) = storage .transactions_dal() diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index d9f09ba49c6e..0a146c51a5e3 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -7,7 +7,6 @@ use zksync_config::{ }; use zksync_dal::ConnectionPool; use zksync_object_store::ObjectStore; -use zksync_system_constants::MAX_TXS_IN_BLOCK; use self::io::MempoolIO; pub use self::{ @@ -46,13 +45,6 @@ pub(crate) async fn create_state_keeper( object_store: Box, stop_receiver: watch::Receiver, ) -> ZkSyncStateKeeper { - assert!( - state_keeper_config.transaction_slots <= MAX_TXS_IN_BLOCK, - "Configured transaction_slots ({}) must be lower than the bootloader constant MAX_TXS_IN_BLOCK={}", - state_keeper_config.transaction_slots, - MAX_TXS_IN_BLOCK - ); - let batch_executor_base = MainBatchExecutorBuilder::new( db_config.state_keeper_db_path.clone(), pool.clone(), diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs index 4c21c41e5e40..41d99b8274b7 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/slots.rs @@ -1,3 +1,4 @@ +use multivm::utils::get_bootloader_max_txs_in_batch; use zksync_types::ProtocolVersionId; use crate::state_keeper::seal_criteria::{ @@ -16,8 +17,15 @@ impl SealCriterion for SlotsCriterion { tx_count: usize, _block_data: &SealData, _tx_data: &SealData, - _protocol_version: ProtocolVersionId, + protocol_version: ProtocolVersionId, ) -> SealResolution { + let max_txs_in_batch = get_bootloader_max_txs_in_batch(protocol_version.into()); + assert!( + config.transaction_slots <= max_txs_in_batch, + "Configured transaction_slots ({}) must be lower than the bootloader constant MAX_TXS_IN_BLOCK={} for protocol version {}", + config.transaction_slots, max_txs_in_batch, protocol_version as u16 + ); + if tx_count >= config.transaction_slots { SealResolution::IncludeAndSeal } else { diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs index ed24e3719338..02683e501d9b 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/tx_encoding_size.rs @@ -1,4 +1,4 @@ -use multivm::vm_latest::constants::BOOTLOADER_TX_ENCODING_SPACE; +use multivm::utils::get_bootloader_encoding_space; use zksync_types::ProtocolVersionId; use crate::state_keeper::seal_criteria::{ @@ -16,18 +16,21 @@ impl SealCriterion for TxEncodingSizeCriterion { _tx_count: usize, block_data: &SealData, tx_data: &SealData, - _protocol_version_id: ProtocolVersionId, + protocol_version_id: ProtocolVersionId, ) -> SealResolution { + let bootloader_tx_encoding_space = + get_bootloader_encoding_space(protocol_version_id.into()); + let reject_bound = - (BOOTLOADER_TX_ENCODING_SPACE as f64 * config.reject_tx_at_geometry_percentage).round(); - let include_and_seal_bound = (BOOTLOADER_TX_ENCODING_SPACE as f64 + (bootloader_tx_encoding_space as f64 * config.reject_tx_at_geometry_percentage).round(); + let include_and_seal_bound = (bootloader_tx_encoding_space as f64 * config.close_block_at_geometry_percentage) .round(); if tx_data.cumulative_size > reject_bound as usize { let message = "Transaction cannot be included due to large encoding size"; SealResolution::Unexecutable(message.into()) - } else if block_data.cumulative_size > BOOTLOADER_TX_ENCODING_SPACE as usize { + } else if block_data.cumulative_size > bootloader_tx_encoding_space as usize { SealResolution::ExcludeAndSeal } else if block_data.cumulative_size > include_and_seal_bound as usize { SealResolution::IncludeAndSeal @@ -47,6 +50,9 @@ mod tests { #[test] fn seal_criterion() { + let bootloader_tx_encoding_space = + get_bootloader_encoding_space(ProtocolVersionId::latest().into()); + // Create an empty config and only setup fields relevant for the test. let config = StateKeeperConfig { reject_tx_at_geometry_percentage: 0.95, @@ -72,7 +78,7 @@ mod tests { 0, &SealData::default(), &SealData { - cumulative_size: BOOTLOADER_TX_ENCODING_SPACE as usize + 1, + cumulative_size: bootloader_tx_encoding_space as usize + 1, ..SealData::default() }, ProtocolVersionId::latest(), @@ -89,7 +95,7 @@ mod tests { 0, 0, &SealData { - cumulative_size: BOOTLOADER_TX_ENCODING_SPACE as usize + 1, + cumulative_size: bootloader_tx_encoding_space as usize + 1, ..SealData::default() }, &SealData { @@ -105,7 +111,7 @@ mod tests { 0, 0, &SealData { - cumulative_size: BOOTLOADER_TX_ENCODING_SPACE as usize, + cumulative_size: bootloader_tx_encoding_space as usize, ..SealData::default() }, &SealData { From ae6e18e5412cadefbc03307a476d6b96c41f04e1 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Fri, 5 Jan 2024 13:01:19 +0200 Subject: [PATCH 15/49] feat(en): Make consistency checker work with pruned data (#742) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Modifies consistency checker so that it works with pruned node data during snapshot recovery. Adds tests for the checker. ## Why ❔ Part of preparations of EN code to support snapshot recovery. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/lib/eth_client/src/clients/generic.rs | 8 +- .../src/api_server/web3/tests/mod.rs | 21 +- .../src/api_server/web3/tests/snapshots.rs | 12 +- .../lib/zksync_core/src/consensus/testonly.rs | 6 +- .../src/consistency_checker/mod.rs | 393 ++++++++---- ...it_l1_batch_200000_testnet_goerli.calldata | Bin 0 -> 35012 bytes ...it_l1_batch_351000-351004_mainnet.calldata | Bin 0 -> 72932 bytes ...mit_l1_batch_4470_testnet_sepolia.calldata | Bin 0 -> 1956 bytes .../src/consistency_checker/tests/mod.rs | 605 ++++++++++++++++++ core/lib/zksync_core/src/eth_sender/tests.rs | 19 +- core/lib/zksync_core/src/genesis.rs | 21 +- .../src/metadata_calculator/tests.rs | 42 +- .../zksync_core/src/reorg_detector/tests.rs | 29 +- .../src/state_keeper/io/tests/mod.rs | 17 +- .../src/state_keeper/io/tests/tester.rs | 25 +- .../zksync_core/src/state_keeper/tests/mod.rs | 57 +- .../src/state_keeper/tests/tester.rs | 25 +- core/lib/zksync_core/src/sync_layer/tests.rs | 6 +- .../src/{utils.rs => utils/mod.rs} | 3 + core/lib/zksync_core/src/utils/testonly.rs | 94 +++ 20 files changed, 1042 insertions(+), 341 deletions(-) create mode 100644 core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_200000_testnet_goerli.calldata create mode 100644 core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_351000-351004_mainnet.calldata create mode 100644 core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_4470_testnet_sepolia.calldata create mode 100644 core/lib/zksync_core/src/consistency_checker/tests/mod.rs rename core/lib/zksync_core/src/{utils.rs => utils/mod.rs} (99%) create mode 100644 core/lib/zksync_core/src/utils/testonly.rs diff --git a/core/lib/eth_client/src/clients/generic.rs b/core/lib/eth_client/src/clients/generic.rs index fa9950d524a9..c54a814d449f 100644 --- a/core/lib/eth_client/src/clients/generic.rs +++ b/core/lib/eth_client/src/clients/generic.rs @@ -160,11 +160,11 @@ impl BoundEthInterface for Arc { self.as_ref().nonce_at(block, component).await } - async fn current_nonce(&self, _: &'static str) -> Result { - self.as_ref().current_nonce("").await + async fn current_nonce(&self, component: &'static str) -> Result { + self.as_ref().current_nonce(component).await } - async fn pending_nonce(&self, _: &'static str) -> Result { - self.as_ref().pending_nonce("").await + async fn pending_nonce(&self, component: &'static str) -> Result { + self.as_ref().pending_nonce(component).await } } diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs index 3333e72faf54..57664629c722 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs @@ -8,13 +8,12 @@ use zksync_config::configs::{ chain::{NetworkConfig, StateKeeperConfig}, ContractsConfig, }; -use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{transactions_dal::L2TxSubmissionResult, ConnectionPool}; use zksync_health_check::CheckHealth; use zksync_state::PostgresStorageCaches; use zksync_types::{ block::MiniblockHeader, fee::TransactionExecutionMetrics, tx::IncludedTxLocation, Address, - L1BatchNumber, ProtocolVersionId, VmEvent, H256, U64, + L1BatchNumber, VmEvent, H256, U64, }; use zksync_web3_decl::{ jsonrpsee::{core::ClientError as RpcError, http_client::HttpClient, types::error::ErrorCode}, @@ -27,7 +26,7 @@ use crate::{ api_server::tx_sender::TxSenderConfig, genesis::{ensure_genesis_state, GenesisParams}, l1_gas_price::L1GasPriceProvider, - state_keeper::tests::create_l2_transaction, + utils::testonly::{create_l2_transaction, create_miniblock}, }; mod snapshots; @@ -209,22 +208,6 @@ fn assert_logs_match(actual_logs: &[api::Log], expected_logs: &[&VmEvent]) { } } -fn create_miniblock(number: u32) -> MiniblockHeader { - MiniblockHeader { - number: MiniblockNumber(number), - timestamp: number.into(), - hash: H256::from_low_u64_be(number.into()), - l1_tx_count: 0, - l2_tx_count: 0, - base_fee_per_gas: 100, - l1_gas_price: 100, - l2_fair_gas_price: 100, - base_system_contracts_hashes: BaseSystemContractsHashes::default(), - protocol_version: Some(ProtocolVersionId::latest()), - virtual_blocks: 1, - } -} - async fn store_miniblock( storage: &mut StorageProcessor<'_>, ) -> anyhow::Result<(MiniblockHeader, H256)> { diff --git a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs index 608f2845065f..e3d233777310 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs @@ -2,23 +2,17 @@ use std::collections::HashSet; -use zksync_types::block::{BlockGasCount, L1BatchHeader}; +use zksync_types::block::BlockGasCount; use zksync_web3_decl::namespaces::SnapshotsNamespaceClient; use super::*; -use crate::state_keeper::tests::create_l1_batch_metadata; +use crate::utils::testonly::{create_l1_batch, create_l1_batch_metadata}; async fn seal_l1_batch( storage: &mut StorageProcessor<'_>, number: L1BatchNumber, ) -> anyhow::Result<()> { - let header = L1BatchHeader::new( - number, - number.0.into(), - Address::repeat_byte(1), - BaseSystemContractsHashes::default(), - ProtocolVersionId::latest(), - ); + let header = create_l1_batch(number.0); storage .blocks_dal() .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 9de6f7085e20..13bae6c9c322 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -12,14 +12,14 @@ use zksync_types::{ use crate::{ genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ - seal_criteria::NoopSealer, - tests::{create_l1_batch_metadata, create_l2_transaction, MockBatchExecutorBuilder}, - MiniblockSealer, ZkSyncStateKeeper, + seal_criteria::NoopSealer, tests::MockBatchExecutorBuilder, MiniblockSealer, + ZkSyncStateKeeper, }, sync_layer::{ sync_action::{ActionQueue, ActionQueueSender, SyncAction}, ExternalIO, MainNodeClient, SyncState, }, + utils::testonly::{create_l1_batch_metadata, create_l2_transaction}, }; #[derive(Debug, Default)] diff --git a/core/lib/zksync_core/src/consistency_checker/mod.rs b/core/lib/zksync_core/src/consistency_checker/mod.rs index 4e3dc44a7fd9..768684e5fbaf 100644 --- a/core/lib/zksync_core/src/consistency_checker/mod.rs +++ b/core/lib/zksync_core/src/consistency_checker/mod.rs @@ -1,204 +1,319 @@ -use std::time::Duration; +use std::{fmt, time::Duration}; +use anyhow::Context as _; +use tokio::sync::watch; use zksync_contracts::PRE_BOOJUM_COMMIT_FUNCTION; -use zksync_dal::ConnectionPool; -use zksync_types::{ - web3::{error, ethabi, transports::Http, types::TransactionId, Web3}, - L1BatchNumber, +use zksync_dal::{ConnectionPool, StorageProcessor}; +use zksync_eth_client::{clients::QueryClient, Error as L1ClientError, EthInterface}; +use zksync_types::{web3::ethabi, L1BatchNumber, H256}; + +use crate::{ + metrics::{CheckerComponent, EN_METRICS}, + utils::wait_for_l1_batch_with_metadata, }; -use crate::metrics::{CheckerComponent, EN_METRICS}; +#[cfg(test)] +mod tests; -#[derive(Debug)] -pub struct ConsistencyChecker { - // ABI of the zkSync contract - contract: ethabi::Contract, - // How many past batches to check when starting - max_batches_to_recheck: u32, - web3: Web3, - db: ConnectionPool, +#[derive(Debug, thiserror::Error)] +enum CheckError { + #[error("Web3 error communicating with L1")] + Web3(#[from] L1ClientError), + #[error("Internal error")] + Internal(#[from] anyhow::Error), } -const SLEEP_DELAY: Duration = Duration::from_secs(5); +impl From for CheckError { + fn from(err: zksync_dal::SqlxError) -> Self { + Self::Internal(err.into()) + } +} -impl ConsistencyChecker { - pub fn new(web3_url: &str, max_batches_to_recheck: u32, db: ConnectionPool) -> Self { - let web3 = Web3::new(Http::new(web3_url).unwrap()); - let contract = zksync_contracts::zksync_contract(); - Self { - web3, - contract, - max_batches_to_recheck, - db, - } +trait UpdateCheckedBatch: fmt::Debug + Send + Sync { + fn update_checked_batch(&mut self, last_checked_batch: L1BatchNumber); +} + +/// Default [`UpdateCheckedBatch`] implementation that reports the batch number as a metric. +impl UpdateCheckedBatch for () { + fn update_checked_batch(&mut self, last_checked_batch: L1BatchNumber) { + EN_METRICS.last_correct_batch[&CheckerComponent::ConsistencyChecker] + .set(last_checked_batch.0.into()); } +} - async fn check_commitments(&self, batch_number: L1BatchNumber) -> Result { - let mut storage = self.db.access_storage().await.unwrap(); +/// Consistency checker behavior when L1 commit data divergence is detected. +// This is a temporary workaround for a bug that sometimes leads to incorrect L1 batch data returned by the server +// (and thus persisted by external nodes). Eventually, we want to go back to bailing on L1 data mismatch; +// for now, it's only enabled for the unit tests. +#[derive(Debug)] +enum L1DataMismatchBehavior { + #[cfg(test)] + Bail, + Log, +} + +/// L1 commit data loaded from Postgres. +#[derive(Debug)] +struct LocalL1BatchCommitData { + is_pre_boojum: bool, + l1_commit_data: ethabi::Token, + commit_tx_hash: H256, +} - let storage_l1_batch = storage +impl LocalL1BatchCommitData { + /// Returns `Ok(None)` if Postgres doesn't contain all data necessary to check L1 commitment + /// for the specified batch. + async fn new( + storage: &mut StorageProcessor<'_>, + batch_number: L1BatchNumber, + ) -> anyhow::Result> { + let Some(storage_l1_batch) = storage .blocks_dal() .get_storage_l1_batch(batch_number) - .await - .unwrap() - .unwrap_or_else(|| panic!("L1 batch #{} not found in the database", batch_number)); + .await? + else { + return Ok(None); + }; - let commit_tx_id = storage_l1_batch - .eth_commit_tx_id - .unwrap_or_else(|| panic!("Commit tx not found for L1 batch #{}", batch_number)) - as u32; + let Some(commit_tx_id) = storage_l1_batch.eth_commit_tx_id else { + return Ok(None); + }; + let commit_tx_hash = storage + .eth_sender_dal() + .get_confirmed_tx_hash_by_eth_tx_id(commit_tx_id as u32) + .await? + .with_context(|| { + format!("Commit tx hash not found in the database for tx id {commit_tx_id}") + })?; - let block_metadata = storage + let Some(l1_batch) = storage .blocks_dal() .get_l1_batch_with_metadata(storage_l1_batch) - .await - .unwrap() - .unwrap_or_else(|| { - panic!( - "Metadata for L1 batch #{} not found in the database", - batch_number - ) - }); + .await? + else { + return Ok(None); + }; - let commit_tx_hash = storage - .eth_sender_dal() - .get_confirmed_tx_hash_by_eth_tx_id(commit_tx_id) - .await - .unwrap() - .unwrap_or_else(|| { - panic!( - "Commit tx hash not found in the database. Commit tx id: {}", - commit_tx_id - ) - }); + let is_pre_boojum = l1_batch + .header + .protocol_version + .map_or(true, |version| version.is_pre_boojum()); + let metadata = &l1_batch.metadata; - tracing::info!( - "Checking commit tx {} for batch {}", + // For Boojum batches, `bootloader_initial_content_commitment` and `events_queue_commitment` + // are (temporarily) only computed by the metadata calculator if it runs with the full tree. + // I.e., for these batches, we may have partial metadata in Postgres, which would not be sufficient + // to compute local L1 commitment. + if !is_pre_boojum + && (metadata.bootloader_initial_content_commitment.is_none() + || metadata.events_queue_commitment.is_none()) + { + return Ok(None); + } + + Ok(Some(Self { + is_pre_boojum, + l1_commit_data: l1_batch.l1_commit_data(), commit_tx_hash, - batch_number.0 - ); + })) + } +} - // we can't get tx calldata from db because it can be fake - let commit_tx = self - .web3 - .eth() - .transaction(TransactionId::Hash(commit_tx_hash)) - .await? - .expect("Commit tx not found on L1"); +#[derive(Debug)] +pub struct ConsistencyChecker { + /// ABI of the zkSync contract + contract: ethabi::Contract, + /// How many past batches to check when starting + max_batches_to_recheck: u32, + sleep_interval: Duration, + l1_client: Box, + l1_batch_updater: Box, + l1_data_mismatch_behavior: L1DataMismatchBehavior, + pool: ConnectionPool, +} + +impl ConsistencyChecker { + const DEFAULT_SLEEP_INTERVAL: Duration = Duration::from_secs(5); + + pub fn new(web3_url: &str, max_batches_to_recheck: u32, pool: ConnectionPool) -> Self { + let web3 = QueryClient::new(web3_url).unwrap(); + Self { + contract: zksync_contracts::zksync_contract(), + max_batches_to_recheck, + sleep_interval: Self::DEFAULT_SLEEP_INTERVAL, + l1_client: Box::new(web3), + l1_batch_updater: Box::new(()), + l1_data_mismatch_behavior: L1DataMismatchBehavior::Log, + pool, + } + } + + async fn check_commitments( + &self, + batch_number: L1BatchNumber, + local: &LocalL1BatchCommitData, + ) -> Result { + let commit_tx_hash = local.commit_tx_hash; + tracing::info!("Checking commit tx {commit_tx_hash} for L1 batch #{batch_number}"); let commit_tx_status = self - .web3 - .eth() - .transaction_receipt(commit_tx_hash) + .l1_client + .get_tx_status(commit_tx_hash, "consistency_checker") .await? - .expect("Commit tx receipt not found on L1") - .status; + .with_context(|| format!("Receipt for tx {commit_tx_hash:?} not found on L1"))?; + if !commit_tx_status.success { + let err = anyhow::anyhow!("Main node gave us a failed commit tx"); + return Err(err.into()); + } - assert_eq!( - commit_tx_status, - Some(1.into()), - "Main node gave us a failed commit tx" - ); + // We can't get tx calldata from db because it can be fake. + let commit_tx_input_data = self + .l1_client + .get_tx(commit_tx_hash, "consistency_checker") + .await? + .with_context(|| format!("Commit for tx {commit_tx_hash:?} not found on L1"))? + .input; + // TODO (PLA-721): Check receiving contract and selector - let commit_function = if block_metadata - .header - .protocol_version - .unwrap() - .is_pre_boojum() - { - PRE_BOOJUM_COMMIT_FUNCTION.clone() + let commit_function = if local.is_pre_boojum { + &*PRE_BOOJUM_COMMIT_FUNCTION } else { - self.contract.function("commitBatches").unwrap().clone() + self.contract + .function("commitBatches") + .context("L1 contract does not have `commitBatches` function")? }; + let commitment = + Self::extract_commit_data(&commit_tx_input_data.0, commit_function, batch_number) + .with_context(|| { + format!("Failed extracting commit data for transaction {commit_tx_hash:?}") + })?; + Ok(commitment == local.l1_commit_data) + } - let commitments = commit_function - .decode_input(&commit_tx.input.0[4..]) - .unwrap() + fn extract_commit_data( + commit_tx_input_data: &[u8], + commit_function: ðabi::Function, + batch_number: L1BatchNumber, + ) -> anyhow::Result { + let mut commit_input_tokens = commit_function + .decode_input(&commit_tx_input_data[4..]) + .with_context(|| format!("Failed decoding calldata for L1 commit function"))?; + let mut commitments = commit_input_tokens .pop() - .unwrap() + .context("Unexpected signature for L1 commit function")? .into_array() - .unwrap(); + .context("Unexpected signature for L1 commit function")?; // Commit transactions usually publish multiple commitments at once, so we need to find // the one that corresponds to the batch we're checking. - let first_batch_number = match &commitments[0] { - ethabi::Token::Tuple(tuple) => tuple[0].clone().into_uint().unwrap().as_usize(), - _ => panic!("ABI does not match the expected one"), + let first_batch_commitment = commitments + .first() + .with_context(|| format!("L1 batch commitment is empty"))?; + let ethabi::Token::Tuple(first_batch_commitment) = first_batch_commitment else { + anyhow::bail!("Unexpected signature for L1 commit function"); }; - let commitment = &commitments[batch_number.0 as usize - first_batch_number]; + let first_batch_number = first_batch_commitment + .first() + .context("Unexpected signature for L1 commit function")?; + let first_batch_number = first_batch_number + .clone() + .into_uint() + .context("Unexpected signature for L1 commit function")?; + let first_batch_number = usize::try_from(first_batch_number) + .map_err(|_| anyhow::anyhow!("Integer overflow for L1 batch number"))?; + // ^ `TryFrom` has `&str` error here, so we can't use `.context()`. - Ok(commitment == &block_metadata.l1_commit_data()) + let commitment = (batch_number.0 as usize) + .checked_sub(first_batch_number) + .and_then(|offset| { + (offset < commitments.len()).then(|| commitments.swap_remove(offset)) + }); + commitment.with_context(|| { + let actual_range = first_batch_number..(first_batch_number + commitments.len()); + format!( + "Malformed commitment data; it should prove L1 batch #{batch_number}, \ + but it actually proves batches #{actual_range:?}" + ) + }) } - async fn last_committed_batch(&self) -> L1BatchNumber { - self.db + async fn last_committed_batch(&self) -> anyhow::Result> { + Ok(self + .pool .access_storage() - .await - .unwrap() + .await? .blocks_dal() .get_number_of_last_l1_batch_committed_on_eth() - .await - .unwrap() - .unwrap_or(L1BatchNumber(0)) + .await?) } - pub async fn run( - self, - stop_receiver: tokio::sync::watch::Receiver, - ) -> anyhow::Result<()> { - let mut batch_number: L1BatchNumber = self + pub async fn run(mut self, mut stop_receiver: watch::Receiver) -> anyhow::Result<()> { + // It doesn't make sense to start the checker until we have at least one L1 batch with metadata. + let earliest_l1_batch_number = + wait_for_l1_batch_with_metadata(&self.pool, self.sleep_interval, &mut stop_receiver) + .await?; + + let Some(earliest_l1_batch_number) = earliest_l1_batch_number else { + return Ok(()); // Stop signal received + }; + + let last_committed_batch = self .last_committed_batch() - .await + .await? + .unwrap_or(earliest_l1_batch_number); + let first_batch_to_check: L1BatchNumber = last_committed_batch .0 .saturating_sub(self.max_batches_to_recheck) - .max(1) .into(); + // We shouldn't check batches not present in the storage, and skip the genesis batch since + // it's not committed on L1. + let first_batch_to_check = first_batch_to_check + .max(earliest_l1_batch_number) + .max(L1BatchNumber(1)); + tracing::info!( + "Last committed L1 batch is #{last_committed_batch}; starting checks from L1 batch #{first_batch_to_check}" + ); - tracing::info!("Starting consistency checker from batch {}", batch_number.0); - + let mut batch_number = first_batch_to_check; loop { if *stop_receiver.borrow() { tracing::info!("Stop signal received, consistency_checker is shutting down"); break; } - let metadata = self - .db - .access_storage() - .await - .unwrap() - .blocks_dal() - .get_l1_batch_metadata(batch_number) - .await - .unwrap(); - let batch_has_metadata = metadata - .map(|m| { - m.metadata.bootloader_initial_content_commitment.is_some() - && m.metadata.events_queue_commitment.is_some() - }) - .unwrap_or(false); - + let mut storage = self.pool.access_storage().await?; // The batch might be already committed but not yet processed by the external node's tree // OR the batch might be processed by the external node's tree but not yet committed. // We need both. - if !batch_has_metadata || self.last_committed_batch().await < batch_number { - tokio::time::sleep(SLEEP_DELAY).await; + let Some(local) = LocalL1BatchCommitData::new(&mut storage, batch_number).await? else { + tokio::time::sleep(self.sleep_interval).await; continue; - } + }; + drop(storage); - match self.check_commitments(batch_number).await { + match self.check_commitments(batch_number, &local).await { Ok(true) => { - tracing::info!("Batch {} is consistent with L1", batch_number.0); - EN_METRICS.last_correct_batch[&CheckerComponent::ConsistencyChecker] - .set(batch_number.0.into()); - batch_number.0 += 1; + tracing::info!("L1 batch #{batch_number} is consistent with L1"); + self.l1_batch_updater.update_checked_batch(batch_number); + batch_number += 1; } - Ok(false) => { - tracing::warn!("Batch {} is inconsistent with L1", batch_number.0); + Ok(false) => match &self.l1_data_mismatch_behavior { + #[cfg(test)] + L1DataMismatchBehavior::Bail => { + anyhow::bail!("L1 Batch #{batch_number} is inconsistent with L1"); + } + L1DataMismatchBehavior::Log => { + tracing::warn!("L1 Batch #{batch_number} is inconsistent with L1"); + } + }, + Err(CheckError::Web3(err)) => { + tracing::warn!("Error accessing L1; will retry after a delay: {err}"); + tokio::time::sleep(self.sleep_interval).await; } - Err(e) => { - tracing::warn!("Consistency checker error: {}", e); - tokio::time::sleep(SLEEP_DELAY).await; + Err(CheckError::Internal(err)) => { + let context = + format!("Failed verifying consistency of L1 batch #{batch_number}"); + return Err(err.context(context)); } } } diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_200000_testnet_goerli.calldata b/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_200000_testnet_goerli.calldata new file mode 100644 index 0000000000000000000000000000000000000000..8018825804e4b15e5708a12061eb8d684b5a912a GIT binary patch literal 35012 zcmc(I2Rv4P^#8Lc<0(>+QKF>m%9fGHEFmjM2`D`qN>lyGKWrV*4eqpBXL@C^Nq4M8|dX){X5%l_${b=Qqymz zsV*H>_;<#GQ)JpxcxG@xCrLr{xTxQ+wZn%!wI|{2A-tm02-aVbiL7iL!Bh`%>mTpL7wZezI;OwBg1^!1ZHX9wMtpYQ`M||9mtk( zr4JtA1qLgPPwM*N!PNN(Me2OUVCsA{bv6vmy#gQ{;3Ep(M7w&wa=Ce9_d6-mOe>qL zzx-TX#p`=RYitq)jZ@QCS_AeqehsI;_59KL^Uv533tL-_N)-?Nej-y*EXrR$o#Cbq z_5pa7Milx&Pl5gqWO@_;QUrmhnFoKt67LYks{_@;AHgv_`dU&C@!X7)*4@<0-h17P{#N;Ib}?V^iZ7ZfiS(Y$Az0kcg-qXziv=a@ z-dhUR?1;D#Ha#7or+8Nu#sA>)S5A+|!&uy;RsTsJJKs3zitUTfzPf2_FFam&_0RTD z_tmPkbvbt`<*>M^s`dxv&-8eTv_iI_CvxC|7KjF+@;<)RC$=Y?7xtgnjK%%Z>OC;> zJNi~+$FHY8{n}X{hjxlLGwyt%lyJZ&oZGV)8d$*no6Ztq5V=!e*DK26hqzFtQl-VE z@M*twru3s7>Q|BmZAP&Ae)~$Sc{+ZJL+ah@udhYo_7#5j4_wD9@tA(k+gzEV{JbFy zE+YmkmkvtfW#rB_5Ce=KrzPxC1;L~;c19O$qDJm)2TO7|bl-!!dA1-I!tsG6Db7qk zEcqktCV|JHSRAxYlHy>jb?G;TMeBk=7-$!oA7pUAB5>}?4O9;fw-c<8!?C$=NF2|} z#Nd9deLw++y1>M?Q#JHEE64}z?xKK$5nZnU1sw5p2vWdF1V4$K_Ca)S1^K97Ys)C$ zV2A?loyx+l`Vem9ju0grY&H9XAL1`D^JC-=v`>+{FRKQ)ZI12xY=_k+LP;M1?$_Ff zlyGVXQ!h93n}Tfc{)tk;5%mw+_sI2+=(##NHnz9QPL0e(T5TbqyzDjSxZL3HPrZnKQG!Zwr|KfxT~}}K67eY zC9H~l^Ow5Y7pcZ)ppMIhS@ATst&I!r^YQ&mdB62Pbux8pyFH6>szYCRI6(sA5jnR8C)K%+b_)^&~6<+S8db0_IyBbT58x#=MI_uI~2v+^}e?cQwtQb4Q}qebu5y@|JKm(^Djo-sgK|lrp9Ll1;(`wn{*dP+K(D)J!t0I zCQ{#_)pAZ~X*Fx-{cIkq&enJ4yB;j>uwiYRHu<(7IW%z>6DW`>Sdu?qx^`6g*__&U znTr!$I|}aeymXz(EHaTOwJ&l=y;~}=EJ_~`yP>iBSFfxv^Ogm%yXUj!celgf*2PF; z<+`gXv?+vUO%4b~jL*ExCYhLeJ&BmriFz=23z7Jzs&k_G(-JY&uMpp@ww>-Ay z=S7@1{Mf9d+sejvbZvIi{D~&Ll5dsF{o3$PQk*|mE!!ukE%kq4p8mIUlgQIaW8qg{ zbUpSS4SeYu`Lo)0=6=uBDQkHsZUpA?NdoER2EEDMo;O(^#sa+~$5Q>Rb5(^|o}EQ- z$sJPi;M!MJC|m_X0r^wQB1<*tu)S+{FSY*171|^~1oFdZ;4y1f5a)#WE63H)zWEPv zHO7zrB}Tk<%aiK|>ep?``zA(!87xqvh(F}ymucLiVErHBbO#Uz?86k}Ot|s!(^{7H zAPk0D_{s`Z$ep%o7BU20+pOG!v2w)mH^H8V+vCW@ z-^xpNfeL`T2)2PZ?&$j~_Af*Iw2hp4k^7B(lLcemb$3;nWVaOgH~nB4+79I}0Nl*$ zYflIAJ}tyw-P2{2A06|-ag)2M#^;WBANSm5gCK#+SbekFe_snEM<_8x1iaj5`IJj0 zIsNVDyy=wYwbP8O`tO|~KM~M3r^UI-snzaHk2HH{-WgE|t?wmPj6SZBK6J`Y2iR)5oCG(7XWbE+}2AV0q*jMVH}tJ^aRT=bex4KYwq>{Z_#l z;lR)rU;i6x_pfckr;9K9_H4>gKWNCI^Yv&8j6g zF;tyDTdJ~sSCmlST^^24G4GJtFJ=|nu4h2|4Qk*zO!d&Quy2Ks#)9z>0& ziknQlv~0eTPNB>5tx3GP%WX}`GTgb`Kw^IFX(H};+xJ1f@wXpe% z>j>|G^v#JA>nr)nLRgr0yeYFvIHz^AjqcZt$Qmng!_^xx`bgqx^5fvG{}_lTIm*!t z;x%sKgWMPn>a9O?e^$nwS(;D;SYmyhGCOeb4=?PCAQro>kv?442Aa88yPYzcVEj`} zpbTmlTl>>`0pr;hY@s@3Y)K&yg@xDfJL|#NrY5jOCBC=_y9mS<;2DEE50+#P#^cH% zVO~!_9zlGO0J_lZaB}elYSg8XY*Yg{MONnEKNz$>qDEaiaW<{Y0DT+_G%s#XI<;Id znJ68*>|{UGJOlApj#nPw^!!6<3Shv_6ZrZVITHV$z?TNsla>6Uf3W}m20@}RdjbV5 zKp)A)PD^OO|F^%gBP~K^hm+UW2em9x^w)3lhXD$5I5J#$TXkRh zw)Ur8wH~pxem~TNxr55bnqPQ|dSv0^+ai&+i9u|KQJq z*8=VN<$0a?oOUE50X#?jLpko4w|^bSt#hqDK60QnHEaLT&wC|6CB{ zqrRy@`8sk~4mX{CJKBa|2shY4-oJps=vkYyw~Kn5orZK!_4`kqT4&J5-O;6A%S6*4Edp$1xe{bwA9a(plp28-Uy4dH<2p@0)0E)flkj^FoK24m>d-n%gt zsJG@0f;iM*#{vdVaz-F$zxl5IY>8tK#|H#^B5n^P6CWJLDF90VR|=MZziiMS7<)fZ zyH_vhu;QSqd+dn1=dRbPV!R0vVXE>eC_!jm060UYDFI}4Jxz3~RBoi>!FjZphS$*O z!%n`|yZZE#9)LhWsge{#A$MM51~Ne6la{IIF4@o-@wF z;!FdNmor=4iEIB8f8<+iV~|^{+2#`*YwxhC<%;V?J=tc6#hEz|!UO6}4#!{G-QIsN z?cu~OUDesLKdZ-IC;sYd`^_qe#hDAaz}A+TPBx;}D|t9euVQ&+raYs+mF;DNybA36 zs!3=)1n!H)aOjcG?#;sn)3aCOGhlI2w(^PlkJDb5x}d48UfvK4`EdZwau(6WChzoX zPRXrCyBP45`l%pV{g&`FyWed&vxF?C%IU*Dgip zf5llUPqrepuqX$Mivi1}qX|uZ98iuAlw6`Y=Id)P8h~}zU`ZEYCF{Bw-H|JO7$TK8 zf>R@5ISq@JMT7!3OFr#khpnt_hwnewENJ-uoK~BolV6HsR2>k`w2~ zlLFq$6oBzdHy#aB1>WDx?-pK~mz3V$Jlyu}uH+wPePt#{j0PSzMxu(40PSy7;+#ZS zA`s(%XW#&EafIS{t+h;Fok#B}VR6gvQ7(d^0y~0iobwg29Zv&E6}VpL`w7%n1Rn_# zpU{L>2b%zH2R<27H!amUkILN6Bk3fHWdb}nll7A+85sOl^@GiX{4^#!Q%emeNYAn{fGBKnDvUn zd!Epnf$g4wh^rJgU2K0S6u9_~k(5so-x0)0;C`@@-%hIYf0=?rlg(w~tN?wyi>s3> zfwjqPc(S+Uamu)L(|&E|fMb0n+d>SSufj8`8*Enj+5EXZNlp%*CO070P^I^dyR3Iw z%c`#D>YNfk7cDQ_eEwdGW`|Fntp9C%hpfTx3u&Dpn3RT3A@kVr*)T?Kf+k4=fdWXY zr@hnFJ0w+fOWbZ+f4=E#8v6H9G4m02@!q1KK8J}G-<4wM8)x`gD_#HIstA;l@c7vC zGabb*?8q4+{RFmqWApnrukg(-PVP%R^>{~s2fMLw<%de=fA#${%X15QovXLDc zJyDR^Df<}2QJG8s5AoTma6mP*RR6>Gi9Jw%SB|HkeC9vIQ_#{6?3<~OUdksZP6GBr z=sgo<7>pC-QU9LsK^@BzaLND&)p*5JR-wV|5(uLjudHhoFh2o>HNbuya~8ytJK$Cw zx~UKQAr^2Q?&L5a4Q_0rQDMi8V^fZ6hPiu=IW0{~!u^$KO~#EIGVz5G&Jb7vxK^+P zaMBD$q4Q#%@tYf!*L!_S@Hi~I@#@6x4=;4n6yI-S+LCqrs zf{b)noLpTOS7zmwh4(?5KiG~xmOng{_xG}2Q`aSVhW67x^ZI^car@R(ztQh`$+4d4 z$i*reUk~g1k%~@SGw=4kve&;DG`L0>i<6fglZ?)@MNju92Oj)@fZhC^)g@HvHuhzD zpy*NCkTuRPS2!f-k+Pnn?DXPM8mlP=$sJ<1Ll-9x_cqfS`|sPixEXps3EUUO+m-27 zQ-kj#GTv^Rq~-b}Bp<|`IxB4;GS*&h8*t$|X8a9gDAm!4terd7*E8R0E!y8{;o9Hd z8R=rWbKP1C#Fy)vBA|R3ppV$kC;L2r`2od2;598DV%kp+#3f2SF$vU`xB!uX@kgj{ z84T1`zd9gr52B!E>9Yt}gpJj)ZEq|=5)dy#{v{zB*IKppea0K80r=5N-%Cj%nJ|MK zX2SZp?gj_OB1Q(l-2(jWm!yjoTv06EY%?(a>OdsL@1-6hO;(xlvOT#1EgRr>O9L3q zP2L0Gt#B76BO;S5Q8IfJoRtfK32*hmcIrg7$ALj;gN*JdYX-F5ZSf1BC|g-e~DP z)Hc``z~_gKbuUhH*ebkV76NcuU>rwgb1=3psi*_&Lc@^Sdng{Un$bUe_V%DwD)Cto zED4Csz%vFH50+#PMk^lWHL2G@9-wbIK1&22B|DN_d`4E!%0yiJu$mM~5U zuH+Yz>il1(Akh?aa-b!kkCfll1AJht`#w~?ui-fTes=ioS3l;GkJhM+p|?6I6G=N} zbrD@v73ybtWS(}USaJ14SW9E_yMIzy?@Nq z*)Zl{>Hy|U>U;#~|IzO)Wl1^Scye2z?Z@Zv1)+(9&Y3oIVrMz3B~d+J5)p@2iqSq5 zS96)r}ckry9_U#1{Ws7Zhv`dXjjW&rQ0bhGSQIUexb>e=UE6AEe|LF7?0P z&|arF7xlVJd%tZ4O?Tw;hDx`jYUBW2SAUhVdECjB>^H%_Wu%Lr=Z!XNJ9RC@sYk8C z`?b{XS0QO(Q??qnZhQshfw}a9f*$&jF2&pYmHa0;TC^(T1U~(;pKV%mG+>%meI!Av z_^=c7y)KoxNIy#_w|Gt2?M{00?|Y?eXU^=Ew!IzlB4p~`wN1?rqg9~m2joY|*q0VA zKDXq^lHspFVg$#p4hTZ9(XJFsZ-tbt9KRd@aezOy4Z>4Cae*wPD&hr_IcnwLwSC3Sh)VKVS{8V-1}F`@!E*!E9LrJ ztO_dcm;67`WnJYQ!~%)qI)dken~!ATIs*R+z(ILw0JmSxqjO|l%0D2Xat(89-?ncr z{m)Jo(;4qQ`$t(z`FB+pXcxdK&xdB2j}$R|N4cCf(?Vp<+28z^04# z=Y%m2VQ~i=c~lxqt5^#zw8gi3-n&uwd$Vab&6{5(j(^%cyj~lrU~y^}SS_m)(wUQs zT!M?e2Lqq!|LSlG+4XdYY?eoV&x_k_`}F-ySV-*rpHj%*1MZ*Z>Q0~T zsKV@fl0S;FD*0V?TKB5*4ma!xy?^Yz+HFDW!&rSY@qbhpUgSIZi~Ij9_tuaa|Ld{PP{^$Er*E$Ag+;~KOX zSJ}09oJ#TWgVrn5P(KfjMOYH)n{X~BoF3pdO%qg!64lIWewBb%{l$0?fBhAQ`yNKgqBm5=vCP_()~c2Mo^9 zoDvQOTXii_N)md1hZ<*{q>O{X1Z&>mt^+D5U=cYsB2at?T(2bs42HaV5sLo+oD~%u z;_7O%cOW0Y5sb@a8HBO?93uTsStMrMkMgxF^T^eBf%ia=K6`Rp9L-Ks5vl@q9dI-< z<-+PdcVEeeAEDlJ|Z7}AS`QAN=iTgYb)Kb80MQ2#a z^EUJjgM6TWm<@~sOf>FCZ;_pEVWVY}>0U$9*}ac!VX>1mab)KE_z0gkAFTVDV_htluy zQeY{tWZ=IPP!zz$5vf>iu05lArX&;UAAq~C%pj^U_Ld4A7;yrr0B+guOojs#b{cZ| z465JNT37-9$b4684(Q`ppe6TR?LWNF{RZ~$AKvF$fH>fHdtqQnbRdJCddoIlTIY+# zGARAN&v$x?Gm$am3$$4hDfu5XarqktQW=u`4M8ph^y^lN%SmK2p6s zK>j8?A!>)^@V*AdPurvKy(=~GY02jwUmdg2igt%z#ig%?6!zq0tNn?nL4(6FNI#w* zrWy(Bu1d_V=egIf6y#7QmZl{CnEyY{-_$Jkgj{VyA35qWZKLIKom_2WFgfZn4T%Ek zjO%k1RtM3)l}%5!zk0-vMTuSMQ##W$+;jN!hMB894SSaHWNV1t^TGQbZA?MyCcHYQ za*OUe{jtbX=erqoy-&-ril6EeCo8o~gllk4F1|YV$llT|_jGDtig<4vPkriq@Q+Jj z2ZhvzFUqWxqpLmG8W5}gir&`pUDxTPb>~Js?}?4LIJk_7yURS~vMO6PMNnJ%ecINS z^J%ue6=SSUIZq@@%XtcZ(7VpBF}vN9ljIROuKAllYPlgH{-Us#DXUy+*2Kh`_aIqTS>U%j{uq;AQTeAOsOQ~(A0Fj&GEdU_iNZ|44%E$&A5hdw@6W=||ilw%Ow8}{L?K&ynk zm|@>Iu%FID zdMTeUd8U=~J?lXg$jldzUdrmQFAP1Z!OffUfh@+1`>yqYltA!o&h!(S2OvKk_#T$h zJfmkjR`p}d=U=`P<~1=dy>SV&NA9&lrX>V8|UV6aIw zk256R-uGP-l0s%aqZnr)mA1STv(v_Th!p){_}FcZtr<`{4+awvTlSYA3rnXEMkhqd zkNl}!AO$s3^=yelT(SPrcyHzjv<_1H?ij`8;p}_E5|Re=)f1~(YAdfxUtlZ7B+Oti z!abH~Psi=yWb#ZNOZtEcus*B-+W=1c+7<3HMCC@W{)wkiNf%lJUcDK%+QpjgY0=&m zuI_RfGhPNV4&|kf^p=ZDuD-9)DTUJN5x&yptF|Y${s&8jx^Ak<+9xe%!eI~TE=V~N*#dp>DTOsMtZ&(8A&)qu| z#;#qx%ZcCe)JHX}z8J7vI-1br$JNkBit`eb1;*)Lu%wpX+MyL5w>se(VEUE4Af!v*3yRkAoN5$`&FQo^BxW;q1&jKJ6dxL&IH zqgc(yA6>9GDDEMz6A`{iK3f2TbJin|0b;8xeaCg4H9ZiH^6yRwuXp9|z@hjQHR{Pu z&aPs{)zkXQTYJuUdCEa{$Lc*5D2NTYRh;-qaG{aXuh1a@sL;fOg z9fxrYqLRmd-xt^>M1L>Qn+bR84UM@TUch2QJSAB)v%ku_iou&WDENl&y4 z^?Gw%%6_}CICA5Z^uAF3eF5P(rIJ5l*WV*&xS?ttWOuY^BWc$limZz6p7Bvw{sac+ zDMl8jTjpPsmw&x`XAQ&##yxaBBv>FHW!c`VTj(8(!4365{v8PpOIThvoH2eI36SV{ zj25wyK!7Q}ec!>!{rzSQf-?__KtAw%Zv&rZ-7M+%+ECe{bwia|W-p>r3c zA%$8ne(_EDKT8-k4p287C8Lx@tPgfmeqjlRU)kVQ0m>k|1fX>V-k7|s4+g{d`fw$8 zK;JYd28?^?dI21HeK436q#mx=29+N5rBDf-yaT|3=GXJnB>nTKzq5?PuQ_#MWa_lR z@*ncG7{3JXd$#ScFTz-j>l!_q9n(dzEl^*lj5au4B$r9Jnz8ku=M~_3$3X(M@7cBq zn)fzres~1L0r3%~JaH;4703~TXACYLEXf{>?O%RsA8`VC1o__-Foa|$`*+(=Lmw&a zzmSRL^~yyJB9NTmrAHh;R_(7VQU_VUKIlvW0SOY$PhxW9FKdg6GlESZ@2mlqqzB2m z0bMsKj%2-p{9`gW;&mg93c@=fbrl;c|RQXKP!=FC2G2XJ~C0`13q8W!n)|s(<+*F zy0UL(O=+g)KUp4Y)@@Na;2|45Fw3-!+#X7xas%HgW%}J;8RHr5FiieQ9y@(uo#-*f zHuYSQGhxFYI{W|I{4|oFFQg6nU{V@Bg#o`orp~}khKNUk{)iLJ;%)W$s8tIScY|ex zwE};gIj^9%tv%MIeWTIF*mImmf3EZnpe%Q6IJTn={;a-|fYo#qK}Kn^+XS;h{;ZfV{Y#cVGab;rZsLL7RcU zT&}k5z(b=b&GQp6oLp8ZlV_p*2pEeyNrfQ_Jq7wdkXM28R|Ew9!|!`A--WE4--P1V zf5>k_`!TRzd|9ER+o1ln=aY^9mSsaU!8$M}eV1uuYtWzvaDHUD%S0%yqW*g!zuRf2 zYl0yWs^7iQ+}GLN%?9GA=Pd=I)?!y2;J&y)ddZyN2G=-dK4RQ0RC_)@83;r={NCkO z#cWY9mM=pI2mg7-h2};z#HOBSED;mVjxGa%RP%@XzaIMRP(dkwNU-MO)>t4PBuUV} zCiEHnHh>d?><(lcn_4irFvJ>~BYJ#||I4k0&;2dV2K{ZbRAWBnm@nhn3i)M#KGPpB zlsjHbd`}zyNETJd%&%_qg`mO|NhvGwRwLZ z26XwB(DHuuS|i4x7X1mD|A0R8YJrxQ#a?U^$;c-T1;IC6nk>=t!!D2Z`1Z{YDBO{U z7DE7Mk?*N=NpxgDLMQ!=oqgWf@E$Y$7x(#-?nlGFI!b3YOJnU?GT!~3oTj*cez#og ziI$tC-?lum9F5JiH?Ihx|J_PdFvFuAs6ZENc zvtwM9Gw>0s&-!zmnyr&}e=y33rn9eV)?8yyuQ>l?W;5Sc9{S2)l?E)%rhmPgbJ(k? ze6@0KaZN<(SM`!2ul&_}60T<#zNpdahn^2$*Y<;GIxL`Ch5IDp2g`tzM`Q8GvyP+d zRNn8mWPfxe>nr3x0yw*ketIK2CbGHDsac)0y!uwtroh_6581efdFG(n_s_;dSi3P` zxpYu^z7+FhYjk{12X}GxVKe~q*MCVDVTIuL{s4|(9N|Hi-j9Ja%75R0#4Z#Q;OW6t ziR}jB7(^wJT}pr7fW&SPkse%=*lsY6K~xgjUGeu(8c-hNAO4;+G|vhDp0oyOE`sdu zqeyT|zmEdqm!;qL5X3_X1yfmXmamLzoa3VSV`t9kPav8~oO20w35apPGX@9c8^Jos zvGxbOz?caxZYo^jr^10M2id|)B@8e(<6G2?)C`5vb((*p9B+p32v7POP_u(no}o<HO-R@bHCOhZp?lhPd@T8N#jXKVzTsioXYu1@!^f>q=}1*M=Jv4hCPsPF-q#arsv!rF(0j zb(gAN%v7MJKVuE42dY|o^y{d3v#<@+gHmFH`IO4!DA#h9$$uOfZk?DlDS1}C2}<<3E9{s8)1 ztu`V&PDhGqT@$L6<@Oik`Qx@Xl(Ureob0!sy7V$*(7*z4$154{bchv54D7C&u`Yjv zG_%ZNUg*`?`Qa&Dp+*qW>K4|n+t0XN>F4scOSr=>qv0A+QNaTXnJSTsk_olK%(IDe z0$ALM_Ir6+&JP~%XNzD7xbx+k$w80K!PxQS<5P#Oc0^{pgYut%U1C3${Bs+gXaV(7 zI)LXZ2=ajO+)4=p(VNhazX{-8lj2ODNOq(m9vj<;;1X2;4$nKnO3*e9KTP>c^SS)w zwQ(#7c%QKJyL@RR(|ZirWbKDc!`Q2Mh5%e4m~RfnqhYGR`-#6Q{lIlV>jCBW6ZeKS zKYuu!@W>JpqvD69!O{R93_Jq|fQy9$$eqxdYZX>?LU|DYx9o>qg8VSDf8T25?}IhO zK~w+meQ+>{L)GSB0&uW_cbvEJJDt&D5)!!QCPYOxek;LaLG6m3?ZEEO1}Q?Bk}DJ2Pk=ez0_SqM#2=bMXFQ5(zkq zU`g9lF0-0H%G=<1(PWuE!~yJb;6t!=+OnyKWryxNWSD7TMk=aq)Rz>8+^%rT;q}I! zJUuZYmZa=|r{Mf=I#Ll5|C_*<1=r`5{3=qN|H~94TA7+gppS>C@d5w4!uK^F{qe=v`&sJo2Au ze{Xth^0@C*?$!U+|3)iPyFTD2uG}l|Ggsi(Q0Fs*1AO88!P@P5v))Mzl?jGSV-jjv z^kE4!YHpJmr%nY{3|brl{#5wkANvD&9Ew#JT2WdCvF?8BV)J-k%<2gqlBYnxR zdUe+~+P0z3#uvVi4ce zA=~TQ4poYr*Vd5ak$Z9f(2kcsoumYxa;GcN`XlVuqF0LXoH%n{8NfK|7dSsk3DQq{ zaaeqjjY7^veskq&%Q7DtreJ^R`()3}HVfgQ9nM4Bd8Zw7LIQgY#F*|BPPTuD<^HP} zMX~!!^00c21202q!-cbrMN-h4kIjgpG>tkox7!Yf+Plc28&|TwUh}Wyk9Rdl1f3?Y zUljjIE4IMbj+)$>|kEgC=zP<0`*rhFssqU%A!gfA+v6hiD2Ez+MRv)M764$ z{_lBOG*XuHzP!K9!ZAPX$e@WU19W@U7OpQS)B7kdzTlZXe0Z1_4r3Est?J`u>!nUH z9S!SHuNvB1#27vwS)};sq?XJ}YklVQ?EJ$fSsFXpPBcBob(BI0kMMb#b%Fo7xPIrl zE@Q#rKWgdex5YQXgS@U6#YJegvT56STb;D?^s?gydqSdxQ}v>v+40i+FVo6Q{A&s% z64Ql{aPH8q{`JxaG!D!J5Zpg!*W3^%^usdqsA~_ODtV-2AU#%>+&NKExgOD9+Y)ni zacUyK&^o`1cb37U`!1W&cb)?`@h!Gg`0niw-SJ58`rEAG{=HiTU6W+mUjJGxdfXXt z@1yz7DP+qd4afdLwS43v4R=of-!*M6e0jI@D`WCG#oriMRrnQE@57s#hCA(Y&R0#F zsvyg@uGL6XecQEVS97NEz_h`cuRGV=cbI;MFTaH`S(mO_EXdwK#6iR(+dDvdZLsu6 zRY9$2LV~Pmn84jr6(ynjW;foo*P1)TQ}wv==HTo~@_}j1vI}0`!OBnMcedMBhwSnh z{4iscJ5@Iw6S)3I|6hFhv&DXfOugH3j3(N9D;QEZd{)WM>ch5<#NWR&taaDt_6I(8 zKKBEK)7*4dU%fk)Hh2dS^lDii`4#@eY4>osY-RuX*Ojn9*DF`51PmVSQQ_S^enXq~ zht1H1D@jF)cbRkJj|ul&vc5WmPF@|hEDvPac7@9F91G5KXvux>JbE!K^~RYO$Uqv8 zTzS=l(Yl=#-Zy?->b^r0WWX!U@G#qDw|1fZGkkfX`qwu%m|!ftKOb$IscM!K)Lop} zlv=TKlFb=GsR8!Br9- zR+$k;d+-G|YW|qNK6oS0nWI%DJ63bg=GnIG0YwSo9LcWJ9;}->n{_(Ge(ru_(sN@_ zhKe{ExwB^UUoUj~P z`BW7szMvorlHn&Kd@o6JYnq-BWgSCX$(YecJ+72EG`!<=^~*@rl=eI-vXFe?k?PCI zx7N%m-LLY_Pj=mPf0Rf2rZ&x7Oq)_jR!}$YJr*T3@MG)_i`Tf%r`xI)Yx%1tPIDIX z`&<><)N$w;Lu^|Gx&U9$^7vF%_Imk`XZ?T9bvVa;81Z!(*ZEYdb$%P)@yWE zHqLvEj0`>rZaIA9ti3VgNvEgf0xVWa*RQgf@vPsUx+Qzv&Oe=o!Up(Yy!z7qDfvyx zqE}k*(1`{&SG^!z#u3x*Bf1vT;U%l<%^Q4K((q*Pt4s1cT@s4#vtlm4D;6_Zh3wg4 zmQbQ3mwA{=$&}ZsQ27wF{s6u#F291W1FX}Z&Jin}4-vc9*Yj5wDWxU)%q+3)Xv`{m z!Lyuae(I?35epEt;R^A|1+(0CH|l*%E?yH|rO@G5BzAhStmN(Y=sI_k;2gsJhx?E! zCWOJM*|uP3(B7J{vF8r2t~2V!xJbkvzGsiC`^{9I@NTd;VfIy9UloTXSJL*Iy-VY= z`r~n4^+g)Hvw!R^9}IuVFjElSe~HIb`M9XwQn{|Z4ivxch&rPxb^jmCDnD7bBn`4UiFTO zhKr|KBEodaEEVTVP1g?V(_X@tCsT*F2{~z|2;I1S-TBHdogcocVLJ4ezvNtZaPn@a z`@1FKI=&3K@!$XO$J-3+5U%NM$GYv-ms}~%ip>hU&NXOlVfVc7vQY-UJefL_2PH%Q z2NF#V0NMrufjq*@3Ibao{>u4VC}2j4Y7J4>SAqcf|B&aU0dc5WlK+1h@t^`@krnc? zNBe10*9U_*s(;_Kyxw1JqcR%^1K#sfzW+9>#f+z>pnf2?iBQf%_X=6aCn(f``l#o5 zg(r(5szDfVy#;uF14LRnA^ogm48t67ZOZxE87|Yv5lsjOTt_BOfTYCy-pK=&v%1s@2TF zd%AW2c*4xz#%L)wVyso|~vHdo*|MK4(UEbCbJ> z`_G-3zrDVAVR7Em$c1xulY6^DcfBiji*&l#m*3;RVCZ)!>8X3?+lFH;Se#Ew$tu1@ z>HPCWj`IZ@yIpkC`&kyVM-QW3hRervJD$bmt52HARBgO%l_nOU_K;rqm(wTYuF@9S zlCWPplqC+#?|BWqUj?q?RK({m&t6#cBvqmQ@{HZ%(^r;LJMp;ekNLc@CfDpH1mrgW zINykyDu}H;&%<{Am{XWem5cE?W|qA7!uy=BZVGq&H%)b8`{#GkBI_Qrw2OPxZc*=A zg_+4`IxH$*Tq}QHs$o?q((=Xi?{ut7;J8k9YhA|c3=g>Dk>D}z5Np-xT~jADex1(R z-Uh{+fL(uw5WgdnXwGVe=JKQM`GJY4Ny#h^oP%~S+r5jwv<~qDYxj)noYwr#6B1+H zvr%rX6H1f6w{MI>2{xCfDt_aTmaV77;sVZ|j~3K{ec?Pb_F*uLimtpekP;_qeNTOpY literal 0 HcmV?d00001 diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_351000-351004_mainnet.calldata b/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_351000-351004_mainnet.calldata new file mode 100644 index 0000000000000000000000000000000000000000..f7983ec06bef5f7ebf4fb441aa932fe44fdd7e61 GIT binary patch literal 72932 zcmeF4XIK%sD6442U^m4y$X<3g#SE*PH|9 zggFNc_kXIZdPdfD-+Q0?Jon4>Lt9i=SDiX_>U4E?{pR$@*65J*tAFOE=>y(e9&yvV z-l*R$g-_nLdf@dT`(7>VoB4Cb4LiDj>C-0W+OICRsbxC;>O7r0bhe^A=0na>Q-%%Q zc>RcR?|o@&y}w4}Ejjq}WmEIfOM86#uxr7%S?c}yW&15mUv{ilOEXof537=MNEe?u-`;#`QT@*CwYS}W-<18gjraHXmX!<4E+2Tc#Sp`)SS=Knaq@Ym-{wfgn>zP~;#t-kf^0(XCX+T1k5uL?ka`G0jD<2%&QZbIj{4Nqbh z?l_p|ZgQ>eM_(j0&Ur0%YS|t4{hx$a3i#Fabnd^#|G|~ywx*~C<4=~^61YBog^$PZ z#Rt0e$kr&`$^K6pZ_4VU-Z*yoli$IJ6Q8C|v=@BdEM2Y!_6E)OEbG7bT-$d^np%U*|5?5An&s9WYKPY^hh!;S@qMEb4_#f)jBk{+ z=Az=pW&@tzzB}Y!^zdKu8s_co_$7Jbt*`5Pmu>#ixXtVPt17ll$-QQbC^6_<#dX8- z{}2w=bLZi2E~jh|)C$J*V0-I-);|Dw*x8Z%qu@g8^kcz}=f3@i4bj@#;90hQ-RLRC4QOG5)&Z(CAs)8rhdSK3Bpi2c;_ zf~{@2nF*%biuZ)=i_+)af2i=O8QmxOH_n?dYSP!_k_CFK{4z4_;U{B`=L_0d%{PBP zzbVxZ9x1!FT&Lk-QI*#|T-M(rPdI(b>RDuuOfD^h zTJ19)4$I}SZC1?uyB`-1SwFc|__HN1UYGXIyVD|XoHj76|E<`1ZcFP2yI%dgHgkvX zTc)K7`rFgDt)TRh~;# zSO5O{t|icG&8$no8Ec)&F{#(_c&mK>-Gp~dR^E-sx255G_mZ#c@61>BQYT{xGCvu5a7cKCO2mFmC4r`|A& z$AeCP9h?96oXNjGtM5`jvUk2|EB=Zqa*!*M(jU3{m`q|oil9-xG~MveyZow z3Xd-*4*7Fm_==fUx&6gs_Zu4Doq3A`+{+@HLTB8u$g(uExJ7npQ+_3td05+GuqmBxBdr!jzD-H6 zS&TL{t0$%BwnBO9>34KC_Xnyi#{kMdos9fxBbrmbeJ~x>8#sDKKccg7dFhylc9HX# zt}6MFa=w(Z$Wq^Y`;k_&Dpd2*@nEl-sTrgEhEb zAeG6xUDaV)p_yKN*HHTVFw|)oT#J=?hWw1hLn*)TA=1p%`sityRgvg>c=Ti@KgPwT zG*>RNh_s<~(!7mK z=@RoOKX8V{3|&E=@~*@gls}c#wmFI{xlkA6sM+pYOg3f0CyUXz&lh&$vad-Po%O3p zrw9x zQCd}*oFfjPr-RdPCz~t{tx9<=UZS)1@l^g@PLfs69RQ!DomQ~FC8SXV-=chtwO z>q&<>22p-3l1I(+2_-$}kE8s6Ih4O4GmY0$S32uQwyNeTN%^k_V=P`An9lNHKh~J` zztgqRuP7%gwC7eKHj>)Cahl3+DnRM+ILh^xJVc$z=lfumZC>evbnFKeWjxO{XSLNv zUqX7JevBH8f+|40cDQJs}@QablZ)ZjUX^>h8xKJI?5P9spE^9R9od+RDb2$ z$X80NfwXPvZ)7?t0okSvfqlvw_5@gr($?|{m3cOkj;FsuT0<_3lq_wYiyY7IK4?X? z6s2o@4^Wx$3E}9|KwC9Z z=b-d?%2#tY!u%RB8FCt|#TFyS^T~HQyY(!NE*Uyg8S6{ZPl5ZCHo_{P*7SwCYgNae3G&phBbbXfK(^dN=0jiB;x z(;~g4%rL6oPLe1UdXR?Cmd3SZuT;g^RG->Y{^`kdykC)yUS!pt(eH3J%}KHiH4A!b zUURZMogG-jVw9U)A4%yIqgYAk7QO{#b`L zN%~1!nMUL`=|MH+qMSg?MBG!D#y*5nPJ#5WmX25BsGbI`VCR#zyrnW(*`3bYLTMR# zh6TEzzk#d!qZf78oTT#Eh9iCLNCfrB_aMCPdcR$)MU=EDX$I4A>p5y=(*Q~re2t@L z^hZkP^utk2$Lv4zILJSAVhD~NKIn^0*}a~wN^=M~haYW&9$L6Eo#!Tf#`xT$YmN2b z*<$aJH7d_AE;gkxv)ftm(9)_8y(p)|4yxzcGD`2>4%yU9Z2Ju$Nk%iA5LlDFCDhYq52nif>j%q%&S^{!|Ri_!Y&{dDAeCT*aze((Tt zu9{wuEhyq8<)o`gM}Lf+P3fJNj=2|+tx8PtgnbsHq0au#lz(#yl}vTk;s*U2DCgFE zI$kNn@<&-z+#^a-&haeJe6QcC(=kSh^pb&67aobHa;CvunPhoDHie zCv|Q}Rkbtxq1@;``XWbcV%jM924nYI`8X;$guH}O)tky^jzXEHrav$*o~xX%xB9@=k2Y3Uxu z@6-kw71a)%nk(GuiyXJv(2=pVKfDae+=oo3TAiVK-mfF+vr>7b;We@nFHe+zU;Z-X zrwOHeb2W?XI=awiN_U)1$JfJ9!{BGoi%atZk(4ugJRP$yr_s&iLFvm6p+PBbE~T%f z!8lC_NpDGmIRL+DQ^qk%YP6X&a)w8$n9)}U{@=&Fw$ zC@1@BY9#|^4m7~*dlM{Ds=GK7owd*ysBY`2eCTExr-Mv}i)-lazT^S5vLC&ZY&jFD z%)u&joW?wb8^&JRe4f06>era^FSoZCO(7#NPFS(Qhf1Gnlt5q77HURkJ@JGiOY_M3 zl{kza?4UNK^H-u)_9fF*zK1DiT5&onLwj=h@kYvNQpmhsh->MB#(_W<{5)^1CBJ)RKF%TDy{3#@zxf!WjMT%WNSN_j;Dtp-$jA8Yw(T9mSUZ;g_kQC)G?GpZY1YadD_7clMR zDNH$5cx`$1&2Y+Jm5Xw`yJ6m`CGe~QITgyk)CRRZct42JPkSMK>S;W#ve>Spe$|fI z0dM;d{xM*NN;;{Ck;1h|W4uUmRlz*PY8W%frtE5tJE(Ob^~*DH8s+CHP5E1~o_1-` z?>y?awt$6WcEr(fe0H))|I^6tRtjDranTI7vlFU;{a=%c@dCq!N z>pjwz#dyAwUA|#e>Js*fW{vV88|koNWqZENd)8EuY|5z?_I$=I$1_WY_^N5_c|Je1 zy_2k|vUafNT`{8Rl1Ni%x0m+3vxEQ66e)f446)}k_;&G)bZM5{lqaG75B6%lrxUkF z+LZ7pUYTo;2O&YO3))#uHc_F3aC_dz*`mIn#HtDQZnd7gZY)7@6ZI;l7vo9Uw74a2 z4r}eJx@dkGxoB7;ZMWzBlDo7=q&##)j6Gkbz?uyrZECAyv@Ya`eL_ejz{OszK=aaTB5@Gx$jjm-NpE}2BQL7pE&sxLE9|)srt6Xp zqt$r15M4EX)RU!+_MO=+rH4v}fn9gn^A2BOx)*86GQK{q%-k}^)6mwHv*fxk^oq@< zdW7;WVZ_skKC9-O_DhHFRCSl6=i#CDYWaQ}eiZHb2I*?eGv zc|6w;U*26*=ALv-lvSmEBB{J+qbFS=^~?OLip!K6&>Cyl8XQz2IQ697>hT?4aUXD0 zF}9&SpW90RTr?aX?bVtz`Yi_x*S20%p=N2!E4HTD0@<0hV%71|UakD7g?-SY0lh`8 z>N!LouUj>O&={LQ1 zU*__D7`}xBh+Ow0-$dDr0qaG*N`NcRb!kM_$^U^~`@C#A0+GwRl3+pLR|2WFt@T1{P^S0=49r3$hnt^M8L4!wtr;Ux^)zK+zd zE4@%m4%PZfAB~W)yb42ni(b5}HTq#{A*Gt{DWCX;ZJ$5um4%r^0cCI+d$%eN50pm3 z1>J@0yQWqxxi4;CtFyhmZHkXDTs+y#)~_1?Bj+}g*j`VzDZ)+7rugUN{>vqPV<)@s znQy0$qmAY5`E-Y`OVlr0Qk*AU8tts44|q{kFYDLczBz}bOB`zW+LM=+cb(}VyqePP zg%}NG-Fkbsxzj(Kg|91!QRZ$h>j?YgK7*Eq9Uf0u`Lr!cLTs zalD&Sd}B<4w1u(-M`Gd7NH{p<(Nr-Q?n$?WN<2@Ouvh)CD`P8^ZJm`5kzB62!5%n0 z|Mq5>h)P*eR=FW`Ci|@{Eh_T}bC}|bqq^tVy;3MjRkDhtGAx*jBlWtAv0yhTQNK-^E^KvR7>o9=aNOczs3`P+ktzP3m5(p8LKo9{_8DUl)5PbIJS# zBsk=bycdRe-&eXx6_}=9mgAxDine3jc&;J7_%glhw4pp#`ub{#P^#+lNl&J(!N*OK zd}@?yx#xE2<#kKpEzR@IylLz6GIo34tqWIXL_b1sFJ-1^jcNC{y;{?IOVUJQvfmZT zR+WytX1ocU&TkfYOmVn?Y2AdSDbXXu$km^Ggx4THd{}~aN!q;Z9@nrO^C63n*z@m3 zdiO)3MemB%FRJT(a0Vfc^5vO5@8?;GnP3sA#9c}<-I;2ynzQ_d5OiqCS8AWkUIdr=5)SW>UO5SFGCa@=>k z?&x=lY`78f<;OMnRc-1htxxjH4C5Lc63O`V2gG=)1~<_+)rKT@3-MKjcGO~(#LQ5$ z_ty(Fd!`%RfODb-<>P0u?r~4LrxuN;#&l#4DGwd2Od7IikY$P0Vb zPgMri!vf}dC1Eg@bN6*|Ryl4~L6i6p8sf{|*A-Rb3zy6g-)be#HGXXp6>QD$I6Tq| zV>rC3P`jGxvAwI;)U6O$DF@H2&0jS~r0B9o7~DM5wXtfac0qBTtD3$okHQ-;@7X#P zZ&{DotB39IzK7nq{UxeULSlsdDbMs?$xYi zLwv^UynsA(IHol-89A;)CVMy6hn8)Uv^B^@ADG8px~fhtv{zi6b@(DlGUVvOl`VZU z_z3&+{CY|_5!F@P?G5n(lXP*$Bu3yQ# z#>5wyQpwTY`p7EZ#26~oux4R3SYKB-&lgLEDK|Uu%WTT}fxKb4)46CNnVMw~m&_1v z(N~2YVd+I#)8K}B*-T6HUfdcha+MI_<_+;7f9pzpb4#qp-iN#t{>URt_%1_y&~v@) z>+X8J17Pi%2d;kWb#VF?X(rB}QP`%Og&XJ2VAT#@vGw#c;oNHACf+K7&N zCT-FSFFPTUrYqTk4E56FJlnAl-tisO)fO^Alu*7`)JG-+&q>VktzHcmis4Y*>gZ)_ zt`cLQY@VmzUoXDbYe@e@Prk3G7kIN*)S%|iAOu#L)#O7bUCZyVnSf}%xWHG20{k*- zt!({xz0%j@Lb_WttPu4o13T#r&%?Clp7*V)7YGq5wkeIK@LVa!rrP>tA(MHo98u%AHx4U@7)|m2Gk}>_6cjUQfU$*zpUafJb$TCMjx@dln_1r=*2(HT4!}S7rBY6Qs#|?IU zLRD$Or;apf!ww;X@}j+{!Sv~%-FJfRI6X(a#m!n90gtfkJZTPSP(h5WI-sv^5Lq&b z5!g@|o`uf=|K@xI+}0P!qI>^DJk@FSD%LT&3@caItFGSH6>E59nW)UBU!(nUp(Zd(6@ylEq9e5}7pt|A3 z4qK+lFvrLr-b0OuJM*6~3N%|zjQ5uE`>r!T5X{$NxN$ZyCJuc8(%rfPu<2Jc- zvKVeP{T97!SV_Ha0rNzzn(2mUO|^999dcRdGtq9pSh(oPNV%Xz^w_YxSwDT`6CK{K z^io|V)=<3(!%ERjrSu8e<1pyR30mmYR0m>GP%jeM4mJ~{~NB5^;t5m3KGvM zO839@%d)uY6LA@QCBF+^?JXWz)iii7$fswc`T8)ITA=4<4Cbxb)bw$pL$gCyiw+Gv z|4yH0(I#GlAwF#kA6xm)fpDOF(UPp_1uA_KgYS7NNiT4=n|*V^I=?-KFFlBP#w9D7 zn-91om2DrC?rXPp`Fd}_O*b2_!Vq7ygs4pkna-~>#2b2x0;iV31#s&K5dAcl-0dso z{ljV#xvd#nXTgkNE*2*0&ufrQ)hnrwcOu^Zu@|p+{e51Sz5acBH)H72n~?oeUm?T$ z@>un0uLo35eQtPu5%b73Z-KnxF-mD1BZO35VyUBzY_+Q5A=9PN?m@cc&uXPNoOLB1 z0YiN5t-2mAH4`;_^Gd_tTjOKXNq(7iLaA!OhM8rC^!$Psc}G3<+D7-#lhQ!_Zgm2# zgAd{syb{os4oU>vB7c_iGVrqUv^f}0?xnqPGiOPu*DaVWOInxAjTQxzq>fxYhIo^B z{x!rm)0g1NynCVswI8~{TXTg~a^rZLT9>TkOW|0R^$sU|I)bZ5x)ax1@6etrJSj=J z{u1>n`&R4qPJ(mi7Gw71P2j^_Q?U$ILNLF%3`e%}X=VyB*5H*H;`1AX@qDUc;knkd z1yLEFhwd$eWZ%3m>Ae`3Q6$yWTlFR~FLj767WR1EmPH@$i9$&1XKTT0kOLR}rq|F) zw1#)baPFFQ8A^=S3#@vgn@O`f_Q_ZMC#<5^KJSD$N}9F2tQ@nm7N0rN=ANgyN?bcv z#jt22@0q+o)a!l}v!1mkn}(OsRc)Cq$|?_Fa$J*LFX*`gu>R(`&pQbjJYSaP>XTz? zcGTyorGY)q+i~TGk}0dAtb3Bl_E`(Hj9@Prn3G+^nvNWNl%`2 z6_Yy0>pUV?O$Yxt<+^DJ99__QVa#gAjk?yxHr8{~!CA0Xm?1vzDIt%tE)sY1{0AG4 zgT$Y6@G-EQDJ1UZQkE}sgp+Yk`XGkRJ?W!EnTteHnU`K~E|<5+Ro(2|Nz0v@WD>F~ zOZ1zOntr5x^V}TXGq9rYi{vk&4Dp%E>rxJm61SbJrN4-JmB(UwN~eyPg?cP5K4N;R z0gH8U(mmu3314Zm+x_MzMzzAgR1f1#7~*p-;Yr!lyRp95@q}m5oUNKoyg0+#$ZC8w zk7EY#JDL>ao6Ef zxq#j`%YwpUgj~MO;$~vbJaLhJk9$*}=K@~H6_s++imR)&&dexRX9fgm|VaCN-)?l9W zsoWWbZb?PM4;`J|KWKfvntw+B?t0&U$RFd}7F3v!etOoio_o!H<5B3gRAlU@MgH6K zzFcs5ZT_m)d=}Pfp1%0$bWQpw+m}ACS!U2<6uKkj88p{on|Uy9XUr4Hnz={L$hT99 zEi2YJIpOQ>zGYAK8-PN0rN&c=Er=QZ?qQBU-enAo+138x>N?r#wS9DAPo@*k{(5q$ zF%+};pNf&zy3Dbp#c{5$%d!o7i%1*3#Hh8dN}Eb$R@1+gHar|d6oK{giiUL^%!r|2 z-G$zp4wS?Hi!yH78E0PnQCg8l*SYyg=3MvtcTLsY{@%R)){vftwcis~oe0dlVchSo ztrsI}Xw*L-LR4Wm_O(?W&m+O3hM(#>W56?)@87>C-<{W4l1yDqCnikkD8~=3_jsAz zoT|W~qt8tHdtF%J)+4ZDTfJGHK-sCP+`y)mIzW5R;=RkY%V#qd9}W!ReE0Mj&)jQo z+TXj7cgF1HvAg84muo?6)4Xr1ZS#0L$Gc#J*`@Nx%HEYPUX%{(4D|5`o)By9+c%9W z2Vs@DjQZ@_GfRhzl`nf*H|(G8jmpr@Q=@rpy3rYNv9#e)b?@;7o69>EHqAj7+oayL zoM0RLSsObt&UY_%%#7~g-ydGf3-&#urKfHjU2l1AoGp=%_U*K>*F562{O{jGGp-5^ z+VJVU&2{Hr8{~lZZT%2Icm+q1B>5w>gFZSMz4)Z!xM3toy zQum23nP%~y;#|wIPil8qp6xH|*UmXNG^qA%X{$Ek)K_(NuWyy&GFlUA&-BLd8Vc51 zK*KEUOx3WXM>@a9r%b6A)qb8nYcAw83LUvN%WoA1ybVU@7m=|@JJZB<%#i$7{d$u= z%_(-VT&r@(SyZP}b*awDy2C>+pTVDtL*0w^7?E|`@um3o)Ik6C1VQTtX?;(?D4h#7 zBz!Hnw_YKm^d?QY)|H13PJFh?=IX!dO;GBr{=?sI9tuUD3zgee7|=W#!@t(GLg>W1 zv4`6Q>`BBQwJbJir)NP&mQ?ne{-98>-W|<^>)oZk8QRS~lzMa+{yR+n6|DDw^;im( zS*dn&kBBQFOOE|QmXyW^2qe;I{qEO5l)(OlAOGQG!*VS<8`iyUoAnIDPkvlNE5LdY zCB}gMe|@NDZ-aOEq1omeGxr*9=Y5?o&4W^HQg`>Oo3}FZ54Sd?TRQ8+juQ{orz&;S z&~9D{AtKU2=!JF5XkWF{I_qmGd+LnikXe&52agR?M~9ggfAUr{j&t>29hth*I#>VW zy(b$ZleWcVKW9$6WBKf9L+y^>psPxX_Mms@$lc7WtV;U+;u>=IU~>X5B_WF>)bbntZJF-@L8!`#*1m!k8J4w;gh*QiJK3$RyrEJ z;NqaZ8UP!}wu5i~kE>n)1${Qf>dA-_A?Nwd8Frn>Ty(FZ%sYez(K79g8y7multAmRNgn zq4xWGRp}Y>rh3<=Wo~TpuiO7>V#)#epZ#5Av~xLGD$>U!dH8ItoUYG{?HfB^c@{aL z#q7H)`uj~ef8goNhpY0Xk9~+;k*-x99y`75+TDbA{WhG7E8}&qLD{LLFQm%dfAH8U zLHCBO!d}vl10#xWkxG@yU+q+C|Mh43P5$Hd&hYJHe@n<*sYvEARUQT99P_&SK1n+8 zZc*&AdsX{Hy_>#q^~e=-dJnrWs)PFpQ@+TEycL?aJs|zpJm@iS>!LL3l<%>7FLdnH z+4ETHSARbkk=HkHbA!BB-iH72+Q;xA`sy{esb^RIQKf@N*Ev`3L@xfL?v{bau6et= zK3;jZ>C3l`w)(*Gi*9%^{mH$MxXKUrm3ZC1b zIC8t{`$$P@GONMFH3@%3G(Tflkh4{e1~X2c`TbSpB|XF5EDv0DCUCV?_RinjyL{t8 zy^7s^)I4Kq-v$|?eR4g&TJf(&@mYH29-e#Qz`Zk14nF3Zc2l(l3Jt-F@>6my?p8SA zLeb9g7m~i7C{u3n+N0kOcKcR)QtXw3FU-ErNBnidJK#-tmF*k-^ZG>$929zc#p;Xd z*Sec#)~f2=_ef!B;bIRTx$5bugT8(&aCL=?tL@eEYp>5`-80-8USY$=Ep;blmXc>3 zC=t#I%`ZB2z1R6AV>k43P4#NOcdPzSw(NiMC^+02_uIMy8Rp%`0R%OM3m%5HhB6*Zey>V1?vQ5^v2`}}GrVYXlywpYZsD%yPvb60P@~tInkUY16 z;Ygd?+@%^C^}*3IX$`hf$8>!Sv>|HybgCimR$Rp(Q)~1YY zK=mw6P227B6hXUnS{ApY#WQ_%q+?%FUp%tVHhi@NZOT@LQq4~AsWWy8?O}ttJiSa+ zDyhMz8}?9|;8U~<`K8e#u_0Q+P65cArSvVf2_4v}%&=2{GY_8vaadsYU|0Flk%6d6 zr&y#tqhH|Y1fMeO6dC|_3LDU|Q;agOQyzz<^V=FtbSjrzrW>f@6);G%uuSk5X2+7FLo3_e9KoZwSseLGJ1Iv6DhM|EKcs)n7K z<(N#&GZ1_VScz=j^Fw#4|C&4H06{glO~?^J`AO?2X9WO54Mf!tR1H3r66eK1+N8Y` zF}qjXBKu(os<~Q&Jinf zBdBQWiGm!@mly?xqZ;H6BVUo^pW9N-U5R?_1f_aLF}=C|fp(*9=$$EZ=1wT-gsH;P z?$MmJU@l7=o5N=^aFy!ERlrlsBuPFSsFU`SK zLRN7tA*<3DBOCWm(6dqYW-Bh^FlB zKq#vmu^1yRrC-aWUCu_0L9CupG|M!IRasw%(mzF6F(PlhaF$S3XaFcHj)byGquafp zd_b&_91tt=0kPs}aiin;%8+@}q?{<}gteLnK7xHpd6xoCVsNWL@^G0)^$^f1)qDYc z%F-(M8Q4{SIyw+n4!3$ng(F|}3&YVfDuU9*%TtR4x*B^lD~vS5T^Z==B3avBK+O)^ z6?Fo4b%AZO5orW{!iU2P3D9-HCD=x;v6x?`)4U+i^W<7ceD`Lp1!SK2l^S#cnYR&?5HupV(^9WOAXe!zw zG!>Q%?}xi#9AK%@chAsS2bwC4c^%Kn_)r-gO+^{ts0OK2s^*jr1Qql86&q7vrv`?d z8l+VF;4>L^it@MJsAfRxXqQk^Ie8LBL>d!ulky$FsiAG{0L+NwZLm-1@DrW=(gNwm z2gn;s*_Ofvil?GGxbg+-AIv_i5TJR?M=UZ{V7$`Y9fy2EPyvc>C;14vIeNSRaj4VWQM64UGijfbRdEk;V5(AFWmuXuVU3;2 zA7DR^U~PbL=V+=|Rmvwo6{DGs(hf8gp9LvL^<<;0L#dEihf=W`jKkb?!clQn$5HWi zXjLfuOx&%d&~uxw(2`5@A#6SZP{rzc^a85iuOYRY5>qvg_y}vUsriAjO2#&reGD_j zI{68#*KlD?H#+Ny*(deQ5JhQ%O`S1Qc*R7V)u2)}=V&Ses1&21R#UUj%=7~e4s2ACrsb!Sz(yh4b-?(hcCDa) zbD#uE!>S(EC`%ivlT<)RWw}}r=$V0}M#IY2RZxa-QfS~;@Tg6Rh3}F|DUcBTYC^}1 z?1@YVM$s04Q5*q`ni(pJIt#)_abQ%rMn5&+gpI-vOr^6xMIk>RQ5*q@8YGtp_!9<( z5{?XS!;ny;_33by4-6FfP5`L#@K5yj+NcewrORN)=zq0SxM*VW992@D&;i zDrKjimw_aBUV{urDD411U2GLP!_K|>|3&!(fJ!$SV7&+AcMEDD0MsbAXn2T5Bn1YF z+B(gGr_G(ZBh}mq{tc}Rqa%T#m@l^|%`i}q9p|gJQw9=TOq!R}WO8I1&90uJ88O()_ zqK_I3$}muz9e_QN?2Dii<%!y#=pQKOh;P#%C9xBTeGZjlLTc(C2MPFOxeWNM|LC_D zQ-*Ysi`dKQB>;T#Z8{23p4taULOaF2K=7RWJF9!r6<$F02nGklKCU`Zo^p5e2`QDY zOT;%~&&N-NPLwiGHro*zG`{m|YW&)B@15N}P zV}~t09O)^0gknb`Ga2cdoBsLQ8eG?2^?ksOZuo9auqLZb+T2X%Uz7%}VmpWDI6ICJDzgGkXQ{eLpYg1Sffv`6njG|Evv6vLFujhGL7ccx}nSy zdme4dJVoEd1e#N9M4P`L-_?DMbnJM|EiGgE`r|2BE z?Xt?G(Zf)d@8lt_Q>!c{{HUJmgm{AM)GlLOr$%1OF{5N&*08_?6W6xEDn8il!CNEn zlkaa12Y;lQiTjc7Vh@*^0`cURf$QXRmbgw0V@}C+3c?y(=j1`FbDWYin3FT0`LR)j z_f6+I&3zU+c1Vvm>5_r#6uTeuVs{zqg&bZCydJM%O|<9`qdoa_A=*=OgoJns#{$Gt z>#-B!2{UORIu(Y9lWvyl~Td;ea{U1tM2d zlu87r*W$gE@w(opMo`_&Q>J*ft?mSHYSbp{J)<_Y-Wvi#LqrvdS014gwZ!yb*)Xy} za#zWT;FL!1O~(rxCk8*W+mBREh=Ig_Ik}iI%ZiGQhKKyl%gPR}(>Q5#W4rIAa)XSK zmJT1b+vhf#yAd1aV*1P#ml61>RR%Zj&vT{7uTC5lEJB+_M0;`rRfzV~CMwaMA{htw zr#TMK$JlAAjXh&PwHn5ZI>&`%IOzEeYuhYEhX8(Vp0{?uICv8vo?23l$;a?AS98@KZA{w+T{9d-*?$Em#qWd(kUh$Ukf8%MhIbbpk)}K3^c7 zQsQ{A71ffot4naIYq7yAE$2;t6oY!SB>*Eo*(@om*6k*r;4=BlD|%(~!5&FCIIhi!dG+KX>^$eWgoN^$1P+wXy74Yu^)`Tn za^p9x)OF#+fin1$)!^E&oU7R3OvGn5mnJI@Li6Q@am8xzr{>BC{^XNY=_c+KF3tTX z2ys-M_>^P53MY%l>ndC_ImXjV7n0adKC!`mYK{oZRfJHc=gW~|aES=jb~WP-!WU85 zc=xBgtVV>ortLNK24p{n4d5CToe~ahP-(zL+BB;(tg7j_7>%f%gZU&kWx5UGM043h zj|yEAJ*tg>YMUKwxIXFe#TVG%DGLZhT6Z*X+`m_8a`%FV!uFP&9fR~1mx@M(Fb`e7#J!R}!$ zFhQlc zziBe4l=V#-Be&C4@^K0m-EeJmtlk7rQ>_;ULQVOK!5v#VHSL%J?_JQrk@z14#6@ijgic3W>k=TGH`9=~&rMM^*Eo-SGR9(%s1^7ArcU>(ex+lRo zv7aNty?-*V46a9a-D;tH;#UF`9a}Z_s0*%8nV>L5i z04pCD0IXt+PE8G#tClRZsta!I;SY4#>$lXs&`@k7*3`zZRqfuT%=6+c%eit-5w7s_ z_(GzFlz=Lq%5}c#Qm%fY57LVeF3&HpRc#FL?mer$;K%s>=ip0qDi!Z)R|FdpSKKV$ z3)TuORa1&&TJG4lOsFZ>51>-5HpA;cqeI}idI2Iyxn=t-&{dUdmVIOF(*@1Zgien# zPE?fDIZ>NOSXMEJDmYPX5iX5R9H-lG3QknBo-r>&MFEenY@%WHM^03X9EebEm&T{P zbkV_nYU0Q-TZ2U36i}XAB}94hq?8LwIdl&5f;M0qlC@KbxYbv@`jr~L4kEUbM@ zJ3OfJNtpueWU4k4eFfBjS&St)uM(WT*5am*o*k28rnc9w`-HbgtU0;il}y1oukGx3 ztPmE)S#6<)OI>r#Q|84PvJUXmi@(O>8Xv|6gLzVp8H8WqU~)*`%zb!^hLfSw#AU>P zYM6&)SP5g}{UiQUn;mx+Id0;Fck+o6f=R{8g7nnXjgNwvL;-?6wZ{p$LnxlqwaK9p z^r>Cue$=9ub?~3cytHC^g#$g(M3*CKtkHSG_AU5jh62(7QJDi9bz(*Fjbkw%Mkj34 zt-{);`j5Hj1gX&_OX*N~>yziXKUFM*cT`J-E{O{j9y0|Os{N=3aiQGPDE%OiHU#cT zZ@6sMoRtsQi=Hob%MY_dM??)nz4Qf|0$Nn87YPw%ua)Euwf`2wHYkQ$52m5bB zk8@-Ful=T-v1+1T8CXya)xmoaI0q(<;povO`sA^V;-V8FT29_pCuG^&ckA#eX`|kV zg9}XwgEG6s3Jd}<&*sc}s@d=YQu9{CdJ3-%z*D>R;%7{uA8jR1fG5{513WdoPwxiQ zM!Od_DPA@>=dlhaMi-PM2iA$z?rzbBvK5YoYj7Fff^x3~mQ%co0Kh4PAX1Z+b?vt+ zk9fpV0B!QIRX}QL<|EyS3FWzNS3@@2wMPhi`2e)Z9=z~~zKcs~9Dj9$z0;=SdDcp} zCE=9;YeJ(sHE~o>otoh?lvDdT2UF zfYhwDqVS9=5Y3nvE-ja&mJw z2-nI`PN82yIeA&2oLpY_q_?~YbN=CCP3Z)9+7#k9`TP1$=o_zr_)XSLDgJAKFsZNK zaIetpNZGSVn2F*qo~ooVJMiXC>#;Mmx%D^udF%(=VZ%no8tq0ph8i zQJvjp4-I-2X{eyeq89{s3U@;MCht7)o7`wq@SEDmGk%l1wBMsdhfMP`>z5I!sa2+C zNUc8z5vj>L4^mT@5lBtZFi6eKGgnsW63OsP?K#qVr&UY4ulMe?D%3}$CbzzRBZcS; z+GIan=LBuy-9B6z7ZP};m~UV*#qA7CrdSYlnjKuJ(C1E!IdvX_kX=H7kn~}P(U?)(@Hd*8OYJ?G3eHmK$tmg9(gJu;47Y#{yib)nJ~v<5W30L-O~? z71erR6`d<8C2#TXq~$&7j{oF5--`~|bUvy2s&d|^Prbjezl z9+CBZgLM~|Psm`%u_r_4^c^E|SZ}vzjzSNm*B`PyY}94q&Qm@C7oNTtoNHQJ%L=J{ z;c`zyEtf{empF|=kEA_krGphm`xMCDuaxh;p$~8GO*d|aeaMJ=buZ>G`$esK9fclC zU#I#64I8!Y${$M>wtOC5d1|@4);WXo9Xm9|ztyLy=`R&Qp(Ls1sq&L*);WBlYj~{v z{Z3nv4;9nxZB;Wqc>k3?J;RDVvi_X3lvhre5@a5CC%$`uuh^D4*eI=N(Id2T)at@n zvg}@GO?R+Fxh~%0bCj%CuoTYp3W{!+s{P?|lgyLHhFh}W7~yM9-V)!(yfpD@VL8!n zf9NLjT3fL1i*&8CU%_a-%8|ZjP&6EfWJ82Gp9L8!4_;w?8ri|eu*r9XF?)$eXs@MP z#(t&mVHege@9X9A5z*+QbRId0A&tne9PB#V8wK1#Hp)JROMT}UODtGyv?i>t?@g5z ze7@5=`&xvb%=J76t8~(*0Y37HK3+sSX2cn{p2hve0@B}iTi2(NLu?NfI@4tVH#^UzSVXBG{+=;B!Ww<@KE zfrbNW7k17(dEA!ASs?u{{^k1oxBx$`DIuprp0Cv}4wIxjeP^47zW&yBb%P$=nm&#H z*ia5m?0^x?)33EWXF=|I!PQ2;AHH=tbh;{k`VNZ=k3woAnFRjdbHoeAoC>)p{MYnR@ICjqNonY;}PnBk;#Tpjw4?1(^@jURX3^Dx?p1 z*Jv{bC$L!v$CJLM>m8o7K>jgo^dX|qaIA9c0*$9fXS+6d*RTpX-%q>Dh{7K{$?*E; zhSqA>B0D1ruRWeuA*EgzQJ8IQdZ^$dBMOb{D{Q(y)vec~3e4CTTA1E_#Iv(2t01#d zP{_6Y@}9Fk3^OD%U(vdjKF#n)ZJGo4zmDZ^X+@`@jpdSGuB};B)iif^clb+sPO_ic2l$-es*_ev$& zAFZDFb$a1O(-ZI$a=-rfUwq<%f*+&Y>n6@ySo?aL#G&8v#WxAg(r@n56{Q9oRq|{M zI2UsI{k(gH5}!XgwejJ)aIYNWEG>-+iz|C0vBu5o_S(lmu9&(7CSq(aMhb2 z`Q|PDRQyrYI-i19XHQ?y`uv9iFZZ@zc4T3L-0f?RIbOcWq63S{AH6*5Sh>o7&(1ez z*glsbLuTc!vwGc>TXV~gyk#BQX2-)L!}6|OU#D@mEWzg{2V6Y<%D3$cY#5X~y_*9K z7}8)%pmVfI9#xW~evYV;a;J|UsH7vdq;phB4e}H5B!ThK=IcjPo+3$_pB}}gR|fV` z{$olryid8+2kDfNBykooBxUJFo!@Z$Mm8Sc^-L^us@6`+Z3#HV$!mhlZ5?Y^D?4MViYLaq`>}IoJlRh zBtd?zl7ELzkc7vXj7v>YG2l-vEt4ko$dl+bus_rW?9cpb><8E4Yv1x49$M4QxNOTtROV-EH{)sU`Y5hj%)w0T*Ti(~#W?9WvCQA^e? zysn5mIi>*V-wFHsN6bkVId>#Rzss0|xK=4>q}CC6()CB+@B3TSsl}gkjy%cXKNf$I z#hKKAKL+{167)Ee@Pzj|Q2mH4i8zyS(2@rB$q|PDc{t&HvefJ`AQdgnqyYQA05Nfj zDrv%xuw&Cb;Da@E>@)>O9$}JVOa6fOMQo$_!jL}eyo57IQ*22@gkne^#g>H40OK?M z5LHsS1CM7@e~IxaUkEI4!uaHfW8EMp#g;Vu8=`0K--_-9PKe%*kt7{(o=ustj7l;% z?}ylui6v=FonuP^)I%Q;TT%n{kOrs+{ii=@F*xITPH`q-lK}NllHyF_McZ2V&qV`^ zQjQMo8F=JLm7-0e{7gDhgh^|M+nR;frSf0cJ&15UYn$$^5tEFfO=`Fv`MUP-=NhP| z1~!2YkJxk@XbEHSo?{q2x3=d*Lq{HAI`FH1Ip=qErgj!edSn zgy$4(5+woQp(J8X;y52QI7gl|ACtG>T88ME0{~K^3?ffrW>Mrx5p%K=YgdarNiioa zITPoiB#S(WBfOCHekksna>NVVO;SXgg!glfHVJ#9dyq1N#o%SpCRG+^l0}%5T&g|C zylY(#0E8A*5^>#fp$09IBq4eP*;GIc6g?70zmJR_8|0 z=g6$gW7G{t?^wi0x%G)S(zZ)ax>m>d(09W4pw*j{uVH+8^ho4r(Id5Jv<|Qj+W3g_ zo^e>En3SkhNUKhfBs;zCf;t`XBvB8GCmH)(B5&jR0BgJgi_+3n6iHGVJPTlo2J=A@ z=mhZ7Tk5GJwj_%xX%LVfVoRcoGwS!HAmup4nREjCoB_UnqIp^~iz&-;< z`c&f{%5i{wu$BW<1CTy=R|nE3M>H)!;OWC;Do>Fo4Hp*7qBOuhR{{3@BjzMUn=~4H zU)`gU4J$#n3#XKU%q;SxhWfFXlScW;^bb@9F(-i%CXdCqSfc^`U^FpD{uy&pk{pm9 zY-1qmPdM;cloWGP1O6}v=`O=#OdhLD zlFYq`+4r&32r5I^AM_8aHjFvB8foQ*FQt#Rz>&q3)ZjnZKg~D?@Mka|FGRiLQ6yiJ zjZG?JfGZN7vVfoKpsZIv`dB)LPb6|h6-0N8ka#F^OL>WD1q=o<*oDe{?NHHTB z>L;7RPrHymC*aT6d_3kL+JbdSInH1f{3B*0EC(?o=?G~x1kfpFq-@$`#js zNx&bT6ueQ}KO#q378&5*81`q>B1bx7f56B#;3|gwiO7)@Gt#YR!YOKn#f&5b(BK6B zVYTCc|9H$u9xW2^pGofYC=hi9WDKSn2>xSHBncqSKoU|ENj%-0r~JU&0tD0|N*b2T z#2o>n$$XI>Q4-~~h>}_?Nk`;J%kH6zQI7)$M4gT(l1}j>U7RCFnhRWgOji}&W-(dD z^m;>*q=*e^kTP_^ebzjFUNn^?1Q0V6v-scQK}y-zhfo>BgM_q%12Pa0mS4z3*c{+L z%xgy^NONJseUx#4fJi$nzqIp?*BdqiH4m$b?NT8ANmNA=W~W`3Z6e_(NYb;Ey0bE&3xgzn#t^?j!1?xR2%~ zW1mtD1#eJ)5d)I+FcoQz{prA;Q93gJ9bHRtALUMec=Ii(hJ@!s#Mdtt4U&H})1yH; zM|gB50)L3`80=S}afN~-1Hm_j^V;N2+opyE9 zDVC%5nJ;B5N0w_oI_t3B8h}4hRuAil-va=$rF{=oZsZpQSS&~FYnXIsC$IvDb{jgX}V=V2}6}4{Sx$wzD~qLL0?*QQVA0X(Q(=1dGav4M`5j&i5h*gegdBx|O!dxMN8 zu1}Tk)->$^8OcA-QmDtt*S0^KkB`$jGUy!NQO&(c|H0cI<2#Ce-m0O8!HamPe};BT z5KXY~j@k$)hxLu43?AgM@BSSg)TR!D2YC&M|ESrK6GsS`@taV)N*p08t8s+vE40QD z@_C4;klIIPr>KzFxIA3{x(#?xY>ogt$ah4rP>|YmaSjCuj!=~Cuts#AMTI;&Uz#U5 zBZVv!B<}@~LhUlk)1EMM-=r@>rwRwrS*t1013Bb`M3x$ zfdeG894I&;2tReb_syyKHb+=U*+CClUOvbboI&nB{k3tDi#^$2-&N&^7ODIQ74l*B z2y+!PT+w4lI`M_hB%xYlxjs#ZB;<;xz>!*fT1VhWEkY#M3NeGc`(Oq&BX?p3e*gtR z2lC1sVIj3xklOqtI#7$f<`e+ZrbJluL2?9uq^OU4m{ml5bfN?K7LJsm9=W`Z_>L;z zzYp{Kbh1WD{+0FZ)lQ)vJ;L0?cq$GgP%SY|Y`$T1Alp{TVnK2}5FIFV20D=M5p)EA z#Mu8p2MT_Z=s*V30YU>E$aX|XpW3w)+l?HcAfp4daHmeOAo0)&B#?)I(2+oCOvV}d z!1`m;1s_-f1vQnZC6etk`-h#~Bgqa>&?x|i-o~*!|VKG}G|r578jI znNC*2eh?N7lGlJ}keXV791Ro$s@%OICa_0XDqYz+`X`4!86yTp`H=$DDwCv3R|LxB zhz6;Jfb^f-DP1HUim<6sv!oQk9<|5>_>l^o4OiZL*T+6w|AheO5RiJ@M~wm$^GL*f zlsG5-kbk#jF5x(qp*+K?mI^I%|5pDkAP1YPLO@tY}aBv zYB(Q@^~eqTrr1-9*EVQ_>00`qk$rqz?g!{rM_`{;g9Gf7!@)IbLANzRPuonNGVCL- zjIcb-OcWieD+_L1+mW?>(>enIH*O{{9V!hDl-l0K?MR~@ zupNp3$-8+An&cfKSdVEPZ-ToZ;!cHY!4wSqM_P~bXzoc#ys$H(r}~HT-IQ{~U~g_4 z21lqzdFW_tBGd-XDe5D!c0!&M;T^S^q3Gcqu@YV(R>BnNQCI;0JCUUDj(l~Zhj%pj zcA2QVV*>0r{{*T|>pkMcl-G4lfgCG0ZC5qYWPSCT=l%nl%f9JVLZtj|MrDp=|66#+ zZ;QqJW|1CwhjhFSt99H-m`Y4V7wpqz(1Sj@1h&4UFYYM*qh?Y+jsJ*8e1LQsbUi#q zr_~r!#~PLnZyv~p#q1N(l&eGo=~w}C_Gm2mv>B_-MKD*9v0H3go|zWHf0KaoP=AYBP^uqhz02g04cdtF2WUI4h@;;yWb@$ zw!TY~94>q;td5wE(x=Lxir9u?nMHTxQyqMs2#Nk<97l000+c6y5yl_CZNPWpB95b& zrxeGL_3|Ha9QoYSgEz{vA~pz*)_+xNF1by~k&VkMIU+X7HNzV52P5h5QXgHtMEEh} zRdU2}#FBZg=vz!32Ycv5_)#239>N`Q97QA~r#Oy40C|sDNJlo@P6!|#~inXARY#th( z`1BJ~l0|x4ceOyMBn_U1<(lRT9MInn@gGl%81A-|p&*&Nu~Pg;zVvkl1Q7~SxYD1q zf-iIgOrZdg+ITw!h-9H4wVC5U0rAxoQ6ZTfIpKZ3V)~eM;fI$T#DOC!q>|@P-4gYv zkY>N>Erp_3C`fMXK>oDbtWzjRynIX)>SOeu22?wuLK?jPbO|;!yJT^z)J2OB$(=$MkGZ%jaUK99pnztG1mx-gl%!dK9N|63ep!t(+3-7Ba|L&3Q}kGphW+tJ zbRoe6Cf~gsh22*FJMkk`fPk8VHJEob50-7W9-lgayF-1v3uCfN3+wIYlNL{@1BC2~ zP7N|9u=lQ;`>IKD(%Lv(9t4oQbS&BR6)GDMERA|y_vstNTzr7*xS6u4 zMr_B>=ih{FXYfl6OKcdiafV!bT-rz-KZqa1arq}oK`KagXhsV1F>YE@bEce9%nB*e zx}=X&Qx+6Ua}wzzOpiQY(9oS_4mJV>dAobp%cdQem_ZJUxTKG2uNmV+aqxT#UoEXh z5?b~6U{dEJKbJF-kaw3mNhoSebRbWw6FnMJZozM#Y|!aILBrCZuFj&>{XD|IV5+)I zkE!7?+l)r1dO8gCbuyQjqVp z|4;r${>Z!eo8-f~7&Iv6j?4dOWze9wBGI5hh5V0vHPu0b2pfyjN(N|Al#rjyx%5#~D6X7S&2YPV+G}WFH-J#2fb~)I98*1tp$LGGLs@F)HBQk|N?vBMoNV}+>f)c>Rwr`8Fvr0T zw_l-AH36h?<%lm%?*=U<)IX6Qr{xi%{meth7IJjiACqBmI9EBcYH*dKs1a0-RoPPq z6{UrIB$jDth@YI`E`6ib)?QwCx+@#=r18zR4do$aBd7FS%0?ygZ-&q;;e$pD*^U&U z8^u6@&K;MVys32hMthz}r{TW;?z(YE41$pV9`cQ-kqsYH@PW*7A!l46+spKg;;RVL zH}avz^o?SC5mzYA736Fbg&@R`&Dx)Eh5VN;9apG!SW-kxh@ZfOVumv+ke%j|P=Pwl zBB($)zK9BBCHuy>$cxT2i|j5V*DSJY8RL?+@(UI2D>ral+*^AmEpPwISz@(acHK+? zDkx8WE>w^~f$YFKB_j+Eb7e!3hf#e0aCsO_QNh`?A(PWb0MXT9!9>H;@iP^ciQ2{bl_tpd-L)6V@d>s_%ChWrfaimRn~s-^10b7 zbC3K*Z#SGX>rP8Rc2AVxr8gSfDe$~R@ARMVui2e8W5#K1wq4lQXK;PL-fa#>STZBg zQzicC>RA~NcK&^8xi<@Ed!KpHa?+^#8wT%7NSLy4!u@n#njq0L<@N2wo72>N9NYHI zq?Om=mgYQQi##8(y4&o2*>AQ=eSX?MNR*&NK3}muZfD!@VN;sDcV?`&r0I%^tEU+) zbss0bj7fPezc&&+S4Iy1ux;%_4BMHjm`~41@k<(CMWPpq zVRDtGX=eX1PszBp!iHQSGZv?6R()8eRDs=l4!QE|(brnvrvvfRZHM4z1utGwu+=dn zxkslGyNrr5Z9+h{*a!GGI@II-zL6c@Kgae&Ga#$!bArci@LN(6TXQQLH}&m1C}dsx zzYM3|SSGJL|8GI6~6@AmB^mzqem{gi>Dv?KN!; z9p2cnLaaajB@?0<+*j$9K?cj!+zcD^e6!J`)zdM7w8PSM>@WUJt9CL3zkKfR`DQ7` z#j^nX?e+8(A}R1IJfco;fFehQ`mN4`lAkN2VrX09i*kdCd)>#o#di1^Kef-;Y`R3= zYX9@JS^Tup3b_wAtBo@G&g&)L`ue)Y!Fn}jMU(q*hvDYm2`!5^mG9kqd+q4Va&0Cd z%Mv?me`ToiNAqp+y@*AQ1|RzEY)kIL%{4X}pXKzHZ+)pdIr8su6Gjos7^rRzeP@0@ zx|e^PS2FW~I#oPORkuEUm|3Q!eK6_VyZDCZ|yxJ^UqpxAXj+rcK9>j z64SN@ZFjd!V=Z6wT9)NKd$d{fd@>%@7T7@Ed_fWGIC-sSpv|wjx_aTHVE=g)gE!;2 zA40_ulxiXE&@})g&A%R8v|z*?^!c6O_7Uk<9wQ`DUa?~PYiN-V850UWS70?fWOriF zgek1&@b_W z7~UXz^PPXLF8r^vS$YBkSC$=@YRNY=G?AHf52Sv{w_IL1$!`IuV3STs=y9Z38KmYH) z_`aeeI>IBkX=v$g-fLf^oRDYFUn47z%RZ@o&BZ(BWG;DO_^^}fR&`3garWl?mFtJE zc-?TxhTcaSo=bjfcHSF3Mm72e58C+Ie`@KUy{jl5H-G-Vq2u@8kNPi9pz*)C$p0&R zUy}Yqf0q9Nf98J1bFbF_ie?IUa|d! z&*}QJe$c-E$@jgz*78c$z9o-eJoTZ(*qKGH^bJU?T&7F>owut#rQh7F>}B^3`QQF_ z*!AgD#i1A#+vW4k$5lL7(GqrgSHOa&3l}WU()8B)*vb3Xw=34WLyvqP#6!Q8Wg)qJ zD}GH;_4l|-FPcn@&oC+fnZZTkIxXmb>*c0g(XUs9zN+FF2!QnJf-2QJR9@F7JgQE(8p3e!ZIaGZR`J} zw6geoR_DXS={^JP`4i0fpJlv!`Qp@rbsGlUzA(eEeTBM7`LF3%t@4H?18coHzJK1A z+czuM+1g@7@d4Re47l2=)txFnUk85;H?+#wD%+8#B{n(A7x2u{t#$o`Omm|mF1_Eh zy+Hal8MA)~9oea5u>~I0tC%PGKi<&TS=_jxU53e{{8~=QQvB4T?)#28_AKhRXwM(7 zie%`rJuj{;f}W?!+2v>(-__Ob4o^F7@}CJFFDjM|TKgbNy9I%7R`f<>r}0$N1=rN# z&8ttWQqQYMrxRULH3@6e@YSeP`JdgG-D`o_q8|Aa5Z$d-snLU0)G|Z_1tw;H)o1$U zjoJ4{L{~h%xl8N47B%0n?5$Jm9&kC$>r01o?ya1+(&c%L7x(yc_1i^t-WB=MxL=5| zHb}{3mYy~94*pW|T(y@auWo-ivqe~?btlhyPVBSxX1W3?bMB2@Qmo9%Nnff}U$Agl zhFMp;uWQ=Z8J2l;qxI=MTfT_CwD;tW*u>)R!z)aF++fJ@4(9qPDxbe|r3*z)UZnVQ z-&k-HYDCf7U^l{Mv8YZ`AbRA9Jwa(md#EEK5zpzaJ;eQXL?fFT7e7qtiHl@JBvOKc z#AlZyBGgjgAdbkHN6%)j5)s4iA{-ItIi0u(VjC$^9Z}6-tiX40!jZF$R$KK!h`UaR zOHS{)6OM?lMog)Wa73)BuZ{M&2uDa!JW1MdTxrPIM$_3mD7k0hhKNp2A~tkUjVOaoZ^*=koM-ef4?=vqI%?SwA~+&m z(y?4rBW?;#BZ9-FH#BsZFqPtIxzZ5aqWX=@5M|rcmugJ{LseM?$^bFasVHaAi});; z85USe`@t|Gw!@_{lRIDfKtTBsfUZj7g=zbh%!ghE7KC%x}~RDv`U9pcSRxSk~D}?7Q`ag6>?D_ zrxcl@kW-3D_(T_tNJn8c(wIc-@!E~p_1fn!&h;unj5_dxTtbMb>D=f?CLv_XK?5ff zA<#pM2_})zi0DU2AfgsbKWGDoh%`(*XnO{3f-#1kNi>7Lu{1WAY*1 zVRcF>1_XPUOu3uxckK6#rCDQRsz7@dFSI;K0NRp6 zxNZbeGU|{shFl3~xsfFwR{L>fs-ZkOJ)n|pDRgHjRUPCe^nfnvP{$jhRHO$4U&i!J zgC`+9QvyQAw~)$40zgaCTurc^e@h_R`kRr4iviu0fOPcID0xIk0F4>WAwFkByhbCU z%_FX(oJ1qySdanYl{A2Q0U+8fJ@O$A(S~qwnmaZQJf5a-P) zlmjS2>}QHU#u6eD2F=apkZM**8TeDs|L6iY9 zK)mWSfJ^{rd0A*Jr6KhnX9lI>kFkMvQ)hY?ia*K?6tJ>q?je+i6n|(>jS9qP^Pw-4 zh?gYMV!A&(LK^3EoR_pAQ{BnM*OHAm8cX4)L zBp~WY3?Q%;S`BP2p`S9h#tYg*LO-?j0d_tG_lJCbru8!wJh75K1^N%;hv+}W)Vhe- z$Q6GuVWh<#kM*q95d9gr+z;nCj6*K>Q#;(8WlkX(KzF&HWc^Ae&{3w~83LZmBTixt zRx5di_FOK2SCanWC?x&!h%~Cy4ly*gT|6Ml1|HC$SO1|OA7Ol+-ZYbPa7?4zhzi8m z$c*zEqXJDOCWEn(NRB(#r{U zl@o%34SfHsD{^xw0dYm4Qv$kVfPx47Mh1w|ayIb0R3I_t=u^E6P{#)1xP{rZ9K?Xg zhs1zZ8&QFr2NW{Em}j)|+ts>ssD%pqHD&DdR*eS1f9#wU~)m zOYtkVamvm&cnvF)nbQ2W1UUqq_ye!wv#+5&kOUgkEE#W8`oeTX9TjL=JQ!y*E(!E1 zzK@kW4bu^GRiMTR8q~%auqInVmr?#1zOpmO{EU<*69#^x3S^Q%4=x5&eBy}9BRwGM zco%10<@A$y+Wv1DU=l)*D*^4KI=d^WC62mwQ0;Cwuu(B+5iKJn{eLsO!6jN>#hLFm_0@>4P<&i zc7Mz8WHZ187G%JjfPPOs-i0@DfFzqF?}-|hBA`P={*VYz+=wR;AiqV%M1bserA`Eh zCyn5twCtXrsc0S<$Y2hC^-g}w1_DstH3aL&@0Q^XBO*`P(JTeFPIaa#)_KajNt<;n zAJC8H;?B1_s>h>waK>9DLjZ`+yEx!nBP*;6Zbm;=G##pkE_fn8W~+8y6wNbXZfSY5 zmZ5qg_KNI({8ltW^;jSSn6Xb<4oOixQBS)ko((PTU=yk*VoeCuV@;_>U!5q9rq(Wd zRmLACrjRl3?)E>f8bi?~w_wWGyKty_cyFCHxFuvGr5_96#c`&J*r219;cLao|<5SCJ?!!a1T+y!@IA>)xJ~6m{XyfA^9%Xd$ zty)(k#py&}*#aSl5yNu;?xFL(Vvau6{*ajbsnrr>9p^$Uzj>z!DY1CS!Uu{Dvg2Yt z_A#rQjZdr~_AIETSWIB!;unWzUbSxD(*`Z|7#=zJWE@1%YmN%zQ-yeg;vwLU zn*kVbC&KcER1LhIy^ zBeZVuks=YG%Y5;uWi}biFVa7D+H37g`|((1Jd;H-%N(c^%W7A|7xNH3Teajn*W5O0g0b!_?}pqoVF8lQ~_-8|eE;9V6y= zk21?1TTF4&wt8r#_)#~C-Wm3dWsl{iv+O|zfybV+y~%2i+3Co&!5K3xieqxSrW|V# zr8Dc{Y6EB-YarF7)3e$pAV!Miz;t?fhEK>6uc&0p6Ok7#j82Vymq8AAUt;HY!FiGW~9jD zji&eF;lcDg+ELMQZ+gOwym+D)U9g)i5D}@OLaiWZ5$EdMB^=$NdN=$dN+-@!zBI#$ ziyxS1IN# z7%s>A!llQv+Cal`B#O~+BCTB$;VJ$#+m*?AdH05DH=ZK4xm!oAmx0EKdbs)TR>pRF z(5^hrL7?OWIFS6+QAahM^iB*3H%d-)sGI3dq$M6s)Kuf)tTvxk@(_f^RfznUfX5e; z^G=?4$a%-cBCed|N!7~icH(F{c^%L=PSoMa7YzmXtRxx&VjbKK=5AR(Qo5Fofuh31DJQk1)v6OMvy+Lv%dL8XyO z>wK57&YnqfOiWWJ+Q9}0PSR|E;P{yoryIcZtZs^nd^1cs|5E15jCHIGYG)hD%iF<4 zzUja=orTUac+o7U>inb4yS+xG%?QL7vXjM+MG2~1KwIB6_AQC1 zjxadhwKffdb4TBD^rbvMMP|NhjJ}^Uz`4#3jzSQdu2} zrAfPEPM?RFo?IVS)$kKIPIOYALUOz_J)IacKXlT`^Oy^nYuyo)hFB$ZoZM4acxqB) zPOf!npfD559*{XXW(b)REv4D#zERHc>j3Y1QYA&4oYPQr(gnXqI5ivO@UYs5juZ79 z&{^BmkT`f!AW9TxW{OReAp2Kf>7E&zE>Q|ipcQ&N=if`r)y2*9} z(#BT-9nyyHX!Ar9GGa~49~;Tzc!6B-8ZJ^~$oJ3WaXjSl&QTYsMMxXV%=2bUK4-fT zc{@GHiFLwF9>=O=-gGq5nKL5YAN{5_Pr|8zNRv-c&=F|`l8EGFf?t+tOO4Jcx3GlP zH_*#Pq?{gC0)_bdgsWjmHOG37L2JCoYU{3b9T8uC*?k}ioAo3yZp$!84h3eN6W5ww zQE4$s$vP+YYE+xzvd$S6WF4%PCyzFXZK-4|FMc_bE$B|Q$+f)DM(tq~S2Zz+n~RQ* zFh;8Ja}N{H$x#dttqLQ!tlve>%)PH)*9v*}jvNv;Hvt{%TXo1pgt?0^A7q+iElKoQ z8TDH0zB~K2#^W4GDJRA`DdqSF-AFWEHsi@jKmyqHhI;t#t0Fr?aB1mCLT@zGv@vkb zU8wfwsO?dlL=&?E%o#tU;8}0=w6e`QBBl>#&O~<+XeJ5|&`fqBN$JFV23;mL8C}K? z!Sr+dV>NXT# z2y`MjJQ`M>8<pd6iLZc!hq21xDy_Q6SM6}FkJB*Ze%O8QC{B$Dnv_s6 zQ6AFAiBYUkV>%K{9rD*3S!LS<_~zNSWzWSWEsosNM?EMNmU+(M{vqBmc?(ng8UV7< zZ%;oh(cLSccC0cctWj|fFxEvmCu$ee9 z>a=v^qLa&{i!jsq=Lj|voz!U0K<%1%PD?os^nyAdXfiXn>g0nK0Go+cO-$)0+uFsQ znadtH>TuYX@4JNd%h^EdIn7nau`;%T>9I0fM|{&|*L>VR$EOJ+%-AOP&oPKd*>9*I z^R*YbBYsLzdimn#T7E(664dEgv-eNOi9=X6|IsgWQo5tK495K^5N2{L5@9C3uw%B1 z@4_}B%vgcBp^h&_WT@jG(-CHfcgFA1;?gy#p-wiXjyvm7ADyk3dbn{j30)RO#9n>er<{phR6;Tx`u4=S4`u~S}$op!3- z`%d0T=Q^_3v8BV~Y?WD#&SRd}A{U>Ao2k(?B<-W@CU#By(ngFKZwa#4$yAFYG3OEqBB$ zr@j8wUJ9Owu*<2oTiz*W#i(STtXCaVPQ45YSCo{y)eEO;=n%^ z)`^p%Can7bk7g%#9d8qR76-a{)$n`d)`iE84;qfCv1?IZQ5`Kye0>^3tnTMJ3jQk%dsgsqn#a?n#|$*Y}%7kGJ!2-Hx zD>i&^zh5#-vyRHh0m{XZa|2HnvUYk|e%XgVZ~qZEanj)2_Oj4tO&^vVrn_dHl=U?> z7%NmLoBq+ZW?kmCvs-UApOJY)E_0yIyx_t$>e-GWQD>!8?{&>QI`8gPPN4Sy7fWdyS4th_ptnp3BF6SZXdWMF)H-8 z=EEmXLZYt9+efFGPWOl}UZGdHiH8arEl<~O?{=b|(!S-vk)4M*_NGRn2&MhuA}`LE zsuT-Pe)DL0Q{^Vd)OVX2Y`Nb$Ps{3M>Q;<+^kYmdM7`Ew1MnH*)VxP{jZ2Fw9dK$h z$AZuSr{+65V|@1~7qLCjjCa7PS)-L9m55UdYu{q)rzTy0DZntb&?WcsI_7Q${gftP z+^tB&X+893nLu9mZpdh4u!b_V+Dkn0Kq%UDeY(1p60EYd;-Vhc4RM=lA;%Kilbeg?{HN7y`QGgCxV-iUk$s6g+f=M#$qNfx%C{0z zug|d2^YLLzfkV0c4~C5=0xf|4J$cO%uz1H!^NXy>{ORBEh4PubEMMNKcD0Drbm=N? zmYXhJ#~;L$$=ee+c%4vQo#WWQEY$z_`>qkGY-rVMt}e>*E+k-LuV0T%QAT`AOd{_nj5{ep=y3sL9X&9MN8A}}*~T}zmmwhg|FItP@P zDmP4W!&4?huKlS(_%~AKj_m>Fa$YB|*9kEOFWM1Q2%B{)cb}O2(1y*&W*$VD)`dp< zOqe>u)Yp`+PdnHerX2r(D4aP*S|YA4u05jbuzxRN9Ta%(*yst|;;J(|cc?i$ccpL4 z>e{ImHm}z9Jm8zXWBllOi)mr-3RV646)yX6$Vg-=kfBw(ZB>m6Qf+wo* zv-c5Bzg}%HwS3~ovD+R#{B_uR+h5)DuSlrbdUMU3la^U7Pgch@{MqN}z5n9FymnT} z7QEB*NOt-o1+*Zg}qrnH;BePymw)hqt;HEo-?YF?k(R!UiR z>(_NvXYSuWLv>X1ZjkBc$N&G25BqPf?*9rOmZbmCpVfZAFS(y;+?(zH@nO70)CsGn zP5HC(ps@FArygoMb8fHWV=Ma2ZT$G=f`5X)`lL-f|AY3W@7F*3&ycG^P|?UOOLs;3 z&)Zme`HF0@i@Rhj_v@*tqx|Y$e*1V(%+Egjzkl z!PBcmHrO!Br`_wm)(_Q^m4EK|<5A-x4zqf@-JD-9_Ka_ycy>^Qu*;8HX3ly)cfs~+ zI@I;9-7C?rd6CdfOLyhN&7tM5>pi>wCF8R;C+3pU6%?EQUY}YC9X_}6KKAHyXi5KfE7wIS4<^=n z5%cBO@iU`tZ@PN4vNEN|VEgb<+U=Uj2} z-ODTcGZ^<~$p33X#=wHl?N#-cj{rt5wGA9%Y7jy!_+StDyG- zKeVsxpR&y2dkcg1`ewdgt6uHc+~ebo>4x;YRk-=-jD1J{`}?yw4W^&>pOM=#x6LxA z^=;?;w{qv{*UkQR*p98gh25}6v|qO=v3b>0T_;)8>(}39n-c2fdt}DOk((Up@`ff( zD|a%^cJJ`n_q#5e;vQPm-IZe+oBsIw<>F*@8(kk&_TH+1y?rB|^{L`zJ%4ZTHA5$} znxk9Yy$1)!3>r6UK!4v|o%U?mc0T>u&rfQ_y-gQ3qjT@$7L_aMfMtB8V6=t@5Vn?c zKf(hUbYmd$rST)2f&o|v6Mld_ zzTax-OcbnY51xTnTZoau-$+A;KvA}*AKqo6Ii!JN4o6|=Rht3`s%6j>R9#PoQ4U_w zW)IVw@_A^#OC|R$XeCr~cKb5q<`T+Dm7MKXGRoo8Gt`c$Tw?r3MrY!)GU)?MClH5QpI0 zEA=S{f~y!%h5Z`=OfXFxh5R?6?aPC4`hN>}h02crbNrA?A(p}ePr!-a1{LurrN z45MZ5kV=&1<~4c+R;4nm%0nvPfK{PSfmPwPa}1?lD_*$>?pp;MVO2OHuqtCx`P-Jh^jTn4IHHDC zc`^kYa)$g3ua1e7<_#{kFcKJ=M%p*EdJ2q2Rhbcm*Koh%sQX{I;6nPW#VMs5M1`Y& zBX(n06-E?mRhQDuyrgPl%0UNI#XR=KQ9k`eBZsNp(At=BTRO!TSs41{2yb?+!#BO%qao8rH3eXQp-b|&3B>-{o^1MMge`38K*_u98 zZ6s29`@f_8E}#l3t(o*$HifC~ZTCVM(9ROQQl<7SFa^qKdSXIf z?EMR)T^YUX6JASYpmD+gES>O86~Z^~COav&)>BZUvzd{3ET7!XETnER% zEKlY4kYgzA(cdWlV3p>z?OsZJYU>j8HQJ4pjRRXhP{Y|?pQV%}h{LFuL^%+!Wm@$Z zEpKeGi~$Ot6S1Gk;uKTdNNSU~bChx;)@rP2cj?uG#uZ1kx{x>y_zQAl`ZxoFw=l-s z;4P%NigTxgx6EExZ?EQKWhRN79@0{3B&48}Z_zS1N?qF4q;Rxe)uTOwCu8O3V3ute z#(>1LmcKSzxD-xVI@Oa>0@6ZxIHW}iIE+$&6b#S8yIcY1E`P%l#5N$!;%8_Xu7JZF zr}A(G9D}zwu!R+P2JMFe4qze!wsi701QAo%zHe+MO0#AZZBJmcQd7Md)m~E`V!vI( zw$M9nz!vrZ*g`EdU<>UGU<-SIY9Z(S7wGj$IOUTvmfBOpwhR*0Vvv?S4bF)e16Qfv zpV0O+l-cB3OI&v;a$A~PA7lk!Z#hr<>`2vk&f=d zxd4BEp!^xswf8V8w{OGQ$8WUlu8CvF7sfLQ;=ZGXbHQF{)q*IeX>~zwj4|($_?ZB`GnHO5VZX7_lWMp! z$(DR6?Fqy+gXiT`mr=sJzL*8dhx-`k-zwuQHK#=2UC)+h*=WbOVpte*&@^#e5QqH$ zzc9`<@C)V9iQv9bz%l#_pW+HQ0>5;~m%-BTSIiEseFKD-n%*Vk3(o?~M%$h0(P}`# zH?De9uK4{<@A_hG0had)*NWxJ#=z{~ip{LQmjnr)PGhDhXj8vTOpETgfFN1oHRhJQh` z3H-}oIWV#t_S-tmrB{N$%y)VfLwY8Nv)Ty%vf4-xC&4c!h$A%|_7M1mx&MW>0e+$6 z1b#6o9NH~@HuAZ%p#@rLUNHJahkxPS&uC8ufvF4vLwz%^!yY?OFkE~1q`{c%zvC2o z>X+MiO(KXRHJpc~V6yQ@<5EM_G4)2lzPH9JiS%!akcsxiMh)?-`_F>1WOF0ni1uRU(%pKP}8QaD; zFT)2P^epN{$Fc`p!yMFU-_(v5+5QlA#@8S)521h~6wLHLKwv0sCYsm2&$CjC=>&0h zBW9)Aa%Wb&Ya7!D-|1oocBhgM7RFI9W2R>LXiwT(^vdxssNt}kp0(;G*5g3Fkh2T< z0`9*VZNf1xu7k6y>+vby@!jxh(}Q1zm6ThF*Ys*RkL$6yH#%SywU)=cHl*y&j#Uib zDUA6ZVVle1zQMw9Bl}4&${BDNW)0vll&A9?%F}@vAtZ7!576egJ@+rY>Qr)EAIIS^ z!oujY7$G_=4Eeu1N&AIF4l6LO;WJY{#@h7CgmQM0$Qi9F_89M`h^90yp&V1mA!Wt} zlpcr}J@kUjAVS1E8@I)%#u>I0jR6f5!x7!Ia`@7WP%UwztYIZ_yGqF!f7h9E+qU!1 zDwaZ2MHa_Y?+)t&6dgYXKA?%b#pJThsnI2C$UFVaFvsJ!+zfN#*1MfZG9C)x?Zm@U z*`0pVnq}}?@Gx_`y^joNWh5ED-{;cG0Znq<86|5RzkQ~+#;HzOIeShI?Ot2~PwkGq zW{rdT0FlPxW}PO5WBRw?vx#Sp<1VTP!nb+UfbGjNk=Bs<#U)u@LdJLiZW6nj=5aH< zDW1_GpQcK1n7DzNRbv7e945*@aF{sScRg-)+dx?hH>Df@nBL=N`v;(jXq^uTtK+dD zE{_{y!-TAW0by)FxC`4pg@*N;k9(aro;z~-9vwU^%JloS%_2O&5)yM#)Xf|?-lWBi zldz5Vre4^lso8K9mP~8)@9|zb$m=KOH9mm3d5u?-p38 zz$9#93hXmTXyENl$imq0v@(O5NDw$rjwtVXo^q4OE_n!IxS0A5H%%&P_k0QBVkOgA z)9|Cp;?OR|vUU5My9Wol#4$9WOJ_n$!gkdIu_U@vhi)xufd|mms`S9ASZ%PP$yp7I zOD6v8vXN*+7ghj_KXK%*b;PE!XKWWODuWDaA^|uro*9F?IBl84pcc4LyAckVnrJ{< zhiI)O5*pwR`_gMi>m&61_7LZ|+x5F$sJxi#)I^;?cgd&cKFKb7$zJrD7Jx{kmz+IG z)oNTM!AP&Fb&kZP#Be3jOJwX)tJxmlIs81fNas^y5;alNdL2TmnCkSsTL3ekglhcm zJ`<{O>Wfe>SwEm&ocdCKU{ND=8lUc@PLr`HM0)WdMx@u@Gs<=Qq%fhHD4N~cSB_T( za)}&Xq?2W5LN$>Csngil)=AQoL|;V6vl<~^M0T($E9aME(dTWFl-+WAJ@G78#_P^E z>=sM>e!`xj35W&iq$pbdEh!eXXH1xC=CCklbr0 zqHF%lcjvw(x>#m9)fp$cg!>OfmmFvsyQTM=*)2syad5f^!Dy+GP#CL-W<4_%{5_9| zF)}3kFv_{OGF6XHryV%1y@bu&4QT8TKQN$0!xtuoRJ3nO*}#N<@lMv^UzQ2U z`pNQo*R#vPLnN3XK4a?~QxJ~EBxz!Flu0h)%9tc=dCfb6^E5^EfPV=|+V%0kXvBor zBr$;RtJ8iaNfQn4GNTdx#Y?Za%xH>rjX_SS=xlzn(&=yDd_Lgwfx zEbuF;#ZiI^lT3PRgq-qg2^*ncqV@`6tVDS=HJa7;W@|^bGH=qPib4!KLY_s(5FjQ1 zVE|&hINXvZGE0w;I9@UxE4Cs33UOd%^rEqHv-oI#Ss})Xd9BSyGRG=darn_ljm9(6 z5oA{5?7pGda>q0{Jn)0n)}o-8R64NCRP<&`+1B>VZABJ3t_;`kQmLjyUsxZSWz5U& z`*6dRxH29asSGKJ%PtmDX*s-Mw*U~7EEOWO2E2)S@C)u`hbBAZMX83P;dd6JmV7Q)XM{e!h^isxM<5x>W{)_N9LeoTRYdjjJ zvy727=GszIE9J+RvTt32HpR3x*l{s`z(OnfcI62zGh)z0H^qoztpgi%#A}q zj5`yB)VMRH#yte^i@ku&WWJhz4Zo*M6MGesuE|Dq!DcSKnjSE-P13#q=vlMbM`g65 z&bo$Qlrw0BAnwfSH$^>zp-tSGC>u%Dc%yjNOD_vl_5vp8=WM@tqt$d$HDG7_V>;|i zDg7C1J}L{O_Qz05H)r*E>+|93m2lMgVE5RFoKB2hE$EWkHTOy zc`ctH59~6j;TA-^{Frw=)L!)Gs{L{^`6GPk5qpz*(!_ONub8JYft{!{F%(<6Y}|yR zi5!xmXwu+BD4I3k*)duzlcH#RB;nLtS36gNKa-uY_oB9Waf94sR2qLKi49a*%~dOk zHbB3CO5<&kgg+zbj5m+nj&H%dU+mDX@%0wd6K4X^^+ctKD^529n@CU@(d?q00MxoA zyKmlx3Xy?LoIt>;P3_jN|3gLRS~I0tAQvZmZ;~oC#;VB!lS)k-f>dgvfxqidW2Q8b zmasK3e>H5)GtXkJLiUt$EOUa5HGG{# zxA48u15e3QZBP@}tlo*SAxfsH)^vU~^fBNezxTy+Mj1Dxfv!UQ6q0q8W9*aXzl~6o z-03mGc}p?@+i%rRz}C$KY^>E4mw*lTYwC#S5$LRII_ypPfohGgH_?T_-sJNv0_7za z#=|e77KXEfXE?;^LkGh#>l(|!<6|Fa#aJ!O>Dy{RD-*D>VdQzK(Jn{9lY2{VjMBN- zaD!o~n34afWwX>QddN|z?#-(IMkz}B6OIluvQN*0Z*4k;uE26_5D+)VO2%M0UNVKO zYjTvhoNDgOn|P#wjZA9nrBCaT z&@I%OE_J4xt?KIZXlnEcTqB7As*!H^riH+7F*Di#_Z1V>r9^Xi(3Ib6;$o2>`Tqca z!|&5$Whu#y#v6izZem;~1>IyBG&>qsqOp2%=8X>n7wCpte7!~A>+ER8fk$U)zlePg zc~kzl@A{@s^cy>bj()?9av*Q)q-?d-=`J!ejr;SuoKQ@Z#+pR&`jScJz=$_KWQcf^ z-&P<>69a-7(fH8-v^o8j3=J7rJ_Wvg&Xg@>FFwYR|535D2eAYLXyf1X5uSl{f%TTb zrA)&L&{7=ZKb2{ojbisZvuw?N)Es~&3C%`Ig>B{F#o3Cc7=J@M8j(Qo7G0%HI^8+ z*Gp>4p3A#tBmPYcnoAn7pf{`mdXtYY-l*S6F_KQ3u_qv(VK3yZJjRpKB2jw@&8(;oOr6y@ALf-hQ=7zlSAF8dk41ddd`X=DqCcddPT`Ag6 z&&1i)iL&g~4)hj}r?c8d_7R+=8q@E+BiNGpE7gdQIN66=#DwyAJJm;?x$LPmpDLr= z#5fAwC|imsH<^QmwlSWKKgA4@Dzii(b*42u4DY2VBnQ_xbk3Y={1A2p*J-?Ddc7Kv zZG5kt$TrcE3a*C4$DBp9P4I4_U%tz`8B+Ga^?=9>=o^2wl1HL_m0XVh3Cby&gadKx zd!fRBIQB6H#7VmRp`4Mp>)v11jo3KZSsh+!3s!-pTG5G(WBIDlF~4Z7?yId8b~l5Y z_>ub`oR|1e)v*Ff@=pcF^HIB7cKbJCF^*1jNymGT(fRUk%mh1V!1xP%^f*s z_MB>cHZLf>`F{0d6P6lJ6*`#M>E8+f&$=u2Rfbn7ZjGDtA=&&^^UOW-Jm1xRW88_< zFG^jBEW2>eV@kojw?@L^@Y;+3OT3x4b(11yyj|d!8uqHoi+6d_ zJr-R8ae{P~C4Q{7H++7kL0heA?!X;`Hq@Lsa$ClnALT|hJsMmtT&zgrUq2aAzUEa*thq#2Bfv2VwUaJJO@5wd{ zdFw9nA3J=I`QDek{=L6W1J7m&z`vGiH%$Swf1STynL&B}-M?sA{uh>n)#JkCAIcT~ z9ifL#4b3!a{x~$|$||z+pD1Vk?HU@=d(qfiIf9f+FUwX|OMedjtLEWGWx%gBkoU4s zhdQCWS*oL#Mmf7&iociBs|O?c)EoZiceiFlU$Vg+YP^|%e1Q4k{qO_ISSd7l zwxb_fzNk(}pS-e1U2fBe#lEjjb2*`kF8HMQRfr5C3|~GGRMeE_H65ZBvwh4mnvjSnl6?>w$Xpr?$txk+Oc+ z9$-#Jd|`0J`e)u&h8J1~dv7+byVmG1!wcKCu6W!L_f)y`s+O*$FTJWQ+Y9g1fqC(Y zHFxdOf2Hj3dhFb>6HDUHJe8B}TD@mk^$xEpC|3fehZ@dw>b#)iE>v_K8xp7Htk+rv zJ(S4i5jMVRT~)!Xm}@HaqB4efwT;PiVJ9{-RlfE9@R{Quk{419c^gJho2$RA^pP#a JmX9C2{C~IK#qR(B literal 0 HcmV?d00001 diff --git a/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_4470_testnet_sepolia.calldata b/core/lib/zksync_core/src/consistency_checker/tests/commit_l1_batch_4470_testnet_sepolia.calldata new file mode 100644 index 0000000000000000000000000000000000000000..d1ecde78199a848733672b25171d809e54479997 GIT binary patch literal 1956 zcmXSrk2uPJ7YLTFe6yD~x18~Jbpii|G{=;_nU--K_8(0e=Ih99*E+Sn4zG5mlWXwG z10|1Ma${`!u52@@u6EO%fzhD_Y{i%(+kKvPNU^=<(#D`Ury#oHvQ~A4hrd^EE|yR^Vpi_D*7)Bt`45YhKRUKE;zfs2 z(Wh^rN=yABUSS41(q| z!<7OV=<%1@xaTo2#b7g+PI6i;ND%ZBV;ENn&`=5k?s@!O95I=GE!I8U%C?HUNuX` z#YPtmj2nPTKO3p4F4&mMz`(Tdt|QZNK>;<}AZDy2o{9Ha7ru zYIeds1TqL1ZlJ)^>SP6T-~^Kw1A}u8LtFC1r40@o24_8TK1v<8-@3JB{VUzGd=ohG zf)*%M-B183%4OIyFMY}9d{+%K@0l8L*BiG^V*20qbe^DPfv?lzhB-&K02Spilzk5p zxjE~bRm;RDf8@$#b~+>*`!~KclDxrrbi)ie!62Zbe1`NjYmc0}a?#w?x#de|u L1BatchWithMetadata { + L1BatchWithMetadata { + header: create_l1_batch(number), + metadata: create_l1_batch_metadata(number), + factory_deps: vec![], + } +} + +const PRE_BOOJUM_PROTOCOL_VERSION: ProtocolVersionId = ProtocolVersionId::Version10; + +fn create_pre_boojum_l1_batch_with_metadata(number: u32) -> L1BatchWithMetadata { + let mut l1_batch = L1BatchWithMetadata { + header: create_l1_batch(number), + metadata: create_l1_batch_metadata(number), + factory_deps: vec![], + }; + l1_batch.header.protocol_version = Some(PRE_BOOJUM_PROTOCOL_VERSION); + l1_batch.metadata.bootloader_initial_content_commitment = None; + l1_batch.metadata.events_queue_commitment = None; + l1_batch +} + +fn build_commit_tx_input_data(batches: &[L1BatchWithMetadata]) -> Vec { + let commit_tokens = batches.iter().map(L1BatchWithMetadata::l1_commit_data); + let commit_tokens = ethabi::Token::Array(commit_tokens.collect()); + + let mut encoded = vec![]; + // Fake Solidity function selector (not checked for now) + encoded.extend_from_slice(b"fake"); + // Mock an additional arg used in real `commitBlocks` / `commitBatches`. In real transactions, + // it's taken from the L1 batch previous to batches[0], but since this arg is not checked, + // it's OK to use batches[0]. + let prev_header_tokens = batches[0].l1_header_data(); + encoded.extend_from_slice(ðabi::encode(&[prev_header_tokens, commit_tokens])); + encoded +} + +fn create_mock_checker(client: MockEthereum, pool: ConnectionPool) -> ConsistencyChecker { + ConsistencyChecker { + contract: zksync_contracts::zksync_contract(), + max_batches_to_recheck: 100, + sleep_interval: Duration::from_millis(10), + l1_client: Box::new(client), + l1_batch_updater: Box::new(()), + l1_data_mismatch_behavior: L1DataMismatchBehavior::Bail, + pool, + } +} + +impl UpdateCheckedBatch for mpsc::UnboundedSender { + fn update_checked_batch(&mut self, last_checked_batch: L1BatchNumber) { + self.send(last_checked_batch).ok(); + } +} + +#[test] +fn build_commit_tx_input_data_is_correct() { + let contract = zksync_contracts::zksync_contract(); + let commit_function = contract.function("commitBatches").unwrap(); + let batches = vec![ + create_l1_batch_with_metadata(1), + create_l1_batch_with_metadata(2), + ]; + + let commit_tx_input_data = build_commit_tx_input_data(&batches); + + for batch in &batches { + let commit_data = ConsistencyChecker::extract_commit_data( + &commit_tx_input_data, + commit_function, + batch.header.number, + ) + .unwrap(); + assert_eq!(commit_data, batch.l1_commit_data()); + } +} + +#[test] +fn extracting_commit_data_for_boojum_batch() { + let contract = zksync_contracts::zksync_contract(); + let commit_function = contract.function("commitBatches").unwrap(); + // Calldata taken from the commit transaction for https://sepolia.explorer.zksync.io/batch/4470; + // https://sepolia.etherscan.io/tx/0x300b9115037028b1f8aa2177abf98148c3df95c9b04f95a4e25baf4dfee7711f + let commit_tx_input_data = include_bytes!("commit_l1_batch_4470_testnet_sepolia.calldata"); + + let commit_data = ConsistencyChecker::extract_commit_data( + commit_tx_input_data, + commit_function, + L1BatchNumber(4_470), + ) + .unwrap(); + + assert_matches!( + commit_data, + ethabi::Token::Tuple(tuple) if tuple[0] == ethabi::Token::Uint(4_470.into()) + ); + + for bogus_l1_batch in [0, 1, 1_000, 4_469, 4_471, 100_000] { + ConsistencyChecker::extract_commit_data( + commit_tx_input_data, + commit_function, + L1BatchNumber(bogus_l1_batch), + ) + .unwrap_err(); + } +} + +#[test] +fn extracting_commit_data_for_multiple_batches() { + let contract = zksync_contracts::zksync_contract(); + let commit_function = contract.function("commitBatches").unwrap(); + // Calldata taken from the commit transaction for https://explorer.zksync.io/batch/351000; + // https://etherscan.io/tx/0xbd8dfe0812df0da534eb95a2d2a4382d65a8172c0b648a147d60c1c2921227fd + let commit_tx_input_data = include_bytes!("commit_l1_batch_351000-351004_mainnet.calldata"); + + for l1_batch in 351_000..=351_004 { + let commit_data = ConsistencyChecker::extract_commit_data( + commit_tx_input_data, + commit_function, + L1BatchNumber(l1_batch), + ) + .unwrap(); + + assert_matches!( + commit_data, + ethabi::Token::Tuple(tuple) if tuple[0] == ethabi::Token::Uint(l1_batch.into()) + ); + } + + for bogus_l1_batch in [350_000, 350_999, 351_005, 352_000] { + ConsistencyChecker::extract_commit_data( + commit_tx_input_data, + commit_function, + L1BatchNumber(bogus_l1_batch), + ) + .unwrap_err(); + } +} + +#[test] +fn extracting_commit_data_for_pre_boojum_batch() { + // Calldata taken from the commit transaction for https://goerli.explorer.zksync.io/batch/200000; + // https://goerli.etherscan.io/tx/0xfd2ef4ccd1223f502cc4a4e0f76c6905feafabc32ba616e5f70257eb968f20a3 + let commit_tx_input_data = include_bytes!("commit_l1_batch_200000_testnet_goerli.calldata"); + + let commit_data = ConsistencyChecker::extract_commit_data( + commit_tx_input_data, + &PRE_BOOJUM_COMMIT_FUNCTION, + L1BatchNumber(200_000), + ) + .unwrap(); + + assert_matches!( + commit_data, + ethabi::Token::Tuple(tuple) if tuple[0] == ethabi::Token::Uint(200_000.into()) + ); +} + +#[derive(Debug, Clone, Copy)] +enum SaveAction<'a> { + InsertBatch(&'a L1BatchWithMetadata), + SaveMetadata(&'a L1BatchWithMetadata), + InsertCommitTx(L1BatchNumber), +} + +impl SaveAction<'_> { + async fn apply( + self, + storage: &mut StorageProcessor<'_>, + commit_tx_hash_by_l1_batch: &HashMap, + ) { + match self { + Self::InsertBatch(l1_batch) => { + storage + .blocks_dal() + .insert_l1_batch(&l1_batch.header, &[], BlockGasCount::default(), &[], &[]) + .await + .unwrap(); + } + Self::SaveMetadata(l1_batch) => { + storage + .blocks_dal() + .save_l1_batch_metadata( + l1_batch.header.number, + &l1_batch.metadata, + H256::default(), + l1_batch.header.protocol_version.unwrap().is_pre_boojum(), + ) + .await + .unwrap(); + } + Self::InsertCommitTx(l1_batch_number) => { + let commit_tx_hash = commit_tx_hash_by_l1_batch[&l1_batch_number]; + storage + .eth_sender_dal() + .insert_bogus_confirmed_eth_tx( + l1_batch_number, + AggregatedActionType::Commit, + commit_tx_hash, + chrono::Utc::now(), + ) + .await + .unwrap(); + } + } + } +} + +type SaveActionMapper = fn(&[L1BatchWithMetadata]) -> Vec>; + +/// Various strategies to persist L1 batches in the DB. Strings are added for debugging failed test cases. +const SAVE_ACTION_MAPPERS: [(&str, SaveActionMapper); 4] = [ + ("sequential_metadata_first", |l1_batches| { + l1_batches + .iter() + .flat_map(|batch| { + [ + SaveAction::InsertBatch(batch), + SaveAction::SaveMetadata(batch), + SaveAction::InsertCommitTx(batch.header.number), + ] + }) + .collect() + }), + ("sequential_commit_txs_first", |l1_batches| { + l1_batches + .iter() + .flat_map(|batch| { + [ + SaveAction::InsertBatch(batch), + SaveAction::InsertCommitTx(batch.header.number), + SaveAction::SaveMetadata(batch), + ] + }) + .collect() + }), + ("all_metadata_first", |l1_batches| { + let commit_tx_actions = l1_batches + .iter() + .map(|batch| SaveAction::InsertCommitTx(batch.header.number)); + l1_batches + .iter() + .map(SaveAction::InsertBatch) + .chain(l1_batches.iter().map(SaveAction::SaveMetadata)) + .chain(commit_tx_actions) + .collect() + }), + ("all_commit_txs_first", |l1_batches| { + let commit_tx_actions = l1_batches + .iter() + .map(|batch| SaveAction::InsertCommitTx(batch.header.number)); + l1_batches + .iter() + .map(SaveAction::InsertBatch) + .chain(commit_tx_actions) + .chain(l1_batches.iter().map(SaveAction::SaveMetadata)) + .collect() + }), +]; + +#[test_casing(12, Product(([10, 3, 1], SAVE_ACTION_MAPPERS)))] +#[tokio::test] +async fn normal_checker_function( + batches_per_transaction: usize, + (mapper_name, save_actions_mapper): (&'static str, SaveActionMapper), +) { + println!("Using save_actions_mapper={mapper_name}"); + + let pool = ConnectionPool::test_pool().await; + let mut storage = pool.access_storage().await.unwrap(); + ensure_genesis_state(&mut storage, L2ChainId::default(), &GenesisParams::mock()) + .await + .unwrap(); + + let l1_batches: Vec<_> = (1..=10).map(create_l1_batch_with_metadata).collect(); + let mut commit_tx_hash_by_l1_batch = HashMap::with_capacity(l1_batches.len()); + let client = MockEthereum::default(); + + for (i, l1_batches) in l1_batches.chunks(batches_per_transaction).enumerate() { + let input_data = build_commit_tx_input_data(l1_batches); + let signed_tx = client.sign_prepared_tx( + input_data.clone(), + Options { + nonce: Some(i.into()), + ..Options::default() + }, + ); + let signed_tx = signed_tx.unwrap(); + client.send_raw_tx(signed_tx.raw_tx).await.unwrap(); + client.execute_tx(signed_tx.hash, true, 1); + + commit_tx_hash_by_l1_batch.extend( + l1_batches + .iter() + .map(|batch| (batch.header.number, signed_tx.hash)), + ); + } + + let (l1_batch_updates_sender, mut l1_batch_updates_receiver) = mpsc::unbounded_channel(); + let checker = ConsistencyChecker { + l1_batch_updater: Box::new(l1_batch_updates_sender), + ..create_mock_checker(client, pool.clone()) + }; + + let (stop_sender, stop_receiver) = watch::channel(false); + let checker_task = tokio::spawn(checker.run(stop_receiver)); + + // Add new batches to the storage. + for save_action in save_actions_mapper(&l1_batches) { + save_action + .apply(&mut storage, &commit_tx_hash_by_l1_batch) + .await; + tokio::time::sleep(Duration::from_millis(7)).await; + } + + // Wait until all batches are checked. + loop { + let checked_batch = l1_batch_updates_receiver.recv().await.unwrap(); + if checked_batch == l1_batches.last().unwrap().header.number { + break; + } + } + + // Send the stop signal to the checker and wait for it to stop. + stop_sender.send_replace(true); + checker_task.await.unwrap().unwrap(); +} + +#[test_casing(4, SAVE_ACTION_MAPPERS)] +#[tokio::test] +async fn checker_processes_pre_boojum_batches( + (mapper_name, save_actions_mapper): (&'static str, SaveActionMapper), +) { + println!("Using save_actions_mapper={mapper_name}"); + + let pool = ConnectionPool::test_pool().await; + let mut storage = pool.access_storage().await.unwrap(); + let genesis_params = GenesisParams { + protocol_version: PRE_BOOJUM_PROTOCOL_VERSION, + ..GenesisParams::mock() + }; + ensure_genesis_state(&mut storage, L2ChainId::default(), &genesis_params) + .await + .unwrap(); + storage + .protocol_versions_dal() + .save_protocol_version_with_tx(ProtocolVersion::default()) + .await; + + let l1_batches: Vec<_> = (1..=5) + .map(create_pre_boojum_l1_batch_with_metadata) + .chain((6..=10).map(create_l1_batch_with_metadata)) + .collect(); + let mut commit_tx_hash_by_l1_batch = HashMap::with_capacity(l1_batches.len()); + let client = MockEthereum::default(); + + for (i, l1_batch) in l1_batches.iter().enumerate() { + let input_data = build_commit_tx_input_data(slice::from_ref(l1_batch)); + let signed_tx = client.sign_prepared_tx( + input_data.clone(), + Options { + nonce: Some(i.into()), + ..Options::default() + }, + ); + let signed_tx = signed_tx.unwrap(); + client.send_raw_tx(signed_tx.raw_tx).await.unwrap(); + client.execute_tx(signed_tx.hash, true, 1); + + commit_tx_hash_by_l1_batch.insert(l1_batch.header.number, signed_tx.hash); + } + + let (l1_batch_updates_sender, mut l1_batch_updates_receiver) = mpsc::unbounded_channel(); + let checker = ConsistencyChecker { + l1_batch_updater: Box::new(l1_batch_updates_sender), + ..create_mock_checker(client, pool.clone()) + }; + + let (stop_sender, stop_receiver) = watch::channel(false); + let checker_task = tokio::spawn(checker.run(stop_receiver)); + + // Add new batches to the storage. + for save_action in save_actions_mapper(&l1_batches) { + save_action + .apply(&mut storage, &commit_tx_hash_by_l1_batch) + .await; + tokio::time::sleep(Duration::from_millis(7)).await; + } + + // Wait until all batches are checked. + loop { + let checked_batch = l1_batch_updates_receiver.recv().await.unwrap(); + if checked_batch == l1_batches.last().unwrap().header.number { + break; + } + } + + // Send the stop signal to the checker and wait for it to stop. + stop_sender.send_replace(true); + checker_task.await.unwrap().unwrap(); +} + +#[test_casing(2, [false, true])] +#[tokio::test] +async fn checker_functions_after_snapshot_recovery(delay_batch_insertion: bool) { + let pool = ConnectionPool::test_pool().await; + let mut storage = pool.access_storage().await.unwrap(); + storage + .protocol_versions_dal() + .save_protocol_version_with_tx(ProtocolVersion::default()) + .await; + + let l1_batch = create_l1_batch_with_metadata(99); + + let commit_tx_input_data = build_commit_tx_input_data(slice::from_ref(&l1_batch)); + let client = MockEthereum::default(); + let signed_tx = client.sign_prepared_tx( + commit_tx_input_data.clone(), + Options { + nonce: Some(0.into()), + ..Options::default() + }, + ); + let signed_tx = signed_tx.unwrap(); + let commit_tx_hash = signed_tx.hash; + client.send_raw_tx(signed_tx.raw_tx).await.unwrap(); + client.execute_tx(commit_tx_hash, true, 1); + + let save_actions = [ + SaveAction::InsertBatch(&l1_batch), + SaveAction::SaveMetadata(&l1_batch), + SaveAction::InsertCommitTx(l1_batch.header.number), + ]; + let commit_tx_hash_by_l1_batch = HashMap::from([(l1_batch.header.number, commit_tx_hash)]); + + if !delay_batch_insertion { + for &save_action in &save_actions { + save_action + .apply(&mut storage, &commit_tx_hash_by_l1_batch) + .await; + } + } + + let (l1_batch_updates_sender, mut l1_batch_updates_receiver) = mpsc::unbounded_channel(); + let checker = ConsistencyChecker { + l1_batch_updater: Box::new(l1_batch_updates_sender), + ..create_mock_checker(client, pool.clone()) + }; + let (stop_sender, stop_receiver) = watch::channel(false); + let checker_task = tokio::spawn(checker.run(stop_receiver)); + + if delay_batch_insertion { + tokio::time::sleep(Duration::from_millis(10)).await; + for &save_action in &save_actions { + save_action + .apply(&mut storage, &commit_tx_hash_by_l1_batch) + .await; + } + } + + // Wait until the batch is checked. + let checked_batch = l1_batch_updates_receiver.recv().await.unwrap(); + assert_eq!(checked_batch, l1_batch.header.number); + + stop_sender.send_replace(true); + checker_task.await.unwrap().unwrap(); +} + +#[derive(Debug, Clone, Copy)] +enum IncorrectDataKind { + MissingStatus, + MismatchedStatus, + BogusCommitDataFormat, + MismatchedCommitDataTimestamp, + CommitDataForAnotherBatch, + CommitDataForPreBoojum, +} + +impl IncorrectDataKind { + const ALL: [Self; 6] = [ + Self::MissingStatus, + Self::MismatchedStatus, + Self::BogusCommitDataFormat, + Self::MismatchedCommitDataTimestamp, + Self::CommitDataForAnotherBatch, + Self::CommitDataForPreBoojum, + ]; + + async fn apply(self, client: &MockEthereum, l1_batch: &L1BatchWithMetadata) -> H256 { + let (commit_tx_input_data, successful_status) = match self { + Self::MissingStatus => { + return H256::zero(); // Do not execute the transaction + } + Self::MismatchedStatus => { + let commit_tx_input_data = build_commit_tx_input_data(slice::from_ref(l1_batch)); + (commit_tx_input_data, false) + } + Self::BogusCommitDataFormat => { + let mut bogus_tx_input_data = b"test".to_vec(); // Preserve the function selector + bogus_tx_input_data + .extend_from_slice(ðabi::encode(&[ethabi::Token::Bool(true)])); + (bogus_tx_input_data, true) + } + Self::MismatchedCommitDataTimestamp => { + let mut l1_batch = create_l1_batch_with_metadata(1); + l1_batch.header.timestamp += 1; + let bogus_tx_input_data = build_commit_tx_input_data(slice::from_ref(&l1_batch)); + (bogus_tx_input_data, true) + } + Self::CommitDataForAnotherBatch => { + let l1_batch = create_l1_batch_with_metadata(100); + let bogus_tx_input_data = build_commit_tx_input_data(slice::from_ref(&l1_batch)); + (bogus_tx_input_data, true) + } + Self::CommitDataForPreBoojum => { + let mut l1_batch = create_l1_batch_with_metadata(1); + l1_batch.header.protocol_version = Some(ProtocolVersionId::Version0); + let bogus_tx_input_data = build_commit_tx_input_data(slice::from_ref(&l1_batch)); + (bogus_tx_input_data, true) + } + }; + + let signed_tx = client.sign_prepared_tx( + commit_tx_input_data, + Options { + nonce: Some(0.into()), + ..Options::default() + }, + ); + let signed_tx = signed_tx.unwrap(); + client.send_raw_tx(signed_tx.raw_tx).await.unwrap(); + client.execute_tx(signed_tx.hash, successful_status, 1); + signed_tx.hash + } +} + +#[test_casing(6, Product((IncorrectDataKind::ALL, [false])))] +// ^ `snapshot_recovery = true` is tested below; we don't want to run it with all incorrect data kinds +#[tokio::test] +async fn checker_detects_incorrect_tx_data(kind: IncorrectDataKind, snapshot_recovery: bool) { + let pool = ConnectionPool::test_pool().await; + let mut storage = pool.access_storage().await.unwrap(); + if snapshot_recovery { + storage + .protocol_versions_dal() + .save_protocol_version_with_tx(ProtocolVersion::default()) + .await; + } else { + ensure_genesis_state(&mut storage, L2ChainId::default(), &GenesisParams::mock()) + .await + .unwrap(); + } + + let l1_batch = create_l1_batch_with_metadata(if snapshot_recovery { 99 } else { 1 }); + let client = MockEthereum::default(); + let commit_tx_hash = kind.apply(&client, &l1_batch).await; + let commit_tx_hash_by_l1_batch = HashMap::from([(l1_batch.header.number, commit_tx_hash)]); + + let save_actions = [ + SaveAction::InsertBatch(&l1_batch), + SaveAction::SaveMetadata(&l1_batch), + SaveAction::InsertCommitTx(l1_batch.header.number), + ]; + for save_action in save_actions { + save_action + .apply(&mut storage, &commit_tx_hash_by_l1_batch) + .await; + } + drop(storage); + + let checker = create_mock_checker(client, pool); + let (_stop_sender, stop_receiver) = watch::channel(false); + // The checker must stop with an error. + tokio::time::timeout(Duration::from_secs(30), checker.run(stop_receiver)) + .await + .expect("Timed out waiting for checker to stop") + .unwrap_err(); +} + +#[tokio::test] +async fn checker_detects_incorrect_tx_data_after_snapshot_recovery() { + checker_detects_incorrect_tx_data(IncorrectDataKind::CommitDataForAnotherBatch, true).await; +} diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index 2fae4cda755f..62a5808e2b7b 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -6,7 +6,6 @@ use zksync_config::{ configs::eth_sender::{ProofSendingMode, SenderConfig}, ContractsConfig, ETHSenderConfig, GasAdjusterConfig, }; -use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_eth_client::{clients::MockEthereum, EthInterface}; use zksync_object_store::ObjectStoreFactory; @@ -27,6 +26,7 @@ use crate::{ eth_tx_manager::L1BlockNumbers, Aggregator, ETHSenderError, EthTxAggregator, EthTxManager, }, l1_gas_price::GasAdjuster, + utils::testonly::create_l1_batch, }; // Alias to conveniently call static methods of ETHSender. @@ -35,13 +35,7 @@ type MockEthTxManager = EthTxManager; static DUMMY_OPERATION: Lazy = Lazy::new(|| { AggregatedOperation::Execute(L1BatchExecuteOperation { l1_batches: vec![L1BatchWithMetadata { - header: L1BatchHeader::new( - L1BatchNumber(1), - 1, - Address::default(), - BaseSystemContractsHashes::default(), - ProtocolVersionId::latest(), - ), + header: create_l1_batch(1), metadata: default_l1_batch_metadata(), factory_deps: Vec::new(), }], @@ -886,14 +880,7 @@ async fn insert_genesis_protocol_version(tester: &EthSenderTester) { } async fn insert_l1_batch(tester: &EthSenderTester, number: L1BatchNumber) -> L1BatchHeader { - let mut header = L1BatchHeader::new( - number, - 0, - Address::zero(), - BaseSystemContractsHashes::default(), - Default::default(), - ); - header.is_finished = true; + let header = create_l1_batch(number.0); // Save L1 batch to the database tester diff --git a/core/lib/zksync_core/src/genesis.rs b/core/lib/zksync_core/src/genesis.rs index bba6a2d6744a..cbd08e329cb0 100644 --- a/core/lib/zksync_core/src/genesis.rs +++ b/core/lib/zksync_core/src/genesis.rs @@ -283,7 +283,7 @@ pub(crate) async fn create_genesis_l1_batch( 0, first_validator_address, base_system_contracts.hashes(), - ProtocolVersionId::latest(), + protocol_version, ); genesis_l1_batch_header.is_finished = true; @@ -297,7 +297,7 @@ pub(crate) async fn create_genesis_l1_batch( l1_gas_price: 0, l2_fair_gas_price: 0, base_system_contracts_hashes: base_system_contracts.hashes(), - protocol_version: Some(ProtocolVersionId::latest()), + protocol_version: Some(protocol_version), virtual_blocks: 0, }; @@ -437,7 +437,7 @@ mod tests { #[tokio::test] async fn running_genesis_with_big_chain_id() { let pool = ConnectionPool::test_pool().await; - let mut conn: StorageProcessor<'_> = pool.access_storage().await.unwrap(); + let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal().delete_genesis().await.unwrap(); let params = GenesisParams { @@ -460,4 +460,19 @@ mod tests { let root_hash = metadata.unwrap().unwrap().metadata.root_hash; assert_ne!(root_hash, H256::zero()); } + + #[tokio::test] + async fn running_genesis_with_non_latest_protocol_version() { + let pool = ConnectionPool::test_pool().await; + let mut conn = pool.access_storage().await.unwrap(); + let params = GenesisParams { + protocol_version: ProtocolVersionId::Version10, + ..GenesisParams::mock() + }; + + ensure_genesis_state(&mut conn, L2ChainId::max(), ¶ms) + .await + .unwrap(); + assert!(!conn.blocks_dal().is_genesis_needed().await.unwrap()); + } } diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index b982039dbc57..ba8c18daf297 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -10,21 +10,23 @@ use zksync_config::configs::{ chain::OperationsManagerConfig, database::{MerkleTreeConfig, MerkleTreeMode}, }; -use zksync_contracts::BaseSystemContracts; use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_health_check::{CheckHealth, HealthStatus}; use zksync_merkle_tree::domain::ZkSyncTree; use zksync_object_store::{ObjectStore, ObjectStoreFactory}; use zksync_types::{ - block::{BlockGasCount, L1BatchHeader, MiniblockHasher, MiniblockHeader}, + block::{BlockGasCount, L1BatchHeader}, proofs::PrepareBasicCircuitsJob, - AccountTreeId, Address, L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, - StorageKey, StorageLog, H256, + AccountTreeId, Address, L1BatchNumber, L2ChainId, MiniblockNumber, StorageKey, StorageLog, + H256, }; use zksync_utils::u32_to_h256; use super::{GenericAsyncTree, L1BatchWithLogs, MetadataCalculator, MetadataCalculatorConfig}; -use crate::genesis::{ensure_genesis_state, GenesisParams}; +use crate::{ + genesis::{ensure_genesis_state, GenesisParams}, + utils::testonly::{create_l1_batch, create_miniblock}, +}; const RUN_TIMEOUT: Duration = Duration::from_secs(30); @@ -485,34 +487,12 @@ pub(super) async fn extend_db_state_from_l1_batch( ) { assert!(storage.in_transaction(), "must be called in DB transaction"); - let base_system_contracts = BaseSystemContracts::load_from_disk(); for (idx, batch_logs) in (next_l1_batch.0..).zip(new_logs) { - let batch_number = L1BatchNumber(idx); - let mut header = L1BatchHeader::new( - batch_number, - 0, - Address::default(), - base_system_contracts.hashes(), - Default::default(), - ); - header.is_finished = true; - + let header = create_l1_batch(idx); + let batch_number = header.number; // Assumes that L1 batch consists of only one miniblock. - let miniblock_number = MiniblockNumber(idx); - let miniblock_header = MiniblockHeader { - number: miniblock_number, - timestamp: header.timestamp, - hash: MiniblockHasher::new(miniblock_number, header.timestamp, H256::zero()) - .finalize(ProtocolVersionId::latest()), - l1_tx_count: header.l1_tx_count, - l2_tx_count: header.l2_tx_count, - base_fee_per_gas: header.base_fee_per_gas, - l1_gas_price: 0, - l2_fair_gas_price: 0, - base_system_contracts_hashes: base_system_contracts.hashes(), - protocol_version: Some(ProtocolVersionId::latest()), - virtual_blocks: 0, - }; + let miniblock_header = create_miniblock(idx); + let miniblock_number = miniblock_header.number; storage .blocks_dal() diff --git a/core/lib/zksync_core/src/reorg_detector/tests.rs b/core/lib/zksync_core/src/reorg_detector/tests.rs index 6f09c0e9fee7..dc5d0711445f 100644 --- a/core/lib/zksync_core/src/reorg_detector/tests.rs +++ b/core/lib/zksync_core/src/reorg_detector/tests.rs @@ -8,29 +8,22 @@ use std::{ use assert_matches::assert_matches; use test_casing::{test_casing, Product}; use tokio::sync::mpsc; -use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::StorageProcessor; use zksync_types::{ - block::{BlockGasCount, L1BatchHeader, MiniblockHeader}, - Address, L2ChainId, ProtocolVersion, ProtocolVersionId, + block::{BlockGasCount, MiniblockHeader}, + L2ChainId, ProtocolVersion, }; use super::*; -use crate::genesis::{ensure_genesis_state, GenesisParams}; +use crate::{ + genesis::{ensure_genesis_state, GenesisParams}, + utils::testonly::{create_l1_batch, create_miniblock}, +}; async fn store_miniblock(storage: &mut StorageProcessor<'_>, number: u32, hash: H256) { let header = MiniblockHeader { - number: MiniblockNumber(number), - timestamp: number.into(), hash, - l1_tx_count: 0, - l2_tx_count: 0, - base_fee_per_gas: 0, - l1_gas_price: 0, - l2_fair_gas_price: 0, - base_system_contracts_hashes: BaseSystemContractsHashes::default(), - protocol_version: Some(ProtocolVersionId::latest()), - virtual_blocks: 1, + ..create_miniblock(number) }; storage .blocks_dal() @@ -40,13 +33,7 @@ async fn store_miniblock(storage: &mut StorageProcessor<'_>, number: u32, hash: } async fn seal_l1_batch(storage: &mut StorageProcessor<'_>, number: u32, hash: H256) { - let header = L1BatchHeader::new( - L1BatchNumber(number), - number.into(), - Address::default(), - BaseSystemContractsHashes::default(), - ProtocolVersionId::latest(), - ); + let header = create_l1_batch(number); storage .blocks_dal() .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index ce3a7714b2be..dcbdac448275 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -12,14 +12,17 @@ use zksync_types::{ use zksync_utils::time::seconds_since_epoch; use self::tester::Tester; -use crate::state_keeper::{ - io::{MiniblockParams, MiniblockSealer, StateKeeperIO}, - mempool_actor::l2_tx_filter, - tests::{ - create_execution_result, create_l1_batch_metadata, create_transaction, - create_updates_manager, default_l1_batch_env, default_vm_block_result, Query, +use crate::{ + state_keeper::{ + io::{MiniblockParams, MiniblockSealer, StateKeeperIO}, + mempool_actor::l2_tx_filter, + tests::{ + create_execution_result, create_transaction, create_updates_manager, + default_l1_batch_env, default_vm_block_result, Query, + }, + updates::{MiniblockSealCommand, MiniblockUpdates, UpdatesManager}, }, - updates::{MiniblockSealCommand, MiniblockUpdates, UpdatesManager}, + utils::testonly::create_l1_batch_metadata, }; mod tester; diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs index ea5714a38944..2f2a38577998 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs @@ -9,16 +9,16 @@ use zksync_dal::ConnectionPool; use zksync_eth_client::clients::MockEthereum; use zksync_object_store::ObjectStoreFactory; use zksync_types::{ - block::{L1BatchHeader, MiniblockHeader}, - protocol_version::L1VerifierConfig, - system_contracts::get_system_smart_contracts, - Address, L1BatchNumber, L2ChainId, MiniblockNumber, PriorityOpId, ProtocolVersionId, H256, + block::MiniblockHeader, protocol_version::L1VerifierConfig, + system_contracts::get_system_smart_contracts, Address, L2ChainId, PriorityOpId, + ProtocolVersionId, H256, }; use crate::{ genesis::create_genesis_l1_batch, l1_gas_price::GasAdjuster, state_keeper::{io::MiniblockSealer, tests::create_transaction, MempoolGuard, MempoolIO}, + utils::testonly::{create_l1_batch, create_miniblock}, }; #[derive(Debug)] @@ -130,32 +130,19 @@ impl Tester { storage .blocks_dal() .insert_miniblock(&MiniblockHeader { - number: MiniblockNumber(number), timestamp: self.current_timestamp, - hash: H256::default(), - l1_tx_count: 0, - l2_tx_count: 0, base_fee_per_gas, l1_gas_price, l2_fair_gas_price, base_system_contracts_hashes: self.base_system_contracts.hashes(), - protocol_version: Some(ProtocolVersionId::latest()), - virtual_blocks: 0, + ..create_miniblock(number) }) .await .unwrap(); } pub(super) async fn insert_sealed_batch(&self, pool: &ConnectionPool, number: u32) { - let mut batch_header = L1BatchHeader::new( - L1BatchNumber(number), - self.current_timestamp, - Address::default(), - self.base_system_contracts.hashes(), - Default::default(), - ); - batch_header.is_finished = true; - + let batch_header = create_l1_batch(number); let mut storage = pool.access_storage_tagged("state_keeper").await.unwrap(); storage .blocks_dal() diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index 044e65742e78..b9d8da1e7e12 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -20,12 +20,8 @@ use zksync_system_constants::ZKPORTER_IS_AVAILABLE; use zksync_types::{ aggregated_operations::AggregatedActionType, block::{BlockGasCount, MiniblockExecutionData, MiniblockHasher}, - commitment::{L1BatchMetaParameters, L1BatchMetadata}, - fee::Fee, - l2::L2Tx, - transaction_request::PaymasterParams, tx::tx_execution_info::ExecutionMetrics, - Address, L1BatchNumber, L2ChainId, LogQuery, MiniblockNumber, Nonce, ProtocolVersionId, + Address, L1BatchNumber, L2ChainId, LogQuery, MiniblockNumber, ProtocolVersionId, StorageLogQuery, StorageLogQueryType, Timestamp, Transaction, H256, U256, }; @@ -47,6 +43,7 @@ use crate::{ types::ExecutionMetricsForCriteria, updates::UpdatesManager, }, + utils::testonly::create_l2_transaction, }; pub(super) static BASE_SYSTEM_CONTRACTS: Lazy = @@ -86,30 +83,6 @@ pub(super) fn default_l1_batch_env( } } -pub(crate) fn create_l1_batch_metadata(number: u32) -> L1BatchMetadata { - L1BatchMetadata { - root_hash: H256::from_low_u64_be(number.into()), - rollup_last_leaf_index: u64::from(number) + 20, - merkle_root_hash: H256::from_low_u64_be(number.into()), - initial_writes_compressed: vec![], - repeated_writes_compressed: vec![], - commitment: H256::from_low_u64_be(number.into()), - l2_l1_messages_compressed: vec![], - l2_l1_merkle_root: H256::from_low_u64_be(number.into()), - block_meta_params: L1BatchMetaParameters { - zkporter_is_available: ZKPORTER_IS_AVAILABLE, - bootloader_code_hash: BASE_SYSTEM_CONTRACTS.bootloader.hash, - default_aa_code_hash: BASE_SYSTEM_CONTRACTS.default_aa.hash, - }, - aux_data_hash: H256::zero(), - meta_parameters_hash: H256::zero(), - pass_through_data_hash: H256::zero(), - events_queue_commitment: Some(H256::zero()), - bootloader_initial_content_commitment: Some(H256::zero()), - state_diffs_compressed: vec![], - } -} - pub(super) fn default_vm_block_result() -> FinishedL1Batch { FinishedL1Batch { block_tip_execution_result: VmExecutionResultAndLogs { @@ -143,32 +116,6 @@ pub(super) fn create_updates_manager() -> UpdatesManager { ) } -pub(crate) fn create_l2_transaction(fee_per_gas: u64, gas_per_pubdata: u32) -> L2Tx { - let fee = Fee { - gas_limit: 1000_u64.into(), - max_fee_per_gas: fee_per_gas.into(), - max_priority_fee_per_gas: 0_u64.into(), - gas_per_pubdata_limit: gas_per_pubdata.into(), - }; - let mut tx = L2Tx::new_signed( - Address::random(), - vec![], - Nonce(0), - fee, - U256::zero(), - L2ChainId::from(271), - &H256::random(), - None, - PaymasterParams::default(), - ) - .unwrap(); - // Input means all transaction data (NOT calldata, but all tx fields) that came from the API. - // This input will be used for the derivation of the tx hash, so put some random to it to be sure - // that the transaction hash is unique. - tx.set_input(H256::random().0.to_vec(), H256::random()); - tx -} - pub(super) fn create_transaction(fee_per_gas: u64, gas_per_pubdata: u32) -> Transaction { create_l2_transaction(fee_per_gas, gas_per_pubdata).into() } diff --git a/core/lib/zksync_core/src/state_keeper/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/tests/tester.rs index ce0293e7e679..83ac7f6b6e88 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/tester.rs @@ -20,16 +20,17 @@ use zksync_types::{ ProtocolVersionId, Transaction, H256, }; -use crate::state_keeper::{ - batch_executor::{BatchExecutorHandle, Command, L1BatchExecutorBuilder, TxExecutionResult}, - io::{MiniblockParams, PendingBatchData, StateKeeperIO}, - seal_criteria::{IoSealCriteria, SequencerSealer}, - tests::{ - create_l2_transaction, default_l1_batch_env, default_vm_block_result, BASE_SYSTEM_CONTRACTS, +use crate::{ + state_keeper::{ + batch_executor::{BatchExecutorHandle, Command, L1BatchExecutorBuilder, TxExecutionResult}, + io::{MiniblockParams, PendingBatchData, StateKeeperIO}, + seal_criteria::{IoSealCriteria, SequencerSealer}, + tests::{default_l1_batch_env, default_vm_block_result, BASE_SYSTEM_CONTRACTS}, + types::ExecutionMetricsForCriteria, + updates::UpdatesManager, + ZkSyncStateKeeper, }, - types::ExecutionMetricsForCriteria, - updates::UpdatesManager, - ZkSyncStateKeeper, + utils::testonly::create_l2_transaction, }; const FEE_ACCOUNT: Address = Address::repeat_byte(0x11); @@ -771,8 +772,8 @@ impl StateKeeperIO for TestIO { } } -/// `L1BatchExecutorBuilder` which doesn't check anything at all. -/// Accepts all transactions. +/// `L1BatchExecutorBuilder` which doesn't check anything at all. Accepts all transactions. +// FIXME: move to `utils`? #[derive(Debug)] pub(crate) struct MockBatchExecutorBuilder; @@ -783,7 +784,7 @@ impl L1BatchExecutorBuilder for MockBatchExecutorBuilder { _l1batch_params: L1BatchEnv, _system_env: SystemEnv, ) -> BatchExecutorHandle { - let (send, recv) = tokio::sync::mpsc::channel(1); + let (send, recv) = mpsc::channel(1); let handle = tokio::task::spawn(async { let mut recv = recv; while let Some(cmd) = recv.recv().await { diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 619ee282e718..d22409e5219c 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -19,10 +19,10 @@ use crate::{ consensus::testonly::MockMainNodeClient, genesis::{ensure_genesis_state, GenesisParams}, state_keeper::{ - seal_criteria::NoopSealer, - tests::{create_l1_batch_metadata, create_l2_transaction, TestBatchExecutorBuilder}, - MiniblockSealer, ZkSyncStateKeeper, + seal_criteria::NoopSealer, tests::TestBatchExecutorBuilder, MiniblockSealer, + ZkSyncStateKeeper, }, + utils::testonly::{create_l1_batch_metadata, create_l2_transaction}, }; const TEST_TIMEOUT: Duration = Duration::from_secs(10); diff --git a/core/lib/zksync_core/src/utils.rs b/core/lib/zksync_core/src/utils/mod.rs similarity index 99% rename from core/lib/zksync_core/src/utils.rs rename to core/lib/zksync_core/src/utils/mod.rs index 351b22350a38..e740c8f0a316 100644 --- a/core/lib/zksync_core/src/utils.rs +++ b/core/lib/zksync_core/src/utils/mod.rs @@ -6,6 +6,9 @@ use tokio::sync::watch; use zksync_dal::ConnectionPool; use zksync_types::L1BatchNumber; +#[cfg(test)] +pub(crate) mod testonly; + /// 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. /// diff --git a/core/lib/zksync_core/src/utils/testonly.rs b/core/lib/zksync_core/src/utils/testonly.rs new file mode 100644 index 000000000000..b026c951ef19 --- /dev/null +++ b/core/lib/zksync_core/src/utils/testonly.rs @@ -0,0 +1,94 @@ +//! Test utils. + +use zksync_contracts::BaseSystemContractsHashes; +use zksync_system_constants::ZKPORTER_IS_AVAILABLE; +use zksync_types::{ + block::{L1BatchHeader, MiniblockHeader}, + commitment::{L1BatchMetaParameters, L1BatchMetadata}, + fee::Fee, + l2::L2Tx, + transaction_request::PaymasterParams, + Address, L1BatchNumber, L2ChainId, MiniblockNumber, Nonce, ProtocolVersionId, H256, U256, +}; + +/// Creates a miniblock header with the specified number and deterministic contents. +pub(crate) fn create_miniblock(number: u32) -> MiniblockHeader { + MiniblockHeader { + number: MiniblockNumber(number), + timestamp: number.into(), + hash: H256::from_low_u64_be(number.into()), + l1_tx_count: 0, + l2_tx_count: 0, + base_fee_per_gas: 100, + l1_gas_price: 100, + l2_fair_gas_price: 100, + base_system_contracts_hashes: BaseSystemContractsHashes::default(), + protocol_version: Some(ProtocolVersionId::latest()), + virtual_blocks: 1, + } +} + +/// Creates an L1 batch header with the specified number and deterministic contents. +pub(crate) fn create_l1_batch(number: u32) -> L1BatchHeader { + let mut header = L1BatchHeader::new( + L1BatchNumber(number), + number.into(), + Address::default(), + BaseSystemContractsHashes::default(), + ProtocolVersionId::latest(), + ); + header.is_finished = true; + header +} + +/// Creates metadata for an L1 batch with the specified number. +pub(crate) fn create_l1_batch_metadata(number: u32) -> L1BatchMetadata { + L1BatchMetadata { + root_hash: H256::from_low_u64_be(number.into()), + rollup_last_leaf_index: u64::from(number) + 20, + merkle_root_hash: H256::from_low_u64_be(number.into()), + initial_writes_compressed: vec![], + repeated_writes_compressed: vec![], + commitment: H256::from_low_u64_be(number.into()), + l2_l1_messages_compressed: vec![], + l2_l1_merkle_root: H256::from_low_u64_be(number.into()), + block_meta_params: L1BatchMetaParameters { + zkporter_is_available: ZKPORTER_IS_AVAILABLE, + bootloader_code_hash: BaseSystemContractsHashes::default().bootloader, + default_aa_code_hash: BaseSystemContractsHashes::default().default_aa, + }, + aux_data_hash: H256::zero(), + meta_parameters_hash: H256::zero(), + pass_through_data_hash: H256::zero(), + events_queue_commitment: Some(H256::zero()), + bootloader_initial_content_commitment: Some(H256::zero()), + state_diffs_compressed: vec![], + } +} + +/// Creates an L2 transaction with randomized parameters. +pub(crate) fn create_l2_transaction(fee_per_gas: u64, gas_per_pubdata: u32) -> L2Tx { + let fee = Fee { + gas_limit: 1000_u64.into(), + max_fee_per_gas: fee_per_gas.into(), + max_priority_fee_per_gas: 0_u64.into(), + gas_per_pubdata_limit: gas_per_pubdata.into(), + }; + let mut tx = L2Tx::new_signed( + Address::random(), + vec![], + Nonce(0), + fee, + U256::zero(), + L2ChainId::from(271), + &H256::random(), + None, + PaymasterParams::default(), + ) + .unwrap(); + // Input means all transaction data (NOT calldata, but all tx fields) that came from the API. + // This input will be used for the derivation of the tx hash, so put some random to it to be sure + // that the transaction hash is unique. + tx.set_input(H256::random().0.to_vec(), H256::random()); + tx +} From 2ceb9114659f4c4583c87b1bbc8ee230eb1c44db Mon Sep 17 00:00:00 2001 From: EmilLuta Date: Fri, 5 Jan 2024 12:32:27 +0100 Subject: [PATCH 16/49] fix(prover): Remove prover-utils from core (#819) Last batch of removing old prover from core. Further improvements will follow up in the core <-> prover integration. These will be tackled as part of prover <-> core separation. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- Cargo.lock | 41 ------ Cargo.toml | 1 - core/lib/prover_utils/Cargo.toml | 26 ---- .../lib/prover_utils/src/gcs_proof_fetcher.rs | 22 --- core/lib/prover_utils/src/lib.rs | 126 ------------------ core/lib/zksync_core/Cargo.toml | 1 - .../zksync_core/src/eth_sender/aggregator.rs | 25 +++- .../src/house_keeper/blocks_state_reporter.rs | 6 +- .../fri_proof_compressor_job_retry_manager.rs | 3 +- .../fri_proof_compressor_queue_monitor.rs | 3 +- .../fri_prover_job_retry_manager.rs | 3 +- .../house_keeper/fri_prover_queue_monitor.rs | 3 +- .../fri_scheduler_circuit_queuer.rs | 3 +- ...ri_witness_generator_jobs_retry_manager.rs | 3 +- .../fri_witness_generator_queue_monitor.rs | 3 +- core/lib/zksync_core/src/house_keeper/mod.rs | 1 + .../src/house_keeper}/periodic_job.rs | 2 +- ...waiting_to_queued_fri_witness_job_mover.rs | 3 +- core/lib/zksync_core/src/lib.rs | 2 +- docs/guides/architecture.md | 1 - prover/Cargo.lock | 30 +---- prover/proof_fri_compressor/Cargo.toml | 2 +- .../src/initial_setup_keys.rs | 59 ++++++++ prover/proof_fri_compressor/src/main.rs | 7 +- prover/prover_fri/Cargo.toml | 3 +- prover/prover_fri/src/main.rs | 3 +- prover/prover_fri_utils/Cargo.toml | 4 + prover/prover_fri_utils/src/lib.rs | 1 + .../prover_fri_utils}/src/region_fetcher.rs | 2 +- .../Cargo.toml | 2 +- .../src/commitment_generator.rs | 8 +- .../src/lib.rs | 1 + .../src/vk_commitment_helper.rs | 0 prover/witness_generator/Cargo.toml | 1 - prover/witness_generator/src/main.rs | 8 +- prover/witness_vector_generator/Cargo.toml | 1 - prover/witness_vector_generator/src/main.rs | 3 +- 37 files changed, 139 insertions(+), 274 deletions(-) delete mode 100644 core/lib/prover_utils/Cargo.toml delete mode 100644 core/lib/prover_utils/src/gcs_proof_fetcher.rs delete mode 100644 core/lib/prover_utils/src/lib.rs rename core/lib/{prover_utils/src => zksync_core/src/house_keeper}/periodic_job.rs (97%) create mode 100644 prover/proof_fri_compressor/src/initial_setup_keys.rs rename {core/lib/prover_utils => prover/prover_fri_utils}/src/region_fetcher.rs (98%) rename {core/lib/prover_utils => prover/vk_setup_data_generator_server_fri}/src/vk_commitment_helper.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 9322e0e2a7b4..461f4df862cc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1369,16 +1369,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" -[[package]] -name = "combine" -version = "4.6.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" -dependencies = [ - "bytes", - "memchr", -] - [[package]] name = "compile-fmt" version = "0.1.0" @@ -7063,17 +7053,6 @@ version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7cda73e2f1397b1262d6dfdcef8aafae14d1de7748d66822d3bfeeb6d03e5e4b" -[[package]] -name = "toml_edit" -version = "0.14.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5376256e44f2443f8896ac012507c19a012df0fe8758b55246ae51a2279db51f" -dependencies = [ - "combine", - "indexmap 1.9.3", - "itertools", -] - [[package]] name = "toml_edit" version = "0.19.15" @@ -8500,7 +8479,6 @@ dependencies = [ "zksync_object_store", "zksync_protobuf", "zksync_protobuf_build", - "zksync_prover_utils", "zksync_queued_job_processor", "zksync_state", "zksync_storage", @@ -8755,25 +8733,6 @@ dependencies = [ "syn 2.0.38", ] -[[package]] -name = "zksync_prover_utils" -version = "0.1.0" -dependencies = [ - "anyhow", - "async-trait", - "ctrlc", - "futures 0.3.28", - "regex", - "reqwest", - "tokio", - "toml_edit 0.14.4", - "tracing", - "zksync_config", - "zksync_object_store", - "zksync_types", - "zksync_utils", -] - [[package]] name = "zksync_queued_job_processor" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 151921aef57e..cd823972a01c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,7 +32,6 @@ members = [ "core/lib/state", "core/lib/storage", "core/lib/types", - "core/lib/prover_utils", "core/lib/utils", "core/lib/vlog", "core/lib/multivm", diff --git a/core/lib/prover_utils/Cargo.toml b/core/lib/prover_utils/Cargo.toml deleted file mode 100644 index 3afa050ace07..000000000000 --- a/core/lib/prover_utils/Cargo.toml +++ /dev/null @@ -1,26 +0,0 @@ -[package] -name = "zksync_prover_utils" -version = "0.1.0" -edition = "2018" -authors = ["The Matter Labs Team "] -homepage = "https://zksync.io/" -repository = "https://github.com/matter-labs/zksync-era" -license = "MIT OR Apache-2.0" -keywords = ["blockchain", "zksync"] -categories = ["cryptography"] - -[dependencies] -zksync_config = { path = "../../lib/config" } -zksync_utils = { path = "../../lib/utils" } -zksync_types = { path = "../../lib/types" } -zksync_object_store = { path = "../../lib/object_store" } - -anyhow = "1.0" -reqwest = { version = "0.11", features = ["blocking"] } -regex = "1.7.2" -tokio = "1.27.0" -futures = { version = "0.3", features = ["compat"] } -ctrlc = { version = "3.1", features = ["termination"] } -toml_edit = "0.14.4" -async-trait = "0.1" -tracing = "0.1" diff --git a/core/lib/prover_utils/src/gcs_proof_fetcher.rs b/core/lib/prover_utils/src/gcs_proof_fetcher.rs deleted file mode 100644 index 26872701a1fe..000000000000 --- a/core/lib/prover_utils/src/gcs_proof_fetcher.rs +++ /dev/null @@ -1,22 +0,0 @@ -use zksync_object_store::{ObjectStore, ObjectStoreError}; -use zksync_types::{aggregated_operations::L1BatchProofForL1, L1BatchNumber}; - -pub async fn load_wrapped_fri_proofs_for_range( - from: L1BatchNumber, - to: L1BatchNumber, - blob_store: &dyn ObjectStore, -) -> Vec { - let mut proofs = Vec::new(); - for l1_batch_number in from.0..=to.0 { - let l1_batch_number = L1BatchNumber(l1_batch_number); - match blob_store.get(l1_batch_number).await { - Ok(proof) => proofs.push(proof), - Err(ObjectStoreError::KeyNotFound(_)) => (), // do nothing, proof is not ready yet - Err(err) => panic!( - "Failed to load proof for batch {}: {}", - l1_batch_number.0, err - ), - } - } - proofs -} diff --git a/core/lib/prover_utils/src/lib.rs b/core/lib/prover_utils/src/lib.rs deleted file mode 100644 index 0ee42ffee065..000000000000 --- a/core/lib/prover_utils/src/lib.rs +++ /dev/null @@ -1,126 +0,0 @@ -#![allow(clippy::upper_case_acronyms, clippy::derive_partial_eq_without_eq)] - -extern crate core; - -use std::{fs::create_dir_all, io::Cursor, path::Path, time::Duration}; - -use futures::{channel::mpsc, executor::block_on, SinkExt}; - -pub mod gcs_proof_fetcher; -pub mod periodic_job; -pub mod region_fetcher; -pub mod vk_commitment_helper; - -fn download_bytes(key_download_url: &str) -> reqwest::Result> { - tracing::info!("Downloading initial setup from {:?}", key_download_url); - - const DOWNLOAD_TIMEOUT: Duration = Duration::from_secs(120); - let client = reqwest::blocking::Client::builder() - .timeout(DOWNLOAD_TIMEOUT) - .build() - .unwrap(); - - const DOWNLOAD_RETRIES: usize = 5; - let mut retry_count = 0; - - while retry_count < DOWNLOAD_RETRIES { - let bytes = client - .get(key_download_url) - .send() - .and_then(|response| response.bytes().map(|bytes| bytes.to_vec())); - match bytes { - Ok(bytes) => return Ok(bytes), - Err(_) => retry_count += 1, - } - - tracing::warn!("Failed to download keys. Backing off for 5 second"); - std::thread::sleep(Duration::from_secs(5)); - } - - client - .get(key_download_url) - .send() - .and_then(|response| response.bytes().map(|bytes| bytes.to_vec())) -} - -pub fn ensure_initial_setup_keys_present(initial_setup_key_path: &str, key_download_url: &str) { - if Path::new(initial_setup_key_path).exists() { - tracing::info!( - "Initial setup already present at {:?}", - initial_setup_key_path - ); - return; - } - - let bytes = download_bytes(key_download_url).expect("Failed downloading initial setup"); - let initial_setup_key_dir = Path::new(initial_setup_key_path).parent().unwrap(); - create_dir_all(initial_setup_key_dir).unwrap_or_else(|_| { - panic!( - "Failed creating dirs recursively: {:?}", - initial_setup_key_dir - ) - }); - let mut file = std::fs::File::create(initial_setup_key_path) - .expect("Cannot create file for the initial setup"); - let mut content = Cursor::new(bytes); - std::io::copy(&mut content, &mut file).expect("Cannot write the downloaded key to the file"); -} - -pub fn numeric_index_to_circuit_name(circuit_numeric_index: u8) -> Option<&'static str> { - match circuit_numeric_index { - 0 => Some("Scheduler"), - 1 => Some("Node aggregation"), - 2 => Some("Leaf aggregation"), - 3 => Some("Main VM"), - 4 => Some("Decommitts sorter"), - 5 => Some("Code decommitter"), - 6 => Some("Log demuxer"), - 7 => Some("Keccak"), - 8 => Some("SHA256"), - 9 => Some("ECRecover"), - 10 => Some("RAM permutation"), - 11 => Some("Storage sorter"), - 12 => Some("Storage application"), - 13 => Some("Initial writes pubdata rehasher"), - 14 => Some("Repeated writes pubdata rehasher"), - 15 => Some("Events sorter"), - 16 => Some("L1 messages sorter"), - 17 => Some("L1 messages rehasher"), - 18 => Some("L1 messages merklizer"), - _ => None, - } -} - -pub fn circuit_name_to_numeric_index(circuit_name: &str) -> Option { - match circuit_name { - "Scheduler" => Some(0), - "Node aggregation" => Some(1), - "Leaf aggregation" => Some(2), - "Main VM" => Some(3), - "Decommitts sorter" => Some(4), - "Code decommitter" => Some(5), - "Log demuxer" => Some(6), - "Keccak" => Some(7), - "SHA256" => Some(8), - "ECRecover" => Some(9), - "RAM permutation" => Some(10), - "Storage sorter" => Some(11), - "Storage application" => Some(12), - "Initial writes pubdata rehasher" => Some(13), - "Repeated writes pubdata rehasher" => Some(14), - "Events sorter" => Some(15), - "L1 messages sorter" => Some(16), - "L1 messages rehasher" => Some(17), - "L1 messages merklizer" => Some(18), - _ => None, - } -} - -pub fn get_stop_signal_receiver() -> mpsc::Receiver { - let (mut stop_signal_sender, stop_signal_receiver) = mpsc::channel(256); - ctrlc::set_handler(move || { - block_on(stop_signal_sender.send(true)).expect("Ctrl+C signal send"); - }) - .expect("Error setting Ctrl+C handler"); - stop_signal_receiver -} diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 64c279ae0cf5..9a86c4e1a1c9 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -22,7 +22,6 @@ zksync_commitment_utils = { path = "../commitment_utils" } zksync_eth_client = { path = "../eth_client" } zksync_eth_signer = { path = "../eth_signer" } zksync_mempool = { path = "../mempool" } -zksync_prover_utils = { path = "../prover_utils" } zksync_queued_job_processor = { path = "../queued_job_processor" } zksync_circuit_breaker = { path = "../circuit_breaker" } zksync_storage = { path = "../storage" } diff --git a/core/lib/zksync_core/src/eth_sender/aggregator.rs b/core/lib/zksync_core/src/eth_sender/aggregator.rs index 7368aaa0cd20..ebb11752894f 100644 --- a/core/lib/zksync_core/src/eth_sender/aggregator.rs +++ b/core/lib/zksync_core/src/eth_sender/aggregator.rs @@ -1,12 +1,11 @@ use zksync_config::configs::eth_sender::{ProofLoadingMode, ProofSendingMode, SenderConfig}; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::StorageProcessor; -use zksync_object_store::ObjectStore; -use zksync_prover_utils::gcs_proof_fetcher::load_wrapped_fri_proofs_for_range; +use zksync_object_store::{ObjectStore, ObjectStoreError}; use zksync_types::{ aggregated_operations::{ AggregatedActionType, AggregatedOperation, L1BatchCommitOperation, L1BatchExecuteOperation, - L1BatchProofOperation, + L1BatchProofForL1, L1BatchProofOperation, }, commitment::L1BatchWithMetadata, helpers::unix_timestamp_ms, @@ -418,3 +417,23 @@ async fn extract_ready_subrange( .collect(), ) } + +pub async fn load_wrapped_fri_proofs_for_range( + from: L1BatchNumber, + to: L1BatchNumber, + blob_store: &dyn ObjectStore, +) -> Vec { + let mut proofs = Vec::new(); + for l1_batch_number in from.0..=to.0 { + let l1_batch_number = L1BatchNumber(l1_batch_number); + match blob_store.get(l1_batch_number).await { + Ok(proof) => proofs.push(proof), + Err(ObjectStoreError::KeyNotFound(_)) => (), // do nothing, proof is not ready yet + Err(err) => panic!( + "Failed to load proof for batch {}: {}", + l1_batch_number.0, err + ), + } + } + proofs +} diff --git a/core/lib/zksync_core/src/house_keeper/blocks_state_reporter.rs b/core/lib/zksync_core/src/house_keeper/blocks_state_reporter.rs index b52edb963758..695c2008e134 100644 --- a/core/lib/zksync_core/src/house_keeper/blocks_state_reporter.rs +++ b/core/lib/zksync_core/src/house_keeper/blocks_state_reporter.rs @@ -1,9 +1,11 @@ use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; use zksync_utils::time::seconds_since_epoch; -use crate::metrics::{BlockL1Stage, BlockStage, L1StageLatencyLabel, APP_METRICS}; +use crate::{ + house_keeper::periodic_job::PeriodicJob, + metrics::{BlockL1Stage, BlockStage, L1StageLatencyLabel, APP_METRICS}, +}; #[derive(Debug)] pub struct L1BatchMetricsReporter { diff --git a/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_job_retry_manager.rs b/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_job_retry_manager.rs index fc26524e992e..7cf2c231b67a 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_job_retry_manager.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_job_retry_manager.rs @@ -2,7 +2,8 @@ use std::time::Duration; use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct FriProofCompressorJobRetryManager { diff --git a/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_queue_monitor.rs index 73c752b69558..88e1f0f64654 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_queue_monitor.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_proof_compressor_queue_monitor.rs @@ -1,8 +1,9 @@ use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; use zksync_types::proofs::JobCountStatistics; +use crate::house_keeper::periodic_job::PeriodicJob; + const PROOF_COMPRESSOR_SERVICE_NAME: &str = "proof_compressor"; #[derive(Debug)] diff --git a/core/lib/zksync_core/src/house_keeper/fri_prover_job_retry_manager.rs b/core/lib/zksync_core/src/house_keeper/fri_prover_job_retry_manager.rs index fefb7333a675..8ff847a5ca92 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_prover_job_retry_manager.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_prover_job_retry_manager.rs @@ -2,7 +2,8 @@ use std::time::Duration; use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct FriProverJobRetryManager { diff --git a/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs index f0ccf0be7333..3ccf09bdf1e8 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs @@ -1,7 +1,8 @@ use async_trait::async_trait; use zksync_config::configs::fri_prover_group::FriProverGroupConfig; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct FriProverStatsReporter { diff --git a/core/lib/zksync_core/src/house_keeper/fri_scheduler_circuit_queuer.rs b/core/lib/zksync_core/src/house_keeper/fri_scheduler_circuit_queuer.rs index 0adfdb470551..70911339a8fd 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_scheduler_circuit_queuer.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_scheduler_circuit_queuer.rs @@ -1,6 +1,7 @@ use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct SchedulerCircuitQueuer { diff --git a/core/lib/zksync_core/src/house_keeper/fri_witness_generator_jobs_retry_manager.rs b/core/lib/zksync_core/src/house_keeper/fri_witness_generator_jobs_retry_manager.rs index f81cb03dc379..3aa21bdd534d 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_witness_generator_jobs_retry_manager.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_witness_generator_jobs_retry_manager.rs @@ -2,7 +2,8 @@ use std::time::Duration; use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct FriWitnessGeneratorJobRetryManager { diff --git a/core/lib/zksync_core/src/house_keeper/fri_witness_generator_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/fri_witness_generator_queue_monitor.rs index 67f81295b44f..f198d27d97b2 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_witness_generator_queue_monitor.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_witness_generator_queue_monitor.rs @@ -2,9 +2,10 @@ use std::collections::HashMap; use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; use zksync_types::proofs::{AggregationRound, JobCountStatistics}; +use crate::house_keeper::periodic_job::PeriodicJob; + const FRI_WITNESS_GENERATOR_SERVICE_NAME: &str = "fri_witness_generator"; #[derive(Debug)] diff --git a/core/lib/zksync_core/src/house_keeper/mod.rs b/core/lib/zksync_core/src/house_keeper/mod.rs index fbf59642d356..2c029d42e779 100644 --- a/core/lib/zksync_core/src/house_keeper/mod.rs +++ b/core/lib/zksync_core/src/house_keeper/mod.rs @@ -6,4 +6,5 @@ pub mod fri_prover_queue_monitor; pub mod fri_scheduler_circuit_queuer; pub mod fri_witness_generator_jobs_retry_manager; pub mod fri_witness_generator_queue_monitor; +pub mod periodic_job; pub mod waiting_to_queued_fri_witness_job_mover; diff --git a/core/lib/prover_utils/src/periodic_job.rs b/core/lib/zksync_core/src/house_keeper/periodic_job.rs similarity index 97% rename from core/lib/prover_utils/src/periodic_job.rs rename to core/lib/zksync_core/src/house_keeper/periodic_job.rs index e58ff33e7890..3f73a01ce200 100644 --- a/core/lib/prover_utils/src/periodic_job.rs +++ b/core/lib/zksync_core/src/house_keeper/periodic_job.rs @@ -1,6 +1,6 @@ use std::time::Duration; -use anyhow::Context as _; +use anyhow::Context; use async_trait::async_trait; use tokio::time::sleep; diff --git a/core/lib/zksync_core/src/house_keeper/waiting_to_queued_fri_witness_job_mover.rs b/core/lib/zksync_core/src/house_keeper/waiting_to_queued_fri_witness_job_mover.rs index 1292ee3f44fa..df9208f1f451 100644 --- a/core/lib/zksync_core/src/house_keeper/waiting_to_queued_fri_witness_job_mover.rs +++ b/core/lib/zksync_core/src/house_keeper/waiting_to_queued_fri_witness_job_mover.rs @@ -1,6 +1,7 @@ use async_trait::async_trait; use zksync_dal::ConnectionPool; -use zksync_prover_utils::periodic_job::PeriodicJob; + +use crate::house_keeper::periodic_job::PeriodicJob; #[derive(Debug)] pub struct WaitingToQueuedFriWitnessJobMover { diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index b3a3d2e580ee..1964651be3ff 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -31,7 +31,6 @@ use zksync_eth_client::{ }; use zksync_health_check::{CheckHealth, HealthStatus, ReactiveHealthCheck}; use zksync_object_store::{ObjectStore, ObjectStoreFactory}; -use zksync_prover_utils::periodic_job::PeriodicJob; use zksync_queued_job_processor::JobProcessor; use zksync_state::PostgresStorageCaches; use zksync_types::{ @@ -62,6 +61,7 @@ use crate::{ fri_scheduler_circuit_queuer::SchedulerCircuitQueuer, fri_witness_generator_jobs_retry_manager::FriWitnessGeneratorJobRetryManager, fri_witness_generator_queue_monitor::FriWitnessGeneratorStatsReporter, + periodic_job::PeriodicJob, waiting_to_queued_fri_witness_job_mover::WaitingToQueuedFriWitnessJobMover, }, l1_gas_price::{GasAdjusterSingleton, L1GasPriceProvider}, diff --git a/docs/guides/architecture.md b/docs/guides/architecture.md index dbac73fa09a8..e87f4bca7e55 100644 --- a/docs/guides/architecture.md +++ b/docs/guides/architecture.md @@ -62,7 +62,6 @@ This section provides a physical map of folders & files in this repository. - `/multivm`: A wrapper over several versions of VM that have been used by the main node. - `/object_store`: Abstraction for storing blobs outside the main data store. - `/prometheus_exporter`: Prometheus data exporter. - - `/prover_utils`: Utilities related to the proof generation. - `/queued_job_processor`: An abstraction for async job processing - `/state`: A state keeper responsible for handling transaction execution and creating miniblocks and L1 batches. - `/storage`: An encapsulated database interface. diff --git a/prover/Cargo.lock b/prover/Cargo.lock index c5c7d1bc7be9..27ac29283485 100644 --- a/prover/Cargo.lock +++ b/prover/Cargo.lock @@ -6217,13 +6217,13 @@ dependencies = [ "serde_json", "shivini", "structopt", + "toml_edit 0.14.4", "tracing", "vlog", "zkevm_test_harness 1.4.0", "zksync_config", "zksync_env_config", "zksync_prover_fri_types", - "zksync_prover_utils", "zksync_types", ] @@ -7017,6 +7017,7 @@ dependencies = [ "ctrlc", "futures 0.3.29", "prometheus_exporter", + "reqwest", "structopt", "tokio", "tracing", @@ -7029,7 +7030,6 @@ dependencies = [ "zksync_env_config", "zksync_object_store", "zksync_prover_fri_types", - "zksync_prover_utils", "zksync_queued_job_processor", "zksync_types", "zksync_utils", @@ -7080,6 +7080,8 @@ dependencies = [ "futures 0.3.29", "local-ip-address", "prometheus_exporter", + "regex", + "reqwest", "serde", "shivini", "tokio", @@ -7094,7 +7096,6 @@ dependencies = [ "zksync_object_store", "zksync_prover_fri_types", "zksync_prover_fri_utils", - "zksync_prover_utils", "zksync_queued_job_processor", "zksync_types", "zksync_utils", @@ -7138,6 +7139,9 @@ dependencies = [ name = "zksync_prover_fri_utils" version = "0.1.0" dependencies = [ + "anyhow", + "regex", + "reqwest", "serde", "tracing", "vise", @@ -7146,24 +7150,6 @@ dependencies = [ "zksync_object_store", "zksync_prover_fri_types", "zksync_types", -] - -[[package]] -name = "zksync_prover_utils" -version = "0.1.0" -dependencies = [ - "anyhow", - "async-trait", - "ctrlc", - "futures 0.3.29", - "regex", - "reqwest", - "tokio", - "toml_edit 0.14.4", - "tracing", - "zksync_config", - "zksync_object_store", - "zksync_types", "zksync_utils", ] @@ -7309,7 +7295,6 @@ dependencies = [ "zksync_object_store", "zksync_prover_fri_types", "zksync_prover_fri_utils", - "zksync_prover_utils", "zksync_queued_job_processor", "zksync_state", "zksync_system_constants", @@ -7341,7 +7326,6 @@ dependencies = [ "zksync_object_store", "zksync_prover_fri_types", "zksync_prover_fri_utils", - "zksync_prover_utils", "zksync_queued_job_processor", "zksync_types", "zksync_utils", diff --git a/prover/proof_fri_compressor/Cargo.toml b/prover/proof_fri_compressor/Cargo.toml index ee5bd340686b..659f1fb4ceb4 100644 --- a/prover/proof_fri_compressor/Cargo.toml +++ b/prover/proof_fri_compressor/Cargo.toml @@ -19,7 +19,6 @@ zksync_prover_fri_types = { path = "../prover_fri_types" } zksync_queued_job_processor = { path = "../../core/lib/queued_job_processor" } vk_setup_data_generator_server_fri = { path = "../vk_setup_data_generator_server_fri" } vlog = { path = "../../core/lib/vlog" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } zkevm_test_harness = { git = "https://github.com/matter-labs/era-zkevm_test_harness.git", branch = "v1.4.0" } @@ -31,3 +30,4 @@ futures = { version = "0.3", features = ["compat"] } ctrlc = { version = "3.1", features = ["termination"] } async-trait = "0.1" bincode = "1.0" +reqwest = { version = "0.11", features = ["blocking"] } diff --git a/prover/proof_fri_compressor/src/initial_setup_keys.rs b/prover/proof_fri_compressor/src/initial_setup_keys.rs new file mode 100644 index 000000000000..222d2ec69cc5 --- /dev/null +++ b/prover/proof_fri_compressor/src/initial_setup_keys.rs @@ -0,0 +1,59 @@ +use std::{fs::create_dir_all, io::Cursor, path::Path, time::Duration}; + +fn download_initial_setup(key_download_url: &str) -> reqwest::Result> { + tracing::info!("Downloading initial setup from {:?}", key_download_url); + + const DOWNLOAD_TIMEOUT: Duration = Duration::from_secs(120); + let client = reqwest::blocking::Client::builder() + .timeout(DOWNLOAD_TIMEOUT) + .build() + .unwrap(); + + const DOWNLOAD_RETRIES: usize = 5; + let mut retry_count = 0; + + while retry_count < DOWNLOAD_RETRIES { + let bytes = client + .get(key_download_url) + .send() + .and_then(|response| response.bytes().map(|bytes| bytes.to_vec())); + match bytes { + Ok(bytes) => return Ok(bytes), + Err(_) => retry_count += 1, + } + + tracing::warn!("Failed to download keys. Backing off for 5 second"); + std::thread::sleep(Duration::from_secs(5)); + } + + client + .get(key_download_url) + .send() + .and_then(|response| response.bytes().map(|bytes| bytes.to_vec())) +} + +pub fn download_initial_setup_keys_if_not_present( + initial_setup_key_path: &str, + key_download_url: &str, +) { + if Path::new(initial_setup_key_path).exists() { + tracing::info!( + "Initial setup already present at {:?}", + initial_setup_key_path + ); + return; + } + + let bytes = download_initial_setup(key_download_url).expect("Failed downloading initial setup"); + let initial_setup_key_dir = Path::new(initial_setup_key_path).parent().unwrap(); + create_dir_all(initial_setup_key_dir).unwrap_or_else(|_| { + panic!( + "Failed creating dirs recursively: {:?}", + initial_setup_key_dir + ) + }); + let mut file = std::fs::File::create(initial_setup_key_path) + .expect("Cannot create file for the initial setup"); + let mut content = Cursor::new(bytes); + std::io::copy(&mut content, &mut file).expect("Cannot write the downloaded key to the file"); +} diff --git a/prover/proof_fri_compressor/src/main.rs b/prover/proof_fri_compressor/src/main.rs index 90d937b6f309..33aaf3b9162b 100644 --- a/prover/proof_fri_compressor/src/main.rs +++ b/prover/proof_fri_compressor/src/main.rs @@ -11,9 +11,12 @@ use zksync_object_store::ObjectStoreFactory; use zksync_queued_job_processor::JobProcessor; use zksync_utils::wait_for_tasks::wait_for_tasks; -use crate::compressor::ProofCompressor; +use crate::{ + compressor::ProofCompressor, initial_setup_keys::download_initial_setup_keys_if_not_present, +}; mod compressor; +mod initial_setup_keys; mod metrics; #[derive(Debug, StructOpt)] @@ -76,7 +79,7 @@ async fn main() -> anyhow::Result<()> { }) .expect("Error setting Ctrl+C handler"); // Setting handler should always succeed. - zksync_prover_utils::ensure_initial_setup_keys_present( + download_initial_setup_keys_if_not_present( &config.universal_setup_path, &config.universal_setup_download_url, ); diff --git a/prover/prover_fri/Cargo.toml b/prover/prover_fri/Cargo.toml index bfef44fd65b3..dca42c966d27 100644 --- a/prover/prover_fri/Cargo.toml +++ b/prover/prover_fri/Cargo.toml @@ -15,7 +15,6 @@ zksync_env_config = { path = "../../core/lib/env_config" } prometheus_exporter = { path = "../../core/lib/prometheus_exporter" } vlog = { path = "../../core/lib/vlog" } zksync_object_store = { path = "../../core/lib/object_store" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } zksync_queued_job_processor = { path = "../../core/lib/queued_job_processor" } zksync_prover_fri_utils = { path = "../prover_fri_utils" } zksync_prover_fri_types = { path = "../prover_fri_types" } @@ -39,6 +38,8 @@ ctrlc = { version = "3.1", features = ["termination"] } serde = { version = "1.0", features = ["derive"] } async-trait = "0.1" local-ip-address = "0.5.0" +reqwest = { version = "0.11", features = ["blocking"] } +regex = "1.7.2" [features] default = [] diff --git a/prover/prover_fri/src/main.rs b/prover/prover_fri/src/main.rs index 980a499f5b48..06a71266a3da 100644 --- a/prover/prover_fri/src/main.rs +++ b/prover/prover_fri/src/main.rs @@ -17,8 +17,7 @@ use zksync_env_config::{ FromEnv, }; use zksync_object_store::{ObjectStore, ObjectStoreFactory}; -use zksync_prover_fri_utils::get_all_circuit_id_round_tuples_for; -use zksync_prover_utils::region_fetcher::get_zone; +use zksync_prover_fri_utils::{get_all_circuit_id_round_tuples_for, region_fetcher::get_zone}; use zksync_queued_job_processor::JobProcessor; use zksync_types::{ basic_fri_types::CircuitIdRoundTuple, diff --git a/prover/prover_fri_utils/Cargo.toml b/prover/prover_fri_utils/Cargo.toml index 148f68f5c0f8..e2023f92b2dc 100644 --- a/prover/prover_fri_utils/Cargo.toml +++ b/prover/prover_fri_utils/Cargo.toml @@ -13,6 +13,10 @@ zksync_config = { path = "../../core/lib/config" } zksync_types = { path = "../../core/lib/types" } zksync_prover_fri_types = { path = "../prover_fri_types" } zksync_dal = { path = "../../core/lib/dal" } +zksync_utils = { path = "../../core/lib/utils" } tracing = "0.1" serde = { version = "1.0", features = ["derive"] } +reqwest = { version = "0.11", features = ["blocking"] } +regex = "1.7.2" +anyhow = "1.0" diff --git a/prover/prover_fri_utils/src/lib.rs b/prover/prover_fri_utils/src/lib.rs index eee7293b591b..991683b7f9b8 100644 --- a/prover/prover_fri_utils/src/lib.rs +++ b/prover/prover_fri_utils/src/lib.rs @@ -19,6 +19,7 @@ use zksync_types::{ use crate::metrics::{CircuitLabels, PROVER_FRI_UTILS_METRICS}; pub mod metrics; +pub mod region_fetcher; pub mod socket_utils; pub async fn fetch_next_circuit( diff --git a/core/lib/prover_utils/src/region_fetcher.rs b/prover/prover_fri_utils/src/region_fetcher.rs similarity index 98% rename from core/lib/prover_utils/src/region_fetcher.rs rename to prover/prover_fri_utils/src/region_fetcher.rs index fceedd287245..cae211c26cbe 100644 --- a/core/lib/prover_utils/src/region_fetcher.rs +++ b/prover/prover_fri_utils/src/region_fetcher.rs @@ -1,4 +1,4 @@ -use anyhow::Context as _; +use anyhow::Context; use regex::Regex; use reqwest::{ header::{HeaderMap, HeaderValue}, diff --git a/prover/vk_setup_data_generator_server_fri/Cargo.toml b/prover/vk_setup_data_generator_server_fri/Cargo.toml index 59df5f37cf89..6186f121b25b 100644 --- a/prover/vk_setup_data_generator_server_fri/Cargo.toml +++ b/prover/vk_setup_data_generator_server_fri/Cargo.toml @@ -23,7 +23,6 @@ path = "src/commitment_generator.rs" [dependencies] vlog = { path = "../../core/lib/vlog" } zksync_types = { path = "../../core/lib/types" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } zksync_prover_fri_types = { path = "../prover_fri_types" } zkevm_test_harness = { git = "https://github.com/matter-labs/era-zkevm_test_harness.git", branch = "v1.4.0" } @@ -43,6 +42,7 @@ itertools = "0.10.5" bincode = "1" structopt = "0.3.26" once_cell = "1.8.0" +toml_edit = "0.14.4" [dev-dependencies] proptest = "1.2.0" diff --git a/prover/vk_setup_data_generator_server_fri/src/commitment_generator.rs b/prover/vk_setup_data_generator_server_fri/src/commitment_generator.rs index dffa1981d1c8..f2dd53b85283 100644 --- a/prover/vk_setup_data_generator_server_fri/src/commitment_generator.rs +++ b/prover/vk_setup_data_generator_server_fri/src/commitment_generator.rs @@ -1,8 +1,8 @@ -use anyhow::Context as _; -use zksync_prover_utils::vk_commitment_helper::{ - get_toml_formatted_value, read_contract_toml, write_contract_toml, +use anyhow::Context; +use zksync_vk_setup_data_server_fri::{ + commitment_utils::generate_commitments, + vk_commitment_helper::{get_toml_formatted_value, read_contract_toml, write_contract_toml}, }; -use zksync_vk_setup_data_server_fri::commitment_utils::generate_commitments; fn main() -> anyhow::Result<()> { tracing::info!("Starting commitment generation!"); diff --git a/prover/vk_setup_data_generator_server_fri/src/lib.rs b/prover/vk_setup_data_generator_server_fri/src/lib.rs index 19fab9af470b..37d0d827bfa2 100644 --- a/prover/vk_setup_data_generator_server_fri/src/lib.rs +++ b/prover/vk_setup_data_generator_server_fri/src/lib.rs @@ -59,6 +59,7 @@ use {shivini::cs::GpuSetup, std::alloc::Global}; pub mod commitment_utils; pub mod utils; +pub mod vk_commitment_helper; #[derive(Debug, Serialize, Deserialize)] #[serde( diff --git a/core/lib/prover_utils/src/vk_commitment_helper.rs b/prover/vk_setup_data_generator_server_fri/src/vk_commitment_helper.rs similarity index 100% rename from core/lib/prover_utils/src/vk_commitment_helper.rs rename to prover/vk_setup_data_generator_server_fri/src/vk_commitment_helper.rs diff --git a/prover/witness_generator/Cargo.toml b/prover/witness_generator/Cargo.toml index 9d8b10d73fa2..153bac7fa296 100644 --- a/prover/witness_generator/Cargo.toml +++ b/prover/witness_generator/Cargo.toml @@ -26,7 +26,6 @@ zksync_types = { path = "../../core/lib/types" } zksync_state = { path = "../../core/lib/state" } zksync_utils = { path = "../../core/lib/utils" } vk_setup_data_generator_server_fri = { path = "../vk_setup_data_generator_server_fri" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } zksync_prover_fri_types = { path = "../prover_fri_types" } zksync_prover_fri_utils = { path = "../prover_fri_utils" } diff --git a/prover/witness_generator/src/main.rs b/prover/witness_generator/src/main.rs index 3b3b8d460339..50b6f4caff87 100644 --- a/prover/witness_generator/src/main.rs +++ b/prover/witness_generator/src/main.rs @@ -3,6 +3,7 @@ use std::time::Instant; use anyhow::{anyhow, Context as _}; +use futures::{channel::mpsc, executor::block_on, SinkExt}; use prometheus_exporter::PrometheusExporterConfig; use structopt::StructOpt; use tokio::sync::watch; @@ -13,7 +14,6 @@ use zksync_config::{ use zksync_dal::ConnectionPool; use zksync_env_config::{object_store::ProverObjectStoreConfig, FromEnv}; use zksync_object_store::ObjectStoreFactory; -use zksync_prover_utils::get_stop_signal_receiver; use zksync_queued_job_processor::JobProcessor; use zksync_types::{proofs::AggregationRound, web3::futures::StreamExt}; use zksync_utils::wait_for_tasks::wait_for_tasks; @@ -229,7 +229,11 @@ async fn main() -> anyhow::Result<()> { SERVER_METRICS.init_latency[&(*round).into()].set(started_at.elapsed()); } - let mut stop_signal_receiver = get_stop_signal_receiver(); + let (mut stop_signal_sender, mut stop_signal_receiver) = mpsc::channel(256); + ctrlc::set_handler(move || { + block_on(stop_signal_sender.send(true)).expect("Ctrl+C signal send"); + }) + .expect("Error setting Ctrl+C handler"); let graceful_shutdown = None::>; let tasks_allowed_to_finish = true; tokio::select! { diff --git a/prover/witness_vector_generator/Cargo.toml b/prover/witness_vector_generator/Cargo.toml index 3f5a6ac55d9e..523514af207f 100644 --- a/prover/witness_vector_generator/Cargo.toml +++ b/prover/witness_vector_generator/Cargo.toml @@ -17,7 +17,6 @@ zksync_prover_fri_utils = { path = "../prover_fri_utils" } zksync_utils = { path = "../../core/lib/utils" } prometheus_exporter = { path = "../../core/lib/prometheus_exporter" } zksync_prover_fri_types = { path = "../prover_fri_types" } -zksync_prover_utils = { path = "../../core/lib/prover_utils" } zksync_queued_job_processor = { path = "../../core/lib/queued_job_processor" } vk_setup_data_generator_server_fri = { path = "../vk_setup_data_generator_server_fri" } vlog = { path = "../../core/lib/vlog" } diff --git a/prover/witness_vector_generator/src/main.rs b/prover/witness_vector_generator/src/main.rs index 552644782c0c..a880cf001829 100644 --- a/prover/witness_vector_generator/src/main.rs +++ b/prover/witness_vector_generator/src/main.rs @@ -11,8 +11,7 @@ use zksync_config::configs::{ use zksync_dal::ConnectionPool; use zksync_env_config::{object_store::ProverObjectStoreConfig, FromEnv}; use zksync_object_store::ObjectStoreFactory; -use zksync_prover_fri_utils::get_all_circuit_id_round_tuples_for; -use zksync_prover_utils::region_fetcher::get_zone; +use zksync_prover_fri_utils::{get_all_circuit_id_round_tuples_for, region_fetcher::get_zone}; use zksync_queued_job_processor::JobProcessor; use zksync_utils::wait_for_tasks::wait_for_tasks; use zksync_vk_setup_data_server_fri::commitment_utils::get_cached_commitments; From 1d1b682015ea7e03388b72d9770dbfcac57bf412 Mon Sep 17 00:00:00 2001 From: Igor Aleksanov Date: Fri, 5 Jan 2024 15:59:44 +0400 Subject: [PATCH 17/49] refactor(object_store): Wrap ObjectStore in Arc instead of Box (#820) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Changes `Box` to `Arc`. ## Why ❔ `ObjectStore` is meant to be a point of customization in ZK Stack, but with the current approach the only way to get it is via `ObjectStoreFactory`, and it yields objects packed in `Box`, e.g. each instance is unique. In order to make it customizable, `ObjectStore` should be universally shareable (to allow it to be created once and then copied), and this is the first step toward that. Later `ObjectStoreFactory` type will be removed completely to grant full control regarding the initial type to the initialization code. ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/snapshots_creator/src/creator.rs | 4 +++- core/bin/snapshots_creator/src/tests.rs | 7 +++++-- core/lib/object_store/src/raw.rs | 14 +++++++------- .../src/basic_witness_input_producer/mod.rs | 2 +- core/lib/zksync_core/src/eth_sender/aggregator.rs | 6 ++++-- core/lib/zksync_core/src/lib.rs | 4 ++-- .../lib/zksync_core/src/metadata_calculator/mod.rs | 5 +++-- .../zksync_core/src/metadata_calculator/tests.rs | 6 +++--- .../zksync_core/src/metadata_calculator/updater.rs | 6 +++--- core/lib/zksync_core/src/proof_data_handler/mod.rs | 4 ++-- .../src/proof_data_handler/request_processor.rs | 4 ++-- .../lib/zksync_core/src/state_keeper/io/mempool.rs | 4 ++-- core/lib/zksync_core/src/state_keeper/mod.rs | 2 +- prover/proof_fri_compressor/src/compressor.rs | 6 +++--- prover/prover_fri/src/gpu_prover_job_processor.rs | 8 ++++---- prover/prover_fri/src/main.rs | 6 +++--- prover/prover_fri/src/prover_job_processor.rs | 8 ++++---- prover/prover_fri_gateway/src/api_data_fetcher.rs | 4 ++-- prover/witness_generator/src/basic_circuits.rs | 6 +++--- prover/witness_generator/src/leaf_aggregation.rs | 4 ++-- prover/witness_generator/src/node_aggregation.rs | 4 ++-- prover/witness_generator/src/scheduler.rs | 4 ++-- prover/witness_vector_generator/src/generator.rs | 9 ++++++--- 23 files changed, 69 insertions(+), 58 deletions(-) diff --git a/core/bin/snapshots_creator/src/creator.rs b/core/bin/snapshots_creator/src/creator.rs index da2ac930dbd4..51a14ce2ccae 100644 --- a/core/bin/snapshots_creator/src/creator.rs +++ b/core/bin/snapshots_creator/src/creator.rs @@ -1,5 +1,7 @@ //! [`SnapshotCreator`] and tightly related types. +use std::sync::Arc; + use anyhow::Context as _; use tokio::sync::Semaphore; use zksync_config::SnapshotsCreatorConfig; @@ -61,7 +63,7 @@ impl SnapshotProgress { /// Creator of a single storage snapshot. #[derive(Debug)] pub(crate) struct SnapshotCreator { - pub blob_store: Box, + pub blob_store: Arc, pub master_pool: ConnectionPool, pub replica_pool: ConnectionPool, #[cfg(test)] diff --git a/core/bin/snapshots_creator/src/tests.rs b/core/bin/snapshots_creator/src/tests.rs index c0e8dd0cbc28..db8501c7d796 100644 --- a/core/bin/snapshots_creator/src/tests.rs +++ b/core/bin/snapshots_creator/src/tests.rs @@ -3,7 +3,10 @@ use std::{ collections::{HashMap, HashSet}, fmt, - sync::atomic::{AtomicUsize, Ordering}, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, + }, }; use rand::{thread_rng, Rng}; @@ -58,7 +61,7 @@ impl HandleEvent for TestEventListener { } impl SnapshotCreator { - fn for_tests(blob_store: Box, pool: ConnectionPool) -> Self { + fn for_tests(blob_store: Arc, pool: ConnectionPool) -> Self { Self { blob_store, master_pool: pool.clone(), diff --git a/core/lib/object_store/src/raw.rs b/core/lib/object_store/src/raw.rs index 72e582deeb2b..764809764dac 100644 --- a/core/lib/object_store/src/raw.rs +++ b/core/lib/object_store/src/raw.rs @@ -88,7 +88,7 @@ impl error::Error for ObjectStoreError { /// /// [`StoredObject`]: crate::StoredObject #[async_trait] -pub trait ObjectStore: fmt::Debug + Send + Sync { +pub trait ObjectStore: 'static + fmt::Debug + Send + Sync { /// Fetches the value for the given key from the given bucket if it exists. /// /// # Errors @@ -178,14 +178,14 @@ impl ObjectStoreFactory { } /// Creates an [`ObjectStore`]. - pub async fn create_store(&self) -> Box { + pub async fn create_store(&self) -> Arc { match &self.origin { ObjectStoreOrigin::Config(config) => Self::create_from_config(config).await, - ObjectStoreOrigin::Mock(store) => Box::new(Arc::clone(store)), + ObjectStoreOrigin::Mock(store) => Arc::new(Arc::clone(store)), } } - async fn create_from_config(config: &ObjectStoreConfig) -> Box { + async fn create_from_config(config: &ObjectStoreConfig) -> Arc { let gcs_credential_file_path = match config.mode { ObjectStoreMode::GCSWithCredentialFile => Some(config.gcs_credential_file_path.clone()), _ => None, @@ -201,7 +201,7 @@ impl ObjectStoreFactory { config.max_retries, ) .await; - Box::new(store) + Arc::new(store) } ObjectStoreMode::GCSWithCredentialFile => { tracing::trace!("Initialized GoogleCloudStorage Object store with credential file"); @@ -211,12 +211,12 @@ impl ObjectStoreFactory { config.max_retries, ) .await; - Box::new(store) + Arc::new(store) } ObjectStoreMode::FileBacked => { tracing::trace!("Initialized FileBacked Object store"); let store = FileBackedObjectStore::new(config.file_backed_base_path.clone()).await; - Box::new(store) + Arc::new(store) } } } diff --git a/core/lib/zksync_core/src/basic_witness_input_producer/mod.rs b/core/lib/zksync_core/src/basic_witness_input_producer/mod.rs index 0367830f9c6f..e91ccc4864eb 100644 --- a/core/lib/zksync_core/src/basic_witness_input_producer/mod.rs +++ b/core/lib/zksync_core/src/basic_witness_input_producer/mod.rs @@ -37,7 +37,7 @@ impl BasicWitnessInputProducer { ) -> anyhow::Result { Ok(BasicWitnessInputProducer { connection_pool, - object_store: store_factory.create_store().await.into(), + object_store: store_factory.create_store().await, l2_chain_id, }) } diff --git a/core/lib/zksync_core/src/eth_sender/aggregator.rs b/core/lib/zksync_core/src/eth_sender/aggregator.rs index ebb11752894f..a043b871b1e8 100644 --- a/core/lib/zksync_core/src/eth_sender/aggregator.rs +++ b/core/lib/zksync_core/src/eth_sender/aggregator.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use zksync_config::configs::eth_sender::{ProofLoadingMode, ProofSendingMode, SenderConfig}; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::StorageProcessor; @@ -24,11 +26,11 @@ pub struct Aggregator { proof_criteria: Vec>, execute_criteria: Vec>, config: SenderConfig, - blob_store: Box, + blob_store: Arc, } impl Aggregator { - pub fn new(config: SenderConfig, blob_store: Box) -> Self { + pub fn new(config: SenderConfig, blob_store: Arc) -> Self { Self { commit_criteria: vec![ Box::from(NumberCriterion { diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 1964651be3ff..fb3b8bd51955 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -674,7 +674,7 @@ async fn add_state_keeper_to_task_futures, - object_store: Box, + object_store: Arc, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { let fair_l2_gas_price = state_keeper_config.fair_l2_gas_price; @@ -793,7 +793,7 @@ async fn run_tree( merkle_tree_config: &MerkleTreeConfig, api_config: Option<&MerkleTreeApiConfig>, operation_manager: &OperationsManagerConfig, - object_store: Option>, + object_store: Option>, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { let started_at = Instant::now(); diff --git a/core/lib/zksync_core/src/metadata_calculator/mod.rs b/core/lib/zksync_core/src/metadata_calculator/mod.rs index 0244094be665..c510eb68fb1d 100644 --- a/core/lib/zksync_core/src/metadata_calculator/mod.rs +++ b/core/lib/zksync_core/src/metadata_calculator/mod.rs @@ -3,6 +3,7 @@ use std::{ future::{self, Future}, + sync::Arc, time::Duration, }; @@ -81,7 +82,7 @@ impl MetadataCalculatorConfig { pub struct MetadataCalculator { tree: GenericAsyncTree, tree_reader: watch::Sender>, - object_store: Option>, + object_store: Option>, delayer: Delayer, health_updater: HealthUpdater, max_l1_batches_per_iter: usize, @@ -91,7 +92,7 @@ impl MetadataCalculator { /// Creates a calculator with the specified `config`. pub async fn new( config: MetadataCalculatorConfig, - object_store: Option>, + object_store: Option>, ) -> Self { assert!( config.max_l1_batches_per_iter > 0, diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index ba8c18daf297..eb9a5693a4a4 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -1,6 +1,6 @@ //! Tests for the metadata calculator component life cycle. -use std::{future::Future, ops, panic, path::Path, time::Duration}; +use std::{future::Future, ops, panic, path::Path, sync::Arc, time::Duration}; use assert_matches::assert_matches; use itertools::Itertools; @@ -360,7 +360,7 @@ async fn postgres_backup_recovery_with_excluded_metadata() { pub(crate) async fn setup_calculator( db_path: &Path, pool: &ConnectionPool, -) -> (MetadataCalculator, Box) { +) -> (MetadataCalculator, Arc) { let store_factory = ObjectStoreFactory::mock(); let store = store_factory.create_store().await; let (merkle_tree_config, operation_manager) = create_config(db_path, MerkleTreeMode::Full); @@ -395,7 +395,7 @@ async fn setup_calculator_with_options( merkle_tree_config: &MerkleTreeConfig, operation_config: &OperationsManagerConfig, pool: &ConnectionPool, - object_store: Option>, + object_store: Option>, ) -> MetadataCalculator { let calculator_config = MetadataCalculatorConfig::for_main_node(merkle_tree_config, operation_config); diff --git a/core/lib/zksync_core/src/metadata_calculator/updater.rs b/core/lib/zksync_core/src/metadata_calculator/updater.rs index 1bed4cbbf080..f3540f25fe94 100644 --- a/core/lib/zksync_core/src/metadata_calculator/updater.rs +++ b/core/lib/zksync_core/src/metadata_calculator/updater.rs @@ -1,6 +1,6 @@ //! Tree updater trait and its implementations. -use std::{ops, time::Instant}; +use std::{ops, sync::Arc, time::Instant}; use anyhow::Context as _; use futures::{future, FutureExt}; @@ -24,14 +24,14 @@ use crate::utils::wait_for_l1_batch; pub(super) struct TreeUpdater { tree: AsyncTree, max_l1_batches_per_iter: usize, - object_store: Option>, + object_store: Option>, } impl TreeUpdater { pub fn new( tree: AsyncTree, max_l1_batches_per_iter: usize, - object_store: Option>, + object_store: Option>, ) -> Self { Self { tree, diff --git a/core/lib/zksync_core/src/proof_data_handler/mod.rs b/core/lib/zksync_core/src/proof_data_handler/mod.rs index f1227d8298c9..7a5b8bc69b32 100644 --- a/core/lib/zksync_core/src/proof_data_handler/mod.rs +++ b/core/lib/zksync_core/src/proof_data_handler/mod.rs @@ -1,4 +1,4 @@ -use std::net::SocketAddr; +use std::{net::SocketAddr, sync::Arc}; use anyhow::Context as _; use axum::{extract::Path, routing::post, Json, Router}; @@ -34,7 +34,7 @@ fn fri_l1_verifier_config(contracts_config: &ContractsConfig) -> L1VerifierConfi pub(crate) async fn run_server( config: ProofDataHandlerConfig, contracts_config: ContractsConfig, - blob_store: Box, + blob_store: Arc, pool: ConnectionPool, mut stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { diff --git a/core/lib/zksync_core/src/proof_data_handler/request_processor.rs b/core/lib/zksync_core/src/proof_data_handler/request_processor.rs index 5a3302ee926d..bc9873d99ed5 100644 --- a/core/lib/zksync_core/src/proof_data_handler/request_processor.rs +++ b/core/lib/zksync_core/src/proof_data_handler/request_processor.rs @@ -65,13 +65,13 @@ impl IntoResponse for RequestProcessorError { impl RequestProcessor { pub(crate) fn new( - blob_store: Box, + blob_store: Arc, pool: ConnectionPool, config: ProofDataHandlerConfig, l1_verifier_config: Option, ) -> Self { Self { - blob_store: Arc::from(blob_store), + blob_store, pool, config, l1_verifier_config, diff --git a/core/lib/zksync_core/src/state_keeper/io/mempool.rs b/core/lib/zksync_core/src/state_keeper/io/mempool.rs index 78a71a6a7130..45484f645b5e 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mempool.rs +++ b/core/lib/zksync_core/src/state_keeper/io/mempool.rs @@ -46,7 +46,7 @@ use crate::{ pub(crate) struct MempoolIO { mempool: MempoolGuard, pool: ConnectionPool, - object_store: Box, + object_store: Arc, timeout_sealer: TimeoutSealer, filter: L2TxFilter, current_miniblock_number: MiniblockNumber, @@ -404,7 +404,7 @@ impl MempoolIO { #[allow(clippy::too_many_arguments)] pub(in crate::state_keeper) async fn new( mempool: MempoolGuard, - object_store: Box, + object_store: Arc, miniblock_sealer_handle: MiniblockSealerHandle, l1_gas_price_provider: Arc, pool: ConnectionPool, diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index 0a146c51a5e3..a17d169bc6ea 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -42,7 +42,7 @@ pub(crate) async fn create_state_keeper( mempool: MempoolGuard, l1_gas_price_provider: Arc, miniblock_sealer_handle: MiniblockSealerHandle, - object_store: Box, + object_store: Arc, stop_receiver: watch::Receiver, ) -> ZkSyncStateKeeper { let batch_executor_base = MainBatchExecutorBuilder::new( diff --git a/prover/proof_fri_compressor/src/compressor.rs b/prover/proof_fri_compressor/src/compressor.rs index b4346305b9f9..3b21ef697bca 100644 --- a/prover/proof_fri_compressor/src/compressor.rs +++ b/prover/proof_fri_compressor/src/compressor.rs @@ -1,4 +1,4 @@ -use std::time::Instant; +use std::{sync::Arc, time::Instant}; use anyhow::Context as _; use async_trait::async_trait; @@ -33,7 +33,7 @@ use zksync_vk_setup_data_server_fri::{get_recursive_layer_vk_for_circuit_type, g use crate::metrics::METRICS; pub struct ProofCompressor { - blob_store: Box, + blob_store: Arc, pool: ConnectionPool, compression_mode: u8, verify_wrapper_proof: bool, @@ -42,7 +42,7 @@ pub struct ProofCompressor { impl ProofCompressor { pub fn new( - blob_store: Box, + blob_store: Arc, pool: ConnectionPool, compression_mode: u8, verify_wrapper_proof: bool, diff --git a/prover/prover_fri/src/gpu_prover_job_processor.rs b/prover/prover_fri/src/gpu_prover_job_processor.rs index 5e576bc114bd..f880a296af5e 100644 --- a/prover/prover_fri/src/gpu_prover_job_processor.rs +++ b/prover/prover_fri/src/gpu_prover_job_processor.rs @@ -49,8 +49,8 @@ pub mod gpu_prover { #[allow(dead_code)] pub struct Prover { - blob_store: Box, - public_blob_store: Option>, + blob_store: Arc, + public_blob_store: Option>, config: Arc, prover_connection_pool: ConnectionPool, setup_load_mode: SetupLoadMode, @@ -66,8 +66,8 @@ pub mod gpu_prover { impl Prover { #[allow(dead_code)] pub fn new( - blob_store: Box, - public_blob_store: Option>, + blob_store: Arc, + public_blob_store: Option>, config: FriProverConfig, prover_connection_pool: ConnectionPool, setup_load_mode: SetupLoadMode, diff --git a/prover/prover_fri/src/main.rs b/prover/prover_fri/src/main.rs index 06a71266a3da..ab2dfa30a9a0 100644 --- a/prover/prover_fri/src/main.rs +++ b/prover/prover_fri/src/main.rs @@ -1,5 +1,5 @@ #![feature(generic_const_exprs)] -use std::future::Future; +use std::{future::Future, sync::Arc}; use anyhow::Context as _; use local_ip_address::local_ip; @@ -170,7 +170,7 @@ async fn get_prover_tasks( prover_config: FriProverConfig, stop_receiver: Receiver, store_factory: ObjectStoreFactory, - public_blob_store: Option>, + public_blob_store: Option>, pool: ConnectionPool, circuit_ids_for_round_to_be_proven: Vec, ) -> anyhow::Result>>> { @@ -204,7 +204,7 @@ async fn get_prover_tasks( prover_config: FriProverConfig, stop_receiver: Receiver, store_factory: ObjectStoreFactory, - public_blob_store: Option>, + public_blob_store: Option>, pool: ConnectionPool, circuit_ids_for_round_to_be_proven: Vec, ) -> anyhow::Result>>> { diff --git a/prover/prover_fri/src/prover_job_processor.rs b/prover/prover_fri/src/prover_job_processor.rs index dbe4bee0c86e..40275b681b70 100644 --- a/prover/prover_fri/src/prover_job_processor.rs +++ b/prover/prover_fri/src/prover_job_processor.rs @@ -43,8 +43,8 @@ pub enum SetupLoadMode { } pub struct Prover { - blob_store: Box, - public_blob_store: Option>, + blob_store: Arc, + public_blob_store: Option>, config: Arc, prover_connection_pool: ConnectionPool, setup_load_mode: SetupLoadMode, @@ -57,8 +57,8 @@ pub struct Prover { impl Prover { #[allow(dead_code)] pub fn new( - blob_store: Box, - public_blob_store: Option>, + blob_store: Arc, + public_blob_store: Option>, config: FriProverConfig, prover_connection_pool: ConnectionPool, setup_load_mode: SetupLoadMode, diff --git a/prover/prover_fri_gateway/src/api_data_fetcher.rs b/prover/prover_fri_gateway/src/api_data_fetcher.rs index f56a9af4cc8f..9e0277d1ea89 100644 --- a/prover/prover_fri_gateway/src/api_data_fetcher.rs +++ b/prover/prover_fri_gateway/src/api_data_fetcher.rs @@ -1,4 +1,4 @@ -use std::time::Duration; +use std::{sync::Arc, time::Duration}; use async_trait::async_trait; use reqwest::Client; @@ -16,7 +16,7 @@ pub(crate) const PROOF_GENERATION_DATA_PATH: &str = "/proof_generation_data"; pub(crate) const SUBMIT_PROOF_PATH: &str = "/submit_proof"; pub(crate) struct PeriodicApiStruct { - pub(crate) blob_store: Box, + pub(crate) blob_store: Arc, pub(crate) pool: ConnectionPool, pub(crate) api_url: String, pub(crate) poll_duration: Duration, diff --git a/prover/witness_generator/src/basic_circuits.rs b/prover/witness_generator/src/basic_circuits.rs index b883c4bb35aa..d0909d79dbff 100644 --- a/prover/witness_generator/src/basic_circuits.rs +++ b/prover/witness_generator/src/basic_circuits.rs @@ -88,7 +88,7 @@ pub struct BasicWitnessGeneratorJob { pub struct BasicWitnessGenerator { config: Arc, object_store: Arc, - public_blob_store: Option>, + public_blob_store: Option>, connection_pool: ConnectionPool, prover_connection_pool: ConnectionPool, protocol_versions: Vec, @@ -98,14 +98,14 @@ impl BasicWitnessGenerator { pub async fn new( config: FriWitnessGeneratorConfig, store_factory: &ObjectStoreFactory, - public_blob_store: Option>, + public_blob_store: Option>, connection_pool: ConnectionPool, prover_connection_pool: ConnectionPool, protocol_versions: Vec, ) -> Self { Self { config: Arc::new(config), - object_store: store_factory.create_store().await.into(), + object_store: store_factory.create_store().await, public_blob_store, connection_pool, prover_connection_pool, diff --git a/prover/witness_generator/src/leaf_aggregation.rs b/prover/witness_generator/src/leaf_aggregation.rs index f190aeb21645..eb28936085fc 100644 --- a/prover/witness_generator/src/leaf_aggregation.rs +++ b/prover/witness_generator/src/leaf_aggregation.rs @@ -1,4 +1,4 @@ -use std::time::Instant; +use std::{sync::Arc, time::Instant}; use anyhow::Context as _; use async_trait::async_trait; @@ -72,7 +72,7 @@ pub struct LeafAggregationWitnessGeneratorJob { #[derive(Debug)] pub struct LeafAggregationWitnessGenerator { config: FriWitnessGeneratorConfig, - object_store: Box, + object_store: Arc, prover_connection_pool: ConnectionPool, protocol_versions: Vec, } diff --git a/prover/witness_generator/src/node_aggregation.rs b/prover/witness_generator/src/node_aggregation.rs index be9e5d0d6225..5f817dd88865 100644 --- a/prover/witness_generator/src/node_aggregation.rs +++ b/prover/witness_generator/src/node_aggregation.rs @@ -1,4 +1,4 @@ -use std::time::Instant; +use std::{sync::Arc, time::Instant}; use anyhow::Context as _; use async_trait::async_trait; @@ -74,7 +74,7 @@ pub struct NodeAggregationWitnessGeneratorJob { #[derive(Debug)] pub struct NodeAggregationWitnessGenerator { config: FriWitnessGeneratorConfig, - object_store: Box, + object_store: Arc, prover_connection_pool: ConnectionPool, protocol_versions: Vec, } diff --git a/prover/witness_generator/src/scheduler.rs b/prover/witness_generator/src/scheduler.rs index 921ba68f4024..6571c6e63c4a 100644 --- a/prover/witness_generator/src/scheduler.rs +++ b/prover/witness_generator/src/scheduler.rs @@ -1,4 +1,4 @@ -use std::{convert::TryInto, time::Instant}; +use std::{convert::TryInto, sync::Arc, time::Instant}; use anyhow::Context as _; use async_trait::async_trait; @@ -51,7 +51,7 @@ pub struct SchedulerWitnessGeneratorJob { #[derive(Debug)] pub struct SchedulerWitnessGenerator { config: FriWitnessGeneratorConfig, - object_store: Box, + object_store: Arc, prover_connection_pool: ConnectionPool, protocol_versions: Vec, } diff --git a/prover/witness_vector_generator/src/generator.rs b/prover/witness_vector_generator/src/generator.rs index 5dfb693af8f8..b51e164042a8 100644 --- a/prover/witness_vector_generator/src/generator.rs +++ b/prover/witness_vector_generator/src/generator.rs @@ -1,4 +1,7 @@ -use std::time::{Duration, Instant}; +use std::{ + sync::Arc, + time::{Duration, Instant}, +}; use anyhow::Context as _; use async_trait::async_trait; @@ -24,7 +27,7 @@ use zksync_vk_setup_data_server_fri::get_finalization_hints; use crate::metrics::METRICS; pub struct WitnessVectorGenerator { - blob_store: Box, + blob_store: Arc, pool: ConnectionPool, circuit_ids_for_round_to_be_proven: Vec, zone: String, @@ -35,7 +38,7 @@ pub struct WitnessVectorGenerator { impl WitnessVectorGenerator { pub fn new( - blob_store: Box, + blob_store: Arc, prover_connection_pool: ConnectionPool, circuit_ids_for_round_to_be_proven: Vec, zone: String, From 011a58ae7d0a22c74e4227db588d14f7998006c0 Mon Sep 17 00:00:00 2001 From: zksync-era-bot <147085853+zksync-era-bot@users.noreply.github.com> Date: Fri, 5 Jan 2024 13:21:15 +0100 Subject: [PATCH 18/49] chore(main): release prover 10.1.0 (#743) :robot: I have created a release *beep* *boop* --- ## [10.1.0](https://github.com/matter-labs/zksync-era/compare/prover-v10.0.2...prover-v10.1.0) (2024-01-05) ### Features * **prover:** Remove circuit-synthesizer ([#801](https://github.com/matter-labs/zksync-era/issues/801)) ([1426b1b](https://github.com/matter-labs/zksync-era/commit/1426b1ba3c8b700e0531087b781ced0756c12e3c)) * **prover:** Remove old prover ([#810](https://github.com/matter-labs/zksync-era/issues/810)) ([8be1925](https://github.com/matter-labs/zksync-era/commit/8be1925b18dcbf268eb03b8ea5f07adfd5330876)) ### Bug Fixes * **prover:** increase DB polling interval for witness vector generators ([#697](https://github.com/matter-labs/zksync-era/issues/697)) ([94579cc](https://github.com/matter-labs/zksync-era/commit/94579cc524514cb867843336cd9787db1b6b99d3)) * **prover:** Remove prover-utils from core ([#819](https://github.com/matter-labs/zksync-era/issues/819)) ([2ceb911](https://github.com/matter-labs/zksync-era/commit/2ceb9114659f4c4583c87b1bbc8ee230eb1c44db)) --- This PR was generated with [Release Please](https://github.com/googleapis/release-please). See [documentation](https://github.com/googleapis/release-please#release-please). --- .github/release-please/manifest.json | 2 +- prover/CHANGELOG.md | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/.github/release-please/manifest.json b/.github/release-please/manifest.json index d75c162b8d51..21b427ed9b36 100644 --- a/.github/release-please/manifest.json +++ b/.github/release-please/manifest.json @@ -1,5 +1,5 @@ { "sdk/zksync-rs": "0.4.0", "core": "18.13.0", - "prover": "10.0.2" + "prover": "10.1.0" } diff --git a/prover/CHANGELOG.md b/prover/CHANGELOG.md index d574826f6305..d0975e59d7d6 100644 --- a/prover/CHANGELOG.md +++ b/prover/CHANGELOG.md @@ -1,5 +1,19 @@ # Changelog +## [10.1.0](https://github.com/matter-labs/zksync-era/compare/prover-v10.0.2...prover-v10.1.0) (2024-01-05) + + +### Features + +* **prover:** Remove circuit-synthesizer ([#801](https://github.com/matter-labs/zksync-era/issues/801)) ([1426b1b](https://github.com/matter-labs/zksync-era/commit/1426b1ba3c8b700e0531087b781ced0756c12e3c)) +* **prover:** Remove old prover ([#810](https://github.com/matter-labs/zksync-era/issues/810)) ([8be1925](https://github.com/matter-labs/zksync-era/commit/8be1925b18dcbf268eb03b8ea5f07adfd5330876)) + + +### Bug Fixes + +* **prover:** increase DB polling interval for witness vector generators ([#697](https://github.com/matter-labs/zksync-era/issues/697)) ([94579cc](https://github.com/matter-labs/zksync-era/commit/94579cc524514cb867843336cd9787db1b6b99d3)) +* **prover:** Remove prover-utils from core ([#819](https://github.com/matter-labs/zksync-era/issues/819)) ([2ceb911](https://github.com/matter-labs/zksync-era/commit/2ceb9114659f4c4583c87b1bbc8ee230eb1c44db)) + ## [10.0.2](https://github.com/matter-labs/zksync-era/compare/prover-v10.0.1...prover-v10.0.2) (2023-12-21) From 3f2858e7a4f4bee17f3241d4b5c2fa2c3f8ecf1b Mon Sep 17 00:00:00 2001 From: zksync-era-bot <147085853+zksync-era-bot@users.noreply.github.com> Date: Fri, 5 Jan 2024 13:33:43 +0100 Subject: [PATCH 19/49] chore(main): release core 19.0.0 (#802) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit :robot: I have created a release *beep* *boop* --- ## [19.0.0](https://github.com/matter-labs/zksync-era/compare/core-v18.13.0...core-v19.0.0) (2024-01-05) ### ⚠ BREAKING CHANGES * **vm:** Release v19 - remove allowlist ([#747](https://github.com/matter-labs/zksync-era/issues/747)) ### Features * **en:** Make consistency checker work with pruned data ([#742](https://github.com/matter-labs/zksync-era/issues/742)) ([ae6e18e](https://github.com/matter-labs/zksync-era/commit/ae6e18e5412cadefbc03307a476d6b96c41f04e1)) * **eth_sender:** Remove generic bounds on L1TxParamsProvider in EthSender ([#799](https://github.com/matter-labs/zksync-era/issues/799)) ([29a4f52](https://github.com/matter-labs/zksync-era/commit/29a4f5299c95e0b338010a6baf83f196ece3a530)) * **merkle tree:** Finalize metadata calculator snapshot recovery logic ([#798](https://github.com/matter-labs/zksync-era/issues/798)) ([c83db35](https://github.com/matter-labs/zksync-era/commit/c83db35f0929a412bc4d89fbee1448d32c54a83f)) * **prover:** Remove circuit-synthesizer ([#801](https://github.com/matter-labs/zksync-era/issues/801)) ([1426b1b](https://github.com/matter-labs/zksync-era/commit/1426b1ba3c8b700e0531087b781ced0756c12e3c)) * **prover:** Remove old prover ([#810](https://github.com/matter-labs/zksync-era/issues/810)) ([8be1925](https://github.com/matter-labs/zksync-era/commit/8be1925b18dcbf268eb03b8ea5f07adfd5330876)) * **snapshot creator:** Make snapshot creator fault-tolerant ([#691](https://github.com/matter-labs/zksync-era/issues/691)) ([286c7d1](https://github.com/matter-labs/zksync-era/commit/286c7d15a623604e01effa7119de3362f0fb4eb9)) * **vm:** Add boojum integration folder ([#805](https://github.com/matter-labs/zksync-era/issues/805)) ([4071e90](https://github.com/matter-labs/zksync-era/commit/4071e90578e0fc8c027a4d2a30d09d96db942b4f)) * **vm:** Make utils version-dependent ([#809](https://github.com/matter-labs/zksync-era/issues/809)) ([e5fbcb5](https://github.com/matter-labs/zksync-era/commit/e5fbcb5dfc2a7d2582f40a481c861fb2f4dd5fb0)) * **vm:** Release v19 - remove allowlist ([#747](https://github.com/matter-labs/zksync-era/issues/747)) ([0e2bc56](https://github.com/matter-labs/zksync-era/commit/0e2bc561b9642b854718adcc86087a3e9762cf5d)) * **vm:** Separate boojum integration vm ([#806](https://github.com/matter-labs/zksync-era/issues/806)) ([61712a6](https://github.com/matter-labs/zksync-era/commit/61712a636f69be70d75719c04f364d679ef624e0)) ### Bug Fixes * **db:** Fix parsing statement timeout from env ([#818](https://github.com/matter-labs/zksync-era/issues/818)) ([3f663ec](https://github.com/matter-labs/zksync-era/commit/3f663eca2f38f4373339ad024e6578099c693af6)) * **prover:** Remove old prover subsystems tables ([#812](https://github.com/matter-labs/zksync-era/issues/812)) ([9d0aefc](https://github.com/matter-labs/zksync-era/commit/9d0aefc1ef4992e19d7b15ec1ce34697e61a3464)) * **prover:** Remove prover-utils from core ([#819](https://github.com/matter-labs/zksync-era/issues/819)) ([2ceb911](https://github.com/matter-labs/zksync-era/commit/2ceb9114659f4c4583c87b1bbc8ee230eb1c44db)) --- This PR was generated with [Release Please](https://github.com/googleapis/release-please). See [documentation](https://github.com/googleapis/release-please#release-please). Co-authored-by: EmilLuta --- .github/release-please/manifest.json | 2 +- core/CHANGELOG.md | 27 +++++++++++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/.github/release-please/manifest.json b/.github/release-please/manifest.json index 21b427ed9b36..e8241e242832 100644 --- a/.github/release-please/manifest.json +++ b/.github/release-please/manifest.json @@ -1,5 +1,5 @@ { "sdk/zksync-rs": "0.4.0", - "core": "18.13.0", + "core": "19.0.0", "prover": "10.1.0" } diff --git a/core/CHANGELOG.md b/core/CHANGELOG.md index 48e53aaf83c0..09749b48c12d 100644 --- a/core/CHANGELOG.md +++ b/core/CHANGELOG.md @@ -1,5 +1,32 @@ # Changelog +## [19.0.0](https://github.com/matter-labs/zksync-era/compare/core-v18.13.0...core-v19.0.0) (2024-01-05) + + +### ⚠ BREAKING CHANGES + +* **vm:** Release v19 - remove allowlist ([#747](https://github.com/matter-labs/zksync-era/issues/747)) + +### Features + +* **en:** Make consistency checker work with pruned data ([#742](https://github.com/matter-labs/zksync-era/issues/742)) ([ae6e18e](https://github.com/matter-labs/zksync-era/commit/ae6e18e5412cadefbc03307a476d6b96c41f04e1)) +* **eth_sender:** Remove generic bounds on L1TxParamsProvider in EthSender ([#799](https://github.com/matter-labs/zksync-era/issues/799)) ([29a4f52](https://github.com/matter-labs/zksync-era/commit/29a4f5299c95e0b338010a6baf83f196ece3a530)) +* **merkle tree:** Finalize metadata calculator snapshot recovery logic ([#798](https://github.com/matter-labs/zksync-era/issues/798)) ([c83db35](https://github.com/matter-labs/zksync-era/commit/c83db35f0929a412bc4d89fbee1448d32c54a83f)) +* **prover:** Remove circuit-synthesizer ([#801](https://github.com/matter-labs/zksync-era/issues/801)) ([1426b1b](https://github.com/matter-labs/zksync-era/commit/1426b1ba3c8b700e0531087b781ced0756c12e3c)) +* **prover:** Remove old prover ([#810](https://github.com/matter-labs/zksync-era/issues/810)) ([8be1925](https://github.com/matter-labs/zksync-era/commit/8be1925b18dcbf268eb03b8ea5f07adfd5330876)) +* **snapshot creator:** Make snapshot creator fault-tolerant ([#691](https://github.com/matter-labs/zksync-era/issues/691)) ([286c7d1](https://github.com/matter-labs/zksync-era/commit/286c7d15a623604e01effa7119de3362f0fb4eb9)) +* **vm:** Add boojum integration folder ([#805](https://github.com/matter-labs/zksync-era/issues/805)) ([4071e90](https://github.com/matter-labs/zksync-era/commit/4071e90578e0fc8c027a4d2a30d09d96db942b4f)) +* **vm:** Make utils version-dependent ([#809](https://github.com/matter-labs/zksync-era/issues/809)) ([e5fbcb5](https://github.com/matter-labs/zksync-era/commit/e5fbcb5dfc2a7d2582f40a481c861fb2f4dd5fb0)) +* **vm:** Release v19 - remove allowlist ([#747](https://github.com/matter-labs/zksync-era/issues/747)) ([0e2bc56](https://github.com/matter-labs/zksync-era/commit/0e2bc561b9642b854718adcc86087a3e9762cf5d)) +* **vm:** Separate boojum integration vm ([#806](https://github.com/matter-labs/zksync-era/issues/806)) ([61712a6](https://github.com/matter-labs/zksync-era/commit/61712a636f69be70d75719c04f364d679ef624e0)) + + +### Bug Fixes + +* **db:** Fix parsing statement timeout from env ([#818](https://github.com/matter-labs/zksync-era/issues/818)) ([3f663ec](https://github.com/matter-labs/zksync-era/commit/3f663eca2f38f4373339ad024e6578099c693af6)) +* **prover:** Remove old prover subsystems tables ([#812](https://github.com/matter-labs/zksync-era/issues/812)) ([9d0aefc](https://github.com/matter-labs/zksync-era/commit/9d0aefc1ef4992e19d7b15ec1ce34697e61a3464)) +* **prover:** Remove prover-utils from core ([#819](https://github.com/matter-labs/zksync-era/issues/819)) ([2ceb911](https://github.com/matter-labs/zksync-era/commit/2ceb9114659f4c4583c87b1bbc8ee230eb1c44db)) + ## [18.13.0](https://github.com/matter-labs/zksync-era/compare/core-v18.12.0...core-v18.13.0) (2024-01-02) From 0fb2877eb19d9ea0273e167542ec6d23482d1e9c Mon Sep 17 00:00:00 2001 From: Marcin M <128217157+mm-zk@users.noreply.github.com> Date: Fri, 5 Jan 2024 15:34:06 +0100 Subject: [PATCH 20/49] chore(eth-sender): Added documentation to commit data (#731) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ * Added documentation to the calldata that we generate in state keeper when calling commit and prove batches * also removed unused 'dead_code' annotations that are no longer needed after we rolled out boojum ## Why ❔ * to increase the understanding of the code - and it make it easier to find during code search. --------- Co-authored-by: Roman Brodetski --- core/lib/types/src/commitment.rs | 24 ++++++++++++++++++++---- spellcheck/era.dic | 3 +++ 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/core/lib/types/src/commitment.rs b/core/lib/types/src/commitment.rs index 91503b7401c1..2d7dac0a34b6 100644 --- a/core/lib/types/src/commitment.rs +++ b/core/lib/types/src/commitment.rs @@ -131,24 +131,34 @@ impl L1BatchWithMetadata { }) } + /// Encodes L1Batch into StorageBatchInfo (see IExecutor.sol) pub fn l1_header_data(&self) -> Token { Token::Tuple(vec![ + // batchNumber Token::Uint(U256::from(self.header.number.0)), + // batchHash Token::FixedBytes(self.metadata.root_hash.as_bytes().to_vec()), + // indexRepeatedStorageChanges Token::Uint(U256::from(self.metadata.rollup_last_leaf_index)), + // numberOfLayer1Txs Token::Uint(U256::from(self.header.l1_tx_count)), + // priorityOperationsHash Token::FixedBytes( self.header .priority_ops_onchain_data_hash() .as_bytes() .to_vec(), ), + // l2LogsTreeRoot Token::FixedBytes(self.metadata.l2_l1_merkle_root.as_bytes().to_vec()), + // timestamp Token::Uint(U256::from(self.header.timestamp)), + // commitment Token::FixedBytes(self.metadata.commitment.as_bytes().to_vec()), ]) } + /// Encodes the L1Batch into CommitBatchInfo (see IExecutor.sol). pub fn l1_commit_data(&self) -> Token { if self.header.protocol_version.unwrap().is_pre_boojum() { Token::Tuple(vec![ @@ -183,17 +193,24 @@ impl L1BatchWithMetadata { ]) } else { Token::Tuple(vec![ + // batchNumber Token::Uint(U256::from(self.header.number.0)), + // timestamp Token::Uint(U256::from(self.header.timestamp)), + // indexRepeatedStorageChanges Token::Uint(U256::from(self.metadata.rollup_last_leaf_index)), + // newStateRoot Token::FixedBytes(self.metadata.merkle_root_hash.as_bytes().to_vec()), + // numberOfLayer1Txs Token::Uint(U256::from(self.header.l1_tx_count)), + // priorityOperationsHash Token::FixedBytes( self.header .priority_ops_onchain_data_hash() .as_bytes() .to_vec(), ), + // bootloaderHeapInitialContentsHash Token::FixedBytes( self.metadata .bootloader_initial_content_commitment @@ -201,6 +218,7 @@ impl L1BatchWithMetadata { .as_bytes() .to_vec(), ), + // eventsQueueStateHash Token::FixedBytes( self.metadata .events_queue_commitment @@ -208,7 +226,9 @@ impl L1BatchWithMetadata { .as_bytes() .to_vec(), ), + // systemLogs Token::Bytes(self.metadata.l2_l1_messages_compressed.clone()), + // totalL2ToL1Pubdata Token::Bytes( self.header .pubdata_input @@ -336,14 +356,10 @@ struct L1BatchAuxiliaryOutput { // The fields below are necessary for boojum. system_logs_compressed: Vec, - #[allow(dead_code)] system_logs_linear_hash: H256, - #[allow(dead_code)] state_diffs_hash: H256, state_diffs_compressed: Vec, - #[allow(dead_code)] bootloader_heap_hash: H256, - #[allow(dead_code)] events_state_queue_hash: H256, is_pre_boojum: bool, } diff --git a/spellcheck/era.dic b/spellcheck/era.dic index a8ad97cc859d..7c289f7b4b44 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -279,6 +279,9 @@ blake2 AR16MT Preimages EN's +StorageBatchInfo +CommitBatchInfo +IExecutor // Names Vyper From c4a86bbbc5697b5391a517299bbd7a5e882a7314 Mon Sep 17 00:00:00 2001 From: perekopskiy <53865202+perekopskiy@users.noreply.github.com> Date: Fri, 5 Jan 2024 16:57:29 +0200 Subject: [PATCH 21/49] feat(state-keeper): circuits seal criterion (#729) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - the number of circuits needed is estimated in VM - replaces `ComputationGasCriterion` with more precise `CircuitsCriterion` ## Why ❔ Introduce `CircuitsCriterion` for efficient batch sealing ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- Cargo.lock | 6 +- core/bin/snapshots_creator/src/tests.rs | 2 +- ...21833_l1-batch-predicted-circuits.down.sql | 2 + ...4121833_l1-batch-predicted-circuits.up.sql | 2 + core/lib/dal/sqlx-data.json | 71 +++--- core/lib/dal/src/blocks_dal.rs | 10 +- core/lib/dal/src/storage_logs_dal.rs | 2 +- core/lib/dal/src/sync_dal.rs | 4 +- core/lib/multivm/Cargo.toml | 3 +- .../src/glue/types/vm/vm_block_result.rs | 6 + .../types/vm/vm_partial_execution_result.rs | 3 + .../types/outputs/execution_result.rs | 1 + .../src/interface/types/outputs/statistic.rs | 1 + .../implementation/execution.rs | 1 + .../implementation/statistics.rs | 2 + .../old_vm/oracles/precompile.rs | 70 ++++-- .../vm_boojum_integration/oracles/storage.rs | 22 +- .../vm_boojum_integration/tests/circuits.rs | 43 ++++ .../vm_boojum_integration/tests/mod.rs | 2 + .../tests/precompiles.rs | 136 ++++++++++ .../vm_boojum_integration/tests/utils.rs | 6 + .../tracers/circuits_capacity.rs | 85 +++++++ .../tracers/circuits_tracer.rs | 199 +++++++++++++++ .../tracers/default_tracers.rs | 13 +- .../vm_boojum_integration/tracers/mod.rs | 3 + .../types/internals/vm_state.rs | 4 +- .../vm_latest/implementation/execution.rs | 1 + .../vm_latest/implementation/statistics.rs | 2 + .../vm_latest/old_vm/oracles/precompile.rs | 70 ++++-- .../src/versions/vm_latest/oracles/storage.rs | 22 +- .../src/versions/vm_latest/tests/circuits.rs | 43 ++++ .../src/versions/vm_latest/tests/mod.rs | 2 + .../versions/vm_latest/tests/precompiles.rs | 136 ++++++++++ .../src/versions/vm_latest/tests/utils.rs | 6 + .../vm_latest/tracers/circuits_capacity.rs | 85 +++++++ .../vm_latest/tracers/circuits_tracer.rs | 199 +++++++++++++++ .../vm_latest/tracers/default_tracers.rs | 13 +- .../src/versions/vm_latest/tracers/mod.rs | 3 + .../vm_latest/types/internals/vm_state.rs | 4 +- .../implementation/statistics.rs | 1 + .../implementation/statistics.rs | 1 + core/lib/state/src/test_utils.rs | 2 +- core/lib/types/src/fee.rs | 1 + core/lib/types/src/tx/tx_execution_info.rs | 3 + .../execution_sandbox/vm_metrics.rs | 1 + .../src/api_server/web3/tests/snapshots.rs | 2 +- .../src/consistency_checker/tests/mod.rs | 2 +- core/lib/zksync_core/src/eth_sender/tests.rs | 2 +- core/lib/zksync_core/src/genesis.rs | 1 + .../src/metadata_calculator/recovery/tests.rs | 2 +- .../src/metadata_calculator/tests.rs | 5 +- .../zksync_core/src/reorg_detector/tests.rs | 2 +- .../src/state_keeper/io/seal_logic.rs | 4 + .../src/state_keeper/io/tests/tester.rs | 2 +- .../seal_criteria/conditional_sealer.rs | 6 +- .../criteria/geometry_seal_criteria.rs | 232 ++---------------- .../seal_criteria/criteria/mod.rs | 9 +- .../zksync_core/src/state_keeper/tests/mod.rs | 1 + .../contracts/precompiles/precompiles.sol | 21 ++ prover/Cargo.lock | 2 +- 60 files changed, 1262 insertions(+), 325 deletions(-) create mode 100644 core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.down.sql create mode 100644 core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.up.sql create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tests/precompiles.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs create mode 100644 core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs create mode 100644 core/lib/multivm/src/versions/vm_latest/tests/circuits.rs create mode 100644 core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs create mode 100644 core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs create mode 100644 core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs create mode 100644 etc/contracts-test-data/contracts/precompiles/precompiles.sol diff --git a/Cargo.lock b/Cargo.lock index 461f4df862cc..d104f9932cbd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4001,6 +4001,7 @@ dependencies = [ "zk_evm 1.3.1", "zk_evm 1.3.3 (git+https://github.com/matter-labs/era-zk_evm.git?tag=v1.3.3-rc2)", "zk_evm 1.4.0", + "zkevm_test_harness 1.4.0", "zksync_contracts", "zksync_eth_signer", "zksync_state", @@ -8022,9 +8023,10 @@ dependencies = [ [[package]] name = "zk_evm_abstractions" version = "0.1.0" -source = "git+https://github.com/matter-labs/era-zk_evm_abstractions.git#15a2af404902d5f10352e3d1fac693cc395fcff9" +source = "git+https://github.com/matter-labs/era-zk_evm_abstractions.git#32dd320953841aa78579d9da08abbc70bcaed175" dependencies = [ "anyhow", + "num_enum", "serde", "static_assertions", "zkevm_opcode_defs 1.3.2", @@ -8104,7 +8106,7 @@ dependencies = [ "codegen 0.2.0", "crossbeam 0.8.2", "derivative", - "env_logger 0.10.0", + "env_logger 0.9.3", "hex", "num-bigint 0.4.4", "num-integer", diff --git a/core/bin/snapshots_creator/src/tests.rs b/core/bin/snapshots_creator/src/tests.rs index db8501c7d796..d344f453e6c9 100644 --- a/core/bin/snapshots_creator/src/tests.rs +++ b/core/bin/snapshots_creator/src/tests.rs @@ -173,7 +173,7 @@ async fn create_l1_batch( ); header.is_finished = true; conn.blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); conn.blocks_dal() diff --git a/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.down.sql b/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.down.sql new file mode 100644 index 000000000000..925299304ebe --- /dev/null +++ b/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.down.sql @@ -0,0 +1,2 @@ +ALTER TABLE l1_batches + DROP COLUMN IF EXISTS predicted_circuits; diff --git a/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.up.sql b/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.up.sql new file mode 100644 index 000000000000..a957fce4d984 --- /dev/null +++ b/core/lib/dal/migrations/20240104121833_l1-batch-predicted-circuits.up.sql @@ -0,0 +1,2 @@ +ALTER TABLE l1_batches + ADD COLUMN IF NOT EXISTS predicted_circuits INT; diff --git a/core/lib/dal/sqlx-data.json b/core/lib/dal/sqlx-data.json index 183ab24fc458..b009e9d8028c 100644 --- a/core/lib/dal/sqlx-data.json +++ b/core/lib/dal/sqlx-data.json @@ -5788,6 +5788,42 @@ }, "query": "\n SELECT\n number,\n hash\n FROM\n miniblocks\n WHERE\n number >= $1\n ORDER BY\n number ASC\n LIMIT\n $2\n " }, + "70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94": { + "describe": { + "columns": [], + "nullable": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Int4", + "Int8", + "Bool", + "Bytea", + "ByteaArray", + "ByteaArray", + "Bytea", + "ByteaArray", + "Int8", + "Int8", + "Int8", + "Jsonb", + "Jsonb", + "Numeric", + "Int8", + "Int8", + "Bytea", + "Bytea", + "Int4", + "ByteaArray", + "Int8Array", + "Bytea", + "Int4" + ] + } + }, + "query": "\n INSERT INTO\n l1_batches (\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n predicted_commit_gas_cost,\n predicted_prove_gas_cost,\n predicted_execute_gas_cost,\n initial_bootloader_heap_content,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n storage_refunds,\n pubdata_input,\n predicted_circuits,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n $19,\n $20,\n $21,\n $22,\n $23,\n $24,\n $25,\n NOW(),\n NOW()\n )\n " + }, "72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42": { "describe": { "columns": [ @@ -8002,41 +8038,6 @@ }, "query": "\n SELECT\n upgrade_tx_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n " }, - "aa8e569cf406cd0975a6ffaeeafa92f632186181ba8b93518e549e0643f58da8": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4", - "Int4", - "Int8", - "Bool", - "Bytea", - "ByteaArray", - "ByteaArray", - "Bytea", - "ByteaArray", - "Int8", - "Int8", - "Int8", - "Jsonb", - "Jsonb", - "Numeric", - "Int8", - "Int8", - "Bytea", - "Bytea", - "Int4", - "ByteaArray", - "Int8Array", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n l1_batches (\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n predicted_commit_gas_cost,\n predicted_prove_gas_cost,\n predicted_execute_gas_cost,\n initial_bootloader_heap_content,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n storage_refunds,\n pubdata_input,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n $19,\n $20,\n $21,\n $22,\n $23,\n $24,\n NOW(),\n NOW()\n )\n " - }, "aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320": { "describe": { "columns": [ diff --git a/core/lib/dal/src/blocks_dal.rs b/core/lib/dal/src/blocks_dal.rs index ae3ef48bcbb6..239f9074800e 100644 --- a/core/lib/dal/src/blocks_dal.rs +++ b/core/lib/dal/src/blocks_dal.rs @@ -450,6 +450,7 @@ impl BlocksDal<'_, '_> { predicted_block_gas: BlockGasCount, events_queue: &[LogQuery], storage_refunds: &[u32], + predicted_circuits: u32, ) -> anyhow::Result<()> { let priority_onchain_data: Vec> = header .priority_ops_onchain_data @@ -509,6 +510,7 @@ impl BlocksDal<'_, '_> { system_logs, storage_refunds, pubdata_input, + predicted_circuits, created_at, updated_at ) @@ -538,6 +540,7 @@ impl BlocksDal<'_, '_> { $22, $23, $24, + $25, NOW(), NOW() ) @@ -566,6 +569,7 @@ impl BlocksDal<'_, '_> { &system_logs, &storage_refunds, pubdata_input, + predicted_circuits as i32, ) .execute(transaction.conn()) .await?; @@ -2341,7 +2345,7 @@ mod tests { header.l2_to_l1_messages.push(vec![33; 33]); conn.blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); @@ -2390,7 +2394,7 @@ mod tests { execute: 10, }; conn.blocks_dal() - .insert_l1_batch(&header, &[], predicted_gas, &[], &[]) + .insert_l1_batch(&header, &[], predicted_gas, &[], &[], 0) .await .unwrap(); @@ -2398,7 +2402,7 @@ mod tests { header.timestamp += 100; predicted_gas += predicted_gas; conn.blocks_dal() - .insert_l1_batch(&header, &[], predicted_gas, &[], &[]) + .insert_l1_batch(&header, &[], predicted_gas, &[], &[], 0) .await .unwrap(); diff --git a/core/lib/dal/src/storage_logs_dal.rs b/core/lib/dal/src/storage_logs_dal.rs index ff757b748e8d..21572f3d3c0f 100644 --- a/core/lib/dal/src/storage_logs_dal.rs +++ b/core/lib/dal/src/storage_logs_dal.rs @@ -733,7 +733,7 @@ mod tests { ); header.is_finished = true; conn.blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); conn.blocks_dal() diff --git a/core/lib/dal/src/sync_dal.rs b/core/lib/dal/src/sync_dal.rs index 4d50f2855bbf..c9a737f581d0 100644 --- a/core/lib/dal/src/sync_dal.rs +++ b/core/lib/dal/src/sync_dal.rs @@ -136,7 +136,7 @@ mod tests { ProtocolVersionId::latest(), ); conn.blocks_dal() - .insert_l1_batch(&l1_batch_header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&l1_batch_header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); conn.blocks_dal() @@ -205,7 +205,7 @@ mod tests { l1_batch_header.number = L1BatchNumber(1); l1_batch_header.timestamp = 1; conn.blocks_dal() - .insert_l1_batch(&l1_batch_header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&l1_batch_header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); conn.blocks_dal() diff --git a/core/lib/multivm/Cargo.toml b/core/lib/multivm/Cargo.toml index 0cf207409904..5f87b5ae5543 100644 --- a/core/lib/multivm/Cargo.toml +++ b/core/lib/multivm/Cargo.toml @@ -14,6 +14,8 @@ zk_evm_1_4_0 = { package = "zk_evm", git = "https://github.com/matter-labs/era-z zk_evm_1_3_3 = { package = "zk_evm", git = "https://github.com/matter-labs/era-zk_evm.git", tag= "v1.3.3-rc2" } zk_evm_1_3_1 = { package = "zk_evm", git = "https://github.com/matter-labs/era-zk_evm.git", tag= "v1.3.1-rc2" } +zkevm_test_harness_1_4_0 = { git = "https://github.com/matter-labs/era-zkevm_test_harness.git", branch = "v1.4.0", package = "zkevm_test_harness" } + zksync_types = { path = "../types" } zksync_state = { path = "../state" } zksync_contracts = { path = "../contracts" } @@ -29,7 +31,6 @@ thiserror = "1.0" tracing = "0.1" vise = { git = "https://github.com/matter-labs/vise.git", version = "0.1.0", rev = "1c9cc500e92cf9ea052b230e114a6f9cce4fb2c1" } - [dev-dependencies] tokio = { version = "1", features = ["time"] } zksync_test_account = { path = "../test_account" } diff --git a/core/lib/multivm/src/glue/types/vm/vm_block_result.rs b/core/lib/multivm/src/glue/types/vm/vm_block_result.rs index 623d3d735d19..f7eda05cc02d 100644 --- a/core/lib/multivm/src/glue/types/vm/vm_block_result.rs +++ b/core/lib/multivm/src/glue/types/vm/vm_block_result.rs @@ -26,6 +26,7 @@ impl GlueFrom for crate::interface::Fi computational_gas_used: value.full_result.gas_used, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), }, @@ -64,6 +65,7 @@ impl GlueFrom for crate::interface::Fi computational_gas_used: value.full_result.computational_gas_used, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), }, @@ -108,6 +110,7 @@ impl GlueFrom for crate::interface: computational_gas_used: value.full_result.computational_gas_used, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), }, @@ -168,6 +171,7 @@ impl GlueFrom computational_gas_used: value.full_result.computational_gas_used, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), } @@ -198,6 +202,7 @@ impl GlueFrom computational_gas_used: 0, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), } @@ -239,6 +244,7 @@ impl GlueFrom computational_gas_used: value.full_result.computational_gas_used, gas_used: value.full_result.gas_used, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), } diff --git a/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs b/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs index 4de727a04c10..6a52c7e66987 100644 --- a/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs +++ b/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs @@ -16,6 +16,7 @@ impl GlueFrom // There are no such fields in m5 computational_gas_used: 0, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: crate::interface::Refunds { gas_refunded: 0, @@ -39,6 +40,7 @@ impl GlueFrom computational_gas_used: value.computational_gas_used, total_log_queries: value.logs.total_log_queries_count, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: crate::interface::Refunds { gas_refunded: 0, @@ -62,6 +64,7 @@ impl GlueFrom computational_gas_used: value.computational_gas_used, total_log_queries: value.logs.total_log_queries_count, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: crate::interface::Refunds { gas_refunded: 0, diff --git a/core/lib/multivm/src/interface/types/outputs/execution_result.rs b/core/lib/multivm/src/interface/types/outputs/execution_result.rs index e177b6300120..6471ca1fe193 100644 --- a/core/lib/multivm/src/interface/types/outputs/execution_result.rs +++ b/core/lib/multivm/src/interface/types/outputs/execution_result.rs @@ -101,6 +101,7 @@ impl VmExecutionResultAndLogs { cycles_used: self.statistics.cycles_used, computational_gas_used: self.statistics.computational_gas_used, pubdata_published: self.statistics.pubdata_published, + estimated_circuits_used: self.statistics.estimated_circuits_used, } } } diff --git a/core/lib/multivm/src/interface/types/outputs/statistic.rs b/core/lib/multivm/src/interface/types/outputs/statistic.rs index c1312fc95da8..1f5b233423c0 100644 --- a/core/lib/multivm/src/interface/types/outputs/statistic.rs +++ b/core/lib/multivm/src/interface/types/outputs/statistic.rs @@ -12,6 +12,7 @@ pub struct VmExecutionStatistics { /// Number of log queries produced by the VM during the tx execution. pub total_log_queries: usize, pub pubdata_published: u32, + pub estimated_circuits_used: f32, } /// Oracle metrics of the VM. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs index 9623b21a1945..1d1d19f92b76 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/execution.rs @@ -80,6 +80,7 @@ impl Vm { spent_pubdata_counter_before, pubdata_published, logs.total_log_queries_count, + tx_tracer.circuits_tracer.estimated_circuits_used, ); let result = tx_tracer.result_tracer.into_result(); diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs index c11165cf78d1..36780c8b8458 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/statistics.rs @@ -24,6 +24,7 @@ impl Vm { spent_pubdata_counter_before: u32, pubdata_published: u32, total_log_queries_count: usize, + estimated_circuits_used: f32, ) -> VmExecutionStatistics { let computational_gas_used = self.calculate_computational_gas_used( tracer, @@ -40,6 +41,7 @@ impl Vm { computational_gas_used, total_log_queries: total_log_queries_count, pubdata_published, + estimated_circuits_used, } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs index 4c798a00a37b..c4986250cbae 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs @@ -1,7 +1,9 @@ +use std::convert::TryFrom; + use zk_evm_1_4_0::{ abstractions::{Memory, PrecompileCyclesWitness, PrecompilesProcessor}, aux_structures::{LogQuery, MemoryQuery, Timestamp}, - zk_evm_abstractions::precompiles::DefaultPrecompilesProcessor, + zk_evm_abstractions::precompiles::{ecrecover, keccak256, sha256, PrecompileAddress}, }; use super::OracleWithHistory; @@ -15,40 +17,44 @@ use crate::vm_boojum_integration::old_vm::history_recorder::{ /// saving timestamps allows us to check the exact number /// of log queries, that were used during the tx execution. #[derive(Debug, Clone)] -pub struct PrecompilesProcessorWithHistory { +pub struct PrecompilesProcessorWithHistory { pub timestamp_history: HistoryRecorder, H>, - pub default_precompiles_processor: DefaultPrecompilesProcessor, + pub precompile_cycles_history: HistoryRecorder, HistoryEnabled>, } -impl Default for PrecompilesProcessorWithHistory { +impl Default for PrecompilesProcessorWithHistory { fn default() -> Self { Self { timestamp_history: Default::default(), - default_precompiles_processor: DefaultPrecompilesProcessor, + precompile_cycles_history: Default::default(), } } } -impl OracleWithHistory for PrecompilesProcessorWithHistory { +impl OracleWithHistory for PrecompilesProcessorWithHistory { fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { self.timestamp_history.rollback_to_timestamp(timestamp); + self.precompile_cycles_history + .rollback_to_timestamp(timestamp); } } -impl PrecompilesProcessorWithHistory { +impl PrecompilesProcessorWithHistory { pub fn get_timestamp_history(&self) -> &Vec { self.timestamp_history.inner() } pub fn delete_history(&mut self) { self.timestamp_history.delete_history(); + self.precompile_cycles_history.delete_history(); } } -impl PrecompilesProcessor for PrecompilesProcessorWithHistory { +impl PrecompilesProcessor for PrecompilesProcessorWithHistory { fn start_frame(&mut self) { - self.default_precompiles_processor.start_frame(); + // there are no precompiles to rollback, do nothing } + fn execute_precompile( &mut self, monotonic_cycle_counter: u32, @@ -62,13 +68,47 @@ impl PrecompilesProcessor for PrecompilesProcesso // where operations and timestamp have different types. self.timestamp_history .push(query.timestamp, query.timestamp); - self.default_precompiles_processor.execute_precompile( - monotonic_cycle_counter, - query, - memory, - ) + + let address_low = u16::from_le_bytes([query.address.0[19], query.address.0[18]]); + if let Ok(precompile_address) = PrecompileAddress::try_from(address_low) { + let rounds = match precompile_address { + PrecompileAddress::Keccak256 => { + // pure function call, non-revertable + keccak256::keccak256_rounds_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + PrecompileAddress::SHA256 => { + // pure function call, non-revertable + sha256::sha256_rounds_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + PrecompileAddress::Ecrecover => { + // pure function call, non-revertable + ecrecover::ecrecover_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + }; + + self.precompile_cycles_history + .push((precompile_address, rounds), query.timestamp); + }; + + None } + fn finish_frame(&mut self, _panicked: bool) { - self.default_precompiles_processor.finish_frame(_panicked); + // there are no revertable precompile yes, so we are ok } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs index 2e051db47481..919fd301c573 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs @@ -59,6 +59,11 @@ pub struct StorageOracle { // Storage refunds that oracle has returned in `estimate_refunds_for_write`. pub(crate) returned_refunds: HistoryRecorder, H>, + + // Keeps track of storage keys that were ever written to. + pub(crate) written_keys: HistoryRecorder, HistoryEnabled>, + // Keeps track of storage keys that were ever read. + pub(crate) read_keys: HistoryRecorder, HistoryEnabled>, } impl OracleWithHistory for StorageOracle { @@ -69,6 +74,8 @@ impl OracleWithHistory for StorageOracle { self.paid_changes.rollback_to_timestamp(timestamp); self.initial_values.rollback_to_timestamp(timestamp); self.returned_refunds.rollback_to_timestamp(timestamp); + self.written_keys.rollback_to_timestamp(timestamp); + self.read_keys.rollback_to_timestamp(timestamp); } } @@ -81,6 +88,8 @@ impl StorageOracle { paid_changes: Default::default(), initial_values: Default::default(), returned_refunds: Default::default(), + written_keys: Default::default(), + read_keys: Default::default(), } } @@ -91,6 +100,8 @@ impl StorageOracle { self.paid_changes.delete_history(); self.initial_values.delete_history(); self.returned_refunds.delete_history(); + self.written_keys.delete_history(); + self.read_keys.delete_history(); } fn is_storage_key_free(&self, key: &StorageKey) -> bool { @@ -108,8 +119,12 @@ impl StorageOracle { } } - pub fn read_value(&mut self, mut query: LogQuery) -> LogQuery { + fn read_value(&mut self, mut query: LogQuery) -> LogQuery { let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + + if !self.read_keys.inner().contains_key(&key) { + self.read_keys.insert(key, (), query.timestamp); + } let current_value = self.storage.read_from_storage(&key); query.read_value = current_value; @@ -127,8 +142,11 @@ impl StorageOracle { query } - pub fn write_value(&mut self, query: LogQuery) -> LogQuery { + fn write_value(&mut self, query: LogQuery) -> LogQuery { let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + if !self.written_keys.inner().contains_key(&key) { + self.written_keys.insert(key, (), query.timestamp); + } let current_value = self.storage .write_to_storage(key, query.written_value, query.timestamp); diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs new file mode 100644 index 000000000000..2630c913e02a --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs @@ -0,0 +1,43 @@ +use zksync_types::{Address, Execute, U256}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_boojum_integration::{constants::BLOCK_GAS_LIMIT, tests::tester::VmTesterBuilder, HistoryEnabled}, +}; + +// Checks that estimated number of circuits for simple transfer doesn't differ much +// from hardcoded expected value. +#[test] +fn test_circuits() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: Address::random(), + calldata: Vec::new(), + value: U256::from(1u8), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let res = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + const EXPECTED_CIRCUITS_USED: f32 = 1.5521; + let delta = + (res.statistics.estimated_circuits_used - EXPECTED_CIRCUITS_USED) / EXPECTED_CIRCUITS_USED; + + if delta.abs() > 0.1 { + panic!( + "Estimation differs from expected result by too much: {}%", + delta * 100.0 + ); + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs index ffb38dd3725a..95377232b3e3 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/mod.rs @@ -4,12 +4,14 @@ mod default_aa; // mod invalid_bytecode; mod bytecode_publishing; mod call_tracer; +mod circuits; mod gas_limit; mod get_used_contracts; mod is_write_initial; mod l1_tx_execution; mod l2_blocks; mod nonce_holder; +mod precompiles; mod refunds; mod require_eip712; mod rollbacks; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/precompiles.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/precompiles.rs new file mode 100644 index 000000000000..516331d574f4 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/precompiles.rs @@ -0,0 +1,136 @@ +use zk_evm_1_4_0::zk_evm_abstractions::precompiles::PrecompileAddress; +use zksync_types::{Address, Execute}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_boojum_integration::{ + constants::BLOCK_GAS_LIMIT, + tests::{tester::VmTesterBuilder, utils::read_precompiles_contract}, + HistoryEnabled, + }, +}; + +#[test] +fn test_keccak() { + // Execute special transaction and check that at least 1000 keccak calls were made. + let contract = read_precompiles_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contract, address, true)]) + .build(); + + // calldata for `doKeccak(1000)`. + let keccak1000_calldata = + "370f20ac00000000000000000000000000000000000000000000000000000000000003e8"; + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: hex::decode(keccak1000_calldata).unwrap(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let keccak_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::Keccak256) + .count(); + + assert!(keccak_count >= 1000); +} + +#[test] +fn test_sha256() { + // Execute special transaction and check that at least 1000 sha256 calls were made. + let contract = read_precompiles_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contract, address, true)]) + .build(); + + // calldata for `doSha256(1000)`. + let sha1000_calldata = + "5d0b4fb500000000000000000000000000000000000000000000000000000000000003e8"; + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: hex::decode(sha1000_calldata).unwrap(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let sha_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::SHA256) + .count(); + + assert!(sha_count >= 1000); +} + +#[test] +fn test_ecrecover() { + // Execute simple transfer and check that exactly 1 ecrecover call was made (it's done during tx validation). + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: account.address, + calldata: Vec::new(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let ecrecover_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::Ecrecover) + .count(); + + assert_eq!(ecrecover_count, 1); +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs index 53ae1c17e917..2dd8e2350eb4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/utils.rs @@ -103,3 +103,9 @@ pub(crate) fn read_max_depth_contract() -> Vec { "core/tests/ts-integration/contracts/zkasm/artifacts/deep_stak.zkasm/deep_stak.zkasm.zbin", ) } + +pub(crate) fn read_precompiles_contract() -> Vec { + read_bytecode( + "etc/contracts-test-data/artifacts-zk/contracts/precompiles/precompiles.sol/Precompiles.json", + ) +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs new file mode 100644 index 000000000000..16f5540172a8 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs @@ -0,0 +1,85 @@ +use zkevm_test_harness_1_4_0::{geometry_config::get_geometry_config, toolset::GeometryConfig}; + +const GEOMETRY_CONFIG: GeometryConfig = get_geometry_config(); +const OVERESTIMATE_PERCENT: f32 = 1.05; + +const MAIN_VM_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_vm_snapshot as f32; + +const CODE_DECOMMITTER_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_code_decommitter_sorter as f32; + +const LOG_DEMUXER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_log_demuxer as f32; + +const STORAGE_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_storage_sorter as f32; + +const EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_events_or_l1_messages_sorter as f32; + +const RAM_PERMUTATION_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_ram_permutation as f32; + +pub(crate) const CODE_DECOMMITTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_code_decommitter as f32; + +pub(crate) const STORAGE_APPLICATION_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_storage_application as f32; + +pub(crate) const KECCAK256_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_keccak256_circuit as f32; + +pub(crate) const SHA256_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_sha256_circuit as f32; + +pub(crate) const ECRECOVER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_ecrecover_circuit as f32; + +// "Rich addressing" opcodes are opcodes that can write their return value/read the input onto the stack +// and so take 1-2 RAM permutations more than an average opcode. +// In the worst case, a rich addressing may take 3 ram permutations +// (1 for reading the opcode, 1 for writing input value, 1 for writing output value). +pub(crate) const RICH_ADDRESSING_OPCODE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 3.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +pub(crate) const AVERAGE_OPCODE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION; + +// Here "base" fraction is a fraction that will be used unconditionally. +// Usage of StorageApplication is being tracked separately as it depends on whether slot was read before or not. +pub(crate) const STORAGE_READ_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + LOG_DEMUXER_CYCLE_FRACTION + + STORAGE_SORTER_CYCLE_FRACTION; + +pub(crate) const EVENT_OR_L1_MESSAGE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + 2.0 * LOG_DEMUXER_CYCLE_FRACTION + + 2.0 * EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION; + +// Here "base" fraction is a fraction that will be used unconditionally. +// Usage of StorageApplication is being tracked separately as it depends on whether slot was written before or not. +pub(crate) const STORAGE_WRITE_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + 2.0 * LOG_DEMUXER_CYCLE_FRACTION + + 2.0 * STORAGE_SORTER_CYCLE_FRACTION; + +pub(crate) const FAR_CALL_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + STORAGE_SORTER_CYCLE_FRACTION + + CODE_DECOMMITTER_SORTER_CYCLE_FRACTION; + +// 5 RAM permutations, because: 1 to read opcode + 2 reads + 2 writes. +// 2 reads and 2 writes are needed because unaligned access is implemented with +// aligned queries. +pub(crate) const UMA_WRITE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 5.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +// 3 RAM permutations, because: 1 to read opcode + 2 reads. +// 2 reads are needed because unaligned access is implemented with aligned queries. +pub(crate) const UMA_READ_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 3.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +pub(crate) const PRECOMPILE_CALL_COMMON_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + LOG_DEMUXER_CYCLE_FRACTION; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs new file mode 100644 index 000000000000..9f26cd057ff0 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs @@ -0,0 +1,199 @@ +use std::marker::PhantomData; + +use zk_evm_1_4_0::{ + tracing::{BeforeExecutionData, VmLocalStateData}, + zk_evm_abstractions::precompiles::PrecompileAddress, + zkevm_opcode_defs::{LogOpcode, Opcode, UMAOpcode}, +}; +use zksync_state::{StoragePtr, WriteStorage}; + +use super::circuits_capacity::*; +use crate::{ + interface::{dyn_tracers::vm_1_4_0::DynTracer, tracer::TracerExecutionStatus}, + vm_boojum_integration::{ + bootloader_state::BootloaderState, + old_vm::{ + history_recorder::{HistoryMode, VectorHistoryEvent}, + memory::SimpleMemory, + }, + tracers::traits::VmTracer, + types::internals::ZkSyncVmState, + }, +}; + +/// Tracer responsible for collecting information about refunds. +#[derive(Debug)] +pub(crate) struct CircuitsTracer { + pub(crate) estimated_circuits_used: f32, + last_decommitment_history_entry_checked: Option, + last_written_keys_history_entry_checked: Option, + last_read_keys_history_entry_checked: Option, + last_precompile_history_entry_checked: Option, + _phantom_data: PhantomData, +} + +impl CircuitsTracer { + pub(crate) fn new() -> Self { + Self { + estimated_circuits_used: 0.0, + last_decommitment_history_entry_checked: None, + last_written_keys_history_entry_checked: None, + last_read_keys_history_entry_checked: None, + last_precompile_history_entry_checked: None, + _phantom_data: Default::default(), + } + } +} + +impl DynTracer> for CircuitsTracer { + fn before_execution( + &mut self, + _state: VmLocalStateData<'_>, + data: BeforeExecutionData, + _memory: &SimpleMemory, + _storage: StoragePtr, + ) { + let used = match data.opcode.variant.opcode { + Opcode::Nop(_) + | Opcode::Add(_) + | Opcode::Sub(_) + | Opcode::Mul(_) + | Opcode::Div(_) + | Opcode::Jump(_) + | Opcode::Binop(_) + | Opcode::Shift(_) + | Opcode::Ptr(_) => RICH_ADDRESSING_OPCODE_FRACTION, + Opcode::Context(_) | Opcode::Ret(_) | Opcode::NearCall(_) => AVERAGE_OPCODE_FRACTION, + Opcode::Log(LogOpcode::StorageRead) => STORAGE_READ_BASE_FRACTION, + Opcode::Log(LogOpcode::StorageWrite) => STORAGE_WRITE_BASE_FRACTION, + Opcode::Log(LogOpcode::ToL1Message) | Opcode::Log(LogOpcode::Event) => { + EVENT_OR_L1_MESSAGE_FRACTION + } + Opcode::Log(LogOpcode::PrecompileCall) => PRECOMPILE_CALL_COMMON_FRACTION, + Opcode::FarCall(_) => FAR_CALL_FRACTION, + Opcode::UMA(UMAOpcode::AuxHeapWrite | UMAOpcode::HeapWrite) => UMA_WRITE_FRACTION, + Opcode::UMA( + UMAOpcode::AuxHeapRead | UMAOpcode::HeapRead | UMAOpcode::FatPointerRead, + ) => UMA_READ_FRACTION, + Opcode::Invalid(_) => unreachable!(), // invalid opcodes are never executed + }; + + self.estimated_circuits_used += used; + } +} + +impl VmTracer for CircuitsTracer { + fn initialize_tracer(&mut self, state: &mut ZkSyncVmState) { + self.last_decommitment_history_entry_checked = Some( + state + .decommittment_processor + .decommitted_code_hashes + .history() + .len(), + ); + + self.last_written_keys_history_entry_checked = + Some(state.storage.written_keys.history().len()); + + self.last_read_keys_history_entry_checked = Some(state.storage.read_keys.history().len()); + + self.last_precompile_history_entry_checked = Some( + state + .precompiles_processor + .precompile_cycles_history + .history() + .len(), + ); + } + + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + // Trace decommitments. + let last_decommitment_history_entry_checked = self + .last_decommitment_history_entry_checked + .expect("Value must be set during init"); + let history = state + .decommittment_processor + .decommitted_code_hashes + .history(); + for (_, history_event) in &history[last_decommitment_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + let bytecode_len = state + .decommittment_processor + .known_bytecodes + .inner() + .get(&history_event.key) + .expect("Bytecode must be known at this point") + .len(); + + // Each cycle of `CodeDecommitter` processes 2 words. + // If the number of words in bytecode is odd, then number of cycles must be rounded up. + let decommitter_cycles_used = (bytecode_len + 1) / 2; + self.estimated_circuits_used += + (decommitter_cycles_used as f32) * CODE_DECOMMITTER_CYCLE_FRACTION; + } + self.last_decommitment_history_entry_checked = Some(history.len()); + + // Process storage writes. + let last_writes_history_entry_checked = self + .last_written_keys_history_entry_checked + .expect("Value must be set during init"); + let history = state.storage.written_keys.history(); + for (_, history_event) in &history[last_writes_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + + self.estimated_circuits_used += 2.0 * STORAGE_APPLICATION_CYCLE_FRACTION; + } + self.last_written_keys_history_entry_checked = Some(history.len()); + + // Process storage reads. + let last_reads_history_entry_checked = self + .last_read_keys_history_entry_checked + .expect("Value must be set during init"); + let history = state.storage.read_keys.history(); + for (_, history_event) in &history[last_reads_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + + // If the slot is already written to, then we've already taken 2 cycles into account. + if !state + .storage + .written_keys + .inner() + .contains_key(&history_event.key) + { + self.estimated_circuits_used += STORAGE_APPLICATION_CYCLE_FRACTION; + } + } + self.last_read_keys_history_entry_checked = Some(history.len()); + + // Process precompiles. + let last_precompile_history_entry_checked = self + .last_precompile_history_entry_checked + .expect("Value must be set during init"); + let history = state + .precompiles_processor + .precompile_cycles_history + .history(); + for (_, history_event) in &history[last_precompile_history_entry_checked..] { + if let VectorHistoryEvent::Push((precompile, cycles)) = history_event { + let fraction = match precompile { + PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, + PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, + PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, + }; + self.estimated_circuits_used += (*cycles as f32) * fraction; + } else { + panic!("Precompile calls should not be rolled back"); + } + } + self.last_precompile_history_entry_checked = Some(history.len()); + + TracerExecutionStatus::Continue + } +} diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs index f0690d996b21..236421726051 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs @@ -32,7 +32,7 @@ use crate::{ computational_gas_price, gas_spent_on_bytecodes_and_long_messages_this_opcode, print_debug_if_needed, VmHook, }, - RefundsTracer, ResultTracer, + CircuitsTracer, RefundsTracer, ResultTracer, }, types::internals::ZkSyncVmState, VmTracer, @@ -62,6 +62,11 @@ pub(crate) struct DefaultExecutionTracer { pub(crate) pubdata_tracer: Option>, pub(crate) dispatcher: TracerDispatcher, ret_from_the_bootloader: Option, + // This tracer tracks what opcodes were executed and calculates how much circuits will be generated. + // It only takes into account circuits that are generated for actual execution. It doesn't + // take into account e.g circuits produced by the initial bootloader memory commitment. + pub(crate) circuits_tracer: CircuitsTracer, + storage: StoragePtr, _phantom: PhantomData, } @@ -88,6 +93,7 @@ impl DefaultExecutionTracer { dispatcher, pubdata_tracer, ret_from_the_bootloader: None, + circuits_tracer: CircuitsTracer::new(), storage, _phantom: PhantomData, } @@ -161,14 +167,15 @@ impl Debug for DefaultExecutionTracer { /// The macro passes the function call to all tracers. macro_rules! dispatch_tracers { ($self:ident.$function:ident($( $params:expr ),*)) => { - $self.result_tracer.$function($( $params ),*); - $self.dispatcher.$function($( $params ),*); + $self.result_tracer.$function($( $params ),*); + $self.dispatcher.$function($( $params ),*); if let Some(tracer) = &mut $self.refund_tracer { tracer.$function($( $params ),*); } if let Some(tracer) = &mut $self.pubdata_tracer { tracer.$function($( $params ),*); } + $self.circuits_tracer.$function($( $params ),*); }; } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs index 33d043de6eb1..1bdb1b6ccdbf 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/mod.rs @@ -1,13 +1,16 @@ +pub(crate) use circuits_tracer::CircuitsTracer; pub(crate) use default_tracers::DefaultExecutionTracer; pub(crate) use pubdata_tracer::PubdataTracer; pub(crate) use refunds::RefundsTracer; pub(crate) use result_tracer::ResultTracer; +pub(crate) mod circuits_tracer; pub(crate) mod default_tracers; pub(crate) mod pubdata_tracer; pub(crate) mod refunds; pub(crate) mod result_tracer; +mod circuits_capacity; pub mod dispatcher; pub(crate) mod traits; pub(crate) mod utils; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs index b775b4b63bca..bff8dbf0f56d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/vm_state.rs @@ -40,7 +40,7 @@ pub type ZkSyncVmState = VmState< StorageOracle, SimpleMemory, InMemoryEventSink, - PrecompilesProcessorWithHistory, + PrecompilesProcessorWithHistory, DecommitterOracle, DummyTracer, >; @@ -84,7 +84,7 @@ pub(crate) fn new_vm_state( let storage_oracle: StorageOracle = StorageOracle::new(storage.clone()); let mut memory = SimpleMemory::default(); let event_sink = InMemoryEventSink::default(); - let precompiles_processor = PrecompilesProcessorWithHistory::::default(); + let precompiles_processor = PrecompilesProcessorWithHistory::::default(); let mut decommittment_processor: DecommitterOracle = DecommitterOracle::new(storage); diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs b/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs index cc76872d29ec..1dfdc5b36499 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/execution.rs @@ -79,6 +79,7 @@ impl Vm { spent_pubdata_counter_before, pubdata_published, logs.total_log_queries_count, + tx_tracer.circuits_tracer.estimated_circuits_used, ); let result = tx_tracer.result_tracer.into_result(); diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs index 8bb3f65bea47..3f88629ffb00 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/statistics.rs @@ -24,6 +24,7 @@ impl Vm { spent_pubdata_counter_before: u32, pubdata_published: u32, total_log_queries_count: usize, + estimated_circuits_used: f32, ) -> VmExecutionStatistics { let computational_gas_used = self.calculate_computational_gas_used( tracer, @@ -40,6 +41,7 @@ impl Vm { computational_gas_used, total_log_queries: total_log_queries_count, pubdata_published, + estimated_circuits_used, } } diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs index 92b88e40fc95..dbc03fb143a5 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs @@ -1,7 +1,9 @@ +use std::convert::TryFrom; + use zk_evm_1_4_0::{ abstractions::{Memory, PrecompileCyclesWitness, PrecompilesProcessor}, aux_structures::{LogQuery, MemoryQuery, Timestamp}, - zk_evm_abstractions::precompiles::DefaultPrecompilesProcessor, + zk_evm_abstractions::precompiles::{ecrecover, keccak256, sha256, PrecompileAddress}, }; use super::OracleWithHistory; @@ -13,40 +15,44 @@ use crate::vm_latest::old_vm::history_recorder::{HistoryEnabled, HistoryMode, Hi /// saving timestamps allows us to check the exact number /// of log queries, that were used during the tx execution. #[derive(Debug, Clone)] -pub struct PrecompilesProcessorWithHistory { +pub struct PrecompilesProcessorWithHistory { pub timestamp_history: HistoryRecorder, H>, - pub default_precompiles_processor: DefaultPrecompilesProcessor, + pub precompile_cycles_history: HistoryRecorder, HistoryEnabled>, } -impl Default for PrecompilesProcessorWithHistory { +impl Default for PrecompilesProcessorWithHistory { fn default() -> Self { Self { timestamp_history: Default::default(), - default_precompiles_processor: DefaultPrecompilesProcessor, + precompile_cycles_history: Default::default(), } } } -impl OracleWithHistory for PrecompilesProcessorWithHistory { +impl OracleWithHistory for PrecompilesProcessorWithHistory { fn rollback_to_timestamp(&mut self, timestamp: Timestamp) { self.timestamp_history.rollback_to_timestamp(timestamp); + self.precompile_cycles_history + .rollback_to_timestamp(timestamp); } } -impl PrecompilesProcessorWithHistory { +impl PrecompilesProcessorWithHistory { pub fn get_timestamp_history(&self) -> &Vec { self.timestamp_history.inner() } pub fn delete_history(&mut self) { self.timestamp_history.delete_history(); + self.precompile_cycles_history.delete_history(); } } -impl PrecompilesProcessor for PrecompilesProcessorWithHistory { +impl PrecompilesProcessor for PrecompilesProcessorWithHistory { fn start_frame(&mut self) { - self.default_precompiles_processor.start_frame(); + // there are no precompiles to rollback, do nothing } + fn execute_precompile( &mut self, monotonic_cycle_counter: u32, @@ -60,13 +66,47 @@ impl PrecompilesProcessor for PrecompilesProcesso // where operations and timestamp have different types. self.timestamp_history .push(query.timestamp, query.timestamp); - self.default_precompiles_processor.execute_precompile( - monotonic_cycle_counter, - query, - memory, - ) + + let address_low = u16::from_le_bytes([query.address.0[19], query.address.0[18]]); + if let Ok(precompile_address) = PrecompileAddress::try_from(address_low) { + let rounds = match precompile_address { + PrecompileAddress::Keccak256 => { + // pure function call, non-revertable + keccak256::keccak256_rounds_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + PrecompileAddress::SHA256 => { + // pure function call, non-revertable + sha256::sha256_rounds_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + PrecompileAddress::Ecrecover => { + // pure function call, non-revertable + ecrecover::ecrecover_function::( + monotonic_cycle_counter, + query, + memory, + ) + .0 + } + }; + + self.precompile_cycles_history + .push((precompile_address, rounds), query.timestamp); + }; + + None } + fn finish_frame(&mut self, _panicked: bool) { - self.default_precompiles_processor.finish_frame(_panicked); + // there are no revertable precompile yes, so we are ok } } diff --git a/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs b/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs index 2b6b5988e06e..b72651c0a9d1 100644 --- a/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs @@ -59,6 +59,11 @@ pub struct StorageOracle { // Storage refunds that oracle has returned in `estimate_refunds_for_write`. pub(crate) returned_refunds: HistoryRecorder, H>, + + // Keeps track of storage keys that were ever written to. + pub(crate) written_keys: HistoryRecorder, HistoryEnabled>, + // Keeps track of storage keys that were ever read. + pub(crate) read_keys: HistoryRecorder, HistoryEnabled>, } impl OracleWithHistory for StorageOracle { @@ -69,6 +74,8 @@ impl OracleWithHistory for StorageOracle { self.paid_changes.rollback_to_timestamp(timestamp); self.initial_values.rollback_to_timestamp(timestamp); self.returned_refunds.rollback_to_timestamp(timestamp); + self.written_keys.rollback_to_timestamp(timestamp); + self.read_keys.rollback_to_timestamp(timestamp); } } @@ -81,6 +88,8 @@ impl StorageOracle { paid_changes: Default::default(), initial_values: Default::default(), returned_refunds: Default::default(), + written_keys: Default::default(), + read_keys: Default::default(), } } @@ -91,6 +100,8 @@ impl StorageOracle { self.paid_changes.delete_history(); self.initial_values.delete_history(); self.returned_refunds.delete_history(); + self.written_keys.delete_history(); + self.read_keys.delete_history(); } fn is_storage_key_free(&self, key: &StorageKey) -> bool { @@ -108,8 +119,12 @@ impl StorageOracle { } } - pub fn read_value(&mut self, mut query: LogQuery) -> LogQuery { + fn read_value(&mut self, mut query: LogQuery) -> LogQuery { let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + + if !self.read_keys.inner().contains_key(&key) { + self.read_keys.insert(key, (), query.timestamp); + } let current_value = self.storage.read_from_storage(&key); query.read_value = current_value; @@ -127,8 +142,11 @@ impl StorageOracle { query } - pub fn write_value(&mut self, query: LogQuery) -> LogQuery { + fn write_value(&mut self, query: LogQuery) -> LogQuery { let key = triplet_to_storage_key(query.shard_id, query.address, query.key); + if !self.written_keys.inner().contains_key(&key) { + self.written_keys.insert(key, (), query.timestamp); + } let current_value = self.storage .write_to_storage(key, query.written_value, query.timestamp); diff --git a/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs b/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs new file mode 100644 index 000000000000..c76f3dc1c729 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs @@ -0,0 +1,43 @@ +use zksync_types::{Address, Execute, U256}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{constants::BLOCK_GAS_LIMIT, tests::tester::VmTesterBuilder, HistoryEnabled}, +}; + +// Checks that estimated number of circuits for simple transfer doesn't differ much +// from hardcoded expected value. +#[test] +fn test_circuits() { + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: Address::random(), + calldata: Vec::new(), + value: U256::from(1u8), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let res = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + const EXPECTED_CIRCUITS_USED: f32 = 1.5521; + let delta = + (res.statistics.estimated_circuits_used - EXPECTED_CIRCUITS_USED) / EXPECTED_CIRCUITS_USED; + + if delta.abs() > 0.1 { + panic!( + "Estimation differs from expected result by too much: {}%", + delta * 100.0 + ); + } +} diff --git a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs index ffb38dd3725a..95377232b3e3 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs @@ -4,12 +4,14 @@ mod default_aa; // mod invalid_bytecode; mod bytecode_publishing; mod call_tracer; +mod circuits; mod gas_limit; mod get_used_contracts; mod is_write_initial; mod l1_tx_execution; mod l2_blocks; mod nonce_holder; +mod precompiles; mod refunds; mod require_eip712; mod rollbacks; diff --git a/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs b/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs new file mode 100644 index 000000000000..adc62a4bd2d6 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs @@ -0,0 +1,136 @@ +use zk_evm_1_4_0::zk_evm_abstractions::precompiles::PrecompileAddress; +use zksync_types::{Address, Execute}; + +use crate::{ + interface::{TxExecutionMode, VmExecutionMode, VmInterface}, + vm_latest::{ + constants::BLOCK_GAS_LIMIT, + tests::{tester::VmTesterBuilder, utils::read_precompiles_contract}, + HistoryEnabled, + }, +}; + +#[test] +fn test_keccak() { + // Execute special transaction and check that at least 1000 keccak calls were made. + let contract = read_precompiles_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contract, address, true)]) + .build(); + + // calldata for `doKeccak(1000)`. + let keccak1000_calldata = + "370f20ac00000000000000000000000000000000000000000000000000000000000003e8"; + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: hex::decode(keccak1000_calldata).unwrap(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let keccak_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::Keccak256) + .count(); + + assert!(keccak_count >= 1000); +} + +#[test] +fn test_sha256() { + // Execute special transaction and check that at least 1000 sha256 calls were made. + let contract = read_precompiles_contract(); + let address = Address::random(); + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .with_custom_contracts(vec![(contract, address, true)]) + .build(); + + // calldata for `doSha256(1000)`. + let sha1000_calldata = + "5d0b4fb500000000000000000000000000000000000000000000000000000000000003e8"; + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: address, + calldata: hex::decode(sha1000_calldata).unwrap(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let sha_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::SHA256) + .count(); + + assert!(sha_count >= 1000); +} + +#[test] +fn test_ecrecover() { + // Execute simple transfer and check that exactly 1 ecrecover call was made (it's done during tx validation). + let mut vm = VmTesterBuilder::new(HistoryEnabled) + .with_empty_in_memory_storage() + .with_random_rich_accounts(1) + .with_deployer() + .with_gas_limit(BLOCK_GAS_LIMIT) + .with_execution_mode(TxExecutionMode::VerifyExecute) + .build(); + + let account = &mut vm.rich_accounts[0]; + let tx = account.get_l2_tx_for_execute( + Execute { + contract_address: account.address, + calldata: Vec::new(), + value: Default::default(), + factory_deps: None, + }, + None, + ); + vm.vm.push_transaction(tx); + let _ = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); + + let ecrecover_count = vm + .vm + .state + .precompiles_processor + .precompile_cycles_history + .inner() + .iter() + .filter(|(precompile, _)| precompile == &PrecompileAddress::Ecrecover) + .count(); + + assert_eq!(ecrecover_count, 1); +} diff --git a/core/lib/multivm/src/versions/vm_latest/tests/utils.rs b/core/lib/multivm/src/versions/vm_latest/tests/utils.rs index c126b50cb574..7c937033a218 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/utils.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/utils.rs @@ -103,3 +103,9 @@ pub(crate) fn read_max_depth_contract() -> Vec { "core/tests/ts-integration/contracts/zkasm/artifacts/deep_stak.zkasm/deep_stak.zkasm.zbin", ) } + +pub(crate) fn read_precompiles_contract() -> Vec { + read_bytecode( + "etc/contracts-test-data/artifacts-zk/contracts/precompiles/precompiles.sol/Precompiles.json", + ) +} diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs new file mode 100644 index 000000000000..16f5540172a8 --- /dev/null +++ b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs @@ -0,0 +1,85 @@ +use zkevm_test_harness_1_4_0::{geometry_config::get_geometry_config, toolset::GeometryConfig}; + +const GEOMETRY_CONFIG: GeometryConfig = get_geometry_config(); +const OVERESTIMATE_PERCENT: f32 = 1.05; + +const MAIN_VM_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_vm_snapshot as f32; + +const CODE_DECOMMITTER_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_code_decommitter_sorter as f32; + +const LOG_DEMUXER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_log_demuxer as f32; + +const STORAGE_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_storage_sorter as f32; + +const EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_events_or_l1_messages_sorter as f32; + +const RAM_PERMUTATION_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_ram_permutation as f32; + +pub(crate) const CODE_DECOMMITTER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_code_decommitter as f32; + +pub(crate) const STORAGE_APPLICATION_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_storage_application as f32; + +pub(crate) const KECCAK256_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_keccak256_circuit as f32; + +pub(crate) const SHA256_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_sha256_circuit as f32; + +pub(crate) const ECRECOVER_CYCLE_FRACTION: f32 = + OVERESTIMATE_PERCENT / GEOMETRY_CONFIG.cycles_per_ecrecover_circuit as f32; + +// "Rich addressing" opcodes are opcodes that can write their return value/read the input onto the stack +// and so take 1-2 RAM permutations more than an average opcode. +// In the worst case, a rich addressing may take 3 ram permutations +// (1 for reading the opcode, 1 for writing input value, 1 for writing output value). +pub(crate) const RICH_ADDRESSING_OPCODE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 3.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +pub(crate) const AVERAGE_OPCODE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION; + +// Here "base" fraction is a fraction that will be used unconditionally. +// Usage of StorageApplication is being tracked separately as it depends on whether slot was read before or not. +pub(crate) const STORAGE_READ_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + LOG_DEMUXER_CYCLE_FRACTION + + STORAGE_SORTER_CYCLE_FRACTION; + +pub(crate) const EVENT_OR_L1_MESSAGE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + 2.0 * LOG_DEMUXER_CYCLE_FRACTION + + 2.0 * EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION; + +// Here "base" fraction is a fraction that will be used unconditionally. +// Usage of StorageApplication is being tracked separately as it depends on whether slot was written before or not. +pub(crate) const STORAGE_WRITE_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + 2.0 * LOG_DEMUXER_CYCLE_FRACTION + + 2.0 * STORAGE_SORTER_CYCLE_FRACTION; + +pub(crate) const FAR_CALL_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + + RAM_PERMUTATION_CYCLE_FRACTION + + STORAGE_SORTER_CYCLE_FRACTION + + CODE_DECOMMITTER_SORTER_CYCLE_FRACTION; + +// 5 RAM permutations, because: 1 to read opcode + 2 reads + 2 writes. +// 2 reads and 2 writes are needed because unaligned access is implemented with +// aligned queries. +pub(crate) const UMA_WRITE_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 5.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +// 3 RAM permutations, because: 1 to read opcode + 2 reads. +// 2 reads are needed because unaligned access is implemented with aligned queries. +pub(crate) const UMA_READ_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + 3.0 * RAM_PERMUTATION_CYCLE_FRACTION; + +pub(crate) const PRECOMPILE_CALL_COMMON_FRACTION: f32 = + MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + LOG_DEMUXER_CYCLE_FRACTION; diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs new file mode 100644 index 000000000000..c2383b11561f --- /dev/null +++ b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs @@ -0,0 +1,199 @@ +use std::marker::PhantomData; + +use zk_evm_1_4_0::{ + tracing::{BeforeExecutionData, VmLocalStateData}, + zk_evm_abstractions::precompiles::PrecompileAddress, + zkevm_opcode_defs::{LogOpcode, Opcode, UMAOpcode}, +}; +use zksync_state::{StoragePtr, WriteStorage}; + +use super::circuits_capacity::*; +use crate::{ + interface::{dyn_tracers::vm_1_4_0::DynTracer, tracer::TracerExecutionStatus}, + vm_latest::{ + bootloader_state::BootloaderState, + old_vm::{ + history_recorder::{HistoryMode, VectorHistoryEvent}, + memory::SimpleMemory, + }, + tracers::traits::VmTracer, + types::internals::ZkSyncVmState, + }, +}; + +/// Tracer responsible for collecting information about refunds. +#[derive(Debug)] +pub(crate) struct CircuitsTracer { + pub(crate) estimated_circuits_used: f32, + last_decommitment_history_entry_checked: Option, + last_written_keys_history_entry_checked: Option, + last_read_keys_history_entry_checked: Option, + last_precompile_history_entry_checked: Option, + _phantom_data: PhantomData, +} + +impl CircuitsTracer { + pub(crate) fn new() -> Self { + Self { + estimated_circuits_used: 0.0, + last_decommitment_history_entry_checked: None, + last_written_keys_history_entry_checked: None, + last_read_keys_history_entry_checked: None, + last_precompile_history_entry_checked: None, + _phantom_data: Default::default(), + } + } +} + +impl DynTracer> for CircuitsTracer { + fn before_execution( + &mut self, + _state: VmLocalStateData<'_>, + data: BeforeExecutionData, + _memory: &SimpleMemory, + _storage: StoragePtr, + ) { + let used = match data.opcode.variant.opcode { + Opcode::Nop(_) + | Opcode::Add(_) + | Opcode::Sub(_) + | Opcode::Mul(_) + | Opcode::Div(_) + | Opcode::Jump(_) + | Opcode::Binop(_) + | Opcode::Shift(_) + | Opcode::Ptr(_) => RICH_ADDRESSING_OPCODE_FRACTION, + Opcode::Context(_) | Opcode::Ret(_) | Opcode::NearCall(_) => AVERAGE_OPCODE_FRACTION, + Opcode::Log(LogOpcode::StorageRead) => STORAGE_READ_BASE_FRACTION, + Opcode::Log(LogOpcode::StorageWrite) => STORAGE_WRITE_BASE_FRACTION, + Opcode::Log(LogOpcode::ToL1Message) | Opcode::Log(LogOpcode::Event) => { + EVENT_OR_L1_MESSAGE_FRACTION + } + Opcode::Log(LogOpcode::PrecompileCall) => PRECOMPILE_CALL_COMMON_FRACTION, + Opcode::FarCall(_) => FAR_CALL_FRACTION, + Opcode::UMA(UMAOpcode::AuxHeapWrite | UMAOpcode::HeapWrite) => UMA_WRITE_FRACTION, + Opcode::UMA( + UMAOpcode::AuxHeapRead | UMAOpcode::HeapRead | UMAOpcode::FatPointerRead, + ) => UMA_READ_FRACTION, + Opcode::Invalid(_) => unreachable!(), // invalid opcodes are never executed + }; + + self.estimated_circuits_used += used; + } +} + +impl VmTracer for CircuitsTracer { + fn initialize_tracer(&mut self, state: &mut ZkSyncVmState) { + self.last_decommitment_history_entry_checked = Some( + state + .decommittment_processor + .decommitted_code_hashes + .history() + .len(), + ); + + self.last_written_keys_history_entry_checked = + Some(state.storage.written_keys.history().len()); + + self.last_read_keys_history_entry_checked = Some(state.storage.read_keys.history().len()); + + self.last_precompile_history_entry_checked = Some( + state + .precompiles_processor + .precompile_cycles_history + .history() + .len(), + ); + } + + fn finish_cycle( + &mut self, + state: &mut ZkSyncVmState, + _bootloader_state: &mut BootloaderState, + ) -> TracerExecutionStatus { + // Trace decommitments. + let last_decommitment_history_entry_checked = self + .last_decommitment_history_entry_checked + .expect("Value must be set during init"); + let history = state + .decommittment_processor + .decommitted_code_hashes + .history(); + for (_, history_event) in &history[last_decommitment_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + let bytecode_len = state + .decommittment_processor + .known_bytecodes + .inner() + .get(&history_event.key) + .expect("Bytecode must be known at this point") + .len(); + + // Each cycle of `CodeDecommitter` processes 2 words. + // If the number of words in bytecode is odd, then number of cycles must be rounded up. + let decommitter_cycles_used = (bytecode_len + 1) / 2; + self.estimated_circuits_used += + (decommitter_cycles_used as f32) * CODE_DECOMMITTER_CYCLE_FRACTION; + } + self.last_decommitment_history_entry_checked = Some(history.len()); + + // Process storage writes. + let last_writes_history_entry_checked = self + .last_written_keys_history_entry_checked + .expect("Value must be set during init"); + let history = state.storage.written_keys.history(); + for (_, history_event) in &history[last_writes_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + + self.estimated_circuits_used += 2.0 * STORAGE_APPLICATION_CYCLE_FRACTION; + } + self.last_written_keys_history_entry_checked = Some(history.len()); + + // Process storage reads. + let last_reads_history_entry_checked = self + .last_read_keys_history_entry_checked + .expect("Value must be set during init"); + let history = state.storage.read_keys.history(); + for (_, history_event) in &history[last_reads_history_entry_checked..] { + // We assume that only insertions may happen during a single VM inspection. + assert!(history_event.value.is_some()); + + // If the slot is already written to, then we've already taken 2 cycles into account. + if !state + .storage + .written_keys + .inner() + .contains_key(&history_event.key) + { + self.estimated_circuits_used += STORAGE_APPLICATION_CYCLE_FRACTION; + } + } + self.last_read_keys_history_entry_checked = Some(history.len()); + + // Process precompiles. + let last_precompile_history_entry_checked = self + .last_precompile_history_entry_checked + .expect("Value must be set during init"); + let history = state + .precompiles_processor + .precompile_cycles_history + .history(); + for (_, history_event) in &history[last_precompile_history_entry_checked..] { + if let VectorHistoryEvent::Push((precompile, cycles)) = history_event { + let fraction = match precompile { + PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, + PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, + PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, + }; + self.estimated_circuits_used += (*cycles as f32) * fraction; + } else { + panic!("Precompile calls should not be rolled back"); + } + } + self.last_precompile_history_entry_checked = Some(history.len()); + + TracerExecutionStatus::Continue + } +} diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs index 0e18d989af62..018272365f8f 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs @@ -32,7 +32,7 @@ use crate::{ computational_gas_price, gas_spent_on_bytecodes_and_long_messages_this_opcode, print_debug_if_needed, VmHook, }, - RefundsTracer, ResultTracer, + CircuitsTracer, RefundsTracer, ResultTracer, }, types::internals::ZkSyncVmState, VmTracer, @@ -62,6 +62,11 @@ pub(crate) struct DefaultExecutionTracer { pub(crate) pubdata_tracer: Option>, pub(crate) dispatcher: TracerDispatcher, ret_from_the_bootloader: Option, + // This tracer tracks what opcodes were executed and calculates how much circuits will be generated. + // It only takes into account circuits that are generated for actual execution. It doesn't + // take into account e.g circuits produced by the initial bootloader memory commitment. + pub(crate) circuits_tracer: CircuitsTracer, + storage: StoragePtr, _phantom: PhantomData, } @@ -88,6 +93,7 @@ impl DefaultExecutionTracer { dispatcher, pubdata_tracer, ret_from_the_bootloader: None, + circuits_tracer: CircuitsTracer::new(), storage, _phantom: PhantomData, } @@ -161,14 +167,15 @@ impl Debug for DefaultExecutionTracer { /// The macro passes the function call to all tracers. macro_rules! dispatch_tracers { ($self:ident.$function:ident($( $params:expr ),*)) => { - $self.result_tracer.$function($( $params ),*); - $self.dispatcher.$function($( $params ),*); + $self.result_tracer.$function($( $params ),*); + $self.dispatcher.$function($( $params ),*); if let Some(tracer) = &mut $self.refund_tracer { tracer.$function($( $params ),*); } if let Some(tracer) = &mut $self.pubdata_tracer { tracer.$function($( $params ),*); } + $self.circuits_tracer.$function($( $params ),*); }; } diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/mod.rs b/core/lib/multivm/src/versions/vm_latest/tracers/mod.rs index 33d043de6eb1..1bdb1b6ccdbf 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/mod.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/mod.rs @@ -1,13 +1,16 @@ +pub(crate) use circuits_tracer::CircuitsTracer; pub(crate) use default_tracers::DefaultExecutionTracer; pub(crate) use pubdata_tracer::PubdataTracer; pub(crate) use refunds::RefundsTracer; pub(crate) use result_tracer::ResultTracer; +pub(crate) mod circuits_tracer; pub(crate) mod default_tracers; pub(crate) mod pubdata_tracer; pub(crate) mod refunds; pub(crate) mod result_tracer; +mod circuits_capacity; pub mod dispatcher; pub(crate) mod traits; pub(crate) mod utils; diff --git a/core/lib/multivm/src/versions/vm_latest/types/internals/vm_state.rs b/core/lib/multivm/src/versions/vm_latest/types/internals/vm_state.rs index da9c81321440..2430e163fe87 100644 --- a/core/lib/multivm/src/versions/vm_latest/types/internals/vm_state.rs +++ b/core/lib/multivm/src/versions/vm_latest/types/internals/vm_state.rs @@ -40,7 +40,7 @@ pub type ZkSyncVmState = VmState< StorageOracle, SimpleMemory, InMemoryEventSink, - PrecompilesProcessorWithHistory, + PrecompilesProcessorWithHistory, DecommitterOracle, DummyTracer, >; @@ -84,7 +84,7 @@ pub(crate) fn new_vm_state( let storage_oracle: StorageOracle = StorageOracle::new(storage.clone()); let mut memory = SimpleMemory::default(); let event_sink = InMemoryEventSink::default(); - let precompiles_processor = PrecompilesProcessorWithHistory::::default(); + let precompiles_processor = PrecompilesProcessorWithHistory::::default(); let mut decommittment_processor: DecommitterOracle = DecommitterOracle::new(storage); diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/statistics.rs index 3e9de5de4ec0..d64e71c3ff20 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/statistics.rs @@ -40,6 +40,7 @@ impl Vm { computational_gas_used, total_log_queries: total_log_queries_count, pubdata_published, + estimated_circuits_used: 0.0, } } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs index 074e8dae56ed..7657babfe4a4 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs @@ -40,6 +40,7 @@ impl Vm { total_log_queries: total_log_queries_count, // This field will be populated by the RefundTracer pubdata_published: 0, + estimated_circuits_used: 0.0, } } diff --git a/core/lib/state/src/test_utils.rs b/core/lib/state/src/test_utils.rs index 2a5664ae0602..21c56678586a 100644 --- a/core/lib/state/src/test_utils.rs +++ b/core/lib/state/src/test_utils.rs @@ -105,7 +105,7 @@ pub(crate) async fn create_l1_batch( ); header.is_finished = true; conn.blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); conn.blocks_dal() diff --git a/core/lib/types/src/fee.rs b/core/lib/types/src/fee.rs index 53e05fbb59a9..fad4d09f5280 100644 --- a/core/lib/types/src/fee.rs +++ b/core/lib/types/src/fee.rs @@ -24,6 +24,7 @@ pub struct TransactionExecutionMetrics { pub computational_gas_used: u32, pub total_updated_values_size: usize, pub pubdata_published: u32, + pub estimated_circuits_used: f32, } #[derive(Debug, Default, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] diff --git a/core/lib/types/src/tx/tx_execution_info.rs b/core/lib/types/src/tx/tx_execution_info.rs index d19757ee970b..968a56d6c55a 100644 --- a/core/lib/types/src/tx/tx_execution_info.rs +++ b/core/lib/types/src/tx/tx_execution_info.rs @@ -69,6 +69,7 @@ pub struct ExecutionMetrics { pub cycles_used: u32, pub computational_gas_used: u32, pub pubdata_published: u32, + pub estimated_circuits_used: f32, } impl ExecutionMetrics { @@ -86,6 +87,7 @@ impl ExecutionMetrics { cycles_used: tx_metrics.cycles_used, computational_gas_used: tx_metrics.computational_gas_used, pubdata_published: tx_metrics.pubdata_published, + estimated_circuits_used: tx_metrics.estimated_circuits_used, } } @@ -119,6 +121,7 @@ impl Add for ExecutionMetrics { cycles_used: self.cycles_used + other.cycles_used, computational_gas_used: self.computational_gas_used + other.computational_gas_used, pubdata_published: self.pubdata_published + other.pubdata_published, + estimated_circuits_used: self.estimated_circuits_used + other.estimated_circuits_used, } } } diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs index 6842fe438f83..82e082d4dd81 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/vm_metrics.rs @@ -240,5 +240,6 @@ pub(super) fn collect_tx_execution_metrics( computational_gas_used: result.statistics.computational_gas_used, total_updated_values_size: writes_metrics.total_updated_values_size, pubdata_published: result.statistics.pubdata_published, + estimated_circuits_used: result.statistics.estimated_circuits_used, } } diff --git a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs index e3d233777310..70ad7d28fa25 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/snapshots.rs @@ -15,7 +15,7 @@ async fn seal_l1_batch( let header = create_l1_batch(number.0); storage .blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await?; storage .blocks_dal() diff --git a/core/lib/zksync_core/src/consistency_checker/tests/mod.rs b/core/lib/zksync_core/src/consistency_checker/tests/mod.rs index 1dac556ec6a7..e8cc9f2c26cc 100644 --- a/core/lib/zksync_core/src/consistency_checker/tests/mod.rs +++ b/core/lib/zksync_core/src/consistency_checker/tests/mod.rs @@ -195,7 +195,7 @@ impl SaveAction<'_> { Self::InsertBatch(l1_batch) => { storage .blocks_dal() - .insert_l1_batch(&l1_batch.header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&l1_batch.header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); } diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index 62a5808e2b7b..ad9764ce76e1 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -887,7 +887,7 @@ async fn insert_l1_batch(tester: &EthSenderTester, number: L1BatchNumber) -> L1B .storage() .await .blocks_dal() - .insert_l1_batch(&header, &[], Default::default(), &[], &[]) + .insert_l1_batch(&header, &[], Default::default(), &[], &[], 0) .await .unwrap(); tester diff --git a/core/lib/zksync_core/src/genesis.rs b/core/lib/zksync_core/src/genesis.rs index cbd08e329cb0..7172968e0ef3 100644 --- a/core/lib/zksync_core/src/genesis.rs +++ b/core/lib/zksync_core/src/genesis.rs @@ -315,6 +315,7 @@ pub(crate) async fn create_genesis_l1_batch( BlockGasCount::default(), &[], &[], + 0, ) .await .unwrap(); diff --git a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs index ee2fc0bb8a76..1b729a06cd44 100644 --- a/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/recovery/tests.rs @@ -400,7 +400,7 @@ async fn prepare_clean_recovery_snapshot( ); storage .blocks_dal() - .insert_l1_batch(&l1_batch, &[], Default::default(), &[], &[]) + .insert_l1_batch(&l1_batch, &[], Default::default(), &[], &[], 0) .await .unwrap(); diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index eb9a5693a4a4..da158ff11ef9 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -288,6 +288,7 @@ async fn test_postgres_backup_recovery( BlockGasCount::default(), &[], &[], + 0, ) .await .unwrap(); @@ -313,7 +314,7 @@ async fn test_postgres_backup_recovery( for batch_header in &removed_batches { let mut txn = storage.start_transaction().await.unwrap(); txn.blocks_dal() - .insert_l1_batch(batch_header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(batch_header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); insert_initial_writes_for_batch(&mut txn, batch_header.number).await; @@ -496,7 +497,7 @@ pub(super) async fn extend_db_state_from_l1_batch( storage .blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); storage diff --git a/core/lib/zksync_core/src/reorg_detector/tests.rs b/core/lib/zksync_core/src/reorg_detector/tests.rs index dc5d0711445f..f9495286b1f8 100644 --- a/core/lib/zksync_core/src/reorg_detector/tests.rs +++ b/core/lib/zksync_core/src/reorg_detector/tests.rs @@ -36,7 +36,7 @@ async fn seal_l1_batch(storage: &mut StorageProcessor<'_>, number: u32, hash: H2 let header = create_l1_batch(number); storage .blocks_dal() - .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[]) + .insert_l1_batch(&header, &[], BlockGasCount::default(), &[], &[], 0) .await .unwrap(); storage diff --git a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs index 15f10ce3ce7d..908e24fd2d5f 100644 --- a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs +++ b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs @@ -149,6 +149,10 @@ impl UpdatesManager { self.l1_batch.l1_gas_count, &events_queue, &finished_batch.final_execution_state.storage_refunds, + self.l1_batch + .block_execution_metrics + .estimated_circuits_used + .ceil() as u32, ) .await .unwrap(); diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs index 2f2a38577998..3adf1f3bff06 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs @@ -146,7 +146,7 @@ impl Tester { let mut storage = pool.access_storage_tagged("state_keeper").await.unwrap(); storage .blocks_dal() - .insert_l1_batch(&batch_header, &[], Default::default(), &[], &[]) + .insert_l1_batch(&batch_header, &[], Default::default(), &[], &[], 0) .await .unwrap(); storage diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs index f239f108f5ba..cc7ba37ef9c8 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/conditional_sealer.rs @@ -133,12 +133,8 @@ impl SequencerSealer { Box::new(criteria::SlotsCriterion), Box::new(criteria::GasCriterion), Box::new(criteria::PubDataBytesCriterion), - Box::new(criteria::InitialWritesCriterion), - Box::new(criteria::RepeatedWritesCriterion), - Box::new(criteria::MaxCyclesCriterion), - Box::new(criteria::ComputationalGasCriterion), + Box::new(criteria::CircuitsCriterion), Box::new(criteria::TxEncodingSizeCriterion), - Box::new(criteria::L2ToL1LogsCriterion), ] } } diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs index 9f99554e58a2..7878621a7296 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/geometry_seal_criteria.rs @@ -1,12 +1,7 @@ use std::fmt; -use multivm::vm_latest::constants::{ERGS_PER_CIRCUIT, MAX_CYCLES_FOR_TX}; use zksync_config::configs::chain::StateKeeperConfig; -use zksync_types::{ - circuit::{GEOMETRY_CONFIG, SCHEDULER_UPPER_BOUND}, - tx::tx_execution_info::{DeduplicatedWritesMetrics, ExecutionMetrics}, - ProtocolVersionId, -}; +use zksync_types::{tx::tx_execution_info::ExecutionMetrics, ProtocolVersionId}; // Local uses use crate::state_keeper::seal_criteria::{SealCriterion, SealData, SealResolution}; @@ -15,20 +10,12 @@ use crate::state_keeper::seal_criteria::{SealCriterion, SealData, SealResolution // Otherwise witness generation will fail and proof won't be generated. #[derive(Debug, Default)] -pub struct RepeatedWritesCriterion; -#[derive(Debug, Default)] -pub struct InitialWritesCriterion; -#[derive(Debug, Default)] -pub struct MaxCyclesCriterion; -#[derive(Debug, Default)] -pub struct ComputationalGasCriterion; -#[derive(Debug, Default)] -pub struct L2ToL1LogsCriterion; +pub struct CircuitsCriterion; trait MetricExtractor { const PROM_METRIC_CRITERION_NAME: &'static str; fn limit_per_block(protocol_version: ProtocolVersionId) -> usize; - fn extract(metric: &ExecutionMetrics, writes: &DeduplicatedWritesMetrics) -> usize; + fn extract(metric: &ExecutionMetrics) -> usize; } impl SealCriterion for T @@ -51,15 +38,13 @@ where * config.close_block_at_geometry_percentage) .round(); - if T::extract(&tx_data.execution_metrics, &tx_data.writes_metrics) > reject_bound as usize { + if T::extract(&tx_data.execution_metrics) > reject_bound as usize { SealResolution::Unexecutable("ZK proof cannot be generated for a transaction".into()) - } else if T::extract(&block_data.execution_metrics, &block_data.writes_metrics) + } else if T::extract(&block_data.execution_metrics) >= T::limit_per_block(protocol_version_id) { SealResolution::ExcludeAndSeal - } else if T::extract(&block_data.execution_metrics, &block_data.writes_metrics) - > close_bound as usize - { + } else if T::extract(&block_data.execution_metrics) > close_bound as usize { SealResolution::IncludeAndSeal } else { SealResolution::NoSeal @@ -71,85 +56,21 @@ where } } -impl MetricExtractor for RepeatedWritesCriterion { - const PROM_METRIC_CRITERION_NAME: &'static str = "repeated_storage_writes"; - - fn limit_per_block(protocol_version_id: ProtocolVersionId) -> usize { - if protocol_version_id.is_pre_boojum() { - GEOMETRY_CONFIG.limit_for_repeated_writes_pubdata_hasher as usize - } else { - // In boojum there is no limit for repeated writes. - usize::MAX - } - } - - fn extract(_metrics: &ExecutionMetrics, writes: &DeduplicatedWritesMetrics) -> usize { - writes.repeated_storage_writes - } -} - -impl MetricExtractor for InitialWritesCriterion { - const PROM_METRIC_CRITERION_NAME: &'static str = "initial_storage_writes"; - - fn limit_per_block(protocol_version_id: ProtocolVersionId) -> usize { - if protocol_version_id.is_pre_boojum() { - GEOMETRY_CONFIG.limit_for_initial_writes_pubdata_hasher as usize - } else { - // In boojum there is no limit for initial writes. - usize::MAX - } - } - - fn extract(_metrics: &ExecutionMetrics, writes: &DeduplicatedWritesMetrics) -> usize { - writes.initial_storage_writes - } -} - -impl MetricExtractor for MaxCyclesCriterion { - const PROM_METRIC_CRITERION_NAME: &'static str = "max_cycles"; - - fn limit_per_block(_protocol_version_id: ProtocolVersionId) -> usize { - MAX_CYCLES_FOR_TX as usize - } - - fn extract(metrics: &ExecutionMetrics, _writes: &DeduplicatedWritesMetrics) -> usize { - metrics.cycles_used as usize - } -} - -impl MetricExtractor for ComputationalGasCriterion { - const PROM_METRIC_CRITERION_NAME: &'static str = "computational_gas"; +impl MetricExtractor for CircuitsCriterion { + const PROM_METRIC_CRITERION_NAME: &'static str = "circuits"; fn limit_per_block(_protocol_version_id: ProtocolVersionId) -> usize { // We subtract constant to take into account that circuits may be not fully filled. // This constant should be greater than number of circuits types // but we keep it larger to be on the safe side. - const MARGIN_NUMBER_OF_CIRCUITS: usize = 100; - const MAX_NUMBER_OF_MUTLIINSTANCE_CIRCUITS: usize = - SCHEDULER_UPPER_BOUND as usize - MARGIN_NUMBER_OF_CIRCUITS; + const MARGIN_NUMBER_OF_CIRCUITS: usize = 10000; + const MAX_NUMBER_OF_CIRCUITS: usize = (1 << 14) + (1 << 13) - MARGIN_NUMBER_OF_CIRCUITS; - MAX_NUMBER_OF_MUTLIINSTANCE_CIRCUITS * ERGS_PER_CIRCUIT as usize + MAX_NUMBER_OF_CIRCUITS } - fn extract(metrics: &ExecutionMetrics, _writes: &DeduplicatedWritesMetrics) -> usize { - metrics.computational_gas_used as usize - } -} - -impl MetricExtractor for L2ToL1LogsCriterion { - const PROM_METRIC_CRITERION_NAME: &'static str = "l2_to_l1_logs"; - - fn limit_per_block(protocol_version_id: ProtocolVersionId) -> usize { - if protocol_version_id.is_pre_boojum() { - GEOMETRY_CONFIG.limit_for_l1_messages_merklizer as usize - } else { - // In boojum there is no limit for L2 to L1 logs. - usize::MAX - } - } - - fn extract(metrics: &ExecutionMetrics, _writes: &DeduplicatedWritesMetrics) -> usize { - metrics.l2_to_l1_logs + fn extract(metrics: &ExecutionMetrics) -> usize { + metrics.estimated_circuits_used.ceil() as usize } } @@ -167,7 +88,6 @@ mod tests { fn test_no_seal_block_resolution( block_execution_metrics: ExecutionMetrics, - block_writes_metrics: DeduplicatedWritesMetrics, criterion: &dyn SealCriterion, protocol_version: ProtocolVersionId, ) { @@ -178,7 +98,6 @@ mod tests { 0, &SealData { execution_metrics: block_execution_metrics, - writes_metrics: block_writes_metrics, ..SealData::default() }, &SealData::default(), @@ -189,7 +108,6 @@ mod tests { fn test_include_and_seal_block_resolution( block_execution_metrics: ExecutionMetrics, - block_writes_metrics: DeduplicatedWritesMetrics, criterion: &dyn SealCriterion, protocol_version: ProtocolVersionId, ) { @@ -200,7 +118,6 @@ mod tests { 0, &SealData { execution_metrics: block_execution_metrics, - writes_metrics: block_writes_metrics, ..SealData::default() }, &SealData::default(), @@ -211,7 +128,6 @@ mod tests { fn test_exclude_and_seal_block_resolution( block_execution_metrics: ExecutionMetrics, - block_writes_metrics: DeduplicatedWritesMetrics, criterion: &dyn SealCriterion, protocol_version: ProtocolVersionId, ) { @@ -222,7 +138,6 @@ mod tests { 0, &SealData { execution_metrics: block_execution_metrics, - writes_metrics: block_writes_metrics, ..SealData::default() }, &SealData::default(), @@ -233,7 +148,6 @@ mod tests { fn test_unexecutable_tx_resolution( tx_execution_metrics: ExecutionMetrics, - tx_writes_metrics: DeduplicatedWritesMetrics, criterion: &dyn SealCriterion, protocol_version: ProtocolVersionId, ) { @@ -245,7 +159,6 @@ mod tests { &SealData::default(), &SealData { execution_metrics: tx_execution_metrics, - writes_metrics: tx_writes_metrics, ..SealData::default() }, protocol_version, @@ -260,17 +173,11 @@ mod tests { macro_rules! test_scenario_execution_metrics { ($criterion: tt, $metric_name: ident, $metric_type: ty, $protocol_version: expr) => { let config = get_config(); - let writes_metrics = DeduplicatedWritesMetrics::default(); let block_execution_metrics = ExecutionMetrics { $metric_name: ($criterion::limit_per_block($protocol_version) / 2) as $metric_type, ..ExecutionMetrics::default() }; - test_no_seal_block_resolution( - block_execution_metrics, - writes_metrics, - &$criterion, - $protocol_version, - ); + test_no_seal_block_resolution(block_execution_metrics, &$criterion, $protocol_version); let block_execution_metrics = ExecutionMetrics { $metric_name: ($criterion::limit_per_block($protocol_version) - 1) as $metric_type, @@ -279,7 +186,6 @@ mod tests { test_include_and_seal_block_resolution( block_execution_metrics, - writes_metrics, &$criterion, $protocol_version, ); @@ -291,7 +197,6 @@ mod tests { test_exclude_and_seal_block_resolution( block_execution_metrics, - writes_metrics, &$criterion, $protocol_version, ); @@ -304,117 +209,16 @@ mod tests { ..ExecutionMetrics::default() }; - test_unexecutable_tx_resolution( - tx_execution_metrics, - writes_metrics, - &$criterion, - $protocol_version, - ); - }; - } - - macro_rules! test_scenario_writes_metrics { - ($criterion:tt, $metric_name:ident, $metric_type:ty, $protocol_version:expr) => { - let config = get_config(); - let execution_metrics = ExecutionMetrics::default(); - let block_writes_metrics = DeduplicatedWritesMetrics { - $metric_name: ($criterion::limit_per_block($protocol_version) / 2) as $metric_type, - ..Default::default() - }; - test_no_seal_block_resolution( - execution_metrics, - block_writes_metrics, - &$criterion, - $protocol_version, - ); - - let block_writes_metrics = DeduplicatedWritesMetrics { - $metric_name: ($criterion::limit_per_block($protocol_version) - 1) as $metric_type, - ..Default::default() - }; - - test_include_and_seal_block_resolution( - execution_metrics, - block_writes_metrics, - &$criterion, - $protocol_version, - ); - - let block_writes_metrics = DeduplicatedWritesMetrics { - $metric_name: ($criterion::limit_per_block($protocol_version)) as $metric_type, - ..Default::default() - }; - - test_exclude_and_seal_block_resolution( - execution_metrics, - block_writes_metrics, - &$criterion, - $protocol_version, - ); - - let tx_writes_metrics = DeduplicatedWritesMetrics { - $metric_name: ($criterion::limit_per_block($protocol_version) as f64 - * config.reject_tx_at_geometry_percentage - + 1f64) - .round() as $metric_type, - ..Default::default() - }; - - test_unexecutable_tx_resolution( - execution_metrics, - tx_writes_metrics, - &$criterion, - $protocol_version, - ); + test_unexecutable_tx_resolution(tx_execution_metrics, &$criterion, $protocol_version); }; } - #[test] - fn repeated_writes_seal_criterion() { - test_scenario_writes_metrics!( - RepeatedWritesCriterion, - repeated_storage_writes, - usize, - ProtocolVersionId::Version17 - ); - } - - #[test] - fn initial_writes_seal_criterion() { - test_scenario_writes_metrics!( - InitialWritesCriterion, - initial_storage_writes, - usize, - ProtocolVersionId::Version17 - ); - } - - #[test] - fn max_cycles_seal_criterion() { - test_scenario_execution_metrics!( - MaxCyclesCriterion, - cycles_used, - u32, - ProtocolVersionId::Version17 - ); - } - #[test] fn computational_gas_seal_criterion() { test_scenario_execution_metrics!( - ComputationalGasCriterion, - computational_gas_used, - u32, - ProtocolVersionId::Version17 - ); - } - - #[test] - fn l2_to_l1_logs_seal_criterion() { - test_scenario_execution_metrics!( - L2ToL1LogsCriterion, - l2_to_l1_logs, - usize, + CircuitsCriterion, + estimated_circuits_used, + f32, ProtocolVersionId::Version17 ); } diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs index 8e0d89e8e0f2..4e30f2a8b608 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/mod.rs @@ -5,12 +5,7 @@ mod slots; mod tx_encoding_size; pub(in crate::state_keeper) use self::{ - gas::GasCriterion, - geometry_seal_criteria::{ - ComputationalGasCriterion, InitialWritesCriterion, L2ToL1LogsCriterion, MaxCyclesCriterion, - RepeatedWritesCriterion, - }, - pubdata_bytes::PubDataBytesCriterion, - slots::SlotsCriterion, + gas::GasCriterion, geometry_seal_criteria::CircuitsCriterion, + pubdata_bytes::PubDataBytesCriterion, slots::SlotsCriterion, tx_encoding_size::TxEncodingSizeCriterion, }; diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index b9d8da1e7e12..afe11b9367f3 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -146,6 +146,7 @@ pub(super) fn create_execution_result( computational_gas_used: 0, total_log_queries, pubdata_published: 0, + estimated_circuits_used: 0.0, }, refunds: Refunds::default(), } diff --git a/etc/contracts-test-data/contracts/precompiles/precompiles.sol b/etc/contracts-test-data/contracts/precompiles/precompiles.sol new file mode 100644 index 000000000000..d9e23c46a6fa --- /dev/null +++ b/etc/contracts-test-data/contracts/precompiles/precompiles.sol @@ -0,0 +1,21 @@ +// SPDX-License-Identifier: UNLICENSED + +pragma solidity ^0.8.0; + +contract Precompiles { + function doKeccak(uint256 iters) public pure returns (uint256) { + uint256 sum = 0; + for (uint256 i = 0; i < iters; i += 1) { + sum += uint(keccak256(abi.encode(i))) % 256; + } + return sum; + } + + function doSha256(uint256 iters) public pure returns (uint256) { + uint256 sum = 0; + for (uint256 i = 0; i < iters; i += 1) { + sum += uint(sha256(abi.encode(i))) % 256; + } + return sum; + } +} diff --git a/prover/Cargo.lock b/prover/Cargo.lock index 27ac29283485..1ed73bdb4d1d 100644 --- a/prover/Cargo.lock +++ b/prover/Cargo.lock @@ -6671,7 +6671,7 @@ dependencies = [ [[package]] name = "zk_evm_abstractions" version = "0.1.0" -source = "git+https://github.com/matter-labs/era-zk_evm_abstractions.git#7502a661d7d38906d849dcd3e7a15e5848af6581" +source = "git+https://github.com/matter-labs/era-zk_evm_abstractions.git#32dd320953841aa78579d9da08abbc70bcaed175" dependencies = [ "anyhow", "serde", From 3839d39eb6b6d111ec556948c88d1eb9c6ab5e4a Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Fri, 5 Jan 2024 15:57:19 -0600 Subject: [PATCH 22/49] feat: fix spelling in dev comments in `core/lib/multivm` - continued (#682) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Series of PRs: This is a part in a series of PRs aimed at enhancing spelling accuracy in this repository. See:** - [ ] https://github.com/matter-labs/zksync-era/pull/681 - [ ] https://github.com/matter-labs/zksync-era/pull/683 - [ ] https://github.com/matter-labs/zksync-era/pull/684 - [ ] https://github.com/matter-labs/zksync-era/pull/685 Once merged, a final PR will enable the `dev_comments: true` in the cargo-spellcheck config file. ## What ❔ - **Spelling Corrections in `core/lib/multivm`:** This PR focuses on rectifying spelling errors in the developer comments within the `core/lib/multivm` directory. - Updates dictionary ## Why ❔ - **Enhancing Code Quality:** The `core/lib/multivm` directory currently has several spelling mistakes in the developer comments. Correcting these errors will enhance the readability and overall quality of our code documentation. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov --- core/lib/multivm/src/glue/mod.rs | 2 +- .../src/glue/types/vm/tx_execution_mode.rs | 8 +- .../src/glue/types/vm/vm_block_result.rs | 2 +- .../types/vm/vm_partial_execution_result.rs | 4 +- .../src/interface/types/errors/halt.rs | 4 +- .../types/errors/tx_revert_reason.rs | 2 +- .../types/errors/vm_revert_reason.rs | 2 +- core/lib/multivm/src/lib.rs | 1 - .../src/tracers/call_tracer/vm_latest/mod.rs | 2 +- .../call_tracer/vm_refunds_enhancement/mod.rs | 2 +- .../call_tracer/vm_virtual_blocks/mod.rs | 2 +- core/lib/multivm/src/tracers/validator/mod.rs | 12 +- .../vm_1_3_2/errors/tx_revert_reason.rs | 8 +- .../vm_1_3_2/errors/vm_revert_reason.rs | 2 +- .../src/versions/vm_1_3_2/event_sink.rs | 2 +- .../src/versions/vm_1_3_2/history_recorder.rs | 4 +- .../multivm/src/versions/vm_1_3_2/memory.rs | 6 +- .../versions/vm_1_3_2/oracles/decommitter.rs | 2 +- .../src/versions/vm_1_3_2/oracles/mod.rs | 4 +- .../src/versions/vm_1_3_2/oracles/storage.rs | 12 +- .../vm_1_3_2/oracles/tracer/bootloader.rs | 2 +- .../versions/vm_1_3_2/oracles/tracer/call.rs | 8 +- .../vm_1_3_2/oracles/tracer/one_tx.rs | 2 +- .../versions/vm_1_3_2/oracles/tracer/utils.rs | 2 +- .../vm_1_3_2/oracles/tracer/validation.rs | 16 +- .../multivm/src/versions/vm_1_3_2/refunds.rs | 24 +- .../src/versions/vm_1_3_2/tests/bootloader.rs | 546 +++++++++--------- .../src/versions/vm_1_3_2/tests/mod.rs | 2 + .../src/versions/vm_1_3_2/tests/upgrades.rs | 106 ++-- .../src/versions/vm_1_3_2/tests/utils.rs | 34 +- .../src/versions/vm_1_3_2/transaction_data.rs | 71 ++- .../multivm/src/versions/vm_1_3_2/utils.rs | 2 +- .../src/versions/vm_1_3_2/vm_instance.rs | 18 +- .../versions/vm_1_3_2/vm_with_bootloader.rs | 12 +- .../old_vm/oracles/decommitter.rs | 2 +- .../vm_latest/bootloader_state/l2_block.rs | 2 +- .../vm_latest/bootloader_state/utils.rs | 4 +- .../vm_latest/implementation/snapshots.rs | 4 +- .../versions/vm_latest/old_vm/event_sink.rs | 2 +- .../vm_latest/old_vm/history_recorder.rs | 4 +- .../src/versions/vm_latest/old_vm/memory.rs | 6 +- .../vm_latest/old_vm/oracles/decommitter.rs | 2 +- .../src/versions/vm_latest/old_vm/utils.rs | 2 +- .../src/versions/vm_latest/oracles/storage.rs | 12 +- .../vm_latest/tests/get_used_contracts.rs | 6 +- .../vm_latest/tests/l1_tx_execution.rs | 6 +- .../src/versions/vm_latest/tests/mod.rs | 2 +- .../versions/vm_latest/tests/nonce_holder.rs | 2 +- .../vm_latest/tests/require_eip712.rs | 8 +- .../vm_latest/tests/tester/vm_tester.rs | 2 +- .../src/versions/vm_latest/tests/upgrade.rs | 14 +- .../vm_latest/tracers/default_tracers.rs | 4 +- .../vm_latest/tracers/pubdata_tracer.rs | 2 +- .../src/versions/vm_latest/tracers/refunds.rs | 4 +- .../vm_latest/tracers/result_tracer.rs | 4 +- .../src/versions/vm_latest/tracers/utils.rs | 2 +- .../vm_latest/types/internals/pubdata.rs | 8 +- .../types/internals/transaction_data.rs | 2 +- .../src/versions/vm_latest/utils/fee.rs | 2 +- .../src/versions/vm_latest/utils/l2_blocks.rs | 2 +- .../src/versions/vm_latest/utils/overhead.rs | 69 ++- .../versions/vm_m5/errors/tx_revert_reason.rs | 8 +- .../src/versions/vm_m5/history_recorder.rs | 6 +- core/lib/multivm/src/versions/vm_m5/memory.rs | 8 +- .../multivm/src/versions/vm_m5/oracles/mod.rs | 4 +- .../src/versions/vm_m5/oracles/storage.rs | 4 +- .../src/versions/vm_m5/oracles/tracer.rs | 20 +- .../lib/multivm/src/versions/vm_m5/refunds.rs | 24 +- .../src/versions/vm_m5/tests/bootloader.rs | 422 +++++++------- .../src/versions/vm_m5/transaction_data.rs | 76 +-- core/lib/multivm/src/versions/vm_m5/utils.rs | 2 +- core/lib/multivm/src/versions/vm_m5/vm.rs | 2 +- .../multivm/src/versions/vm_m5/vm_instance.rs | 14 +- .../src/versions/vm_m5/vm_with_bootloader.rs | 12 +- .../versions/vm_m6/errors/tx_revert_reason.rs | 8 +- .../versions/vm_m6/errors/vm_revert_reason.rs | 2 +- .../src/versions/vm_m6/history_recorder.rs | 4 +- core/lib/multivm/src/versions/vm_m6/memory.rs | 8 +- .../src/versions/vm_m6/oracles/decommitter.rs | 2 +- .../multivm/src/versions/vm_m6/oracles/mod.rs | 4 +- .../src/versions/vm_m6/oracles/storage.rs | 8 +- .../vm_m6/oracles/tracer/bootloader.rs | 2 +- .../src/versions/vm_m6/oracles/tracer/call.rs | 8 +- .../versions/vm_m6/oracles/tracer/one_tx.rs | 2 +- .../versions/vm_m6/oracles/tracer/utils.rs | 2 +- .../vm_m6/oracles/tracer/validation.rs | 16 +- .../lib/multivm/src/versions/vm_m6/refunds.rs | 24 +- .../src/versions/vm_m6/tests/bootloader.rs | 544 ++++++++--------- .../src/versions/vm_m6/transaction_data.rs | 77 +-- core/lib/multivm/src/versions/vm_m6/utils.rs | 2 +- core/lib/multivm/src/versions/vm_m6/vm.rs | 2 +- .../multivm/src/versions/vm_m6/vm_instance.rs | 18 +- .../src/versions/vm_m6/vm_with_bootloader.rs | 12 +- .../bootloader_state/l2_block.rs | 2 +- .../bootloader_state/utils.rs | 4 +- .../implementation/snapshots.rs | 4 +- .../old_vm/event_sink.rs | 2 +- .../old_vm/history_recorder.rs | 4 +- .../vm_refunds_enhancement/old_vm/memory.rs | 6 +- .../old_vm/oracles/decommitter.rs | 2 +- .../vm_refunds_enhancement/old_vm/utils.rs | 2 +- .../vm_refunds_enhancement/oracles/storage.rs | 14 +- .../tracers/default_tracers.rs | 4 +- .../vm_refunds_enhancement/tracers/refunds.rs | 4 +- .../tracers/result_tracer.rs | 4 +- .../vm_refunds_enhancement/tracers/utils.rs | 2 +- .../types/internals/transaction_data.rs | 2 +- .../vm_refunds_enhancement/utils/fee.rs | 2 +- .../vm_refunds_enhancement/utils/l2_blocks.rs | 2 +- .../vm_refunds_enhancement/utils/overhead.rs | 78 +-- .../bootloader_state/l2_block.rs | 2 +- .../bootloader_state/utils.rs | 4 +- .../implementation/snapshots.rs | 4 +- .../implementation/statistics.rs | 2 +- .../vm_virtual_blocks/old_vm/event_sink.rs | 2 +- .../old_vm/history_recorder.rs | 4 +- .../vm_virtual_blocks/old_vm/memory.rs | 6 +- .../old_vm/oracles/decommitter.rs | 2 +- .../old_vm/oracles/storage.rs | 12 +- .../vm_virtual_blocks/old_vm/utils.rs | 2 +- .../tracers/default_tracers.rs | 2 +- .../vm_virtual_blocks/tracers/refunds.rs | 4 +- .../tracers/result_tracer.rs | 4 +- .../vm_virtual_blocks/tracers/utils.rs | 2 +- .../types/internals/transaction_data.rs | 2 +- .../versions/vm_virtual_blocks/utils/fee.rs | 2 +- .../vm_virtual_blocks/utils/l2_blocks.rs | 2 +- .../vm_virtual_blocks/utils/overhead.rs | 62 +- spellcheck/era.dic | 251 +++++++- 129 files changed, 1632 insertions(+), 1331 deletions(-) diff --git a/core/lib/multivm/src/glue/mod.rs b/core/lib/multivm/src/glue/mod.rs index 299093532bd4..5347b79d3c2f 100644 --- a/core/lib/multivm/src/glue/mod.rs +++ b/core/lib/multivm/src/glue/mod.rs @@ -29,7 +29,7 @@ pub trait GlueInto: Sized { fn glue_into(self) -> T; } -// Blaknet `GlueInto` impl for any type that implements `GlueFrom`. +// Blanket `GlueInto` impl for any type that implements `GlueFrom`. impl GlueInto for T where U: GlueFrom, diff --git a/core/lib/multivm/src/glue/types/vm/tx_execution_mode.rs b/core/lib/multivm/src/glue/types/vm/tx_execution_mode.rs index 1dd90e104a5a..0709b13de782 100644 --- a/core/lib/multivm/src/glue/types/vm/tx_execution_mode.rs +++ b/core/lib/multivm/src/glue/types/vm/tx_execution_mode.rs @@ -19,12 +19,12 @@ impl GlueFrom match value { crate::interface::TxExecutionMode::VerifyExecute => Self::VerifyExecute, crate::interface::TxExecutionMode::EstimateFee => Self::EstimateFee { - // We used it only for api services we don't have limit for storage invocation inside statekeeper + // We used it only for API services we don't have limit for storage invocation inside statekeeper // It's impossible to recover this value for the vm integration after virtual blocks missed_storage_invocation_limit: usize::MAX, }, crate::interface::TxExecutionMode::EthCall => Self::EthCall { - // We used it only for api services we don't have limit for storage invocation inside statekeeper + // We used it only for API services we don't have limit for storage invocation inside statekeeper // It's impossible to recover this value for the vm integration after virtual blocks missed_storage_invocation_limit: usize::MAX, }, @@ -39,12 +39,12 @@ impl GlueFrom match value { crate::interface::TxExecutionMode::VerifyExecute => Self::VerifyExecute, crate::interface::TxExecutionMode::EstimateFee => Self::EstimateFee { - // We used it only for api services we don't have limit for storage invocation inside statekeeper + // We used it only for API services we don't have limit for storage invocation inside statekeeper // It's impossible to recover this value for the vm integration after virtual blocks missed_storage_invocation_limit: usize::MAX, }, crate::interface::TxExecutionMode::EthCall => Self::EthCall { - // We used it only for api services we don't have limit for storage invocation inside statekeeper + // We used it only for API services we don't have limit for storage invocation inside statekeeper // It's impossible to recover this value for the vm integration after virtual blocks missed_storage_invocation_limit: usize::MAX, }, diff --git a/core/lib/multivm/src/glue/types/vm/vm_block_result.rs b/core/lib/multivm/src/glue/types/vm/vm_block_result.rs index f7eda05cc02d..cc76ce22ca07 100644 --- a/core/lib/multivm/src/glue/types/vm/vm_block_result.rs +++ b/core/lib/multivm/src/glue/types/vm/vm_block_result.rs @@ -8,7 +8,7 @@ use crate::{ }, }; -// Note: In version after vm VmVirtualBlocks the bootloader memory knowledge is encapsulated into the VM. +// Note: In version after vm `VmVirtualBlocks` the bootloader memory knowledge is encapsulated into the VM. // But before it was a part of a public API. // Bootloader memory required only for producing witnesses, // and server doesn't need to generate witnesses for old blocks diff --git a/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs b/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs index 6a52c7e66987..7b25c1ff3e05 100644 --- a/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs +++ b/core/lib/multivm/src/glue/types/vm/vm_partial_execution_result.rs @@ -11,9 +11,9 @@ impl GlueFrom contracts_used: value.contracts_used, cycles_used: value.cycles_used, total_log_queries: value.logs.total_log_queries_count, - // There are no such fields in m5 + // There are no such fields in `m5` gas_used: 0, - // There are no such fields in m5 + // There are no such fields in `m5` computational_gas_used: 0, pubdata_published: 0, estimated_circuits_used: 0.0, diff --git a/core/lib/multivm/src/interface/types/errors/halt.rs b/core/lib/multivm/src/interface/types/errors/halt.rs index 3323a128c682..c302467dacee 100644 --- a/core/lib/multivm/src/interface/types/errors/halt.rs +++ b/core/lib/multivm/src/interface/types/errors/halt.rs @@ -7,7 +7,7 @@ use super::VmRevertReason; /// Differentiates VM-specific issues from contract-related errors. #[derive(Debug, Clone, PartialEq)] pub enum Halt { - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` ValidationFailed(VmRevertReason), PaymasterValidationFailed(VmRevertReason), PrePaymasterPreparationFailed(VmRevertReason), @@ -16,7 +16,7 @@ pub enum Halt { FailedToChargeFee(VmRevertReason), // Emitted when trying to call a transaction from an account that has not // been deployed as an account (i.e. the `from` is just a contract). - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` FromIsNotAnAccount, // Currently cannot be returned. Should be removed when refactoring errors. InnerTxError, diff --git a/core/lib/multivm/src/interface/types/errors/tx_revert_reason.rs b/core/lib/multivm/src/interface/types/errors/tx_revert_reason.rs index 9578a06ea0a5..d863e387e019 100644 --- a/core/lib/multivm/src/interface/types/errors/tx_revert_reason.rs +++ b/core/lib/multivm/src/interface/types/errors/tx_revert_reason.rs @@ -55,7 +55,7 @@ impl TxRevertReason { BootloaderErrorCode::UnacceptablePubdataPrice => { Self::Halt(Halt::UnexpectedVMBehavior("UnacceptablePubdataPrice".to_owned())) } - // This is different from AccountTxValidationFailed error in a way that it means that + // This is different from `AccountTxValidationFailed` error in a way that it means that // the error was not produced by the account itself, but for some other unknown reason (most likely not enough gas) BootloaderErrorCode::TxValidationError => Self::Halt(Halt::ValidationFailed(revert_reason)), // Note, that `InnerTxError` is derived only after the actual tx execution, so diff --git a/core/lib/multivm/src/interface/types/errors/vm_revert_reason.rs b/core/lib/multivm/src/interface/types/errors/vm_revert_reason.rs index 4a6457491264..25b394ce2582 100644 --- a/core/lib/multivm/src/interface/types/errors/vm_revert_reason.rs +++ b/core/lib/multivm/src/interface/types/errors/vm_revert_reason.rs @@ -68,7 +68,7 @@ impl VmRevertReason { pub fn to_user_friendly_string(&self) -> String { match self { - // In case of `Unknown` reason we suppress it to prevent verbose Error function_selector = 0x{} + // In case of `Unknown` reason we suppress it to prevent verbose `Error function_selector = 0x{}` // message shown to user. VmRevertReason::Unknown { .. } => "".to_owned(), _ => self.to_string(), diff --git a/core/lib/multivm/src/lib.rs b/core/lib/multivm/src/lib.rs index bed4586d428e..a2327eba6f89 100644 --- a/core/lib/multivm/src/lib.rs +++ b/core/lib/multivm/src/lib.rs @@ -1,4 +1,3 @@ -// #![deny(unreachable_pub)] #![deny(unused_crate_dependencies)] #![warn(unused_extern_crates)] #![warn(unused_imports)] diff --git a/core/lib/multivm/src/tracers/call_tracer/vm_latest/mod.rs b/core/lib/multivm/src/tracers/call_tracer/vm_latest/mod.rs index 09b5b828fc03..1c6221f1bf7d 100644 --- a/core/lib/multivm/src/tracers/call_tracer/vm_latest/mod.rs +++ b/core/lib/multivm/src/tracers/call_tracer/vm_latest/mod.rs @@ -139,7 +139,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { diff --git a/core/lib/multivm/src/tracers/call_tracer/vm_refunds_enhancement/mod.rs b/core/lib/multivm/src/tracers/call_tracer/vm_refunds_enhancement/mod.rs index 3bc4426e8cc2..6a97d791e8e0 100644 --- a/core/lib/multivm/src/tracers/call_tracer/vm_refunds_enhancement/mod.rs +++ b/core/lib/multivm/src/tracers/call_tracer/vm_refunds_enhancement/mod.rs @@ -140,7 +140,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { diff --git a/core/lib/multivm/src/tracers/call_tracer/vm_virtual_blocks/mod.rs b/core/lib/multivm/src/tracers/call_tracer/vm_virtual_blocks/mod.rs index f96b2cb0f58f..f1713fc5e9ff 100644 --- a/core/lib/multivm/src/tracers/call_tracer/vm_virtual_blocks/mod.rs +++ b/core/lib/multivm/src/tracers/call_tracer/vm_virtual_blocks/mod.rs @@ -138,7 +138,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { diff --git a/core/lib/multivm/src/tracers/validator/mod.rs b/core/lib/multivm/src/tracers/validator/mod.rs index 36d767dac632..aef11924af87 100644 --- a/core/lib/multivm/src/tracers/validator/mod.rs +++ b/core/lib/multivm/src/tracers/validator/mod.rs @@ -102,7 +102,7 @@ impl ValidationTracer { return true; } - // The pair of MSG_VALUE_SIMULATOR_ADDRESS & L2_ETH_TOKEN_ADDRESS simulates the behavior of transferring ETH + // The pair of `MSG_VALUE_SIMULATOR_ADDRESS` & `L2_ETH_TOKEN_ADDRESS` simulates the behavior of transferring ETH // that is safe for the DDoS protection rules. if valid_eth_token_call(address, msg_sender) { return true; @@ -146,11 +146,11 @@ impl ValidationTracer { let (potential_address_bytes, potential_position_bytes) = calldata.split_at(32); let potential_address = be_bytes_to_safe_address(potential_address_bytes); - // If the validation_address is equal to the potential_address, - // then it is a request that could be used for mapping of kind mapping(address => ...). + // If the `validation_address` is equal to the `potential_address`, + // then it is a request that could be used for mapping of kind `mapping(address => ...).` // - // If the potential_position_bytes were already allowed before, then this keccak might be used - // for ERC-20 allowance or any other of mapping(address => mapping(...)) + // If the `potential_position_bytes` were already allowed before, then this keccak might be used + // for ERC-20 allowance or any other of `mapping(address => mapping(...))` if potential_address == Some(validated_address) || self .auxilary_allowed_slots @@ -188,7 +188,7 @@ fn touches_allowed_context(address: Address, key: U256) -> bool { return false; } - // Only chain_id is allowed to be touched. + // Only `chain_id` is allowed to be touched. key == U256::from(0u32) } diff --git a/core/lib/multivm/src/versions/vm_1_3_2/errors/tx_revert_reason.rs b/core/lib/multivm/src/versions/vm_1_3_2/errors/tx_revert_reason.rs index 4775d8339f79..3ddaa0684614 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/errors/tx_revert_reason.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/errors/tx_revert_reason.rs @@ -7,11 +7,11 @@ use super::{BootloaderErrorCode, VmRevertReason}; // Reasons why the transaction executed inside the bootloader could fail. #[derive(Debug, Clone, PartialEq)] pub enum TxRevertReason { - // Can only be returned in EthCall execution mode (=ExecuteOnly) + // Can only be returned in EthCall execution mode `(=ExecuteOnly)` EthCall(VmRevertReason), // Returned when the execution of an L2 transaction has failed TxReverted(VmRevertReason), - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` ValidationFailed(VmRevertReason), PaymasterValidationFailed(VmRevertReason), PrePaymasterPreparationFailed(VmRevertReason), @@ -20,7 +20,7 @@ pub enum TxRevertReason { FailedToChargeFee(VmRevertReason), // Emitted when trying to call a transaction from an account that has not // been deployed as an account (i.e. the `from` is just a contract). - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` FromIsNotAnAccount, // Currently cannot be returned. Should be removed when refactoring errors. InnerTxError, @@ -101,7 +101,7 @@ impl TxRevertReason { BootloaderErrorCode::UnacceptablePubdataPrice => { Self::UnexpectedVMBehavior("UnacceptablePubdataPrice".to_owned()) } - // This is different from AccountTxValidationFailed error in a way that it means that + // This is different from `AccountTxValidationFailed` error in a way that it means that // the error was not produced by the account itself, but for some other unknown reason (most likely not enough gas) BootloaderErrorCode::TxValidationError => Self::ValidationFailed(revert_reason), // Note, that `InnerTxError` is derived only after the actual tx execution, so diff --git a/core/lib/multivm/src/versions/vm_1_3_2/errors/vm_revert_reason.rs b/core/lib/multivm/src/versions/vm_1_3_2/errors/vm_revert_reason.rs index 70c954425f46..ed17ffc4c39b 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/errors/vm_revert_reason.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/errors/vm_revert_reason.rs @@ -73,7 +73,7 @@ impl VmRevertReason { pub fn to_user_friendly_string(&self) -> String { match self { - // In case of `Unknown` reason we suppress it to prevent verbose Error function_selector = 0x{} + // In case of `Unknown` reason we suppress it to prevent verbose `Error function_selector = 0x{}` // message shown to user. VmRevertReason::Unknown { .. } => "".to_owned(), _ => self.to_string(), diff --git a/core/lib/multivm/src/versions/vm_1_3_2/event_sink.rs b/core/lib/multivm/src/versions/vm_1_3_2/event_sink.rs index cbf7c183d3a7..b9aea7e09afc 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/event_sink.rs @@ -50,7 +50,7 @@ impl InMemoryEventSink { pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { let events = self.frames_stack.forward().current_frame(); - // Select all of the last elements where e.timestamp >= from_timestamp. + // Select all of the last elements where `e.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. events .rsplit(|e| e.timestamp < from_timestamp) diff --git a/core/lib/multivm/src/versions/vm_1_3_2/history_recorder.rs b/core/lib/multivm/src/versions/vm_1_3_2/history_recorder.rs index 263c1f023dd2..bb3c12580c4f 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/history_recorder.rs @@ -12,14 +12,14 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; pub type MemoryWithHistory = HistoryRecorder; pub type IntFrameManagerWithHistory = HistoryRecorder, H>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` // can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_1_3_2/memory.rs b/core/lib/multivm/src/versions/vm_1_3_2/memory.rs index 91fdbe223d8b..c9f97c0c2254 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/memory.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/memory.rs @@ -282,7 +282,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -299,7 +299,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -307,7 +307,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs index d58b501b244d..8bf0e70026b8 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_1_3_2::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/mod.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/mod.rs index 08eb1c6e1747..59b0601e1483 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/mod.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/mod.rs @@ -1,9 +1,9 @@ use zk_evm_1_3_3::aux_structures::Timestamp; // All the changes to the events in the DB will be applied after the tx is executed, -// so fow now it is fine. +// so for now it is fine. pub use zk_evm_1_3_3::reference_impls::event_sink::InMemoryEventSink as EventSinkOracle; // We will discard RAM as soon as the execution of a tx ends, so -// it is ok for now to use SimpleMemory +// it is ok for now to use `SimpleMemory` pub use zk_evm_1_3_3::reference_impls::memory::SimpleMemory as RamOracle; pub use zk_evm_1_3_3::testing::simple_tracer::NoopTracer; diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/storage.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/storage.rs index ea2ecf83a3d6..745dcad5050e 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/storage.rs @@ -167,7 +167,7 @@ impl StorageOracle { ) -> &[Box] { let logs = self.frames_stack.forward().current_frame(); - // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Select all of the last elements where `l.log_query.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. logs.rsplit(|l| l.log_query.timestamp < from_timestamp) .next() @@ -208,13 +208,14 @@ impl StorageOracle { } impl VmStorageOracle for StorageOracle { - // Perform a storage read/write access by taking an partially filled query + // Perform a storage read / write access by taking an partially filled query // and returning filled query and cold/warm marker for pricing purposes fn execute_partial_query( &mut self, _monotonic_cycle_counter: u32, query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -224,6 +225,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -303,7 +305,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -317,8 +319,8 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). -// While new_value is always 32 bytes long, for key we use the following optimization: +// we basically need to publish the following pair: `()`. +// While `new_value` is always 32 bytes long, for key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. // - The second time we publish it, we will use this 8-byte instead of the 32 bytes of the entire key. diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/bootloader.rs index 3bc80f311194..fac4a74a1ebd 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/bootloader.rs @@ -100,7 +100,7 @@ impl PubdataSpentTracer for BootloaderTracer { impl BootloaderTracer { fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/call.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/call.rs index 88b21818fc34..3f31d7b71234 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/call.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/call.rs @@ -188,7 +188,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { @@ -255,8 +255,8 @@ impl CallTracer { // Filter all near calls from the call stack // Important that the very first call is near call - // And this NearCall includes several Normal or Mimic calls - // So we return all childrens of this NearCall + // And this `NearCall` includes several Normal or Mimic calls + // So we return all children of this `NearCall` pub fn extract_calls(&mut self) -> Vec { if let Some(current_call) = self.stack.pop() { filter_near_call(current_call) @@ -267,7 +267,7 @@ impl CallTracer { } // Filter all near calls from the call stack -// Normally wr are not interested in NearCall, because it's just a wrapper for internal calls +// Normally we are not interested in `NearCall`, because it's just a wrapper for internal calls fn filter_near_call(mut call: Call) -> Vec { let mut calls = vec![]; let original_calls = std::mem::take(&mut call.calls); diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/one_tx.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/one_tx.rs index 896befb8abc5..9bf5a9b7d224 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/one_tx.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/one_tx.rs @@ -19,7 +19,7 @@ use crate::vm_1_3_2::{ }; /// Allows any opcodes, but tells the VM to end the execution once the tx is over. -// Internally depeds on Bootloader's VMHooks to get the notification once the transaction is finished. +// Internally depends on Bootloader's `VMHooks` to get the notification once the transaction is finished. #[derive(Debug)] pub struct OneTxTracer { tx_has_been_processed: bool, diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/utils.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/utils.rs index 76890b042de1..5ee8d8554b65 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/utils.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/utils.rs @@ -48,7 +48,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/validation.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/validation.rs index d3308c7ea2dc..caea2688bb99 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/validation.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/tracer/validation.rs @@ -100,7 +100,7 @@ fn touches_allowed_context(address: Address, key: U256) -> bool { return false; } - // Only chain_id is allowed to be touched. + // Only `chain_id` is allowed to be touched. key == U256::from(0u32) } @@ -223,7 +223,7 @@ impl ValidationTracer { return true; } - // The pair of MSG_VALUE_SIMULATOR_ADDRESS & L2_ETH_TOKEN_ADDRESS simulates the behavior of transferring ETH + // The pair of `MSG_VALUE_SIMULATOR_ADDRESS` & `L2_ETH_TOKEN_ADDRESS` simulates the behavior of transferring ETH // that is safe for the DDoS protection rules. if valid_eth_token_call(address, msg_sender) { return true; @@ -267,20 +267,20 @@ impl ValidationTracer { let (potential_address_bytes, potential_position_bytes) = calldata.split_at(32); let potential_address = be_bytes_to_safe_address(potential_address_bytes); - // If the validation_address is equal to the potential_address, - // then it is a request that could be used for mapping of kind mapping(address => ...). + // If the `validation_address` is equal to the `potential_address`, + // then it is a request that could be used for mapping of kind `mapping(address => ...)`. // - // If the potential_position_bytes were already allowed before, then this keccak might be used - // for ERC-20 allowance or any other of mapping(address => mapping(...)) + // If the `potential_position_bytes` were already allowed before, then this keccak might be used + // for ERC-20 allowance or any other of `mapping(address => mapping(...))` if potential_address == Some(validated_address) || self .auxilary_allowed_slots .contains(&H256::from_slice(potential_position_bytes)) { - // This is request that could be used for mapping of kind mapping(address => ...) + // This is request that could be used for mapping of kind `mapping(address => ...)` // We could theoretically wait for the slot number to be returned by the - // keccak256 precompile itself, but this would complicate the code even further + // `keccak256` precompile itself, but this would complicate the code even further // so let's calculate it here. let slot = keccak256(calldata); diff --git a/core/lib/multivm/src/versions/vm_1_3_2/refunds.rs b/core/lib/multivm/src/versions/vm_1_3_2/refunds.rs index 9de2ee9676bf..555dd0f643ea 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/refunds.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/refunds.rs @@ -76,7 +76,7 @@ impl VmInstance { ) -> u32 { // TODO (SMA-1715): Make users pay for the block overhead 0 - + // ``` // let pubdata_published = self.pubdata_published(from_timestamp); // // let total_gas_spent = gas_remaining_before - self.gas_remaining(); @@ -121,6 +121,7 @@ impl VmInstance { // ); // 0 // } + // ``` } // TODO (SMA-1715): Make users pay for the block overhead @@ -134,39 +135,39 @@ impl VmInstance { _l2_l1_logs: usize, ) -> u32 { 0 - + // ``` // let overhead_for_block_gas = U256::from(crate::transaction_data::block_overhead_gas( // gas_per_pubdata_byte_limit, // )); - + // // let encoded_len = U256::from(encoded_len); // let pubdata_published = U256::from(pubdata_published); // let gas_spent_on_computation = U256::from(gas_spent_on_computation); // let number_of_decommitment_requests = U256::from(number_of_decommitment_requests); // let l2_l1_logs = U256::from(l2_l1_logs); - + // // let tx_slot_overhead = ceil_div_u256(overhead_for_block_gas, MAX_TXS_IN_BLOCK.into()); - + // // let overhead_for_length = ceil_div_u256( // encoded_len * overhead_for_block_gas, // BOOTLOADER_TX_ENCODING_SPACE.into(), // ); - + // // let actual_overhead_for_pubdata = ceil_div_u256( // pubdata_published * overhead_for_block_gas, // MAX_PUBDATA_PER_BLOCK.into(), // ); - + // // let actual_gas_limit_overhead = ceil_div_u256( // gas_spent_on_computation * overhead_for_block_gas, // MAX_BLOCK_MULTIINSTANCE_GAS_LIMIT.into(), // ); - + // // let code_decommitter_sorter_circuit_overhead = ceil_div_u256( // number_of_decommitment_requests * overhead_for_block_gas, // GEOMETRY_CONFIG.limit_for_code_decommitter_sorter.into(), // ); - + // // let l1_l2_logs_overhead = ceil_div_u256( // l2_l1_logs * overhead_for_block_gas, // std::cmp::min( @@ -175,7 +176,7 @@ impl VmInstance { // ) // .into(), // ); - + // // let overhead = vec![ // tx_slot_overhead, // overhead_for_length, @@ -187,8 +188,9 @@ impl VmInstance { // .into_iter() // .max() // .unwrap(); - + // // overhead.as_u32() + // ``` } /// Returns the given transactions' gas limit - by reading it directly from the VM memory. diff --git a/core/lib/multivm/src/versions/vm_1_3_2/tests/bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/tests/bootloader.rs index b42c17363b0a..2e5b55c945dc 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/tests/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/tests/bootloader.rs @@ -1,3 +1,4 @@ +// ``` // //! // //! Tests for the bootloader // //! The description for each of the tests can be found in the corresponding `.yul` file. @@ -8,7 +9,7 @@ // convert::{TryFrom, TryInto}, // }; // use zksync_eth_signer::{raw_ethereum_tx::TransactionParameters, EthereumSigner, PrivateKeySigner}; - +// // use crate::{ // errors::VmRevertReason, // history_recorder::HistoryMode, @@ -37,7 +38,7 @@ // }, // HistoryEnabled, OracleTools, TxRevertReason, VmBlockResult, VmExecutionResult, VmInstance, // }; - +// // use zk_evm_1_3_3::{ // aux_structures::Timestamp, block_properties::BlockProperties, zkevm_opcode_defs::FarCallOpcode, // }; @@ -69,11 +70,11 @@ // test_utils::LoadnextContractExecutionParams, // {bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}, // }; - +// // use zksync_contracts::{ // get_loadnext_contract, load_contract, SystemContractCode, PLAYGROUND_BLOCK_BOOTLOADER_CODE, // }; - +// // use super::utils::{read_many_owners_custom_account_contract, read_nonce_holder_tester}; // /// Helper struct for tests, that takes care of setting the database and provides some functions to get and set balances. // /// Example use: @@ -92,7 +93,7 @@ // pub block_properties: BlockProperties, // pub storage_ptr: Box>, // } - +// // impl VmTestEnv { // /// Creates a new test helper with a bunch of already deployed contracts. // pub fn new_with_contracts(contracts: &[(H160, Vec)]) -> Self { @@ -100,39 +101,39 @@ // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let mut raw_storage = InMemoryStorage::with_system_contracts(hash_bytecode); // for (address, bytecode) in contracts { // let account = DeployedContract { // account_id: AccountTreeId::new(*address), // bytecode: bytecode.clone(), // }; - +// // insert_contracts(&mut raw_storage, vec![(account, true)]); // } - +// // let storage_ptr = Box::new(StorageView::new(raw_storage)); - +// // VmTestEnv { // block_context, // block_properties, // storage_ptr, // } // } - +// // /// Gets the current ETH balance for a given account. // pub fn get_eth_balance(&mut self, address: &H160) -> U256 { // get_eth_balance(address, self.storage_ptr.as_mut()) // } - +// // /// Sets a large balance for a given account. // pub fn set_rich_account(&mut self, address: &H160) { // let key = storage_key_for_eth_balance(address); - +// // self.storage_ptr // .set_value(key, u256_to_h256(U256::from(10u64.pow(19)))); // } - +// // /// Runs a given transaction in a VM. // // Note: that storage changes will be preserved, but not changed to events etc. // // Strongly suggest to use this function only if this is the only transaction executed within the test. @@ -146,7 +147,7 @@ // ); // (result, tx_has_failed) // } - +// // /// Runs a given transaction in a VM and asserts if it fails. // pub fn run_vm_or_die(&mut self, transaction_data: TransactionData) { // let (result, tx_has_failed) = self.run_vm(transaction_data); @@ -157,13 +158,13 @@ // ); // } // } - +// // impl Default for VmTestEnv { // fn default() -> Self { // VmTestEnv::new_with_contracts(&[]) // } // } - +// // /// Helper struct to create a default VM for a given environment. // #[derive(Debug)] // pub struct VmTestHelper<'a> { @@ -172,12 +173,12 @@ // pub block_properties: BlockProperties, // vm_created: bool, // } - +// // impl<'a> VmTestHelper<'a> { // pub fn new(test_env: &'a mut VmTestEnv) -> Self { // let block_context = test_env.block_context; // let block_properties = test_env.block_properties; - +// // let oracle_tools = OracleTools::new(test_env.storage_ptr.as_mut(), HistoryEnabled); // VmTestHelper { // oracle_tools, @@ -186,7 +187,7 @@ // vm_created: false, // } // } - +// // /// Creates the VM that can be used in tests. // pub fn vm(&'a mut self) -> Box> { // assert!(!self.vm_created, "Vm can be created only once"); @@ -202,7 +203,7 @@ // vm // } // } - +// // fn run_vm_with_custom_factory_deps<'a, H: HistoryMode>( // oracle_tools: &'a mut OracleTools<'a, false, H>, // block_context: BlockContext, @@ -221,7 +222,7 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // vm.bootloader_state.add_tx_data(encoded_tx.len()); // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, @@ -238,23 +239,23 @@ // ), // Timestamp(0), // ); - +// // let result = vm.execute_next_tx(u32::MAX, false).err(); - +// // assert_eq!(expected_error, result); // } - +// // fn get_balance(token_id: AccountTreeId, account: &Address, main_storage: StoragePtr) -> U256 { // let key = storage_key_for_standard_token_balance(token_id, account); // h256_to_u256(main_storage.borrow_mut().read_value(&key)) // } - +// // fn get_eth_balance(account: &Address, main_storage: &mut StorageView) -> U256 { // let key = // storage_key_for_standard_token_balance(AccountTreeId::new(L2_ETH_TOKEN_ADDRESS), account); // h256_to_u256(main_storage.read_value(&key)) // } - +// // #[test] // fn test_dummy_bootloader() { // let mut vm_test_env = VmTestEnv::default(); @@ -262,12 +263,12 @@ // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(vm_test_env.block_context, Default::default()), @@ -276,37 +277,37 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::BlockPostprocessing); - +// // // Dummy bootloader should not panic // assert!(res.revert_reason.is_none()); - +// // let correct_first_cell = U256::from_str_radix("123123123", 16).unwrap(); - +// // verify_required_memory( // &vm.state, // vec![(correct_first_cell, BOOTLOADER_HEAP_PAGE, 0)], // ); // } - +// // #[test] // fn test_bootloader_out_of_gas() { // let mut vm_test_env = VmTestEnv::default(); // let mut oracle_tools = OracleTools::new(vm_test_env.storage_ptr.as_mut(), HistoryEnabled); - +// // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); - +// // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // // init vm with only 10 ergs // let mut vm = init_vm_inner( // &mut oracle_tools, @@ -316,12 +317,12 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let res = vm.execute_block_tip(); - +// // assert_eq!(res.revert_reason, Some(TxRevertReason::BootloaderOutOfGas)); // } - +// // fn verify_required_memory( // state: &ZkSyncVmState<'_, H>, // required_values: Vec<(U256, u32, u32)>, @@ -334,14 +335,14 @@ // assert_eq!(current_value, required_value); // } // } - +// // #[test] // fn test_default_aa_interaction() { // // In this test, we aim to test whether a simple account interaction (without any fee logic) // // will work. The account will try to deploy a simple contract from integration tests. - +// // let mut vm_test_env = VmTestEnv::default(); - +// // let operator_address = vm_test_env.block_context.context.operator_address; // let base_fee = vm_test_env.block_context.base_fee; // // We deploy here counter contract, because its logic is trivial @@ -362,27 +363,27 @@ // ) // .into(); // let tx_data: TransactionData = tx.clone().into(); - +// // let maximal_fee = tx_data.gas_limit * tx_data.max_fee_per_gas; // let sender_address = tx_data.from(); - +// // vm_test_env.set_rich_account(&sender_address); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let tx_execution_result = vm // .execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); @@ -392,28 +393,28 @@ // "Bootloader was not expected to revert: {:?}", // res.revert_reason // ); - +// // // Both deployment and ordinary nonce should be incremented by one. // let account_nonce_key = get_nonce_key(&sender_address); // let expected_nonce = TX_NONCE_INCREMENT + DEPLOYMENT_NONCE_INCREMENT; - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(expected_nonce), account_nonce_key), // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert!(!tx_has_failed(&vm.state, 0)); - +// // let expected_fee = // maximal_fee - U256::from(tx_execution_result.gas_refunded) * U256::from(base_fee); // let operator_balance = get_balance( @@ -421,13 +422,13 @@ // &operator_address, // vm.state.storage.storage.get_ptr(), // ); - +// // assert_eq!( // operator_balance, expected_fee, // "Operator did not receive his fee" // ); // } - +// // fn execute_vm_with_predetermined_refund( // txs: Vec, // refunds: Vec, @@ -435,15 +436,15 @@ // ) -> VmBlockResult { // let mut vm_test_env = VmTestEnv::default(); // let block_context = vm_test_env.block_context; - +// // for tx in txs.iter() { // let sender_address = tx.initiator_account(); // vm_test_env.set_rich_account(&sender_address); // } - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // let codes_for_decommiter = txs // .iter() // .flat_map(|tx| { @@ -456,12 +457,12 @@ // .collect::)>>() // }) // .collect(); - +// // vm.state.decommittment_processor.populate( // codes_for_decommiter, // Timestamp(vm.state.local_state.timestamp), // ); - +// // let memory_with_suggested_refund = get_bootloader_memory( // txs.into_iter().map(Into::into).collect(), // refunds, @@ -469,24 +470,24 @@ // TxExecutionMode::VerifyExecute, // BlockContextMode::NewBlock(block_context, Default::default()), // ); - +// // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, // memory_with_suggested_refund, // Timestamp(0), // ); - +// // vm.execute_till_block_end(BootloaderJobType::TransactionExecution) // } - +// // #[test] // fn test_predetermined_refunded_gas() { // // In this test, we compare the execution of the bootloader with the predefined // // refunded gas and without them - +// // let mut vm_test_env = VmTestEnv::default(); // let base_fee = vm_test_env.block_context.base_fee; - +// // // We deploy here counter contract, because its logic is trivial // let contract_code = read_test_contract(); // let published_bytecode = CompressedBytecodeInfo::from_original(contract_code.clone()).unwrap(); @@ -504,27 +505,27 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); - +// // // set balance // vm_test_env.set_rich_account(&sender_address); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let tx_execution_result = vm // .execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // // If the refund provided by the operator or the final refund are the 0 // // there is no impact of the operator's refund at all and so this test does not // // make much sense. @@ -536,14 +537,14 @@ // tx_execution_result.gas_refunded > 0, // "The final refund is 0" // ); - +// // let mut result = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); // assert!( // result.full_result.revert_reason.is_none(), // "Bootloader was not expected to revert: {:?}", // result.full_result.revert_reason // ); - +// // let mut result_with_predetermined_refund = execute_vm_with_predetermined_refund( // vec![tx], // vec![tx_execution_result.operator_suggested_refund], @@ -555,7 +556,7 @@ // .full_result // .used_contract_hashes // .sort(); - +// // assert_eq!( // result.full_result.events, // result_with_predetermined_refund.full_result.events @@ -577,18 +578,18 @@ // .used_contract_hashes // ); // } - +// // #[derive(Debug, Clone)] // enum TransactionRollbackTestInfo { // Rejected(Transaction, TxRevertReason), // Processed(Transaction, bool, TxExecutionStatus), // } - +// // impl TransactionRollbackTestInfo { // fn new_rejected(transaction: Transaction, revert_reason: TxRevertReason) -> Self { // Self::Rejected(transaction, revert_reason) // } - +// // fn new_processed( // transaction: Transaction, // should_be_rollbacked: bool, @@ -596,28 +597,28 @@ // ) -> Self { // Self::Processed(transaction, should_be_rollbacked, expected_status) // } - +// // fn get_transaction(&self) -> &Transaction { // match self { // TransactionRollbackTestInfo::Rejected(tx, _) => tx, // TransactionRollbackTestInfo::Processed(tx, _, _) => tx, // } // } - +// // fn rejection_reason(&self) -> Option { // match self { // TransactionRollbackTestInfo::Rejected(_, revert_reason) => Some(revert_reason.clone()), // TransactionRollbackTestInfo::Processed(_, _, _) => None, // } // } - +// // fn should_rollback(&self) -> bool { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => true, // TransactionRollbackTestInfo::Processed(_, x, _) => *x, // } // } - +// // fn expected_status(&self) -> TxExecutionStatus { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => { @@ -627,7 +628,7 @@ // } // } // } - +// // // Accepts the address of the sender as well as the list of pairs of its transactions // // and whether these transactions should succeed. // fn execute_vm_with_possible_rollbacks( @@ -641,13 +642,13 @@ // block_properties, // ..Default::default() // }; - +// // // Setting infinite balance for the sender. // vm_test_env.set_rich_account(&sender_address); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // for test_info in transactions { // vm.save_current_vm_as_snapshot(); // let vm_state_before_tx = vm.dump_inner_state(); @@ -657,7 +658,7 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // match vm.execute_next_tx(u32::MAX, false) { // Err(reason) => { // assert_eq!(test_info.rejection_reason(), Some(reason)); @@ -671,11 +672,11 @@ // ); // } // }; - +// // if test_info.should_rollback() { // // Some error has occurred, we should reject the transaction // vm.rollback_to_latest_snapshot(); - +// // // vm_state_before_tx. // let state_after_rollback = vm.dump_inner_state(); // assert_eq!( @@ -684,7 +685,7 @@ // ); // } // } - +// // let VmBlockResult { // full_result: mut result, // .. @@ -692,10 +693,10 @@ // // Used contract hashes are retrieved in unordered manner. // // However it must be sorted for the comparisons in tests to work // result.used_contract_hashes.sort(); - +// // result // } - +// // // Sets the signature for an L2 transaction and returns the same transaction // // but this different signature. // fn change_signature(mut tx: Transaction, signature: Vec) -> Transaction { @@ -706,22 +707,22 @@ // } // _ => unreachable!(), // }; - +// // tx // } - +// // #[test] // fn test_vm_rollbacks() { // let (block_context, block_properties): (DerivedBlockContext, BlockProperties) = { // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let base_fee = U256::from(block_context.base_fee); - +// // let sender_private_key = H256::random(); // let contract_code = read_test_contract(); - +// // let tx_nonce_0: Transaction = get_deploy_tx( // sender_private_key, // Nonce(0), @@ -764,13 +765,13 @@ // }, // ) // .into(); - +// // let wrong_signature_length_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 32]); // let wrong_v_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 65]); // let wrong_signature_tx = change_signature(tx_nonce_0.clone(), vec![27u8; 65]); - +// // let sender_address = tx_nonce_0.initiator_account(); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -794,7 +795,7 @@ // block_context, // block_properties, // ); - +// // let incorrect_nonce = TxRevertReason::ValidationFailed(VmRevertReason::General { // msg: "Incorrect nonce".to_string(), // data: vec![ @@ -837,7 +838,7 @@ // msg: "Account validation returned invalid magic value. Most often this means that the signature is incorrect".to_string(), // data: vec![], // }); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -882,11 +883,11 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); - +// // let loadnext_contract = get_loadnext_contract(); - +// // let loadnext_constructor_data = encode(&[Token::Uint(U256::from(100))]); // let loadnext_deploy_tx: Transaction = get_deploy_tx( // sender_private_key, @@ -909,7 +910,7 @@ // false, // TxExecutionStatus::Success, // ); - +// // let get_load_next_tx = |params: LoadnextContractExecutionParams, nonce: Nonce| { // // Here we test loadnext with various kinds of operations // let tx: Transaction = mock_loadnext_test_call( @@ -925,10 +926,10 @@ // params, // ) // .into(); - +// // tx // }; - +// // let loadnext_tx_0 = get_load_next_tx( // LoadnextContractExecutionParams { // reads: 100, @@ -951,7 +952,7 @@ // }, // Nonce(2), // ); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -970,7 +971,7 @@ // block_context, // block_properties, // ); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -1011,10 +1012,10 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); // } - +// // // Inserts the contracts into the test environment, bypassing the // // deployer system contract. Besides the reference to storage // // it accepts a `contracts` tuple of information about the contract @@ -1023,16 +1024,16 @@ // for (contract, is_account) in contracts { // let deployer_code_key = get_code_key(contract.account_id.address()); // raw_storage.set_value(deployer_code_key, hash_bytecode(&contract.bytecode)); - +// // if is_account { // let is_account_key = get_is_account_key(contract.account_id.address()); // raw_storage.set_value(is_account_key, u256_to_h256(1_u32.into())); // } - +// // raw_storage.store_factory_dep(hash_bytecode(&contract.bytecode), contract.bytecode); // } // } - +// // enum NonceHolderTestMode { // SetValueUnderNonce, // IncreaseMinNonceBy5, @@ -1041,7 +1042,7 @@ // IncreaseMinNonceBy1, // SwitchToArbitraryOrdering, // } - +// // impl From for u8 { // fn from(mode: NonceHolderTestMode) -> u8 { // match mode { @@ -1054,7 +1055,7 @@ // } // } // } - +// // fn get_nonce_holder_test_tx( // nonce: U256, // account_address: Address, @@ -1076,11 +1077,11 @@ // reserved: [U256::zero(); 4], // data: vec![12], // signature: vec![test_mode.into()], - +// // ..Default::default() // } // } - +// // fn run_vm_with_raw_tx<'a, H: HistoryMode>( // oracle_tools: &'a mut OracleTools<'a, false, H>, // block_context: DerivedBlockContext, @@ -1097,9 +1098,9 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let block_gas_price_per_pubdata = block_context.context.block_gas_price_per_pubdata(); - +// // let overhead = tx.overhead_gas(block_gas_price_per_pubdata as u32); // push_raw_transaction_to_bootloader_memory( // &mut vm, @@ -1112,18 +1113,18 @@ // full_result: result, // .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); - +// // (result, tx_has_failed(&vm.state, 0)) // } - +// // #[test] // fn test_nonce_holder() { // let account_address = H160::random(); // let mut vm_test_env = // VmTestEnv::new_with_contracts(&[(account_address, read_nonce_holder_tester())]); - +// // vm_test_env.set_rich_account(&account_address); - +// // let mut run_nonce_test = |nonce: U256, // test_mode: NonceHolderTestMode, // error_message: Option, @@ -1134,7 +1135,7 @@ // test_mode, // &vm_test_env.block_context, // ); - +// // let (result, tx_has_failed) = vm_test_env.run_vm(tx); // if let Some(msg) = error_message { // let expected_error = @@ -1153,7 +1154,7 @@ // assert!(!tx_has_failed, "{}", comment); // } // }; - +// // // Test 1: trying to set value under non sequential nonce value. // run_nonce_test( // 1u32.into(), @@ -1161,7 +1162,7 @@ // Some("Previous nonce has not been used".to_string()), // "Allowed to set value under non sequential value", // ); - +// // // Test 2: increase min nonce by 1 with sequential nonce ordering: // run_nonce_test( // 0u32.into(), @@ -1169,7 +1170,7 @@ // None, // "Failed to increment nonce by 1 for sequential account", // ); - +// // // Test 3: correctly set value under nonce with sequential nonce ordering: // run_nonce_test( // 1u32.into(), @@ -1177,7 +1178,7 @@ // None, // "Failed to set value under nonce sequential value", // ); - +// // // Test 5: migrate to the arbitrary nonce ordering: // run_nonce_test( // 2u32.into(), @@ -1185,7 +1186,7 @@ // None, // "Failed to switch to arbitrary ordering", // ); - +// // // Test 6: increase min nonce by 5 // run_nonce_test( // 6u32.into(), @@ -1193,7 +1194,7 @@ // None, // "Failed to increase min nonce by 5", // ); - +// // // Test 7: since the nonces in range [6,10] are no longer allowed, the // // tx with nonce 10 should not be allowed // run_nonce_test( @@ -1202,7 +1203,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse nonce below the minimal one", // ); - +// // // Test 8: we should be able to use nonce 13 // run_nonce_test( // 13u32.into(), @@ -1210,7 +1211,7 @@ // None, // "Did not allow to use unused nonce 10", // ); - +// // // Test 9: we should not be able to reuse nonce 13 // run_nonce_test( // 13u32.into(), @@ -1218,7 +1219,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse the same nonce twice", // ); - +// // // Test 10: we should be able to simply use nonce 14, while bumping the minimal nonce by 5 // run_nonce_test( // 14u32.into(), @@ -1226,7 +1227,7 @@ // None, // "Did not allow to use a bumped nonce", // ); - +// // // Test 6: Do not allow bumping nonce by too much // run_nonce_test( // 16u32.into(), @@ -1234,7 +1235,7 @@ // Some("The value for incrementing the nonce is too high".to_string()), // "Allowed for incrementing min nonce too much", // ); - +// // // Test 7: Do not allow not setting a nonce as used // run_nonce_test( // 16u32.into(), @@ -1243,7 +1244,7 @@ // "Allowed to leave nonce as unused", // ); // } - +// // #[test] // fn test_l1_tx_execution() { // // In this test, we try to execute a contract deployment from L1 @@ -1255,7 +1256,7 @@ // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_tx = get_l1_deploy_tx(&contract_code, &[]); // let l1_deploy_tx_data: TransactionData = l1_deploy_tx.clone().into(); - +// // let required_l2_to_l1_logs = vec![ // L2ToL1Log { // shard_id: 0, @@ -1274,9 +1275,9 @@ // value: u256_to_h256(U256::from(1u32)), // }, // ]; - +// // let sender_address = l1_deploy_tx_data.from(); - +// // vm_helper.oracle_tools.decommittment_processor.populate( // vec![( // h256_to_u256(contract_code_hash), @@ -1284,38 +1285,38 @@ // )], // Timestamp(0), // ); - +// // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &l1_deploy_tx, // TxExecutionMode::VerifyExecute, // None, // ); - +// // let res = vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; // assert!(!tx_has_failed(&vm.state, 0)); - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert_eq!(res.result.logs.l2_to_l1_logs, required_l2_to_l1_logs); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, true); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = StorageWritesDeduplicator::apply_on_empty_state( // &vm.execute_next_tx(u32::MAX, false) // .unwrap() @@ -1324,7 +1325,7 @@ // .storage_logs, // ); // assert_eq!(res.initial_storage_writes, 0); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, false); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); // let res = StorageWritesDeduplicator::apply_on_empty_state( @@ -1335,9 +1336,9 @@ // .storage_logs, // ); // assert_eq!(res.initial_storage_writes, 2); - +// // let repeated_writes = res.repeated_storage_writes; - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); // let res = StorageWritesDeduplicator::apply_on_empty_state( // &vm.execute_next_tx(u32::MAX, false) @@ -1349,7 +1350,7 @@ // assert_eq!(res.initial_storage_writes, 1); // // We do the same storage write, so it will be deduplicated // assert_eq!(res.repeated_storage_writes, repeated_writes); - +// // let mut tx = get_l1_execute_test_contract_tx(deployed_address, false); // tx.execute.value = U256::from(1); // match &mut tx.common_data { @@ -1366,35 +1367,35 @@ // TxExecutionStatus::Failure, // "The transaction should fail" // ); - +// // let res = // StorageWritesDeduplicator::apply_on_empty_state(&execution_result.result.logs.storage_logs); - +// // // There are 2 initial writes here: // // - totalSupply of ETH token // // - balance of the refund recipient // assert_eq!(res.initial_storage_writes, 2); // } - +// // #[test] // fn test_invalid_bytecode() { // let mut vm_test_env = VmTestEnv::default(); - +// // let block_gas_per_pubdata = vm_test_env // .block_context // .context // .block_gas_price_per_pubdata(); - +// // let mut test_vm_with_custom_bytecode_hash = // |bytecode_hash: H256, expected_revert_reason: Option| { // let mut oracle_tools = // OracleTools::new(vm_test_env.storage_ptr.as_mut(), HistoryEnabled); - +// // let (encoded_tx, predefined_overhead) = get_l1_tx_with_custom_bytecode_hash( // h256_to_u256(bytecode_hash), // block_gas_per_pubdata as u32, // ); - +// // run_vm_with_custom_factory_deps( // &mut oracle_tools, // vm_test_env.block_context.context, @@ -1404,14 +1405,14 @@ // expected_revert_reason, // ); // }; - +// // let failed_to_mark_factory_deps = |msg: &str, data: Vec| { // TxRevertReason::FailedToMarkFactoryDependencies(VmRevertReason::General { // msg: msg.to_string(), // data, // }) // }; - +// // // Here we provide the correctly-formatted bytecode hash of // // odd length, so it should work. // test_vm_with_custom_bytecode_hash( @@ -1421,7 +1422,7 @@ // ]), // None, // ); - +// // // Here we provide correctly formatted bytecode of even length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1440,7 +1441,7 @@ // ], // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1460,7 +1461,7 @@ // ], // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1481,17 +1482,17 @@ // )), // ); // } - +// // #[test] // fn test_tracing_of_execution_errors() { // // In this test, we are checking that the execution errors are transmitted correctly from the bootloader. // let contract_address = Address::random(); - +// // let mut vm_test_env = // VmTestEnv::new_with_contracts(&[(contract_address, read_error_contract())]); - +// // let private_key = H256::random(); - +// // let tx = get_error_tx( // private_key, // Nonce(0), @@ -1503,25 +1504,25 @@ // gas_per_pubdata_limit: U256::from(MAX_GAS_PER_PUBDATA_BYTE), // }, // ); - +// // vm_test_env.set_rich_account(&tx.common_data.initiator_address); // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &tx.into(), // TxExecutionMode::VerifyExecute, // None, // ); - +// // let mut tracer = TransactionResultTracer::new(usize::MAX, false); // assert_eq!( // vm.execute_with_custom_tracer(&mut tracer), // VmExecutionStopReason::VmFinished, // "Tracer should never request stop" // ); - +// // match tracer.revert_reason { // Some(revert_reason) => { // let revert_reason = VmRevertReason::try_from(&revert_reason as &[u8]).unwrap(); @@ -1544,7 +1545,7 @@ // tracer.revert_reason // ), // } - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); // let tx = get_error_tx( @@ -1564,7 +1565,7 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // let mut tracer = TransactionResultTracer::new(10, false); // assert_eq!( // vm.execute_with_custom_tracer(&mut tracer), @@ -1572,13 +1573,13 @@ // ); // assert!(tracer.is_limit_reached()); // } - +// // /// Checks that `TX_GAS_LIMIT_OFFSET` constant is correct. // #[test] // fn test_tx_gas_limit_offset() { // let gas_limit = U256::from(999999); // let mut vm_test_env = VmTestEnv::default(); - +// // let contract_code = read_test_contract(); // let tx: Transaction = get_deploy_tx( // H256::random(), @@ -1592,11 +1593,11 @@ // }, // ) // .into(); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let gas_limit_from_memory = vm // .state // .memory @@ -1607,12 +1608,12 @@ // .value; // assert_eq!(gas_limit_from_memory, gas_limit); // } - +// // #[test] // fn test_is_write_initial_behaviour() { // // In this test, we check result of `is_write_initial` at different stages. // let mut vm_test_env = VmTestEnv::default(); - +// // let base_fee = vm_test_env.block_context.base_fee; // let account_pk = H256::random(); // let contract_code = read_test_contract(); @@ -1630,27 +1631,27 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); // let nonce_key = get_nonce_key(&sender_address); - +// // // Check that the next write to the nonce key will be initial. // assert!(vm_test_env.storage_ptr.is_write_initial(&nonce_key)); - +// // // Set balance to be able to pay fee for txs. // vm_test_env.set_rich_account(&sender_address); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // vm.execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing the first transaction"); // // Check that `is_write_initial` still returns true for the nonce key. // assert!(vm_test_env.storage_ptr.is_write_initial(&nonce_key)); // } - +// // pub fn get_l1_tx_with_custom_bytecode_hash( // bytecode_hash: U256, // block_gas_per_pubdata: u32, @@ -1659,10 +1660,10 @@ // let predefined_overhead = // tx.overhead_gas_with_custom_factory_deps(vec![bytecode_hash], block_gas_per_pubdata); // let tx_bytes = tx.abi_encode_with_custom_factory_deps(vec![bytecode_hash]); - +// // (bytes_to_be_words(tx_bytes), predefined_overhead) // } - +// // pub fn get_l1_execute_test_contract_tx(deployed_address: Address, with_panic: bool) -> Transaction { // let sender = H160::random(); // get_l1_execute_test_contract_tx_with_sender( @@ -1673,18 +1674,18 @@ // false, // ) // } - +// // pub fn get_l1_tx_with_large_output(sender: Address, deployed_address: Address) -> Transaction { // let test_contract = load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/long-return-data/long-return-data.sol/LongReturnData.json", // ); - +// // let function = test_contract.function("longReturnData").unwrap(); - +// // let calldata = function // .encode_input(&[]) // .expect("failed to encode parameters"); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender, @@ -1701,23 +1702,23 @@ // received_timestamp_ms: 0, // } // } - +// // #[test] // fn test_call_tracer() { // let mut vm_test_env = VmTestEnv::default(); - +// // let sender = H160::random(); - +// // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_tx = get_l1_deploy_tx(&contract_code, &[]); // let l1_deploy_tx_data: TransactionData = l1_deploy_tx.clone().into(); - +// // let sender_address_counter = l1_deploy_tx_data.from(); - +// // vm_test_env.set_rich_account(&sender_address_counter); // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); - +// // vm_helper.oracle_tools.decommittment_processor.populate( // vec![( // h256_to_u256(contract_code_hash), @@ -1725,7 +1726,7 @@ // )], // Timestamp(0), // ); - +// // let contract_code = read_long_return_data_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_long_return_data_tx = get_l1_deploy_tx(&contract_code, &[]); @@ -1736,21 +1737,21 @@ // )], // Timestamp(0), // ); - +// // let tx_data: TransactionData = l1_deploy_long_return_data_tx.clone().into(); // let sender_long_return_address = tx_data.from(); // // The contract should be deployed successfully. // let deployed_address_long_return_data = // deployed_address_create(sender_long_return_address, U256::zero()); // let mut vm = vm_helper.vm(); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &l1_deploy_tx, // TxExecutionMode::VerifyExecute, // None, // ); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address_counter, U256::zero()); // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); @@ -1791,16 +1792,16 @@ // calls: vec![], // }; // assert_eq!(create_call.unwrap(), expected); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &l1_deploy_long_return_data_tx, // TxExecutionMode::VerifyExecute, // None, // ); - +// // vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // let tx = get_l1_execute_test_contract_tx_with_sender( // sender, // deployed_address, @@ -1808,13 +1809,13 @@ // U256::from(1u8), // true, // ); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; - +// // // We don't want to compare gas used, because it's not fully deterministic. // let expected = Call { // r#type: CallType::Call(FarCallOpcode::Mimic), @@ -1833,7 +1834,7 @@ // revert_reason: None, // calls: vec![], // }; - +// // // First loop filter out the bootloaders calls and // // the second loop filters out the calls msg value simulator calls // for call in calls { @@ -1845,7 +1846,7 @@ // } // } // } - +// // let tx = get_l1_execute_test_contract_tx_with_sender( // sender, // deployed_address, @@ -1853,13 +1854,13 @@ // U256::from(1u8), // true, // ); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; - +// // let expected = Call { // r#type: CallType::Call(FarCallOpcode::Mimic), // to: deployed_address, @@ -1874,7 +1875,7 @@ // revert_reason: Some("This method always reverts".to_string()), // calls: vec![], // }; - +// // for call in calls { // if let CallType::Call(FarCallOpcode::Mimic) = call.r#type { // for call in call.calls { @@ -1884,12 +1885,12 @@ // } // } // } - +// // let tx = get_l1_tx_with_large_output(sender, deployed_address_long_return_data); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // assert_ne!(deployed_address_long_return_data, deployed_address); // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; @@ -1907,30 +1908,30 @@ // } // } // } - +// // #[test] // fn test_get_used_contracts() { // let mut vm_test_env = VmTestEnv::default(); - +// // let mut vm_helper = VmTestHelper::new(&mut vm_test_env); // let mut vm = vm_helper.vm(); - +// // assert!(known_bytecodes_without_aa_code(&vm).is_empty()); - +// // // create and push and execute some not-empty factory deps transaction with success status // // to check that get_used_contracts() updates // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let tx1 = get_l1_deploy_tx(&contract_code, &[]); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx1, TxExecutionMode::VerifyExecute, None); - +// // let res1 = vm.execute_next_tx(u32::MAX, true).unwrap(); // assert_eq!(res1.status, TxExecutionStatus::Success); // assert!(vm // .get_used_contracts() // .contains(&h256_to_u256(contract_code_hash))); - +// // assert_eq!( // vm.get_used_contracts() // .into_iter() @@ -1940,13 +1941,13 @@ // .cloned() // .collect::>() // ); - +// // // create push and execute some non-empty factory deps transaction that fails // // (known_bytecodes will be updated but we expect get_used_contracts() to not be updated) - +// // let mut tx2 = tx1; // tx2.execute.contract_address = L1_MESSENGER_ADDRESS; - +// // let calldata = vec![1, 2, 3]; // let big_calldata: Vec = calldata // .iter() @@ -1954,16 +1955,16 @@ // .take(calldata.len() * 1024) // .cloned() // .collect(); - +// // tx2.execute.calldata = big_calldata; // tx2.execute.factory_deps = Some(vec![vec![1; 32]]); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx2, TxExecutionMode::VerifyExecute, None); - +// // let res2 = vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // assert_eq!(res2.status, TxExecutionStatus::Failure); - +// // for factory_dep in tx2.execute.factory_deps.unwrap() { // let hash = hash_bytecode(&factory_dep); // let hash_to_u256 = h256_to_u256(hash); @@ -1973,7 +1974,7 @@ // assert!(!vm.get_used_contracts().contains(&hash_to_u256)); // } // } - +// // fn known_bytecodes_without_aa_code(vm: &VmInstance) -> HashMap> { // let mut known_bytecodes_without_aa_code = vm // .state @@ -1981,14 +1982,14 @@ // .known_bytecodes // .inner() // .clone(); - +// // known_bytecodes_without_aa_code // .remove(&h256_to_u256(BASE_SYSTEM_CONTRACTS.default_aa.hash)) // .unwrap(); - +// // known_bytecodes_without_aa_code // } - +// // #[tokio::test] // /// This test deploys 'buggy' account abstraction code, and then tries accessing it both with legacy // /// and EIP712 transactions. @@ -1999,31 +2000,31 @@ // // - account_address - AA account, where the contract is deployed // // - beneficiary - an EOA account, where we'll try to transfer the tokens. // let account_address = H160::random(); - +// // let (bytecode, contract) = read_many_owners_custom_account_contract(); - +// // let mut vm_test_env = VmTestEnv::new_with_contracts(&[(account_address, bytecode)]); - +// // let beneficiary = H160::random(); - +// // assert_eq!(vm_test_env.get_eth_balance(&beneficiary), U256::from(0)); - +// // let private_key = H256::random(); // let private_address = PackedEthSignature::address_from_private_key(&private_key).unwrap(); // let pk_signer = PrivateKeySigner::new(private_key); - +// // vm_test_env.set_rich_account(&account_address); // vm_test_env.set_rich_account(&private_address); - +// // let chain_id: u16 = 270; - +// // // First, let's set the owners of the AA account to the private_address. // // (so that messages signed by private_address, are authorized to act on behalf of the AA account). // { // let set_owners_function = contract.function("setOwners").unwrap(); // let encoded_input = set_owners_function // .encode_input(&[Token::Array(vec![Token::Address(private_address)])]); - +// // // Create a legacy transaction to set the owners. // let raw_tx = TransactionParameters { // nonce: U256::from(0), @@ -2039,19 +2040,19 @@ // max_priority_fee_per_gas: U256::from(1000000000), // }; // let txn = pk_signer.sign_transaction(raw_tx).await.unwrap(); - +// // let (txn_request, hash) = TransactionRequest::from_bytes(&txn, chain_id).unwrap(); - +// // let mut l2_tx: L2Tx = L2Tx::from_request(txn_request, 100000).unwrap(); // l2_tx.set_input(txn, hash); // let transaction: Transaction = l2_tx.try_into().unwrap(); // let transaction_data: TransactionData = transaction.try_into().unwrap(); - +// // vm_test_env.run_vm_or_die(transaction_data); // } - +// // let private_account_balance = vm_test_env.get_eth_balance(&private_address); - +// // // And now let's do the transfer from the 'account abstraction' to 'beneficiary' (using 'legacy' transaction). // // Normally this would not work - unless the operator is malicious. // { @@ -2068,20 +2069,20 @@ // max_fee_per_gas: U256::from(1000000000), // max_priority_fee_per_gas: U256::from(1000000000), // }; - +// // let aa_txn = pk_signer.sign_transaction(aa_raw_tx).await.unwrap(); - +// // let (aa_txn_request, aa_hash) = TransactionRequest::from_bytes(&aa_txn, 270).unwrap(); - +// // let mut l2_tx: L2Tx = L2Tx::from_request(aa_txn_request, 100000).unwrap(); // l2_tx.set_input(aa_txn, aa_hash); // // Pretend that operator is malicious and sets the initiator to the AA account. // l2_tx.common_data.initiator_address = account_address; - +// // let transaction: Transaction = l2_tx.try_into().unwrap(); - +// // let transaction_data: TransactionData = transaction.try_into().unwrap(); - +// // vm_test_env.run_vm_or_die(transaction_data); // assert_eq!( // vm_test_env.get_eth_balance(&beneficiary), @@ -2093,7 +2094,7 @@ // vm_test_env.get_eth_balance(&private_address) // ) // } - +// // // Now send the 'classic' EIP712 transaction // { // let tx_712 = L2Tx::new( @@ -2111,27 +2112,27 @@ // None, // Default::default(), // ); - +// // let transaction_request: TransactionRequest = tx_712.into(); - +// // let domain = Eip712Domain::new(L2ChainId(chain_id)); // let signature = pk_signer // .sign_typed_data(&domain, &transaction_request) // .await // .unwrap(); // let encoded_tx = transaction_request.get_signed_bytes(&signature, L2ChainId(chain_id)); - +// // let (aa_txn_request, aa_hash) = // TransactionRequest::from_bytes(&encoded_tx, chain_id).unwrap(); - +// // let mut l2_tx: L2Tx = L2Tx::from_request(aa_txn_request, 100000).unwrap(); // l2_tx.set_input(encoded_tx, aa_hash); - +// // let transaction: Transaction = l2_tx.try_into().unwrap(); // let transaction_data: TransactionData = transaction.try_into().unwrap(); - +// // vm_test_env.run_vm_or_die(transaction_data); - +// // assert_eq!( // vm_test_env.get_eth_balance(&beneficiary), // U256::from(916375026) @@ -2142,3 +2143,4 @@ // ); // } // } +// ``` \ No newline at end of file diff --git a/core/lib/multivm/src/versions/vm_1_3_2/tests/mod.rs b/core/lib/multivm/src/versions/vm_1_3_2/tests/mod.rs index af4748e38645..04448987b1cc 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/tests/mod.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/tests/mod.rs @@ -1,4 +1,6 @@ +// ``` // mod bootloader; // mod upgrades; // mod utils; +// ``` diff --git a/core/lib/multivm/src/versions/vm_1_3_2/tests/upgrades.rs b/core/lib/multivm/src/versions/vm_1_3_2/tests/upgrades.rs index 47e9ad5d4eb8..cd3857d46da9 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/tests/upgrades.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/tests/upgrades.rs @@ -1,3 +1,4 @@ +// ``` // use crate::{ // test_utils::verify_required_storage, // tests::utils::get_l1_deploy_tx, @@ -7,9 +8,9 @@ // vm_with_bootloader::{BlockContextMode, TxExecutionMode}, // HistoryEnabled, OracleTools, TxRevertReason, // }; - +// // use zk_evm_1_3_3::aux_structures::Timestamp; - +// // use zksync_types::{ // ethabi::Contract, // tx::tx_execution_info::TxExecutionStatus, @@ -18,17 +19,17 @@ // {ethabi::Token, Address, ExecuteTransactionCommon, Transaction, H256, U256}, // {get_code_key, get_known_code_key, H160}, // }; - +// // use zksync_utils::{bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}; - +// // use zksync_contracts::{deployer_contract, load_contract, load_sys_contract, read_bytecode}; // use zksync_state::WriteStorage; - +// // use crate::tests::utils::create_storage_view; // use zksync_types::protocol_version::ProtocolUpgradeTxCommonData; - +// // use super::utils::read_test_contract; - +// // /// In this test we ensure that the requirements for protocol upgrade transactions are enforced by the bootloader: // /// - This transaction must be the only one in block // /// - If present, this transaction must be the first one in block @@ -37,9 +38,9 @@ // let mut storage_view = create_storage_view(); // let mut oracle_tools = OracleTools::new(&mut storage_view, HistoryEnabled); // let (block_context, block_properties) = create_test_block_params(); - +// // let bytecode_hash = hash_bytecode(&read_test_contract()); - +// // // Here we just use some random transaction of protocol upgrade type: // let protocol_upgrade_transaction = get_forced_deploy_tx(&[ForceDeployment { // // The bytecode hash to put on an address @@ -53,9 +54,9 @@ // // The constructor calldata // input: vec![], // }]); - +// // let normal_l1_transaction = get_l1_deploy_tx(&read_test_contract(), &[]); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -64,14 +65,14 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // let expected_error = TxRevertReason::UnexpectedVMBehavior( // "Assertion error: Protocol upgrade tx not first".to_string(), // ); - +// // // Test 1: there must be only one system transaction in block // vm.save_current_vm_as_snapshot(); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &protocol_upgrade_transaction, @@ -90,15 +91,15 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // vm.execute_next_tx(u32::MAX, false).unwrap(); // vm.execute_next_tx(u32::MAX, false).unwrap(); // let res = vm.execute_next_tx(u32::MAX, false); // assert_eq!(res, Err(expected_error.clone())); - +// // // Test 2: the protocol upgrade tx must be the first one in block // vm.rollback_to_latest_snapshot(); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &normal_l1_transaction, @@ -111,26 +112,26 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // vm.execute_next_tx(u32::MAX, false).unwrap(); // let res = vm.execute_next_tx(u32::MAX, false); // assert_eq!(res, Err(expected_error)); // } - +// // /// In this test we try to test how force deployments could be done via protocol upgrade transactions. // #[test] // fn test_force_deploy_upgrade() { // let mut storage_view = create_storage_view(); - +// // let bytecode_hash = hash_bytecode(&read_test_contract()); - +// // let known_code_key = get_known_code_key(&bytecode_hash); // // It is generally expected that all the keys will be set as known prior to the protocol upgrade. // storage_view.set_value(known_code_key, u256_to_h256(1.into())); - +// // let mut oracle_tools = OracleTools::new(&mut storage_view, HistoryEnabled); // let (block_context, block_properties) = create_test_block_params(); - +// // let address_to_deploy = H160::random(); // // Here we just use some random transaction of protocol upgrade type: // let transaction = get_forced_deploy_tx(&[ForceDeployment { @@ -145,7 +146,7 @@ // // The constructor calldata // input: vec![], // }]); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -167,33 +168,33 @@ // "The force upgrade was not successful" // ); // assert!(!tx_has_failed(&vm.state, 0)); - +// // let expected_slots = vec![(bytecode_hash, get_code_key(&address_to_deploy))]; - +// // // Verify that the bytecode has been set correctly // verify_required_storage(&vm.state, expected_slots); // } - +// // /// Here we show how the work with the complex upgrader could be done // #[test] // fn test_complex_upgrader() { // let mut storage_view = create_storage_view(); - +// // let bytecode_hash = hash_bytecode(&read_complex_upgrade()); // let msg_sender_test_hash = hash_bytecode(&read_msg_sender_test()); - +// // // Let's assume that the bytecode for the implementation of the complex upgrade // // is already deployed in some address in userspace // let upgrade_impl = H160::random(); // let account_code_key = get_code_key(&upgrade_impl); - +// // storage_view.set_value(get_known_code_key(&bytecode_hash), u256_to_h256(1.into())); // storage_view.set_value( // get_known_code_key(&msg_sender_test_hash), // u256_to_h256(1.into()), // ); // storage_view.set_value(account_code_key, bytecode_hash); - +// // let mut oracle_tools: OracleTools = // OracleTools::new(&mut storage_view, HistoryEnabled); // oracle_tools.decommittment_processor.populate( @@ -209,19 +210,19 @@ // ], // Timestamp(0), // ); - +// // let (block_context, block_properties) = create_test_block_params(); - +// // let address_to_deploy1 = H160::random(); // let address_to_deploy2 = H160::random(); - +// // let transaction = get_complex_upgrade_tx( // upgrade_impl, // address_to_deploy1, // address_to_deploy2, // bytecode_hash, // ); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -243,16 +244,16 @@ // "The force upgrade was not successful" // ); // assert!(!tx_has_failed(&vm.state, 0)); - +// // let expected_slots = vec![ // (bytecode_hash, get_code_key(&address_to_deploy1)), // (bytecode_hash, get_code_key(&address_to_deploy2)), // ]; - +// // // Verify that the bytecode has been set correctly // verify_required_storage(&vm.state, expected_slots); // } - +// // #[derive(Debug, Clone)] // struct ForceDeployment { // // The bytecode hash to put on an address @@ -266,11 +267,11 @@ // // The constructor calldata // input: Vec, // } - +// // fn get_forced_deploy_tx(deployment: &[ForceDeployment]) -> Transaction { // let deployer = deployer_contract(); // let contract_function = deployer.function("forceDeployOnAddresses").unwrap(); - +// // let encoded_deployments: Vec<_> = deployment // .iter() // .map(|deployment| { @@ -283,20 +284,20 @@ // ]) // }) // .collect(); - +// // let params = [Token::Array(encoded_deployments)]; - +// // let calldata = contract_function // .encode_input(¶ms) // .expect("failed to encode parameters"); - +// // let execute = Execute { // contract_address: CONTRACT_DEPLOYER_ADDRESS, // calldata, // factory_deps: None, // value: U256::zero(), // }; - +// // Transaction { // common_data: ExecuteTransactionCommon::ProtocolUpgrade(ProtocolUpgradeTxCommonData { // sender: CONTRACT_FORCE_DEPLOYER_ADDRESS, @@ -308,7 +309,7 @@ // received_timestamp_ms: 0, // } // } - +// // // Returns the transaction that performs a complex protocol upgrade. // // The first param is the address of the implementation of the complex upgrade // // in user-space, while the next 3 params are params of the implenentaiton itself @@ -329,7 +330,7 @@ // Token::FixedBytes(bytecode_hash.as_bytes().to_vec()), // ]) // .unwrap(); - +// // let complex_upgrader = get_complex_upgrader_abi(); // let upgrade_function = complex_upgrader.function("upgrade").unwrap(); // let complex_upgrader_calldata = upgrade_function @@ -338,14 +339,14 @@ // Token::Bytes(impl_calldata), // ]) // .unwrap(); - +// // let execute = Execute { // contract_address: COMPLEX_UPGRADER_ADDRESS, // calldata: complex_upgrader_calldata, // factory_deps: None, // value: U256::zero(), // }; - +// // Transaction { // common_data: ExecuteTransactionCommon::ProtocolUpgrade(ProtocolUpgradeTxCommonData { // sender: CONTRACT_FORCE_DEPLOYER_ADDRESS, @@ -357,21 +358,22 @@ // received_timestamp_ms: 0, // } // } - +// // fn read_complex_upgrade() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/complex-upgrade.sol/ComplexUpgrade.json") // } - +// // fn read_msg_sender_test() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/msg-sender.sol/MsgSenderTest.json") // } - +// // fn get_complex_upgrade_abi() -> Contract { // load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/complex-upgrade/complex-upgrade.sol/ComplexUpgrade.json" // ) // } - +// // fn get_complex_upgrader_abi() -> Contract { // load_sys_contract("ComplexUpgrader") // } +// ``` \ No newline at end of file diff --git a/core/lib/multivm/src/versions/vm_1_3_2/tests/utils.rs b/core/lib/multivm/src/versions/vm_1_3_2/tests/utils.rs index 865f8503701c..b2231f05cf17 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/tests/utils.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/tests/utils.rs @@ -1,3 +1,4 @@ +// ``` // //! // //! Tests for the bootloader // //! The description for each of the tests can be found in the corresponding `.yul` file. @@ -7,36 +8,36 @@ // Execute, L1TxCommonData, H160, REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE, // {ethabi::Token, Address, ExecuteTransactionCommon, Transaction, U256}, // }; - +// // use zksync_contracts::{load_contract, read_bytecode}; // use zksync_state::{InMemoryStorage, StorageView}; // use zksync_utils::bytecode::hash_bytecode; - +// // use crate::test_utils::get_create_execute; - +// // pub fn read_test_contract() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json") // } - +// // pub fn read_long_return_data_contract() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/long-return-data/long-return-data.sol/LongReturnData.json") // } - +// // pub fn read_nonce_holder_tester() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/custom-account/nonce-holder-test.sol/NonceHolderTest.json") // } - +// // pub fn read_error_contract() -> Vec { // read_bytecode( // "etc/contracts-test-data/artifacts-zk/contracts/error/error.sol/SimpleRequire.json", // ) // } - +// // pub fn read_many_owners_custom_account_contract() -> (Vec, Contract) { // let path = "etc/contracts-test-data/artifacts-zk/contracts/custom-account/many-owners-custom-account.sol/ManyOwnersCustomAccount.json"; // (read_bytecode(path), load_contract(path)) // } - +// // pub fn get_l1_execute_test_contract_tx_with_sender( // sender: Address, // deployed_address: Address, @@ -45,7 +46,7 @@ // payable: bool, // ) -> Transaction { // let execute = execute_test_contract(deployed_address, with_panic, value, payable); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender, @@ -58,7 +59,7 @@ // received_timestamp_ms: 0, // } // } - +// // fn execute_test_contract( // address: Address, // with_panic: bool, @@ -68,7 +69,7 @@ // let test_contract = load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json", // ); - +// // let function = if payable { // test_contract // .function("incrementWithRevertPayable") @@ -76,11 +77,11 @@ // } else { // test_contract.function("incrementWithRevert").unwrap() // }; - +// // let calldata = function // .encode_input(&[Token::Uint(U256::from(1u8)), Token::Bool(with_panic)]) // .expect("failed to encode parameters"); - +// // Execute { // contract_address: address, // calldata, @@ -88,10 +89,10 @@ // factory_deps: None, // } // } - +// // pub fn get_l1_deploy_tx(code: &[u8], calldata: &[u8]) -> Transaction { // let execute = get_create_execute(code, calldata); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender: H160::random(), @@ -103,8 +104,9 @@ // received_timestamp_ms: 0, // } // } - +// // pub fn create_storage_view() -> StorageView { // let raw_storage = InMemoryStorage::with_system_contracts(hash_bytecode); // StorageView::new(raw_storage) // } +// ``` \ No newline at end of file diff --git a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs index d1a22f1a40f2..63d0418c2969 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs @@ -235,7 +235,7 @@ pub(crate) fn derive_overhead( encoded_len: usize, coefficients: OverheadCoefficients, ) -> u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); @@ -244,8 +244,8 @@ pub(crate) fn derive_overhead( let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -259,15 +259,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); + // ``` vec![ (coefficients.ergs_limit_overhead_coeficient @@ -285,7 +287,7 @@ pub(crate) fn derive_overhead( /// Contains the coefficients with which the overhead for transactions will be calculated. /// All of the coefficients should be <= 1. There are here to provide a certain "discount" for normal transactions /// at the risk of malicious transactions that may close the block prematurely. -/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coeficient` MUST +/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coefficient` MUST /// result in an integer number #[derive(Debug, Clone, Copy)] pub struct OverheadCoefficients { @@ -323,8 +325,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -352,28 +354,28 @@ pub fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -384,7 +386,7 @@ pub fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -397,13 +399,17 @@ pub fn get_amortized_overhead( }; // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, - // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. + // since the pubdata is not published. If decided to use the pubdata overhead, it needs to be updated. + // ``` // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + // Throwing off the `ceil`, while may provide marginally lower // overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // ``` // ceil(OB * (TL - OE) / (EP * MP)) >= OE // // OB * (TL - OE) / (MP * EP) > OE - 1 @@ -416,7 +422,7 @@ pub fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -425,7 +431,7 @@ pub fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - + // // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) @@ -434,6 +440,7 @@ pub fn get_amortized_overhead( // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + // ``` let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from( @@ -448,16 +455,16 @@ pub fn get_amortized_overhead( let overhead = vec![tx_slot_overhead, overhead_for_length, overhead_for_gas] .into_iter() .max() - // For the sake of consistency making sure that total_gas_limit >= max_overhead + // For the sake of consistency making sure that `total_gas_limit >= max_overhead` .map(|max_overhead| std::cmp::min(max_overhead, total_gas_limit.as_u32())) .unwrap(); let limit_after_deducting_overhead = total_gas_limit - overhead; // During double checking of the overhead, the bootloader will assume that the - // body of the transaction does not have any more than MAX_L2_TX_GAS_LIMIT ergs available to it. + // body of the transaction does not have any more than `MAX_L2_TX_GAS_LIMIT` ergs available to it. if limit_after_deducting_overhead.as_u64() > MAX_L2_TX_GAS_LIMIT { - // We derive the same overhead that would exist for the MAX_L2_TX_GAS_LIMIT ergs + // We derive the same overhead that would exist for the `MAX_L2_TX_GAS_LIMIT` ergs derive_overhead( MAX_L2_TX_GAS_LIMIT as u32, gas_per_pubdata_byte_limit, diff --git a/core/lib/multivm/src/versions/vm_1_3_2/utils.rs b/core/lib/multivm/src/versions/vm_1_3_2/utils.rs index e462725c2431..a7956d473ab3 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/utils.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/utils.rs @@ -190,7 +190,7 @@ impl IntoFixedLengthByteIterator<32> for U256 { /// Receives sorted slice of timestamps. /// Returns count of timestamps that are greater than or equal to `from_timestamp`. -/// Works in O(log(sorted_timestamps.len())). +/// Works in `O(log(sorted_timestamps.len()))`. pub fn precompile_calls_count_after_timestamp( sorted_timestamps: &[Timestamp], from_timestamp: Timestamp, diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs index 5f48a276ac79..3fe3f1929fd7 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_instance.rs @@ -184,7 +184,7 @@ fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(NewVmExecutionResult::Panic) } else { @@ -217,7 +217,7 @@ fn vm_may_have_ended( NewVmExecutionResult::Ok(data) => { Some(VmExecutionResult { // The correct `events` value for this field should be set separately - // later on based on the information inside the event_sink oracle. + // later on based on the information inside the `event_sink` oracle. events: vec![], storage_log_queries: vm.state.storage.get_final_log_queries(), used_contract_hashes: vm.get_used_contracts(), @@ -490,8 +490,8 @@ impl VmInstance { ); } - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = tracer.requested_refund() { assert!( operator_refund.is_none(), @@ -587,8 +587,8 @@ impl VmInstance { /// Panics if there are no new transactions in bootloader. /// Internally uses the OneTxTracer to stop the VM when the last opcode from the transaction is reached. // Err when transaction is rejected. - // Ok(status: TxExecutionStatus::Success) when the transaction succeeded - // Ok(status: TxExecutionStatus::Failure) when the transaction failed. + // `Ok(status: TxExecutionStatus::Success)` when the transaction succeeded + // `Ok(status: TxExecutionStatus::Failure)` when the transaction failed. // Note that failed transactions are considered properly processed and are included in blocks pub fn execute_next_tx( &mut self, @@ -648,7 +648,7 @@ impl VmInstance { revert_reason: None, // getting contracts used during this transaction // at least for now the number returned here is always <= to the number - // of the code hashes actually used by the transaction, since it might've + // of the code hashes actually used by the transaction, since it might have // reused bytecode hashes from some of the previous ones. contracts_used: self .state @@ -913,8 +913,8 @@ impl VmInstance { pub fn save_current_vm_as_snapshot(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs index b8db8a480da3..5ecbea6ff150 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs @@ -36,8 +36,8 @@ use crate::vm_1_3_2::{ OracleTools, VmInstance, }; -// TODO (SMA-1703): move these to config and make them programmatically generatable. -// fill these values in the similar fasion as other overhead-related constants +// TODO (SMA-1703): move these to config and make them programmatically generable. +// fill these values in the similar fashion as other overhead-related constants pub const BLOCK_OVERHEAD_GAS: u32 = 1200000; pub const BLOCK_OVERHEAD_L1_GAS: u32 = 1000000; pub const BLOCK_OVERHEAD_PUBDATA: u32 = BLOCK_OVERHEAD_L1_GAS / L1_GAS_PER_PUBDATA_BYTE; @@ -89,7 +89,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, @@ -254,12 +254,12 @@ pub fn init_vm_with_gas_limit( } #[derive(Debug, Clone, Copy)] -// The block.number/block.timestamp data are stored in the CONTEXT_SYSTEM_CONTRACT. +// The `block.number` / `block.timestamp` data are stored in the `CONTEXT_SYSTEM_CONTRACT`. // The bootloader can support execution in two modes: -// - "NewBlock" when the new block is created. It is enforced that the block.number is incremented by 1 +// - `NewBlock` when the new block is created. It is enforced that the block.number is incremented by 1 // and the timestamp is non-decreasing. Also, the L2->L1 message used to verify the correctness of the previous root hash is sent. // This is the mode that should be used in the state keeper. -// - "OverrideCurrent" when we need to provide custom block.number and block.timestamp. ONLY to be used in testing/ethCalls. +// - `OverrideCurrent` when we need to provide custom block.number and block.timestamp. ONLY to be used in testing / `ethCalls`. pub enum BlockContextMode { NewBlock(DerivedBlockContext, U256), OverrideCurrent(DerivedBlockContext), diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs index 299f7b170b54..6ff63e17ce00 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_boojum_integration::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_latest/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_latest/bootloader_state/l2_block.rs index 146e8713c69e..9a4ae2df871b 100644 --- a/core/lib/multivm/src/versions/vm_latest/bootloader_state/l2_block.rs +++ b/core/lib/multivm/src/versions/vm_latest/bootloader_state/l2_block.rs @@ -19,7 +19,7 @@ pub(crate) struct BootloaderL2Block { pub(crate) timestamp: u64, pub(crate) txs_rolling_hash: H256, // The rolling hash of all the transactions in the miniblock pub(crate) prev_block_hash: H256, - // Number of the first l2 block tx in l1 batch + // Number of the first L2 block tx in L1 batch pub(crate) first_tx_index: usize, pub(crate) max_virtual_blocks_to_create: u32, pub(super) txs: Vec, diff --git a/core/lib/multivm/src/versions/vm_latest/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_latest/bootloader_state/utils.rs index 93ec255e7740..346c1bde5368 100644 --- a/core/lib/multivm/src/versions/vm_latest/bootloader_state/utils.rs +++ b/core/lib/multivm/src/versions/vm_latest/bootloader_state/utils.rs @@ -94,8 +94,8 @@ pub(crate) fn apply_l2_block( bootloader_l2_block: &BootloaderL2Block, txs_index: usize, ) { - // Since L2 block infos start from the TX_OPERATOR_L2_BLOCK_INFO_OFFSET and each - // L2 block info takes TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots, the position where the L2 block info + // Since L2 block information start from the `TX_OPERATOR_L2_BLOCK_INFO_OFFSET` and each + // L2 block info takes `TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO` slots, the position where the L2 block info // for this transaction needs to be written is: let block_position = diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_latest/implementation/snapshots.rs index b6b452834388..d7cba6ffd24b 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/snapshots.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/snapshots.rs @@ -36,8 +36,8 @@ impl Vm { pub(crate) fn make_snapshot_inner(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/event_sink.rs index 8e7f4d447b4d..564d26fcbde3 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/event_sink.rs @@ -54,7 +54,7 @@ impl InMemoryEventSink { pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { let events = self.frames_stack.forward().current_frame(); - // Select all of the last elements where e.timestamp >= from_timestamp. + // Select all of the last elements where `e.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. events .rsplit(|e| e.timestamp < from_timestamp) diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/history_recorder.rs index 83a22f35b4a1..d82fb60dd6e7 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/history_recorder.rs @@ -12,14 +12,14 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; pub(crate) type MemoryWithHistory = HistoryRecorder; pub(crate) type IntFrameManagerWithHistory = HistoryRecorder, H>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` // can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/memory.rs index 5a7592ce9654..388ed31b5134 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/memory.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/memory.rs @@ -282,7 +282,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -299,7 +299,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -307,7 +307,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/decommitter.rs index 4a718917a21d..f9f0e57a2d14 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_latest::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/utils.rs index 1dbe82a81d4f..0fca670949d1 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/utils.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/utils.rs @@ -122,7 +122,7 @@ pub(crate) fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(VmExecutionResult::Panic) } else { diff --git a/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs b/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs index b72651c0a9d1..d80a545ff67f 100644 --- a/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_latest/oracles/storage.rs @@ -53,7 +53,7 @@ pub struct StorageOracle { pub(crate) paid_changes: HistoryRecorder, H>, // The map that contains all the first values read from storage for each slot. - // While formally it does not have to be rollbackable, we still do it to avoid memory bloat + // While formally it does not have to be capable of rolling back, we still do it to avoid memory bloat // for unused slots. pub(crate) initial_values: HistoryRecorder, H>, @@ -212,7 +212,7 @@ impl StorageOracle { let required_pubdata = self.base_price_for_write(&key, first_slot_value, current_slot_value); - // We assume that "prepaid_for_slot" represents both the number of pubdata published and the number of bytes paid by the previous transactions + // We assume that `prepaid_for_slot` represents both the number of pubdata published and the number of bytes paid by the previous transactions // as they should be identical. let prepaid_for_slot = self .pre_paid_changes @@ -292,7 +292,7 @@ impl StorageOracle { ) -> &[Box] { let logs = self.frames_stack.forward().current_frame(); - // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Select all of the last elements where `l.log_query.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. logs.rsplit(|l| l.log_query.timestamp < from_timestamp) .next() @@ -340,6 +340,7 @@ impl VmStorageOracle for StorageOracle { _monotonic_cycle_counter: u32, mut query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -349,6 +350,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -443,7 +445,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -457,7 +459,7 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). +// we basically need to publish the following pair: `()`. // For key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. diff --git a/core/lib/multivm/src/versions/vm_latest/tests/get_used_contracts.rs b/core/lib/multivm/src/versions/vm_latest/tests/get_used_contracts.rs index b82057bef8b7..38a4d7cbb43c 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/get_used_contracts.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/get_used_contracts.rs @@ -29,7 +29,7 @@ fn test_get_used_contracts() { assert!(known_bytecodes_without_aa_code(&vm.vm).is_empty()); // create and push and execute some not-empty factory deps transaction with success status - // to check that get_used_contracts() updates + // to check that `get_used_contracts()` updates let contract_code = read_test_contract(); let mut account = Account::random(); let tx = account.get_deploy_tx(&contract_code, None, TxType::L1 { serial_id: 0 }); @@ -42,7 +42,7 @@ fn test_get_used_contracts() { .get_used_contracts() .contains(&h256_to_u256(tx.bytecode_hash))); - // Note: Default_AA will be in the list of used contracts if l2 tx is used + // Note: `Default_AA` will be in the list of used contracts if L2 tx is used assert_eq!( vm.vm .get_used_contracts() @@ -55,7 +55,7 @@ fn test_get_used_contracts() { ); // create push and execute some non-empty factory deps transaction that fails - // (known_bytecodes will be updated but we expect get_used_contracts() to not be updated) + // (`known_bytecodes` will be updated but we expect `get_used_contracts()` to not be updated) let calldata = [1, 2, 3]; let big_calldata: Vec = calldata diff --git a/core/lib/multivm/src/versions/vm_latest/tests/l1_tx_execution.rs b/core/lib/multivm/src/versions/vm_latest/tests/l1_tx_execution.rs index 4f61dd90fad7..a8e53021848e 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/l1_tx_execution.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/l1_tx_execution.rs @@ -26,13 +26,13 @@ fn test_l1_tx_execution() { // using L1->L2 communication, the same it would likely be done during the priority mode. // There are always at least 7 initial writes here, because we pay fees from l1: - // - totalSupply of ETH token + // - `totalSupply` of ETH token // - balance of the refund recipient // - balance of the bootloader - // - tx_rolling hash + // - `tx_rolling` hash // - rolling hash of L2->L1 logs // - transaction number in block counter - // - L2->L1 log counter in L1Messenger + // - L2->L1 log counter in `L1Messenger` // TODO(PLA-537): right now we are using 4 slots instead of 7 due to 0 fee for transaction. let basic_initial_writes = 4; diff --git a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs index 95377232b3e3..b6c2cb654a82 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs @@ -1,7 +1,7 @@ mod bootloader; mod default_aa; // TODO - fix this test -// mod invalid_bytecode; +// `mod invalid_bytecode;` mod bytecode_publishing; mod call_tracer; mod circuits; diff --git a/core/lib/multivm/src/versions/vm_latest/tests/nonce_holder.rs b/core/lib/multivm/src/versions/vm_latest/tests/nonce_holder.rs index 2de5e23bdd23..309e26120af3 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/nonce_holder.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/nonce_holder.rs @@ -59,7 +59,7 @@ fn test_nonce_holder() { comment: &'static str| { // In this test we have to reset VM state after each test case. Because once bootloader failed during the validation of the transaction, // it will fail again and again. At the same time we have to keep the same storage, because we want to keep the nonce holder contract state. - // The easiest way in terms of lifetimes is to reuse vm_builder to achieve it. + // The easiest way in terms of lifetimes is to reuse `vm_builder` to achieve it. vm.reset_state(true); let mut transaction_data: TransactionData = account .get_l2_tx_for_execute_with_nonce( diff --git a/core/lib/multivm/src/versions/vm_latest/tests/require_eip712.rs b/core/lib/multivm/src/versions/vm_latest/tests/require_eip712.rs index c03e5fe64212..de4f27436afb 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/require_eip712.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/require_eip712.rs @@ -37,8 +37,8 @@ impl VmTester { /// Currently we support both, but in the future, we should allow only EIP712 transactions to access the AA accounts. async fn test_require_eip712() { // Use 3 accounts: - // - private_address - EOA account, where we have the key - // - account_address - AA account, where the contract is deployed + // - `private_address` - EOA account, where we have the key + // - `account_address` - AA account, where the contract is deployed // - beneficiary - an EOA account, where we'll try to transfer the tokens. let account_abstraction = Account::random(); let mut private_account = Account::random(); @@ -56,8 +56,8 @@ async fn test_require_eip712() { let chain_id: u32 = 270; - // First, let's set the owners of the AA account to the private_address. - // (so that messages signed by private_address, are authorized to act on behalf of the AA account). + // First, let's set the owners of the AA account to the `private_address`. + // (so that messages signed by `private_address`, are authorized to act on behalf of the AA account). let set_owners_function = contract.function("setOwners").unwrap(); let encoded_input = set_owners_function .encode_input(&[Token::Array(vec![Token::Address(private_account.address)])]) diff --git a/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs b/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs index 25f1361f14d0..9574c434062d 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs @@ -76,7 +76,7 @@ impl VmTester { if !self.custom_contracts.is_empty() { println!("Inserting custom contracts is not yet supported") - // insert_contracts(&mut self.storage, &self.custom_contracts); + // `insert_contracts(&mut self.storage, &self.custom_contracts);` } let mut l1_batch = self.vm.batch_env.clone(); diff --git a/core/lib/multivm/src/versions/vm_latest/tests/upgrade.rs b/core/lib/multivm/src/versions/vm_latest/tests/upgrade.rs index b5c493ca7075..05ef5f57132d 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/upgrade.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/upgrade.rs @@ -45,7 +45,7 @@ fn test_protocol_upgrade_is_first() { let protocol_upgrade_transaction = get_forced_deploy_tx(&[ForceDeployment { // The bytecode hash to put on an address bytecode_hash, - // The address on which to deploy the bytecodehash to + // The address on which to deploy the bytecode hash to address: H160::random(), // Whether to run the constructor on the force deployment call_constructor: false, @@ -59,7 +59,7 @@ fn test_protocol_upgrade_is_first() { let another_protocol_upgrade_transaction = get_forced_deploy_tx(&[ForceDeployment { // The bytecode hash to put on an address bytecode_hash, - // The address on which to deploy the bytecodehash to + // The address on which to deploy the bytecode hash to address: H160::random(), // Whether to run the constructor on the force deployment call_constructor: false, @@ -141,7 +141,7 @@ fn test_force_deploy_upgrade() { let transaction = get_forced_deploy_tx(&[ForceDeployment { // The bytecode hash to put on an address bytecode_hash, - // The address on which to deploy the bytecodehash to + // The address on which to deploy the bytecode hash to address: address_to_deploy, // Whether to run the constructor on the force deployment call_constructor: false, @@ -180,7 +180,7 @@ fn test_complex_upgrader() { let msg_sender_test_hash = hash_bytecode(&read_msg_sender_test()); // Let's assume that the bytecode for the implementation of the complex upgrade - // is already deployed in some address in userspace + // is already deployed in some address in user space let upgrade_impl = H160::random(); let account_code_key = get_code_key(&upgrade_impl); @@ -240,7 +240,7 @@ fn test_complex_upgrader() { struct ForceDeployment { // The bytecode hash to put on an address bytecode_hash: H256, - // The address on which to deploy the bytecodehash to + // The address on which to deploy the bytecode hash to address: Address, // Whether to run the constructor on the force deployment call_constructor: bool, @@ -295,8 +295,8 @@ fn get_forced_deploy_tx(deployment: &[ForceDeployment]) -> Transaction { // Returns the transaction that performs a complex protocol upgrade. // The first param is the address of the implementation of the complex upgrade -// in user-space, while the next 3 params are params of the implenentaiton itself -// For the explanatation for the parameters, please refer to: +// in user-space, while the next 3 params are params of the implementation itself +// For the explanation for the parameters, please refer to: // etc/contracts-test-data/complex-upgrade/complex-upgrade.sol fn get_complex_upgrade_tx( implementation_address: Address, diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs index 018272365f8f..cba23d14d23a 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs @@ -54,7 +54,7 @@ pub(crate) struct DefaultExecutionTracer { pub(crate) result_tracer: ResultTracer, // This tracer is designed specifically for calculating refunds. Its separation from the custom tracer // ensures static dispatch, enhancing performance by avoiding dynamic dispatch overhead. - // Additionally, being an internal tracer, it saves the results directly to VmResultAndLogs. + // Additionally, being an internal tracer, it saves the results directly to `VmResultAndLogs`. pub(crate) refund_tracer: Option>, // The pubdata tracer is responsible for inserting the pubdata packing information into the bootloader // memory at the end of the batch. Its separation from the custom tracer @@ -298,7 +298,7 @@ impl DefaultExecutionTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/pubdata_tracer.rs b/core/lib/multivm/src/versions/vm_latest/tracers/pubdata_tracer.rs index 5773e3797e2b..92213619055c 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/pubdata_tracer.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/pubdata_tracer.rs @@ -56,7 +56,7 @@ impl PubdataTracer { impl PubdataTracer { // Packs part of L1 Messenger total pubdata that corresponds to - // L2toL1Logs sent in the block + // `L2toL1Logs` sent in the block fn get_total_user_logs( &self, state: &ZkSyncVmState, diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs index e852fba1dac8..c67f086dc119 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs @@ -214,8 +214,8 @@ impl VmTracer for RefundsTracer { #[vise::register] static METRICS: vise::Global = vise::Global::new(); - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = self.requested_refund() { assert!( self.operator_refund.is_none(), diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_latest/tracers/result_tracer.rs index b3412587725c..6894e9167e3d 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/result_tracer.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/result_tracer.rs @@ -54,7 +54,7 @@ impl ResultTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 @@ -150,7 +150,7 @@ impl ResultTracer { }); } VmExecutionResult::Revert(output) => { - // Unlike VmHook::ExecutionResult, vm has completely finished and returned not only the revert reason, + // Unlike `VmHook::ExecutionResult`, vm has completely finished and returned not only the revert reason, // but with bytecode, which represents the type of error from the bootloader side let revert_reason = TxRevertReason::parse_error(&output); diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/utils.rs b/core/lib/multivm/src/versions/vm_latest/tracers/utils.rs index 93710586fdac..1046c7a4c4a0 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/utils.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/utils.rs @@ -57,7 +57,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_latest/types/internals/pubdata.rs b/core/lib/multivm/src/versions/vm_latest/types/internals/pubdata.rs index aa9e8a7eda70..38489a6c8e92 100644 --- a/core/lib/multivm/src/versions/vm_latest/types/internals/pubdata.rs +++ b/core/lib/multivm/src/versions/vm_latest/types/internals/pubdata.rs @@ -24,14 +24,14 @@ impl PubdataInput { } = self; // Encoding user L2->L1 logs. - // Format: [(numberOfL2ToL1Logs as u32) || l2tol1logs[1] || ... || l2tol1logs[n]] + // Format: `[(numberOfL2ToL1Logs as u32) || l2tol1logs[1] || ... || l2tol1logs[n]]` l1_messenger_pubdata.extend((user_logs.len() as u32).to_be_bytes()); for l2tol1log in user_logs { l1_messenger_pubdata.extend(l2tol1log.packed_encoding()); } // Encoding L2->L1 messages - // Format: [(numberOfMessages as u32) || (messages[1].len() as u32) || messages[1] || ... || (messages[n].len() as u32) || messages[n]] + // Format: `[(numberOfMessages as u32) || (messages[1].len() as u32) || messages[1] || ... || (messages[n].len() as u32) || messages[n]]` l1_messenger_pubdata.extend((l2_to_l1_messages.len() as u32).to_be_bytes()); for message in l2_to_l1_messages { l1_messenger_pubdata.extend((message.len() as u32).to_be_bytes()); @@ -39,7 +39,7 @@ impl PubdataInput { } // Encoding bytecodes - // Format: [(numberOfBytecodes as u32) || (bytecodes[1].len() as u32) || bytecodes[1] || ... || (bytecodes[n].len() as u32) || bytecodes[n]] + // Format: `[(numberOfBytecodes as u32) || (bytecodes[1].len() as u32) || bytecodes[1] || ... || (bytecodes[n].len() as u32) || bytecodes[n]]` l1_messenger_pubdata.extend((published_bytecodes.len() as u32).to_be_bytes()); for bytecode in published_bytecodes { l1_messenger_pubdata.extend((bytecode.len() as u32).to_be_bytes()); @@ -47,7 +47,7 @@ impl PubdataInput { } // Encoding state diffs - // Format: [size of compressed state diffs u32 || compressed state diffs || (# state diffs: intial + repeated) as u32 || sorted state diffs by ] + // Format: `[size of compressed state diffs u32 || compressed state diffs || (# state diffs: intial + repeated) as u32 || sorted state diffs by ]` let state_diffs_compressed = compress_state_diffs(state_diffs.clone()); l1_messenger_pubdata.extend(state_diffs_compressed); diff --git a/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs index f5b97ca9793e..b312ada5b209 100644 --- a/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs @@ -234,7 +234,7 @@ impl TransactionData { let l2_tx: L2Tx = self.clone().try_into().unwrap(); let transaction_request: TransactionRequest = l2_tx.into(); - // It is assumed that the TransactionData always has all the necessary components to recover the hash. + // It is assumed that the `TransactionData` always has all the necessary components to recover the hash. transaction_request .get_tx_hash(chain_id) .expect("Could not recover L2 transaction hash") diff --git a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs index ef0b26eeea1a..9711b12ecbd4 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs @@ -19,7 +19,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, diff --git a/core/lib/multivm/src/versions/vm_latest/utils/l2_blocks.rs b/core/lib/multivm/src/versions/vm_latest/utils/l2_blocks.rs index 5dd26c4c0277..e5832f7f5879 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/l2_blocks.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/l2_blocks.rs @@ -68,7 +68,7 @@ pub fn load_last_l2_block(storage: StoragePtr) -> Option u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); @@ -23,8 +23,8 @@ pub(crate) fn derive_overhead( let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -38,15 +38,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); + // ``` vec![ (coefficients.ergs_limit_overhead_coeficient @@ -64,7 +66,7 @@ pub(crate) fn derive_overhead( /// Contains the coefficients with which the overhead for transactions will be calculated. /// All of the coefficients should be <= 1. There are here to provide a certain "discount" for normal transactions /// at the risk of malicious transactions that may close the block prematurely. -/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coeficient` MUST +/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coefficient` MUST /// result in an integer number #[derive(Debug, Clone, Copy)] pub struct OverheadCoefficients { @@ -102,8 +104,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -132,28 +134,28 @@ pub(crate) fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -164,7 +166,7 @@ pub(crate) fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -177,13 +179,17 @@ pub(crate) fn get_amortized_overhead( }; // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, - // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. + // since the pubdata is not published. If decided to use the pubdata overhead, it needs to be updated. + // ``` // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + // Throwing off the `ceil`, while may provide marginally lower // overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // ``` // ceil(OB * (TL - OE) / (EP * MP)) >= OE // // OB * (TL - OE) / (MP * EP) > OE - 1 @@ -196,7 +202,7 @@ pub(crate) fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -205,7 +211,7 @@ pub(crate) fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - + // // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) @@ -214,6 +220,7 @@ pub(crate) fn get_amortized_overhead( // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + // ``` let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from( @@ -235,9 +242,9 @@ pub(crate) fn get_amortized_overhead( let limit_after_deducting_overhead = total_gas_limit - overhead; // During double checking of the overhead, the bootloader will assume that the - // body of the transaction does not have any more than MAX_L2_TX_GAS_LIMIT ergs available to it. + // body of the transaction does not have any more than `MAX_L2_TX_GAS_LIMIT` ergs available to it. if limit_after_deducting_overhead.as_u64() > MAX_L2_TX_GAS_LIMIT { - // We derive the same overhead that would exist for the MAX_L2_TX_GAS_LIMIT ergs + // We derive the same overhead that would exist for the `MAX_L2_TX_GAS_LIMIT` ergs derive_overhead( MAX_L2_TX_GAS_LIMIT as u32, gas_per_pubdata_byte_limit, diff --git a/core/lib/multivm/src/versions/vm_m5/errors/tx_revert_reason.rs b/core/lib/multivm/src/versions/vm_m5/errors/tx_revert_reason.rs index 9259dd87a376..439524108a95 100644 --- a/core/lib/multivm/src/versions/vm_m5/errors/tx_revert_reason.rs +++ b/core/lib/multivm/src/versions/vm_m5/errors/tx_revert_reason.rs @@ -7,11 +7,11 @@ use super::{BootloaderErrorCode, VmRevertReason}; // Reasons why the transaction executed inside the bootloader could fail. #[derive(Debug, Clone, PartialEq)] pub enum TxRevertReason { - // Can only be returned in EthCall execution mode (=ExecuteOnly) + // Can only be returned in EthCall execution mode `(=ExecuteOnly)` EthCall(VmRevertReason), // Returned when the execution of an L2 transaction has failed TxReverted(VmRevertReason), - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` ValidationFailed(VmRevertReason), PaymasterValidationFailed(VmRevertReason), PrePaymasterPreparationFailed(VmRevertReason), @@ -20,7 +20,7 @@ pub enum TxRevertReason { FailedToChargeFee(VmRevertReason), // Emitted when trying to call a transaction from an account that has not // been deployed as an account (i.e. the `from` is just a contract). - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` FromIsNotAnAccount, // Currently cannot be returned. Should be removed when refactoring errors. InnerTxError, @@ -98,7 +98,7 @@ impl TxRevertReason { BootloaderErrorCode::UnacceptablePubdataPrice => { Self::UnexpectedVMBehavior("UnacceptablePubdataPrice".to_owned()) } - // This is different from AccountTxValidationFailed error in a way that it means that + // This is different from `AccountTxValidationFailed` error in a way that it means that // the error was not produced by the account itself, but for some other unknown reason (most likely not enough gas) BootloaderErrorCode::TxValidationError => Self::ValidationFailed(revert_reason), // Note, that `InnerTxError` is derived only after the actual tx execution, so diff --git a/core/lib/multivm/src/versions/vm_m5/history_recorder.rs b/core/lib/multivm/src/versions/vm_m5/history_recorder.rs index 7a158b4dea79..f744be32d0bf 100644 --- a/core/lib/multivm/src/versions/vm_m5/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_m5/history_recorder.rs @@ -19,13 +19,13 @@ pub type MemoryWithHistory = HistoryRecorder; pub type FrameManagerWithHistory = HistoryRecorder>; pub type IntFrameManagerWithHistory = FrameManagerWithHistory>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 -// can be used. This can sometimes vioalate monotonicity of the timestamp within the +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` +// can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_m5/memory.rs b/core/lib/multivm/src/versions/vm_m5/memory.rs index dc58450263e4..34c083b21f79 100644 --- a/core/lib/multivm/src/versions/vm_m5/memory.rs +++ b/core/lib/multivm/src/versions/vm_m5/memory.rs @@ -34,7 +34,7 @@ impl OracleWithHistory for SimpleMemory { impl SimpleMemory { pub fn populate(&mut self, elements: Vec<(u32, Vec)>, timestamp: Timestamp) { for (page, values) in elements.into_iter() { - // Resizing the pages array to fit the page. + // Re-sizing the pages array to fit the page. let len = values.len(); assert!(len <= MEMORY_CELLS_OTHER_PAGES); @@ -261,7 +261,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -276,7 +276,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -284,7 +284,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_m5/oracles/mod.rs b/core/lib/multivm/src/versions/vm_m5/oracles/mod.rs index 6b821c68e9d2..c43c9987de5d 100644 --- a/core/lib/multivm/src/versions/vm_m5/oracles/mod.rs +++ b/core/lib/multivm/src/versions/vm_m5/oracles/mod.rs @@ -1,9 +1,9 @@ use zk_evm_1_3_1::aux_structures::Timestamp; // All the changes to the events in the DB will be applied after the tx is executed, -// so fow now it is fine. +// so for now it is fine. pub use zk_evm_1_3_1::reference_impls::event_sink::InMemoryEventSink as EventSinkOracle; // We will discard RAM as soon as the execution of a tx ends, so -// it is ok for now to use SimpleMemory +// it is ok for now to use `SimpleMemory` pub use zk_evm_1_3_1::reference_impls::memory::SimpleMemory as RamOracle; pub use zk_evm_1_3_1::testing::simple_tracer::NoopTracer; diff --git a/core/lib/multivm/src/versions/vm_m5/oracles/storage.rs b/core/lib/multivm/src/versions/vm_m5/oracles/storage.rs index c81b90f9c9cf..b38da4051f3b 100644 --- a/core/lib/multivm/src/versions/vm_m5/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_m5/oracles/storage.rs @@ -183,6 +183,7 @@ impl VmStorageOracle for StorageOracle { _monotonic_cycle_counter: u32, query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -192,6 +193,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -274,7 +276,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } diff --git a/core/lib/multivm/src/versions/vm_m5/oracles/tracer.rs b/core/lib/multivm/src/versions/vm_m5/oracles/tracer.rs index ac370f832e4c..481e8a7e02e6 100644 --- a/core/lib/multivm/src/versions/vm_m5/oracles/tracer.rs +++ b/core/lib/multivm/src/versions/vm_m5/oracles/tracer.rs @@ -178,7 +178,7 @@ fn touches_allowed_context(address: Address, key: U256) -> bool { return false; } - // Only chain_id is allowed to be touched. + // Only `chain_id` is allowed to be touched. key == U256::from(0u32) } @@ -303,7 +303,7 @@ impl ValidationTracer { return true; } - // The pair of MSG_VALUE_SIMULATOR_ADDRESS & L2_ETH_TOKEN_ADDRESS simulates the behavior of transferring ETH + // The pair of `MSG_VALUE_SIMULATOR_ADDRESS` & `L2_ETH_TOKEN_ADDRESS` simulates the behavior of transferring ETH // that is safe for the DDoS protection rules. if valid_eth_token_call(address, msg_sender) { return true; @@ -347,20 +347,20 @@ impl ValidationTracer { let (potential_address_bytes, potential_position_bytes) = calldata.split_at(32); let potential_address = be_bytes_to_safe_address(potential_address_bytes); - // If the validation_address is equal to the potential_address, - // then it is a request that could be used for mapping of kind mapping(address => ...). + // If the `validation_address` is equal to the `potential_address`, + // then it is a request that could be used for mapping of kind `mapping(address => ...)`. // - // If the potential_position_bytes were already allowed before, then this keccak might be used - // for ERC-20 allowance or any other of mapping(address => mapping(...)) + // If the `potential_position_bytes` were already allowed before, then this keccak might be used + // for ERC-20 allowance or any other of `mapping(address => mapping(...))` if potential_address == Some(validated_address) || self .auxilary_allowed_slots .contains(&H256::from_slice(potential_position_bytes)) { - // This is request that could be used for mapping of kind mapping(address => ...) + // This is request that could be used for mapping of kind `mapping(address => ...)` // We could theoretically wait for the slot number to be returned by the - // keccak256 precompile itself, but this would complicate the code even further + // `keccak256` precompile itself, but this would complicate the code even further // so let's calculate it here. let slot = keccak256(calldata); @@ -719,7 +719,7 @@ impl PubdataSpentTracer for BootloaderTracer {} impl BootloaderTracer { fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 @@ -762,7 +762,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_m5/refunds.rs b/core/lib/multivm/src/versions/vm_m5/refunds.rs index 8e084fd9ee32..fd4e2788f035 100644 --- a/core/lib/multivm/src/versions/vm_m5/refunds.rs +++ b/core/lib/multivm/src/versions/vm_m5/refunds.rs @@ -75,7 +75,7 @@ impl VmInstance { ) -> u32 { // TODO (SMA-1715): Make users pay for the block overhead 0 - + // ``` // let pubdata_published = self.pubdata_published(from_timestamp); // // let total_gas_spent = gas_remaining_before - self.gas_remaining(); @@ -120,6 +120,7 @@ impl VmInstance { // ); // 0 // } + // ``` } // TODO (SMA-1715): Make users pay for the block overhead @@ -133,39 +134,39 @@ impl VmInstance { _l2_l1_logs: usize, ) -> u32 { 0 - + // ``` // let overhead_for_block_gas = U256::from(crate::transaction_data::block_overhead_gas( // gas_per_pubdata_byte_limit, // )); - + // // let encoded_len = U256::from(encoded_len); // let pubdata_published = U256::from(pubdata_published); // let gas_spent_on_computation = U256::from(gas_spent_on_computation); // let number_of_decommitment_requests = U256::from(number_of_decommitment_requests); // let l2_l1_logs = U256::from(l2_l1_logs); - + // // let tx_slot_overhead = ceil_div_u256(overhead_for_block_gas, MAX_TXS_IN_BLOCK.into()); - + // // let overhead_for_length = ceil_div_u256( // encoded_len * overhead_for_block_gas, // BOOTLOADER_TX_ENCODING_SPACE.into(), // ); - + // // let actual_overhead_for_pubdata = ceil_div_u256( // pubdata_published * overhead_for_block_gas, // MAX_PUBDATA_PER_BLOCK.into(), // ); - + // // let actual_gas_limit_overhead = ceil_div_u256( // gas_spent_on_computation * overhead_for_block_gas, // MAX_BLOCK_MULTIINSTANCE_GAS_LIMIT.into(), // ); - + // // let code_decommitter_sorter_circuit_overhead = ceil_div_u256( // number_of_decommitment_requests * overhead_for_block_gas, // GEOMETRY_CONFIG.limit_for_code_decommitter_sorter.into(), // ); - + // // let l1_l2_logs_overhead = ceil_div_u256( // l2_l1_logs * overhead_for_block_gas, // std::cmp::min( @@ -174,7 +175,7 @@ impl VmInstance { // ) // .into(), // ); - + // // let overhead = vec![ // tx_slot_overhead, // overhead_for_length, @@ -186,8 +187,9 @@ impl VmInstance { // .into_iter() // .max() // .unwrap(); - + // // overhead.as_u32() + // ``` } pub(crate) fn get_tx_gas_limit(&self, tx_index: usize) -> u32 { diff --git a/core/lib/multivm/src/versions/vm_m5/tests/bootloader.rs b/core/lib/multivm/src/versions/vm_m5/tests/bootloader.rs index 1034e8595936..d9e07c5068d0 100644 --- a/core/lib/multivm/src/versions/vm_m5/tests/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m5/tests/bootloader.rs @@ -1,9 +1,10 @@ +// ``` // //! // //! Tests for the bootloader // //! The description for each of the tests can be found in the corresponding `.yul` file. // //! // #![cfg_attr(test, allow(unused_imports))] - +// // use crate::errors::{VmRevertReason, VmRevertReasonParsingResult}; // use crate::memory::SimpleMemory; // use crate::oracles::tracer::{ @@ -58,7 +59,7 @@ // u256_to_h256, // }; // use zksync_utils::{h256_to_account_address, u256_to_account_address}; - +// // use crate::{transaction_data::TransactionData, OracleTools}; // use std::time; // use zksync_contracts::{ @@ -88,10 +89,10 @@ // MAX_TXS_IN_BLOCK, SYSTEM_CONTEXT_ADDRESS, SYSTEM_CONTEXT_GAS_PRICE_POSITION, // SYSTEM_CONTEXT_MINIMAL_BASE_FEE, SYSTEM_CONTEXT_TX_ORIGIN_POSITION, // }; - +// // use once_cell::sync::Lazy; // use zksync_system_constants::ZKPORTER_IS_AVAILABLE; - +// // fn run_vm_with_custom_factory_deps<'a>( // oracle_tools: &'a mut OracleTools<'a, false>, // block_context: BlockContext, @@ -110,7 +111,7 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // vm.bootloader_state.add_tx_data(encoded_tx.len()); // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, @@ -124,17 +125,17 @@ // ), // Timestamp(0), // ); - +// // let result = vm.execute_next_tx().err(); - +// // assert_eq!(expected_error, result); // } - +// // fn get_balance(token_id: AccountTreeId, account: &Address, main_storage: StoragePtr<'_>) -> U256 { // let key = storage_key_for_standard_token_balance(token_id, account); // h256_to_u256(main_storage.borrow_mut().get_value(&key)) // } - +// // #[test] // fn test_dummy_bootloader() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -143,18 +144,18 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr); // let (block_context, block_properties) = create_test_block_params(); // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -163,22 +164,22 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::BlockPostprocessing); - +// // // Dummy bootloader should not panic // assert!(res.revert_reason.is_none()); - +// // let correct_first_cell = U256::from_str_radix("123123123", 16).unwrap(); - +// // verify_required_memory( // &vm.state, // vec![(correct_first_cell, BOOTLOADER_HEAP_PAGE, 0)], // ); // } - +// // #[test] // fn test_bootloader_out_of_gas() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -187,20 +188,20 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr); // let (block_context, block_properties) = create_test_block_params(); - +// // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); - +// // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // // init vm with only 100 ergs // let mut vm = init_vm_inner( // &mut oracle_tools, @@ -210,16 +211,16 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let res = vm.execute_block_tip(); - +// // assert_eq!(res.revert_reason, Some(TxRevertReason::BootloaderOutOfGas)); // } - +// // fn verify_required_storage(state: &ZkSyncVmState<'_>, required_values: Vec<(H256, StorageKey)>) { // for (required_value, key) in required_values { // let current_value = state.storage.storage.read_from_storage(&key); - +// // assert_eq!( // u256_to_h256(current_value), // required_value, @@ -227,7 +228,7 @@ // ); // } // } - +// // fn verify_required_memory(state: &ZkSyncVmState<'_>, required_values: Vec<(U256, u32, u32)>) { // for (required_value, memory_page, cell) in required_values { // let current_value = state @@ -236,21 +237,21 @@ // assert_eq!(current_value, required_value); // } // } - +// // #[test] // fn test_default_aa_interaction() { // // In this test, we aim to test whether a simple account interaction (without any fee logic) // // will work. The account will try to deploy a simple contract from integration tests. - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let operator_address = block_context.context.operator_address; // let base_fee = block_context.base_fee; // // We deploy here counter contract, because its logic is trivial @@ -271,16 +272,16 @@ // ) // .into(); // let tx_data: TransactionData = tx.clone().into(); - +// // let maximal_fee = tx_data.gas_limit * tx_data.max_fee_per_gas; // let sender_address = tx_data.from(); // // set balance - +// // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -290,17 +291,17 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); - +// // let tx_execution_result = vm // .execute_next_tx() // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); @@ -310,28 +311,28 @@ // "Bootloader was not expected to revert: {:?}", // res.revert_reason // ); - +// // // Both deployment and ordinary nonce should be incremented by one. // let account_nonce_key = get_nonce_key(&sender_address); // let expected_nonce = TX_NONCE_INCREMENT + DEPLOYMENT_NONCE_INCREMENT; - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(expected_nonce), account_nonce_key), // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert!(!tx_has_failed(&vm.state, 0)); - +// // let expected_fee = // maximal_fee - U256::from(tx_execution_result.gas_refunded) * U256::from(base_fee); // let operator_balance = get_balance( @@ -339,32 +340,32 @@ // &operator_address, // vm.state.storage.storage.get_ptr(), // ); - +// // assert!( // operator_balance == expected_fee, // "Operator did not receive his fee" // ); // } - +// // fn execute_vm_with_predetermined_refund(txs: Vec, refunds: Vec) -> VmBlockResult { // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // set balance // for tx in txs.iter() { // let sender_address = tx.initiator_account(); // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); // } - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -373,7 +374,7 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // let codes_for_decommiter = txs // .iter() // .flat_map(|tx| { @@ -386,42 +387,42 @@ // .collect::)>>() // }) // .collect(); - +// // vm.state.decommittment_processor.populate( // codes_for_decommiter, // Timestamp(vm.state.local_state.timestamp), // ); - +// // let memory_with_suggested_refund = get_bootloader_memory( // txs.into_iter().map(Into::into).collect(), // refunds, // TxExecutionMode::VerifyExecute, // BlockContextMode::NewBlock(block_context, Default::default()), // ); - +// // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, // memory_with_suggested_refund, // Timestamp(0), // ); - +// // vm.execute_till_block_end(BootloaderJobType::TransactionExecution) // } - +// // #[test] // fn test_predetermined_refunded_gas() { // // In this test, we compare the execution of the bootloader with the predefined // // refunded gas and without them - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let base_fee = block_context.base_fee; // // We deploy here counter contract, because its logic is trivial // let contract_code = read_test_contract(); @@ -439,15 +440,15 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); - +// // // set balance // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -456,19 +457,19 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); - +// // let tx_execution_result = vm // .execute_next_tx() // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // // If the refund provided by the operator or the final refund are the 0 // // there is no impact of the operator's refund at all and so this test does not // // make much sense. @@ -480,14 +481,14 @@ // tx_execution_result.gas_refunded > 0, // "The final refund is 0" // ); - +// // let mut result = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); // assert!( // result.full_result.revert_reason.is_none(), // "Bootloader was not expected to revert: {:?}", // result.full_result.revert_reason // ); - +// // let mut result_with_predetermined_refund = execute_vm_with_predetermined_refund( // vec![tx], // vec![tx_execution_result.operator_suggested_refund], @@ -498,7 +499,7 @@ // .full_result // .used_contract_hashes // .sort(); - +// // assert_eq!( // result.full_result.events, // result_with_predetermined_refund.full_result.events @@ -520,18 +521,18 @@ // .used_contract_hashes // ); // } - +// // #[derive(Debug, Clone)] // enum TransactionRollbackTestInfo { // Rejected(Transaction, TxRevertReason), // Processed(Transaction, bool, TxExecutionStatus), // } - +// // impl TransactionRollbackTestInfo { // fn new_rejected(transaction: Transaction, revert_reason: TxRevertReason) -> Self { // Self::Rejected(transaction, revert_reason) // } - +// // fn new_processed( // transaction: Transaction, // should_be_rollbacked: bool, @@ -539,28 +540,28 @@ // ) -> Self { // Self::Processed(transaction, should_be_rollbacked, expected_status) // } - +// // fn get_transaction(&self) -> &Transaction { // match self { // TransactionRollbackTestInfo::Rejected(tx, _) => tx, // TransactionRollbackTestInfo::Processed(tx, _, _) => tx, // } // } - +// // fn rejection_reason(&self) -> Option { // match self { // TransactionRollbackTestInfo::Rejected(_, revert_reason) => Some(revert_reason.clone()), // TransactionRollbackTestInfo::Processed(_, _, _) => None, // } // } - +// // fn should_rollback(&self) -> bool { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => true, // TransactionRollbackTestInfo::Processed(_, x, _) => *x, // } // } - +// // fn expected_status(&self) -> TxExecutionStatus { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => { @@ -570,7 +571,7 @@ // } // } // } - +// // // Accepts the address of the sender as well as the list of pairs of its transactions // // and whether these transactions should succeed. // fn execute_vm_with_possible_rollbacks( @@ -584,13 +585,13 @@ // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // Setting infinite balance for the sender. // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -599,7 +600,7 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // for test_info in transactions { // vm.save_current_vm_as_snapshot(); // let vm_state_before_tx = vm.dump_inner_state(); @@ -608,7 +609,7 @@ // test_info.get_transaction(), // TxExecutionMode::VerifyExecute, // ); - +// // match vm.execute_next_tx() { // Err(reason) => { // assert_eq!(test_info.rejection_reason(), Some(reason)); @@ -622,11 +623,11 @@ // ); // } // }; - +// // if test_info.should_rollback() { // // Some error has occurred, we should reject the transaction // vm.rollback_to_latest_snapshot(); - +// // // vm_state_before_tx. // let state_after_rollback = vm.dump_inner_state(); // assert_eq!( @@ -635,7 +636,7 @@ // ); // } // } - +// // let VmBlockResult { // full_result: mut result, // .. @@ -643,10 +644,10 @@ // // Used contract hashes are retrieved in unordered manner. // // However it must be sorted for the comparisons in tests to work // result.used_contract_hashes.sort(); - +// // result // } - +// // // Sets the signature for an L2 transaction and returns the same transaction // // but this different signature. // fn change_signature(mut tx: Transaction, signature: Vec) -> Transaction { @@ -657,22 +658,22 @@ // } // _ => unreachable!(), // }; - +// // tx // } - +// // #[test] // fn test_vm_rollbacks() { // let (block_context, block_properties): (DerivedBlockContext, BlockProperties) = { // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let base_fee = U256::from(block_context.base_fee); - +// // let sender_private_key = H256::random(); // let contract_code = read_test_contract(); - +// // let tx_nonce_0: Transaction = get_deploy_tx( // sender_private_key, // Nonce(0), @@ -715,13 +716,13 @@ // }, // ) // .into(); - +// // let wrong_signature_length_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 32]); // let wrong_v_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 65]); // let wrong_signature_tx = change_signature(tx_nonce_0.clone(), vec![27u8; 65]); - +// // let sender_address = tx_nonce_0.initiator_account(); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -745,7 +746,7 @@ // block_context, // block_properties, // ); - +// // let incorrect_nonce = TxRevertReason::ValidationFailed(VmRevertReason::General { // msg: "Incorrect nonce".to_string(), // }); @@ -761,7 +762,7 @@ // let signature_is_incorrect = TxRevertReason::ValidationFailed(VmRevertReason::General { // msg: "Account validation returned invalid magic value. Most often this means that the signature is incorrect".to_string(), // }); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -806,11 +807,11 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); - +// // let loadnext_contract = get_loadnext_contract(); - +// // let loadnext_constructor_data = encode(&[Token::Uint(U256::from(100))]); // let loadnext_deploy_tx: Transaction = get_deploy_tx( // sender_private_key, @@ -833,7 +834,7 @@ // false, // TxExecutionStatus::Success, // ); - +// // let get_load_next_tx = |params: LoadnextContractExecutionParams, nonce: Nonce| { // // Here we test loadnext with various kinds of operations // let tx: Transaction = mock_loadnext_test_call( @@ -849,10 +850,10 @@ // params, // ) // .into(); - +// // tx // }; - +// // let loadnext_tx_0 = get_load_next_tx( // LoadnextContractExecutionParams { // reads: 100, @@ -875,7 +876,7 @@ // }, // Nonce(2), // ); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -894,7 +895,7 @@ // block_context, // block_properties, // ); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -935,10 +936,10 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); // } - +// // // Inserts the contracts into the test environment, bypassing the // // deployer system contract. Besides the reference to storage // // it accepts a `contracts` tuple of information about the contract @@ -951,13 +952,13 @@ // .iter() // .flat_map(|(contract, is_account)| { // let mut new_logs = vec![]; - +// // let deployer_code_key = get_code_key(contract.account_id.address()); // new_logs.push(StorageLog::new_write_log( // deployer_code_key, // hash_bytecode(&contract.bytecode), // )); - +// // if *is_account { // let is_account_key = get_is_account_key(contract.account_id.address()); // new_logs.push(StorageLog::new_write_log( @@ -965,19 +966,19 @@ // u256_to_h256(1u32.into()), // )); // } - +// // new_logs // }) // .collect(); // raw_storage.process_transaction_logs(&logs); - +// // for (contract, _) in contracts { // raw_storage.store_contract(*contract.account_id.address(), contract.bytecode.clone()); // raw_storage.store_factory_dep(hash_bytecode(&contract.bytecode), contract.bytecode); // } // raw_storage.save(L1BatchNumber(0)); // } - +// // enum NonceHolderTestMode { // SetValueUnderNonce, // IncreaseMinNonceBy5, @@ -986,7 +987,7 @@ // IncreaseMinNonceBy1, // SwitchToArbitraryOrdering, // } - +// // impl From for u8 { // fn from(mode: NonceHolderTestMode) -> u8 { // match mode { @@ -999,7 +1000,7 @@ // } // } // } - +// // fn get_nonce_holder_test_tx( // nonce: U256, // account_address: Address, @@ -1021,11 +1022,11 @@ // reserved: [U256::zero(); 4], // data: vec![12], // signature: vec![test_mode.into()], - +// // ..Default::default() // } // } - +// // fn run_vm_with_raw_tx<'a>( // oracle_tools: &'a mut OracleTools<'a, false>, // block_context: DerivedBlockContext, @@ -1042,7 +1043,7 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let overhead = tx.overhead_gas(); // push_raw_transaction_to_bootloader_memory( // &mut vm, @@ -1054,43 +1055,43 @@ // full_result: result, // .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); - +// // (result, tx_has_failed(&vm.state, 0)) // } - +// // #[test] // fn test_nonce_holder() { // let (block_context, block_properties): (DerivedBlockContext, BlockProperties) = { // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); - +// // let account_address = H160::random(); // let account = DeployedContract { // account_id: AccountTreeId::new(account_address), // bytecode: read_nonce_holder_tester(), // }; - +// // insert_contracts(&mut raw_storage, vec![(account, true)]); - +// // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // We deploy here counter contract, because its logic is trivial - +// // let key = storage_key_for_eth_balance(&account_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut run_nonce_test = |nonce: U256, // test_mode: NonceHolderTestMode, // error_message: Option, // comment: &'static str| { // let tx = get_nonce_holder_test_tx(nonce, account_address, test_mode, &block_context); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); // let (result, tx_has_failed) = // run_vm_with_raw_tx(&mut oracle_tools, block_context, &block_properties, tx); @@ -1109,7 +1110,7 @@ // assert!(!tx_has_failed, "{}", comment); // } // }; - +// // // Test 1: trying to set value under non sequential nonce value. // run_nonce_test( // 1u32.into(), @@ -1117,7 +1118,7 @@ // Some("Previous nonce has not been used".to_string()), // "Allowed to set value under non sequential value", // ); - +// // // Test 2: increase min nonce by 1 with sequential nonce ordering: // run_nonce_test( // 0u32.into(), @@ -1125,7 +1126,7 @@ // None, // "Failed to increment nonce by 1 for sequential account", // ); - +// // // Test 3: correctly set value under nonce with sequential nonce ordering: // run_nonce_test( // 1u32.into(), @@ -1133,7 +1134,7 @@ // None, // "Failed to set value under nonce sequential value", // ); - +// // // Test 5: migrate to the arbitrary nonce ordering: // run_nonce_test( // 2u32.into(), @@ -1141,7 +1142,7 @@ // None, // "Failed to switch to arbitrary ordering", // ); - +// // // Test 6: increase min nonce by 5 // run_nonce_test( // 6u32.into(), @@ -1149,7 +1150,7 @@ // None, // "Failed to increase min nonce by 5", // ); - +// // // Test 7: since the nonces in range [6,10] are no longer allowed, the // // tx with nonce 10 should not be allowed // run_nonce_test( @@ -1158,7 +1159,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse nonce below the minimal one", // ); - +// // // Test 8: we should be able to use nonce 13 // run_nonce_test( // 13u32.into(), @@ -1166,7 +1167,7 @@ // None, // "Did not allow to use unused nonce 10", // ); - +// // // Test 9: we should not be able to reuse nonce 13 // run_nonce_test( // 13u32.into(), @@ -1174,7 +1175,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse the same nonce twice", // ); - +// // // Test 10: we should be able to simply use nonce 14, while bumping the minimal nonce by 5 // run_nonce_test( // 14u32.into(), @@ -1182,7 +1183,7 @@ // None, // "Did not allow to use a bumped nonce", // ); - +// // // Test 6: Do not allow bumping nonce by too much // run_nonce_test( // 16u32.into(), @@ -1190,7 +1191,7 @@ // Some("The value for incrementing the nonce is too high".to_string()), // "Allowed for incrementing min nonce too much", // ); - +// // // Test 7: Do not allow not setting a nonce as used // run_nonce_test( // 16u32.into(), @@ -1199,7 +1200,7 @@ // "Allowed to leave nonce as unused", // ); // } - +// // #[test] // fn test_l1_tx_execution() { // // In this test, we try to execute a contract deployment from L1 @@ -1209,17 +1210,17 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr); // let (block_context, block_properties) = create_test_block_params(); - +// // // Here instead of marking code hash via the bootloader means, we will // // using L1->L2 communication, the same it would likely be done during the priority mode. // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_tx = get_l1_deploy_tx(&contract_code, &[]); // let l1_deploy_tx_data: TransactionData = l1_deploy_tx.clone().into(); - +// // let required_l2_to_l1_logs = vec![ // L2ToL1Log { // shard_id: 0, @@ -1238,9 +1239,9 @@ // value: u256_to_h256(U256::from(1u32)), // }, // ]; - +// // let sender_address = l1_deploy_tx_data.from(); - +// // oracle_tools.decommittment_processor.populate( // vec![( // h256_to_u256(contract_code_hash), @@ -1248,7 +1249,7 @@ // )], // Timestamp(0), // ); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -1258,44 +1259,44 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &l1_deploy_tx, TxExecutionMode::VerifyExecute); - +// // let res = vm.execute_next_tx().unwrap(); - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; // assert!(!tx_has_failed(&vm.state, 0)); - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert_eq!(res.result.logs.l2_to_l1_logs, required_l2_to_l1_logs); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, true); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); // let res = ExecutionMetrics::new(&vm.execute_next_tx().unwrap().result.logs, 0, 0, 0, 0); // assert_eq!(res.initial_storage_writes, 0); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, false); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); // let res = ExecutionMetrics::new(&vm.execute_next_tx().unwrap().result.logs, 0, 0, 0, 0); // assert_eq!(res.initial_storage_writes, 2); - +// // let repeated_writes = res.repeated_storage_writes; - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); // let res = ExecutionMetrics::new(&vm.execute_next_tx().unwrap().result.logs, 0, 0, 0, 0); // assert_eq!(res.initial_storage_writes, 1); // // We do the same storage write, so it will be deduplicated // assert_eq!(res.repeated_storage_writes, repeated_writes); - +// // let mut tx = get_l1_execute_test_contract_tx(deployed_address, false); // tx.execute.value = U256::from(1); // match &mut tx.common_data { @@ -1312,15 +1313,15 @@ // TxExecutionStatus::Failure, // "The transaction should fail" // ); - +// // let res = ExecutionMetrics::new(&execution_result.result.logs, 0, 0, 0, 0); - +// // // There are 2 initial writes here: // // - totalSupply of ETH token // // - balance of the refund recipient // assert_eq!(res.initial_storage_writes, 2); // } - +// // #[test] // fn test_invalid_bytecode() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -1328,16 +1329,16 @@ // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let (block_context, block_properties) = create_test_block_params(); - +// // let test_vm_with_custom_bytecode_hash = // |bytecode_hash: H256, expected_revert_reason: Option| { // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let (encoded_tx, predefined_overhead) = // get_l1_tx_with_custom_bytecode_hash(h256_to_u256(bytecode_hash)); - +// // run_vm_with_custom_factory_deps( // &mut oracle_tools, // block_context, @@ -1347,13 +1348,13 @@ // expected_revert_reason, // ); // }; - +// // let failed_to_mark_factory_deps = |msg: &str| { // TxRevertReason::FailedToMarkFactoryDependencies(VmRevertReason::General { // msg: msg.to_string(), // }) // }; - +// // // Here we provide the correctly-formatted bytecode hash of // // odd length, so it should work. // test_vm_with_custom_bytecode_hash( @@ -1363,7 +1364,7 @@ // ]), // None, // ); - +// // // Here we provide correctly formatted bytecode of even length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1375,7 +1376,7 @@ // "Code length in words must be odd", // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1387,7 +1388,7 @@ // "Incorrectly formatted bytecodeHash", // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1400,24 +1401,24 @@ // )), // ); // } - +// // #[test] // fn test_tracing_of_execution_errors() { // // In this test, we are checking that the execution errors are transmitted correctly from the bootloader. // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); - +// // let contract_address = Address::random(); // let error_contract = DeployedContract { // account_id: AccountTreeId::new(contract_address), // bytecode: read_error_contract(), // }; - +// // let tx = get_error_tx( // H256::random(), // Nonce(0), @@ -1429,16 +1430,16 @@ // gas_per_pubdata_limit: U256::from(50000u32), // }, // ); - +// // insert_contracts(&mut raw_storage, vec![(error_contract, false)]); - +// // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let key = storage_key_for_eth_balance(&tx.common_data.initiator_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1448,14 +1449,14 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &tx.into(), TxExecutionMode::VerifyExecute); - +// // let mut tracer = TransactionResultTracer::default(); // assert_eq!( // vm.execute_with_custom_tracer(&mut tracer), // VmExecutionStopReason::VmFinished, // "Tracer should never request stop" // ); - +// // match tracer.revert_reason { // Some(revert_reason) => { // let revert_reason = VmRevertReason::try_from(&revert_reason as &[u8]).unwrap(); @@ -1472,20 +1473,20 @@ // ), // } // } - +// // /// Checks that `TX_GAS_LIMIT_OFFSET` constant is correct. // #[test] // fn test_tx_gas_limit_offset() { // let gas_limit = U256::from(999999); - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let raw_storage = SecondaryStateStorage::new(db); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let contract_code = read_test_contract(); // let tx: Transaction = get_deploy_tx( // H256::random(), @@ -1499,9 +1500,9 @@ // }, // ) // .into(); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1511,7 +1512,7 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); - +// // let gas_limit_from_memory = vm // .state // .memory @@ -1522,20 +1523,20 @@ // .value; // assert_eq!(gas_limit_from_memory, gas_limit); // } - +// // #[test] // fn test_is_write_initial_behaviour() { // // In this test, we check result of `is_write_initial` at different stages. - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let base_fee = block_context.base_fee; // let account_pk = H256::random(); // let contract_code = read_test_contract(); @@ -1553,19 +1554,19 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); // let nonce_key = get_nonce_key(&sender_address); - +// // // Check that the next write to the nonce key will be initial. // assert!(storage_ptr.is_write_initial(&nonce_key)); - +// // // Set balance to be able to pay fee for txs. // let balance_key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&balance_key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1574,25 +1575,25 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute); - +// // vm.execute_next_tx() // .expect("Bootloader failed while processing the first transaction"); // // Check that `is_write_initial` still returns true for the nonce key. // assert!(storage_ptr.is_write_initial(&nonce_key)); // } - +// // pub fn get_l1_tx_with_custom_bytecode_hash(bytecode_hash: U256) -> (Vec, u32) { // let tx: TransactionData = get_l1_execute_test_contract_tx(Default::default(), false).into(); // let predefined_overhead = tx.overhead_gas_with_custom_factory_deps(vec![bytecode_hash]); // let tx_bytes = tx.abi_encode_with_custom_factory_deps(vec![bytecode_hash]); - +// // (bytes_to_be_words(tx_bytes), predefined_overhead) // } - +// // const L1_TEST_GAS_PER_PUBDATA_BYTE: u32 = 800; - +// // pub fn get_l1_execute_test_contract_tx(deployed_address: Address, with_panic: bool) -> Transaction { // let execute = execute_test_contract(deployed_address, with_panic); // Transaction { @@ -1606,10 +1607,10 @@ // received_timestamp_ms: 0, // } // } - +// // pub fn get_l1_deploy_tx(code: &[u8], calldata: &[u8]) -> Transaction { // let execute = get_create_execute(code, calldata); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender: H160::random(), @@ -1621,28 +1622,28 @@ // received_timestamp_ms: 0, // } // } - +// // fn read_test_contract() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json") // } - +// // fn read_nonce_holder_tester() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/custom-account/nonce-holder-test.sol/NonceHolderTest.json") // } - +// // fn read_error_contract() -> Vec { // read_bytecode( // "etc/contracts-test-data/artifacts-zk/contracts/error/error.sol/SimpleRequire.json", // ) // } - +// // fn execute_test_contract(address: Address, with_panic: bool) -> Execute { // let test_contract = load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json", // ); - +// // let function = test_contract.function("incrementWithRevert").unwrap(); - +// // let calldata = function // .encode_input(&[Token::Uint(U256::from(1u8)), Token::Bool(with_panic)]) // .expect("failed to encode parameters"); @@ -1653,3 +1654,4 @@ // factory_deps: None, // } // } +// ``` diff --git a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs index 2069f190598b..6545f79b5019 100644 --- a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs @@ -184,7 +184,7 @@ impl TransactionData { get_maximal_allowed_overhead(total_gas_limit, gas_per_pubdata_byte_limit, encoded_len) } - + // ``` // #[cfg(test)] // pub(crate) fn overhead_gas_with_custom_factory_deps( // &self, @@ -201,19 +201,20 @@ impl TransactionData { // ); // get_maximal_allowed_overhead(total_gas_limit, gas_per_pubdata_byte_limit, encoded_len) // } - + // // #[cfg(test)] // pub(crate) fn canonical_l1_tx_hash(&self) -> zksync_types::H256 { // use zksync_types::web3::signing::keccak256; - + // // if self.tx_type != L1_TX_TYPE { // panic!("Trying to get L1 tx hash for non-L1 tx"); // } - + // // let encoded_bytes = self.clone().abi_encode(); - + // // zksync_types::H256(keccak256(&encoded_bytes)) // } + // ``` } pub(crate) fn derive_overhead( @@ -232,8 +233,8 @@ pub(crate) fn derive_overhead( let gas_price_per_pubdata = U256::from(gas_price_per_pubdata); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -247,7 +248,7 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); // The maximal potential overhead from pubdata @@ -281,53 +282,55 @@ pub fn get_maximal_allowed_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: // 1. The overhead for occupying a single tx slot is a constant: let tx_slot_overhead = ceil_div_u256(overhead_for_block_gas, MAX_TXS_IN_BLOCK.into()); - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, BOOTLOADER_TX_ENCODING_SPACE.into(), ); - + // ``` // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower - // overhead to the operator, provides substantially easier formula to work with. + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + //Throwing off the `ceil`, while may provide marginally lower + //overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE - // ceil(OB * (TL - OE) / (EP * MP)) >= OE + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // `ceil(OB * (TL - OE) / (EP * MP)) >= OE` // - // OB * (TL - OE) / (MP * EP) > OE - 1 - // OB * (TL - OE) > (OE - 1) * EP * MP - // OB * TL + EP * MP > OE * EP * MP + OE * OB - // (OB * TL + EP * MP) / (EP * MP + OB) > OE - // OE = floor((OB * TL + EP * MP) / (EP * MP + OB)) with possible -1 if the division is without remainder + // `OB * (TL - OE) / (MP * EP) > OE - 1` + // `OB * (TL - OE) > (OE - 1) * EP * MP` + // `OB * TL + EP * MP > OE * EP * MP + OE * OB` + // `(OB * TL + EP * MP) / (EP * MP + OB) > OE` + // `OE = floor((OB * TL + EP * MP) / (EP * MP + OB))` with possible -1 if the division is without remainder let overhead_for_pubdata = { let numerator: U256 = overhead_for_block_gas * total_gas_limit + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); @@ -342,7 +345,7 @@ pub fn get_maximal_allowed_overhead( (numerator - 1) / denominator } }; - + // ``` // 4. ceil(O4 * overhead_for_block_gas) >= overhead_gas // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= OE @@ -351,6 +354,7 @@ pub fn get_maximal_allowed_overhead( // OB * (TL - OE) > OE * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT + OB) // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT + OB)), with possible -1 if the division is without remainder + // ``` let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from(MAX_TX_ERGS_LIMIT) + overhead_for_block_gas; @@ -395,7 +399,7 @@ mod tests { } else { 0u32 }; - // Safe cast: the gas_limit for a transaction can not be larger than 2^32 + // Safe cast: the gas_limit for a transaction can not be larger than `2^32` let mut right_bound = total_gas_limit; // The closure returns whether a certain overhead would be accepted by the bootloader. diff --git a/core/lib/multivm/src/versions/vm_m5/utils.rs b/core/lib/multivm/src/versions/vm_m5/utils.rs index fb0c25fdb13c..addab5c78afe 100644 --- a/core/lib/multivm/src/versions/vm_m5/utils.rs +++ b/core/lib/multivm/src/versions/vm_m5/utils.rs @@ -224,7 +224,7 @@ pub fn collect_log_queries_after_timestamp( /// Receives sorted slice of timestamps. /// Returns count of timestamps that are greater than or equal to `from_timestamp`. -/// Works in O(log(sorted_timestamps.len())). +/// Works in `O(log(sorted_timestamps.len()))`. pub fn precompile_calls_count_after_timestamp( sorted_timestamps: &[Timestamp], from_timestamp: Timestamp, diff --git a/core/lib/multivm/src/versions/vm_m5/vm.rs b/core/lib/multivm/src/versions/vm_m5/vm.rs index 67c4f126309d..037388d10612 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm.rs @@ -161,7 +161,7 @@ impl VmInterface for Vm { user_l2_to_l1_logs: l2_to_l1_logs, total_log_queries, cycles_used: self.vm.state.local_state.monotonic_cycle_counter, - // It's not applicable for vm5 + // It's not applicable for `vm5` deduplicated_events_logs: vec![], storage_refunds: vec![], } diff --git a/core/lib/multivm/src/versions/vm_m5/vm_instance.rs b/core/lib/multivm/src/versions/vm_m5/vm_instance.rs index eafac7356d3d..2a78a8179999 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_instance.rs @@ -205,7 +205,7 @@ fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(NewVmExecutionResult::Panic) } else { @@ -233,7 +233,7 @@ fn vm_may_have_ended(vm: &VmInstance, gas_before: u32) -> Option< NewVmExecutionResult::Ok(data) => { Some(VmExecutionResult { // The correct `events` value for this field should be set separately - // later on based on the information inside the event_sink oracle. + // later on based on the information inside the `event_sink` oracle. events: vec![], storage_log_queries: vm.get_final_log_queries(), used_contract_hashes: vm.get_used_contracts(), @@ -380,8 +380,8 @@ impl VmInstance { pub fn save_current_vm_as_snapshot(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), @@ -626,8 +626,8 @@ impl VmInstance { } // Err when transaction is rejected. - // Ok(status: TxExecutionStatus::Success) when the transaction succeeded - // Ok(status: TxExecutionStatus::Failure) when the transaction failed. + // `Ok(status: TxExecutionStatus::Success)` when the transaction succeeded + // `Ok(status: TxExecutionStatus::Failure)` when the transaction failed. // Note that failed transactions are considered properly processed and are included in blocks pub fn execute_next_tx(&mut self) -> Result { let tx_index = self.bootloader_state.next_unexecuted_tx() as u32; @@ -672,7 +672,7 @@ impl VmInstance { revert_reason: None, // getting contracts used during this transaction // at least for now the number returned here is always <= to the number - // of the code hashes actually used by the transaction, since it might've + // of the code hashes actually used by the transaction, since it might have // reused bytecode hashes from some of the previous ones. contracts_used: self .state diff --git a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs index 8826db6171cd..766103fee4fb 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs @@ -31,8 +31,8 @@ use crate::vm_m5::{ OracleTools, }; -// TODO (SMA-1703): move these to config and make them programmatically generatable. -// fill these values in the similar fasion as other overhead-related constants +// TODO (SMA-1703): move these to config and make them programmatically generable. +// fill these values in the similar fashion as other overhead-related constants pub const BLOCK_OVERHEAD_GAS: u32 = 1200000; pub const BLOCK_OVERHEAD_L1_GAS: u32 = 1000000; pub const BLOCK_OVERHEAD_PUBDATA: u32 = BLOCK_OVERHEAD_L1_GAS / L1_GAS_PER_PUBDATA_BYTE; @@ -78,7 +78,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible to a transaciton to + // The `baseFee` is set in such a way that it is always possible to a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, @@ -221,12 +221,12 @@ pub fn init_vm_with_gas_limit( } #[derive(Debug, Clone, Copy)] -// The block.number/block.timestamp data are stored in the CONTEXT_SYSTEM_CONTRACT. +// The `block.number` / `block.timestamp` data are stored in the `CONTEXT_SYSTEM_CONTRACT`. // The bootloader can support execution in two modes: -// - "NewBlock" when the new block is created. It is enforced that the block.number is incremented by 1 +// - `NewBlock` when the new block is created. It is enforced that the block.number is incremented by 1 // and the timestamp is non-decreasing. Also, the L2->L1 message used to verify the correctness of the previous root hash is sent. // This is the mode that should be used in the state keeper. -// - "OverrideCurrent" when we need to provide custom block.number and block.timestamp. ONLY to be used in testing/ethCalls. +// - `OverrideCurrent` when we need to provide custom `block.number` and `block.timestamp`. ONLY to be used in testing / `ethCalls`. pub enum BlockContextMode { NewBlock(DerivedBlockContext, U256), OverrideCurrent(DerivedBlockContext), diff --git a/core/lib/multivm/src/versions/vm_m6/errors/tx_revert_reason.rs b/core/lib/multivm/src/versions/vm_m6/errors/tx_revert_reason.rs index 4775d8339f79..3ddaa0684614 100644 --- a/core/lib/multivm/src/versions/vm_m6/errors/tx_revert_reason.rs +++ b/core/lib/multivm/src/versions/vm_m6/errors/tx_revert_reason.rs @@ -7,11 +7,11 @@ use super::{BootloaderErrorCode, VmRevertReason}; // Reasons why the transaction executed inside the bootloader could fail. #[derive(Debug, Clone, PartialEq)] pub enum TxRevertReason { - // Can only be returned in EthCall execution mode (=ExecuteOnly) + // Can only be returned in EthCall execution mode `(=ExecuteOnly)` EthCall(VmRevertReason), // Returned when the execution of an L2 transaction has failed TxReverted(VmRevertReason), - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` ValidationFailed(VmRevertReason), PaymasterValidationFailed(VmRevertReason), PrePaymasterPreparationFailed(VmRevertReason), @@ -20,7 +20,7 @@ pub enum TxRevertReason { FailedToChargeFee(VmRevertReason), // Emitted when trying to call a transaction from an account that has not // been deployed as an account (i.e. the `from` is just a contract). - // Can only be returned in VerifyAndExecute + // Can only be returned in `VerifyAndExecute` FromIsNotAnAccount, // Currently cannot be returned. Should be removed when refactoring errors. InnerTxError, @@ -101,7 +101,7 @@ impl TxRevertReason { BootloaderErrorCode::UnacceptablePubdataPrice => { Self::UnexpectedVMBehavior("UnacceptablePubdataPrice".to_owned()) } - // This is different from AccountTxValidationFailed error in a way that it means that + // This is different from `AccountTxValidationFailed` error in a way that it means that // the error was not produced by the account itself, but for some other unknown reason (most likely not enough gas) BootloaderErrorCode::TxValidationError => Self::ValidationFailed(revert_reason), // Note, that `InnerTxError` is derived only after the actual tx execution, so diff --git a/core/lib/multivm/src/versions/vm_m6/errors/vm_revert_reason.rs b/core/lib/multivm/src/versions/vm_m6/errors/vm_revert_reason.rs index fb2341c0b2eb..0e5bf9fd8346 100644 --- a/core/lib/multivm/src/versions/vm_m6/errors/vm_revert_reason.rs +++ b/core/lib/multivm/src/versions/vm_m6/errors/vm_revert_reason.rs @@ -73,7 +73,7 @@ impl VmRevertReason { pub fn to_user_friendly_string(&self) -> String { match self { - // In case of `Unknown` reason we suppress it to prevent verbose Error function_selector = 0x{} + // In case of `Unknown` reason we suppress it to prevent verbose `Error function_selector = 0x{}` // message shown to user. VmRevertReason::Unknown { .. } => "".to_owned(), _ => self.to_string(), diff --git a/core/lib/multivm/src/versions/vm_m6/history_recorder.rs b/core/lib/multivm/src/versions/vm_m6/history_recorder.rs index 7ec8b2fde3bf..63dc9be4933a 100644 --- a/core/lib/multivm/src/versions/vm_m6/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_m6/history_recorder.rs @@ -18,13 +18,13 @@ pub type MemoryWithHistory = HistoryRecorder; pub type IntFrameManagerWithHistory = HistoryRecorder, H>; // Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 -// can be used. This can sometimes vioalate monotonicity of the timestamp within the +// can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_m6/memory.rs b/core/lib/multivm/src/versions/vm_m6/memory.rs index 5a5042e5657f..6ad92c3a1e08 100644 --- a/core/lib/multivm/src/versions/vm_m6/memory.rs +++ b/core/lib/multivm/src/versions/vm_m6/memory.rs @@ -31,7 +31,7 @@ impl OracleWithHistory for SimpleMemory { impl SimpleMemory { pub fn populate(&mut self, elements: Vec<(u32, Vec)>, timestamp: Timestamp) { for (page, values) in elements.into_iter() { - // Resizing the pages array to fit the page. + // Re-sizing the pages array to fit the page. let len = values.len(); assert!(len <= MEMORY_CELLS_OTHER_PAGES); @@ -281,7 +281,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -298,7 +298,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -306,7 +306,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs index a1c2a97edf9f..fe59580e2ce9 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_m6::{ storage::{Storage, StoragePtr}, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/mod.rs b/core/lib/multivm/src/versions/vm_m6/oracles/mod.rs index 2b7aa3a49f7d..450ed4cf1e0c 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/mod.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/mod.rs @@ -1,9 +1,9 @@ use zk_evm_1_3_1::aux_structures::Timestamp; // All the changes to the events in the DB will be applied after the tx is executed, -// so fow now it is fine. +// so for now it is fine. pub use zk_evm_1_3_1::reference_impls::event_sink::InMemoryEventSink as EventSinkOracle; // We will discard RAM as soon as the execution of a tx ends, so -// it is ok for now to use SimpleMemory +// it is ok for now to use `SimpleMemory` pub use zk_evm_1_3_1::reference_impls::memory::SimpleMemory as RamOracle; pub use zk_evm_1_3_1::testing::simple_tracer::NoopTracer; diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/storage.rs b/core/lib/multivm/src/versions/vm_m6/oracles/storage.rs index 7ceab94bd472..4eafbacbebd3 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/storage.rs @@ -190,6 +190,7 @@ impl VmStorageOracle for StorageOracle { _monotonic_cycle_counter: u32, query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -199,6 +200,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -278,7 +280,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -292,8 +294,8 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). -// While new_value is always 32 bytes long, for key we use the following optimization: +// we basically need to publish the following pair: `()`. +// While `new_value` is always 32 bytes long, for key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. // - The second time we publish it, we will use this 8-byte instead of the 32 bytes of the entire key. diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/bootloader.rs b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/bootloader.rs index 5509cef90832..f2780c6ae80f 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/bootloader.rs @@ -100,7 +100,7 @@ impl PubdataSpentTracer for BootloaderTracer { impl BootloaderTracer { fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/call.rs b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/call.rs index 1166e7a8cdb6..19cbf13b2756 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/call.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/call.rs @@ -189,7 +189,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { @@ -256,8 +256,8 @@ impl CallTracer { // Filter all near calls from the call stack // Important that the very first call is near call - // And this NearCall includes several Normal or Mimic calls - // So we return all childrens of this NearCall + // And this `NearCall` includes several Normal or Mimic calls + // So we return all children of this `NearCall` pub fn extract_calls(&mut self) -> Vec { if let Some(current_call) = self.stack.pop() { filter_near_call(current_call) @@ -268,7 +268,7 @@ impl CallTracer { } // Filter all near calls from the call stack -// Normally wr are not interested in NearCall, because it's just a wrapper for internal calls +// Normally we are not interested in `NearCall`, because it's just a wrapper for internal calls fn filter_near_call(mut call: Call) -> Vec { let mut calls = vec![]; let original_calls = std::mem::take(&mut call.calls); diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/one_tx.rs b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/one_tx.rs index 346daba21317..53e5e4ee2f6a 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/one_tx.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/one_tx.rs @@ -19,7 +19,7 @@ use crate::vm_m6::{ }; /// Allows any opcodes, but tells the VM to end the execution once the tx is over. -// Internally depeds on Bootloader's VMHooks to get the notification once the transaction is finished. +// Internally depends on Bootloader's `VMHooks` to get the notification once the transaction is finished. #[derive(Debug)] pub struct OneTxTracer { tx_has_been_processed: bool, diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/utils.rs b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/utils.rs index d29476ea4ccc..2df22aa2d3f8 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/utils.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/utils.rs @@ -48,7 +48,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/validation.rs b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/validation.rs index e75a9f34a4ba..704a967548be 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/tracer/validation.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/tracer/validation.rs @@ -100,7 +100,7 @@ fn touches_allowed_context(address: Address, key: U256) -> bool { return false; } - // Only chain_id is allowed to be touched. + // Only `chain_id` is allowed to be touched. key == U256::from(0u32) } @@ -235,7 +235,7 @@ impl ValidationTracer { return true; } - // The pair of MSG_VALUE_SIMULATOR_ADDRESS & L2_ETH_TOKEN_ADDRESS simulates the behavior of transferring ETH + // The pair of `MSG_VALUE_SIMULATOR_ADDRESS` & `L2_ETH_TOKEN_ADDRESS` simulates the behavior of transferring ETH // that is safe for the DDoS protection rules. if valid_eth_token_call(address, msg_sender) { return true; @@ -279,20 +279,20 @@ impl ValidationTracer { let (potential_address_bytes, potential_position_bytes) = calldata.split_at(32); let potential_address = be_bytes_to_safe_address(potential_address_bytes); - // If the validation_address is equal to the potential_address, - // then it is a request that could be used for mapping of kind mapping(address => ...). + // If the `validation_address` is equal to the `potential_address`, + // then it is a request that could be used for mapping of kind `mapping(address => ...)`. // - // If the potential_position_bytes were already allowed before, then this keccak might be used - // for ERC-20 allowance or any other of mapping(address => mapping(...)) + // If the `potential_position_bytes` were already allowed before, then this keccak might be used + // for ERC-20 allowance or any other of `mapping(address => mapping(...))` if potential_address == Some(validated_address) || self .auxilary_allowed_slots .contains(&H256::from_slice(potential_position_bytes)) { - // This is request that could be used for mapping of kind mapping(address => ...) + // This is request that could be used for mapping of kind `mapping(address => ...)` // We could theoretically wait for the slot number to be returned by the - // keccak256 precompile itself, but this would complicate the code even further + // `keccak256` precompile itself, but this would complicate the code even further // so let's calculate it here. let slot = keccak256(calldata); diff --git a/core/lib/multivm/src/versions/vm_m6/refunds.rs b/core/lib/multivm/src/versions/vm_m6/refunds.rs index 4b4229b306b9..406bf380a0b2 100644 --- a/core/lib/multivm/src/versions/vm_m6/refunds.rs +++ b/core/lib/multivm/src/versions/vm_m6/refunds.rs @@ -76,7 +76,7 @@ impl VmInstance { ) -> u32 { // TODO (SMA-1715): Make users pay for the block overhead 0 - + // ``` // let pubdata_published = self.pubdata_published(from_timestamp); // // let total_gas_spent = gas_remaining_before - self.gas_remaining(); @@ -121,6 +121,7 @@ impl VmInstance { // ); // 0 // } + // ``` } // TODO (SMA-1715): Make users pay for the block overhead @@ -134,39 +135,39 @@ impl VmInstance { _l2_l1_logs: usize, ) -> u32 { 0 - + // ``` // let overhead_for_block_gas = U256::from(crate::transaction_data::block_overhead_gas( // gas_per_pubdata_byte_limit, // )); - + // // let encoded_len = U256::from(encoded_len); // let pubdata_published = U256::from(pubdata_published); // let gas_spent_on_computation = U256::from(gas_spent_on_computation); // let number_of_decommitment_requests = U256::from(number_of_decommitment_requests); // let l2_l1_logs = U256::from(l2_l1_logs); - + // // let tx_slot_overhead = ceil_div_u256(overhead_for_block_gas, MAX_TXS_IN_BLOCK.into()); - + // // let overhead_for_length = ceil_div_u256( // encoded_len * overhead_for_block_gas, // BOOTLOADER_TX_ENCODING_SPACE.into(), // ); - + // // let actual_overhead_for_pubdata = ceil_div_u256( // pubdata_published * overhead_for_block_gas, // MAX_PUBDATA_PER_BLOCK.into(), // ); - + // // let actual_gas_limit_overhead = ceil_div_u256( // gas_spent_on_computation * overhead_for_block_gas, // MAX_BLOCK_MULTIINSTANCE_GAS_LIMIT.into(), // ); - + // // let code_decommitter_sorter_circuit_overhead = ceil_div_u256( // number_of_decommitment_requests * overhead_for_block_gas, // GEOMETRY_CONFIG.limit_for_code_decommitter_sorter.into(), // ); - + // // let l1_l2_logs_overhead = ceil_div_u256( // l2_l1_logs * overhead_for_block_gas, // std::cmp::min( @@ -175,7 +176,7 @@ impl VmInstance { // ) // .into(), // ); - + // // let overhead = vec![ // tx_slot_overhead, // overhead_for_length, @@ -187,8 +188,9 @@ impl VmInstance { // .into_iter() // .max() // .unwrap(); - + // // overhead.as_u32() + // ``` } /// Returns the given transactions' gas limit - by reading it directly from the VM memory. diff --git a/core/lib/multivm/src/versions/vm_m6/tests/bootloader.rs b/core/lib/multivm/src/versions/vm_m6/tests/bootloader.rs index be840e16a142..16d2b7f47d21 100644 --- a/core/lib/multivm/src/versions/vm_m6/tests/bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/tests/bootloader.rs @@ -1,3 +1,4 @@ +// ``` // //! // //! Tests for the bootloader // //! The description for each of the tests can be found in the corresponding `.yul` file. @@ -8,7 +9,7 @@ // convert::TryFrom, // }; // use tempfile::TempDir; - +// // use crate::{ // errors::VmRevertReason, // history_recorder::HistoryMode, @@ -35,11 +36,11 @@ // }, // HistoryEnabled, OracleTools, TxRevertReason, VmBlockResult, VmExecutionResult, VmInstance, // }; - +// // use zk_evm_1_3_1::{ // aux_structures::Timestamp, block_properties::BlockProperties, zkevm_opcode_defs::FarCallOpcode, // }; - +// // use zksync_types::{ // block::DeployedContract, // ethabi::encode, @@ -60,21 +61,21 @@ // L2_ETH_TOKEN_ADDRESS, MAX_GAS_PER_PUBDATA_BYTE, SYSTEM_CONTEXT_ADDRESS, // }, // }; - +// // use zksync_utils::{ // bytecode::CompressedBytecodeInfo, // test_utils::LoadnextContractExecutionParams, // {bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, u256_to_h256}, // }; - +// // use zksync_contracts::{ // get_loadnext_contract, load_contract, read_bytecode, SystemContractCode, // PLAYGROUND_BLOCK_BOOTLOADER_CODE, // }; - +// // use zksync_state::{secondary_storage::SecondaryStateStorage, storage_view::StorageView}; // use zksync_storage::{db::Database, RocksDB}; - +// // fn run_vm_with_custom_factory_deps<'a, H: HistoryMode>( // oracle_tools: &'a mut OracleTools<'a, false, H>, // block_context: BlockContext, @@ -93,7 +94,7 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // vm.bootloader_state.add_tx_data(encoded_tx.len()); // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, @@ -110,17 +111,17 @@ // ), // Timestamp(0), // ); - +// // let result = vm.execute_next_tx(u32::MAX, false).err(); - +// // assert_eq!(expected_error, result); // } - +// // fn get_balance(token_id: AccountTreeId, account: &Address, main_storage: StoragePtr<'_>) -> U256 { // let key = storage_key_for_standard_token_balance(token_id, account); // h256_to_u256(main_storage.borrow_mut().get_value(&key)) // } - +// // #[test] // fn test_dummy_bootloader() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -129,18 +130,18 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); // let (block_context, block_properties) = create_test_block_params(); // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -149,22 +150,22 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::BlockPostprocessing); - +// // // Dummy bootloader should not panic // assert!(res.revert_reason.is_none()); - +// // let correct_first_cell = U256::from_str_radix("123123123", 16).unwrap(); - +// // verify_required_memory( // &vm.state, // vec![(correct_first_cell, BOOTLOADER_HEAP_PAGE, 0)], // ); // } - +// // #[test] // fn test_bootloader_out_of_gas() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -173,20 +174,20 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); // let (block_context, block_properties) = create_test_block_params(); - +// // let mut base_system_contracts = BASE_SYSTEM_CONTRACTS.clone(); - +// // let bootloader_code = read_bootloader_test_code("dummy"); // let bootloader_hash = hash_bytecode(&bootloader_code); - +// // base_system_contracts.bootloader = SystemContractCode { // code: bytes_to_be_words(bootloader_code), // hash: bootloader_hash, // }; - +// // // init vm with only 10 ergs // let mut vm = init_vm_inner( // &mut oracle_tools, @@ -196,19 +197,19 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let res = vm.execute_block_tip(); - +// // assert_eq!(res.revert_reason, Some(TxRevertReason::BootloaderOutOfGas)); // } - +// // fn verify_required_storage( // state: &ZkSyncVmState<'_, H>, // required_values: Vec<(H256, StorageKey)>, // ) { // for (required_value, key) in required_values { // let current_value = state.storage.storage.read_from_storage(&key); - +// // assert_eq!( // u256_to_h256(current_value), // required_value, @@ -216,7 +217,7 @@ // ); // } // } - +// // fn verify_required_memory( // state: &ZkSyncVmState<'_, H>, // required_values: Vec<(U256, u32, u32)>, @@ -229,21 +230,21 @@ // assert_eq!(current_value, required_value); // } // } - +// // #[test] // fn test_default_aa_interaction() { // // In this test, we aim to test whether a simple account interaction (without any fee logic) // // will work. The account will try to deploy a simple contract from integration tests. - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let operator_address = block_context.context.operator_address; // let base_fee = block_context.base_fee; // // We deploy here counter contract, because its logic is trivial @@ -264,16 +265,16 @@ // ) // .into(); // let tx_data: TransactionData = tx.clone().into(); - +// // let maximal_fee = tx_data.gas_limit * tx_data.max_fee_per_gas; // let sender_address = tx_data.from(); // // set balance - +// // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -283,17 +284,17 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let tx_execution_result = vm // .execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // let VmBlockResult { // full_result: res, .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); @@ -303,28 +304,28 @@ // "Bootloader was not expected to revert: {:?}", // res.revert_reason // ); - +// // // Both deployment and ordinary nonce should be incremented by one. // let account_nonce_key = get_nonce_key(&sender_address); // let expected_nonce = TX_NONCE_INCREMENT + DEPLOYMENT_NONCE_INCREMENT; - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(expected_nonce), account_nonce_key), // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert!(!tx_has_failed(&vm.state, 0)); - +// // let expected_fee = // maximal_fee - U256::from(tx_execution_result.gas_refunded) * U256::from(base_fee); // let operator_balance = get_balance( @@ -332,13 +333,13 @@ // &operator_address, // vm.state.storage.storage.get_ptr(), // ); - +// // assert!( // operator_balance == expected_fee, // "Operator did not receive his fee" // ); // } - +// // fn execute_vm_with_predetermined_refund( // txs: Vec, // refunds: Vec, @@ -346,22 +347,22 @@ // ) -> VmBlockResult { // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // set balance // for tx in txs.iter() { // let sender_address = tx.initiator_account(); // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); // } - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -370,7 +371,7 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // let codes_for_decommiter = txs // .iter() // .flat_map(|tx| { @@ -383,12 +384,12 @@ // .collect::)>>() // }) // .collect(); - +// // vm.state.decommittment_processor.populate( // codes_for_decommiter, // Timestamp(vm.state.local_state.timestamp), // ); - +// // let memory_with_suggested_refund = get_bootloader_memory( // txs.into_iter().map(Into::into).collect(), // refunds, @@ -396,32 +397,32 @@ // TxExecutionMode::VerifyExecute, // BlockContextMode::NewBlock(block_context, Default::default()), // ); - +// // vm.state.memory.populate_page( // BOOTLOADER_HEAP_PAGE as usize, // memory_with_suggested_refund, // Timestamp(0), // ); - +// // vm.execute_till_block_end(BootloaderJobType::TransactionExecution) // } - +// // #[test] // fn test_predetermined_refunded_gas() { // // In this test, we compare the execution of the bootloader with the predefined // // refunded gas and without them - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let base_fee = block_context.base_fee; - +// // // We deploy here counter contract, because its logic is trivial // let contract_code = read_test_contract(); // let published_bytecode = CompressedBytecodeInfo::from_original(contract_code.clone()).unwrap(); @@ -439,15 +440,15 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); - +// // // set balance // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -456,19 +457,19 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let tx_execution_result = vm // .execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing transaction"); - +// // assert_eq!( // tx_execution_result.status, // TxExecutionStatus::Success, // "Transaction wasn't successful" // ); - +// // // If the refund provided by the operator or the final refund are the 0 // // there is no impact of the operator's refund at all and so this test does not // // make much sense. @@ -480,14 +481,14 @@ // tx_execution_result.gas_refunded > 0, // "The final refund is 0" // ); - +// // let mut result = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); // assert!( // result.full_result.revert_reason.is_none(), // "Bootloader was not expected to revert: {:?}", // result.full_result.revert_reason // ); - +// // let mut result_with_predetermined_refund = execute_vm_with_predetermined_refund( // vec![tx], // vec![tx_execution_result.operator_suggested_refund], @@ -499,7 +500,7 @@ // .full_result // .used_contract_hashes // .sort(); - +// // assert_eq!( // result.full_result.events, // result_with_predetermined_refund.full_result.events @@ -521,18 +522,18 @@ // .used_contract_hashes // ); // } - +// // #[derive(Debug, Clone)] // enum TransactionRollbackTestInfo { // Rejected(Transaction, TxRevertReason), // Processed(Transaction, bool, TxExecutionStatus), // } - +// // impl TransactionRollbackTestInfo { // fn new_rejected(transaction: Transaction, revert_reason: TxRevertReason) -> Self { // Self::Rejected(transaction, revert_reason) // } - +// // fn new_processed( // transaction: Transaction, // should_be_rollbacked: bool, @@ -540,28 +541,28 @@ // ) -> Self { // Self::Processed(transaction, should_be_rollbacked, expected_status) // } - +// // fn get_transaction(&self) -> &Transaction { // match self { // TransactionRollbackTestInfo::Rejected(tx, _) => tx, // TransactionRollbackTestInfo::Processed(tx, _, _) => tx, // } // } - +// // fn rejection_reason(&self) -> Option { // match self { // TransactionRollbackTestInfo::Rejected(_, revert_reason) => Some(revert_reason.clone()), // TransactionRollbackTestInfo::Processed(_, _, _) => None, // } // } - +// // fn should_rollback(&self) -> bool { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => true, // TransactionRollbackTestInfo::Processed(_, x, _) => *x, // } // } - +// // fn expected_status(&self) -> TxExecutionStatus { // match self { // TransactionRollbackTestInfo::Rejected(_, _) => { @@ -571,7 +572,7 @@ // } // } // } - +// // // Accepts the address of the sender as well as the list of pairs of its transactions // // and whether these transactions should succeed. // fn execute_vm_with_possible_rollbacks( @@ -585,13 +586,13 @@ // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // Setting infinite balance for the sender. // let key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -600,7 +601,7 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // for test_info in transactions { // vm.save_current_vm_as_snapshot(); // let vm_state_before_tx = vm.dump_inner_state(); @@ -610,7 +611,7 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // match vm.execute_next_tx(u32::MAX, false) { // Err(reason) => { // assert_eq!(test_info.rejection_reason(), Some(reason)); @@ -624,11 +625,11 @@ // ); // } // }; - +// // if test_info.should_rollback() { // // Some error has occurred, we should reject the transaction // vm.rollback_to_latest_snapshot(); - +// // // vm_state_before_tx. // let state_after_rollback = vm.dump_inner_state(); // assert_eq!( @@ -637,7 +638,7 @@ // ); // } // } - +// // let VmBlockResult { // full_result: mut result, // .. @@ -645,10 +646,10 @@ // // Used contract hashes are retrieved in unordered manner. // // However it must be sorted for the comparisons in tests to work // result.used_contract_hashes.sort(); - +// // result // } - +// // // Sets the signature for an L2 transaction and returns the same transaction // // but this different signature. // fn change_signature(mut tx: Transaction, signature: Vec) -> Transaction { @@ -659,22 +660,22 @@ // } // _ => unreachable!(), // }; - +// // tx // } - +// // #[test] // fn test_vm_rollbacks() { // let (block_context, block_properties): (DerivedBlockContext, BlockProperties) = { // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let base_fee = U256::from(block_context.base_fee); - +// // let sender_private_key = H256::random(); // let contract_code = read_test_contract(); - +// // let tx_nonce_0: Transaction = get_deploy_tx( // sender_private_key, // Nonce(0), @@ -717,13 +718,13 @@ // }, // ) // .into(); - +// // let wrong_signature_length_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 32]); // let wrong_v_tx = change_signature(tx_nonce_0.clone(), vec![1u8; 65]); // let wrong_signature_tx = change_signature(tx_nonce_0.clone(), vec![27u8; 65]); - +// // let sender_address = tx_nonce_0.initiator_account(); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -747,7 +748,7 @@ // block_context, // block_properties, // ); - +// // let incorrect_nonce = TxRevertReason::ValidationFailed(VmRevertReason::General { // msg: "Incorrect nonce".to_string(), // data: vec![ @@ -790,7 +791,7 @@ // msg: "Account validation returned invalid magic value. Most often this means that the signature is incorrect".to_string(), // data: vec![], // }); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -835,11 +836,11 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); - +// // let loadnext_contract = get_loadnext_contract(); - +// // let loadnext_constructor_data = encode(&[Token::Uint(U256::from(100))]); // let loadnext_deploy_tx: Transaction = get_deploy_tx( // sender_private_key, @@ -862,7 +863,7 @@ // false, // TxExecutionStatus::Success, // ); - +// // let get_load_next_tx = |params: LoadnextContractExecutionParams, nonce: Nonce| { // // Here we test loadnext with various kinds of operations // let tx: Transaction = mock_loadnext_test_call( @@ -878,10 +879,10 @@ // params, // ) // .into(); - +// // tx // }; - +// // let loadnext_tx_0 = get_load_next_tx( // LoadnextContractExecutionParams { // reads: 100, @@ -904,7 +905,7 @@ // }, // Nonce(2), // ); - +// // let result_without_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -923,7 +924,7 @@ // block_context, // block_properties, // ); - +// // let result_with_rollbacks = execute_vm_with_possible_rollbacks( // sender_address, // vec![ @@ -964,10 +965,10 @@ // block_context, // block_properties, // ); - +// // assert_eq!(result_without_rollbacks, result_with_rollbacks); // } - +// // // Inserts the contracts into the test environment, bypassing the // // deployer system contract. Besides the reference to storage // // it accepts a `contracts` tuple of information about the contract @@ -980,13 +981,13 @@ // .iter() // .flat_map(|(contract, is_account)| { // let mut new_logs = vec![]; - +// // let deployer_code_key = get_code_key(contract.account_id.address()); // new_logs.push(StorageLog::new_write_log( // deployer_code_key, // hash_bytecode(&contract.bytecode), // )); - +// // if *is_account { // let is_account_key = get_is_account_key(contract.account_id.address()); // new_logs.push(StorageLog::new_write_log( @@ -994,18 +995,18 @@ // u256_to_h256(1u32.into()), // )); // } - +// // new_logs // }) // .collect(); // raw_storage.process_transaction_logs(&logs); - +// // for (contract, _) in contracts { // raw_storage.store_factory_dep(hash_bytecode(&contract.bytecode), contract.bytecode); // } // raw_storage.save(L1BatchNumber(0)); // } - +// // enum NonceHolderTestMode { // SetValueUnderNonce, // IncreaseMinNonceBy5, @@ -1014,7 +1015,7 @@ // IncreaseMinNonceBy1, // SwitchToArbitraryOrdering, // } - +// // impl From for u8 { // fn from(mode: NonceHolderTestMode) -> u8 { // match mode { @@ -1027,7 +1028,7 @@ // } // } // } - +// // fn get_nonce_holder_test_tx( // nonce: U256, // account_address: Address, @@ -1049,11 +1050,11 @@ // reserved: [U256::zero(); 4], // data: vec![12], // signature: vec![test_mode.into()], - +// // ..Default::default() // } // } - +// // fn run_vm_with_raw_tx<'a, H: HistoryMode>( // oracle_tools: &'a mut OracleTools<'a, false, H>, // block_context: DerivedBlockContext, @@ -1070,9 +1071,9 @@ // &base_system_contracts, // TxExecutionMode::VerifyExecute, // ); - +// // let block_gas_price_per_pubdata = block_context.context.block_gas_price_per_pubdata(); - +// // let overhead = tx.overhead_gas(block_gas_price_per_pubdata as u32); // push_raw_transaction_to_bootloader_memory( // &mut vm, @@ -1085,43 +1086,43 @@ // full_result: result, // .. // } = vm.execute_till_block_end(BootloaderJobType::TransactionExecution); - +// // (result, tx_has_failed(&vm.state, 0)) // } - +// // #[test] // fn test_nonce_holder() { // let (block_context, block_properties): (DerivedBlockContext, BlockProperties) = { // let (block_context, block_properties) = create_test_block_params(); // (block_context.into(), block_properties) // }; - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); - +// // let account_address = H160::random(); // let account = DeployedContract { // account_id: AccountTreeId::new(account_address), // bytecode: read_nonce_holder_tester(), // }; - +// // insert_contracts(&mut raw_storage, vec![(account, true)]); - +// // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // // We deploy here counter contract, because its logic is trivial - +// // let key = storage_key_for_eth_balance(&account_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut run_nonce_test = |nonce: U256, // test_mode: NonceHolderTestMode, // error_message: Option, // comment: &'static str| { // let tx = get_nonce_holder_test_tx(nonce, account_address, test_mode, &block_context); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); // let (result, tx_has_failed) = // run_vm_with_raw_tx(&mut oracle_tools, block_context, &block_properties, tx); @@ -1142,7 +1143,7 @@ // assert!(!tx_has_failed, "{}", comment); // } // }; - +// // // Test 1: trying to set value under non sequential nonce value. // run_nonce_test( // 1u32.into(), @@ -1150,7 +1151,7 @@ // Some("Previous nonce has not been used".to_string()), // "Allowed to set value under non sequential value", // ); - +// // // Test 2: increase min nonce by 1 with sequential nonce ordering: // run_nonce_test( // 0u32.into(), @@ -1158,7 +1159,7 @@ // None, // "Failed to increment nonce by 1 for sequential account", // ); - +// // // Test 3: correctly set value under nonce with sequential nonce ordering: // run_nonce_test( // 1u32.into(), @@ -1166,7 +1167,7 @@ // None, // "Failed to set value under nonce sequential value", // ); - +// // // Test 5: migrate to the arbitrary nonce ordering: // run_nonce_test( // 2u32.into(), @@ -1174,7 +1175,7 @@ // None, // "Failed to switch to arbitrary ordering", // ); - +// // // Test 6: increase min nonce by 5 // run_nonce_test( // 6u32.into(), @@ -1182,7 +1183,7 @@ // None, // "Failed to increase min nonce by 5", // ); - +// // // Test 7: since the nonces in range [6,10] are no longer allowed, the // // tx with nonce 10 should not be allowed // run_nonce_test( @@ -1191,7 +1192,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse nonce below the minimal one", // ); - +// // // Test 8: we should be able to use nonce 13 // run_nonce_test( // 13u32.into(), @@ -1199,7 +1200,7 @@ // None, // "Did not allow to use unused nonce 10", // ); - +// // // Test 9: we should not be able to reuse nonce 13 // run_nonce_test( // 13u32.into(), @@ -1207,7 +1208,7 @@ // Some("Reusing the same nonce twice".to_string()), // "Allowed to reuse the same nonce twice", // ); - +// // // Test 10: we should be able to simply use nonce 14, while bumping the minimal nonce by 5 // run_nonce_test( // 14u32.into(), @@ -1215,7 +1216,7 @@ // None, // "Did not allow to use a bumped nonce", // ); - +// // // Test 6: Do not allow bumping nonce by too much // run_nonce_test( // 16u32.into(), @@ -1223,7 +1224,7 @@ // Some("The value for incrementing the nonce is too high".to_string()), // "Allowed for incrementing min nonce too much", // ); - +// // // Test 7: Do not allow not setting a nonce as used // run_nonce_test( // 16u32.into(), @@ -1232,7 +1233,7 @@ // "Allowed to leave nonce as unused", // ); // } - +// // #[test] // fn test_l1_tx_execution() { // // In this test, we try to execute a contract deployment from L1 @@ -1242,17 +1243,17 @@ // insert_system_contracts(&mut raw_storage); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); // let (block_context, block_properties) = create_test_block_params(); - +// // // Here instead of marking code hash via the bootloader means, we will // // using L1->L2 communication, the same it would likely be done during the priority mode. // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_tx = get_l1_deploy_tx(&contract_code, &[]); // let l1_deploy_tx_data: TransactionData = l1_deploy_tx.clone().into(); - +// // let required_l2_to_l1_logs = vec![ // L2ToL1Log { // shard_id: 0, @@ -1271,9 +1272,9 @@ // value: u256_to_h256(U256::from(1u32)), // }, // ]; - +// // let sender_address = l1_deploy_tx_data.from(); - +// // oracle_tools.decommittment_processor.populate( // vec![( // h256_to_u256(contract_code_hash), @@ -1281,7 +1282,7 @@ // )], // Timestamp(0), // ); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context.into(), Default::default()), @@ -1296,29 +1297,29 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // let res = vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // // The code hash of the deployed contract should be marked as republished. // let known_codes_key = get_known_code_key(&contract_code_hash); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address, U256::zero()); // let account_code_key = get_code_key(&deployed_address); - +// // let expected_slots = vec![ // (u256_to_h256(U256::from(1u32)), known_codes_key), // (contract_code_hash, account_code_key), // ]; // assert!(!tx_has_failed(&vm.state, 0)); - +// // verify_required_storage(&vm.state, expected_slots); - +// // assert_eq!(res.result.logs.l2_to_l1_logs, required_l2_to_l1_logs); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, true); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = StorageWritesDeduplicator::apply_on_empty_state( // &vm.execute_next_tx(u32::MAX, false) // .unwrap() @@ -1327,7 +1328,7 @@ // .storage_logs, // ); // assert_eq!(res.initial_storage_writes, 0); - +// // let tx = get_l1_execute_test_contract_tx(deployed_address, false); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); // let res = StorageWritesDeduplicator::apply_on_empty_state( @@ -1338,9 +1339,9 @@ // .storage_logs, // ); // assert_eq!(res.initial_storage_writes, 2); - +// // let repeated_writes = res.repeated_storage_writes; - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); // let res = StorageWritesDeduplicator::apply_on_empty_state( // &vm.execute_next_tx(u32::MAX, false) @@ -1352,7 +1353,7 @@ // assert_eq!(res.initial_storage_writes, 1); // // We do the same storage write, so it will be deduplicated // assert_eq!(res.repeated_storage_writes, repeated_writes); - +// // let mut tx = get_l1_execute_test_contract_tx(deployed_address, false); // tx.execute.value = U256::from(1); // match &mut tx.common_data { @@ -1369,16 +1370,16 @@ // TxExecutionStatus::Failure, // "The transaction should fail" // ); - +// // let res = // StorageWritesDeduplicator::apply_on_empty_state(&execution_result.result.logs.storage_logs); - +// // // There are 2 initial writes here: // // - totalSupply of ETH token // // - balance of the refund recipient // assert_eq!(res.initial_storage_writes, 2); // } - +// // #[test] // fn test_invalid_bytecode() { // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); @@ -1387,18 +1388,18 @@ // insert_system_contracts(&mut raw_storage); // let (block_context, block_properties) = create_test_block_params(); // let block_gas_per_pubdata = block_context.block_gas_price_per_pubdata(); - +// // let test_vm_with_custom_bytecode_hash = // |bytecode_hash: H256, expected_revert_reason: Option| { // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let (encoded_tx, predefined_overhead) = get_l1_tx_with_custom_bytecode_hash( // h256_to_u256(bytecode_hash), // block_gas_per_pubdata as u32, // ); - +// // run_vm_with_custom_factory_deps( // &mut oracle_tools, // block_context, @@ -1408,14 +1409,14 @@ // expected_revert_reason, // ); // }; - +// // let failed_to_mark_factory_deps = |msg: &str, data: Vec| { // TxRevertReason::FailedToMarkFactoryDependencies(VmRevertReason::General { // msg: msg.to_string(), // data, // }) // }; - +// // // Here we provide the correctly-formatted bytecode hash of // // odd length, so it should work. // test_vm_with_custom_bytecode_hash( @@ -1425,7 +1426,7 @@ // ]), // None, // ); - +// // // Here we provide correctly formatted bytecode of even length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1444,7 +1445,7 @@ // ], // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1464,7 +1465,7 @@ // ], // )), // ); - +// // // Here we provide incorrectly formatted bytecode of odd length, so // // it should fail. // test_vm_with_custom_bytecode_hash( @@ -1485,25 +1486,25 @@ // )), // ); // } - +// // #[test] // fn test_tracing_of_execution_errors() { // // In this test, we are checking that the execution errors are transmitted correctly from the bootloader. // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let private_key = H256::random(); - +// // let contract_address = Address::random(); // let error_contract = DeployedContract { // account_id: AccountTreeId::new(contract_address), // bytecode: read_error_contract(), // }; - +// // let tx = get_error_tx( // private_key, // Nonce(0), @@ -1515,16 +1516,16 @@ // gas_per_pubdata_limit: U256::from(MAX_GAS_PER_PUBDATA_BYTE), // }, // ); - +// // insert_contracts(&mut raw_storage, vec![(error_contract, false)]); - +// // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let key = storage_key_for_eth_balance(&tx.common_data.initiator_address); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1539,14 +1540,14 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // let mut tracer = TransactionResultTracer::new(usize::MAX, false); // assert_eq!( // vm.execute_with_custom_tracer(&mut tracer), // VmExecutionStopReason::VmFinished, // "Tracer should never request stop" // ); - +// // match tracer.revert_reason { // Some(revert_reason) => { // let revert_reason = VmRevertReason::try_from(&revert_reason as &[u8]).unwrap(); @@ -1570,7 +1571,7 @@ // ), // } // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1596,7 +1597,7 @@ // TxExecutionMode::VerifyExecute, // None, // ); - +// // let mut tracer = TransactionResultTracer::new(10, false); // assert_eq!( // vm.execute_with_custom_tracer(&mut tracer), @@ -1604,20 +1605,20 @@ // ); // assert!(tracer.is_limit_reached()); // } - +// // /// Checks that `TX_GAS_LIMIT_OFFSET` constant is correct. // #[test] // fn test_tx_gas_limit_offset() { // let gas_limit = U256::from(999999); - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let raw_storage = SecondaryStateStorage::new(db); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let contract_code = read_test_contract(); // let tx: Transaction = get_deploy_tx( // H256::random(), @@ -1631,9 +1632,9 @@ // }, // ) // .into(); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1643,7 +1644,7 @@ // TxExecutionMode::VerifyExecute, // ); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let gas_limit_from_memory = vm // .state // .memory @@ -1654,20 +1655,20 @@ // .value; // assert_eq!(gas_limit_from_memory, gas_limit); // } - +// // #[test] // fn test_is_write_initial_behaviour() { // // In this test, we check result of `is_write_initial` at different stages. - +// // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); - +// // let base_fee = block_context.base_fee; // let account_pk = H256::random(); // let contract_code = read_test_contract(); @@ -1685,19 +1686,19 @@ // }, // ) // .into(); - +// // let sender_address = tx.initiator_account(); // let nonce_key = get_nonce_key(&sender_address); - +// // // Check that the next write to the nonce key will be initial. // assert!(storage_ptr.is_write_initial(&nonce_key)); - +// // // Set balance to be able to pay fee for txs. // let balance_key = storage_key_for_eth_balance(&sender_address); // storage_ptr.set_value(&balance_key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // let mut vm = init_vm_inner( // &mut oracle_tools, // BlockContextMode::NewBlock(block_context, Default::default()), @@ -1706,15 +1707,15 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // vm.execute_next_tx(u32::MAX, false) // .expect("Bootloader failed while processing the first transaction"); // // Check that `is_write_initial` still returns true for the nonce key. // assert!(storage_ptr.is_write_initial(&nonce_key)); // } - +// // pub fn get_l1_tx_with_custom_bytecode_hash( // bytecode_hash: U256, // block_gas_per_pubdata: u32, @@ -1723,12 +1724,12 @@ // let predefined_overhead = // tx.overhead_gas_with_custom_factory_deps(vec![bytecode_hash], block_gas_per_pubdata); // let tx_bytes = tx.abi_encode_with_custom_factory_deps(vec![bytecode_hash]); - +// // (bytes_to_be_words(tx_bytes), predefined_overhead) // } - +// // const L1_TEST_GAS_PER_PUBDATA_BYTE: u32 = 800; - +// // pub fn get_l1_execute_test_contract_tx(deployed_address: Address, with_panic: bool) -> Transaction { // let sender = H160::random(); // get_l1_execute_test_contract_tx_with_sender( @@ -1739,18 +1740,18 @@ // false, // ) // } - +// // pub fn get_l1_tx_with_large_output(sender: Address, deployed_address: Address) -> Transaction { // let test_contract = load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/long-return-data/long-return-data.sol/LongReturnData.json", // ); - +// // let function = test_contract.function("longReturnData").unwrap(); - +// // let calldata = function // .encode_input(&[]) // .expect("failed to encode parameters"); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender, @@ -1767,7 +1768,7 @@ // received_timestamp_ms: 0, // } // } - +// // pub fn get_l1_execute_test_contract_tx_with_sender( // sender: Address, // deployed_address: Address, @@ -1776,7 +1777,7 @@ // payable: bool, // ) -> Transaction { // let execute = execute_test_contract(deployed_address, with_panic, value, payable); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender, @@ -1789,10 +1790,10 @@ // received_timestamp_ms: 0, // } // } - +// // pub fn get_l1_deploy_tx(code: &[u8], calldata: &[u8]) -> Transaction { // let execute = get_create_execute(code, calldata); - +// // Transaction { // common_data: ExecuteTransactionCommon::L1(L1TxCommonData { // sender: H160::random(), @@ -1804,25 +1805,25 @@ // received_timestamp_ms: 0, // } // } - +// // fn read_test_contract() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json") // } - +// // fn read_long_return_data_contract() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/long-return-data/long-return-data.sol/LongReturnData.json") // } - +// // fn read_nonce_holder_tester() -> Vec { // read_bytecode("etc/contracts-test-data/artifacts-zk/contracts/custom-account/nonce-holder-test.sol/NonceHolderTest.json") // } - +// // fn read_error_contract() -> Vec { // read_bytecode( // "etc/contracts-test-data/artifacts-zk/contracts/error/error.sol/SimpleRequire.json", // ) // } - +// // fn execute_test_contract( // address: Address, // with_panic: bool, @@ -1832,7 +1833,7 @@ // let test_contract = load_contract( // "etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json", // ); - +// // let function = if payable { // test_contract // .function("incrementWithRevertPayable") @@ -1840,11 +1841,11 @@ // } else { // test_contract.function("incrementWithRevert").unwrap() // }; - +// // let calldata = function // .encode_input(&[Token::Uint(U256::from(1u8)), Token::Bool(with_panic)]) // .expect("failed to encode parameters"); - +// // Execute { // contract_address: address, // calldata, @@ -1852,7 +1853,7 @@ // factory_deps: None, // } // } - +// // #[test] // fn test_call_tracer() { // let sender = H160::random(); @@ -1860,21 +1861,21 @@ // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); - +// // let (block_context, block_properties) = create_test_block_params(); - +// // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_tx = get_l1_deploy_tx(&contract_code, &[]); // let l1_deploy_tx_data: TransactionData = l1_deploy_tx.clone().into(); - +// // let sender_address_counter = l1_deploy_tx_data.from(); // let mut storage_accessor = StorageView::new(&raw_storage); // let storage_ptr: &mut dyn Storage = &mut storage_accessor; - +// // let key = storage_key_for_eth_balance(&sender_address_counter); // storage_ptr.set_value(&key, u256_to_h256(U256([0, 0, 1, 0]))); - +// // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); // oracle_tools.decommittment_processor.populate( // vec![( @@ -1883,7 +1884,7 @@ // )], // Timestamp(0), // ); - +// // let contract_code = read_long_return_data_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let l1_deploy_long_return_data_tx = get_l1_deploy_tx(&contract_code, &[]); @@ -1894,7 +1895,7 @@ // )], // Timestamp(0), // ); - +// // let tx_data: TransactionData = l1_deploy_long_return_data_tx.clone().into(); // let sender_long_return_address = tx_data.from(); // // The contract should be deployed successfully. @@ -1908,14 +1909,14 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &l1_deploy_tx, // TxExecutionMode::VerifyExecute, // None, // ); - +// // // The contract should be deployed successfully. // let deployed_address = deployed_address_create(sender_address_counter, U256::zero()); // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); @@ -1956,16 +1957,16 @@ // calls: vec![], // }; // assert_eq!(create_call.unwrap(), expected); - +// // push_transaction_to_bootloader_memory( // &mut vm, // &l1_deploy_long_return_data_tx, // TxExecutionMode::VerifyExecute, // None, // ); - +// // vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // let tx = get_l1_execute_test_contract_tx_with_sender( // sender, // deployed_address, @@ -1973,13 +1974,13 @@ // U256::from(1u8), // true, // ); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; - +// // // We don't want to compare gas used, because it's not fully deterministic. // let expected = Call { // r#type: CallType::Call(FarCallOpcode::Mimic), @@ -1998,7 +1999,7 @@ // revert_reason: None, // calls: vec![], // }; - +// // // First loop filter out the bootloaders calls and // // the second loop filters out the calls msg value simulator calls // for call in calls { @@ -2010,7 +2011,7 @@ // } // } // } - +// // let tx = get_l1_execute_test_contract_tx_with_sender( // sender, // deployed_address, @@ -2018,13 +2019,13 @@ // U256::from(1u8), // true, // ); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; - +// // let expected = Call { // r#type: CallType::Call(FarCallOpcode::Mimic), // to: deployed_address, @@ -2039,7 +2040,7 @@ // revert_reason: Some("This method always reverts".to_string()), // calls: vec![], // }; - +// // for call in calls { // if let CallType::Call(FarCallOpcode::Mimic) = call.r#type { // for call in call.calls { @@ -2049,12 +2050,12 @@ // } // } // } - +// // let tx = get_l1_tx_with_large_output(sender, deployed_address_long_return_data); - +// // let tx_data: TransactionData = tx.clone().into(); // push_transaction_to_bootloader_memory(&mut vm, &tx, TxExecutionMode::VerifyExecute, None); - +// // assert_ne!(deployed_address_long_return_data, deployed_address); // let res = vm.execute_next_tx(u32::MAX, true).unwrap(); // let calls = res.call_traces; @@ -2072,23 +2073,23 @@ // } // } // } - +// // #[test] // fn test_get_used_contracts() { // // get block context // let (block_context, block_properties) = create_test_block_params(); // let block_context: DerivedBlockContext = block_context.into(); - +// // // insert system contracts to avoid vm errors during initialization // let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); // let db = RocksDB::new(Database::StateKeeper, temp_dir.as_ref(), false); // let mut raw_storage = SecondaryStateStorage::new(db); // insert_system_contracts(&mut raw_storage); - +// // // get oracle tools // let storage_ptr: &mut dyn Storage = &mut StorageView::new(&raw_storage); // let mut oracle_tools = OracleTools::new(storage_ptr, HistoryEnabled); - +// // // init vm // let mut vm = init_vm_inner( // &mut oracle_tools, @@ -2098,23 +2099,23 @@ // &BASE_SYSTEM_CONTRACTS, // TxExecutionMode::VerifyExecute, // ); - +// // assert!(known_bytecodes_without_aa_code(&vm).is_empty()); - +// // // create and push and execute some not-empty factory deps transaction with success status // // to check that get_used_contracts() updates // let contract_code = read_test_contract(); // let contract_code_hash = hash_bytecode(&contract_code); // let tx1 = get_l1_deploy_tx(&contract_code, &[]); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx1, TxExecutionMode::VerifyExecute, None); - +// // let res1 = vm.execute_next_tx(u32::MAX, true).unwrap(); // assert_eq!(res1.status, TxExecutionStatus::Success); // assert!(vm // .get_used_contracts() // .contains(&h256_to_u256(contract_code_hash))); - +// // assert_eq!( // vm.get_used_contracts() // .into_iter() @@ -2124,13 +2125,13 @@ // .cloned() // .collect::>() // ); - +// // // create push and execute some non-empty factory deps transaction that fails // // (known_bytecodes will be updated but we expect get_used_contracts() to not be updated) - +// // let mut tx2 = tx1; // tx2.execute.contract_address = L1_MESSENGER_ADDRESS; - +// // let calldata = vec![1, 2, 3]; // let big_calldata: Vec = calldata // .iter() @@ -2138,16 +2139,16 @@ // .take(calldata.len() * 1024) // .cloned() // .collect(); - +// // tx2.execute.calldata = big_calldata; // tx2.execute.factory_deps = Some(vec![vec![1; 32]]); - +// // push_transaction_to_bootloader_memory(&mut vm, &tx2, TxExecutionMode::VerifyExecute, None); - +// // let res2 = vm.execute_next_tx(u32::MAX, false).unwrap(); - +// // assert_eq!(res2.status, TxExecutionStatus::Failure); - +// // for factory_dep in tx2.execute.factory_deps.unwrap() { // let hash = hash_bytecode(&factory_dep); // let hash_to_u256 = h256_to_u256(hash); @@ -2157,7 +2158,7 @@ // assert!(!vm.get_used_contracts().contains(&hash_to_u256)); // } // } - +// // fn known_bytecodes_without_aa_code(vm: &VmInstance) -> HashMap> { // let mut known_bytecodes_without_aa_code = vm // .state @@ -2165,10 +2166,11 @@ // .known_bytecodes // .inner() // .clone(); - +// // known_bytecodes_without_aa_code // .remove(&h256_to_u256(BASE_SYSTEM_CONTRACTS.default_aa.hash)) // .unwrap(); - +// // known_bytecodes_without_aa_code // } +// ``` diff --git a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs index 3536159f658e..19eefbb1a001 100644 --- a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs @@ -238,7 +238,7 @@ pub(crate) fn derive_overhead( encoded_len: usize, coefficients: OverheadCoefficients, ) -> u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); @@ -247,8 +247,8 @@ pub(crate) fn derive_overhead( let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -262,16 +262,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); - + // ``` vec![ (coefficients.ergs_limit_overhead_coeficient * overhead_for_single_instance_circuits.as_u32() as f64) @@ -288,7 +289,7 @@ pub(crate) fn derive_overhead( /// Contains the coefficients with which the overhead for transactions will be calculated. /// All of the coefficients should be <= 1. There are here to provide a certain "discount" for normal transactions /// at the risk of malicious transactions that may close the block prematurely. -/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coeficient` MUST +/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coefficient` MUST /// result in an integer number #[derive(Debug, Clone, Copy)] pub struct OverheadCoefficients { @@ -326,8 +327,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -355,28 +356,28 @@ pub fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -387,7 +388,7 @@ pub fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -400,15 +401,18 @@ pub fn get_amortized_overhead( }; // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, - // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. + // since the pubdata is not published. If decided to use the pubdata overhead, it needs to be updated. + // ``` // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + // Throwing off the `ceil`, while may provide marginally lower // overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE - // ceil(OB * (TL - OE) / (EP * MP)) >= OE - // + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // `ceil(OB * (TL - OE) / (EP * MP)) >= OE` + // ``` // OB * (TL - OE) / (MP * EP) > OE - 1 // OB * (TL - OE) > (OE - 1) * EP * MP // OB * TL + EP * MP > OE * EP * MP + OE * OB @@ -419,7 +423,7 @@ pub fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -428,7 +432,7 @@ pub fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - + // // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) @@ -437,6 +441,7 @@ pub fn get_amortized_overhead( // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + // ``` let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from( @@ -451,16 +456,16 @@ pub fn get_amortized_overhead( let overhead = vec![tx_slot_overhead, overhead_for_length, overhead_for_gas] .into_iter() .max() - // For the sake of consistency making sure that total_gas_limit >= max_overhead + // For the sake of consistency making sure that `total_gas_limit >= max_overhead` .map(|max_overhead| std::cmp::min(max_overhead, total_gas_limit.as_u32())) .unwrap(); let limit_after_deducting_overhead = total_gas_limit - overhead; // During double checking of the overhead, the bootloader will assume that the - // body of the transaction does not have any more than MAX_L2_TX_GAS_LIMIT ergs available to it. + // body of the transaction does not have any more than `MAX_L2_TX_GAS_LIMIT` ergs available to it. if limit_after_deducting_overhead.as_u64() > MAX_L2_TX_GAS_LIMIT { - // We derive the same overhead that would exist for the MAX_L2_TX_GAS_LIMIT ergs + // We derive the same overhead that would exist for the `MAX_L2_TX_GAS_LIMIT` ergs derive_overhead( MAX_L2_TX_GAS_LIMIT as u32, gas_per_pubdata_byte_limit, @@ -495,7 +500,7 @@ mod tests { } else { 0u32 }; - // Safe cast: the gas_limit for a transaction can not be larger than 2^32 + // Safe cast: the gas_limit for a transaction can not be larger than `2^32` let mut right_bound = total_gas_limit; // The closure returns whether a certain overhead would be accepted by the bootloader. diff --git a/core/lib/multivm/src/versions/vm_m6/utils.rs b/core/lib/multivm/src/versions/vm_m6/utils.rs index b383f7c1753f..4cabab82c9c8 100644 --- a/core/lib/multivm/src/versions/vm_m6/utils.rs +++ b/core/lib/multivm/src/versions/vm_m6/utils.rs @@ -227,7 +227,7 @@ pub fn collect_log_queries_after_timestamp( /// Receives sorted slice of timestamps. /// Returns count of timestamps that are greater than or equal to `from_timestamp`. -/// Works in O(log(sorted_timestamps.len())). +/// Works in `O(log(sorted_timestamps.len()))`. pub fn precompile_calls_count_after_timestamp( sorted_timestamps: &[Timestamp], from_timestamp: Timestamp, diff --git a/core/lib/multivm/src/versions/vm_m6/vm.rs b/core/lib/multivm/src/versions/vm_m6/vm.rs index e82d51d2bf3b..938b47784de9 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm.rs @@ -167,7 +167,7 @@ impl VmInterface for Vm { system_logs: vec![], total_log_queries, cycles_used: self.vm.state.local_state.monotonic_cycle_counter, - // It's not applicable for vm6 + // It's not applicable for `vm6` deduplicated_events_logs: vec![], storage_refunds: vec![], user_l2_to_l1_logs: l2_to_l1_logs, diff --git a/core/lib/multivm/src/versions/vm_m6/vm_instance.rs b/core/lib/multivm/src/versions/vm_m6/vm_instance.rs index 3c77c1159149..1e792d308f1f 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_instance.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_instance.rs @@ -202,7 +202,7 @@ fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(NewVmExecutionResult::Panic) } else { @@ -235,7 +235,7 @@ fn vm_may_have_ended( NewVmExecutionResult::Ok(data) => { Some(VmExecutionResult { // The correct `events` value for this field should be set separately - // later on based on the information inside the event_sink oracle. + // later on based on the information inside the `event_sink` oracle. events: vec![], storage_log_queries: vm.get_final_log_queries(), used_contract_hashes: vm.get_used_contracts(), @@ -508,8 +508,8 @@ impl VmInstance { ); } - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = tracer.requested_refund() { assert!( operator_refund.is_none(), @@ -605,8 +605,8 @@ impl VmInstance { /// Panics if there are no new transactions in bootloader. /// Internally uses the OneTxTracer to stop the VM when the last opcode from the transaction is reached. // Err when transaction is rejected. - // Ok(status: TxExecutionStatus::Success) when the transaction succeeded - // Ok(status: TxExecutionStatus::Failure) when the transaction failed. + // `Ok(status: TxExecutionStatus::Success)` when the transaction succeeded + // `Ok(status: TxExecutionStatus::Failure)` when the transaction failed. // Note that failed transactions are considered properly processed and are included in blocks pub fn execute_next_tx( &mut self, @@ -666,7 +666,7 @@ impl VmInstance { revert_reason: None, // getting contracts used during this transaction // at least for now the number returned here is always <= to the number - // of the code hashes actually used by the transaction, since it might've + // of the code hashes actually used by the transaction, since it might have // reused bytecode hashes from some of the previous ones. contracts_used: self .state @@ -951,8 +951,8 @@ impl VmInstance { pub fn save_current_vm_as_snapshot(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs index 66b2f3d07d78..02dfc0f313a4 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs @@ -34,8 +34,8 @@ use crate::vm_m6::{ OracleTools, VmInstance, }; -// TODO (SMA-1703): move these to config and make them programmatically generatable. -// fill these values in the similar fasion as other overhead-related constants +// TODO (SMA-1703): move these to config and make them programmatically generable. +// fill these values in the similar fashion as other overhead-related constants pub const BLOCK_OVERHEAD_GAS: u32 = 1200000; pub const BLOCK_OVERHEAD_L1_GAS: u32 = 1000000; pub const BLOCK_OVERHEAD_PUBDATA: u32 = BLOCK_OVERHEAD_L1_GAS / L1_GAS_PER_PUBDATA_BYTE; @@ -87,7 +87,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, @@ -256,12 +256,12 @@ pub fn init_vm_with_gas_limit( } #[derive(Debug, Clone, Copy)] -// The block.number/block.timestamp data are stored in the CONTEXT_SYSTEM_CONTRACT. +// The `block.number` / `block.timestamp` data are stored in the `CONTEXT_SYSTEM_CONTRACT`. // The bootloader can support execution in two modes: -// - "NewBlock" when the new block is created. It is enforced that the block.number is incremented by 1 +// - `NewBlock` when the new block is created. It is enforced that the block.number is incremented by 1 // and the timestamp is non-decreasing. Also, the L2->L1 message used to verify the correctness of the previous root hash is sent. // This is the mode that should be used in the state keeper. -// - "OverrideCurrent" when we need to provide custom block.number and block.timestamp. ONLY to be used in testing/ethCalls. +// - `OverrideCurrent` when we need to provide custom `block.number` and `block.timestamp`. ONLY to be used in testing / `ethCalls`. pub enum BlockContextMode { NewBlock(DerivedBlockContext, U256), OverrideCurrent(DerivedBlockContext), diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/l2_block.rs index 6cd1096b3bd3..03544d8b054b 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/l2_block.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/l2_block.rs @@ -19,7 +19,7 @@ pub(crate) struct BootloaderL2Block { pub(crate) timestamp: u64, pub(crate) txs_rolling_hash: H256, // The rolling hash of all the transactions in the miniblock pub(crate) prev_block_hash: H256, - // Number of the first l2 block tx in l1 batch + // Number of the first L2 block tx in L1 batch pub(crate) first_tx_index: usize, pub(crate) max_virtual_blocks_to_create: u32, pub(super) txs: Vec, diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/utils.rs index 8adeb3e0b428..f47b95d6cbf7 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/utils.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/bootloader_state/utils.rs @@ -92,8 +92,8 @@ pub(crate) fn apply_l2_block( bootloader_l2_block: &BootloaderL2Block, txs_index: usize, ) { - // Since L2 block infos start from the TX_OPERATOR_L2_BLOCK_INFO_OFFSET and each - // L2 block info takes TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots, the position where the L2 block info + // Since L2 block information start from the `TX_OPERATOR_L2_BLOCK_INFO_OFFSET` and each + // L2 block info takes `TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO` slots, the position where the L2 block info // for this transaction needs to be written is: let block_position = diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/snapshots.rs index c34535726c04..56c219fffa4b 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/snapshots.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/snapshots.rs @@ -37,8 +37,8 @@ impl Vm { pub(crate) fn make_snapshot_inner(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/event_sink.rs index 43019cce1ce7..74dca71d10f6 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/event_sink.rs @@ -50,7 +50,7 @@ impl InMemoryEventSink { pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { let events = self.frames_stack.forward().current_frame(); - // Select all of the last elements where e.timestamp >= from_timestamp. + // Select all of the last elements where `e.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. events .rsplit(|e| e.timestamp < from_timestamp) diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/history_recorder.rs index fc6179cb1481..e862f57898ae 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/history_recorder.rs @@ -12,14 +12,14 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; pub(crate) type MemoryWithHistory = HistoryRecorder; pub(crate) type IntFrameManagerWithHistory = HistoryRecorder, H>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` // can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/memory.rs index 8568d6c72157..9219126d76e8 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/memory.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/memory.rs @@ -282,7 +282,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -299,7 +299,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -307,7 +307,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs index 6705831dbade..9a7addc97e11 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_refunds_enhancement::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/utils.rs index bc4b2c3eff15..c2478edf7a89 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/utils.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/utils.rs @@ -122,7 +122,7 @@ pub(crate) fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(VmExecutionResult::Panic) } else { diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/oracles/storage.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/oracles/storage.rs index b970a8a95f73..6e58b8b30925 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/oracles/storage.rs @@ -48,7 +48,7 @@ pub struct StorageOracle { pub(crate) paid_changes: HistoryRecorder, H>, // The map that contains all the first values read from storage for each slot. - // While formally it does not have to be rollbackable, we still do it to avoid memory bloat + // While formally it does not have to be capable of rolling back, we still do it to avoid memory bloat // for unused slots. pub(crate) initial_values: HistoryRecorder, H>, @@ -182,7 +182,7 @@ impl StorageOracle { let required_pubdata = self.base_price_for_write(&key, first_slot_value, current_slot_value); - // We assume that "prepaid_for_slot" represents both the number of pubdata published and the number of bytes paid by the previous transactions + // We assume that `prepaid_for_slot` represents both the number of pubdata published and the number of bytes paid by the previous transactions // as they should be identical. let prepaid_for_slot = self .pre_paid_changes @@ -252,7 +252,7 @@ impl StorageOracle { ) -> &[Box] { let logs = self.frames_stack.forward().current_frame(); - // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Select all of the last elements where `l.log_query.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. logs.rsplit(|l| l.log_query.timestamp < from_timestamp) .next() @@ -300,6 +300,7 @@ impl VmStorageOracle for StorageOracle { _monotonic_cycle_counter: u32, query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -309,6 +310,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -394,7 +396,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -408,8 +410,8 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). -// While new_value is always 32 bytes long, for key we use the following optimization: +// we basically need to publish the following pair: `()`. +// While `new_value` is always 32 bytes long, for key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. // - The second time we publish it, we will use this 8-byte instead of the 32 bytes of the entire key. diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/default_tracers.rs index 8e9c0f11aba4..47fe3142abac 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/default_tracers.rs @@ -49,7 +49,7 @@ pub(crate) struct DefaultExecutionTracer { pub(crate) result_tracer: ResultTracer, // This tracer is designed specifically for calculating refunds. Its separation from the custom tracer // ensures static dispatch, enhancing performance by avoiding dynamic dispatch overhead. - // Additionally, being an internal tracer, it saves the results directly to VmResultAndLogs. + // Additionally, being an internal tracer, it saves the results directly to `VmResultAndLogs`. pub(crate) refund_tracer: Option, pub(crate) dispatcher: TracerDispatcher, ret_from_the_bootloader: Option, @@ -291,7 +291,7 @@ impl VmTracer for DefaultExecutionTracer< } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs index f906cef6230b..b66918b476b1 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs @@ -209,8 +209,8 @@ impl VmTracer for RefundsTracer { #[vise::register] static METRICS: vise::Global = vise::Global::new(); - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = self.requested_refund() { assert!( self.operator_refund.is_none(), diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/result_tracer.rs index 1281b416bb47..22cf08c8ef93 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/result_tracer.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/result_tracer.rs @@ -52,7 +52,7 @@ impl ResultTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 @@ -151,7 +151,7 @@ impl ResultTracer { }); } VmExecutionResult::Revert(output) => { - // Unlike VmHook::ExecutionResult, vm has completely finished and returned not only the revert reason, + // Unlike `VmHook::ExecutionResult`, vm has completely finished and returned not only the revert reason, // but with bytecode, which represents the type of error from the bootloader side let revert_reason = TxRevertReason::parse_error(&output); diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/utils.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/utils.rs index 8de2ad181f47..ccacea0cd7e4 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/utils.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/utils.rs @@ -55,7 +55,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs index 4b70a79fdd4c..82b79c012f20 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs @@ -236,7 +236,7 @@ impl TransactionData { let l2_tx: L2Tx = self.clone().try_into().unwrap(); let transaction_request: TransactionRequest = l2_tx.into(); - // It is assumed that the TransactionData always has all the necessary components to recover the hash. + // It is assumed that the `TransactionData` always has all the necessary components to recover the hash. transaction_request .get_tx_hash(chain_id) .expect("Could not recover L2 transaction hash") diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs index 6df858efbf7c..e48f4e8c9654 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs @@ -18,7 +18,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/l2_blocks.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/l2_blocks.rs index 5dd26c4c0277..e5832f7f5879 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/l2_blocks.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/l2_blocks.rs @@ -68,7 +68,7 @@ pub fn load_last_l2_block(storage: StoragePtr) -> Option u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); @@ -23,8 +23,8 @@ pub(crate) fn derive_overhead( let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -38,15 +38,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); + // ``` vec![ (coefficients.ergs_limit_overhead_coeficient @@ -102,8 +104,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -132,28 +134,28 @@ pub(crate) fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -164,7 +166,7 @@ pub(crate) fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -177,15 +179,18 @@ pub(crate) fn get_amortized_overhead( }; // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, - // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. - // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas - // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // since the pubdata is not published. If decided to use the pubdata overhead, it needs to be updated. + // ``` + // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK` + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + // Throwing off the `ceil`, while may provide marginally lower // overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE - // ceil(OB * (TL - OE) / (EP * MP)) >= OE - // + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // `ceil(OB * (TL - OE) / (EP * MP)) >= OE` + // ``` // OB * (TL - OE) / (MP * EP) > OE - 1 // OB * (TL - OE) > (OE - 1) * EP * MP // OB * TL + EP * MP > OE * EP * MP + OE * OB @@ -196,7 +201,7 @@ pub(crate) fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -205,7 +210,7 @@ pub(crate) fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - + // // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) @@ -214,6 +219,7 @@ pub(crate) fn get_amortized_overhead( // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + // ``` let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from( @@ -228,16 +234,16 @@ pub(crate) fn get_amortized_overhead( let overhead = vec![tx_slot_overhead, overhead_for_length, overhead_for_gas] .into_iter() .max() - // For the sake of consistency making sure that total_gas_limit >= max_overhead + // For the sake of consistency making sure that `total_gas_limit >= max_overhead` .map(|max_overhead| std::cmp::min(max_overhead, total_gas_limit.as_u32())) .unwrap(); let limit_after_deducting_overhead = total_gas_limit - overhead; // During double checking of the overhead, the bootloader will assume that the - // body of the transaction does not have any more than MAX_L2_TX_GAS_LIMIT ergs available to it. + // body of the transaction does not have any more than `MAX_L2_TX_GAS_LIMIT` ergs available to it. if limit_after_deducting_overhead.as_u64() > MAX_L2_TX_GAS_LIMIT { - // We derive the same overhead that would exist for the MAX_L2_TX_GAS_LIMIT ergs + // We derive the same overhead that would exist for the `MAX_L2_TX_GAS_LIMIT` ergs derive_overhead( MAX_L2_TX_GAS_LIMIT as u32, gas_per_pubdata_byte_limit, @@ -270,7 +276,7 @@ mod tests { } else { 0u32 }; - // Safe cast: the gas_limit for a transaction can not be larger than 2^32 + // Safe cast: the `gas_limit` for a transaction can not be larger than `2^32` let mut right_bound = total_gas_limit; // The closure returns whether a certain overhead would be accepted by the bootloader. diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/l2_block.rs index 8ce851d6699c..48284bcc2ac8 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/l2_block.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/l2_block.rs @@ -19,7 +19,7 @@ pub(crate) struct BootloaderL2Block { pub(crate) timestamp: u64, pub(crate) txs_rolling_hash: H256, // The rolling hash of all the transactions in the miniblock pub(crate) prev_block_hash: H256, - // Number of the first l2 block tx in l1 batch + // Number of the first L2 block tx in L1 batch pub(crate) first_tx_index: usize, pub(crate) max_virtual_blocks_to_create: u32, pub(super) txs: Vec, diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/utils.rs index a3986d6fe463..9a682da3a5ab 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/utils.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/bootloader_state/utils.rs @@ -92,8 +92,8 @@ pub(crate) fn apply_l2_block( bootloader_l2_block: &BootloaderL2Block, txs_index: usize, ) { - // Since L2 block infos start from the TX_OPERATOR_L2_BLOCK_INFO_OFFSET and each - // L2 block info takes TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots, the position where the L2 block info + // Since L2 block information start from the `TX_OPERATOR_L2_BLOCK_INFO_OFFSET` and each + // L2 block info takes `TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO` slots, the position where the L2 block info // for this transaction needs to be written is: let block_position = diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/snapshots.rs index 569e11150390..2b653333a5c1 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/snapshots.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/snapshots.rs @@ -35,8 +35,8 @@ impl Vm { pub(crate) fn make_snapshot_inner(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs index 7657babfe4a4..1421a7b35f48 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/statistics.rs @@ -38,7 +38,7 @@ impl Vm { gas_used: gas_remaining_before - gas_remaining_after, computational_gas_used, total_log_queries: total_log_queries_count, - // This field will be populated by the RefundTracer + // This field will be populated by the `RefundTracer` pubdata_published: 0, estimated_circuits_used: 0.0, } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/event_sink.rs index 02938594b5cf..00a03ca0adb8 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/event_sink.rs @@ -50,7 +50,7 @@ impl InMemoryEventSink { pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { let events = self.frames_stack.forward().current_frame(); - // Select all of the last elements where e.timestamp >= from_timestamp. + // Select all of the last elements where `e.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. events .rsplit(|e| e.timestamp < from_timestamp) diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/history_recorder.rs index 6154dc8e24d6..baed63c14b89 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/history_recorder.rs @@ -12,14 +12,14 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; pub(crate) type MemoryWithHistory = HistoryRecorder; pub(crate) type IntFrameManagerWithHistory = HistoryRecorder, H>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` // can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/memory.rs index c78f8a9e7798..a48620db11ce 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/memory.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/memory.rs @@ -282,7 +282,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -299,7 +299,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -307,7 +307,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs index 061912f83c92..f01394cebb52 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs @@ -15,7 +15,7 @@ use crate::vm_virtual_blocks::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }; -/// The main job of the DecommiterOracle is to implement the DecommittmentProcessor trait - that is +/// The main job of the DecommiterOracle is to implement the DecommitmentProcessor trait - that is /// used by the VM to 'load' bytecodes into memory. #[derive(Debug)] pub struct DecommitterOracle { diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/storage.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/storage.rs index 91c293f4ac80..2555f57fc7e1 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/storage.rs @@ -167,7 +167,7 @@ impl StorageOracle { ) -> &[Box] { let logs = self.frames_stack.forward().current_frame(); - // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Select all of the last elements where `l.log_query.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. logs.rsplit(|l| l.log_query.timestamp < from_timestamp) .next() @@ -208,13 +208,14 @@ impl StorageOracle { } impl VmStorageOracle for StorageOracle { - // Perform a storage read/write access by taking an partially filled query + // Perform a storage read / write access by taking an partially filled query // and returning filled query and cold/warm marker for pricing purposes fn execute_partial_query( &mut self, _monotonic_cycle_counter: u32, query: LogQuery, ) -> LogQuery { + // ``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -224,6 +225,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + // ``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -303,7 +305,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -317,8 +319,8 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). -// While new_value is always 32 bytes long, for key we use the following optimization: +// we basically need to publish the following pair: `()`. +// While `new_value` is always 32 bytes long, for key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. // - The second time we publish it, we will use this 8-byte instead of the 32 bytes of the entire key. diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/utils.rs index 7d38ba1058df..5be62e384378 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/utils.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/utils.rs @@ -122,7 +122,7 @@ pub(crate) fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(VmExecutionResult::Panic) } else { diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/default_tracers.rs index 463bdaa4f359..f6007214494d 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/default_tracers.rs @@ -271,7 +271,7 @@ impl DefaultExecutionTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs index 6051cd7bb7d8..9cdf30da9ce1 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs @@ -208,8 +208,8 @@ impl ExecutionProcessing for RefundsTrace #[vise::register] static METRICS: vise::Global = vise::Global::new(); - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = self.requested_refund() { assert!( self.operator_refund.is_none(), diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/result_tracer.rs index 8c6a5d1793f8..3ba396fd0c4c 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/result_tracer.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/result_tracer.rs @@ -53,7 +53,7 @@ impl ResultTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 @@ -152,7 +152,7 @@ impl ResultTracer { }); } VmExecutionResult::Revert(output) => { - // Unlike VmHook::ExecutionResult, vm has completely finished and returned not only the revert reason, + // Unlike `VmHook::ExecutionResult`, vm has completely finished and returned not only the revert reason, // but with bytecode, which represents the type of error from the bootloader side let revert_reason = TxRevertReason::parse_error(&output); diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/utils.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/utils.rs index b2358602fe05..1f3d27d9d20e 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/utils.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/utils.rs @@ -55,7 +55,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs index 6d2fe36868b0..8bebd5ab321d 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs @@ -234,7 +234,7 @@ impl TransactionData { let l2_tx: L2Tx = self.clone().try_into().unwrap(); let transaction_request: TransactionRequest = l2_tx.into(); - // It is assumed that the TransactionData always has all the necessary components to recover the hash. + // It is assumed that the `TransactionData` always has all the necessary components to recover the hash. transaction_request .get_tx_hash(chain_id) .expect("Could not recover L2 transaction hash") diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs index 2762eb447161..64bf1462c777 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs @@ -18,7 +18,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/l2_blocks.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/l2_blocks.rs index 5dd26c4c0277..e5832f7f5879 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/l2_blocks.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/l2_blocks.rs @@ -68,7 +68,7 @@ pub fn load_last_l2_block(storage: StoragePtr) -> Option u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); @@ -23,8 +23,8 @@ pub(crate) fn derive_overhead( let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -38,15 +38,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in O(1) + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); + // ``` vec![ (coefficients.ergs_limit_overhead_coeficient @@ -64,7 +66,7 @@ pub(crate) fn derive_overhead( /// Contains the coefficients with which the overhead for transactions will be calculated. /// All of the coefficients should be <= 1. There are here to provide a certain "discount" for normal transactions /// at the risk of malicious transactions that may close the block prematurely. -/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coeficient` MUST +/// IMPORTANT: to perform correct computations, `MAX_TX_ERGS_LIMIT / coefficients.ergs_limit_overhead_coefficient` MUST /// result in an integer number #[derive(Debug, Clone, Copy)] pub struct OverheadCoefficients { @@ -102,8 +104,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -132,28 +134,28 @@ pub(crate) fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -164,7 +166,7 @@ pub(crate) fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -177,13 +179,17 @@ pub(crate) fn get_amortized_overhead( }; // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, - // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. + // since the pubdata is not published. If decided to use the pubdata overhead, it needs to be updated. + // ``` // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas // O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK = ceil(gas_limit / gas_per_pubdata_byte_limit) / MAX_PUBDATA_PER_BLOCK - // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). Throwing off the `ceil`, while may provide marginally lower + // >= (gas_limit / (gas_per_pubdata_byte_limit * MAX_PUBDATA_PER_BLOCK). + // ``` + // Throwing off the `ceil`, while may provide marginally lower // overhead to the operator, provides substantially easier formula to work with. // - // For better clarity, let's denote gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE + // For better clarity, let's denote `gas_limit = GL, MAX_PUBDATA_PER_BLOCK = MP, gas_per_pubdata_byte_limit = EP, overhead_for_block_gas = OB, total_gas_limit = TL, overhead_gas = OE` + // ``` // ceil(OB * (TL - OE) / (EP * MP)) >= OE // // OB * (TL - OE) / (MP * EP) > OE - 1 @@ -196,7 +202,7 @@ pub(crate) fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -205,7 +211,7 @@ pub(crate) fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - + // // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 7c289f7b4b44..1dfdd03e5231 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -14,39 +14,69 @@ < > % +^ 0x00 0x01 0x02 +0x20 ~10x u32 u64 +u8 +1B H256 10e18 10^9 +2^32 +2^128 +2^24 +10^32 +10^* +2^16 +2^64 +10^8 U256 +12.5% +5% +10% +20% +*% +90% +f64 k M kb 50M +2M +130µs – 18kb 128kb 10k 100k 120k +800k 24k 500k +50k 120kb +18kb +12GB +20GB 500B 100M ~100us 10ms +1_000ms 1us ~100 +gwei ABI vlog +const L2 +L2s L1 json l1 @@ -57,6 +87,7 @@ ZKSYNC_HOME MultiVMTracer vm_virtual_blocks eth_node +EthCall BaseSystemContracts eth_calls refactor @@ -115,6 +146,7 @@ unfinalizable meterer Timedout bootloader +bootloader's testkit Sepolia Goerli @@ -146,11 +178,6 @@ param HistoryDisabled HistoryEnabled sorted_timestamps -DecommiterOracle -DecommittmentProcessor -encodings -DecommittmentProcessor -decommitment known_bytecodes returndata namespaces @@ -159,6 +186,19 @@ BYTES_PER_ENUMERATION_INDEX derived_key prefill reorg +precompile +Init +init +enqueued +stage2 +testnets +ethCalls +generable +Serde +tokenize +EOAs +zeroized +value // zkSync-related words matterlabs @@ -188,6 +228,7 @@ serializer serializable deserializer Deserializes +deserializes serializing deserializing deserialization @@ -225,7 +266,6 @@ metadata boojum deps Precalculated -decommitted WASM DefaultPrecompilesProcessor LSB @@ -279,6 +319,93 @@ blake2 AR16MT Preimages EN's +SystemContext +StorageOracle +intrinsics +chunked +chunking +deadbeef01 +deadbeef0 +deadbeef +unsynced +computable +DevEx +Workspace +NFT +preimage +subcalls +hashmaps +monotonicity +subquery +RPCs +programmatically +stdin +stderr +Linter +SmallRng +ZkPorter +StateDiffs +HashMaps +encodings +CTPOP +decommitter +Decommitter +Decommitments +Decommitment +decommitment +decommitments +Decommit +decommit +decommits +DecommiterOracle +DecommitmentProcessor +decommitted +decommit +decommitting +Demuxer +demultiplex +recid +inversed +plux +Binop +Arithmetization +arithmetization +nocapture +Plonky +permissioned +mathbb +Invb +REDC +iszero +skept +ECADD +ECMUL +preds +inttoptr +syncvm +nasm +rodata +ISZERO +JUMPI +ethir +ptrtoint +lshr +getu +zext +noprofile +umin +cccond +ccret +prodm +prodl +prodeh +prodh +interm +signv +ashr +noalias +immediates +prode StorageBatchInfo CommitBatchInfo IExecutor @@ -306,6 +433,50 @@ pepe Arweave Streamr dutterbutter +NixOS +CLI +SQLx +Rustup +nextest +NTFS +toolchains +toolchain +IDE +M1 +M2 +MacOS +OpenSSL +Xcode +LLVM +nvm +LTS +logout +WSL +orchestrator +TypeScript +Cryptographical +cryptographical +microservices +Executables +subcomponents +v2 +v1 +rmSync +SSL +setup_2^26 +uncomment +toml +GCP +dev +workspace +subcommand +Kubernetes +Etherscan +cryptographic +hashers +MacBook +DDR5 +~ // Used libraries numberish @@ -410,6 +581,16 @@ RLP DAL zkSync's l2_to_l1 +PoW +coinbase +FIXME +ASC +DESC +Versioning +initializer +refactoring +prefetch +unformatted // crypto events Edcon @@ -426,12 +607,17 @@ Armeabi scijava gluk @Deniallugo's +emilluta // Programming related words bytecode bytecodes +impl +subrange timeframe +leaf_count mkdir +librocksdb zksolc zksyncrobot precompiles @@ -448,6 +634,8 @@ Zerion Maverik zk_evm_1_3_3 vk +vks +CORS verifier crypto callee @@ -495,6 +683,7 @@ l2_block submodule enums deserialized +deserialize hashmap vm_m5 SDK @@ -586,20 +775,36 @@ eth_getLogs façade virtual_blocks_per_miniblock virtual_block_interval +max_overhead +total_gas_limit cloneable timestamped healthcheck +Healthcheck +HealthCheck readonly upgrader startup +PingCAP +witgen +ok +hacky +ceil +Infura +synth AUTOGENERATED x19Ethereum block_timestamp SYSTEM_BLOCK_INFO_BLOCK_NUMBER_MULTIPLIER +MAX_L2_TX_GAS_LIMIT +MAX_TX_ERGS_LIMIT OneTxTracer multicall +Multicall's Multicall3 +proxied +scalers updatable instantiation unexecuted @@ -612,4 +817,36 @@ compactions M6 compiler_common noop -tokenized \ No newline at end of file +rustc +sqlx +zkevm +Boojum +Sepolia +psql +Cuda +cuda +hdcaa +impls +abda +edaf +unsynchronized +CUDA +gcloud +NVME +OTLP +multiVM +Deduplicator +lobkc +sread +myfunction +merklelization +beaf +subcalls +unallowed +Nuxt +Merklized +satisfiability +demultiplex +precompile +statekeeper +tokenized From b46c2e9cbbcd048647f998810c8d550f8ad0c1f4 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Fri, 5 Jan 2024 17:22:30 -0600 Subject: [PATCH 23/49] feat: fix spelling in dev comments in `core/lib/*` - continued (#684) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Series of PRs: This is a part in a series of PRs aimed at enhancing spelling accuracy in this repository. See:** - [ ] https://github.com/matter-labs/zksync-era/pull/681 - [ ] https://github.com/matter-labs/zksync-era/pull/682 - [ ] https://github.com/matter-labs/zksync-era/pull/683 - [ ] https://github.com/matter-labs/zksync-era/pull/685 Once merged, a final PR will enable the `dev_comments: true` in the cargo-spellcheck config file. ## What ❔ - **Spelling Corrections**: This PR focuses on rectifying spelling errors found in the developer comments within multiple core libraries: `core/lib/types/`, `core/lib/eth_client/`, ` core/lib/eth_signer` , `core/lib/mempool`, `core/lib/merkle_tree`, `core/lib/state` - Updates dictionary ## Why ❔ - **Consistency and Readability:** The addressed directories contain numerous spelling inaccuracies. Correcting these enhances the readability and consistency of our documentation, ensuring clear understanding for current and future developer ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov --- core/lib/eth_client/src/clients/http/query.rs | 4 ++-- core/lib/eth_client/src/clients/http/signing.rs | 2 +- core/lib/eth_client/src/clients/mock.rs | 4 ++-- core/lib/eth_signer/src/json_rpc_signer.rs | 2 +- core/lib/eth_signer/src/pk_signer.rs | 4 ++-- core/lib/eth_signer/src/raw_ethereum_tx.rs | 4 ++-- core/lib/mempool/src/tests.rs | 2 +- core/lib/merkle_tree/src/consistency.rs | 4 ++-- core/lib/merkle_tree/src/storage/tests.rs | 4 ++-- core/lib/merkle_tree/src/types/internal.rs | 6 +++--- core/lib/state/src/rocksdb/mod.rs | 2 +- core/lib/storage/src/db.rs | 2 +- core/lib/types/src/commitment.rs | 6 +++--- core/lib/types/src/eth_sender.rs | 2 +- core/lib/types/src/event.rs | 12 ++++++------ core/lib/types/src/l1/mod.rs | 8 ++++---- core/lib/types/src/l2/mod.rs | 4 ++-- core/lib/types/src/proofs.rs | 2 +- core/lib/types/src/protocol_version.rs | 8 ++++---- core/lib/types/src/snapshots.rs | 4 ++-- core/lib/types/src/storage/log.rs | 2 +- core/lib/types/src/storage/mod.rs | 2 +- core/lib/types/src/system_contracts.rs | 4 ++-- core/lib/types/src/transaction_request.rs | 10 +++++----- core/lib/types/src/tx/execute.rs | 2 +- .../tx/primitives/eip712_signature/struct_builder.rs | 2 +- .../primitives/eip712_signature/typed_structure.rs | 2 +- core/lib/types/src/utils.rs | 2 +- core/lib/utils/src/bytecode.rs | 8 ++++---- spellcheck/era.dic | 6 ++++++ 30 files changed, 66 insertions(+), 60 deletions(-) diff --git a/core/lib/eth_client/src/clients/http/query.rs b/core/lib/eth_client/src/clients/http/query.rs index de8a51a731eb..d1abb74c46d6 100644 --- a/core/lib/eth_client/src/clients/http/query.rs +++ b/core/lib/eth_client/src/clients/http/query.rs @@ -97,8 +97,8 @@ impl EthInterface for QueryClient { let mut history = Vec::with_capacity(block_count); let from_block = upto_block.saturating_sub(block_count); - // Here we are requesting fee_history from blocks - // (from_block; upto_block] in chunks of size MAX_REQUEST_CHUNK + // Here we are requesting `fee_history` from blocks + // `(from_block; upto_block)` in chunks of size `MAX_REQUEST_CHUNK` // starting from the oldest block. for chunk_start in (from_block..=upto_block).step_by(MAX_REQUEST_CHUNK) { let chunk_end = (chunk_start + MAX_REQUEST_CHUNK).min(upto_block); diff --git a/core/lib/eth_client/src/clients/http/signing.rs b/core/lib/eth_client/src/clients/http/signing.rs index 269d773a2588..6e3dd3d223d8 100644 --- a/core/lib/eth_client/src/clients/http/signing.rs +++ b/core/lib/eth_client/src/clients/http/signing.rs @@ -226,7 +226,7 @@ impl BoundEthInterface for SigningClient { None => self.inner.default_priority_fee_per_gas, }; - // Fetch current base fee and add max_priority_fee_per_gas + // Fetch current base fee and add `max_priority_fee_per_gas` let max_fee_per_gas = match options.max_fee_per_gas { Some(max_fee_per_gas) => max_fee_per_gas, None => { diff --git a/core/lib/eth_client/src/clients/mock.rs b/core/lib/eth_client/src/clients/mock.rs index ae223bb47101..5541dd1d198b 100644 --- a/core/lib/eth_client/src/clients/mock.rs +++ b/core/lib/eth_client/src/clients/mock.rs @@ -177,14 +177,14 @@ impl MockEthereum { .unwrap_or(self.max_priority_fee_per_gas); let nonce = options.nonce.expect("Nonce must be set for every tx"); - // Nonce and gas_price are appended to distinguish the same transactions + // Nonce and `gas_price` are appended to distinguish the same transactions // with different gas by their hash in tests. raw_tx.append(&mut ethabi::encode(&max_fee_per_gas.into_tokens())); raw_tx.append(&mut ethabi::encode(&max_priority_fee_per_gas.into_tokens())); raw_tx.append(&mut ethabi::encode(&nonce.into_tokens())); let hash = Self::fake_sha256(&raw_tx); // Okay for test purposes. - // Concatenate raw_tx plus hash for test purposes + // Concatenate `raw_tx` plus hash for test purposes let mut new_raw_tx = hash.as_bytes().to_vec(); new_raw_tx.extend(raw_tx); Ok(SignedCallResult { diff --git a/core/lib/eth_signer/src/json_rpc_signer.rs b/core/lib/eth_signer/src/json_rpc_signer.rs index 66a7b33e9894..38e9a04e19aa 100644 --- a/core/lib/eth_signer/src/json_rpc_signer.rs +++ b/core/lib/eth_signer/src/json_rpc_signer.rs @@ -171,7 +171,7 @@ impl JsonRpcSigner { None => AddressOrIndex::Index(0), }; - // EthereumSigner can support many different addresses, + // `EthereumSigner` can support many different addresses, // we define only the one we need by the index // of receiving from the server or by the address itself. signer.detect_address(address_or_index).await?; diff --git a/core/lib/eth_signer/src/pk_signer.rs b/core/lib/eth_signer/src/pk_signer.rs index 4f9795dca865..0ea68e2a6df9 100644 --- a/core/lib/eth_signer/src/pk_signer.rs +++ b/core/lib/eth_signer/src/pk_signer.rs @@ -62,7 +62,7 @@ impl EthereumSigner for PrivateKeySigner { let key = SecretKey::from_slice(self.private_key.as_bytes()).unwrap(); // According to the code in web3 - // We should use max_fee_per_gas as gas_price if we use EIP1559 + // We should use `max_fee_per_gas` as `gas_price` if we use EIP1559 let gas_price = raw_tx.max_fee_per_gas; let max_priority_fee_per_gas = raw_tx.max_priority_fee_per_gas; @@ -113,7 +113,7 @@ mod test { .await .unwrap(); assert_ne!(raw_tx.len(), 1); - // precalculated signature with right algorithm implementation + // pre-calculated signature with right algorithm implementation let precalculated_raw_tx: Vec = vec![ 1, 248, 100, 130, 1, 14, 1, 2, 128, 148, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 128, 131, 1, 2, 3, 192, 1, 160, 98, 201, 238, 158, 215, 98, 23, 231, diff --git a/core/lib/eth_signer/src/raw_ethereum_tx.rs b/core/lib/eth_signer/src/raw_ethereum_tx.rs index 124c09965de4..03d085d91336 100644 --- a/core/lib/eth_signer/src/raw_ethereum_tx.rs +++ b/core/lib/eth_signer/src/raw_ethereum_tx.rs @@ -104,7 +104,7 @@ impl Transaction { let list_size = if signature.is_some() { 11 } else { 8 }; stream.begin_list(list_size); - // append chain_id. from EIP-2930: chainId is defined to be an integer of arbitrary size. + // append `chain_id`. from EIP-2930: `chainId` is defined to be an integer of arbitrary size. stream.append(&chain_id); self.rlp_append_legacy(&mut stream); @@ -123,7 +123,7 @@ impl Transaction { let list_size = if signature.is_some() { 12 } else { 9 }; stream.begin_list(list_size); - // append chain_id. from EIP-2930: chainId is defined to be an integer of arbitrary size. + // append `chain_id`. from EIP-2930: `chainId` is defined to be an integer of arbitrary size. stream.append(&chain_id); stream.append(&self.nonce); diff --git a/core/lib/mempool/src/tests.rs b/core/lib/mempool/src/tests.rs index cd595509ec56..38d4b62aafaf 100644 --- a/core/lib/mempool/src/tests.rs +++ b/core/lib/mempool/src/tests.rs @@ -45,7 +45,7 @@ fn basic_flow() { (account0, 3) ); assert_eq!(mempool.next_transaction(&L2TxFilter::default()), None); - // unclog second account and insert more txns + // unclog second account and insert more transactions mempool.insert( vec![gen_l2_tx(account1, Nonce(0)), gen_l2_tx(account0, Nonce(3))], HashMap::new(), diff --git a/core/lib/merkle_tree/src/consistency.rs b/core/lib/merkle_tree/src/consistency.rs index 659befbe048e..7b30e8b44e01 100644 --- a/core/lib/merkle_tree/src/consistency.rs +++ b/core/lib/merkle_tree/src/consistency.rs @@ -161,8 +161,8 @@ impl MerkleTree { is_leaf: child_ref.is_leaf, })?; - // Recursion here is OK; the tree isn't that deep (~8 nibbles for a tree with - // ~1B entries). + // Recursion here is OK; the tree isn't that deep (approximately 8 nibbles for a tree with + // approximately 1B entries). let child_hash = self.validate_node(&child, child_key, leaf_data)?; if child_hash == child_ref.hash { Ok(()) diff --git a/core/lib/merkle_tree/src/storage/tests.rs b/core/lib/merkle_tree/src/storage/tests.rs index a0c1ae4c9494..8bcaab710814 100644 --- a/core/lib/merkle_tree/src/storage/tests.rs +++ b/core/lib/merkle_tree/src/storage/tests.rs @@ -79,7 +79,7 @@ fn inserting_entries_in_empty_database() { fn assert_storage_with_2_keys(updater: &TreeUpdater) { // Check the internal nodes with a single child that should be created at keys - // '', 'd', 'de', ..., 'deadbeef'. + // `'', 'd', 'de', ..., 'deadbeef'`. let internal_node_nibbles = (0..8).map(|i| { let nibbles = Nibbles::new(&FIRST_KEY, i); let next_nibble = Nibbles::nibble(&FIRST_KEY, i); @@ -96,7 +96,7 @@ fn assert_storage_with_2_keys(updater: &TreeUpdater) { assert!(!child_ref.is_leaf); } - // Check the final internal node with 2 leaf children at 'deadbeef0'. + // Check the final internal node with 2 leaf children at `deadbeef0`. let nibbles = Nibbles::new(&FIRST_KEY, 9); let node = updater.patch_set.get(&nibbles).unwrap(); let Node::Internal(node) = node else { diff --git a/core/lib/merkle_tree/src/types/internal.rs b/core/lib/merkle_tree/src/types/internal.rs index 4351b7b8647e..06923bac33f2 100644 --- a/core/lib/merkle_tree/src/types/internal.rs +++ b/core/lib/merkle_tree/src/types/internal.rs @@ -314,12 +314,12 @@ impl NodeKey { #[allow(clippy::cast_possible_truncation)] pub(crate) fn to_db_key(self) -> Vec { let nibbles_byte_len = (self.nibbles.nibble_count + 1) / 2; - // ^ equivalent to ceil(self.nibble_count / 2) + // ^ equivalent to `ceil(self.nibble_count / 2)` let mut bytes = Vec::with_capacity(9 + nibbles_byte_len); // ^ 8 bytes for `version` + 1 byte for nibble count bytes.extend_from_slice(&self.version.to_be_bytes()); bytes.push(self.nibbles.nibble_count as u8); - // ^ conversion is safe: nibble_count <= 64 + // ^ conversion is safe: `nibble_count <= 64` bytes.extend_from_slice(&self.nibbles.bytes[..nibbles_byte_len]); bytes } @@ -568,7 +568,7 @@ mod tests { use super::*; // `U256` uses little-endian `u64` ordering; i.e., this is - // 0x_dead_beef_0000_0000_.._0000. + // `0x_dead_beef_0000_0000_.._0000.` const TEST_KEY: Key = U256([0, 0, 0, 0x_dead_beef_0000_0000]); #[test] diff --git a/core/lib/state/src/rocksdb/mod.rs b/core/lib/state/src/rocksdb/mod.rs index 30f1fe2779e9..6e0bb7233eec 100644 --- a/core/lib/state/src/rocksdb/mod.rs +++ b/core/lib/state/src/rocksdb/mod.rs @@ -676,7 +676,7 @@ mod tests { storage.update_from_postgres(&mut conn).await; assert_eq!(storage.l1_batch_number(), L1BatchNumber(2)); - // Check that enum indices are correct after syncing with postgres. + // Check that enum indices are correct after syncing with Postgres. for log in &storage_logs { let expected_index = enum_indices[&log.key.hashed_key()]; assert_eq!( diff --git a/core/lib/storage/src/db.rs b/core/lib/storage/src/db.rs index f6237d49950a..24502493a60d 100644 --- a/core/lib/storage/src/db.rs +++ b/core/lib/storage/src/db.rs @@ -529,7 +529,7 @@ impl RocksDB { .iterator_cf_opt(cf, options, IteratorMode::Start) .map(Result::unwrap) .fuse() - // ^ The rocksdb docs say that a raw iterator (which is used by the returned ordinary iterator) + // ^ The RocksDB docs say that a raw iterator (which is used by the returned ordinary iterator) // can become invalid "when it reaches the end of its defined range, or when it encounters an error." // We panic on RocksDB errors elsewhere and fuse it to prevent polling after the end of the range. // Thus, `unwrap()` should be safe. diff --git a/core/lib/types/src/commitment.rs b/core/lib/types/src/commitment.rs index 2d7dac0a34b6..e11bdbdc6487 100644 --- a/core/lib/types/src/commitment.rs +++ b/core/lib/types/src/commitment.rs @@ -255,7 +255,7 @@ impl L1BatchWithMetadata { res.extend(l2_to_l1_log.0.to_bytes()); } - // Process and Pack Msgs + // Process and Pack Messages res.extend((self.header.l2_to_l1_messages.len() as u32).to_be_bytes()); for msg in &self.header.l2_to_l1_messages { res.extend((msg.len() as u32).to_be_bytes()); @@ -347,7 +347,7 @@ struct L1BatchAuxiliaryOutput { l2_l1_logs_merkle_root: H256, // Once cut over to boojum, these fields are no longer required as their values - // are covered by state_diffs_compressed and its hash. + // are covered by `state_diffs_compressed` and its hash. // Task to remove: PLA-640 initial_writes_compressed: Vec, initial_writes_hash: H256, @@ -601,7 +601,7 @@ impl L1BatchCommitment { last_leaf_index: rollup_last_leaf_index, root_hash: rollup_root_hash, }, - // Despite the fact that zk_porter is not available we have to add params about it. + // Despite the fact that `zk_porter` is not available we have to add params about it. RootState { last_leaf_index: 0, root_hash: H256::zero(), diff --git a/core/lib/types/src/eth_sender.rs b/core/lib/types/src/eth_sender.rs index 6c8d268888e2..7778d8252080 100644 --- a/core/lib/types/src/eth_sender.rs +++ b/core/lib/types/src/eth_sender.rs @@ -13,7 +13,7 @@ pub struct EthTx { impl std::fmt::Debug for EthTx { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // Do not print raw_tx + // Do not print `raw_tx` f.debug_struct("EthTx") .field("id", &self.id) .field("nonce", &self.nonce) diff --git a/core/lib/types/src/event.rs b/core/lib/types/src/event.rs index 01561912cceb..dc4bcdc6045b 100644 --- a/core/lib/types/src/event.rs +++ b/core/lib/types/src/event.rs @@ -204,7 +204,7 @@ fn extract_added_token_info_from_addresses( .collect() } -// moved from RuntimeContext +// moved from `RuntimeContext` // Extracts all the "long" L2->L1 messages that were submitted by the // L1Messenger contract pub fn extract_long_l2_to_l1_messages(all_generated_events: &[VmEvent]) -> Vec> { @@ -226,8 +226,8 @@ pub fn extract_long_l2_to_l1_messages(all_generated_events: &[VmEvent]) -> Vec Vec { @@ -370,9 +370,9 @@ mod tests { value.to_big_endian(&mut val_arr); let tokens = vec![ - /*l2ShardId*/ Token::Uint(U256::from(0)), - /*isService*/ Token::Bool(true), - /*txNumberInBlock*/ Token::Uint(tx_number), + /*`l2ShardId`*/ Token::Uint(U256::from(0)), + /*`isService`*/ Token::Bool(true), + /*`txNumberInBlock`*/ Token::Uint(tx_number), /*sender*/ Token::Address(sender), /*key*/ Token::FixedBytes(key_arr.to_vec()), /*value*/ Token::FixedBytes(val_arr.to_vec()), diff --git a/core/lib/types/src/l1/mod.rs b/core/lib/types/src/l1/mod.rs index 16ce192bf8b3..a37f535cfd16 100644 --- a/core/lib/types/src/l1/mod.rs +++ b/core/lib/types/src/l1/mod.rs @@ -199,11 +199,11 @@ impl TryFrom for L1Tx { fn try_from(event: Log) -> Result { // TODO: refactor according to tx type let transaction_param_type = ParamType::Tuple(vec![ - ParamType::Uint(8), // txType + ParamType::Uint(8), // `txType` ParamType::Address, // sender ParamType::Address, // to ParamType::Uint(256), // gasLimit - ParamType::Uint(256), // gasPerPubdataLimit + ParamType::Uint(256), // `gasPerPubdataLimit` ParamType::Uint(256), // maxFeePerGas ParamType::Uint(256), // maxPriorityFeePerGas ParamType::Address, // paymaster @@ -214,7 +214,7 @@ impl TryFrom for L1Tx { ParamType::Bytes, // signature ParamType::Array(Box::new(ParamType::Uint(256))), // factory deps ParamType::Bytes, // paymaster input - ParamType::Bytes, // reservedDynamic + ParamType::Bytes, // `reservedDynamic` ]); let mut dec_ev = decode( @@ -302,7 +302,7 @@ impl TryFrom for L1Tx { let signature = transaction.remove(0).into_bytes().unwrap(); assert_eq!(signature.len(), 0); - // TODO (SMA-1621): check that reservedDynamic are constructed correctly. + // TODO (SMA-1621): check that `reservedDynamic` are constructed correctly. let _factory_deps_hashes = transaction.remove(0).into_array().unwrap(); let _paymaster_input = transaction.remove(0).into_bytes().unwrap(); let _reserved_dynamic = transaction.remove(0).into_bytes().unwrap(); diff --git a/core/lib/types/src/l2/mod.rs b/core/lib/types/src/l2/mod.rs index 61a505909b2b..6827f0421912 100644 --- a/core/lib/types/src/l2/mod.rs +++ b/core/lib/types/src/l2/mod.rs @@ -29,7 +29,7 @@ pub enum TransactionType { EIP2930Transaction = 1, EIP1559Transaction = 2, - // Eip 712 transaction with additional fields specified for zksync + // EIP 712 transaction with additional fields specified for zkSync EIP712Transaction = EIP_712_TX_TYPE as u32, PriorityOpTransaction = PRIORITY_OPERATION_L2_TX_TYPE as u32, ProtocolUpgradeTransaction = PROTOCOL_UPGRADE_TX_TYPE as u32, @@ -289,7 +289,7 @@ impl L2Tx { fn signature_to_vrs(signature: &[u8], tx_type: u32) -> (Option, Option, Option) { let signature = if tx_type == LEGACY_TX_TYPE as u32 { // Note that we use `deserialize_packed_no_v_check` here, because we want to preserve the original `v` value. - // This is needed due to inconsistent behaviour on Ethereum where the `v` value is >= 27 for legacy transactions + // This is needed due to inconsistent behavior on Ethereum where the `v` value is >= 27 for legacy transactions // and is either 0 or 1 for other ones. PackedEthSignature::deserialize_packed_no_v_check(signature) } else { diff --git a/core/lib/types/src/proofs.rs b/core/lib/types/src/proofs.rs index a23b0f44416a..392369f645db 100644 --- a/core/lib/types/src/proofs.rs +++ b/core/lib/types/src/proofs.rs @@ -36,7 +36,7 @@ pub struct StorageLogMetadata { pub merkle_paths: Vec<[u8; HASH_LEN]>, pub leaf_hashed_key: U256, pub leaf_enumeration_index: u64, - // **NB.** For compatibility reasons, `#[serde_as(as = "Bytes")]` attrs are not added below. + // **NB.** For compatibility reasons, `#[serde_as(as = "Bytes")]` attributes are not added below. pub value_written: [u8; HASH_LEN], pub value_read: [u8; HASH_LEN], } diff --git a/core/lib/types/src/protocol_version.rs b/core/lib/types/src/protocol_version.rs index dabb9d127545..75333880ef4b 100644 --- a/core/lib/types/src/protocol_version.rs +++ b/core/lib/types/src/protocol_version.rs @@ -245,11 +245,11 @@ impl TryFrom for ProtocolUpgrade { }; let transaction_param_type = ParamType::Tuple(vec![ - ParamType::Uint(256), // txType + ParamType::Uint(256), // `txType` ParamType::Uint(256), // sender ParamType::Uint(256), // to ParamType::Uint(256), // gasLimit - ParamType::Uint(256), // gasPerPubdataLimit + ParamType::Uint(256), // `gasPerPubdataLimit` ParamType::Uint(256), // maxFeePerGas ParamType::Uint(256), // maxPriorityFeePerGas ParamType::Uint(256), // paymaster @@ -260,7 +260,7 @@ impl TryFrom for ProtocolUpgrade { ParamType::Bytes, // signature ParamType::Array(Box::new(ParamType::Uint(256))), // factory deps ParamType::Bytes, // paymaster input - ParamType::Bytes, // reservedDynamic + ParamType::Bytes, // `reservedDynamic` ]); let verifier_params_type = ParamType::Tuple(vec![ ParamType::FixedBytes(32), @@ -355,7 +355,7 @@ impl TryFrom for ProtocolUpgrade { let paymaster_input = transaction.remove(0).into_bytes().unwrap(); assert_eq!(paymaster_input.len(), 0); - // TODO (SMA-1621): check that reservedDynamic are constructed correctly. + // TODO (SMA-1621): check that `reservedDynamic` are constructed correctly. let reserved_dynamic = transaction.remove(0).into_bytes().unwrap(); assert_eq!(reserved_dynamic.len(), 0); diff --git a/core/lib/types/src/snapshots.rs b/core/lib/types/src/snapshots.rs index b71a8e34b4be..19f818bb5d1e 100644 --- a/core/lib/types/src/snapshots.rs +++ b/core/lib/types/src/snapshots.rs @@ -36,7 +36,7 @@ impl SnapshotMetadata { } /// Snapshot data returned by using JSON-RPC API. -/// Contains all data not contained in factory deps / storage logs files to perform restore process. +/// Contains all data not contained in `factory_deps` / `storage_logs` files to perform restore process. #[derive(Debug, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct SnapshotHeader { @@ -52,7 +52,7 @@ pub struct SnapshotHeader { #[serde(rename_all = "camelCase")] pub struct SnapshotStorageLogsChunkMetadata { pub chunk_id: u64, - // can be either be a file available under http(s) or local filesystem path + // can be either be a file available under HTTP(s) or local filesystem path pub filepath: String, } diff --git a/core/lib/types/src/storage/log.rs b/core/lib/types/src/storage/log.rs index 11756f7175ae..a64bbb502207 100644 --- a/core/lib/types/src/storage/log.rs +++ b/core/lib/types/src/storage/log.rs @@ -7,7 +7,7 @@ use zksync_utils::u256_to_h256; use crate::{StorageKey, StorageValue, U256}; -// TODO (SMA-1269): Refactor StorageLog/StorageLogQuery and StorageLogKind/StorageLongQueryType. +// TODO (SMA-1269): Refactor `StorageLog/StorageLogQuery and StorageLogKind/StorageLongQueryType`. #[derive(Debug, Clone, Copy, PartialEq, Serialize, Deserialize)] pub enum StorageLogKind { Read, diff --git a/core/lib/types/src/storage/mod.rs b/core/lib/types/src/storage/mod.rs index 46b98575f124..54694f63c504 100644 --- a/core/lib/types/src/storage/mod.rs +++ b/core/lib/types/src/storage/mod.rs @@ -67,7 +67,7 @@ fn get_address_mapping_key(address: &Address, position: H256) -> H256 { pub fn get_nonce_key(account: &Address) -> StorageKey { let nonce_manager = AccountTreeId::new(NONCE_HOLDER_ADDRESS); - // The `minNonce` (used as nonce for EOAs) is stored in a mapping inside the NONCE_HOLDER system contract + // The `minNonce` (used as nonce for EOAs) is stored in a mapping inside the `NONCE_HOLDER` system contract let key = get_address_mapping_key(account, H256::zero()); StorageKey::new(nonce_manager, key) diff --git a/core/lib/types/src/system_contracts.rs b/core/lib/types/src/system_contracts.rs index 7cd4107ad41e..464a562b9272 100644 --- a/core/lib/types/src/system_contracts.rs +++ b/core/lib/types/src/system_contracts.rs @@ -16,9 +16,9 @@ use crate::{ SHA256_PRECOMPILE_ADDRESS, SYSTEM_CONTEXT_ADDRESS, }; -// Note, that in the NONCE_HOLDER_ADDRESS's storage the nonces of accounts +// Note, that in the `NONCE_HOLDER_ADDRESS` storage the nonces of accounts // are stored in the following form: -// 2^128 * deployment_nonce + tx_nonce, +// `2^128 * deployment_nonce + tx_nonce`, // where `tx_nonce` should be number of transactions, the account has processed // and the `deployment_nonce` should be the number of contracts. pub const TX_NONCE_INCREMENT: U256 = U256([1, 0, 0, 0]); // 1 diff --git a/core/lib/types/src/transaction_request.rs b/core/lib/types/src/transaction_request.rs index e66c2495afe3..8f14f44f87b5 100644 --- a/core/lib/types/src/transaction_request.rs +++ b/core/lib/types/src/transaction_request.rs @@ -442,7 +442,7 @@ impl TransactionRequest { match self.transaction_type { // EIP-2930 (0x01) Some(x) if x == EIP_2930_TX_TYPE.into() => { - // rlp_opt(rlp, &self.chain_id); + // `rlp_opt(rlp, &self.chain_id);` rlp.append(&chain_id); rlp.append(&self.nonce); rlp.append(&self.gas_price); @@ -454,7 +454,7 @@ impl TransactionRequest { } // EIP-1559 (0x02) Some(x) if x == EIP_1559_TX_TYPE.into() => { - // rlp_opt(rlp, &self.chain_id); + // `rlp_opt(rlp, &self.chain_id);` rlp.append(&chain_id); rlp.append(&self.nonce); rlp_opt(rlp, &self.max_priority_fee_per_gas); @@ -821,7 +821,7 @@ impl L2Tx { /// Ensures that encoded transaction size is not greater than `max_tx_size`. fn check_encoded_size(&self, max_tx_size: usize) -> Result<(), SerializationTransactionError> { - // since abi_encoding_len returns 32-byte words multiplication on 32 is needed + // since `abi_encoding_len` returns 32-byte words multiplication on 32 is needed let tx_size = self.abi_encoding_len() * 32; if tx_size > max_tx_size { return Err(SerializationTransactionError::OversizedData( @@ -890,7 +890,7 @@ impl TryFrom for L1Tx { let total_needed_eth = tx.execute.value + tx.common_data.fee.max_fee_per_gas * tx.common_data.fee.gas_limit; - // Note, that we do not set refund_recipient here, to keep it explicitly 0, + // Note, that we do not set `refund_recipient` here, to keep it explicitly 0, // so that during fee estimation it is taken into account that the refund recipient may be a different address let common_data = L1TxCommonData { sender: tx.common_data.initiator_address, @@ -1396,7 +1396,7 @@ mod tests { let random_tx_max_size = 1_000_000; // bytes let private_key = H256::random(); let address = PackedEthSignature::address_from_private_key(&private_key).unwrap(); - // choose some number that devides on 8 and is > 1_000_000 + // choose some number that divides on 8 and is `> 1_000_000` let factory_dep = vec![2u8; 1600000]; let factory_deps: Vec> = factory_dep.chunks(32).map(|s| s.into()).collect(); let mut tx = TransactionRequest { diff --git a/core/lib/types/src/tx/execute.rs b/core/lib/types/src/tx/execute.rs index 50340230cb9e..21f0b401cce2 100644 --- a/core/lib/types/src/tx/execute.rs +++ b/core/lib/types/src/tx/execute.rs @@ -31,7 +31,7 @@ impl EIP712TypedStructure for Execute { builder.add_member("data", &self.calldata.as_slice()); // Factory deps are not included into the transaction signature, since they are parsed from the // transaction metadata. - // Note that for the deploy transactions all the dependencies are implicitly included into the "calldataHash" + // Note that for the deploy transactions all the dependencies are implicitly included into the `calldataHash` // field, because the deps are referenced in the bytecode of the "main" contract bytecode. } } diff --git a/core/lib/types/src/tx/primitives/eip712_signature/struct_builder.rs b/core/lib/types/src/tx/primitives/eip712_signature/struct_builder.rs index 2093042b9f7a..1b3260993ea9 100644 --- a/core/lib/types/src/tx/primitives/eip712_signature/struct_builder.rs +++ b/core/lib/types/src/tx/primitives/eip712_signature/struct_builder.rs @@ -87,7 +87,7 @@ pub(crate) struct EncodeBuilder { impl EncodeBuilder { /// Returns the concatenation of the encoded member values in the order that they appear in the type. pub fn encode_data(&self) -> Vec { - // encodeData(s : 𝕊) = enc(value₁) ‖ enc(value₂) ‖ … ‖ enc(valueₙ). + // `encodeData(s : 𝕊) = enc(value₁) ‖ enc(value₂) ‖ … ‖ enc(valueₙ).` self.members.iter().map(|(_, data)| *data).collect() } diff --git a/core/lib/types/src/tx/primitives/eip712_signature/typed_structure.rs b/core/lib/types/src/tx/primitives/eip712_signature/typed_structure.rs index daf1c9698ee1..421944e5d46f 100644 --- a/core/lib/types/src/tx/primitives/eip712_signature/typed_structure.rs +++ b/core/lib/types/src/tx/primitives/eip712_signature/typed_structure.rs @@ -123,7 +123,7 @@ pub trait EIP712TypedStructure: Serialize { } fn hash_struct(&self) -> H256 { - // hashStruct(s : 𝕊) = keccak256(keccak256(encodeType(typeOf(s))) ‖ encodeData(s)). + // `hashStruct(s : 𝕊) = keccak256(keccak256(encodeType(typeOf(s))) ‖ encodeData(s)).` let type_hash = { let encode_type = self.encode_type(); keccak256(encode_type.as_bytes()) diff --git a/core/lib/types/src/utils.rs b/core/lib/types/src/utils.rs index 7828f2d1262b..b13887000cb2 100644 --- a/core/lib/types/src/utils.rs +++ b/core/lib/types/src/utils.rs @@ -48,7 +48,7 @@ pub fn storage_key_for_standard_token_balance( token_contract: AccountTreeId, address: &Address, ) -> StorageKey { - // We have different implementation of the standard erc20 contract and native + // We have different implementation of the standard ERC20 contract and native // eth contract. The key for the balance is different for each. let key = if token_contract.address() == &L2_ETH_TOKEN_ADDRESS { key_for_eth_balance(address) diff --git a/core/lib/utils/src/bytecode.rs b/core/lib/utils/src/bytecode.rs index c533642d240f..f9554c6f72bd 100644 --- a/core/lib/utils/src/bytecode.rs +++ b/core/lib/utils/src/bytecode.rs @@ -58,7 +58,7 @@ pub fn compress_bytecode(code: &[u8]) -> Result, FailedToCompressBytecod return Err(FailedToCompressBytecodeError::DictionaryOverflow); } - // Fill the dictionary with the pmost popular chunks. + // Fill the dictionary with the most popular chunks. // The most popular chunks will be encoded with the smallest indexes, so that // the 255 most popular chunks will be encoded with one zero byte. // And the encoded data will be filled with more zeros, so @@ -214,9 +214,9 @@ mod test { let example_code = hex::decode("0000000000000000111111111111111111111111111111112222222222222222") .unwrap(); - // The size of the dictionary should be 0x0003 - // The dictionary itself should put the most common chunk first, i.e. 0x1111111111111111 - // Then, the ordering does not matter, but the algorithm will return the one with the highest position, i.e. 0x2222222222222222 + // The size of the dictionary should be `0x0003` + // The dictionary itself should put the most common chunk first, i.e. `0x1111111111111111` + // Then, the ordering does not matter, but the algorithm will return the one with the highest position, i.e. `0x2222222222222222` let expected_encoding = hex::decode("00031111111111111111222222222222222200000000000000000002000000000001") .unwrap(); diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 1dfdd03e5231..0973f8399328 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -849,4 +849,10 @@ satisfiability demultiplex precompile statekeeper +crаsh +protobuf +L1Tx +EIP +DecommittmentProcessor +decommitment tokenized From 70c3febbf0445d2e0c22a942eaf643828aee045d Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Sat, 6 Jan 2024 01:40:20 -0600 Subject: [PATCH 24/49] feat: fix spelling in dev comments in `core/lib/zksync_core` - continued (#685) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Series of PRs: This is a part in a series of PRs aimed at enhancing spelling accuracy in this repository. See:** - [ ] https://github.com/matter-labs/zksync-era/pull/681 - [ ] https://github.com/matter-labs/zksync-era/pull/682 - [ ] https://github.com/matter-labs/zksync-era/pull/683 - [ ] https://github.com/matter-labs/zksync-era/pull/684 Once merged, a final PR will enable the `dev_comments: true` in the cargo-spellcheck config file. ## What ❔ - - **Spelling Corrections**: This PR focuses on rectifying spelling errors found in the developer comments within core libraries: `core/lib/zksync_core` - Updates dictionary ## Why ❔ - **Consistency and Readability:** The addressed directories contain numerous spelling inaccuracies. Correcting these enhances the readability and consistency of our documentation, ensuring clear understanding for current and future developer ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .../src/api_server/contract_verification/mod.rs | 2 +- .../src/api_server/execution_sandbox/apply.rs | 2 +- .../src/api_server/execution_sandbox/execute.rs | 2 +- .../zksync_core/src/api_server/healthcheck.rs | 2 +- core/lib/zksync_core/src/api_server/tree/mod.rs | 2 +- .../zksync_core/src/api_server/tx_sender/mod.rs | 6 +++--- core/lib/zksync_core/src/consensus/storage.rs | 2 +- core/lib/zksync_core/src/consensus/testonly.rs | 2 +- core/lib/zksync_core/src/consensus/tests.rs | 2 +- .../src/eth_sender/eth_tx_aggregator.rs | 4 ++-- .../zksync_core/src/eth_sender/eth_tx_manager.rs | 16 ++++++++-------- core/lib/zksync_core/src/eth_sender/tests.rs | 6 +++--- .../event_processors/governance_upgrades.rs | 2 +- .../src/eth_watch/event_processors/upgrades.rs | 2 +- .../lib/zksync_core/src/gas_tracker/constants.rs | 4 ++-- .../src/house_keeper/fri_prover_queue_monitor.rs | 4 ++-- .../src/l1_gas_price/gas_adjuster/mod.rs | 10 +++++----- .../src/metadata_calculator/helpers.rs | 2 +- core/lib/zksync_core/src/reorg_detector/mod.rs | 4 ++-- .../src/state_keeper/batch_executor/mod.rs | 10 +++++----- .../zksync_core/src/state_keeper/io/tests/mod.rs | 6 +++--- .../seal_criteria/criteria/pubdata_bytes.rs | 2 +- .../zksync_core/src/state_keeper/tests/mod.rs | 2 +- .../src/sync_layer/batch_status_updater.rs | 2 +- core/lib/zksync_core/src/sync_layer/fetcher.rs | 2 +- .../src/sync_layer/gossip/buffered/mod.rs | 6 +++--- .../src/sync_layer/gossip/storage/mod.rs | 4 ++-- .../zksync_core/src/sync_layer/sync_action.rs | 4 ++-- .../lib/zksync_core/src/sync_layer/sync_state.rs | 10 +++++----- core/lib/zksync_core/src/sync_layer/tests.rs | 2 +- core/lib/zksync_core/src/temp_config_store.rs | 2 +- spellcheck/era.dic | 3 +++ 32 files changed, 67 insertions(+), 64 deletions(-) diff --git a/core/lib/zksync_core/src/api_server/contract_verification/mod.rs b/core/lib/zksync_core/src/api_server/contract_verification/mod.rs index a1bf980a49a1..3c10b91ff713 100644 --- a/core/lib/zksync_core/src/api_server/contract_verification/mod.rs +++ b/core/lib/zksync_core/src/api_server/contract_verification/mod.rs @@ -26,7 +26,7 @@ fn start_server(api: RestApi, bind_to: SocketAddr, threads: usize) -> Server { .allow_any_method(), ) .service(api.into_scope()) - // Endpoint needed for js isReachable + // Endpoint needed for js `isReachable` .route( "/favicon.ico", web::get().to(|| async { HttpResponse::Ok().finish() }), diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs index 13b215d047f4..33e8e8f8bfc7 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs @@ -107,7 +107,7 @@ pub(super) fn apply_vm_in_sandbox( } else if current_l2_block_info.l2_block_number == 0 { // Special case: // - For environments, where genesis block was created before virtual block upgrade it doesn't matter what we put here. - // - Otherwise, we need to put actual values here. We cannot create next l2 block with block_number=0 and max_virtual_blocks_to_create=0 + // - Otherwise, we need to put actual values here. We cannot create next L2 block with block_number=0 and `max_virtual_blocks_to_create=0` // because of SystemContext requirements. But, due to intrinsics of SystemContext, block.number still will be resolved to 0. L2BlockEnv { number: 1, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs index b915866ebe64..7e65cd9a8931 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs @@ -92,7 +92,7 @@ pub(crate) async fn execute_tx_eth_call( // Protection against infinite-loop eth_calls and alike: // limiting the amount of gas the call can use. - // We can't use BLOCK_ERGS_LIMIT here since the VM itself has some overhead. + // We can't use `BLOCK_ERGS_LIMIT` here since the VM itself has some overhead. tx.common_data.fee.gas_limit = ETH_CALL_GAS_LIMIT.into(); let (vm_result, _) = execute_tx_in_sandbox( vm_permit, diff --git a/core/lib/zksync_core/src/api_server/healthcheck.rs b/core/lib/zksync_core/src/api_server/healthcheck.rs index 58444c30dc94..7010d29fb4b5 100644 --- a/core/lib/zksync_core/src/api_server/healthcheck.rs +++ b/core/lib/zksync_core/src/api_server/healthcheck.rs @@ -74,7 +74,7 @@ impl HealthCheckHandle { pub async fn stop(self) { // Paradoxically, `hyper` server is quite slow to shut down if it isn't queried during shutdown: - // https://github.com/hyperium/hyper/issues/3188. It is thus recommended to set a timeout for shutdown. + // . It is thus recommended to set a timeout for shutdown. const GRACEFUL_SHUTDOWN_WAIT: Duration = Duration::from_secs(10); self.stop_sender.send(true).ok(); diff --git a/core/lib/zksync_core/src/api_server/tree/mod.rs b/core/lib/zksync_core/src/api_server/tree/mod.rs index 00a5fd285546..a6b6d51fbaae 100644 --- a/core/lib/zksync_core/src/api_server/tree/mod.rs +++ b/core/lib/zksync_core/src/api_server/tree/mod.rs @@ -73,7 +73,7 @@ impl IntoResponse for TreeApiError { } }; - // Loosely conforms to HTTP Problem Details RFC: https://datatracker.ietf.org/doc/html/rfc7807 + // Loosely conforms to HTTP Problem Details RFC: let body = serde_json::json!({ "type": "/errors#l1-batch-not-found", "title": title, 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 d4d6d54dd9ee..d79daf8f2575 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 @@ -646,7 +646,7 @@ impl TxSender { let current_l1_gas_price = ((effective_gas_price as f64) * self.0.sender_config.gas_price_scale_factor) as u64; - // In order for execution to pass smoothly, we need to ensure that block's required gasPerPubdata will be + // In order for execution to pass smoothly, we need to ensure that block's required `gasPerPubdata` will be // <= to the one in the transaction itself. adjust_l1_gas_price_for_tx( current_l1_gas_price, @@ -738,7 +738,7 @@ impl TxSender { }; // We are using binary search to find the minimal values of gas_limit under which - // the transaction succeedes + // the transaction succeeds let mut lower_bound = 0; let mut upper_bound = MAX_L2_TX_GAS_LIMIT as u32; let tx_id = format!( @@ -756,7 +756,7 @@ impl TxSender { while lower_bound + acceptable_overestimation < upper_bound { let mid = (lower_bound + upper_bound) / 2; // There is no way to distinct between errors due to out of gas - // or normal exeuction errors, so we just hope that increasing the + // or normal execution errors, so we just hope that increasing the // gas limit will make the transaction successful let iteration_started_at = Instant::now(); let try_gas_limit = gas_for_bytecodes_pubdata + mid; diff --git a/core/lib/zksync_core/src/consensus/storage.rs b/core/lib/zksync_core/src/consensus/storage.rs index d0feecf77949..f47fc75b6480 100644 --- a/core/lib/zksync_core/src/consensus/storage.rs +++ b/core/lib/zksync_core/src/consensus/storage.rs @@ -214,7 +214,7 @@ impl SignedBlockStore { genesis: &validator::FinalBlock, operator_address: Address, ) -> anyhow::Result { - // Ensure that genesis block has consensus field set in postgres. + // Ensure that genesis block has consensus field set in Postgres. let head = { let mut storage = storage(ctx, &pool).await.wrap("storage()")?; storage diff --git a/core/lib/zksync_core/src/consensus/testonly.rs b/core/lib/zksync_core/src/consensus/testonly.rs index 13bae6c9c322..b32e495eef72 100644 --- a/core/lib/zksync_core/src/consensus/testonly.rs +++ b/core/lib/zksync_core/src/consensus/testonly.rs @@ -226,7 +226,7 @@ impl StateKeeperHandle { pub async fn push_random_blocks(&mut self, rng: &mut impl Rng, count: usize) { for _ in 0..count { // 20% chance to seal an L1 batch. - // seal_batch() also produces a (fictive) block. + // `seal_batch()` also produces a (fictive) block. if rng.gen_range(0..100) < 20 { self.seal_batch().await; } else { diff --git a/core/lib/zksync_core/src/consensus/tests.rs b/core/lib/zksync_core/src/consensus/tests.rs index be8ff9bacb23..25ac048ae627 100644 --- a/core/lib/zksync_core/src/consensus/tests.rs +++ b/core/lib/zksync_core/src/consensus/tests.rs @@ -8,7 +8,7 @@ use super::*; // In the current implementation, consensus certificates are created asynchronously // for the miniblocks constructed by the StateKeeper. This means that consensus actor -// is effectively just backfilling the consensus certificates for the miniblocks in storage. +// is effectively just back filling the consensus certificates for the miniblocks in storage. #[tokio::test(flavor = "multi_thread")] async fn test_backfill() { const OPERATOR_ADDRESS: Address = Address::repeat_byte(17); diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs index b6413509641b..6211b6363849 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_aggregator.rs @@ -182,8 +182,8 @@ impl EthTxAggregator { ] } - // The role of the method below is to detokenize multicall call's result, which is actually a token. - // This token is an array of tuples like (bool, bytes), that contain the status and result for each contract call. + // The role of the method below is to de-tokenize multicall call's result, which is actually a token. + // This token is an array of tuples like `(bool, bytes)`, that contain the status and result for each contract call. pub(super) fn parse_multicall_data( &self, token: Token, diff --git a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs index ae5c8a128298..99d46e31068c 100644 --- a/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs +++ b/core/lib/zksync_core/src/eth_sender/eth_tx_manager.rs @@ -174,7 +174,7 @@ impl EthTxManager { return Err(ETHSenderError::from(Error::from(Web3Error::Internal))); } - // Increase `priority_fee_per_gas` by at least 20% to prevent "replacement transaction underpriced" error. + // Increase `priority_fee_per_gas` by at least 20% to prevent "replacement transaction under-priced" error. Ok((previous_priority_fee + (previous_priority_fee / 5) + 1) .max(self.gas_adjuster.get_priority_fee())) } @@ -303,7 +303,7 @@ impl EthTxManager { Ok(L1BlockNumbers { finalized, latest }) } - // Monitors the inflight transactions, marks mined ones as confirmed, + // Monitors the in-flight transactions, marks mined ones as confirmed, // returns the one that has to be resent (if there is one). pub(super) async fn monitor_inflight_transactions( &mut self, @@ -333,7 +333,7 @@ impl EthTxManager { // If the `operator_nonce.latest` <= `tx.nonce`, this means // that `tx` is not mined and we should resend it. - // We only resend the first unmined transaction. + // We only resend the first un-mined transaction. if operator_nonce.latest <= tx.nonce { // None means txs hasn't been sent yet let first_sent_at_block = storage @@ -367,9 +367,9 @@ impl EthTxManager { } None => { // The nonce has increased but we did not find the receipt. - // This is an error because such a big reorg may cause transactions that were + // This is an error because such a big re-org may cause transactions that were // previously recorded as confirmed to become pending again and we have to - // make sure it's not the case - otherwise eth_sender may not work properly. + // make sure it's not the case - otherwise `eth_sender` may not work properly. tracing::error!( "Possible block reorgs: finalized nonce increase detected, but no tx receipt found for tx {:?}", &tx @@ -410,7 +410,7 @@ impl EthTxManager { ) { for tx in storage.eth_sender_dal().get_unsent_txs().await.unwrap() { // Check already sent txs not marked as sent and mark them as sent. - // The common reason for this behaviour is that we sent tx and stop the server + // The common reason for this behavior is that we sent tx and stop the server // before updating the database let tx_status = self.get_tx_status(tx.tx_hash).await; @@ -561,8 +561,8 @@ impl EthTxManager { self.send_unsent_txs(&mut storage, l1_block_numbers).await; } - // It's mandatory to set last_known_l1_block to zero, otherwise the first iteration - // will never check inflight txs status + // It's mandatory to set `last_known_l1_block` to zero, otherwise the first iteration + // will never check in-flight txs status let mut last_known_l1_block = L1BlockNumber(0); loop { let mut storage = pool.access_storage_tagged("eth_sender").await.unwrap(); diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index ad9764ce76e1..d7e64a8928bb 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -214,7 +214,7 @@ async fn confirm_many() -> anyhow::Result<()> { Ok(()) } -// Tests that we resend first unmined transaction every block with an increased gas price. +// Tests that we resend first un-mined transaction every block with an increased gas price. #[tokio::test] async fn resend_each_block() -> anyhow::Result<()> { let connection_pool = ConnectionPool::test_pool().await; @@ -378,7 +378,7 @@ async fn dont_resend_already_mined() -> anyhow::Result<()> { ) .await?; - // check that transaction is still considered inflight + // check that transaction is still considered in-flight assert_eq!( tester .storage() @@ -449,7 +449,7 @@ async fn three_scenarios() -> anyhow::Result<()> { .await? .expect("we should be trying to resend the last tx"); - // check that last 2 transactions are still considered inflight + // check that last 2 transactions are still considered in-flight assert_eq!( tester .storage() diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs b/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs index 9bec83d26833..6008f4a05e96 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/governance_upgrades.rs @@ -66,7 +66,7 @@ impl EventProcessor for GovernanceUpgradesEventProcessor { ); continue; }; - // Scheduler VK is not present in proposal event. It is hardcoded in verifier contract. + // Scheduler VK is not present in proposal event. It is hard coded in verifier contract. let scheduler_vk_hash = if let Some(address) = upgrade.verifier_address { Some(client.scheduler_vk_hash(address).await?) } else { diff --git a/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs b/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs index a46b3f21cb53..e7f906cdf070 100644 --- a/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs +++ b/core/lib/zksync_core/src/eth_watch/event_processors/upgrades.rs @@ -43,7 +43,7 @@ impl EventProcessor for UpgradesEventProcessor { { let upgrade = ProtocolUpgrade::try_from(event) .map_err(|err| Error::LogParse(format!("{:?}", err)))?; - // Scheduler VK is not present in proposal event. It is hardcoded in verifier contract. + // Scheduler VK is not present in proposal event. It is hard coded in verifier contract. let scheduler_vk_hash = if let Some(address) = upgrade.verifier_address { Some(client.scheduler_vk_hash(address).await?) } else { diff --git a/core/lib/zksync_core/src/gas_tracker/constants.rs b/core/lib/zksync_core/src/gas_tracker/constants.rs index 4eb9475cb0f5..00c96486a72e 100644 --- a/core/lib/zksync_core/src/gas_tracker/constants.rs +++ b/core/lib/zksync_core/src/gas_tracker/constants.rs @@ -1,5 +1,5 @@ -// Currently, every AGGR_* cost is overestimated, -// so there are safety margins around 100_000 -- 200_000 +// Currently, every `AGGR_* cost` is overestimated, +// so there are safety margins around `100_000 -- 200_000` pub(super) const AGGR_L1_BATCH_COMMIT_BASE_COST: u32 = 242_000; pub(super) const AGGR_L1_BATCH_PROVE_BASE_COST: u32 = 1_000_000; diff --git a/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs b/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs index 3ccf09bdf1e8..928c4cbce6d4 100644 --- a/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs +++ b/core/lib/zksync_core/src/house_keeper/fri_prover_queue_monitor.rs @@ -39,11 +39,11 @@ impl PeriodicJob for FriProverStatsReporter { for ((circuit_id, aggregation_round), stats) in stats.into_iter() { // BEWARE, HERE BE DRAGONS. - // In database, the circuit_id stored is the circuit for which the aggregation is done, + // In database, the `circuit_id` stored is the circuit for which the aggregation is done, // not the circuit which is running. // There is a single node level aggregation circuit, which is circuit 2. // This can aggregate multiple leaf nodes (which may belong to different circuits). - // This reporting is a hacky forced way to use circuit_id 2 which will solve autoscalers. + // This reporting is a hacky forced way to use `circuit_id` 2 which will solve auto scalers. // A proper fix will be later provided to solve this at database level. let circuit_id = if aggregation_round == 2 { 2 diff --git a/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs b/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs index 0fbb52768436..2adcd852c323 100644 --- a/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs +++ b/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs @@ -141,7 +141,7 @@ impl L1TxParamsProvider for GasAdjuster { // Currently we use an exponential formula. // The alternative is a linear one: - // let scale_factor = a + b * time_in_mempool as f64; + // `let scale_factor = a + b * time_in_mempool as f64;` let scale_factor = a * b.powf(time_in_mempool as f64); let median = self.statistics.median(); METRICS.median_base_fee_per_gas.set(median); @@ -158,11 +158,11 @@ impl L1TxParamsProvider for GasAdjuster { // Priority fee is set to constant, sourced from config. // Reasoning behind this is the following: - // High priority_fee means high demand for block space, - // which means base_fee will increase, which means priority_fee + // High `priority_fee` means high demand for block space, + // which means `base_fee` will increase, which means `priority_fee` // will decrease. The EIP-1559 mechanism is designed such that - // base_fee will balance out priority_fee in such a way that - // priority_fee will be a small fraction of the overall fee. + // `base_fee` will balance out `priority_fee` in such a way that + // `priority_fee` will be a small fraction of the overall fee. fn get_priority_fee(&self) -> u64 { self.config.default_priority_fee_per_gas } diff --git a/core/lib/zksync_core/src/metadata_calculator/helpers.rs b/core/lib/zksync_core/src/metadata_calculator/helpers.rs index e0471cb238e9..563e643d7e11 100644 --- a/core/lib/zksync_core/src/metadata_calculator/helpers.rs +++ b/core/lib/zksync_core/src/metadata_calculator/helpers.rs @@ -319,7 +319,7 @@ pub(super) struct Delayer { delay_interval: Duration, // Notifies the tests about the next L1 batch number and tree root hash when the calculator // runs out of L1 batches to process. (Since RocksDB is exclusive, we cannot just create - // another instance to check these params on the test side without stopping the calc.) + // another instance to check these params on the test side without stopping the calculation.) #[cfg(test)] pub delay_notifier: mpsc::UnboundedSender<(L1BatchNumber, H256)>, } diff --git a/core/lib/zksync_core/src/reorg_detector/mod.rs b/core/lib/zksync_core/src/reorg_detector/mod.rs index f0ae656ce099..106202389ad3 100644 --- a/core/lib/zksync_core/src/reorg_detector/mod.rs +++ b/core/lib/zksync_core/src/reorg_detector/mod.rs @@ -267,12 +267,12 @@ impl ReorgDetector { let miniblock_hashes_match = self.miniblock_hashes_match(sealed_miniblock_number).await?; - // The only event that triggers reorg detection and node rollback is if the + // The only event that triggers re-org detection and node rollback is if the // hash mismatch at the same block height is detected, be it miniblocks or batches. // // In other cases either there is only a height mismatch which means that one of // the nodes needs to do catching up; however, it is not certain that there is actually - // a reorg taking place. + // a re-org taking place. if root_hashes_match && miniblock_hashes_match { self.block_updater .update_correct_block(sealed_miniblock_number, sealed_l1_batch_number); diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs index 2267792297fd..dc8a6b548b9f 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs @@ -325,7 +325,7 @@ impl BatchExecutor { }; resp.send((vm_block_result, witness_block_state)).unwrap(); - // storage_view cannot be accessed while borrowed by the VM, + // `storage_view` cannot be accessed while borrowed by the VM, // so this is the only point at which storage metrics can be obtained let metrics = storage_view.as_ref().borrow_mut().metrics(); EXECUTOR_METRICS.batch_storage_interaction_duration[&InteractionType::GetValue] @@ -433,8 +433,8 @@ impl BatchExecutor { } // Err when transaction is rejected. - // Ok(TxExecutionStatus::Success) when the transaction succeeded - // Ok(TxExecutionStatus::Failure) when the transaction failed. + // `Ok(TxExecutionStatus::Success)` when the transaction succeeded + // `Ok(TxExecutionStatus::Failure)` when the transaction failed. // Note that failed transactions are considered properly processed and are included in blocks fn execute_tx_in_vm( &self, @@ -451,8 +451,8 @@ impl BatchExecutor { // that will not be published (e.g. due to out of gas), we use the following scheme: // We try to execute the transaction with compressed bytecodes. // If it fails and the compressed bytecodes have not been published, - // it means that there is no sense in pollutting the space of compressed bytecodes, - // and so we reexecute the transaction, but without compressions. + // it means that there is no sense in polluting the space of compressed bytecodes, + // and so we re-execute the transaction, but without compression. // Saving the snapshot before executing vm.make_snapshot(); diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index dcbdac448275..2135187362c9 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -49,7 +49,7 @@ async fn test_filter_with_pending_batch() { tester.genesis(&connection_pool).await; - // Insert a sealed batch so there will be a prev_l1_batch_state_root. + // Insert a sealed batch so there will be a `prev_l1_batch_state_root`. // These gas values are random and don't matter for filter calculation as there will be a // pending batch the filter will be based off of. tester @@ -88,14 +88,14 @@ async fn test_filter_with_pending_batch() { assert_eq!(mempool.filter(), &want_filter); } -/// Ensure that MempoolIO.filter is modified correctly if there is no pending batch. +/// Ensure that `MempoolIO.filter` is modified correctly if there is no pending batch. #[tokio::test] async fn test_filter_with_no_pending_batch() { let connection_pool = ConnectionPool::test_pool().await; let tester = Tester::new(); tester.genesis(&connection_pool).await; - // Insert a sealed batch so there will be a prev_l1_batch_state_root. + // Insert a sealed batch so there will be a `prev_l1_batch_state_root`. // These gas values are random and don't matter for filter calculation. tester .insert_miniblock(&connection_pool, 1, 5, 55, 555) diff --git a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs index 61f30d724a70..ec778cdf0836 100644 --- a/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs +++ b/core/lib/zksync_core/src/state_keeper/seal_criteria/criteria/pubdata_bytes.rs @@ -26,7 +26,7 @@ impl SealCriterion for PubDataBytesCriterion { let block_size = block_data.execution_metrics.size() + block_data.writes_metrics.size(protocol_version); // For backward compatibility, we need to keep calculating the size of the pubdata based - // StorageDeduplication metrics. All vm versions + // `StorageDeduplication` metrics. All vm versions // after vm with virtual blocks will provide the size of the pubdata in the execution metrics. let tx_size = if tx_data.execution_metrics.pubdata_published == 0 { tx_data.execution_metrics.size() + tx_data.writes_metrics.size(protocol_version) diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index afe11b9367f3..7d0c5fbd93da 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -222,7 +222,7 @@ async fn sealed_by_gas() { let sealer = SequencerSealer::with_sealers(config, vec![Box::new(GasCriterion)]); let l1_gas_per_tx = BlockGasCount { - commit: 1, // Both txs together with block_base_cost would bring it over the block 31_001 commit bound. + commit: 1, // Both txs together with `block_base_cost` would bring it over the block `31_001` commit bound. prove: 0, execute: 0, }; diff --git a/core/lib/zksync_core/src/sync_layer/batch_status_updater.rs b/core/lib/zksync_core/src/sync_layer/batch_status_updater.rs index 8924fa5c5db5..f3027c1d65e8 100644 --- a/core/lib/zksync_core/src/sync_layer/batch_status_updater.rs +++ b/core/lib/zksync_core/src/sync_layer/batch_status_updater.rs @@ -155,7 +155,7 @@ impl BatchStatusUpdater { }; request_latency.observe(); - // We could've used any miniblock from the range, all of them share the same info. + // We could have used any miniblock from the range, all of them share the same info. let request_latency = FETCHER_METRICS.requests[&FetchStage::GetBlockDetails].start(); let Some(batch_info) = self .client diff --git a/core/lib/zksync_core/src/sync_layer/fetcher.rs b/core/lib/zksync_core/src/sync_layer/fetcher.rs index 17f205e87370..1c52864b0d13 100644 --- a/core/lib/zksync_core/src/sync_layer/fetcher.rs +++ b/core/lib/zksync_core/src/sync_layer/fetcher.rs @@ -248,7 +248,7 @@ impl MainNodeFetcher { { tracing::warn!("Following transport error occurred: {err}"); tracing::info!("Trying again after a delay"); - tokio::time::sleep(RETRY_DELAY_INTERVAL).await; // TODO (BFT-100): Implement the fibonacci backoff. + tokio::time::sleep(RETRY_DELAY_INTERVAL).await; // TODO (BFT-100): Implement the Fibonacci back-off. } else { return Err(err.context("Unexpected error in the fetcher")); } diff --git a/core/lib/zksync_core/src/sync_layer/gossip/buffered/mod.rs b/core/lib/zksync_core/src/sync_layer/gossip/buffered/mod.rs index c4680cc3d7d2..4c6a1a73f038 100644 --- a/core/lib/zksync_core/src/sync_layer/gossip/buffered/mod.rs +++ b/core/lib/zksync_core/src/sync_layer/gossip/buffered/mod.rs @@ -323,10 +323,10 @@ impl WriteBlockStore for Buffered { _payload: &Payload, ) -> ctx::Result<()> { // This is storage for non-validator nodes (aka full nodes), - // so verify_payload() won't be called. + // so `verify_payload()` won't be called. // Still, it probably would be better to either - // * move verify_payload() to BlockStore, so that Buffered can just forward the call - // * create another separate trait for verify_payload. + // * move `verify_payload()` to `BlockStore`, so that Buffered can just forward the call + // * create another separate trait for `verify_payload`. // It will be clear what needs to be done when we implement multi-validator consensus for // zksync-era. unimplemented!() diff --git a/core/lib/zksync_core/src/sync_layer/gossip/storage/mod.rs b/core/lib/zksync_core/src/sync_layer/gossip/storage/mod.rs index bc5aeb946b27..cae4ea4f06c2 100644 --- a/core/lib/zksync_core/src/sync_layer/gossip/storage/mod.rs +++ b/core/lib/zksync_core/src/sync_layer/gossip/storage/mod.rs @@ -167,7 +167,7 @@ impl PostgresBlockStorage { let actual_consensus_fields = ConsensusBlockFields::decode(actual_consensus_fields) .context("ConsensusBlockFields::decode()")?; // While justifications may differ among nodes for an arbitrary block, we assume that - // the genesis block has a hardcoded justification. + // the genesis block has a hard coded justification. if actual_consensus_fields != expected_consensus_fields { return Err(anyhow::anyhow!( "Genesis block consensus fields in Postgres {actual_consensus_fields:?} do not match \ @@ -344,7 +344,7 @@ impl BlockStore for PostgresBlockStorage { #[async_trait] impl ContiguousBlockStore for PostgresBlockStorage { async fn schedule_next_block(&self, ctx: &ctx::Ctx, block: &FinalBlock) -> ctx::Result<()> { - // last_in_batch` is always set to `false` by this call; it is properly set by `CursorWithCachedBlock`. + // `last_in_batch` is always set to `false` by this call; it is properly set by `CursorWithCachedBlock`. let fetched_block = FetchedBlock::from_gossip_block(block, false).context("from_gossip_block()")?; let actions = sync::lock(ctx, &self.cursor).await?.advance(fetched_block); 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 994676def491..ecbccdd237da 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_action.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_action.rs @@ -246,7 +246,7 @@ mod tests { // Unexpected tx (vec![tx()], "Unexpected Tx"), (vec![open_batch(), seal_miniblock(), tx()], "Unexpected Tx"), - // Unexpected OpenBatch/Miniblock + // Unexpected `OpenBatch/Miniblock` ( vec![miniblock(), miniblock()], "Unexpected OpenBatch/Miniblock", @@ -259,7 +259,7 @@ mod tests { vec![open_batch(), miniblock()], "Unexpected OpenBatch/Miniblock", ), - // Unexpected SealMiniblock + // Unexpected `SealMiniblock` (vec![seal_miniblock()], "Unexpected SealMiniblock"), ( vec![miniblock(), seal_miniblock(), seal_miniblock()], 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 8ba2cbb4e98c..78b78d4aa4dd 100644 --- a/core/lib/zksync_core/src/sync_layer/sync_state.rs +++ b/core/lib/zksync_core/src/sync_layer/sync_state.rs @@ -40,7 +40,7 @@ impl SyncState { let mut inner = self.inner.write().unwrap(); if let Some(local_block) = inner.local_block { if block.0 < local_block.0 { - // Probably it's fine -- will be checked by the reorg detector. + // Probably it's fine -- will be checked by the re-org detector. tracing::warn!( "main_node_block({}) is less than local_block({})", block, @@ -56,7 +56,7 @@ impl SyncState { let mut inner = self.inner.write().unwrap(); if let Some(main_node_block) = inner.main_node_block { if block.0 > main_node_block.0 { - // Probably it's fine -- will be checked by the reorg detector. + // Probably it's fine -- will be checked by the re-org detector. tracing::warn!( "local_block({}) is greater than main_node_block({})", block, @@ -86,7 +86,7 @@ impl SyncState { (inner.main_node_block, inner.local_block) { let Some(block_diff) = main_node_block.0.checked_sub(local_block.0) else { - // We're ahead of the main node, this situation is handled by the reorg detector. + // We're ahead of the main node, this situation is handled by the re-org detector. return (true, Some(0)); }; (block_diff <= SYNC_MINIBLOCK_DELTA, Some(block_diff)) @@ -137,7 +137,7 @@ mod tests { sync_state.set_main_node_block(MiniblockNumber(1)); sync_state.set_local_block(MiniblockNumber(2)); - // ^ should not panic, as we defer the situation to the reorg detector. + // ^ should not panic, as we defer the situation to the re-org detector. // At the same time, we should consider ourselves synced unless `ReorgDetector` tells us otherwise. assert!(sync_state.is_synced()); @@ -149,7 +149,7 @@ mod tests { sync_state.set_local_block(MiniblockNumber(2)); sync_state.set_main_node_block(MiniblockNumber(1)); - // ^ should not panic, as we defer the situation to the reorg detector. + // ^ should not panic, as we defer the situation to the re-org detector. // At the same time, we should consider ourselves synced unless `ReorgDetector` tells us otherwise. assert!(sync_state.is_synced()); diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index d22409e5219c..826f4e343e92 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -425,7 +425,7 @@ async fn fetcher_basics() { ); let fetcher_task = tokio::spawn(fetcher.run()); - // Check that sync_state is updated. + // Check that `sync_state` is updated. while sync_state.get_main_node_block() < MiniblockNumber(5) { tokio::time::sleep(POLL_INTERVAL).await; } diff --git a/core/lib/zksync_core/src/temp_config_store.rs b/core/lib/zksync_core/src/temp_config_store.rs index 393cd014be82..cfa9ceed3791 100644 --- a/core/lib/zksync_core/src/temp_config_store.rs +++ b/core/lib/zksync_core/src/temp_config_store.rs @@ -14,7 +14,7 @@ use zksync_config::{ GasAdjusterConfig, ObjectStoreConfig, PostgresConfig, }; -// TODO (QIT-22): This structure is going to be removed when components will be respnsible for their own configs. +// TODO (QIT-22): This structure is going to be removed when components will be responsible for their own configs. /// A temporary config store allowing to pass deserialized configs from `zksync_server` to `zksync_core`. /// All the configs are optional, since for some component combination it is not needed to pass all the configs. #[derive(Debug)] diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 0973f8399328..0d5271b93268 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -856,3 +856,6 @@ EIP DecommittmentProcessor decommitment tokenized +Aggregator +DecommittmentProcessor +decommitment From 0421fe6b3e9629fdad2fb88ad5710200825adc91 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Sat, 6 Jan 2024 02:32:33 -0600 Subject: [PATCH 25/49] feat: fix spelling in dev comments in `core/lib/*` - continued (#683) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Series of PRs: This is a part in a series of PRs aimed at enhancing spelling accuracy in this repository. See:** - [ ] https://github.com/matter-labs/zksync-era/pull/681 - [ ] https://github.com/matter-labs/zksync-era/pull/682 - [ ] https://github.com/matter-labs/zksync-era/pull/684 - [ ] https://github.com/matter-labs/zksync-era/pull/685 Once merged, a final PR will enable the `dev_comments: true` in the cargo-spellcheck config file. ## What ❔ - **Spelling Corrections**: This PR focuses on rectifying spelling errors found in the developer comments within multiple core libraries: `core/lib/basic_types/`, `core/lib/config/src/configs/`, `core/lib/constants/`, `core/lib/contracts/`, and `core/lib/dal/`. - Updates dictionary ## Why ❔ - **Consistency and Readability:** The addressed directories contain numerous spelling inaccuracies. Correcting these enhances the readability and consistency of our documentation, ensuring clear understanding for current and future developer ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov --- core/lib/basic_types/src/lib.rs | 4 ++-- core/lib/constants/src/blocks.rs | 4 ++-- core/lib/constants/src/contracts.rs | 2 +- core/lib/constants/src/ethereum.rs | 4 ++-- core/lib/constants/src/system_context.rs | 10 +++++----- core/lib/contracts/src/lib.rs | 2 +- core/lib/dal/src/models/storage_block.rs | 4 ++-- core/lib/dal/src/models/storage_transaction.rs | 2 +- core/lib/dal/src/storage_dal.rs | 4 ++-- core/lib/dal/src/system_dal.rs | 2 +- core/lib/dal/src/transactions_dal.rs | 6 +++--- core/lib/dal/src/transactions_web3_dal.rs | 2 +- core/lib/merkle_tree/README.md | 2 +- core/lib/mini_merkle_tree/README.md | 4 ++-- spellcheck/era.dic | 4 ++++ 15 files changed, 30 insertions(+), 26 deletions(-) diff --git a/core/lib/basic_types/src/lib.rs b/core/lib/basic_types/src/lib.rs index aa9bf615c915..5c8b4e6ee69a 100644 --- a/core/lib/basic_types/src/lib.rs +++ b/core/lib/basic_types/src/lib.rs @@ -115,9 +115,9 @@ impl FromStr for L2ChainId { impl L2ChainId { /// The maximum value of the L2 chain ID. - // 2^53 - 1 is a max safe integer in JS. In ethereum JS libs chain ID should be the safe integer. + // `2^53 - 1` is a max safe integer in JS. In Ethereum JS libraries chain ID should be the safe integer. // Next arithmetic operation: subtract 36 and divide by 2 comes from `v` calculation: - // v = 2*chainId + 36, that should be save integer as well. + // `v = 2*chainId + 36`, that should be save integer as well. const MAX: u64 = ((1 << 53) - 1 - 36) / 2; pub fn max() -> Self { diff --git a/core/lib/constants/src/blocks.rs b/core/lib/constants/src/blocks.rs index 7579b408f0c0..5f7f83c2de2c 100644 --- a/core/lib/constants/src/blocks.rs +++ b/core/lib/constants/src/blocks.rs @@ -1,7 +1,7 @@ use zksync_basic_types::H256; -// By design we don't have a term: uncle blocks. Hence we have to use rlp hash -// from empty list for ethereum compatibility. +// By design we don't have a term: uncle blocks. Hence we have to use RLP hash +// from empty list for Ethereum compatibility. pub const EMPTY_UNCLES_HASH: H256 = H256([ 0x1d, 0xcc, 0x4d, 0xe8, 0xde, 0xc7, 0x5d, 0x7a, 0xab, 0x85, 0xb5, 0x67, 0xb6, 0xcc, 0xd4, 0x1a, 0xd3, 0x12, 0x45, 0x1b, 0x94, 0x8a, 0x74, 0x13, 0xf0, 0xa1, 0x42, 0xfd, 0x40, 0xd4, 0x93, 0x47, diff --git a/core/lib/constants/src/contracts.rs b/core/lib/constants/src/contracts.rs index 5e49f83e5e1d..9d167f7346a3 100644 --- a/core/lib/constants/src/contracts.rs +++ b/core/lib/constants/src/contracts.rs @@ -113,5 +113,5 @@ pub const ERC20_TRANSFER_TOPIC: H256 = H256([ // TODO (SMA-240): Research whether using zero address is ok pub const MINT_AND_BURN_ADDRESS: H160 = H160::zero(); -// The storage_log.value database value for a contract that was deployed in a failed transaction. +// The `storage_log.value` database value for a contract that was deployed in a failed transaction. pub const FAILED_CONTRACT_DEPLOYMENT_BYTECODE_HASH: H256 = H256::zero(); diff --git a/core/lib/constants/src/ethereum.rs b/core/lib/constants/src/ethereum.rs index 299b08e3d0d2..9e88e77c0dfb 100644 --- a/core/lib/constants/src/ethereum.rs +++ b/core/lib/constants/src/ethereum.rs @@ -15,11 +15,11 @@ pub const GUARANTEED_PUBDATA_PER_L1_BATCH: u64 = 4000; /// The 18kb margin is left in case of any impreciseness of the pubdata calculation. pub const MAX_PUBDATA_PER_L1_BATCH: u64 = 110000; -// TODO: import from zkevm_opcode_defs once VM1.3 is supported +// TODO: import from `zkevm_opcode_defs` once `VM1.3` is supported pub const MAX_L2_TX_GAS_LIMIT: u64 = 80000000; // The users should always be able to provide `MAX_GAS_PER_PUBDATA_BYTE` gas per pubdata in their -// transactions so that they are able to send at least GUARANTEED_PUBDATA_PER_L1_BATCH bytes per +// transactions so that they are able to send at least `GUARANTEED_PUBDATA_PER_L1_BATCH` bytes per // transaction. pub const MAX_GAS_PER_PUBDATA_BYTE: u64 = MAX_L2_TX_GAS_LIMIT / GUARANTEED_PUBDATA_PER_L1_BATCH; diff --git a/core/lib/constants/src/system_context.rs b/core/lib/constants/src/system_context.rs index 6fa2def5f315..6a90469fb1f2 100644 --- a/core/lib/constants/src/system_context.rs +++ b/core/lib/constants/src/system_context.rs @@ -35,7 +35,7 @@ pub const SYSTEM_CONTEXT_DIFFICULTY_POSITION: H256 = H256([ 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x05, ]); -// 2500000000000000. THe number is chosen for compatibility with other L2s. +// 2500000000000000. The number is chosen for compatibility with other L2s. pub const SYSTEM_CONTEXT_DIFFICULTY: H256 = H256([ 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x08, 0xE1, 0xBC, 0x9B, 0xF0, 0x40, 0x00, @@ -48,7 +48,7 @@ pub const SYSTEM_CONTEXT_BASE_FEE_POSITION: H256 = H256([ // Tenth of a gwei in wei. 1 gwei is 10^9 wei, so 0.1 gwei is 10^8 wei. const TENTH_OF_GWEI: u64 = 10u64.pow(8); -// The base fee in wei. u64 as u32 would limit this price to be ~4.3 gwei. +// The base fee in wei. u64 as u32 would limit this price to be approximately 4.3 gwei. pub const SYSTEM_CONTEXT_MINIMAL_BASE_FEE: u64 = TENTH_OF_GWEI; pub const SYSTEM_CONTEXT_BLOCK_INFO_POSITION: H256 = H256([ @@ -78,12 +78,12 @@ pub const SYSTEM_CONTEXT_CURRENT_L2_BLOCK_HASHES_POSITION: H256 = H256([ pub const SYSTEM_CONTEXT_STORED_L2_BLOCK_HASHES: u32 = 257; -// It is equal to SYSTEM_CONTEXT_CURRENT_L2_BLOCK_HASHES_POSITION + SYSTEM_CONTEXT_STORED_L2_BLOCK_HASHES +// It is equal to `SYSTEM_CONTEXT_CURRENT_L2_BLOCK_HASHES_POSITION + SYSTEM_CONTEXT_STORED_L2_BLOCK_HASHES` pub const CURRENT_VIRTUAL_BLOCK_INFO_POSITION: H256 = H256([ 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x0c, ]); -/// Block info is stored compactly as SYSTEM_BLOCK_INFO_BLOCK_NUMBER_MULTIPLIER * block_number + block_timestamp. -/// This number is equal to 2**128 +/// Block info is stored compactly as `SYSTEM_BLOCK_INFO_BLOCK_NUMBER_MULTIPLIER * block_number + block_timestamp`. +/// This number is equal to `2**128` pub const SYSTEM_BLOCK_INFO_BLOCK_NUMBER_MULTIPLIER: U256 = U256([0, 0, 1, 0]); diff --git a/core/lib/contracts/src/lib.rs b/core/lib/contracts/src/lib.rs index d114c725bc7d..de1a591b95f7 100644 --- a/core/lib/contracts/src/lib.rs +++ b/core/lib/contracts/src/lib.rs @@ -190,7 +190,7 @@ pub static DEFAULT_SYSTEM_CONTRACTS_REPO: Lazy = /// fetching contracts that are located there. /// As most of the static methods in this file, is loading data based on ZKSYNC_HOME environment variable. pub struct SystemContractsRepo { - // Path to the root of the system contracts repo. + // Path to the root of the system contracts repository. pub root: PathBuf, } diff --git a/core/lib/dal/src/models/storage_block.rs b/core/lib/dal/src/models/storage_block.rs index c48e3d8c7104..4c5db47566a8 100644 --- a/core/lib/dal/src/models/storage_block.rs +++ b/core/lib/dal/src/models/storage_block.rs @@ -339,7 +339,7 @@ pub fn bind_block_where_sql_params<'q>( query: Query<'q, Postgres, PgArguments>, ) -> Query<'q, Postgres, PgArguments> { match block_id { - // these block_id types result in `$1` in the query string, which we have to `bind` + // these `block_id` types result in `$1` in the query string, which we have to `bind` api::BlockId::Hash(block_hash) => query.bind(block_hash.as_bytes()), api::BlockId::Number(api::BlockNumber::Number(number)) => { query.bind(number.as_u64() as i64) @@ -515,7 +515,7 @@ pub struct StorageMiniblockHeader { // The maximal number of virtual blocks that can be created with this miniblock. // If this value is greater than zero, then at least 1 will be created, but no more than - // min(virtual_blocks, miniblock_number - virtual_block_number), i.e. making sure that virtual blocks + // `min(virtual_blocks`, `miniblock_number - virtual_block_number`), i.e. making sure that virtual blocks // never go beyond the miniblock they are based on. pub virtual_blocks: i64, } diff --git a/core/lib/dal/src/models/storage_transaction.rs b/core/lib/dal/src/models/storage_transaction.rs index 8e03590dcc59..1e252a4b8e43 100644 --- a/core/lib/dal/src/models/storage_transaction.rs +++ b/core/lib/dal/src/models/storage_transaction.rs @@ -119,7 +119,7 @@ impl From for L1TxCommonData { // `tx.hash` represents the transaction hash obtained from the execution results, // and it should be exactly the same as the canonical tx hash calculated from the - // transaction data, so we don't store it as a separate "canonical_tx_hash" field. + // transaction data, so we don't store it as a separate `canonical_tx_hash` field. let canonical_tx_hash = H256::from_slice(&tx.hash); L1TxCommonData { diff --git a/core/lib/dal/src/storage_dal.rs b/core/lib/dal/src/storage_dal.rs index 8f08f65b4405..eaefdaef0322 100644 --- a/core/lib/dal/src/storage_dal.rs +++ b/core/lib/dal/src/storage_dal.rs @@ -25,7 +25,7 @@ impl StorageDal<'_, '_> { .map(|dep| (dep.0.as_bytes(), dep.1.as_slice())) .unzip(); - // Copy from stdin can't be used here because of 'ON CONFLICT'. + // Copy from stdin can't be used here because of `ON CONFLICT`. sqlx::query!( r#" INSERT INTO @@ -186,7 +186,7 @@ impl StorageDal<'_, '_> { Vec<_>, ) = query_parts.multiunzip(); - // Copy from stdin can't be used here because of 'ON CONFLICT'. + // Copy from stdin can't be used here because of `ON CONFLICT`. sqlx::query!( r#" INSERT INTO diff --git a/core/lib/dal/src/system_dal.rs b/core/lib/dal/src/system_dal.rs index e9b020943052..e5cf2cf29d4f 100644 --- a/core/lib/dal/src/system_dal.rs +++ b/core/lib/dal/src/system_dal.rs @@ -9,7 +9,7 @@ pub struct SystemDal<'a, 'c> { impl SystemDal<'_, '_> { pub async fn get_replication_lag_sec(&mut self) -> u32 { // NOTE: lag (seconds) has a special meaning here - // (it is not the same that replay_lag/write_lag/flush_lag from pg_stat_replication view) + // (it is not the same that `replay_lag/write_lag/flush_lag` from `pg_stat_replication` view) // and it is only useful when synced column is false, // because lag means how many seconds elapsed since the last action was committed. let pg_row = sqlx::query( diff --git a/core/lib/dal/src/transactions_dal.rs b/core/lib/dal/src/transactions_dal.rs index 7041d4e20ac7..19c19646bbfa 100644 --- a/core/lib/dal/src/transactions_dal.rs +++ b/core/lib/dal/src/transactions_dal.rs @@ -284,7 +284,7 @@ impl TransactionsDal<'_, '_> { // 3) WHERE clause conditions for DO UPDATE block were not met, so the transaction can't be replaced // the subquery in RETURNING clause looks into pre-UPDATE state of the table. So if the subquery will return NULL // transaction is fresh and was added to db(the second condition of RETURNING clause checks it). - // Otherwise, if the subquery won't return NULL it means that there is already tx with such nonce and initiator_address in DB + // Otherwise, if the subquery won't return NULL it means that there is already tx with such nonce and `initiator_address` in DB // and we can replace it WHERE clause conditions are met. // It is worth mentioning that if WHERE clause conditions are not met, None will be returned. let query_result = sqlx::query!( @@ -408,7 +408,7 @@ impl TransactionsDal<'_, '_> { // another tx with the same tx hash is supposed to have the same data // In this case we identify it as Duplicate // Note, this error can happen because of the race condition (tx can be taken by several - // api servers, that simultaneously start execute it and try to inserted to DB) + // API servers, that simultaneously start execute it and try to inserted to DB) if let error::Error::Database(ref error) = err { if let Some(constraint) = error.constraint() { if constraint == "transactions_pkey" { @@ -597,7 +597,7 @@ impl TransactionsDal<'_, '_> { }); if !l2_hashes.is_empty() { - // Update l2 txs + // Update L2 txs // Due to the current tx replacement model, it's possible that tx has been replaced, // but the original was executed in memory, diff --git a/core/lib/dal/src/transactions_web3_dal.rs b/core/lib/dal/src/transactions_web3_dal.rs index 3eb22bf05479..e41d6cd27041 100644 --- a/core/lib/dal/src/transactions_web3_dal.rs +++ b/core/lib/dal/src/transactions_web3_dal.rs @@ -124,7 +124,7 @@ impl TransactionsWeb3Dal<'_, '_> { root: block_hash, logs_bloom: Default::default(), // Even though the Rust SDK recommends us to supply "None" for legacy transactions - // we always supply some number anyway to have the same behaviour as most popular RPCs + // we always supply some number anyway to have the same behavior as most popular RPCs transaction_type: Some(tx_type), } }); diff --git a/core/lib/merkle_tree/README.md b/core/lib/merkle_tree/README.md index edf2ec20c417..b3c8a31c9980 100644 --- a/core/lib/merkle_tree/README.md +++ b/core/lib/merkle_tree/README.md @@ -111,7 +111,7 @@ following order of RocksDB storage consumption at the end of the test: [gauge] rocksdb.total_mem_table_size{db=merkle_tree, cf=stale_keys} = 19924992 bytes ``` -I.e., pruning reduces RocksDB size ~8.7 times in this case. +I.e., pruning reduces RocksDB size approximately 8.7 times in this case. [jellyfish merkle tree]: https://developers.diem.com/papers/jellyfish-merkle-tree/2021-01-14.pdf [`insta`]: https://docs.rs/insta/ diff --git a/core/lib/mini_merkle_tree/README.md b/core/lib/mini_merkle_tree/README.md index a0608d78f71e..afac2fc9ebd2 100644 --- a/core/lib/mini_merkle_tree/README.md +++ b/core/lib/mini_merkle_tree/README.md @@ -12,5 +12,5 @@ cargo bench -p zksync_mini_merkle_tree --bench tree ``` The order of timings should be 2M elements/s for all tree sizes (measured on MacBook Pro with 12-core Apple M2 Max CPU), -both for calculating the root and the root + Merkle path. This translates to ~130µs for a tree with 512 leaves (the tree -size used for `L2ToL1Log`s). +both for calculating the root and the root + Merkle path. This translates to approximately 130µs for a tree with 512 +leaves (the tree size used for `L2ToL1Log`s). diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 0d5271b93268..db9f1e021bfd 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -849,6 +849,10 @@ satisfiability demultiplex precompile statekeeper +matchers +lifecycle +dedup +deduped crаsh protobuf L1Tx From 1fd0afdcd9b6c344e1f5dac93fda5aa25c106b2f Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Sun, 7 Jan 2024 02:20:44 -0600 Subject: [PATCH 26/49] feat: address remaining spelling issues in dev comments and turns on dev_comments in cfg (#827) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Related to: https://github.com/matter-labs/zksync-era/pull/681 ## What ❔ - Sets dev_comments to true - Addresses new / remaining spelling issues in dev comments ## Why ❔ - Dev comments should be checked for spelling issues for improved readability ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov --- core/bin/block_reverter/src/main.rs | 2 +- core/bin/contract-verifier/src/main.rs | 2 +- core/bin/external_node/src/config/mod.rs | 2 +- .../src/intrinsic_costs.rs | 6 +- .../call_tracer/vm_boojum_integration/mod.rs | 2 +- .../bootloader_state/l2_block.rs | 2 +- .../bootloader_state/state.rs | 2 +- .../bootloader_state/utils.rs | 14 ++-- .../implementation/snapshots.rs | 4 +- .../old_vm/event_sink.rs | 2 +- .../old_vm/history_recorder.rs | 4 +- .../vm_boojum_integration/old_vm/memory.rs | 6 +- .../old_vm/oracles/precompile.rs | 2 +- .../vm_boojum_integration/old_vm/utils.rs | 2 +- .../vm_boojum_integration/oracles/storage.rs | 12 +-- .../tracers/circuits_capacity.rs | 4 +- .../tracers/default_tracers.rs | 4 +- .../tracers/pubdata_tracer.rs | 2 +- .../vm_boojum_integration/tracers/refunds.rs | 4 +- .../tracers/result_tracer.rs | 4 +- .../vm_boojum_integration/tracers/utils.rs | 2 +- .../types/internals/pubdata.rs | 8 +- .../types/internals/transaction_data.rs | 2 +- .../vm_boojum_integration/utils/fee.rs | 2 +- .../vm_boojum_integration/utils/l2_blocks.rs | 2 +- .../vm_boojum_integration/utils/overhead.rs | 74 ++++++++++--------- .../vm_latest/old_vm/oracles/precompile.rs | 2 +- .../versions/vm_latest/tests/precompiles.rs | 4 +- .../vm_latest/tracers/circuits_capacity.rs | 4 +- .../src/versions/vm_latest/utils/fee.rs | 2 + core/lib/types/src/commitment.rs | 34 ++++----- .../src/consistency_checker/tests/mod.rs | 18 ++--- core/lib/zksync_core/src/eth_sender/tests.rs | 6 +- .../src/checker.rs | 2 +- core/tests/loadnext/src/account/mod.rs | 2 +- .../src/account/tx_command_executor.rs | 2 +- core/tests/loadnext/src/account_pool.rs | 2 +- core/tests/loadnext/src/command/api.rs | 2 +- core/tests/loadnext/src/config.rs | 12 +-- core/tests/loadnext/src/executor.rs | 2 +- .../src/report_collector/metrics_collector.rs | 2 +- core/tests/loadnext/src/rng.rs | 2 +- core/tests/vm-benchmark/harness/src/lib.rs | 2 +- sdk/zksync-rs/src/ethereum/mod.rs | 6 +- spellcheck/era.cfg | 2 +- spellcheck/era.dic | 1 + 46 files changed, 144 insertions(+), 137 deletions(-) diff --git a/core/bin/block_reverter/src/main.rs b/core/bin/block_reverter/src/main.rs index c1b02a1a1201..f7cbc20f554d 100644 --- a/core/bin/block_reverter/src/main.rs +++ b/core/bin/block_reverter/src/main.rs @@ -32,7 +32,7 @@ enum Command { #[arg(long)] l1_batch_number: u32, /// Priority fee used for rollback Ethereum transaction. - // We operate only by priority fee because we want to use base fee from ethereum + // We operate only by priority fee because we want to use base fee from Ethereum // and send transaction as soon as possible without any resend logic #[arg(long)] priority_fee_per_gas: Option, diff --git a/core/bin/contract-verifier/src/main.rs b/core/bin/contract-verifier/src/main.rs index 33090697c510..477b4dc0722d 100644 --- a/core/bin/contract-verifier/src/main.rs +++ b/core/bin/contract-verifier/src/main.rs @@ -178,7 +178,7 @@ async fn main() -> anyhow::Result<()> { let contract_verifier = ContractVerifier::new(verifier_config, pool); let tasks = vec![ - // todo PLA-335: Leftovers after the prover DB split. + // TODO PLA-335: Leftovers after the prover DB split. // The prover connection pool is not used by the contract verifier, but we need to pass it // since `JobProcessor` trait requires it. tokio::spawn(contract_verifier.run(stop_receiver.clone(), opt.jobs_number)), diff --git a/core/bin/external_node/src/config/mod.rs b/core/bin/external_node/src/config/mod.rs index 47c6f74f17fb..e25b5390b0ff 100644 --- a/core/bin/external_node/src/config/mod.rs +++ b/core/bin/external_node/src/config/mod.rs @@ -437,7 +437,7 @@ impl ExternalNodeConfig { .context("Unable to fetch required config values from the main node")?; // We can query them from main node, but it's better to set them explicitly - // as well to avoid connecting to wrong envs unintentionally. + // as well to avoid connecting to wrong environment variables unintentionally. let eth_chain_id = HttpClientBuilder::default() .build(required.eth_client_url()?) .expect("Unable to build HTTP client for L1 client") diff --git a/core/bin/system-constants-generator/src/intrinsic_costs.rs b/core/bin/system-constants-generator/src/intrinsic_costs.rs index 697d539f0c20..4f5e988e7b1a 100644 --- a/core/bin/system-constants-generator/src/intrinsic_costs.rs +++ b/core/bin/system-constants-generator/src/intrinsic_costs.rs @@ -81,7 +81,7 @@ pub(crate) fn l2_gas_constants() -> IntrinsicSystemGasConstants { true, ); - // This price does not include the overhead for the transaction itself, but rather auxilary parts + // This price does not include the overhead for the transaction itself, but rather auxiliary parts // that must be done by the transaction and it can not be enforced by the operator to not to accept // the transaction if it does not cover the minimal costs. let min_l1_tx_price = empty_l1_tx_result.gas_consumed - bootloader_intrinsic_gas; @@ -107,7 +107,7 @@ pub(crate) fn l2_gas_constants() -> IntrinsicSystemGasConstants { let delta_from_544_bytes = lengthier_tx_result.gas_consumed - empty_l1_tx_result.gas_consumed; - // The number of public data per factory dep should not depend on the size/structure of the factory + // The number of public data per factory dependencies should not depend on the size/structure of the factory // dependency, since the dependency has already been published on L1. let tx_with_more_factory_deps_result = execute_user_txs_in_test_gas_vm( vec![get_l1_tx( @@ -180,7 +180,7 @@ fn get_intrinsic_overheads_for_tx_type(tx_generator: &TransactionGenerator) -> I let bootloader_intrinsic_pubdata = result_0.pubdata_published; // For various small reasons the overhead for the first transaction and for all the subsequent ones - // might differ a bit, so we will calculate both and will use the maximum one as the result for l2 txs. + // might differ a bit, so we will calculate both and will use the maximum one as the result for L2 txs. let (tx1_intrinsic_gas, tx1_intrinsic_pubdata) = get_intrinsic_price(result_0, result_1); let (tx2_intrinsic_gas, tx2_intrinsic_pubdata) = get_intrinsic_price(result_1, result_2); diff --git a/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs b/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs index 8b6742da4619..e2e884e26a1e 100644 --- a/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs +++ b/core/lib/multivm/src/tracers/call_tracer/vm_boojum_integration/mod.rs @@ -143,7 +143,7 @@ impl CallTracer { let fat_data_pointer = state.vm_local_state.registers[RET_IMPLICIT_RETURNDATA_PARAMS_REGISTER as usize]; - // if fat_data_pointer is not a pointer then there is no output + // if `fat_data_pointer` is not a pointer then there is no output let output = if fat_data_pointer.is_pointer { let fat_data_pointer = FatPointer::from_u256(fat_data_pointer.value); if !fat_data_pointer.is_trivial() { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs index a1bd6323b848..f032c301c948 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/l2_block.rs @@ -19,7 +19,7 @@ pub(crate) struct BootloaderL2Block { pub(crate) timestamp: u64, pub(crate) txs_rolling_hash: H256, // The rolling hash of all the transactions in the miniblock pub(crate) prev_block_hash: H256, - // Number of the first l2 block tx in l1 batch + // Number of the first L2 block tx in L1 batch pub(crate) first_tx_index: usize, pub(crate) max_virtual_blocks_to_create: u32, pub(super) txs: Vec, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs index 255dc6aaebb9..db13d2aace5d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/state.rs @@ -68,7 +68,7 @@ impl BootloaderState { pub(crate) fn set_refund_for_current_tx(&mut self, refund: u32) { let current_tx = self.current_tx(); - // We can't set the refund for the latest tx or using the latest l2_block for fining tx + // We can't set the refund for the latest tx or using the latest `l2_block` for fining tx // Because we can fill the whole batch first and then execute txs one by one let tx = self.find_tx_mut(current_tx); tx.refund = refund; diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs index 091bca449f75..77a8ed2ce9b9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/bootloader_state/utils.rs @@ -74,7 +74,7 @@ pub(super) fn apply_tx_to_memory( }; apply_l2_block(memory, &bootloader_l2_block, tx_index); - // Note, +1 is moving for pointer + // Note, `+1` is moving for pointer let compressed_bytecodes_offset = COMPRESSED_BYTECODES_OFFSET + 1 + compressed_bytecodes_size; let encoded_compressed_bytecodes = @@ -94,8 +94,8 @@ pub(crate) fn apply_l2_block( bootloader_l2_block: &BootloaderL2Block, txs_index: usize, ) { - // Since L2 block infos start from the TX_OPERATOR_L2_BLOCK_INFO_OFFSET and each - // L2 block info takes TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots, the position where the L2 block info + // Since L2 block information starts from the `TX_OPERATOR_L2_BLOCK_INFO_OFFSET` and each + // L2 block info takes `TX_OPERATOR_SLOTS_PER_L2_BLOCK_INFO slots`, the position where the L2 block info // for this transaction needs to be written is: let block_position = @@ -120,12 +120,12 @@ pub(crate) fn apply_pubdata_to_memory( pubdata_information: PubdataInput, ) { // Skipping two slots as they will be filled by the bootloader itself: - // - One slot is for the selector of the call to the L1Messenger. + // - One slot is for the selector of the call to the `L1Messenger`. // - The other slot is for the 0x20 offset for the calldata. let l1_messenger_pubdata_start_slot = OPERATOR_PROVIDED_L1_MESSENGER_PUBDATA_OFFSET + 2; // Need to skip first word as it represents array offset - // while bootloader expects only [len || data] + // while bootloader expects only `[len || data]` let pubdata = ethabi::encode(&[ethabi::Token::Bytes( pubdata_information.build_pubdata(true), )])[32..] @@ -156,8 +156,8 @@ pub(crate) fn apply_pubdata_to_memory( /// For example, when checking validity, we don't want to actually execute transaction and have side effects. /// /// Possible values: -/// - 0x00: validate & execute (normal mode) -/// - 0x02: execute but DO NOT validate +/// - `0x00`: validate & execute (normal mode) +/// - `0x02`: execute but DO NOT validate /// /// - 31 byte (LSB): whether to execute transaction or not (at all). pub(super) fn assemble_tx_meta(execution_mode: TxExecutionMode, execute_tx: bool) -> U256 { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs index 98c4baad99f8..b5b09c0fd6d2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/snapshots.rs @@ -34,8 +34,8 @@ impl Vm { pub(crate) fn make_snapshot_inner(&mut self) { self.snapshots.push(VmSnapshot { // Vm local state contains O(1) various parameters (registers/etc). - // The only "expensive" copying here is copying of the callstack. - // It will take O(callstack_depth) to copy it. + // The only "expensive" copying here is copying of the call stack. + // It will take `O(callstack_depth)` to copy it. // So it is generally recommended to get snapshots of the bootloader frame, // where the depth is 1. local_state: self.state.local_state.clone(), diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs index 0693ad35742a..6638057643d4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/event_sink.rs @@ -54,7 +54,7 @@ impl InMemoryEventSink { pub fn log_queries_after_timestamp(&self, from_timestamp: Timestamp) -> &[Box] { let events = self.frames_stack.forward().current_frame(); - // Select all of the last elements where e.timestamp >= from_timestamp. + // Select all of the last elements where `e.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. events .rsplit(|e| e.timestamp < from_timestamp) diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs index 10e138885aa4..90d0c868ea33 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/history_recorder.rs @@ -12,14 +12,14 @@ use zksync_utils::{h256_to_u256, u256_to_h256}; pub(crate) type MemoryWithHistory = HistoryRecorder; pub(crate) type IntFrameManagerWithHistory = HistoryRecorder, H>; -// Within the same cycle, timestamps in range timestamp..timestamp+TIME_DELTA_PER_CYCLE-1 +// Within the same cycle, timestamps in range `timestamp..timestamp+TIME_DELTA_PER_CYCLE-1` // can be used. This can sometimes violate monotonicity of the timestamp within the // same cycle, so it should be normalized. #[inline] fn normalize_timestamp(timestamp: Timestamp) -> Timestamp { let timestamp = timestamp.0; - // Making sure it is divisible by TIME_DELTA_PER_CYCLE + // Making sure it is divisible by `TIME_DELTA_PER_CYCLE` Timestamp(timestamp - timestamp % zkevm_opcode_defs::TIME_DELTA_PER_CYCLE) } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs index f151278ffa4b..8229727b6dd9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/memory.rs @@ -282,7 +282,7 @@ impl Memory for SimpleMemory { let returndata_page = returndata_fat_pointer.memory_page; for &page in current_observable_pages { - // If the page's number is greater than or equal to the base_page, + // If the page's number is greater than or equal to the `base_page`, // it means that it was created by the internal calls of this contract. // We need to add this check as the calldata pointer is also part of the // observable pages. @@ -299,7 +299,7 @@ impl Memory for SimpleMemory { } } -// It is expected that there is some intersection between [word_number*32..word_number*32+31] and [start, end] +// It is expected that there is some intersection between `[word_number*32..word_number*32+31]` and `[start, end]` fn extract_needed_bytes_from_word( word_value: Vec, word_number: usize, @@ -307,7 +307,7 @@ fn extract_needed_bytes_from_word( end: usize, ) -> Vec { let word_start = word_number * 32; - let word_end = word_start + 31; // Note, that at word_start + 32 a new word already starts + let word_end = word_start + 31; // Note, that at `word_start + 32` a new word already starts let intersection_left = std::cmp::max(word_start, start); let intersection_right = std::cmp::min(word_end, end); diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs index c4986250cbae..b23c72669d70 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs @@ -109,6 +109,6 @@ impl PrecompilesProcessor for PrecompilesProcessorWithHistory } fn finish_frame(&mut self, _panicked: bool) { - // there are no revertable precompile yes, so we are ok + // there are no revertible precompile yes, so we are ok } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs index 3aa4beb8e8de..342cc64ea2ac 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/utils.rs @@ -122,7 +122,7 @@ pub(crate) fn vm_may_have_ended_inner( } (false, _) => None, (true, l) if l == outer_eh_location => { - // check r1,r2,r3 + // check `r1,r2,r3` if vm.local_state.flags.overflow_or_less_than_flag { Some(VmExecutionResult::Panic) } else { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs index 919fd301c573..1367f83f4e5d 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/oracles/storage.rs @@ -53,7 +53,7 @@ pub struct StorageOracle { pub(crate) paid_changes: HistoryRecorder, H>, // The map that contains all the first values read from storage for each slot. - // While formally it does not have to be rollbackable, we still do it to avoid memory bloat + // While formally it does not have to be rolled back, we still do it to avoid memory bloat // for unused slots. pub(crate) initial_values: HistoryRecorder, H>, @@ -212,7 +212,7 @@ impl StorageOracle { let required_pubdata = self.base_price_for_write(&key, first_slot_value, current_slot_value); - // We assume that "prepaid_for_slot" represents both the number of pubdata published and the number of bytes paid by the previous transactions + // We assume that `prepaid_for_slot` represents both the number of pubdata published and the number of bytes paid by the previous transactions // as they should be identical. let prepaid_for_slot = self .pre_paid_changes @@ -292,7 +292,7 @@ impl StorageOracle { ) -> &[Box] { let logs = self.frames_stack.forward().current_frame(); - // Select all of the last elements where l.log_query.timestamp >= from_timestamp. + // Select all of the last elements where `l.log_query.timestamp >= from_timestamp`. // Note, that using binary search here is dangerous, because the logs are not sorted by timestamp. logs.rsplit(|l| l.log_query.timestamp < from_timestamp) .next() @@ -340,6 +340,7 @@ impl VmStorageOracle for StorageOracle { _monotonic_cycle_counter: u32, mut query: LogQuery, ) -> LogQuery { + //``` // tracing::trace!( // "execute partial query cyc {:?} addr {:?} key {:?}, rw {:?}, wr {:?}, tx {:?}", // _monotonic_cycle_counter, @@ -349,6 +350,7 @@ impl VmStorageOracle for StorageOracle { // query.written_value, // query.tx_number_in_block // ); + //``` assert!(!query.rollback); if query.rw_flag { // The number of bytes that have been compensated by the user to perform this write @@ -443,7 +445,7 @@ impl VmStorageOracle for StorageOracle { ); // Additional validation that the current value was correct - // Unwrap is safe because the return value from write_inner is the previous value in this leaf. + // Unwrap is safe because the return value from `write_inner` is the previous value in this leaf. // It is impossible to set leaf value to `None` assert_eq!(current_value, written_value); } @@ -457,7 +459,7 @@ impl VmStorageOracle for StorageOracle { /// Returns the number of bytes needed to publish a slot. // Since we need to publish the state diffs onchain, for each of the updated storage slot -// we basically need to publish the following pair: (). +// we basically need to publish the following pair: `()`. // For key we use the following optimization: // - The first time we publish it, we use 32 bytes. // Then, we remember a 8-byte id for this slot and assign it to it. We call this initial write. diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs index 16f5540172a8..33fa6677de27 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_capacity.rs @@ -47,7 +47,7 @@ pub(crate) const AVERAGE_OPCODE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION; // Here "base" fraction is a fraction that will be used unconditionally. -// Usage of StorageApplication is being tracked separately as it depends on whether slot was read before or not. +// Usage of `StorageApplication` is being tracked separately as it depends on whether slot was read before or not. pub(crate) const STORAGE_READ_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + LOG_DEMUXER_CYCLE_FRACTION @@ -59,7 +59,7 @@ pub(crate) const EVENT_OR_L1_MESSAGE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + 2.0 * EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION; // Here "base" fraction is a fraction that will be used unconditionally. -// Usage of StorageApplication is being tracked separately as it depends on whether slot was written before or not. +// Usage of `StorageApplication` is being tracked separately as it depends on whether slot was written before or not. pub(crate) const STORAGE_WRITE_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + 2.0 * LOG_DEMUXER_CYCLE_FRACTION diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs index 236421726051..e41451ae8ef4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs @@ -54,7 +54,7 @@ pub(crate) struct DefaultExecutionTracer { pub(crate) result_tracer: ResultTracer, // This tracer is designed specifically for calculating refunds. Its separation from the custom tracer // ensures static dispatch, enhancing performance by avoiding dynamic dispatch overhead. - // Additionally, being an internal tracer, it saves the results directly to VmResultAndLogs. + // Additionally, being an internal tracer, it saves the results directly to `VmResultAndLogs`. pub(crate) refund_tracer: Option>, // The pubdata tracer is responsible for inserting the pubdata packing information into the bootloader // memory at the end of the batch. Its separation from the custom tracer @@ -298,7 +298,7 @@ impl DefaultExecutionTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs index 78c8160d1af9..3e3075cb45f9 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/pubdata_tracer.rs @@ -56,7 +56,7 @@ impl PubdataTracer { impl PubdataTracer { // Packs part of L1 Messenger total pubdata that corresponds to - // L2toL1Logs sent in the block + // `L2toL1Logs` sent in the block fn get_total_user_logs( &self, state: &ZkSyncVmState, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs index 5605661a2ec2..9f2d6837030c 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs @@ -214,8 +214,8 @@ impl VmTracer for RefundsTracer { #[vise::register] static METRICS: vise::Global = vise::Global::new(); - // This means that the bootloader has informed the system (usually via VMHooks) - that some gas - // should be refunded back (see askOperatorForRefund in bootloader.yul for details). + // This means that the bootloader has informed the system (usually via `VMHooks`) - that some gas + // should be refunded back (see `askOperatorForRefund` in `bootloader.yul` for details). if let Some(bootloader_refund) = self.requested_refund() { assert!( self.operator_refund.is_none(), diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs index b8b00b265e03..2293273228b1 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/result_tracer.rs @@ -54,7 +54,7 @@ impl ResultTracer { } fn current_frame_is_bootloader(local_state: &VmLocalState) -> bool { - // The current frame is bootloader if the callstack depth is 1. + // The current frame is bootloader if the call stack depth is 1. // Some of the near calls inside the bootloader can be out of gas, which is totally normal behavior // and it shouldn't result in `is_bootloader_out_of_gas` becoming true. local_state.callstack.inner.len() == 1 @@ -150,7 +150,7 @@ impl ResultTracer { }); } VmExecutionResult::Revert(output) => { - // Unlike VmHook::ExecutionResult, vm has completely finished and returned not only the revert reason, + // Unlike `VmHook::ExecutionResult`, vm has completely finished and returned not only the revert reason, // but with bytecode, which represents the type of error from the bootloader side let revert_reason = TxRevertReason::parse_error(&output); diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs index 592429677109..58264d89c8ea 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/utils.rs @@ -57,7 +57,7 @@ impl VmHook { let value = data.src1_value.value; - // Only UMA opcodes in the bootloader serve for vm hooks + // Only `UMA` opcodes in the bootloader serve for vm hooks if !matches!(opcode_variant.opcode, Opcode::UMA(UMAOpcode::HeapWrite)) || heap_page != BOOTLOADER_HEAP_PAGE || fat_ptr.offset != VM_HOOK_POSITION * 32 diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs index aa9e8a7eda70..5451201c5bcf 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/pubdata.rs @@ -24,14 +24,14 @@ impl PubdataInput { } = self; // Encoding user L2->L1 logs. - // Format: [(numberOfL2ToL1Logs as u32) || l2tol1logs[1] || ... || l2tol1logs[n]] + // Format: `[(numberOfL2ToL1Logs as u32) || l2tol1logs[1] || ... || l2tol1logs[n]]` l1_messenger_pubdata.extend((user_logs.len() as u32).to_be_bytes()); for l2tol1log in user_logs { l1_messenger_pubdata.extend(l2tol1log.packed_encoding()); } // Encoding L2->L1 messages - // Format: [(numberOfMessages as u32) || (messages[1].len() as u32) || messages[1] || ... || (messages[n].len() as u32) || messages[n]] + // Format: `[(numberOfMessages as u32) || (messages[1].len() as u32) || messages[1] || ... || (messages[n].len() as u32) || messages[n]]` l1_messenger_pubdata.extend((l2_to_l1_messages.len() as u32).to_be_bytes()); for message in l2_to_l1_messages { l1_messenger_pubdata.extend((message.len() as u32).to_be_bytes()); @@ -39,7 +39,7 @@ impl PubdataInput { } // Encoding bytecodes - // Format: [(numberOfBytecodes as u32) || (bytecodes[1].len() as u32) || bytecodes[1] || ... || (bytecodes[n].len() as u32) || bytecodes[n]] + // Format: `[(numberOfBytecodes as u32) || (bytecodes[1].len() as u32) || bytecodes[1] || ... || (bytecodes[n].len() as u32) || bytecodes[n]]` l1_messenger_pubdata.extend((published_bytecodes.len() as u32).to_be_bytes()); for bytecode in published_bytecodes { l1_messenger_pubdata.extend((bytecode.len() as u32).to_be_bytes()); @@ -47,7 +47,7 @@ impl PubdataInput { } // Encoding state diffs - // Format: [size of compressed state diffs u32 || compressed state diffs || (# state diffs: intial + repeated) as u32 || sorted state diffs by ] + // Format: `[size of compressed state diffs u32 || compressed state diffs || (# state diffs: initial + repeated) as u32 || sorted state diffs by ]` let state_diffs_compressed = compress_state_diffs(state_diffs.clone()); l1_messenger_pubdata.extend(state_diffs_compressed); diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs index a2a9f57d2742..277affb39ef3 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs @@ -234,7 +234,7 @@ impl TransactionData { let l2_tx: L2Tx = self.clone().try_into().unwrap(); let transaction_request: TransactionRequest = l2_tx.into(); - // It is assumed that the TransactionData always has all the necessary components to recover the hash. + // It is assumed that the `TransactionData` always has all the necessary components to recover the hash. transaction_request .get_tx_hash(chain_id) .expect("Could not recover L2 transaction hash") diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs index c19b401e3fce..b8d5bd1a38f2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs @@ -18,7 +18,7 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) -> (u64, u64) { let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); - // The baseFee is set in such a way that it is always possible for a transaction to + // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( fair_gas_price, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs index 5dd26c4c0277..e5832f7f5879 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/l2_blocks.rs @@ -68,7 +68,7 @@ pub fn load_last_l2_block(storage: StoragePtr) -> Option u32 { - // Even if the gas limit is greater than the MAX_TX_ERGS_LIMIT, we assume that everything beyond MAX_TX_ERGS_LIMIT + // Even if the gas limit is greater than the `MAX_TX_ERGS_LIMIT`, we assume that everything beyond `MAX_TX_ERGS_LIMIT` // will be spent entirely on publishing bytecodes and so we derive the overhead solely based on the capped value let gas_limit = std::cmp::min(MAX_TX_ERGS_LIMIT, gas_limit); - // Using large U256 type to avoid overflow + // Using large `U256` type to avoid overflow let max_block_overhead = U256::from(block_overhead_gas(gas_price_per_pubdata)); let gas_limit = U256::from(gas_limit); let encoded_len = U256::from(encoded_len); - // The MAX_TX_ERGS_LIMIT is formed in a way that may fulfills a single-instance circuits - // if used in full. That is, within MAX_TX_ERGS_LIMIT it is possible to fully saturate all the single-instance + // The `MAX_TX_ERGS_LIMIT` is formed in a way that may fulfills a single-instance circuits + // if used in full. That is, within `MAX_TX_ERGS_LIMIT` it is possible to fully saturate all the single-instance // circuits. let overhead_for_single_instance_circuits = ceil_div_u256(gas_limit * max_block_overhead, MAX_TX_ERGS_LIMIT.into()); @@ -38,15 +38,17 @@ pub(crate) fn derive_overhead( // The overhead for occupying a single tx slot let tx_slot_overhead = ceil_div_u256(max_block_overhead, MAX_TXS_IN_BLOCK.into()); - // We use "ceil" here for formal reasons to allow easier approach for calculating the overhead in O(1) - // let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata); + // We use `ceil` here for formal reasons to allow easier approach for calculating the overhead in `O(1)` + // `let max_pubdata_in_tx = ceil_div_u256(gas_limit, gas_price_per_pubdata);` // The maximal potential overhead from pubdata // TODO (EVM-67): possibly use overhead for pubdata + // ``` // let pubdata_overhead = ceil_div_u256( // max_pubdata_in_tx * max_block_overhead, // MAX_PUBDATA_PER_BLOCK.into(), // ); + //``` vec![ (coefficients.ergs_limit_overhead_coeficient @@ -102,8 +104,8 @@ impl OverheadCoefficients { OverheadCoefficients::new_checked( 1.0, 1.0, // For L2 transactions we allow a certain default discount with regard to the number of ergs. - // Multiinstance circuits can in theory be spawned infinite times, while projected future limitations - // on gas per pubdata allow for roughly 800kk gas per L1 batch, so the rough trust "discount" on the proof's part + // Multi-instance circuits can in theory be spawned infinite times, while projected future limitations + // on gas per pubdata allow for roughly 800k gas per L1 batch, so the rough trust "discount" on the proof's part // to be paid by the users is 0.1. 0.1, ) @@ -132,28 +134,28 @@ pub(crate) fn get_amortized_overhead( let encoded_len = U256::from(encoded_len); // Derivation of overhead consists of 4 parts: - // 1. The overhead for taking up a transaction's slot. (O1): O1 = 1 / MAX_TXS_IN_BLOCK - // 2. The overhead for taking up the bootloader's memory (O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE - // 3. The overhead for possible usage of pubdata. (O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK - // 4. The overhead for possible usage of all the single-instance circuits. (O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT + // 1. The overhead for taking up a transaction's slot. `(O1): O1 = 1 / MAX_TXS_IN_BLOCK` + // 2. The overhead for taking up the bootloader's memory `(O2): O2 = encoded_len / BOOTLOADER_TX_ENCODING_SPACE` + // 3. The overhead for possible usage of pubdata. `(O3): O3 = max_pubdata_in_tx / MAX_PUBDATA_PER_BLOCK` + // 4. The overhead for possible usage of all the single-instance circuits. `(O4): O4 = gas_limit / MAX_TX_ERGS_LIMIT` // // The maximum of these is taken to derive the part of the block's overhead to be paid by the users: // - // max_overhead = max(O1, O2, O3, O4) - // overhead_gas = ceil(max_overhead * overhead_for_block_gas). Thus, overhead_gas is a function of - // tx_gas_limit, gas_per_pubdata_byte_limit and encoded_len. + // `max_overhead = max(O1, O2, O3, O4)` + // `overhead_gas = ceil(max_overhead * overhead_for_block_gas)`. Thus, `overhead_gas` is a function of + // `tx_gas_limit`, `gas_per_pubdata_byte_limit` and `encoded_len`. // - // While it is possible to derive the overhead with binary search in O(log n), it is too expensive to be done - // on L1, so here is a reference implementation of finding the overhead for transaction in O(1): + // While it is possible to derive the overhead with binary search in `O(log n)`, it is too expensive to be done + // on L1, so here is a reference implementation of finding the overhead for transaction in `O(1)`: // - // Given total_gas_limit = tx_gas_limit + overhead_gas, we need to find overhead_gas and tx_gas_limit, such that: - // 1. overhead_gas is maximal possible (the operator is paid fairly) - // 2. overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas (the user does not overpay) + // Given `total_gas_limit = tx_gas_limit + overhead_gas`, we need to find `overhead_gas` and `tx_gas_limit`, such that: + // 1. `overhead_gas` is maximal possible (the operator is paid fairly) + // 2. `overhead_gas(tx_gas_limit, gas_per_pubdata_byte_limit, encoded_len) >= overhead_gas` (the user does not overpay) // The third part boils to the following 4 inequalities (at least one of these must hold): - // ceil(O1 * overhead_for_block_gas) >= overhead_gas - // ceil(O2 * overhead_for_block_gas) >= overhead_gas - // ceil(O3 * overhead_for_block_gas) >= overhead_gas - // ceil(O4 * overhead_for_block_gas) >= overhead_gas + // `ceil(O1 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O2 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O3 * overhead_for_block_gas) >= overhead_gas` + // `ceil(O4 * overhead_for_block_gas) >= overhead_gas` // // Now, we need to solve each of these separately: @@ -164,7 +166,7 @@ pub(crate) fn get_amortized_overhead( (coefficients.slot_overhead_coeficient * tx_slot_overhead as f64).floor() as u32 }; - // 2. The overhead for occupying the bootloader memory can be derived from encoded_len + // 2. The overhead for occupying the bootloader memory can be derived from `encoded_len` let overhead_for_length = { let overhead_for_length = ceil_div_u256( encoded_len * overhead_for_block_gas, @@ -175,7 +177,7 @@ pub(crate) fn get_amortized_overhead( (coefficients.bootloader_memory_overhead_coeficient * overhead_for_length as f64).floor() as u32 }; - + //``` // TODO (EVM-67): possibly include the overhead for pubdata. The formula below has not been properly maintained, // since the pubdat is not published. If decided to use the pubdata overhead, it needs to be updated. // 3. ceil(O3 * overhead_for_block_gas) >= overhead_gas @@ -196,7 +198,7 @@ pub(crate) fn get_amortized_overhead( // + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); // let denominator = // gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; - + // // // Corner case: if `total_gas_limit` = `gas_per_pubdata_byte_limit` = 0 // // then the numerator will be 0 and subtracting 1 will cause a panic, so we just return a zero. // if numerator.is_zero() { @@ -205,15 +207,15 @@ pub(crate) fn get_amortized_overhead( // (numerator - 1) / denominator // } // }; - - // 4. K * ceil(O4 * overhead_for_block_gas) >= overhead_gas, where K is the discount - // O4 = gas_limit / MAX_TX_ERGS_LIMIT. Using the notation from the previous equation: - // ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K) - // ceil(OB * (TL - OE) / MAX_TX_ERGS_LIMIT) >= (OE/K) - // OB * (TL - OE) / MAX_TX_ERGS_LIMIT > (OE/K) - 1 - // OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT - // OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB) - // OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB)), with possible -1 if the division is without remainder + //``` + // 4. `K * ceil(O4 * overhead_for_block_gas) >= overhead_gas`, where K is the discount + // `O4 = gas_limit / MAX_TX_ERGS_LIMIT`. Using the notation from the previous equation: + // `ceil(OB * GL / MAX_TX_ERGS_LIMIT) >= (OE / K)` + // `ceil(OB * (TL - OE) / MAX_TX_ERGS_LIMIT) >= (OE/K)` + // `OB * (TL - OE) / MAX_TX_ERGS_LIMIT > (OE/K) - 1` + // `OB * (TL - OE) > (OE/K) * MAX_TX_ERGS_LIMIT - MAX_TX_ERGS_LIMIT` + // `OB * TL + MAX_TX_ERGS_LIMIT > OE * ( MAX_TX_ERGS_LIMIT/K + OB)` + // `OE = floor(OB * TL + MAX_TX_ERGS_LIMIT / (MAX_TX_ERGS_LIMIT/K + OB))`, with possible -1 if the division is without remainder let overhead_for_gas = { let numerator = overhead_for_block_gas * total_gas_limit + U256::from(MAX_TX_ERGS_LIMIT); let denominator: U256 = U256::from( diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs index dbc03fb143a5..0e750fe69356 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs @@ -107,6 +107,6 @@ impl PrecompilesProcessor for PrecompilesProcessorWithHistory } fn finish_frame(&mut self, _panicked: bool) { - // there are no revertable precompile yes, so we are ok + // there are no revertible precompile yes, so we are ok } } diff --git a/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs b/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs index adc62a4bd2d6..4f78319bc42e 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/precompiles.rs @@ -56,7 +56,7 @@ fn test_keccak() { #[test] fn test_sha256() { - // Execute special transaction and check that at least 1000 sha256 calls were made. + // Execute special transaction and check that at least 1000 `sha256` calls were made. let contract = read_precompiles_contract(); let address = Address::random(); let mut vm = VmTesterBuilder::new(HistoryEnabled) @@ -100,7 +100,7 @@ fn test_sha256() { #[test] fn test_ecrecover() { - // Execute simple transfer and check that exactly 1 ecrecover call was made (it's done during tx validation). + // Execute simple transfer and check that exactly 1 `ecrecover` call was made (it's done during tx validation). let mut vm = VmTesterBuilder::new(HistoryEnabled) .with_empty_in_memory_storage() .with_random_rich_accounts(1) diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs index 16f5540172a8..33fa6677de27 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_capacity.rs @@ -47,7 +47,7 @@ pub(crate) const AVERAGE_OPCODE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION; // Here "base" fraction is a fraction that will be used unconditionally. -// Usage of StorageApplication is being tracked separately as it depends on whether slot was read before or not. +// Usage of `StorageApplication` is being tracked separately as it depends on whether slot was read before or not. pub(crate) const STORAGE_READ_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + LOG_DEMUXER_CYCLE_FRACTION @@ -59,7 +59,7 @@ pub(crate) const EVENT_OR_L1_MESSAGE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + 2.0 * EVENTS_OR_L1_MESSAGES_SORTER_CYCLE_FRACTION; // Here "base" fraction is a fraction that will be used unconditionally. -// Usage of StorageApplication is being tracked separately as it depends on whether slot was written before or not. +// Usage of `StorageApplication` is being tracked separately as it depends on whether slot was written before or not. pub(crate) const STORAGE_WRITE_BASE_FRACTION: f32 = MAIN_VM_CYCLE_FRACTION + RAM_PERMUTATION_CYCLE_FRACTION + 2.0 * LOG_DEMUXER_CYCLE_FRACTION diff --git a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs index 9711b12ecbd4..6efd9274523d 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs @@ -38,9 +38,11 @@ pub(crate) fn adjust_l1_gas_price_for_tx( fair_l2_gas_price: u64, tx_gas_per_pubdata_limit: U256, ) -> u64 { + //``` // gasPerPubdata = ceil(17 * l1gasprice / fair_l2_gas_price) // gasPerPubdata <= 17 * l1gasprice / fair_l2_gas_price + 1 // fair_l2_gas_price(gasPerPubdata - 1) / 17 <= l1gasprice + //``` let l1_gas_price = U256::from(fair_l2_gas_price) * (tx_gas_per_pubdata_limit - U256::from(1u32)) / U256::from(17); diff --git a/core/lib/types/src/commitment.rs b/core/lib/types/src/commitment.rs index e11bdbdc6487..27851c3434e9 100644 --- a/core/lib/types/src/commitment.rs +++ b/core/lib/types/src/commitment.rs @@ -131,25 +131,25 @@ impl L1BatchWithMetadata { }) } - /// Encodes L1Batch into StorageBatchInfo (see IExecutor.sol) + /// Encodes L1Batch into `StorageBatchInfo` (see `IExecutor.sol`) pub fn l1_header_data(&self) -> Token { Token::Tuple(vec![ - // batchNumber + // `batchNumber` Token::Uint(U256::from(self.header.number.0)), - // batchHash + // `batchHash` Token::FixedBytes(self.metadata.root_hash.as_bytes().to_vec()), - // indexRepeatedStorageChanges + // `indexRepeatedStorageChanges` Token::Uint(U256::from(self.metadata.rollup_last_leaf_index)), - // numberOfLayer1Txs + // `numberOfLayer1Txs` Token::Uint(U256::from(self.header.l1_tx_count)), - // priorityOperationsHash + // `priorityOperationsHash` Token::FixedBytes( self.header .priority_ops_onchain_data_hash() .as_bytes() .to_vec(), ), - // l2LogsTreeRoot + // `l2LogsTreeRoot` Token::FixedBytes(self.metadata.l2_l1_merkle_root.as_bytes().to_vec()), // timestamp Token::Uint(U256::from(self.header.timestamp)), @@ -193,24 +193,24 @@ impl L1BatchWithMetadata { ]) } else { Token::Tuple(vec![ - // batchNumber + // `batchNumber` Token::Uint(U256::from(self.header.number.0)), - // timestamp + // `timestamp` Token::Uint(U256::from(self.header.timestamp)), - // indexRepeatedStorageChanges + // `indexRepeatedStorageChanges` Token::Uint(U256::from(self.metadata.rollup_last_leaf_index)), - // newStateRoot + // `newStateRoot` Token::FixedBytes(self.metadata.merkle_root_hash.as_bytes().to_vec()), - // numberOfLayer1Txs + // `numberOfLayer1Txs` Token::Uint(U256::from(self.header.l1_tx_count)), - // priorityOperationsHash + // `priorityOperationsHash` Token::FixedBytes( self.header .priority_ops_onchain_data_hash() .as_bytes() .to_vec(), ), - // bootloaderHeapInitialContentsHash + // `bootloaderHeapInitialContentsHash` Token::FixedBytes( self.metadata .bootloader_initial_content_commitment @@ -218,7 +218,7 @@ impl L1BatchWithMetadata { .as_bytes() .to_vec(), ), - // eventsQueueStateHash + // `eventsQueueStateHash` Token::FixedBytes( self.metadata .events_queue_commitment @@ -226,9 +226,9 @@ impl L1BatchWithMetadata { .as_bytes() .to_vec(), ), - // systemLogs + // `systemLogs` Token::Bytes(self.metadata.l2_l1_messages_compressed.clone()), - // totalL2ToL1Pubdata + // `totalL2ToL1Pubdata` Token::Bytes( self.header .pubdata_input diff --git a/core/lib/zksync_core/src/consistency_checker/tests/mod.rs b/core/lib/zksync_core/src/consistency_checker/tests/mod.rs index e8cc9f2c26cc..bdea516cf9f4 100644 --- a/core/lib/zksync_core/src/consistency_checker/tests/mod.rs +++ b/core/lib/zksync_core/src/consistency_checker/tests/mod.rs @@ -49,9 +49,9 @@ fn build_commit_tx_input_data(batches: &[L1BatchWithMetadata]) -> Vec { let mut encoded = vec![]; // Fake Solidity function selector (not checked for now) encoded.extend_from_slice(b"fake"); - // Mock an additional arg used in real `commitBlocks` / `commitBatches`. In real transactions, - // it's taken from the L1 batch previous to batches[0], but since this arg is not checked, - // it's OK to use batches[0]. + // Mock an additional argument used in real `commitBlocks` / `commitBatches`. In real transactions, + // it's taken from the L1 batch previous to `batches[0]`, but since this argument is not checked, + // it's OK to use `batches[0]`. let prev_header_tokens = batches[0].l1_header_data(); encoded.extend_from_slice(ðabi::encode(&[prev_header_tokens, commit_tokens])); encoded @@ -101,8 +101,8 @@ fn build_commit_tx_input_data_is_correct() { fn extracting_commit_data_for_boojum_batch() { let contract = zksync_contracts::zksync_contract(); let commit_function = contract.function("commitBatches").unwrap(); - // Calldata taken from the commit transaction for https://sepolia.explorer.zksync.io/batch/4470; - // https://sepolia.etherscan.io/tx/0x300b9115037028b1f8aa2177abf98148c3df95c9b04f95a4e25baf4dfee7711f + // Calldata taken from the commit transaction for `https://sepolia.explorer.zksync.io/batch/4470`; + // `https://sepolia.etherscan.io/tx/0x300b9115037028b1f8aa2177abf98148c3df95c9b04f95a4e25baf4dfee7711f` let commit_tx_input_data = include_bytes!("commit_l1_batch_4470_testnet_sepolia.calldata"); let commit_data = ConsistencyChecker::extract_commit_data( @@ -131,8 +131,8 @@ fn extracting_commit_data_for_boojum_batch() { fn extracting_commit_data_for_multiple_batches() { let contract = zksync_contracts::zksync_contract(); let commit_function = contract.function("commitBatches").unwrap(); - // Calldata taken from the commit transaction for https://explorer.zksync.io/batch/351000; - // https://etherscan.io/tx/0xbd8dfe0812df0da534eb95a2d2a4382d65a8172c0b648a147d60c1c2921227fd + // Calldata taken from the commit transaction for `https://explorer.zksync.io/batch/351000`; + // `https://etherscan.io/tx/0xbd8dfe0812df0da534eb95a2d2a4382d65a8172c0b648a147d60c1c2921227fd` let commit_tx_input_data = include_bytes!("commit_l1_batch_351000-351004_mainnet.calldata"); for l1_batch in 351_000..=351_004 { @@ -161,8 +161,8 @@ fn extracting_commit_data_for_multiple_batches() { #[test] fn extracting_commit_data_for_pre_boojum_batch() { - // Calldata taken from the commit transaction for https://goerli.explorer.zksync.io/batch/200000; - // https://goerli.etherscan.io/tx/0xfd2ef4ccd1223f502cc4a4e0f76c6905feafabc32ba616e5f70257eb968f20a3 + // Calldata taken from the commit transaction for `https://goerli.explorer.zksync.io/batch/200000`; + // `https://goerli.etherscan.io/tx/0xfd2ef4ccd1223f502cc4a4e0f76c6905feafabc32ba616e5f70257eb968f20a3` let commit_tx_input_data = include_bytes!("commit_l1_batch_200000_testnet_goerli.calldata"); let commit_data = ConsistencyChecker::extract_commit_data( diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index d7e64a8928bb..73f484a1fd77 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -29,7 +29,7 @@ use crate::{ utils::testonly::create_l1_batch, }; -// Alias to conveniently call static methods of ETHSender. +// Alias to conveniently call static methods of `ETHSender`. type MockEthTxManager = EthTxManager; static DUMMY_OPERATION: Lazy = Lazy::new(|| { @@ -268,7 +268,7 @@ async fn resend_each_block() -> anyhow::Result<()> { assert_eq!(sent_tx.nonce, 0.into()); assert_eq!( sent_tx.max_fee_per_gas.unwrap() - sent_tx.max_priority_fee_per_gas.unwrap(), - 18.into() // 6 * 3 * 2^0 + 18.into() // `6 * 3 * 2^0` ); // now, median is 5 @@ -318,7 +318,7 @@ async fn resend_each_block() -> anyhow::Result<()> { assert_eq!(resent_tx.nonce, 0.into()); assert_eq!( resent_tx.max_fee_per_gas.unwrap() - resent_tx.max_priority_fee_per_gas.unwrap(), - 30.into() // 5 * 3 * 2^1 + 30.into() // `5 * 3 * 2^1` ); Ok(()) diff --git a/core/tests/cross_external_nodes_checker/src/checker.rs b/core/tests/cross_external_nodes_checker/src/checker.rs index 2aa8aa9c4e6c..0ddd179c2667 100644 --- a/core/tests/cross_external_nodes_checker/src/checker.rs +++ b/core/tests/cross_external_nodes_checker/src/checker.rs @@ -367,7 +367,7 @@ impl Checker { tracing::debug!("Maybe checking batch {}", miniblock_batch_number); // We should check batches only the first time we encounter them per instance - // (i.e., next_instance_batch_to_check == miniblock_batch_number) + // (i.e., `next_instance_batch_to_check == miniblock_batch_number`) match instance_batch_to_check.cmp(&miniblock_batch_number) { Greater => return Ok(()), // This batch has already been checked. Less => { diff --git a/core/tests/loadnext/src/account/mod.rs b/core/tests/loadnext/src/account/mod.rs index 46b5d04ba235..9a1243151d0e 100644 --- a/core/tests/loadnext/src/account/mod.rs +++ b/core/tests/loadnext/src/account/mod.rs @@ -136,7 +136,7 @@ impl AccountLifespan { let is_l1_transaction = matches!(command.command_type, TxType::L1Execute | TxType::Deposit); if is_l1_transaction && l1_tx_count >= MAX_L1_TRANSACTIONS { - continue; // Skip command to not run out of ethereum on L1 + continue; // Skip command to not run out of Ethereum on L1 } // The new transaction should be sent only if mempool is not full diff --git a/core/tests/loadnext/src/account/tx_command_executor.rs b/core/tests/loadnext/src/account/tx_command_executor.rs index 5c51a68376d5..599656cd034e 100644 --- a/core/tests/loadnext/src/account/tx_command_executor.rs +++ b/core/tests/loadnext/src/account/tx_command_executor.rs @@ -201,7 +201,7 @@ impl AccountLifespan { }?; // Update current nonce for future txs - // If the transaction has a tx_hash and is small enough to be included in a block, this tx will change the nonce. + // If the transaction has a `tx_hash` and is small enough to be included in a block, this tx will change the nonce. // We can be sure that the nonce will be changed based on this assumption. if let SubmitResult::TxHash(_) = &result { self.current_nonce = Some(nonce + 1) diff --git a/core/tests/loadnext/src/account_pool.rs b/core/tests/loadnext/src/account_pool.rs index 730a6d07b48d..a6ae8cd68163 100644 --- a/core/tests/loadnext/src/account_pool.rs +++ b/core/tests/loadnext/src/account_pool.rs @@ -90,7 +90,7 @@ impl AccountPool { /// Generates all the required test accounts and prepares `Wallet` objects. pub async fn new(config: &LoadtestConfig) -> anyhow::Result { let l2_chain_id = L2ChainId::try_from(config.l2_chain_id).unwrap(); - // Create a client for pinging the rpc. + // Create a client for pinging the RPC. let client = HttpClientBuilder::default() .build(&config.l2_rpc_address) .unwrap(); diff --git a/core/tests/loadnext/src/command/api.rs b/core/tests/loadnext/src/command/api.rs index 76ed5db57470..278f4e1a7496 100644 --- a/core/tests/loadnext/src/command/api.rs +++ b/core/tests/loadnext/src/command/api.rs @@ -73,7 +73,7 @@ async fn random_block_number(wallet: &SyncWallet, rng: &mut LoadtestRng) -> api: match block_number { BlockNumber::Committed => api::BlockNumber::Committed, BlockNumber::Number => { - // Choose a random block in the range [0, latest_committed_block_number). + // Choose a random block in the range `[0, latest_committed_block_number)`. match wallet .provider .get_block_by_number(api::BlockNumber::Committed, false) diff --git a/core/tests/loadnext/src/config.rs b/core/tests/loadnext/src/config.rs index 6620e3aed579..2cc6317059dc 100644 --- a/core/tests/loadnext/src/config.rs +++ b/core/tests/loadnext/src/config.rs @@ -152,8 +152,8 @@ fn default_l1_rpc_address() -> String { fn default_master_wallet_pk() -> String { // Use this key only for localhost because it is compromised! - // Using this key for rinkeby will result in losing rinkeby ETH. - // Corresponding wallet is 0x36615Cf349d7F6344891B1e7CA7C72883F5dc049 + // Using this key for Rinkeby will result in losing Rinkeby ETH. + // Corresponding wallet is `0x36615Cf349d7F6344891B1e7CA7C72883F5dc049` let result = "7726827caac94a7f9e1b160f7ea819f172f7b6f9d2a97f992c38edeab82d4110".to_string(); tracing::info!("Using default MASTER_WALLET_PK: {result}"); result @@ -181,7 +181,7 @@ fn default_main_token() -> H160 { // Read token addresses from `etc/tokens/localhost.json`. Use the first one // as a main token since all of them are suitable. - // 0xeb8f08a975Ab53E34D8a0330E0D34de942C95926 for rinkeby + // `0xeb8f08a975Ab53E34D8a0330E0D34de942C95926` for Rinkeby let tokens = read_tokens(Network::Localhost).expect("Failed to parse tokens file"); let main_token = tokens.first().expect("Loaded tokens list is empty"); tracing::info!("Main token: {main_token:?}"); @@ -225,7 +225,7 @@ fn default_seed() -> Option { } fn default_l2_chain_id() -> u64 { - // 270 for rinkeby + // 270 for Rinkeby let result = L2ChainId::default().as_u64(); tracing::info!("Using default L2_CHAIN_ID: {result}"); result @@ -236,14 +236,14 @@ pub fn get_default_l2_rpc_address() -> String { } fn default_l2_rpc_address() -> String { - // https://z2-dev-api.zksync.dev:443 for stage2 + // `https://z2-dev-api.zksync.dev:443` for stage2 let result = get_default_l2_rpc_address(); tracing::info!("Using default L2_RPC_ADDRESS: {result}"); result } fn default_l2_ws_rpc_address() -> String { - // ws://z2-dev-api.zksync.dev:80/ws for stage2 + // `ws://z2-dev-api.zksync.dev:80/ws` for stage2 let result = "ws://127.0.0.1:3051".to_string(); tracing::info!("Using default L2_WS_RPC_ADDRESS: {result}"); result diff --git a/core/tests/loadnext/src/executor.rs b/core/tests/loadnext/src/executor.rs index d6e233a1f491..e87fd22b867e 100644 --- a/core/tests/loadnext/src/executor.rs +++ b/core/tests/loadnext/src/executor.rs @@ -54,7 +54,7 @@ impl Executor { ) -> anyhow::Result { let pool = AccountPool::new(&config).await?; - // derive l2 main token address + // derive L2 main token address let l2_main_token = pool .master_wallet .ethereum(&config.l1_rpc_address) diff --git a/core/tests/loadnext/src/report_collector/metrics_collector.rs b/core/tests/loadnext/src/report_collector/metrics_collector.rs index 6c5867901b12..90775f039ebf 100644 --- a/core/tests/loadnext/src/report_collector/metrics_collector.rs +++ b/core/tests/loadnext/src/report_collector/metrics_collector.rs @@ -151,7 +151,7 @@ mod tests { #[test] fn histogram_window_size() { - // Vector of ((window_idx, window_size), expected_range)). + // Vector of `((window_idx, window_size), expected_range))`. let test_vector = [ ((0, 100), (0, 99)), ((1, 100), (100, 199)), diff --git a/core/tests/loadnext/src/rng.rs b/core/tests/loadnext/src/rng.rs index 3612a7c2a847..20f163e69ee2 100644 --- a/core/tests/loadnext/src/rng.rs +++ b/core/tests/loadnext/src/rng.rs @@ -45,7 +45,7 @@ impl LoadtestRng { // We chain the current seed bytes and the Ethereum private key together, // and then calculate the hash of this data. // This way we obtain a derived seed, unique for each wallet, which will result in - // an uniques set of operations for each account. + // an unique set of operations for each account. let input_bytes: Vec = self .seed .iter() diff --git a/core/tests/vm-benchmark/harness/src/lib.rs b/core/tests/vm-benchmark/harness/src/lib.rs index 33042adaaeb8..08d52dd67a69 100644 --- a/core/tests/vm-benchmark/harness/src/lib.rs +++ b/core/tests/vm-benchmark/harness/src/lib.rs @@ -39,7 +39,7 @@ pub fn cut_to_allowed_bytecode_size(bytes: &[u8]) -> Option<&[u8]> { static STORAGE: Lazy = Lazy::new(|| { let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); - // give PRIVATE_KEY some money + // give `PRIVATE_KEY` some money let my_addr = PackedEthSignature::address_from_private_key(&PRIVATE_KEY).unwrap(); let key = storage_key_for_eth_balance(&my_addr); storage.set_value(key, zksync_utils::u256_to_h256(U256([0, 0, 1, 0]))); diff --git a/sdk/zksync-rs/src/ethereum/mod.rs b/sdk/zksync-rs/src/ethereum/mod.rs index a21c7727cb1f..26b76017ed83 100644 --- a/sdk/zksync-rs/src/ethereum/mod.rs +++ b/sdk/zksync-rs/src/ethereum/mod.rs @@ -39,7 +39,7 @@ const ZKSYNC_INTERFACE: &str = include_str!("../abi/IZkSync.json"); const L1_DEFAULT_BRIDGE_INTERFACE: &str = include_str!("../abi/IL1Bridge.json"); const RAW_ERC20_DEPOSIT_GAS_LIMIT: &str = include_str!("DepositERC20GasLimit.json"); -// The gasPerPubdata to be used in L1->L2 requests. It may be almost any number, but here we 800 +// The `gasPerPubdata` to be used in L1->L2 requests. It may be almost any number, but here we 800 // as an optimal one. In the future, it will be estimated. const L1_TO_L2_GAS_PER_PUBDATA: u32 = 800; @@ -72,7 +72,7 @@ pub struct EthereumProvider { polling_interval: Duration, } -// TODO (SMA-1623): create a way to pass `Options` (e.g. nonce, gas_limit, priority_fee_per_gas) +// TODO (SMA-1623): create a way to pass `Options` (e.g. `nonce`, `gas_limit`, `priority_fee_per_gas`) // into methods that perform L1 transactions. The unit is wei. pub const DEFAULT_PRIORITY_FEE: u64 = 2_000_000_000; @@ -472,7 +472,7 @@ impl EthereumProvider { let mut options = eth_options.unwrap_or_default(); - // If the user has already provided max_fee_per_gas or gas_price, we will use + // If the user has already provided `max_fee_per_gas` or `gas_price`, we will use // it to calculate the base cost for the transaction let gas_price = if let Some(max_fee_per_gas) = options.max_fee_per_gas { max_fee_per_gas diff --git a/spellcheck/era.cfg b/spellcheck/era.cfg index c00c2d7a64ce..ea5b65543f8b 100644 --- a/spellcheck/era.cfg +++ b/spellcheck/era.cfg @@ -2,7 +2,7 @@ # ${CARGO_MANIFEST_DIR}/.config/spellcheck.toml # Also take into account developer comments -dev_comments = false +dev_comments = true # Skip the README.md file as defined in the cargo manifest skip_readme = false diff --git a/spellcheck/era.dic b/spellcheck/era.dic index db9f1e021bfd..9bc9025729cd 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -863,3 +863,4 @@ tokenized Aggregator DecommittmentProcessor decommitment +hardcoded \ No newline at end of file From 5129b825ff811f3fb167bf43af241a249ab13555 Mon Sep 17 00:00:00 2001 From: Maksym Date: Sun, 7 Jan 2024 22:15:32 +0200 Subject: [PATCH 27/49] ci: add arm64 build for en (#811) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Refactored workflows with multiplatform support ## Why ❔ For multiplatform builds ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .github/workflows/build-core-template.yml | 82 ++++++++++++++------- .github/workflows/build-docker-from-tag.yml | 2 - .github/workflows/build-prover-template.yml | 7 +- .github/workflows/ci.yml | 2 - .github/workflows/release-test-stage.yml | 2 - infrastructure/zk/src/docker.ts | 39 +++++----- 6 files changed, 77 insertions(+), 57 deletions(-) diff --git a/.github/workflows/build-core-template.yml b/.github/workflows/build-core-template.yml index 727f4028817a..1ad14918af75 100644 --- a/.github/workflows/build-core-template.yml +++ b/.github/workflows/build-core-template.yml @@ -9,10 +9,6 @@ on: description: "DOCKERHUB_TOKEN" required: true inputs: - image_tag: - description: "Tag of a built image to deploy" - type: string - required: true image_tag_suffix: description: "Optional suffix to override tag name generation" type: string @@ -26,27 +22,21 @@ jobs: build-images: name: Build and Push Docker Images env: - image_tag: ${{ inputs.image_tag }} IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} - runs-on: ${{ matrix.component.runner }} + runs-on: ${{ fromJSON('["matterlabs-ci-runner", "matterlabs-ci-runner-arm"]')[contains(matrix.platforms, 'arm')] }} strategy: matrix: - component: - - name: server-v2 - platforms: linux/amd64 - runner: matterlabs-ci-runner - - name: external-node - platforms: linux/amd64 - runner: matterlabs-ci-runner - - name: contract-verifier - platforms: linux/amd64 - runner: matterlabs-ci-runner - - name: cross-external-nodes-checker - platforms: linux/amd64 - runner: matterlabs-ci-runner - - name: snapshots-creator - platforms: linux/amd64 - runner: matterlabs-ci-runner + components: + - server-v2 + - external-node + - contract-verifier + - cross-external-nodes-checker + - snapshots-creator + platforms: + - linux/amd64 + include: + - components: external-node + platforms: linux/arm64 steps: - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 @@ -88,18 +78,58 @@ jobs: if: github.event_name != 'pull_request' && (github.ref == 'refs/heads/main' || startsWith(github.ref, 'refs/tags/')) run: | ci_run docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} - ci_run gcloud auth configure-docker us-docker.pkg.dev,asia-docker.pkg.dev,europe-docker.pkg.dev -q + ci_run gcloud auth configure-docker us-docker.pkg.dev -q - name: update-images env: DOCKER_ACTION: ${{ inputs.action }} - COMPONENT: ${{ matrix.component.name }} - PLATFORMS: ${{ matrix.component.platforms }} + COMPONENT: ${{ matrix.components }} + PLATFORM: ${{ matrix.platforms }} run: | ci_run rustup default nightly-2023-08-21 - ci_run zk docker $DOCKER_ACTION --platforms=${PLATFORMS} $COMPONENT + ci_run zk docker $DOCKER_ACTION --platform=${PLATFORM} $COMPONENT - name: Show sccache stats if: always() run: | ci_run sccache --show-stats ci_run cat /tmp/sccache_log.txt + + create_manifest: + name: Create release manifest + runs-on: matterlabs-ci-runner + needs: build-images + if: ${{ inputs.action == 'push' }} + strategy: + matrix: + component: + - name: server-v2 + platform: linux/amd64 + - name: external-node + platform: linux/amd64,linux/arm64 + - name: contract-verifier + platform: linux/amd64 + - name: cross-external-nodes-checker + platform: linux/amd64 + - name: snapshots-creator + platform: linux/amd64 + env: + IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} + steps: + - name: login to Docker registries + run: | + docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} + gcloud auth configure-docker us-docker.pkg.dev -q + + - name: Create Docker manifest + run: | + docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/${{ matrix.component.name }}") + for repo in "${docker_repositories[@]}"; do + repo_tags=() + for platform in ${{ matrix.component.platform }}; do + platform=$(echo $platform | tr '/' '-') + repo_tags+=("--amend ${repo}:${IMAGE_TAG_SUFFIX}-${platform}") + done + + docker manifest create "${repo}:${IMAGE_TAG_SUFFIX}" "${repo_tags[@]}" + docker manifest push "${repo}:${IMAGE_TAG_SUFFIX}" + done diff --git a/.github/workflows/build-docker-from-tag.yml b/.github/workflows/build-docker-from-tag.yml index 9f01a2b56f3a..023aff6cd95a 100644 --- a/.github/workflows/build-docker-from-tag.yml +++ b/.github/workflows/build-docker-from-tag.yml @@ -63,7 +63,6 @@ jobs: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} build-push-prover-images: @@ -72,7 +71,6 @@ jobs: uses: ./.github/workflows/build-prover-template.yml if: contains(github.ref_name, 'prover') with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} CUDA_ARCH: "60;70;75;89" diff --git a/.github/workflows/build-prover-template.yml b/.github/workflows/build-prover-template.yml index ca8792a62054..023a90076641 100644 --- a/.github/workflows/build-prover-template.yml +++ b/.github/workflows/build-prover-template.yml @@ -13,10 +13,6 @@ on: description: "ERA_BELLMAN_CUDA_RELEASE" type: string required: true - image_tag: - description: "Tag of a built image to deploy" - type: string - required: true image_tag_suffix: description: "Optional suffix to override tag name generation" type: string @@ -41,7 +37,6 @@ jobs: build-images: name: Build and Push Docker Images env: - image_tag: ${{ inputs.image_tag }} IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} RUNNER_COMPOSE_FILE: "docker-compose-runner-nightly.yml" ERA_BELLMAN_CUDA_RELEASE: ${{ inputs.ERA_BELLMAN_CUDA_RELEASE }} @@ -93,7 +88,7 @@ jobs: if: github.event_name != 'pull_request' && (github.ref == 'refs/heads/main' || startsWith(github.ref, 'refs/tags/')) run: | ci_run docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} - ci_run gcloud auth configure-docker us-docker.pkg.dev,asia-docker.pkg.dev,europe-docker.pkg.dev -q + ci_run gcloud auth configure-docker us-docker.pkg.dev -q # We need to run this only when ERA_BELLMAN_CUDA_RELEASE is not available # In our case it happens only when PR is created from fork diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3451451bb025..f15efa8a6a35 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -108,7 +108,6 @@ jobs: if: ${{ (needs.changed_files.outputs.core == 'true' || needs.changed_files.outputs.all == 'true') && !contains(github.ref_name, 'release-please--branches') }} uses: ./.github/workflows/build-core-template.yml with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} action: "build" secrets: @@ -121,7 +120,6 @@ jobs: if: ${{ (needs.changed_files.outputs.prover == 'true' || needs.changed_files.outputs.all == 'true') && !contains(github.ref_name, 'release-please--branches') }} uses: ./.github/workflows/build-prover-template.yml with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} action: "build" ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} diff --git a/.github/workflows/release-test-stage.yml b/.github/workflows/release-test-stage.yml index 1b942f7c5f6e..fae77fca0b53 100644 --- a/.github/workflows/release-test-stage.yml +++ b/.github/workflows/release-test-stage.yml @@ -74,7 +74,6 @@ jobs: uses: ./.github/workflows/build-core-template.yml if: needs.changed_files.outputs.core == 'true' || needs.changed_files.outputs.all == 'true' with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} @@ -86,7 +85,6 @@ jobs: uses: ./.github/workflows/build-prover-template.yml if: needs.changed_files.outputs.prover == 'true' || needs.changed_files.outputs.all == 'true' with: - image_tag: ${{ needs.setup.outputs.image_tag }} image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} CUDA_ARCH: "60;70;75;89" diff --git a/infrastructure/zk/src/docker.ts b/infrastructure/zk/src/docker.ts index 89eac3be45e4..7b571ebcc344 100644 --- a/infrastructure/zk/src/docker.ts +++ b/infrastructure/zk/src/docker.ts @@ -27,7 +27,7 @@ const UNIX_TIMESTAMP = Date.now(); async function dockerCommand( command: 'push' | 'build', image: string, - platforms: string[] = ['linux/amd64'], + platform: string = '', customTag?: string, buildExtraArgs: string = '', dockerOrg: string = 'matterlabs' @@ -41,8 +41,8 @@ async function dockerCommand( // We want an alternative flow for Rust image if (image == 'rust') { - await dockerCommand(command, 'server-v2', platforms, customTag, dockerOrg); - await dockerCommand(command, 'prover', platforms, customTag, dockerOrg); + await dockerCommand(command, 'server-v2', platform, customTag, dockerOrg); + await dockerCommand(command, 'prover', platform, customTag, dockerOrg); return; } if (!IMAGES.includes(image)) { @@ -58,7 +58,7 @@ async function dockerCommand( // Main build\push flow switch (command) { case 'build': - await _build(image, tagList, dockerOrg, platforms, buildExtraArgs); + await _build(image, tagList, dockerOrg, platform, buildExtraArgs); break; default: console.log(`Unknown command for docker ${command}.`); @@ -89,25 +89,27 @@ function defaultTagList(image: string, imageTagSha: string, imageTagShaTS: strin return tagList; } -async function _build( - image: string, - tagList: string[], - dockerOrg: string, - platforms: string[], - extraArgs: string = '' -) { +async function _build(image: string, tagList: string[], dockerOrg: string, platform: string, extraArgs: string = '') { let tagsToBuild = ''; for (const tag of tagList) { for (const registry of DOCKER_REGISTRIES) { - tagsToBuild = tagsToBuild + `-t ${registry}/${image}:${tag} `; + if (platform != '') { + let platformSuffix = platform.replace('/', '-'); + tagsToBuild = tagsToBuild + `-t ${registry}/${image}:${tag}-${platformSuffix} `; + } else { + tagsToBuild = tagsToBuild + `-t ${registry}/${image}:${tag} `; + } } } let buildArgs = ''; + if (platform != '') { + buildArgs += `--platform=${platform} `; + } if (image === 'prover-v2') { const eraBellmanCudaRelease = process.env.ERA_BELLMAN_CUDA_RELEASE; - buildArgs += `--build-arg ERA_BELLMAN_CUDA_RELEASE=${eraBellmanCudaRelease}`; + buildArgs += `--build-arg ERA_BELLMAN_CUDA_RELEASE=${eraBellmanCudaRelease} `; } if (image === 'prover-gpu-fri') { const cudaArch = process.env.CUDA_ARCH; @@ -119,7 +121,6 @@ async function _build( const buildCommand = `DOCKER_BUILDKIT=1 docker buildx build ${tagsToBuild}` + - ` --platform=${platforms.join(',')}` + (buildArgs ? ` ${buildArgs}` : '') + ` -f ./docker/${imagePath}/Dockerfile .`; @@ -127,15 +128,15 @@ async function _build( } export async function build(image: string, cmd: Command) { - await dockerCommand('build', image, cmd.platforms, cmd.customTag); + await dockerCommand('build', image, cmd.platform, cmd.customTag); } export async function customBuildForHyperchain(image: string, dockerOrg: string) { - await dockerCommand('build', image, ['linux/amd64'], dockerOrg); + await dockerCommand('build', image, 'linux/amd64', dockerOrg); } export async function push(image: string, cmd: Command) { - await dockerCommand('build', image, cmd.platforms, cmd.customTag, '--push'); + await dockerCommand('build', image, cmd.platform, cmd.customTag, '--push'); } export async function restart(container: string) { @@ -151,13 +152,13 @@ export const command = new Command('docker').description('docker management'); command .command('build ') .option('--custom-tag ', 'Custom tag for image') - .option('--platforms ', 'Comma-separated list of platforms', (val) => val.split(','), ['linux/amd64']) + .option('--platform ', 'Docker platform') .description('build docker image') .action(build); command .command('push ') .option('--custom-tag ', 'Custom tag for image') - .option('--platforms ', 'Comma-separated list of platforms', (val) => val.split(','), ['linux/amd64']) + .option('--platform ', 'Docker platform') .action(push); command.command('pull').description('pull all containers').action(pull); command.command('restart ').description('restart container in docker-compose.yml').action(restart); From e6502466db93655c6ef9ab42e376e4c69f772a89 Mon Sep 17 00:00:00 2001 From: Maksym Kryva Date: Sun, 7 Jan 2024 22:46:19 +0200 Subject: [PATCH 28/49] ci: fix foop for merge docker manifest --- .github/workflows/build-core-template.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build-core-template.yml b/.github/workflows/build-core-template.yml index 1ad14918af75..dd773b25074e 100644 --- a/.github/workflows/build-core-template.yml +++ b/.github/workflows/build-core-template.yml @@ -123,9 +123,10 @@ jobs: - name: Create Docker manifest run: | docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/${{ matrix.component.name }}") + platforms=${{ matrix.component.platform }} for repo in "${docker_repositories[@]}"; do repo_tags=() - for platform in ${{ matrix.component.platform }}; do + for platform in ${platforms//,/ }; do platform=$(echo $platform | tr '/' '-') repo_tags+=("--amend ${repo}:${IMAGE_TAG_SUFFIX}-${platform}") done From 3f1add8b1be49f90fb03847a5c741bc2600e4a89 Mon Sep 17 00:00:00 2001 From: Maksym Kryva Date: Sun, 7 Jan 2024 23:16:41 +0200 Subject: [PATCH 29/49] ci: revert arm64 and merge manifest job --- .github/workflows/build-core-template.yml | 46 +---------------------- 1 file changed, 1 insertion(+), 45 deletions(-) diff --git a/.github/workflows/build-core-template.yml b/.github/workflows/build-core-template.yml index dd773b25074e..c27829968f59 100644 --- a/.github/workflows/build-core-template.yml +++ b/.github/workflows/build-core-template.yml @@ -34,9 +34,6 @@ jobs: - snapshots-creator platforms: - linux/amd64 - include: - - components: external-node - platforms: linux/arm64 steps: - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 @@ -87,50 +84,9 @@ jobs: PLATFORM: ${{ matrix.platforms }} run: | ci_run rustup default nightly-2023-08-21 - ci_run zk docker $DOCKER_ACTION --platform=${PLATFORM} $COMPONENT + ci_run zk docker $DOCKER_ACTION $COMPONENT - name: Show sccache stats if: always() run: | ci_run sccache --show-stats ci_run cat /tmp/sccache_log.txt - - create_manifest: - name: Create release manifest - runs-on: matterlabs-ci-runner - needs: build-images - if: ${{ inputs.action == 'push' }} - strategy: - matrix: - component: - - name: server-v2 - platform: linux/amd64 - - name: external-node - platform: linux/amd64,linux/arm64 - - name: contract-verifier - platform: linux/amd64 - - name: cross-external-nodes-checker - platform: linux/amd64 - - name: snapshots-creator - platform: linux/amd64 - env: - IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} - steps: - - name: login to Docker registries - run: | - docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} - gcloud auth configure-docker us-docker.pkg.dev -q - - - name: Create Docker manifest - run: | - docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/${{ matrix.component.name }}") - platforms=${{ matrix.component.platform }} - for repo in "${docker_repositories[@]}"; do - repo_tags=() - for platform in ${platforms//,/ }; do - platform=$(echo $platform | tr '/' '-') - repo_tags+=("--amend ${repo}:${IMAGE_TAG_SUFFIX}-${platform}") - done - - docker manifest create "${repo}:${IMAGE_TAG_SUFFIX}" "${repo_tags[@]}" - docker manifest push "${repo}:${IMAGE_TAG_SUFFIX}" - done From 7a07f91fb27ec734fc2cc0c78b14de99e01a8498 Mon Sep 17 00:00:00 2001 From: Fedor Sakharov Date: Mon, 8 Jan 2024 10:11:25 +0100 Subject: [PATCH 30/49] chore(dal): bumps sqlx dependency to 0.7.3 (#725) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Bumps `sqlx` dependency to `0.7.3`. ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Alex Ostrovski --- .github/workflows/ci-core-lint-reusable.yml | 2 +- .github/workflows/ci-core-reusable.yml | 4 + Cargo.lock | 652 +- core/bin/external_node/src/main.rs | 9 +- core/lib/constants/Cargo.toml | 2 +- ...f6e1df560ab1e8935564355236e90b6147d2f.json | 15 + ...381d82925556a4801f678b8786235d525d784.json | 16 + ...07703b2581dda4fe3c00be6c5422c78980c4b.json | 20 + ...e57a83f37da8999849377dfad60b44989be39.json | 107 + ...82d56bbace9013f1d95ea0926eef1fb48039b.json | 34 + ...a1b4ba7fe355ebc02ea49a054aa073ce324ba.json | 15 + ...4f3670813e5a5356ddcb7ac482a0201d045f7.json | 108 + ...c2563a3e061bcc6505d8c295b1b2517f85f1b.json | 20 + ...c7e3b6bdb2a08149d5f443310607d3c78988c.json | 22 + ...2b5ea100f59295cd693d14ee0d5ee089c7981.json | 20 + ...994723ebe57b3ed209069bd3db906921ef1a3.json | 28 + ...1a2ee78eb59628d0429e89b27d185f83512be.json | 28 + ...33d196907ebd599e926d305e5ef9f26afa2fa.json | 24 + ...4710daa723e2d9a23317c347f6081465c3643.json | 52 + ...1e4ee6682a89fb86f3b715a240805d44e6d87.json | 15 + ...c6f03469d78bf4f45f5fd1eaf37171db2f04a.json | 20 + ...dcde721ca1c652ae2f8db41fb75cecdecb674.json | 22 + ...fced141fb29dd8b6c32dd2dc3452dc294fe1f.json | 23 + ...e5f7edcafa4fc6757264a96a46dbf7dd1f9cc.json | 31 + ...8048558243ff878b59586eb3a8b22794395d8.json | 259 + ...bc6e326e15dca141050bc9d7dacae98a430e8.json | 22 + ...dfbf962636347864fc365fafa46c7a2da5f30.json | 22 + ...a52daa202279bf612a9278f663deb78bc6e41.json | 22 + ...986511265c541d81b1d21f0a751ae1399c626.json | 72 + ...661a097308d9f81024fdfef24a14418202730.json | 22 + ...700a95e4c37a7a18531b3cdf120394cb055b9.json | 22 + ...aff3a06b7a9c1866132d62e4449fa9436c7c4.json | 15 + ...263298abc0521734f807fbf45484158b167b2.json | 20 + ...c68e8e15a831f1a45dc3b2240d5c6809d5ef2.json | 82 + ...28a20420763a3a22899ad0e5f4b953b615a9e.json | 25 + ...3c17c7a7939195dd2d9aa42512b1479d2ed20.json | 94 + ...a2d505a1aabf52ff4136d2ed1b39c70dd1632.json | 230 + ...a36408510bb05c15285dfa7708bc0b044d0c5.json | 259 + ...bfdf10fb1cb43781219586c169fb976204331.json | 22 + ...9f3f0b5d629fdb5c36ea1bfb93ed246be968e.json | 88 + ...f50e423eb6134659fe52bcc2b27ad16740c96.json | 14 + ...7ea964610e131d45f720f7a7b2a130fe9ed89.json | 17 + ...5d98a3d9f7df162ae22a98c3c7620fcd13bd2.json | 80 + ...62786c58e54f132588c48f07d9537cf21d3ed.json | 22 + ...7b84c5fd52f0352f0f0e311d054cb9e45b07e.json | 22 + ...0a01a6c7cbe9297cbb55d42533fddc18719b6.json | 20 + ...1610ffa7f169d560e79e89b99eedf681c6773.json | 16 + ...0fa6d65c05d8144bdfd143480834ead4bd8d5.json | 76 + ...4d26e48e968c79834858c98b7a7f9dfc81910.json | 14 + ...be3b85419fde77038dd3de6c55aa9c55d375c.json | 61 + ...5f6760cfeac185330d1d9c5cdb5b383ed8ed4.json | 30 + ...099c4591ce3f8d51f3df99db0165e086f96af.json | 22 + ...a5a088ceebfef0228701fcbed8255b74b1050.json | 232 + ...5fc7f60618209d0132e7937a1347b7e94b212.json | 20 + ...710ce910d0c9138d85cb55e16459c757dea03.json | 20 + ...b2d0413d0f89c309de4b31254c309116ea60c.json | 17 + ...de995ddddc621ee2149f08f905af2d8aadd03.json | 34 + ...492cf2545944291dd0d42b937c0d0c7eefd47.json | 106 + ...cd9b0128f82306e27e4c7258116082a54ab95.json | 29 + ...b19a3a7feec950cb3e503588cf55d954a493a.json | 22 + ...8eeab159533211d2ddbef41e6ff0ba937d04a.json | 14 + ...16065fcad42c6b621efb3a135a16b477dcfd9.json | 86 + ...6ca08324bc86a72e4ea85c9c7962a6c8c9e30.json | 16 + ...5234d536b064951a67d826ac49b7a3a095a1a.json | 28 + ...717b0dd3cec64b0b29f59b273f1c6952e01da.json | 22 + ...eda1da000848ed4abf309b68989da33e1ce5a.json | 124 + ...07997b7e24682da56f84d812da2b9aeb14ca2.json | 40 + ...6381a9c4165e2727542eaeef3bbedd13a4f20.json | 15 + ...42d4ef93ca238fd2fbc9155e6d012d0d1e113.json | 15 + ...23efc18e51e6d4092188e081f4fafe92fe0ef.json | 34 + ...095596297be0d728a0aa2d9b180fb72de222b.json | 22 + ...83f115cd97586bd795ee31f58fc14e56d58cb.json | 14 + ...52964b268b93c32fd280c49bf9f932884300d.json | 20 + ...3595fe783b7b894748d07fed9735c58fb28fb.json | 20 + ...52554ccfb5b83f00efdc12bed0f60ef439785.json | 25 + ...f45d57f1f8b2558fdb374bf02e84d3c825a23.json | 20 + ...bffc78d92adb3c1e3ca60b12163e38c67047e.json | 22 + ...feb5f094160627bc09db4bda2dda9a8c11c44.json | 15 + ...5fb7a093b73727f75e0cb7db9cea480c95f5c.json | 35 + ...76085385c3a79d72f49669b4215a9454323c2.json | 22 + ...78f42a877039094c3d6354928a03dad29451a.json | 15 + ...40e73a2778633bbf9d8e591ec912634639af9.json | 232 + ...eb89c728357c0744f209090824695a5d35fa3.json | 22 + ...4d6b3b1d15b818974e4e403f4ac418ed40c2c.json | 26 + ...119a0a60dc9804175b2baf8b45939c74bd583.json | 15 + ...05d7d1c9e649663f6e9444c4425821d0a5b71.json | 22 + ...da2b4afc314a3b3e78b041b44c8a020a0ee12.json | 14 + ...5ef6676b2eac623c0f78634cae9072fe0498a.json | 16 + ...85d4402ff6005446bbded68c8d3dbca510f37.json | 26 + ...95f3823926f69cb1d54874878e6d6c301bfd8.json | 20 + ...e9ebdda648667d48d6b27ddf155f2fe01d77a.json | 16 + ...b04ba5868d1897553df85aac15b169ebb0732.json | 28 + ...22ff6372f63ecadb504a329499b02e7d3550e.json | 26 + ...26b0b3185a231bbf0b8b132a1a95bc157e827.json | 34 + ...f9044ae85b579c7051301b40bd5f94df1f530.json | 15 + ...7e5afa28d855d87ea2f8c93e79c436749068a.json | 258 + ...41a21691357494c2f9919bfcbcbb0e42315c2.json | 22 + ...e2f7e49cd6600bd3d56cf41795c2c9e082e4c.json | 20 + ...279601234a489f73d843f2f314252ed4cb8b0.json | 28 + ...ac77b63879ab97a32eed8392b48cc46116a28.json | 14 + ...96ec397767a063dc21aa3add974cb9b070361.json | 16 + ...6387f2f3eda07a630503324bd6576dbdf8231.json | 22 + ...6769dbb04d3a61cf232892236c974660ffe64.json | 35 + ...489745438eae73a07b577aa25bd08adf95354.json | 16 + ...8b87ead36f593488437c6f67da629ca81e4fa.json | 14 + ...4cee5c602d275bb812022cc8fdabf0a60e151.json | 56 + ...4de79b67833f17d252b5b0e8e00cd6e75b5c1.json | 20 + ...5da82065836fe17687ffad04126a6a8b2b27c.json | 15 + ...7931a02fe5ffaf2c4dc2f1e7a48c0e932c060.json | 50 + ...77cb8f1340b483faedbbc2b71055aa5451cae.json | 20 + ...f82918afa1b0ac410325e4d00cd1c4fdd1e8b.json | 134 + ...d772d8801b0ae673b7173ae08a1fa6cbf67b2.json | 59 + ...6ea7ef2c1178b1b0394a17442185b79f2d77d.json | 22 + ...c6d47e8e1b4b5124c82c1f35d405204fcf783.json | 82 + ...afca34d61347e0e2e29fb07ca3d1b8b1f309c.json | 18 + ...37d8d542b4f14cf560972c005ab3cc13d1f63.json | 23 + ...ec427a099492905a1feee512dc43f39d10047.json | 15 + ...9a8f447824a5ab466bb6eea1710e8aeaa2c56.json | 15 + ...a7ea81d8d10c76bbae540013667e13230e2ea.json | 22 + ...d94f28b7b2b60d551d552a9b0bab1f1791e39.json | 22 + ...a8d4768c6a803a1a90889e5a1b8254c315231.json | 22 + ...252f26335699a4e7f0347d7e894320245271d.json | 15 + ...ecd50a04799ffd36be0e17c56f47fcdbc8f60.json | 14 + ...40e7eb9f598999c28015a504605f88bf84b33.json | 88 + ...6009503182c300877e74a8539f231050e6f85.json | 15 + ...5bb2036d21f60d4c6934f9145730ac35c04de.json | 20 + ...63f089c91aead2bc9abb284697e65840f1e8f.json | 16 + ...5db07667280abef27cc73159d2fd9c95b209b.json | 256 + ...49374a2d931e62615bc7e6ecd0af957b42a13.json | 88 + ...97b16599abaf51df0f19effe1a536376cf6a6.json | 28 + ...7effac442434c6e734d977e6682a7484abe7f.json | 35 + ...cd0bc0e8923a6bae64f22f09242766835ee0c.json | 74 + ...3d55ff2b1510868dfe80d14fffa3f5ff07b83.json | 15 + ...52aeb5f06c26f68d131dd242f6ed68816c513.json | 22 + ...f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49.json | 20 + ...3ce80f9b2b27758651ccfc09df61a4ae8a363.json | 88 + ...8927f9e976319a305e0928ff366d535a97104.json | 92 + ...843eb48b5e26ee92cd8476f50251e8c32610d.json | 26 + ...b734d5d255e36e48668b3bfc25a0f86ea52e7.json | 40 + ...e7ee767e4c98706246eb113498c0f817f5f38.json | 17 + ...91a8d67abcdb743a619ed0d1b9c16206a3c20.json | 12 + ...3260b49ce42daaf9dbe8075daf0a8e0ad9914.json | 12 + ...ab01cb2c55bf86d2c8c99abb1b7ca21cf75c0.json | 14 + ...a896e34b757421fca9682aecb3a98b54695a6.json | 86 + ...8a58479487686d774e6b2b1391347bdafe06d.json | 29 + ...ad7fe3464f2619cee2d583accaa829aa12b94.json | 38 + ...2826095e9290f0c1157094bd0c44e06012e42.json | 232 + ...98129b44534062f524823666ed432d2fcd345.json | 12 + ...f396f9d193615d70184d4327378a7fc8a5665.json | 30 + ...26313c255ce356a9a4e25c098484d3129c914.json | 14 + ...7f92c4cfad9313b1461e12242d9becb59e0b0.json | 22 + ...ce56b53cc4ca0a8c6ee7cac1b9a5863000be3.json | 256 + ...8bf92c7f2c55386c8208e3a82b30456abd5b4.json | 90 + ...4a42b4ead6964edd17bfcacb4a828492bba60.json | 20 + ...dcf2324f380932698386aa8f9751b1fa24a7b.json | 15 + ...6bf94a013e49bf50ce23f4de4968781df0796.json | 15 + ...26dc7bb98e0f7feaf14656307e20bd2bb0b6c.json | 14 + ...1caa4cca66d6ad74b2cd1a34ea5f7bc1e6909.json | 28 + ...715e903f3b399886c2c73e838bd924fed6776.json | 18 + ...50d625f057acf1440b6550375ce5509a816a6.json | 107 + ...b139e943f3ad2a41387b610606a42b7f03283.json | 29 + ...7227120a8279db1875d26ccae5ee0785f46a9.json | 22 + ...1578db18c29cdca85b8b6aad86fe2a9bf6bbe.json | 32 + ...77b39429d56072f63b3530c576fb31d7a56f9.json | 18 + ...534014b9ab9ca7725e14fb17aa050d9f35eb8.json | 23 + ...f03c0aad3ac30d85176e0a4e35f72bbb21b12.json | 256 + ...19a00c170cf7725d95dd6eb8b753fa5facec8.json | 235 + ...0c5f885240d99ea69140a4636b00469d08497.json | 22 + ...c70aff49e60b7def3d93b9206e650c259168b.json | 20 + ...63595237a0c54f67b8c669dfbb4fca32757e4.json | 20 + ...1af63b86e8da6d908d48998012231e1d66a60.json | 29 + ...b4207b69cc48b4ba648e797211631be612b69.json | 28 + ...567e2667c63a033baa6b427bd8a0898c08bf2.json | 22 + ...02c690c33686c889d318b1d64bdd7fa6374db.json | 20 + ...c8dc4bd57e32dfefe1017acba9a15fc14b895.json | 36 + ...cbb724af0f0216433a70f19d784e3f2afbc9f.json | 22 + ...b8ba0b771cb45573aca81db254f6bcfc17c77.json | 20 + ...b17305343df99ebc55f240278b5c4e63f89f5.json | 22 + ...2767e112b95f4d103c27acd6f7ede108bd300.json | 16 + ...4125a2633ddb6abaa129f2b12389140d83c3f.json | 40 + ...f0a9676e26f422506545ccc90b7e8a36c8d47.json | 35 + ...d9b6a1a6fe629f9a02c8e33330a0efda64068.json | 32 + ...4561347b81f8931cc2addefa907c9aa9355e6.json | 82 + ...1e30606fddf6631c859ab03819ec476bcf005.json | 22 + ...578179e6269c6ff36877cedee264067ccdafc.json | 65 + ...693831e2ab6d6d3c1805df5aa51d76994ec19.json | 16 + ...e3b0fd67aa1f5f7ea0de673a2fbe1f742db86.json | 22 + ...c9317c01d6e35db3b07d0a31f436e7e3c7c40.json | 14 + ...8e221a8ef9014458cc7f1dbc60c056a0768a0.json | 16 + ...a36532fee1450733849852dfd20e18ded1f03.json | 15 + ...d2e2326d7ace079b095def52723a45b65d3f3.json | 15 + ...2cfbaf09164aecfa5eed8d7142518ad96abea.json | 22 + ...5ca679a0459a012899a373c67393d30d12601.json | 14 + ...d92f5a547150590b8c221c4065eab11175c7a.json | 20 + ...d7cbb0bc526ebe61a07f735d4ab587058b22c.json | 22 + ...a69138206dfeb41f3daff4a3eef1de0bed4e4.json | 16 + ...5d93e391600ab9da2e5fd4e8b139ab3d77583.json | 15 + ...6c6e7b4595f3e7c3dca1d1590de5437187733.json | 29 + ...c9a64904026506914abae2946e5d353d6a604.json | 23 + ...e2757dbc13be6b30f5840744e5e0569adc66e.json | 22 + ...9f41220c51f58a03c61d6b7789eab0504e320.json | 32 + ...477dcf21955e0921ba648ba2e751dbfc3cb45.json | 38 + ...9bdc9efc6b89fc0444caf8271edd7dfe4a3bc.json | 20 + ...1788feea2fbb5682de09120dd109899510820.json | 28 + ...cd6ca26aa2fa565fcf208b6985f461c1868f2.json | 28 + ...63f436a4f16dbeb784d0d28be392ad96b1c49.json | 14 + ...895006e23ec73018f4b4e0a364a741f5c9781.json | 22 + ...f724216807ffd481cd6f7f19968e42e52b284.json | 14 + ...1ac4ab2d73bda6022700aeb0a630a2563a4b4.json | 15 + ...f8dcbcb014b4f808c6232abd9a83354c995ac.json | 35 + ...663a9c5ea62ea7c99a77941eb8f05d4454125.json | 18 + ...7b56187686173327498ac75424593547c19c5.json | 22 + ...9871731c41cef01da3d6aaf2c53f7b17c47b2.json | 23 + ...f8c12deeca6b8843fe3869cc2b02b30da5de6.json | 22 + ...49b6370c211a7fc24ad03a5f0e327f9d18040.json | 22 + ...d964d4bb39b9dcd18fb03bc11ce2fb32b7fb3.json | 83 + ...61b86b1abd915739533f9982fea9d8e21b9e9.json | 14 + ...0103263af3ff5cb6c9dc5df59d3cd2a5e56b4.json | 17 + ...0c2cfff08e6fef3c3824d20dfdf2d0f73e671.json | 34 + ...5386a2fd53492f3df05545edbfb0ec0f059d2.json | 15 + ...697e36b3c2a997038c30553f7727cdfa17361.json | 20 + ...651aa072f030c70a5e6de38843a1d9afdf329.json | 16 + ...180abbd76956e073432af8d8500327b74e488.json | 22 + ...9c9a85e3c24fca1bf87905f7fc68fe2ce3276.json | 16 + ...6d5ee4341edadb8a86b459a07427b7e265e98.json | 136 + ...a4b4e2af48907fa9321064ddb24ac02ab17cd.json | 20 + ...5661d1df4ec053797d75cfd32272be4f485e7.json | 54 + ...b742349e78e6e4ce3e7c9a0dcf6447eedc6d8.json | 94 + ...912d57f8eb2a38bdb7884fc812a2897a3a660.json | 35 + ...709dbee508ad6d1cae43e477cf1bef8cb4aa9.json | 23 + ...f995b654adfe328cb825a44ad36b4bf9ec8f2.json | 94 + ...379f3b2e9ff1bc6e8e38f473fb4950c8e4b77.json | 20 + ...a653852f7e43670076eb2ebcd49542a870539.json | 14 + ...c1d66aeaeb6e2d36fddcf96775f01716a8a74.json | 14 + ...432d865852afe0c60e11a2c1800d30599aa61.json | 14 + ...8aa0f8525832cb4075e831c0d4b23c5675b99.json | 24 + ...e899e360650afccb34f5cc301b5cbac4a3d36.json | 15 + ...38d60d6e93bcb34fd20422e227714fccbf6b7.json | 34 + ...8d43c31ec7441a7f6c5040e120810ebbb72f7.json | 21 + ...64420a25eef488de2b67e91ed39657667bd4a.json | 26 + ...2b7c4d79cbd404e0267794ec203df0cbb169d.json | 20 + ...2c212918b31dabebf08a84efdfe1feee85765.json | 20 + ...0d7eaeeb4549ed59b58f8d984be2a22a80355.json | 22 + ...ac429aac3c030f7e226a1264243d8cdae038d.json | 17 + ...03d8bd356f2f9cc249cd8b90caa5a8b5918e3.json | 23 + ...cb21a635037d89ce24dd3ad58ffaadb59594a.json | 20 + ...b4afd384063ae394a847b26304dd18d490ab4.json | 28 + ...41368993009bb4bd90c2ad177ce56317aa04c.json | 257 + ...3e67f08f2ead5f55bfb6594e50346bf9cf2ef.json | 32 + ...32432295b2f7ee40bf13522a6edbd236f1970.json | 29 + ...3b438f14b68535111cf2cedd363fc646aac99.json | 20 + ...0015eeb3ef2643ceda3024504a471b24d1283.json | 254 + ...8242aad3e9a47400f6d6837a35f4c54a216b9.json | 20 + ...fd17f833fb15b58b0110c7cc566946db98e76.json | 94 + ...3d2839f0f928c06b8020eecec38e895f99b42.json | 28 + ...813d2b2d411bd5faf8306cd48db500532b711.json | 29 + ...33c83477d65a6f8cb2673f67b3148cd95b436.json | 20 + ...75da2787332deca5708d1d08730cdbfc73541.json | 136 + ...7e88abd0f8164c2413dc83c91c29665ca645e.json | 35 + ...6aba34bfab67494fee578613b03ef7255324d.json | 15 + ...e8ee5e54b170de9da863bbdbc79e3f206640b.json | 14 + ...2060fbea775dc185f639139fbfd23e4d5f3c6.json | 15 + ...76fb01e6629e8c982c265f2af1d5000090572.json | 20 + ...c82627a936f7ea9f6c354eca4bea76fac6b10.json | 20 + ...2e2ca92fdfdf01cfd0b11f5ce24f0458a5e48.json | 26 + ...550b8db817630d1a9341db4a1f453f12e64fb.json | 34 + ...3aa4239d3a8be71b47178b4b8fb11fe898b31.json | 16 + ...70a4e629b2a1cde641e74e4e55bb100df809f.json | 22 + ...8f8af02aa297d85a2695c5f448ed14b2d7386.json | 19 + ...f35d55901fb1e6805f23413ea43e3637540a0.json | 28 + ...b957e92cd375ec33fe16f855f319ffc0b208e.json | 118 + ...8c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b.json | 22 + ...6bc6419ba51514519652e055c769b096015f6.json | 22 + ...cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705.json | 28 + ...71ababa66e4a443fbefbfffca72b7540b075b.json | 15 + ...85b19e1294b3aa6e084d2051bb532408672be.json | 12 + ...91a9984685eaaaa0a8b223410d560a15a3034.json | 61 + ...6686e655206601854799139c22c017a214744.json | 19 + ...78a8a0ec739f4ddec8ffeb3a87253aeb18d30.json | 14 + ...c3465e2211ef3013386feb12d4cc04e0eade9.json | 60 + ...15aaade450980719933089824eb8c494d64a4.json | 15 + ...4312b2b780563a9fde48bae5e51650475670f.json | 82 + ...d056cc9bae25bc070c5de8424f88fab20e5ea.json | 28 + ...bae63443c51deb818dd0affd1a0949b161737.json | 16 + ...5ff66f2c3b2b83d45360818a8782e56aa3d66.json | 36 + ...6efe8d41759bbdcdd6d39db93208f2664f03a.json | 22 + ...cfdc050b85e6d773b13f9699f15c335d42593.json | 22 + ...4867ed765dcb9dc60754df9df8700d4244bfb.json | 44 + ...17db60405a887f9f7fa0ca60aa7fc879ce630.json | 16 + ...227ecaa45d76d1ba260074898fe57c059a15a.json | 232 + ...5e6f8b7f88a0894a7f9e27fc26f93997d37c7.json | 24 + ...63a1e89945f8d5e0f4da42ecf6313c4f5967e.json | 20 + ...d425bac8ed3d2780c60c9b63afbcea131f9a0.json | 15 + ...c7dfb7aad7261e5fc402d845aedc3b91a4e99.json | 23 + ...304e8a35fd65bf37e976b7106f57c57e70b9b.json | 16 + ...a58f82acd817932415f04bcbd05442ad80c2b.json | 23 + ...5d9ec9b3f628c3a4cf5e10580ea6e5e3a2429.json | 14 + ...eb09b538a67d1c39fda052c4f4ddb23ce0084.json | 22 + core/lib/dal/Cargo.toml | 11 +- core/lib/dal/sqlx-data.json | 11447 ---------------- core/lib/dal/src/blocks_dal.rs | 4 +- core/lib/dal/src/eth_sender_dal.rs | 2 +- core/lib/dal/src/snapshots_creator_dal.rs | 2 +- core/lib/types/Cargo.toml | 2 +- core/lib/utils/Cargo.toml | 4 +- core/lib/web3_decl/Cargo.toml | 2 +- core/lib/zksync_core/Cargo.toml | 2 +- .../zksync_core/src/api_server/web3/mod.rs | 2 +- .../zksync_core/src/api_server/web3/pubsub.rs | 22 +- .../src/api_server/web3/tests/mod.rs | 15 +- docker/external-node/Dockerfile | 2 +- .../20.04_amd64_cuda_11_8.Dockerfile | 2 +- .../20.04_amd64_cuda_12_0.Dockerfile | 2 +- docker/zk-environment/Dockerfile | 2 +- docs/guides/setup-dev.md | 4 +- 315 files changed, 12566 insertions(+), 11660 deletions(-) create mode 100644 core/lib/dal/.sqlx/query-00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f.json create mode 100644 core/lib/dal/.sqlx/query-012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784.json create mode 100644 core/lib/dal/.sqlx/query-015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b.json create mode 100644 core/lib/dal/.sqlx/query-01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39.json create mode 100644 core/lib/dal/.sqlx/query-01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b.json create mode 100644 core/lib/dal/.sqlx/query-01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba.json create mode 100644 core/lib/dal/.sqlx/query-02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7.json create mode 100644 core/lib/dal/.sqlx/query-026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b.json create mode 100644 core/lib/dal/.sqlx/query-03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c.json create mode 100644 core/lib/dal/.sqlx/query-040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981.json create mode 100644 core/lib/dal/.sqlx/query-04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3.json create mode 100644 core/lib/dal/.sqlx/query-05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be.json create mode 100644 core/lib/dal/.sqlx/query-07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa.json create mode 100644 core/lib/dal/.sqlx/query-083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643.json create mode 100644 core/lib/dal/.sqlx/query-08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87.json create mode 100644 core/lib/dal/.sqlx/query-0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a.json create mode 100644 core/lib/dal/.sqlx/query-0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674.json create mode 100644 core/lib/dal/.sqlx/query-0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f.json create mode 100644 core/lib/dal/.sqlx/query-0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc.json create mode 100644 core/lib/dal/.sqlx/query-0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8.json create mode 100644 core/lib/dal/.sqlx/query-0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8.json create mode 100644 core/lib/dal/.sqlx/query-0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30.json create mode 100644 core/lib/dal/.sqlx/query-0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41.json create mode 100644 core/lib/dal/.sqlx/query-0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626.json create mode 100644 core/lib/dal/.sqlx/query-10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730.json create mode 100644 core/lib/dal/.sqlx/query-11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9.json create mode 100644 core/lib/dal/.sqlx/query-12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4.json create mode 100644 core/lib/dal/.sqlx/query-12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2.json create mode 100644 core/lib/dal/.sqlx/query-136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2.json create mode 100644 core/lib/dal/.sqlx/query-15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e.json create mode 100644 core/lib/dal/.sqlx/query-15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20.json create mode 100644 core/lib/dal/.sqlx/query-1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632.json create mode 100644 core/lib/dal/.sqlx/query-16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5.json create mode 100644 core/lib/dal/.sqlx/query-1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331.json create mode 100644 core/lib/dal/.sqlx/query-1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e.json create mode 100644 core/lib/dal/.sqlx/query-18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96.json create mode 100644 core/lib/dal/.sqlx/query-19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89.json create mode 100644 core/lib/dal/.sqlx/query-19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2.json create mode 100644 core/lib/dal/.sqlx/query-19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed.json create mode 100644 core/lib/dal/.sqlx/query-1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e.json create mode 100644 core/lib/dal/.sqlx/query-1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6.json create mode 100644 core/lib/dal/.sqlx/query-1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773.json create mode 100644 core/lib/dal/.sqlx/query-1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5.json create mode 100644 core/lib/dal/.sqlx/query-1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910.json create mode 100644 core/lib/dal/.sqlx/query-1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c.json create mode 100644 core/lib/dal/.sqlx/query-1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4.json create mode 100644 core/lib/dal/.sqlx/query-1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af.json create mode 100644 core/lib/dal/.sqlx/query-1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050.json create mode 100644 core/lib/dal/.sqlx/query-1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212.json create mode 100644 core/lib/dal/.sqlx/query-1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03.json create mode 100644 core/lib/dal/.sqlx/query-1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c.json create mode 100644 core/lib/dal/.sqlx/query-1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03.json create mode 100644 core/lib/dal/.sqlx/query-1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47.json create mode 100644 core/lib/dal/.sqlx/query-2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95.json create mode 100644 core/lib/dal/.sqlx/query-2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a.json create mode 100644 core/lib/dal/.sqlx/query-20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a.json create mode 100644 core/lib/dal/.sqlx/query-23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9.json create mode 100644 core/lib/dal/.sqlx/query-245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30.json create mode 100644 core/lib/dal/.sqlx/query-24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a.json create mode 100644 core/lib/dal/.sqlx/query-249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da.json create mode 100644 core/lib/dal/.sqlx/query-25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a.json create mode 100644 core/lib/dal/.sqlx/query-26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2.json create mode 100644 core/lib/dal/.sqlx/query-26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20.json create mode 100644 core/lib/dal/.sqlx/query-2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113.json create mode 100644 core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json create mode 100644 core/lib/dal/.sqlx/query-280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b.json create mode 100644 core/lib/dal/.sqlx/query-293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb.json create mode 100644 core/lib/dal/.sqlx/query-2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d.json create mode 100644 core/lib/dal/.sqlx/query-2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb.json create mode 100644 core/lib/dal/.sqlx/query-2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785.json create mode 100644 core/lib/dal/.sqlx/query-2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23.json create mode 100644 core/lib/dal/.sqlx/query-2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e.json create mode 100644 core/lib/dal/.sqlx/query-2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44.json create mode 100644 core/lib/dal/.sqlx/query-2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c.json create mode 100644 core/lib/dal/.sqlx/query-2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2.json create mode 100644 core/lib/dal/.sqlx/query-2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a.json create mode 100644 core/lib/dal/.sqlx/query-2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9.json create mode 100644 core/lib/dal/.sqlx/query-2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3.json create mode 100644 core/lib/dal/.sqlx/query-2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c.json create mode 100644 core/lib/dal/.sqlx/query-2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583.json create mode 100644 core/lib/dal/.sqlx/query-2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71.json create mode 100644 core/lib/dal/.sqlx/query-2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12.json create mode 100644 core/lib/dal/.sqlx/query-31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a.json create mode 100644 core/lib/dal/.sqlx/query-314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37.json create mode 100644 core/lib/dal/.sqlx/query-31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8.json create mode 100644 core/lib/dal/.sqlx/query-322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a.json create mode 100644 core/lib/dal/.sqlx/query-32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732.json create mode 100644 core/lib/dal/.sqlx/query-33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e.json create mode 100644 core/lib/dal/.sqlx/query-3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827.json create mode 100644 core/lib/dal/.sqlx/query-35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530.json create mode 100644 core/lib/dal/.sqlx/query-3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a.json create mode 100644 core/lib/dal/.sqlx/query-367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2.json create mode 100644 core/lib/dal/.sqlx/query-38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c.json create mode 100644 core/lib/dal/.sqlx/query-3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0.json create mode 100644 core/lib/dal/.sqlx/query-3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28.json create mode 100644 core/lib/dal/.sqlx/query-3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361.json create mode 100644 core/lib/dal/.sqlx/query-3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231.json create mode 100644 core/lib/dal/.sqlx/query-3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64.json create mode 100644 core/lib/dal/.sqlx/query-3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354.json create mode 100644 core/lib/dal/.sqlx/query-3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa.json create mode 100644 core/lib/dal/.sqlx/query-41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151.json create mode 100644 core/lib/dal/.sqlx/query-43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1.json create mode 100644 core/lib/dal/.sqlx/query-46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c.json create mode 100644 core/lib/dal/.sqlx/query-47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060.json create mode 100644 core/lib/dal/.sqlx/query-481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae.json create mode 100644 core/lib/dal/.sqlx/query-4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b.json create mode 100644 core/lib/dal/.sqlx/query-4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2.json create mode 100644 core/lib/dal/.sqlx/query-4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d.json create mode 100644 core/lib/dal/.sqlx/query-4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783.json create mode 100644 core/lib/dal/.sqlx/query-4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c.json create mode 100644 core/lib/dal/.sqlx/query-525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63.json create mode 100644 core/lib/dal/.sqlx/query-532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047.json create mode 100644 core/lib/dal/.sqlx/query-534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56.json create mode 100644 core/lib/dal/.sqlx/query-53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea.json create mode 100644 core/lib/dal/.sqlx/query-53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39.json create mode 100644 core/lib/dal/.sqlx/query-5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231.json create mode 100644 core/lib/dal/.sqlx/query-556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d.json create mode 100644 core/lib/dal/.sqlx/query-55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60.json create mode 100644 core/lib/dal/.sqlx/query-5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33.json create mode 100644 core/lib/dal/.sqlx/query-5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85.json create mode 100644 core/lib/dal/.sqlx/query-58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de.json create mode 100644 core/lib/dal/.sqlx/query-59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f.json create mode 100644 core/lib/dal/.sqlx/query-5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b.json create mode 100644 core/lib/dal/.sqlx/query-5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13.json create mode 100644 core/lib/dal/.sqlx/query-5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6.json create mode 100644 core/lib/dal/.sqlx/query-5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f.json create mode 100644 core/lib/dal/.sqlx/query-5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c.json create mode 100644 core/lib/dal/.sqlx/query-5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83.json create mode 100644 core/lib/dal/.sqlx/query-61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513.json create mode 100644 core/lib/dal/.sqlx/query-61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49.json create mode 100644 core/lib/dal/.sqlx/query-6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363.json create mode 100644 core/lib/dal/.sqlx/query-66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104.json create mode 100644 core/lib/dal/.sqlx/query-68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d.json create mode 100644 core/lib/dal/.sqlx/query-68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7.json create mode 100644 core/lib/dal/.sqlx/query-6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38.json create mode 100644 core/lib/dal/.sqlx/query-6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20.json create mode 100644 core/lib/dal/.sqlx/query-6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914.json create mode 100644 core/lib/dal/.sqlx/query-6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0.json create mode 100644 core/lib/dal/.sqlx/query-6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6.json create mode 100644 core/lib/dal/.sqlx/query-708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d.json create mode 100644 core/lib/dal/.sqlx/query-70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94.json create mode 100644 core/lib/dal/.sqlx/query-72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42.json create mode 100644 core/lib/dal/.sqlx/query-72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345.json create mode 100644 core/lib/dal/.sqlx/query-73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665.json create mode 100644 core/lib/dal/.sqlx/query-7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914.json create mode 100644 core/lib/dal/.sqlx/query-759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0.json create mode 100644 core/lib/dal/.sqlx/query-75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3.json create mode 100644 core/lib/dal/.sqlx/query-75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4.json create mode 100644 core/lib/dal/.sqlx/query-75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60.json create mode 100644 core/lib/dal/.sqlx/query-76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b.json create mode 100644 core/lib/dal/.sqlx/query-77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796.json create mode 100644 core/lib/dal/.sqlx/query-77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c.json create mode 100644 core/lib/dal/.sqlx/query-78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909.json create mode 100644 core/lib/dal/.sqlx/query-7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776.json create mode 100644 core/lib/dal/.sqlx/query-7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6.json create mode 100644 core/lib/dal/.sqlx/query-7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283.json create mode 100644 core/lib/dal/.sqlx/query-806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9.json create mode 100644 core/lib/dal/.sqlx/query-8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe.json create mode 100644 core/lib/dal/.sqlx/query-81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9.json create mode 100644 core/lib/dal/.sqlx/query-83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8.json create mode 100644 core/lib/dal/.sqlx/query-84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12.json create mode 100644 core/lib/dal/.sqlx/query-852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8.json create mode 100644 core/lib/dal/.sqlx/query-8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497.json create mode 100644 core/lib/dal/.sqlx/query-877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b.json create mode 100644 core/lib/dal/.sqlx/query-878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4.json create mode 100644 core/lib/dal/.sqlx/query-88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60.json create mode 100644 core/lib/dal/.sqlx/query-8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69.json create mode 100644 core/lib/dal/.sqlx/query-894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2.json create mode 100644 core/lib/dal/.sqlx/query-8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db.json create mode 100644 core/lib/dal/.sqlx/query-8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895.json create mode 100644 core/lib/dal/.sqlx/query-8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f.json create mode 100644 core/lib/dal/.sqlx/query-90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77.json create mode 100644 core/lib/dal/.sqlx/query-9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5.json create mode 100644 core/lib/dal/.sqlx/query-95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300.json create mode 100644 core/lib/dal/.sqlx/query-966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f.json create mode 100644 core/lib/dal/.sqlx/query-9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47.json create mode 100644 core/lib/dal/.sqlx/query-995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068.json create mode 100644 core/lib/dal/.sqlx/query-99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6.json create mode 100644 core/lib/dal/.sqlx/query-99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005.json create mode 100644 core/lib/dal/.sqlx/query-99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc.json create mode 100644 core/lib/dal/.sqlx/query-9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19.json create mode 100644 core/lib/dal/.sqlx/query-9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86.json create mode 100644 core/lib/dal/.sqlx/query-9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40.json create mode 100644 core/lib/dal/.sqlx/query-9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0.json create mode 100644 core/lib/dal/.sqlx/query-9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03.json create mode 100644 core/lib/dal/.sqlx/query-a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3.json create mode 100644 core/lib/dal/.sqlx/query-a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea.json create mode 100644 core/lib/dal/.sqlx/query-a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601.json create mode 100644 core/lib/dal/.sqlx/query-a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a.json create mode 100644 core/lib/dal/.sqlx/query-a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c.json create mode 100644 core/lib/dal/.sqlx/query-a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4.json create mode 100644 core/lib/dal/.sqlx/query-a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583.json create mode 100644 core/lib/dal/.sqlx/query-a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733.json create mode 100644 core/lib/dal/.sqlx/query-a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604.json create mode 100644 core/lib/dal/.sqlx/query-a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e.json create mode 100644 core/lib/dal/.sqlx/query-aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320.json create mode 100644 core/lib/dal/.sqlx/query-aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45.json create mode 100644 core/lib/dal/.sqlx/query-ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc.json create mode 100644 core/lib/dal/.sqlx/query-ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820.json create mode 100644 core/lib/dal/.sqlx/query-ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2.json create mode 100644 core/lib/dal/.sqlx/query-aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49.json create mode 100644 core/lib/dal/.sqlx/query-aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781.json create mode 100644 core/lib/dal/.sqlx/query-af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284.json create mode 100644 core/lib/dal/.sqlx/query-afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4.json create mode 100644 core/lib/dal/.sqlx/query-b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac.json create mode 100644 core/lib/dal/.sqlx/query-b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125.json create mode 100644 core/lib/dal/.sqlx/query-b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5.json create mode 100644 core/lib/dal/.sqlx/query-b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2.json create mode 100644 core/lib/dal/.sqlx/query-b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6.json create mode 100644 core/lib/dal/.sqlx/query-b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040.json create mode 100644 core/lib/dal/.sqlx/query-b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3.json create mode 100644 core/lib/dal/.sqlx/query-b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9.json create mode 100644 core/lib/dal/.sqlx/query-b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4.json create mode 100644 core/lib/dal/.sqlx/query-b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671.json create mode 100644 core/lib/dal/.sqlx/query-b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2.json create mode 100644 core/lib/dal/.sqlx/query-b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361.json create mode 100644 core/lib/dal/.sqlx/query-b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329.json create mode 100644 core/lib/dal/.sqlx/query-b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488.json create mode 100644 core/lib/dal/.sqlx/query-b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276.json create mode 100644 core/lib/dal/.sqlx/query-bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98.json create mode 100644 core/lib/dal/.sqlx/query-bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd.json create mode 100644 core/lib/dal/.sqlx/query-bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7.json create mode 100644 core/lib/dal/.sqlx/query-be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8.json create mode 100644 core/lib/dal/.sqlx/query-bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660.json create mode 100644 core/lib/dal/.sqlx/query-bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9.json create mode 100644 core/lib/dal/.sqlx/query-c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2.json create mode 100644 core/lib/dal/.sqlx/query-c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77.json create mode 100644 core/lib/dal/.sqlx/query-c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539.json create mode 100644 core/lib/dal/.sqlx/query-c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74.json create mode 100644 core/lib/dal/.sqlx/query-c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61.json create mode 100644 core/lib/dal/.sqlx/query-c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99.json create mode 100644 core/lib/dal/.sqlx/query-c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36.json create mode 100644 core/lib/dal/.sqlx/query-c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7.json create mode 100644 core/lib/dal/.sqlx/query-c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7.json create mode 100644 core/lib/dal/.sqlx/query-c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a.json create mode 100644 core/lib/dal/.sqlx/query-c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d.json create mode 100644 core/lib/dal/.sqlx/query-c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765.json create mode 100644 core/lib/dal/.sqlx/query-c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355.json create mode 100644 core/lib/dal/.sqlx/query-c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d.json create mode 100644 core/lib/dal/.sqlx/query-c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3.json create mode 100644 core/lib/dal/.sqlx/query-ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a.json create mode 100644 core/lib/dal/.sqlx/query-cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4.json create mode 100644 core/lib/dal/.sqlx/query-cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c.json create mode 100644 core/lib/dal/.sqlx/query-ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef.json create mode 100644 core/lib/dal/.sqlx/query-cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970.json create mode 100644 core/lib/dal/.sqlx/query-d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99.json create mode 100644 core/lib/dal/.sqlx/query-d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283.json create mode 100644 core/lib/dal/.sqlx/query-d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9.json create mode 100644 core/lib/dal/.sqlx/query-d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76.json create mode 100644 core/lib/dal/.sqlx/query-d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42.json create mode 100644 core/lib/dal/.sqlx/query-d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711.json create mode 100644 core/lib/dal/.sqlx/query-d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436.json create mode 100644 core/lib/dal/.sqlx/query-d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541.json create mode 100644 core/lib/dal/.sqlx/query-d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e.json create mode 100644 core/lib/dal/.sqlx/query-d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d.json create mode 100644 core/lib/dal/.sqlx/query-da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b.json create mode 100644 core/lib/dal/.sqlx/query-db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6.json create mode 100644 core/lib/dal/.sqlx/query-dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572.json create mode 100644 core/lib/dal/.sqlx/query-dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10.json create mode 100644 core/lib/dal/.sqlx/query-dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48.json create mode 100644 core/lib/dal/.sqlx/query-dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb.json create mode 100644 core/lib/dal/.sqlx/query-dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31.json create mode 100644 core/lib/dal/.sqlx/query-df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f.json create mode 100644 core/lib/dal/.sqlx/query-df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386.json create mode 100644 core/lib/dal/.sqlx/query-e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0.json create mode 100644 core/lib/dal/.sqlx/query-e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e.json create mode 100644 core/lib/dal/.sqlx/query-e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b.json create mode 100644 core/lib/dal/.sqlx/query-e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6.json create mode 100644 core/lib/dal/.sqlx/query-e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705.json create mode 100644 core/lib/dal/.sqlx/query-e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b.json create mode 100644 core/lib/dal/.sqlx/query-e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be.json create mode 100644 core/lib/dal/.sqlx/query-e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034.json create mode 100644 core/lib/dal/.sqlx/query-e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744.json create mode 100644 core/lib/dal/.sqlx/query-ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30.json create mode 100644 core/lib/dal/.sqlx/query-ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9.json create mode 100644 core/lib/dal/.sqlx/query-edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4.json create mode 100644 core/lib/dal/.sqlx/query-ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f.json create mode 100644 core/lib/dal/.sqlx/query-ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea.json create mode 100644 core/lib/dal/.sqlx/query-ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737.json create mode 100644 core/lib/dal/.sqlx/query-f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66.json create mode 100644 core/lib/dal/.sqlx/query-f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a.json create mode 100644 core/lib/dal/.sqlx/query-f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593.json create mode 100644 core/lib/dal/.sqlx/query-f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb.json create mode 100644 core/lib/dal/.sqlx/query-f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630.json create mode 100644 core/lib/dal/.sqlx/query-f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a.json create mode 100644 core/lib/dal/.sqlx/query-f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7.json create mode 100644 core/lib/dal/.sqlx/query-f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e.json create mode 100644 core/lib/dal/.sqlx/query-f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0.json create mode 100644 core/lib/dal/.sqlx/query-fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99.json create mode 100644 core/lib/dal/.sqlx/query-fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b.json create mode 100644 core/lib/dal/.sqlx/query-fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b.json create mode 100644 core/lib/dal/.sqlx/query-fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429.json create mode 100644 core/lib/dal/.sqlx/query-fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084.json delete mode 100644 core/lib/dal/sqlx-data.json diff --git a/.github/workflows/ci-core-lint-reusable.yml b/.github/workflows/ci-core-lint-reusable.yml index f0155168c592..365d477c64bb 100644 --- a/.github/workflows/ci-core-lint-reusable.yml +++ b/.github/workflows/ci-core-lint-reusable.yml @@ -36,5 +36,5 @@ jobs: ci_run zk lint js --check ci_run zk lint ts --check ci_run zk lint md --check - ci_run zk db check-sqlx-data + # ci_run zk db check-sqlx-data diff --git a/.github/workflows/ci-core-reusable.yml b/.github/workflows/ci-core-reusable.yml index b466fc87d39e..76666210cb11 100644 --- a/.github/workflows/ci-core-reusable.yml +++ b/.github/workflows/ci-core-reusable.yml @@ -326,6 +326,10 @@ jobs: if: always() run: ci_run cat server.log || true + - name: Show ext-node.log logs + if: always() + run: ci_run cat ext-node.log || true + - name: Show contract_verifier.log logs if: always() run: ci_run cat ext-node.log || true diff --git a/Cargo.lock b/Cargo.lock index d104f9932cbd..fa420d7c48d3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -44,7 +44,7 @@ dependencies = [ "actix-rt", "actix-service", "actix-utils", - "ahash 0.8.5", + "ahash 0.8.7", "base64 0.21.5", "bitflags 2.4.1", "brotli", @@ -160,7 +160,7 @@ dependencies = [ "actix-service", "actix-utils", "actix-web-codegen", - "ahash 0.8.5", + "ahash 0.8.7", "bytes", "bytestring", "cfg-if 1.0.0", @@ -314,9 +314,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.5" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd7d5a2cecb58716e47d67d5703a249964b14c7be1ec3cad3affc295b2d1c35d" +checksum = "77c3a9648d43b9cd48db467b3f87fdd6e146bcc88ab0180006cef2179fe11d01" dependencies = [ "cfg-if 1.0.0", "getrandom 0.2.10", @@ -349,6 +349,12 @@ dependencies = [ "alloc-no-stdlib", ] +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + [[package]] name = "android-tzdata" version = "0.1.1" @@ -549,13 +555,23 @@ dependencies = [ [[package]] name = "atoi" -version = "0.4.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "616896e05fc0e2649463a93a15183c6a16bf03413a7af88ef1285ddedfa9cda5" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" dependencies = [ "num-traits", ] +[[package]] +name = "atomic-write-file" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edcdbedc2236483ab103a53415653d6b4442ea6141baf1ffa85df29635e88436" +dependencies = [ + "nix", + "rand 0.8.5", +] + [[package]] name = "atty" version = "0.2.14" @@ -738,11 +754,11 @@ dependencies = [ [[package]] name = "bigdecimal" -version = "0.2.2" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1e50562e37200edf7c6c43e54a08e64a5553bfb59d9c297d5572512aa517256" +checksum = "a6773ddc0eafc0e509fb60e48dff7f450f8e674a0686ae8605e8d9901bd5eefa" dependencies = [ - "num-bigint 0.3.3", + "num-bigint 0.4.4", "num-integer", "num-traits", "serde", @@ -798,6 +814,9 @@ name = "bitflags" version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +dependencies = [ + "serde", +] [[package]] name = "bitmaps" @@ -981,7 +1000,7 @@ dependencies = [ "derivative", "ethereum-types 0.14.1", "firestorm", - "itertools", + "itertools 0.10.5", "lazy_static", "num-modular", "num_cpus", @@ -996,6 +1015,30 @@ dependencies = [ "unroll", ] +[[package]] +name = "borsh" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d4d6dafc1a3bb54687538972158f07b2c948bc57d5890df22c0739098b3028" +dependencies = [ + "borsh-derive", + "cfg_aliases", +] + +[[package]] +name = "borsh-derive" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf4918709cc4dd777ad2b6303ed03cb37f3ca0ccede8c1b0d28ac6db8f4710e0" +dependencies = [ + "once_cell", + "proc-macro-crate 2.0.1", + "proc-macro2 1.0.69", + "quote 1.0.33", + "syn 2.0.38", + "syn_derive", +] + [[package]] name = "brotli" version = "3.4.0" @@ -1029,6 +1072,28 @@ version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c3ac9f8b63eca6fd385229b3675f6cc0dc5c8a5c8a54a59d4f52ffd670d87b0c" +[[package]] +name = "bytecheck" +version = "0.6.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6372023ac861f6e6dc89c8344a8f398fb42aaba2b5dbc649ca0c0e9dbcb627" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7ec4c6f261935ad534c0c22dbef2201b45918860eb1c574b972bd213a76af61" +dependencies = [ + "proc-macro2 1.0.69", + "quote 1.0.33", + "syn 1.0.109", +] + [[package]] name = "bytecount" version = "0.6.5" @@ -1135,6 +1200,12 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +[[package]] +name = "cfg_aliases" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" + [[package]] name = "chacha20" version = "0.9.1" @@ -1480,18 +1551,18 @@ dependencies = [ [[package]] name = "crc" -version = "2.1.0" +version = "3.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49fc9a695bca7f35f5f4c15cddc84415f66a74ea78eef08e90c5024f2b540e23" +checksum = "86ec7a15cbe22e59248fc7eadb1907dab5ba09372595da4d73dd805ed4417dfe" dependencies = [ "crc-catalog", ] [[package]] name = "crc-catalog" -version = "1.1.1" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccaeedb56da03b09f598226e25e80088cb4cd25f316e6e4df7d695f0feeb1403" +checksum = "19d374276b40fb8bbdee95aef7c7fa6b5316ec764510eb64b8dd0e2ed0d7e7f5" [[package]] name = "crc32fast" @@ -1514,7 +1585,7 @@ dependencies = [ "ciborium", "clap 3.2.25", "criterion-plot", - "itertools", + "itertools 0.10.5", "lazy_static", "num-traits", "oorandom", @@ -1535,7 +1606,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" dependencies = [ "cast", - "itertools", + "itertools 0.10.5", ] [[package]] @@ -1878,7 +1949,7 @@ dependencies = [ "hashbrown 0.14.2", "lock_api", "once_cell", - "parking_lot_core 0.9.9", + "parking_lot_core", ] [[package]] @@ -1962,35 +2033,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer 0.10.4", + "const-oid", "crypto-common", "subtle", ] [[package]] -name = "dirs" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3aa72a6f96ea37bbc5aa912f6788242832f75369bdfdadcb0e38423f100059" -dependencies = [ - "dirs-sys", -] - -[[package]] -name = "dirs-sys" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b1d1d91c932ef41c0f2663aa8b0ca0342d444d842c06914aa0a7e352d0bada6" -dependencies = [ - "libc", - "redox_users", - "winapi", -] - -[[package]] -name = "dotenv" -version = "0.15.0" +name = "dotenvy" +version = "0.15.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" +checksum = "1aaf95b3e5c8f23aa320147307562d361db0ae0d51242340f558153b4eb2439b" [[package]] name = "dtoa" @@ -2147,6 +2199,17 @@ dependencies = [ "version_check", ] +[[package]] +name = "etcetera" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "136d1b5283a1ab77bd9257427ffd09d8667ced0570b6f938942bc7568ed5b943" +dependencies = [ + "cfg-if 1.0.0", + "home", + "windows-sys 0.48.0", +] + [[package]] name = "ethabi" version = "18.0.0" @@ -2368,6 +2431,17 @@ dependencies = [ "miniz_oxide", ] +[[package]] +name = "flume" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55ac459de2512911e4b674ce33cf20befaba382d05b62b008afc1c8b57cbf181" +dependencies = [ + "futures-core", + "futures-sink", + "spin 0.9.8", +] + [[package]] name = "fnv" version = "1.0.7" @@ -2413,7 +2487,7 @@ dependencies = [ "digest 0.9.0", "hex", "indexmap 1.9.3", - "itertools", + "itertools 0.10.5", "lazy_static", "num-bigint 0.4.4", "num-derive 0.2.5", @@ -2445,7 +2519,7 @@ dependencies = [ "digest 0.9.0", "hex", "indexmap 1.9.3", - "itertools", + "itertools 0.10.5", "lazy_static", "num-bigint 0.4.4", "num-derive 0.2.5", @@ -2529,13 +2603,13 @@ dependencies = [ [[package]] name = "futures-intrusive" -version = "0.4.2" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a604f7a68fbf8103337523b1fadc8ade7361ee3f112f7c680ad179651616aed5" +checksum = "1d930c203dd0b6ff06e0201a4a2fe9149b43c684fd4420555b26d21b1a02956f" dependencies = [ "futures-core", "lock_api", - "parking_lot 0.11.2", + "parking_lot", ] [[package]] @@ -2780,7 +2854,7 @@ dependencies = [ "futures-timer", "no-std-compat", "nonzero_ext", - "parking_lot 0.12.1", + "parking_lot", "quanta 0.9.3", "rand 0.8.5", "smallvec", @@ -2838,26 +2912,20 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.11.2" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ "ahash 0.7.7", ] -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" - [[package]] name = "hashbrown" version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ff8ae62cd3a9102e5637afc8452c55acf3844001bd5374e0b0bd7b6616c038" dependencies = [ - "ahash 0.8.5", + "ahash 0.8.7", ] [[package]] @@ -2865,14 +2933,18 @@ name = "hashbrown" version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" +dependencies = [ + "ahash 0.8.7", + "allocator-api2", +] [[package]] name = "hashlink" -version = "0.7.0" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7249a3129cbc1ffccd74857f81464a323a152173cdb134e0fd81bc803b29facf" +checksum = "e8094feaf31ff591f651a2664fb9cfd92bba7a60ce3197265e9482ebe753c8f7" dependencies = [ - "hashbrown 0.11.2", + "hashbrown 0.14.2", ] [[package]] @@ -3270,9 +3342,12 @@ checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" [[package]] name = "ipnetwork" -version = "0.17.0" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02c3eaab3ac0ede60ffa41add21970a7df7d91772c03383aac6c2c3d53cc716b" +checksum = "bf466541e9d546596ee94f9f69590f89473455f88372423e0008fc1a7daf100e" +dependencies = [ + "serde", +] [[package]] name = "iri-string" @@ -3304,6 +3379,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.9" @@ -3399,7 +3483,7 @@ dependencies = [ "futures-util", "hyper", "jsonrpsee-types", - "parking_lot 0.12.1", + "parking_lot", "pin-project", "rand 0.8.5", "rustc-hash", @@ -3552,6 +3636,9 @@ name = "lazy_static" version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +dependencies = [ + "spin 0.5.2", +] [[package]] name = "lazycell" @@ -3601,6 +3688,17 @@ dependencies = [ "libz-sys", ] +[[package]] +name = "libsqlite3-sys" +version = "0.27.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf4e226dcd58b4be396f7bd3c20da8fdee2911400705297ba7d2d7cc2c30f716" +dependencies = [ + "cc", + "pkg-config", + "vcpkg", +] + [[package]] name = "libz-sys" version = "1.1.12" @@ -3849,7 +3947,7 @@ version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fde3af1a009ed76a778cb84fdef9e7dbbdf5775ae3e4cc1f434a6a307f6f76c5" dependencies = [ - "ahash 0.8.5", + "ahash 0.8.7", "metrics-macros", "portable-atomic", ] @@ -3992,7 +4090,7 @@ dependencies = [ "anyhow", "ethabi", "hex", - "itertools", + "itertools 0.10.5", "once_cell", "thiserror", "tokio", @@ -4130,6 +4228,23 @@ dependencies = [ "serde", ] +[[package]] +name = "num-bigint-dig" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc84195820f291c7697304f3cbdadd1cb7199c0efc917ff5eafd71225c136151" +dependencies = [ + "byteorder", + "lazy_static", + "libm", + "num-integer", + "num-iter", + "num-traits", + "rand 0.8.5", + "smallvec", + "zeroize", +] + [[package]] name = "num-complex" version = "0.3.1" @@ -4147,6 +4262,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ba157ca0885411de85d6ca030ba7e2a83a28636056c7c699b07c8b6f7383214" dependencies = [ "num-traits", + "serde", ] [[package]] @@ -4225,6 +4341,7 @@ dependencies = [ "num-bigint 0.4.4", "num-integer", "num-traits", + "serde", ] [[package]] @@ -4500,17 +4617,6 @@ version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" -[[package]] -name = "parking_lot" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d17b78036a60663b797adeaee46f5c9dfebb86948d1255007a1d6be0271ff99" -dependencies = [ - "instant", - "lock_api", - "parking_lot_core 0.8.6", -] - [[package]] name = "parking_lot" version = "0.12.1" @@ -4518,21 +4624,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" dependencies = [ "lock_api", - "parking_lot_core 0.9.9", -] - -[[package]] -name = "parking_lot_core" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60a2cfe6f0ad2bfc16aefa463b497d5c7a5ecd44a23efa72aa342d90177356dc" -dependencies = [ - "cfg-if 1.0.0", - "instant", - "libc", - "redox_syscall 0.2.16", - "smallvec", - "winapi", + "parking_lot_core", ] [[package]] @@ -4703,6 +4795,17 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pkcs1" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8ffb9f10fa047879315e6625af03c164b16962a5368d724ed16323b68ace47f" +dependencies = [ + "der 0.7.8", + "pkcs8 0.10.2", + "spki 0.7.2", +] + [[package]] name = "pkcs8" version = "0.9.0" @@ -4916,7 +5019,7 @@ checksum = "3c99afa9a01501019ac3a14d71d9f94050346f55ca471ce90c799a15c58f61e2" dependencies = [ "dtoa", "itoa", - "parking_lot 0.12.1", + "parking_lot", "prometheus-client-derive-encode", ] @@ -4961,7 +5064,7 @@ checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", "heck 0.4.1", - "itertools", + "itertools 0.10.5", "log", "multimap", "once_cell", @@ -4982,7 +5085,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", - "itertools", + "itertools 0.10.5", "proc-macro2 1.0.69", "quote 1.0.33", "syn 2.0.38", @@ -5040,6 +5143,26 @@ dependencies = [ "thiserror", ] +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2 1.0.69", + "quote 1.0.33", + "syn 1.0.109", +] + [[package]] name = "pulldown-cmark" version = "0.9.3" @@ -5357,15 +5480,6 @@ dependencies = [ "rand_core 0.3.1", ] -[[package]] -name = "redox_syscall" -version = "0.2.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" -dependencies = [ - "bitflags 1.3.2", -] - [[package]] name = "redox_syscall" version = "0.3.5" @@ -5384,17 +5498,6 @@ dependencies = [ "bitflags 1.3.2", ] -[[package]] -name = "redox_users" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" -dependencies = [ - "getrandom 0.2.10", - "redox_syscall 0.2.16", - "thiserror", -] - [[package]] name = "regex" version = "1.10.2" @@ -5448,6 +5551,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "rend" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2571463863a6bd50c32f94402933f03457a3fbaf697a707c5be741e459f08fd" +dependencies = [ + "bytecheck", +] + [[package]] name = "reqwest" version = "0.11.22" @@ -5587,6 +5699,35 @@ dependencies = [ "opaque-debug", ] +[[package]] +name = "rkyv" +version = "0.7.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "527a97cdfef66f65998b5f3b637c26f5a5ec09cc52a3f9932313ac645f4190f5" +dependencies = [ + "bitvec 1.0.1", + "bytecheck", + "bytes", + "hashbrown 0.12.3", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5c462a1328c8e67e4d6dbad1eb0355dd43e8ab432c6e227a43657f16ade5033" +dependencies = [ + "proc-macro2 1.0.69", + "quote 1.0.33", + "syn 1.0.109", +] + [[package]] name = "rlp" version = "0.5.2" @@ -5613,6 +5754,42 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "afab94fb28594581f62d981211a9a4d53cc8130bbcbbb89a0440d9b8e81a7746" +[[package]] +name = "rsa" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af6c4b23d99685a1408194da11270ef8e9809aff951cc70ec9b17350b087e474" +dependencies = [ + "const-oid", + "digest 0.10.7", + "num-bigint-dig", + "num-integer", + "num-traits", + "pkcs1", + "pkcs8 0.10.2", + "rand_core 0.6.4", + "signature 2.2.0", + "spki 0.7.2", + "subtle", + "zeroize", +] + +[[package]] +name = "rust_decimal" +version = "1.33.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06676aec5ccb8fc1da723cc8c0f9a46549f21ebb8753d3915c6c41db1e7f1dc4" +dependencies = [ + "arrayvec 0.7.4", + "borsh", + "bytes", + "num-traits", + "rand 0.8.5", + "rkyv", + "serde", + "serde_json", +] + [[package]] name = "rustc-demangle" version = "0.1.23" @@ -5821,6 +5998,12 @@ dependencies = [ "untrusted 0.7.1", ] +[[package]] +name = "seahash" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" + [[package]] name = "sec1" version = "0.3.0" @@ -6118,17 +6301,6 @@ dependencies = [ "opaque-debug", ] -[[package]] -name = "sha-1" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5058ada175748e33390e40e872bd0fe59a19f265d0158daa551c5a88a76009c" -dependencies = [ - "cfg-if 1.0.0", - "cpufeatures", - "digest 0.10.7", -] - [[package]] name = "sha1" version = "0.10.6" @@ -6245,9 +6417,16 @@ version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77549399552de45a898a580c1b41d445bf730df867cc44e6c0233bbc4b8329de" dependencies = [ + "digest 0.10.7", "rand_core 0.6.4", ] +[[package]] +name = "simdutf8" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" + [[package]] name = "similar" version = "2.3.0" @@ -6394,7 +6573,7 @@ dependencies = [ "httparse", "log", "rand 0.8.5", - "sha-1 0.9.8", + "sha-1", ] [[package]] @@ -6408,6 +6587,9 @@ name = "spin" version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" +dependencies = [ + "lock_api", +] [[package]] name = "spki" @@ -6437,85 +6619,94 @@ checksum = "c85070f382340e8b23a75808e83573ddf65f9ad9143df9573ca37c1ed2ee956a" [[package]] name = "sqlformat" -version = "0.1.8" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b7922be017ee70900be125523f38bdd644f4f06a1b16e8fa5a8ee8c34bffd4" +checksum = "ce81b7bd7c4493975347ef60d8c7e8b742d4694f4c49f93e0a12ea263938176c" dependencies = [ - "itertools", + "itertools 0.12.0", "nom", "unicode_categories", ] [[package]] name = "sqlx" -version = "0.5.13" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "551873805652ba0d912fec5bbb0f8b4cdd96baf8e2ebf5970e5671092966019b" +checksum = "dba03c279da73694ef99763320dea58b51095dfe87d001b1d4b5fe78ba8763cf" dependencies = [ "sqlx-core", "sqlx-macros", + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", ] [[package]] name = "sqlx-core" -version = "0.5.13" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e48c61941ccf5ddcada342cd59e3e5173b007c509e1e8e990dafc830294d9dc5" +checksum = "d84b0a3c3739e220d94b3239fd69fb1f74bc36e16643423bd99de3b43c21bfbd" dependencies = [ - "ahash 0.7.7", + "ahash 0.8.7", "atoi", - "base64 0.13.1", "bigdecimal", - "bitflags 1.3.2", "byteorder", "bytes", "chrono", "crc", "crossbeam-queue 0.3.8", - "dirs", + "dotenvy", "either", "event-listener 2.5.3", "futures-channel", "futures-core", "futures-intrusive", + "futures-io", "futures-util", "hashlink", "hex", - "hkdf", - "hmac 0.12.1", - "indexmap 1.9.3", + "indexmap 2.1.0", "ipnetwork", - "itoa", - "libc", "log", - "md-5", "memchr", - "num-bigint 0.3.3", + "native-tls", "once_cell", "paste", "percent-encoding", - "rand 0.8.5", + "rust_decimal", "serde", "serde_json", - "sha-1 0.10.1", "sha2 0.10.8", "smallvec", "sqlformat", - "sqlx-rt", - "stringprep", "thiserror", + "tokio", "tokio-stream", + "tracing", "url", - "whoami", ] [[package]] name = "sqlx-macros" -version = "0.5.13" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc0fba2b0cae21fc00fe6046f8baa4c7fcb49e379f0f592b04696607f69ed2e1" +checksum = "89961c00dc4d7dffb7aee214964b065072bff69e36ddb9e2c107541f75e4f2a5" dependencies = [ - "dotenv", + "proc-macro2 1.0.69", + "quote 1.0.33", + "sqlx-core", + "sqlx-macros-core", + "syn 1.0.109", +] + +[[package]] +name = "sqlx-macros-core" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0bd4519486723648186a08785143599760f7cc81c52334a55d6a83ea1e20841" +dependencies = [ + "atomic-write-file", + "dotenvy", "either", "heck 0.4.1", "hex", @@ -6526,21 +6717,126 @@ dependencies = [ "serde_json", "sha2 0.10.8", "sqlx-core", - "sqlx-rt", + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", "syn 1.0.109", + "tempfile", + "tokio", "url", ] [[package]] -name = "sqlx-rt" -version = "0.5.13" +name = "sqlx-mysql" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4db708cd3e459078f85f39f96a00960bd841f66ee2a669e90bf36907f5a79aae" +checksum = "e37195395df71fd068f6e2082247891bc11e3289624bbc776a0cdfa1ca7f1ea4" dependencies = [ - "native-tls", + "atoi", + "base64 0.21.5", + "bigdecimal", + "bitflags 2.4.1", + "byteorder", + "bytes", + "chrono", + "crc", + "digest 0.10.7", + "dotenvy", + "either", + "futures-channel", + "futures-core", + "futures-io", + "futures-util", + "generic-array", + "hex", + "hkdf", + "hmac 0.12.1", + "itoa", + "log", + "md-5", + "memchr", "once_cell", - "tokio", - "tokio-native-tls", + "percent-encoding", + "rand 0.8.5", + "rsa", + "rust_decimal", + "serde", + "sha1", + "sha2 0.10.8", + "smallvec", + "sqlx-core", + "stringprep", + "thiserror", + "tracing", + "whoami", +] + +[[package]] +name = "sqlx-postgres" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6ac0ac3b7ccd10cc96c7ab29791a7dd236bd94021f31eec7ba3d46a74aa1c24" +dependencies = [ + "atoi", + "base64 0.21.5", + "bigdecimal", + "bitflags 2.4.1", + "byteorder", + "chrono", + "crc", + "dotenvy", + "etcetera", + "futures-channel", + "futures-core", + "futures-io", + "futures-util", + "hex", + "hkdf", + "hmac 0.12.1", + "home", + "ipnetwork", + "itoa", + "log", + "md-5", + "memchr", + "num-bigint 0.4.4", + "once_cell", + "rand 0.8.5", + "rust_decimal", + "serde", + "serde_json", + "sha1", + "sha2 0.10.8", + "smallvec", + "sqlx-core", + "stringprep", + "thiserror", + "tracing", + "whoami", +] + +[[package]] +name = "sqlx-sqlite" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "210976b7d948c7ba9fced8ca835b11cbb2d677c59c79de41ac0d397e14547490" +dependencies = [ + "atoi", + "chrono", + "flume", + "futures-channel", + "futures-core", + "futures-executor", + "futures-intrusive", + "futures-util", + "libsqlite3-sys", + "log", + "percent-encoding", + "serde", + "sqlx-core", + "tracing", + "url", + "urlencoding", ] [[package]] @@ -6675,6 +6971,18 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "syn_derive" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1329189c02ff984e9736652b1631330da25eaa6bc639089ed4915d25446cbe7b" +dependencies = [ + "proc-macro-error", + "proc-macro2 1.0.69", + "quote 1.0.33", + "syn 2.0.38", +] + [[package]] name = "sync_vm" version = "1.3.3" @@ -6685,7 +6993,7 @@ dependencies = [ "derivative", "franklin-crypto 0.0.5 (git+https://github.com/matter-labs/franklin-crypto?branch=dev)", "hex", - "itertools", + "itertools 0.10.5", "num-bigint 0.4.4", "num-derive 0.3.3", "num-integer", @@ -6972,7 +7280,7 @@ dependencies = [ "libc", "mio", "num_cpus", - "parking_lot 0.12.1", + "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2 0.5.5", @@ -7655,7 +7963,7 @@ dependencies = [ "jsonrpc-core", "log", "once_cell", - "parking_lot 0.12.1", + "parking_lot", "pin-project", "reqwest", "rlp", @@ -7698,10 +8006,6 @@ name = "whoami" version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "22fc3756b8a9133049b26c7f61ab35416c130e8c09b660f5b3958b446f52cc50" -dependencies = [ - "wasm-bindgen", - "web-sys", -] [[package]] name = "winapi" @@ -8062,7 +8366,7 @@ dependencies = [ "cs_derive 0.1.0 (git+https://github.com/matter-labs/era-boojum.git?branch=main)", "derivative", "hex", - "itertools", + "itertools 0.10.5", "rand 0.4.6", "rand 0.8.5", "serde", @@ -8106,7 +8410,7 @@ dependencies = [ "codegen 0.2.0", "crossbeam 0.8.2", "derivative", - "env_logger 0.9.3", + "env_logger 0.10.0", "hex", "num-bigint 0.4.4", "num-integer", @@ -8440,7 +8744,7 @@ dependencies = [ "futures 0.3.28", "governor", "hex", - "itertools", + "itertools 0.10.5", "jsonrpsee", "lru", "metrics", @@ -8515,8 +8819,8 @@ dependencies = [ "bigdecimal", "bincode", "hex", - "itertools", - "num 0.3.1", + "itertools 0.10.5", + "num 0.4.1", "once_cell", "prost", "rand 0.8.5", @@ -8771,7 +9075,7 @@ name = "zksync_state" version = "0.1.0" dependencies = [ "anyhow", - "itertools", + "itertools 0.10.5", "mini-moka", "rand 0.8.5", "tempfile", @@ -8836,7 +9140,7 @@ dependencies = [ "codegen 0.1.0", "ethereum-types 0.12.1", "hex", - "num 0.3.1", + "num 0.4.1", "num_enum", "once_cell", "parity-crypto", @@ -8870,9 +9174,9 @@ dependencies = [ "bigdecimal", "futures 0.3.28", "hex", - "itertools", + "itertools 0.10.5", "metrics", - "num 0.3.1", + "num 0.4.1", "reqwest", "serde", "serde_json", @@ -8890,7 +9194,7 @@ version = "0.1.0" dependencies = [ "bigdecimal", "chrono", - "itertools", + "itertools 0.10.5", "jsonrpsee", "rlp", "serde", diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 01b3a1260802..b20ab0a9fb81 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -390,7 +390,6 @@ async fn main() -> anyhow::Result<()> { .build() .await .context("failed to build a connection_pool")?; - if opt.revert_pending_l1_batch { tracing::info!("Rolling pending L1 batch back.."); let reverter = BlockReverter::new( @@ -484,11 +483,17 @@ async fn main() -> anyhow::Result<()> { if let Some(last_correct_batch) = reorg_detector_last_correct_batch { tracing::info!("Performing rollback to L1 batch #{last_correct_batch}"); + let block_reverter_connection_pool = + ConnectionPool::builder(&config.postgres.database_url, 1) + .build() + .await + .context("failed to build a block reverter connection pool")?; + let reverter = BlockReverter::new( config.required.state_cache_path, config.required.merkle_tree_path, None, - connection_pool, + block_reverter_connection_pool, L1ExecutedBatchesRevert::Allowed, ); reverter diff --git a/core/lib/constants/Cargo.toml b/core/lib/constants/Cargo.toml index e7e12206da2c..0d3d09f83a76 100644 --- a/core/lib/constants/Cargo.toml +++ b/core/lib/constants/Cargo.toml @@ -20,5 +20,5 @@ num = "0.3.1" serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" once_cell = "1.13.0" -bigdecimal = "0.2.2" +bigdecimal = "0.3.0" hex = "0.4" diff --git a/core/lib/dal/.sqlx/query-00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f.json b/core/lib/dal/.sqlx/query-00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f.json new file mode 100644 index 000000000000..49a533897ce3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Time", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f" +} diff --git a/core/lib/dal/.sqlx/query-012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784.json b/core/lib/dal/.sqlx/query-012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784.json new file mode 100644 index 000000000000..fbeefdfbf956 --- /dev/null +++ b/core/lib/dal/.sqlx/query-012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n eth_commit_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int8", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784" +} diff --git a/core/lib/dal/.sqlx/query-015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b.json b/core/lib/dal/.sqlx/query-015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b.json new file mode 100644 index 000000000000..d8495583ba97 --- /dev/null +++ b/core/lib/dal/.sqlx/query-015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(id) AS \"max?\"\n FROM\n protocol_versions\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "max?", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b" +} diff --git a/core/lib/dal/.sqlx/query-01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39.json b/core/lib/dal/.sqlx/query-01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39.json new file mode 100644 index 000000000000..8ca4bb693c23 --- /dev/null +++ b/core/lib/dal/.sqlx/query-01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39.json @@ -0,0 +1,107 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n node_aggregation_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC,\n depth ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n node_aggregation_witness_jobs_fri.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "depth", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "attempts", + "type_info": "Int2" + }, + { + "ordinal": 6, + "name": "aggregations_url", + "type_info": "Text" + }, + { + "ordinal": 7, + "name": "processing_started_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "time_taken", + "type_info": "Time" + }, + { + "ordinal": 9, + "name": "error", + "type_info": "Text" + }, + { + "ordinal": 10, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 11, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 12, + "name": "number_of_dependent_jobs", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "picked_by", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true + ] + }, + "hash": "01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39" +} diff --git a/core/lib/dal/.sqlx/query-01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b.json b/core/lib/dal/.sqlx/query-01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b.json new file mode 100644 index 000000000000..4a229dbe78d6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "factory_deps_filepath", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "storage_logs_filepaths", + "type_info": "TextArray" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b" +} diff --git a/core/lib/dal/.sqlx/query-01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba.json b/core/lib/dal/.sqlx/query-01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba.json new file mode 100644 index 000000000000..e28c68abc281 --- /dev/null +++ b/core/lib/dal/.sqlx/query-01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE storage\n SET\n value = u.value\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (key, value)\n WHERE\n u.key = hashed_key\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba" +} diff --git a/core/lib/dal/.sqlx/query-02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7.json b/core/lib/dal/.sqlx/query-02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7.json new file mode 100644 index 000000000000..41a37726f484 --- /dev/null +++ b/core/lib/dal/.sqlx/query-02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7.json @@ -0,0 +1,108 @@ +{ + "db_name": "PostgreSQL", + "query": "\n WITH\n sl AS (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.address = $1\n AND storage_logs.tx_hash = $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n )\n SELECT\n transactions.hash AS tx_hash,\n transactions.index_in_block AS index_in_block,\n transactions.l1_batch_tx_index AS l1_batch_tx_index,\n transactions.miniblock_number AS \"block_number!\",\n transactions.error AS error,\n transactions.effective_gas_price AS effective_gas_price,\n transactions.initiator_address AS initiator_address,\n transactions.data -> 'to' AS \"transfer_to?\",\n transactions.data -> 'contractAddress' AS \"execute_contract_address?\",\n transactions.tx_format AS \"tx_format?\",\n transactions.refunded_gas AS refunded_gas,\n transactions.gas_limit AS gas_limit,\n miniblocks.hash AS \"block_hash\",\n miniblocks.l1_batch_number AS \"l1_batch_number?\",\n sl.key AS \"contract_address?\"\n FROM\n transactions\n JOIN miniblocks ON miniblocks.number = transactions.miniblock_number\n LEFT JOIN sl ON sl.value != $3\n WHERE\n transactions.hash = $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "block_number!", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 5, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 6, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "transfer_to?", + "type_info": "Jsonb" + }, + { + "ordinal": 8, + "name": "execute_contract_address?", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "tx_format?", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 12, + "name": "block_hash", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "l1_batch_number?", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "contract_address?", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false, + true, + true, + true, + true, + true, + false, + null, + null, + true, + false, + true, + false, + true, + false + ] + }, + "hash": "02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7" +} diff --git a/core/lib/dal/.sqlx/query-026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b.json b/core/lib/dal/.sqlx/query-026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b.json new file mode 100644 index 000000000000..d98798241f7e --- /dev/null +++ b/core/lib/dal/.sqlx/query-026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS prove_tx ON (l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id)\n WHERE\n prove_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b" +} diff --git a/core/lib/dal/.sqlx/query-03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c.json b/core/lib/dal/.sqlx/query-03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c.json new file mode 100644 index 000000000000..9c811e9f87cc --- /dev/null +++ b/core/lib/dal/.sqlx/query-03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n storage_refunds\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "storage_refunds", + "type_info": "Int8Array" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c" +} diff --git a/core/lib/dal/.sqlx/query-040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981.json b/core/lib/dal/.sqlx/query-040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981.json new file mode 100644 index 000000000000..c0e0c777cc52 --- /dev/null +++ b/core/lib/dal/.sqlx/query-040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n snapshots\n WHERE\n NOT (''::TEXT = ANY (storage_logs_filepaths))\n ORDER BY\n l1_batch_number DESC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981" +} diff --git a/core/lib/dal/.sqlx/query-04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3.json b/core/lib/dal/.sqlx/query-04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3.json new file mode 100644 index 000000000000..00f94f7c864e --- /dev/null +++ b/core/lib/dal/.sqlx/query-04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MIN(miniblocks.number) AS \"min?\",\n MAX(miniblocks.number) AS \"max?\"\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "min?", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "max?", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + null, + null + ] + }, + "hash": "04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3" +} diff --git a/core/lib/dal/.sqlx/query-05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be.json b/core/lib/dal/.sqlx/query-05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be.json new file mode 100644 index 000000000000..81b6ad9687b6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n call_traces\n WHERE\n tx_hash IN (\n SELECT\n hash\n FROM\n transactions\n WHERE\n miniblock_number = $1\n )\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "call_trace", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be" +} diff --git a/core/lib/dal/.sqlx/query-07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa.json b/core/lib/dal/.sqlx/query-07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa.json new file mode 100644 index 000000000000..a293d217645c --- /dev/null +++ b/core/lib/dal/.sqlx/query-07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa.json @@ -0,0 +1,24 @@ +{ + "db_name": "PostgreSQL", + "query": "INSERT INTO eth_txs_history (eth_tx_id, base_fee_per_gas, priority_fee_per_gas, tx_hash, signed_raw_tx, created_at, updated_at, confirmed_at) VALUES ($1, 0, 0, $2, '\\x00', now(), now(), $3) RETURNING id", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int4", + "Text", + "Timestamp" + ] + }, + "nullable": [ + false + ] + }, + "hash": "07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa" +} diff --git a/core/lib/dal/.sqlx/query-083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643.json b/core/lib/dal/.sqlx/query-083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643.json new file mode 100644 index 000000000000..e2c3a7105562 --- /dev/null +++ b/core/lib/dal/.sqlx/query-083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643.json @@ -0,0 +1,52 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $3,\n error = $4\n WHERE\n l1_batch_number = $2\n AND status != $5\n RETURNING\n basic_witness_input_producer_jobs.attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + }, + "Int8", + "Time", + "Text", + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + } + ] + }, + "nullable": [ + false + ] + }, + "hash": "083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643" +} diff --git a/core/lib/dal/.sqlx/query-08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87.json b/core/lib/dal/.sqlx/query-08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87.json new file mode 100644 index 000000000000..0c3ca92c10c5 --- /dev/null +++ b/core/lib/dal/.sqlx/query-08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n proof_generation_details (l1_batch_number, status, proof_gen_data_blob_url, created_at, updated_at)\n VALUES\n ($1, 'ready_to_be_proven', $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text" + ] + }, + "nullable": [] + }, + "hash": "08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87" +} diff --git a/core/lib/dal/.sqlx/query-0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a.json b/core/lib/dal/.sqlx/query-0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a.json new file mode 100644 index 000000000000..b8e36d109065 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status NOT IN ('generated', 'skipped')\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a" +} diff --git a/core/lib/dal/.sqlx/query-0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674.json b/core/lib/dal/.sqlx/query-0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674.json new file mode 100644 index 000000000000..f0e439d0e0b8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "SELECT COUNT(*) FROM storage_logs WHERE miniblock_number = $1", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + null + ] + }, + "hash": "0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674" +} diff --git a/core/lib/dal/.sqlx/query-0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f.json b/core/lib/dal/.sqlx/query-0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f.json new file mode 100644 index 000000000000..854e34b4f184 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n protocol_versions (\n id,\n timestamp,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n bootloader_code_hash,\n default_account_code_hash,\n verifier_address,\n upgrade_tx_hash,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, NOW())\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int8", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f" +} diff --git a/core/lib/dal/.sqlx/query-0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc.json b/core/lib/dal/.sqlx/query-0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc.json new file mode 100644 index 000000000000..00379abe6df7 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc.json @@ -0,0 +1,31 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n gas_limit,\n max_fee_per_gas,\n gas_per_pubdata_limit,\n data,\n priority_op_id,\n full_fee,\n layer_2_tip_fee,\n contract_address,\n l1_block_number,\n value,\n paymaster,\n paymaster_input,\n tx_format,\n l1_tx_mint,\n l1_tx_refund_recipient,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n TRUE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n NOW(),\n NOW()\n )\n ON CONFLICT (hash) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Numeric", + "Numeric", + "Numeric", + "Jsonb", + "Int8", + "Numeric", + "Numeric", + "Bytea", + "Int4", + "Numeric", + "Bytea", + "Bytea", + "Int4", + "Numeric", + "Bytea", + "Timestamp" + ] + }, + "nullable": [] + }, + "hash": "0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc" +} diff --git a/core/lib/dal/.sqlx/query-0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8.json b/core/lib/dal/.sqlx/query-0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8.json new file mode 100644 index 000000000000..688a7373d05c --- /dev/null +++ b/core/lib/dal/.sqlx/query-0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8.json @@ -0,0 +1,259 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n l1_batches.timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n JOIN protocol_versions ON protocol_versions.id = l1_batches.protocol_version\n WHERE\n eth_commit_tx_id IS NULL\n AND number != 0\n AND protocol_versions.bootloader_code_hash = $1\n AND protocol_versions.default_account_code_hash = $2\n AND commitment IS NOT NULL\n AND (\n protocol_versions.id = $3\n OR protocol_versions.upgrade_tx_hash IS NULL\n )\n ORDER BY\n number\n LIMIT\n $4\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Int4", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8" +} diff --git a/core/lib/dal/.sqlx/query-0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8.json b/core/lib/dal/.sqlx/query-0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8.json new file mode 100644 index 000000000000..e5ed48130726 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8" +} diff --git a/core/lib/dal/.sqlx/query-0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30.json b/core/lib/dal/.sqlx/query-0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30.json new file mode 100644 index 000000000000..35c1633fc557 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "virtual_blocks", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30" +} diff --git a/core/lib/dal/.sqlx/query-0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41.json b/core/lib/dal/.sqlx/query-0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41.json new file mode 100644 index 000000000000..100761f54b41 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n protocol_version\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "protocol_version", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41" +} diff --git a/core/lib/dal/.sqlx/query-0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626.json b/core/lib/dal/.sqlx/query-0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626.json new file mode 100644 index 000000000000..8b5605f078a5 --- /dev/null +++ b/core/lib/dal/.sqlx/query-0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626.json @@ -0,0 +1,72 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = 'reserved',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n gpu_prover_queue_fri\n WHERE\n specialized_prover_group_id = $2\n AND zone = $3\n AND (\n instance_status = 'available'\n OR (\n instance_status = 'reserved'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n )\n ORDER BY\n updated_at ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n gpu_prover_queue_fri.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "instance_host", + "type_info": "Inet" + }, + { + "ordinal": 2, + "name": "instance_port", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "instance_status", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "specialized_prover_group_id", + "type_info": "Int2" + }, + { + "ordinal": 5, + "name": "zone", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "processing_started_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false, + false, + true + ] + }, + "hash": "0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626" +} diff --git a/core/lib/dal/.sqlx/query-10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730.json b/core/lib/dal/.sqlx/query-10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730.json new file mode 100644 index 000000000000..8f929a7a7336 --- /dev/null +++ b/core/lib/dal/.sqlx/query-10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n verification_info\n FROM\n contracts_verification_info\n WHERE\n address = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "verification_info", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + true + ] + }, + "hash": "10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730" +} diff --git a/core/lib/dal/.sqlx/query-11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9.json b/core/lib/dal/.sqlx/query-11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9.json new file mode 100644 index 000000000000..ed211d7dc9d8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_generation_details\n SET\n status = 'picked_by_prover',\n updated_at = NOW(),\n prover_taken_at = NOW()\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status = 'ready_to_be_proven'\n OR (\n status = 'picked_by_prover'\n AND prover_taken_at < NOW() - $1::INTERVAL\n )\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n proof_generation_details.l1_batch_number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Interval" + ] + }, + "nullable": [ + false + ] + }, + "hash": "11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9" +} diff --git a/core/lib/dal/.sqlx/query-12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4.json b/core/lib/dal/.sqlx/query-12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4.json new file mode 100644 index 000000000000..5441bce3e016 --- /dev/null +++ b/core/lib/dal/.sqlx/query-12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4" +} diff --git a/core/lib/dal/.sqlx/query-12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2.json b/core/lib/dal/.sqlx/query-12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2.json new file mode 100644 index 000000000000..556867a21ff4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_address\n FROM\n tokens\n WHERE\n well_known = FALSE\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_address", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2" +} diff --git a/core/lib/dal/.sqlx/query-136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2.json b/core/lib/dal/.sqlx/query-136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2.json new file mode 100644 index 000000000000..fc33c9693033 --- /dev/null +++ b/core/lib/dal/.sqlx/query-136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2.json @@ -0,0 +1,82 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n protocol_versions\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "recursion_scheduler_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "recursion_node_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "recursion_leaf_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "recursion_circuits_set_vks_hash", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "default_account_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "verifier_address", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "upgrade_tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "created_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + false + ] + }, + "hash": "136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2" +} diff --git a/core/lib/dal/.sqlx/query-15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e.json b/core/lib/dal/.sqlx/query-15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e.json new file mode 100644 index 000000000000..ac0e433a9195 --- /dev/null +++ b/core/lib/dal/.sqlx/query-15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e.json @@ -0,0 +1,25 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id\n FROM\n prover_fri_protocol_versions\n WHERE\n recursion_circuits_set_vks_hash = $1\n AND recursion_leaf_level_vk_hash = $2\n AND recursion_node_level_vk_hash = $3\n AND recursion_scheduler_level_vk_hash = $4\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e" +} diff --git a/core/lib/dal/.sqlx/query-15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20.json b/core/lib/dal/.sqlx/query-15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20.json new file mode 100644 index 000000000000..8e4c16cca415 --- /dev/null +++ b/core/lib/dal/.sqlx/query-15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20.json @@ -0,0 +1,94 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n miniblocks.number,\n COALESCE(\n miniblocks.l1_batch_number,\n (\n SELECT\n (MAX(number) + 1)\n FROM\n l1_batches\n )\n ) AS \"l1_batch_number!\",\n (\n SELECT\n MAX(m2.number)\n FROM\n miniblocks m2\n WHERE\n miniblocks.l1_batch_number = m2.l1_batch_number\n ) AS \"last_batch_miniblock?\",\n miniblocks.timestamp,\n miniblocks.l1_gas_price,\n miniblocks.l2_fair_gas_price,\n miniblocks.bootloader_code_hash,\n miniblocks.default_aa_code_hash,\n miniblocks.virtual_blocks,\n miniblocks.hash,\n miniblocks.consensus,\n miniblocks.protocol_version AS \"protocol_version!\",\n l1_batches.fee_account_address AS \"fee_account_address?\"\n FROM\n miniblocks\n LEFT JOIN l1_batches ON miniblocks.l1_batch_number = l1_batches.number\n WHERE\n miniblocks.number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number!", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "last_batch_miniblock?", + "type_info": "Int8" + }, + { + "ordinal": 3, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 5, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "virtual_blocks", + "type_info": "Int8" + }, + { + "ordinal": 9, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "consensus", + "type_info": "Jsonb" + }, + { + "ordinal": 11, + "name": "protocol_version!", + "type_info": "Int4" + }, + { + "ordinal": 12, + "name": "fee_account_address?", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + null, + null, + false, + false, + false, + true, + true, + false, + false, + true, + true, + false + ] + }, + "hash": "15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20" +} diff --git a/core/lib/dal/.sqlx/query-1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632.json b/core/lib/dal/.sqlx/query-1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632.json new file mode 100644 index 000000000000..7b939d137db9 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632.json @@ -0,0 +1,230 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number IS NOT NULL\n AND l1_batch_number IS NULL\n ORDER BY\n miniblock_number,\n index_in_block\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632" +} diff --git a/core/lib/dal/.sqlx/query-16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5.json b/core/lib/dal/.sqlx/query-16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5.json new file mode 100644 index 000000000000..3ba2e9b5448b --- /dev/null +++ b/core/lib/dal/.sqlx/query-16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5.json @@ -0,0 +1,259 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n l1_batches.timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n JOIN protocol_versions ON protocol_versions.id = l1_batches.protocol_version\n WHERE\n eth_commit_tx_id IS NULL\n AND number != 0\n AND protocol_versions.bootloader_code_hash = $1\n AND protocol_versions.default_account_code_hash = $2\n AND commitment IS NOT NULL\n AND (\n protocol_versions.id = $3\n OR protocol_versions.upgrade_tx_hash IS NULL\n )\n AND events_queue_commitment IS NOT NULL\n AND bootloader_initial_content_commitment IS NOT NULL\n ORDER BY\n number\n LIMIT\n $4\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Int4", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5" +} diff --git a/core/lib/dal/.sqlx/query-1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331.json b/core/lib/dal/.sqlx/query-1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331.json new file mode 100644 index 000000000000..74c11e4c9a01 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n initial_writes\n WHERE\n hashed_key = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331" +} diff --git a/core/lib/dal/.sqlx/query-1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e.json b/core/lib/dal/.sqlx/query-1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e.json new file mode 100644 index 000000000000..1bb2d641befb --- /dev/null +++ b/core/lib/dal/.sqlx/query-1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e.json @@ -0,0 +1,88 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n transactions.is_priority,\n transactions.initiator_address,\n transactions.gas_limit,\n transactions.gas_per_pubdata_limit,\n transactions.received_at,\n transactions.miniblock_number,\n transactions.error,\n transactions.effective_gas_price,\n transactions.refunded_gas,\n commit_tx.tx_hash AS \"eth_commit_tx_hash?\",\n prove_tx.tx_hash AS \"eth_prove_tx_hash?\",\n execute_tx.tx_hash AS \"eth_execute_tx_hash?\"\n FROM\n transactions\n LEFT JOIN miniblocks ON miniblocks.number = transactions.miniblock_number\n LEFT JOIN l1_batches ON l1_batches.number = miniblocks.l1_batch_number\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n transactions.hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 1, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 5, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 7, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 8, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 9, + "name": "eth_commit_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 10, + "name": "eth_prove_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 11, + "name": "eth_execute_tx_hash?", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + false, + false + ] + }, + "hash": "1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e" +} diff --git a/core/lib/dal/.sqlx/query-18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96.json b/core/lib/dal/.sqlx/query-18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96.json new file mode 100644 index 000000000000..73887716338f --- /dev/null +++ b/core/lib/dal/.sqlx/query-18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM transactions\n WHERE\n in_mempool = TRUE\n AND initiator_address = ANY ($1)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96" +} diff --git a/core/lib/dal/.sqlx/query-19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89.json b/core/lib/dal/.sqlx/query-19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89.json new file mode 100644 index 000000000000..88093dcee18e --- /dev/null +++ b/core/lib/dal/.sqlx/query-19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89.json @@ -0,0 +1,17 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE contract_verification_requests\n SET\n status = 'failed',\n updated_at = NOW(),\n error = $2,\n compilation_errors = $3,\n panic_message = $4\n WHERE\n id = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text", + "Jsonb", + "Text" + ] + }, + "nullable": [] + }, + "hash": "19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89" +} diff --git a/core/lib/dal/.sqlx/query-19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2.json b/core/lib/dal/.sqlx/query-19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2.json new file mode 100644 index 000000000000..3273d9654aa4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2.json @@ -0,0 +1,80 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n protocol_versions\n ORDER BY\n id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "recursion_scheduler_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "recursion_node_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "recursion_leaf_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "recursion_circuits_set_vks_hash", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "default_account_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "verifier_address", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "upgrade_tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "created_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + false + ] + }, + "hash": "19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2" +} diff --git a/core/lib/dal/.sqlx/query-19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed.json b/core/lib/dal/.sqlx/query-19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed.json new file mode 100644 index 000000000000..b1156c907d47 --- /dev/null +++ b/core/lib/dal/.sqlx/query-19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "SELECT sent_at_block FROM eth_txs_history WHERE eth_tx_id = $1 AND sent_at_block IS NOT NULL ORDER BY created_at ASC LIMIT 1", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "sent_at_block", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + true + ] + }, + "hash": "19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed" +} diff --git a/core/lib/dal/.sqlx/query-1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e.json b/core/lib/dal/.sqlx/query-1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e.json new file mode 100644 index 000000000000..460f81615bf4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT DISTINCT\n ON (hashed_key) hashed_key\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n miniblock_number > $1\n ) inn\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e" +} diff --git a/core/lib/dal/.sqlx/query-1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6.json b/core/lib/dal/.sqlx/query-1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6.json new file mode 100644 index 000000000000..8b9995b3b0f7 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(priority_op_id) AS \"op_id\"\n FROM\n transactions\n WHERE\n is_priority = TRUE\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "op_id", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6" +} diff --git a/core/lib/dal/.sqlx/query-1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773.json b/core/lib/dal/.sqlx/query-1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773.json new file mode 100644 index 000000000000..0351691c3955 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n scheduler_witness_jobs_fri (\n l1_batch_number,\n scheduler_partial_input_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773" +} diff --git a/core/lib/dal/.sqlx/query-1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5.json b/core/lib/dal/.sqlx/query-1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5.json new file mode 100644 index 000000000000..a9d5b42d2148 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5.json @@ -0,0 +1,76 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE contract_verification_requests\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id = (\n SELECT\n id\n FROM\n contract_verification_requests\n WHERE\n status = 'queued'\n OR (\n status = 'in_progress'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n ORDER BY\n created_at\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n id,\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "source_code", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "contract_name", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "zk_compiler_version", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "compiler_version", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "optimization_used", + "type_info": "Bool" + }, + { + "ordinal": 7, + "name": "optimizer_mode", + "type_info": "Text" + }, + { + "ordinal": 8, + "name": "constructor_arguments", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "is_system", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [ + "Interval" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + true, + false, + false + ] + }, + "hash": "1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5" +} diff --git a/core/lib/dal/.sqlx/query-1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910.json b/core/lib/dal/.sqlx/query-1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910.json new file mode 100644 index 000000000000..747105fb444b --- /dev/null +++ b/core/lib/dal/.sqlx/query-1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM l2_to_l1_logs\n WHERE\n miniblock_number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910" +} diff --git a/core/lib/dal/.sqlx/query-1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c.json b/core/lib/dal/.sqlx/query-1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c.json new file mode 100644 index 000000000000..b8929febf761 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c.json @@ -0,0 +1,61 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n storage.value AS \"value!\",\n tokens.l1_address AS \"l1_address!\",\n tokens.l2_address AS \"l2_address!\",\n tokens.symbol AS \"symbol!\",\n tokens.name AS \"name!\",\n tokens.decimals AS \"decimals!\",\n tokens.usd_price AS \"usd_price?\"\n FROM\n storage\n INNER JOIN tokens ON storage.address = tokens.l2_address\n OR (\n storage.address = $2\n AND tokens.l2_address = $3\n )\n WHERE\n storage.hashed_key = ANY ($1)\n AND storage.value != $4\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "value!", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "l1_address!", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "l2_address!", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "symbol!", + "type_info": "Varchar" + }, + { + "ordinal": 4, + "name": "name!", + "type_info": "Varchar" + }, + { + "ordinal": 5, + "name": "decimals!", + "type_info": "Int4" + }, + { + "ordinal": 6, + "name": "usd_price?", + "type_info": "Numeric" + } + ], + "parameters": { + "Left": [ + "ByteaArray", + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + true + ] + }, + "hash": "1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c" +} diff --git a/core/lib/dal/.sqlx/query-1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4.json b/core/lib/dal/.sqlx/query-1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4.json new file mode 100644 index 000000000000..7531f7ed4c00 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4.json @@ -0,0 +1,30 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n contract_verification_requests (\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, 'queued', NOW(), NOW())\n RETURNING\n id\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Text", + "Text", + "Text", + "Text", + "Bool", + "Text", + "Bytea", + "Bool" + ] + }, + "nullable": [ + false + ] + }, + "hash": "1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4" +} diff --git a/core/lib/dal/.sqlx/query-1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af.json b/core/lib/dal/.sqlx/query-1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af.json new file mode 100644 index 000000000000..dd142601d007 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n bytecode_hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af" +} diff --git a/core/lib/dal/.sqlx/query-1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050.json b/core/lib/dal/.sqlx/query-1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050.json new file mode 100644 index 000000000000..7e970780d8ed --- /dev/null +++ b/core/lib/dal/.sqlx/query-1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050.json @@ -0,0 +1,232 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number = $1\n ORDER BY\n index_in_block\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050" +} diff --git a/core/lib/dal/.sqlx/query-1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212.json b/core/lib/dal/.sqlx/query-1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212.json new file mode 100644 index 000000000000..1ae6117e4a15 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212" +} diff --git a/core/lib/dal/.sqlx/query-1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03.json b/core/lib/dal/.sqlx/query-1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03.json new file mode 100644 index 000000000000..9cf4cc1e68e1 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n protocol_version\n FROM\n l1_batches\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "protocol_version", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + true + ] + }, + "hash": "1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03" +} diff --git a/core/lib/dal/.sqlx/query-1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c.json b/core/lib/dal/.sqlx/query-1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c.json new file mode 100644 index 000000000000..b535ae5a8636 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c.json @@ -0,0 +1,17 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE tokens\n SET\n token_list_name = $2,\n token_list_symbol = $3,\n token_list_decimals = $4,\n well_known = TRUE,\n updated_at = NOW()\n WHERE\n l1_address = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Varchar", + "Varchar", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c" +} diff --git a/core/lib/dal/.sqlx/query-1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03.json b/core/lib/dal/.sqlx/query-1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03.json new file mode 100644 index 000000000000..077554611f4f --- /dev/null +++ b/core/lib/dal/.sqlx/query-1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n hash = data_table.hash,\n signature = data_table.signature,\n gas_limit = data_table.gas_limit,\n max_fee_per_gas = data_table.max_fee_per_gas,\n max_priority_fee_per_gas = data_table.max_priority_fee_per_gas,\n gas_per_pubdata_limit = data_table.gas_per_pubdata_limit,\n input = data_table.input,\n data = data_table.data,\n tx_format = data_table.tx_format,\n miniblock_number = $21,\n index_in_block = data_table.index_in_block,\n error = NULLIF(data_table.error, ''),\n effective_gas_price = data_table.effective_gas_price,\n execution_info = data_table.new_execution_info,\n refunded_gas = data_table.refunded_gas,\n value = data_table.value,\n contract_address = data_table.contract_address,\n paymaster = data_table.paymaster,\n paymaster_input = data_table.paymaster_input,\n in_mempool = FALSE,\n updated_at = NOW()\n FROM\n (\n SELECT\n data_table_temp.*\n FROM\n (\n SELECT\n UNNEST($1::bytea[]) AS initiator_address,\n UNNEST($2::INT[]) AS nonce,\n UNNEST($3::bytea[]) AS hash,\n UNNEST($4::bytea[]) AS signature,\n UNNEST($5::NUMERIC[]) AS gas_limit,\n UNNEST($6::NUMERIC[]) AS max_fee_per_gas,\n UNNEST($7::NUMERIC[]) AS max_priority_fee_per_gas,\n UNNEST($8::NUMERIC[]) AS gas_per_pubdata_limit,\n UNNEST($9::INT[]) AS tx_format,\n UNNEST($10::INTEGER[]) AS index_in_block,\n UNNEST($11::VARCHAR[]) AS error,\n UNNEST($12::NUMERIC[]) AS effective_gas_price,\n UNNEST($13::jsonb[]) AS new_execution_info,\n UNNEST($14::bytea[]) AS input,\n UNNEST($15::jsonb[]) AS data,\n UNNEST($16::BIGINT[]) AS refunded_gas,\n UNNEST($17::NUMERIC[]) AS value,\n UNNEST($18::bytea[]) AS contract_address,\n UNNEST($19::bytea[]) AS paymaster,\n UNNEST($20::bytea[]) AS paymaster_input\n ) AS data_table_temp\n JOIN transactions ON transactions.initiator_address = data_table_temp.initiator_address\n AND transactions.nonce = data_table_temp.nonce\n ORDER BY\n transactions.hash\n ) AS data_table\n WHERE\n transactions.initiator_address = data_table.initiator_address\n AND transactions.nonce = data_table.nonce\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "Int4Array", + "ByteaArray", + "ByteaArray", + "NumericArray", + "NumericArray", + "NumericArray", + "NumericArray", + "Int4Array", + "Int4Array", + "VarcharArray", + "NumericArray", + "JsonbArray", + "ByteaArray", + "JsonbArray", + "Int8Array", + "NumericArray", + "ByteaArray", + "ByteaArray", + "ByteaArray", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03" +} diff --git a/core/lib/dal/.sqlx/query-1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47.json b/core/lib/dal/.sqlx/query-1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47.json new file mode 100644 index 000000000000..362c775ea5a7 --- /dev/null +++ b/core/lib/dal/.sqlx/query-1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47.json @@ -0,0 +1,106 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batches.number,\n l1_batches.timestamp,\n l1_batches.l1_tx_count,\n l1_batches.l2_tx_count,\n l1_batches.hash AS \"root_hash?\",\n commit_tx.tx_hash AS \"commit_tx_hash?\",\n commit_tx.confirmed_at AS \"committed_at?\",\n prove_tx.tx_hash AS \"prove_tx_hash?\",\n prove_tx.confirmed_at AS \"proven_at?\",\n execute_tx.tx_hash AS \"execute_tx_hash?\",\n execute_tx.confirmed_at AS \"executed_at?\",\n l1_batches.l1_gas_price,\n l1_batches.l2_fair_gas_price,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n l1_batches.number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "root_hash?", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "commit_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "committed_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "prove_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 8, + "name": "proven_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 9, + "name": "execute_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 10, + "name": "executed_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 11, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 14, + "name": "default_aa_code_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + true, + false, + true, + false, + true, + false, + true, + false, + false, + true, + true + ] + }, + "hash": "1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47" +} diff --git a/core/lib/dal/.sqlx/query-2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95.json b/core/lib/dal/.sqlx/query-2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95.json new file mode 100644 index 000000000000..77177e405f12 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n transactions.hash,\n transactions.received_at\n FROM\n transactions\n LEFT JOIN miniblocks ON miniblocks.number = miniblock_number\n WHERE\n received_at > $1\n ORDER BY\n received_at ASC\n LIMIT\n $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "received_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Timestamp", + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95" +} diff --git a/core/lib/dal/.sqlx/query-2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a.json b/core/lib/dal/.sqlx/query-2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a.json new file mode 100644 index 000000000000..8aaefe3c6ba6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n miniblock_number <= $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a" +} diff --git a/core/lib/dal/.sqlx/query-20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a.json b/core/lib/dal/.sqlx/query-20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a.json new file mode 100644 index 000000000000..5f7048a8a20a --- /dev/null +++ b/core/lib/dal/.sqlx/query-20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n skip_proof = TRUE\n WHERE\n number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a" +} diff --git a/core/lib/dal/.sqlx/query-23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9.json b/core/lib/dal/.sqlx/query-23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9.json new file mode 100644 index 000000000000..8c63a924c0aa --- /dev/null +++ b/core/lib/dal/.sqlx/query-23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9.json @@ -0,0 +1,86 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n confirmed_eth_tx_history_id IS NULL\n AND id <= (\n SELECT\n COALESCE(MAX(eth_tx_id), 0)\n FROM\n eth_txs_history\n WHERE\n sent_at_block IS NOT NULL\n )\n ORDER BY\n id\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "contract_address", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "tx_type", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "gas_used", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "has_failed", + "type_info": "Bool" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "confirmed_eth_tx_history_id", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "predicted_gas_cost", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false, + false, + false, + true, + true, + false + ] + }, + "hash": "23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9" +} diff --git a/core/lib/dal/.sqlx/query-245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30.json b/core/lib/dal/.sqlx/query-245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30.json new file mode 100644 index 000000000000..0b9c4aa59b7a --- /dev/null +++ b/core/lib/dal/.sqlx/query-245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n eth_prove_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int8", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30" +} diff --git a/core/lib/dal/.sqlx/query-24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a.json b/core/lib/dal/.sqlx/query-24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a.json new file mode 100644 index 000000000000..194f4faedb1b --- /dev/null +++ b/core/lib/dal/.sqlx/query-24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hashed_key,\n INDEX\n FROM\n initial_writes\n WHERE\n l1_batch_number = $1\n ORDER BY\n INDEX\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a" +} diff --git a/core/lib/dal/.sqlx/query-249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da.json b/core/lib/dal/.sqlx/query-249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da.json new file mode 100644 index 000000000000..38419af111f0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode_hash\n FROM\n factory_deps\n WHERE\n miniblock_number > $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da" +} diff --git a/core/lib/dal/.sqlx/query-25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a.json b/core/lib/dal/.sqlx/query-25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a.json new file mode 100644 index 000000000000..d966ff14c99a --- /dev/null +++ b/core/lib/dal/.sqlx/query-25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a.json @@ -0,0 +1,124 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n miniblocks.number,\n COALESCE(\n miniblocks.l1_batch_number,\n (\n SELECT\n (MAX(number) + 1)\n FROM\n l1_batches\n )\n ) AS \"l1_batch_number!\",\n miniblocks.timestamp,\n miniblocks.l1_tx_count,\n miniblocks.l2_tx_count,\n miniblocks.hash AS \"root_hash?\",\n commit_tx.tx_hash AS \"commit_tx_hash?\",\n commit_tx.confirmed_at AS \"committed_at?\",\n prove_tx.tx_hash AS \"prove_tx_hash?\",\n prove_tx.confirmed_at AS \"proven_at?\",\n execute_tx.tx_hash AS \"execute_tx_hash?\",\n execute_tx.confirmed_at AS \"executed_at?\",\n miniblocks.l1_gas_price,\n miniblocks.l2_fair_gas_price,\n miniblocks.bootloader_code_hash,\n miniblocks.default_aa_code_hash,\n miniblocks.protocol_version,\n l1_batches.fee_account_address AS \"fee_account_address?\"\n FROM\n miniblocks\n LEFT JOIN l1_batches ON miniblocks.l1_batch_number = l1_batches.number\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n miniblocks.number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number!", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "root_hash?", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "commit_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 7, + "name": "committed_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "prove_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 9, + "name": "proven_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "execute_tx_hash?", + "type_info": "Text" + }, + { + "ordinal": 11, + "name": "executed_at?", + "type_info": "Timestamp" + }, + { + "ordinal": 12, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 15, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 16, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 17, + "name": "fee_account_address?", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + null, + false, + false, + false, + false, + false, + true, + false, + true, + false, + true, + false, + false, + true, + true, + true, + false + ] + }, + "hash": "25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a" +} diff --git a/core/lib/dal/.sqlx/query-26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2.json b/core/lib/dal/.sqlx/query-26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2.json new file mode 100644 index 000000000000..58ba7c33f2b6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2.json @@ -0,0 +1,40 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n miniblock_number AS \"miniblock_number!\",\n hash,\n index_in_block AS \"index_in_block!\",\n l1_batch_tx_index AS \"l1_batch_tx_index!\"\n FROM\n transactions\n WHERE\n l1_batch_number = $1\n ORDER BY\n miniblock_number,\n index_in_block\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "miniblock_number!", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "index_in_block!", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "l1_batch_tx_index!", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true, + false, + true, + true + ] + }, + "hash": "26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2" +} diff --git a/core/lib/dal/.sqlx/query-26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20.json b/core/lib/dal/.sqlx/query-26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20.json new file mode 100644 index 000000000000..30738bc2094a --- /dev/null +++ b/core/lib/dal/.sqlx/query-26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_generation_details\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20" +} diff --git a/core/lib/dal/.sqlx/query-2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113.json b/core/lib/dal/.sqlx/query-2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113.json new file mode 100644 index 000000000000..67b9c0566821 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n error = $1,\n updated_at = NOW()\n WHERE\n hash = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Varchar", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113" +} diff --git a/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json b/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json new file mode 100644 index 000000000000..bb47b8254aca --- /dev/null +++ b/core/lib/dal/.sqlx/query-2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bootloader_code_hash,\n default_account_code_hash,\n id\n FROM\n protocol_versions\n WHERE\n timestamp <= $1\n ORDER BY\n id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "default_account_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef" +} diff --git a/core/lib/dal/.sqlx/query-280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b.json b/core/lib/dal/.sqlx/query-280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b.json new file mode 100644 index 000000000000..5b49941ed18f --- /dev/null +++ b/core/lib/dal/.sqlx/query-280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n basic_witness_input_producer_jobs\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b" +} diff --git a/core/lib/dal/.sqlx/query-293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb.json b/core/lib/dal/.sqlx/query-293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb.json new file mode 100644 index 000000000000..2b07c3b02e04 --- /dev/null +++ b/core/lib/dal/.sqlx/query-293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM events\n WHERE\n miniblock_number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb" +} diff --git a/core/lib/dal/.sqlx/query-2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d.json b/core/lib/dal/.sqlx/query-2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d.json new file mode 100644 index 000000000000..7c3a261d1f6e --- /dev/null +++ b/core/lib/dal/.sqlx/query-2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d" +} diff --git a/core/lib/dal/.sqlx/query-2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb.json b/core/lib/dal/.sqlx/query-2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb.json new file mode 100644 index 000000000000..8bcd8098baeb --- /dev/null +++ b/core/lib/dal/.sqlx/query-2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number\n FROM\n miniblocks\n WHERE\n consensus IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb" +} diff --git a/core/lib/dal/.sqlx/query-2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785.json b/core/lib/dal/.sqlx/query-2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785.json new file mode 100644 index 000000000000..db810604cd88 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785.json @@ -0,0 +1,25 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n l1_batch_number = $1\n AND circuit_id = $2\n AND aggregation_round = $3\n AND depth = $4\n AND status = 'successful'\n ORDER BY\n sequence_number ASC;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int2", + "Int2", + "Int4" + ] + }, + "nullable": [ + false + ] + }, + "hash": "2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785" +} diff --git a/core/lib/dal/.sqlx/query-2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23.json b/core/lib/dal/.sqlx/query-2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23.json new file mode 100644 index 000000000000..9ec94fee0d34 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n miniblocks\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23" +} diff --git a/core/lib/dal/.sqlx/query-2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e.json b/core/lib/dal/.sqlx/query-2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e.json new file mode 100644 index 000000000000..f61f39e3b0b0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n contracts_verification_info\n WHERE\n address = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + null + ] + }, + "hash": "2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e" +} diff --git a/core/lib/dal/.sqlx/query-2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44.json b/core/lib/dal/.sqlx/query-2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44.json new file mode 100644 index 000000000000..1d9c276b0786 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n contracts_verification_info (address, verification_info)\n VALUES\n ($1, $2)\n ON CONFLICT (address) DO\n UPDATE\n SET\n verification_info = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Jsonb" + ] + }, + "nullable": [] + }, + "hash": "2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44" +} diff --git a/core/lib/dal/.sqlx/query-2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c.json b/core/lib/dal/.sqlx/query-2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c.json new file mode 100644 index 000000000000..c4bcd6ea4915 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'in_gpu_proof'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n id,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c" +} diff --git a/core/lib/dal/.sqlx/query-2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2.json b/core/lib/dal/.sqlx/query-2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2.json new file mode 100644 index 000000000000..e08f0c855175 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n INDEX\n FROM\n initial_writes\n WHERE\n l1_batch_number <= $1\n ORDER BY\n l1_batch_number DESC,\n INDEX DESC\n LIMIT\n 1;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2" +} diff --git a/core/lib/dal/.sqlx/query-2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a.json b/core/lib/dal/.sqlx/query-2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a.json new file mode 100644 index 000000000000..dbd2e21c1a3e --- /dev/null +++ b/core/lib/dal/.sqlx/query-2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM storage_logs\n WHERE\n miniblock_number = $1\n AND operation_number != ALL ($2)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int4Array" + ] + }, + "nullable": [] + }, + "hash": "2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a" +} diff --git a/core/lib/dal/.sqlx/query-2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9.json b/core/lib/dal/.sqlx/query-2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9.json new file mode 100644 index 000000000000..bb81e7c31943 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9.json @@ -0,0 +1,232 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number = $1\n ORDER BY\n index_in_block\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9" +} diff --git a/core/lib/dal/.sqlx/query-2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3.json b/core/lib/dal/.sqlx/query-2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3.json new file mode 100644 index 000000000000..68b48f325091 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n l1_batch_number = NULL,\n miniblock_number = NULL,\n error = NULL,\n index_in_block = NULL,\n execution_info = '{}'\n WHERE\n miniblock_number > $1\n RETURNING\n hash\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3" +} diff --git a/core/lib/dal/.sqlx/query-2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c.json b/core/lib/dal/.sqlx/query-2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c.json new file mode 100644 index 000000000000..c2cba82b7a6b --- /dev/null +++ b/core/lib/dal/.sqlx/query-2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n eth_txs_history (\n eth_tx_id,\n base_fee_per_gas,\n priority_fee_per_gas,\n tx_hash,\n signed_raw_tx,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW(), NOW())\n ON CONFLICT (tx_hash) DO NOTHING\n RETURNING\n id\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int4", + "Int8", + "Int8", + "Text", + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c" +} diff --git a/core/lib/dal/.sqlx/query-2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583.json b/core/lib/dal/.sqlx/query-2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583.json new file mode 100644 index 000000000000..205487768306 --- /dev/null +++ b/core/lib/dal/.sqlx/query-2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n compiler_versions (VERSION, compiler, created_at, updated_at)\n SELECT\n u.version,\n $2,\n NOW(),\n NOW()\n FROM\n UNNEST($1::TEXT[]) AS u (VERSION)\n ON CONFLICT (VERSION, compiler) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "TextArray", + "Text" + ] + }, + "nullable": [] + }, + "hash": "2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583" +} diff --git a/core/lib/dal/.sqlx/query-2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71.json b/core/lib/dal/.sqlx/query-2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71.json new file mode 100644 index 000000000000..6b0ddef258fc --- /dev/null +++ b/core/lib/dal/.sqlx/query-2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n eth_commit_tx_id\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "eth_commit_tx_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71" +} diff --git a/core/lib/dal/.sqlx/query-2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12.json b/core/lib/dal/.sqlx/query-2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12.json new file mode 100644 index 000000000000..3baa8596a3ba --- /dev/null +++ b/core/lib/dal/.sqlx/query-2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n in_mempool = FALSE\n FROM\n UNNEST($1::bytea[]) AS s (address)\n WHERE\n transactions.in_mempool = TRUE\n AND transactions.initiator_address = s.address\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12" +} diff --git a/core/lib/dal/.sqlx/query-31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a.json b/core/lib/dal/.sqlx/query-31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a.json new file mode 100644 index 000000000000..31b129a89288 --- /dev/null +++ b/core/lib/dal/.sqlx/query-31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n snapshots (\n l1_batch_number,\n storage_logs_filepaths,\n factory_deps_filepath,\n created_at,\n updated_at\n )\n VALUES\n ($1, ARRAY_FILL(''::TEXT, ARRAY[$2::INTEGER]), $3, NOW(), NOW())\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Text" + ] + }, + "nullable": [] + }, + "hash": "31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a" +} diff --git a/core/lib/dal/.sqlx/query-314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37.json b/core/lib/dal/.sqlx/query-314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37.json new file mode 100644 index 000000000000..3034791742c4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n miniblocks (\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n gas_per_pubdata_limit,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, NOW(), NOW())\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int8", + "Bytea", + "Int4", + "Int4", + "Numeric", + "Int8", + "Int8", + "Int8", + "Bytea", + "Bytea", + "Int4", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37" +} diff --git a/core/lib/dal/.sqlx/query-31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8.json b/core/lib/dal/.sqlx/query-31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8.json new file mode 100644 index 000000000000..c63ea98db44b --- /dev/null +++ b/core/lib/dal/.sqlx/query-31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8" +} diff --git a/core/lib/dal/.sqlx/query-322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a.json b/core/lib/dal/.sqlx/query-322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a.json new file mode 100644 index 000000000000..804940f674df --- /dev/null +++ b/core/lib/dal/.sqlx/query-322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n commitment = $2,\n aux_data_hash = $3,\n updated_at = NOW()\n WHERE\n number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a" +} diff --git a/core/lib/dal/.sqlx/query-32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732.json b/core/lib/dal/.sqlx/query-32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732.json new file mode 100644 index 000000000000..32b36d7fed96 --- /dev/null +++ b/core/lib/dal/.sqlx/query-32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n basic_witness_input_producer_jobs (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + } + ] + }, + "nullable": [] + }, + "hash": "32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732" +} diff --git a/core/lib/dal/.sqlx/query-33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e.json b/core/lib/dal/.sqlx/query-33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e.json new file mode 100644 index 000000000000..76483cd73d31 --- /dev/null +++ b/core/lib/dal/.sqlx/query-33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id\n FROM\n prover_jobs_fri\n JOIN leaf_aggregation_witness_jobs_fri lawj ON prover_jobs_fri.l1_batch_number = lawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = lawj.circuit_id\n WHERE\n lawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 0\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n lawj.number_of_basic_circuits\n HAVING\n COUNT(*) = lawj.number_of_basic_circuits\n )\n RETURNING\n l1_batch_number,\n circuit_id;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "circuit_id", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false + ] + }, + "hash": "33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e" +} diff --git a/core/lib/dal/.sqlx/query-3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827.json b/core/lib/dal/.sqlx/query-3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827.json new file mode 100644 index 000000000000..3275e94936a1 --- /dev/null +++ b/core/lib/dal/.sqlx/query-3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hashed_key,\n l1_batch_number,\n INDEX\n FROM\n initial_writes\n WHERE\n hashed_key = ANY ($1::bytea[])\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827" +} diff --git a/core/lib/dal/.sqlx/query-35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530.json b/core/lib/dal/.sqlx/query-35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530.json new file mode 100644 index 000000000000..a11e154326ec --- /dev/null +++ b/core/lib/dal/.sqlx/query-35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530" +} diff --git a/core/lib/dal/.sqlx/query-3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a.json b/core/lib/dal/.sqlx/query-3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a.json new file mode 100644 index 000000000000..1e8d851ab07a --- /dev/null +++ b/core/lib/dal/.sqlx/query-3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a.json @@ -0,0 +1,258 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n LIMIT\n $3\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a" +} diff --git a/core/lib/dal/.sqlx/query-367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2.json b/core/lib/dal/.sqlx/query-367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2.json new file mode 100644 index 000000000000..0eadba8f7f53 --- /dev/null +++ b/core/lib/dal/.sqlx/query-367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n miniblocks\n WHERE\n number = $1\n AND consensus IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + null + ] + }, + "hash": "367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2" +} diff --git a/core/lib/dal/.sqlx/query-38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c.json b/core/lib/dal/.sqlx/query-38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c.json new file mode 100644 index 000000000000..9b989a9ba251 --- /dev/null +++ b/core/lib/dal/.sqlx/query-38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c" +} diff --git a/core/lib/dal/.sqlx/query-3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0.json b/core/lib/dal/.sqlx/query-3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0.json new file mode 100644 index 000000000000..39781954d486 --- /dev/null +++ b/core/lib/dal/.sqlx/query-3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hashed_key,\n value AS \"value!\"\n FROM\n storage\n WHERE\n hashed_key = ANY ($1)\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "value!", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0" +} diff --git a/core/lib/dal/.sqlx/query-3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28.json b/core/lib/dal/.sqlx/query-3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28.json new file mode 100644 index 000000000000..326915adb2fc --- /dev/null +++ b/core/lib/dal/.sqlx/query-3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM call_traces\n WHERE\n tx_hash = ANY ($1)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28" +} diff --git a/core/lib/dal/.sqlx/query-3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361.json b/core/lib/dal/.sqlx/query-3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361.json new file mode 100644 index 000000000000..38890ae58f2e --- /dev/null +++ b/core/lib/dal/.sqlx/query-3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n factory_deps (bytecode_hash, bytecode, miniblock_number, created_at, updated_at)\n SELECT\n u.bytecode_hash,\n u.bytecode,\n $3,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (bytecode_hash, bytecode)\n ON CONFLICT (bytecode_hash) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "ByteaArray", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361" +} diff --git a/core/lib/dal/.sqlx/query-3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231.json b/core/lib/dal/.sqlx/query-3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231.json new file mode 100644 index 000000000000..ad5c726ea130 --- /dev/null +++ b/core/lib/dal/.sqlx/query-3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n trace\n FROM\n transaction_traces\n WHERE\n tx_hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "trace", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231" +} diff --git a/core/lib/dal/.sqlx/query-3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64.json b/core/lib/dal/.sqlx/query-3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64.json new file mode 100644 index 000000000000..56d8b1fa9956 --- /dev/null +++ b/core/lib/dal/.sqlx/query-3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64" +} diff --git a/core/lib/dal/.sqlx/query-3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354.json b/core/lib/dal/.sqlx/query-3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354.json new file mode 100644 index 000000000000..2290d558cead --- /dev/null +++ b/core/lib/dal/.sqlx/query-3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM tokens\n WHERE\n l2_address IN (\n SELECT\n SUBSTRING(key, 12, 20)\n FROM\n storage_logs\n WHERE\n storage_logs.address = $1\n AND miniblock_number > $2\n AND NOT EXISTS (\n SELECT\n 1\n FROM\n storage_logs AS s\n WHERE\n s.hashed_key = storage_logs.hashed_key\n AND (s.miniblock_number, s.operation_number) >= (storage_logs.miniblock_number, storage_logs.operation_number)\n AND s.value = $3\n )\n )\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Int8", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354" +} diff --git a/core/lib/dal/.sqlx/query-3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa.json b/core/lib/dal/.sqlx/query-3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa.json new file mode 100644 index 000000000000..5815e65636cb --- /dev/null +++ b/core/lib/dal/.sqlx/query-3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n l1_batch_number = $1\n AND status != 'successful'\n AND status != 'in_progress'\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa" +} diff --git a/core/lib/dal/.sqlx/query-41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151.json b/core/lib/dal/.sqlx/query-41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151.json new file mode 100644 index 000000000000..8c51c26131bd --- /dev/null +++ b/core/lib/dal/.sqlx/query-41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151.json @@ -0,0 +1,56 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n eth_txs_history.id,\n eth_txs_history.eth_tx_id,\n eth_txs_history.tx_hash,\n eth_txs_history.base_fee_per_gas,\n eth_txs_history.priority_fee_per_gas,\n eth_txs_history.signed_raw_tx,\n eth_txs.nonce\n FROM\n eth_txs_history\n JOIN eth_txs ON eth_txs.id = eth_txs_history.eth_tx_id\n WHERE\n eth_txs_history.sent_at_block IS NULL\n AND eth_txs.confirmed_eth_tx_history_id IS NULL\n ORDER BY\n eth_txs_history.id DESC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "eth_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "tx_hash", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "base_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "priority_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 5, + "name": "signed_raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "nonce", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false + ] + }, + "hash": "41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151" +} diff --git a/core/lib/dal/.sqlx/query-43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1.json b/core/lib/dal/.sqlx/query-43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1.json new file mode 100644 index 000000000000..56fcdb389430 --- /dev/null +++ b/core/lib/dal/.sqlx/query-43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MIN(number) AS \"number\"\n FROM\n l1_batches\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1" +} diff --git a/core/lib/dal/.sqlx/query-46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c.json b/core/lib/dal/.sqlx/query-46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c.json new file mode 100644 index 000000000000..5ebb1951966d --- /dev/null +++ b/core/lib/dal/.sqlx/query-46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Time", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c" +} diff --git a/core/lib/dal/.sqlx/query-47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060.json b/core/lib/dal/.sqlx/query-47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060.json new file mode 100644 index 000000000000..fe8a346d1e21 --- /dev/null +++ b/core/lib/dal/.sqlx/query-47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060.json @@ -0,0 +1,50 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number,\n l1_batch_root_hash,\n miniblock_number,\n miniblock_root_hash,\n last_finished_chunk_id,\n total_chunk_count\n FROM\n snapshot_recovery\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 3, + "name": "miniblock_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "last_finished_chunk_id", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "total_chunk_count", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + true, + false + ] + }, + "hash": "47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060" +} diff --git a/core/lib/dal/.sqlx/query-481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae.json b/core/lib/dal/.sqlx/query-481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae.json new file mode 100644 index 000000000000..3a9c7616c9c1 --- /dev/null +++ b/core/lib/dal/.sqlx/query-481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n is_finished = TRUE\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae" +} diff --git a/core/lib/dal/.sqlx/query-4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b.json b/core/lib/dal/.sqlx/query-4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b.json new file mode 100644 index 000000000000..6210b70e4d68 --- /dev/null +++ b/core/lib/dal/.sqlx/query-4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b.json @@ -0,0 +1,134 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n pubdata_input\n FROM\n l1_batches\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 7, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 10, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 11, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 12, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 15, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 16, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 17, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 18, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + false, + true + ] + }, + "hash": "4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b" +} diff --git a/core/lib/dal/.sqlx/query-4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2.json b/core/lib/dal/.sqlx/query-4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2.json new file mode 100644 index 000000000000..b0fb8d4be23c --- /dev/null +++ b/core/lib/dal/.sqlx/query-4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2.json @@ -0,0 +1,59 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n aggregation_round DESC,\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "aggregation_round", + "type_info": "Int2" + }, + { + "ordinal": 4, + "name": "sequence_number", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "depth", + "type_info": "Int4" + }, + { + "ordinal": 6, + "name": "is_node_final_proof", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [ + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false + ] + }, + "hash": "4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2" +} diff --git a/core/lib/dal/.sqlx/query-4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d.json b/core/lib/dal/.sqlx/query-4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d.json new file mode 100644 index 000000000000..e9a9425da3ce --- /dev/null +++ b/core/lib/dal/.sqlx/query-4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "SELECT eth_txs.id FROM eth_txs_history JOIN eth_txs ON eth_txs.confirmed_eth_tx_history_id = eth_txs_history.id WHERE eth_txs_history.tx_hash = $1", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Text" + ] + }, + "nullable": [ + false + ] + }, + "hash": "4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d" +} diff --git a/core/lib/dal/.sqlx/query-4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783.json b/core/lib/dal/.sqlx/query-4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783.json new file mode 100644 index 000000000000..44d1506ac937 --- /dev/null +++ b/core/lib/dal/.sqlx/query-4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783.json @@ -0,0 +1,82 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n ORDER BY\n created_at DESC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "eth_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "tx_hash", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 4, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 5, + "name": "base_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "priority_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 7, + "name": "confirmed_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "signed_raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "sent_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true + ] + }, + "hash": "4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783" +} diff --git a/core/lib/dal/.sqlx/query-4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c.json b/core/lib/dal/.sqlx/query-4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c.json new file mode 100644 index 000000000000..f7ae37f4b7b3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c.json @@ -0,0 +1,18 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n prover_fri_protocol_versions (\n id,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW())\n ON CONFLICT (id) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Bytea", + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c" +} diff --git a/core/lib/dal/.sqlx/query-525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63.json b/core/lib/dal/.sqlx/query-525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63.json new file mode 100644 index 000000000000..7764425aa214 --- /dev/null +++ b/core/lib/dal/.sqlx/query-525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hash\n FROM\n miniblocks\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63" +} diff --git a/core/lib/dal/.sqlx/query-532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047.json b/core/lib/dal/.sqlx/query-532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047.json new file mode 100644 index 000000000000..629dca2ea7f0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE eth_txs_history\n SET\n sent_at_block = $2,\n sent_at = NOW()\n WHERE\n id = $1\n AND sent_at_block IS NULL\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047" +} diff --git a/core/lib/dal/.sqlx/query-534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56.json b/core/lib/dal/.sqlx/query-534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56.json new file mode 100644 index 000000000000..a85b4895b451 --- /dev/null +++ b/core/lib/dal/.sqlx/query-534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56" +} diff --git a/core/lib/dal/.sqlx/query-53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea.json b/core/lib/dal/.sqlx/query-53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea.json new file mode 100644 index 000000000000..e07b9192b5f4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n fee_account_address\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "fee_account_address", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea" +} diff --git a/core/lib/dal/.sqlx/query-53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39.json b/core/lib/dal/.sqlx/query-53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39.json new file mode 100644 index 000000000000..15a10f7ce3c5 --- /dev/null +++ b/core/lib/dal/.sqlx/query-53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n leaf_aggregation_witness_jobs_fri\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39" +} diff --git a/core/lib/dal/.sqlx/query-5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231.json b/core/lib/dal/.sqlx/query-5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231.json new file mode 100644 index 000000000000..5f27c7549b47 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "INSERT INTO eth_txs (raw_tx, nonce, tx_type, contract_address, predicted_gas_cost, created_at, updated_at) VALUES ('\\x00', 0, $1, '', 0, now(), now()) RETURNING id", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Text" + ] + }, + "nullable": [ + false + ] + }, + "hash": "5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231" +} diff --git a/core/lib/dal/.sqlx/query-556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d.json b/core/lib/dal/.sqlx/query-556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d.json new file mode 100644 index 000000000000..1dcfa982c51d --- /dev/null +++ b/core/lib/dal/.sqlx/query-556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n events_queue (l1_batch_number, serialized_events_queue)\n VALUES\n ($1, $2)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Jsonb" + ] + }, + "nullable": [] + }, + "hash": "556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d" +} diff --git a/core/lib/dal/.sqlx/query-55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60.json b/core/lib/dal/.sqlx/query-55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60.json new file mode 100644 index 000000000000..6478bb53538d --- /dev/null +++ b/core/lib/dal/.sqlx/query-55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM l1_batches\n WHERE\n number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60" +} diff --git a/core/lib/dal/.sqlx/query-5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33.json b/core/lib/dal/.sqlx/query-5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33.json new file mode 100644 index 000000000000..399b0d028459 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33.json @@ -0,0 +1,88 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n id > (\n SELECT\n COALESCE(MAX(eth_tx_id), 0)\n FROM\n eth_txs_history\n )\n ORDER BY\n id\n LIMIT\n $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "contract_address", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "tx_type", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "gas_used", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "has_failed", + "type_info": "Bool" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "confirmed_eth_tx_history_id", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "predicted_gas_cost", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false, + false, + false, + true, + true, + false + ] + }, + "hash": "5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33" +} diff --git a/core/lib/dal/.sqlx/query-5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85.json b/core/lib/dal/.sqlx/query-5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85.json new file mode 100644 index 000000000000..86877a48dd4d --- /dev/null +++ b/core/lib/dal/.sqlx/query-5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE witness_inputs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85" +} diff --git a/core/lib/dal/.sqlx/query-58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de.json b/core/lib/dal/.sqlx/query-58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de.json new file mode 100644 index 000000000000..502d14e05ea5 --- /dev/null +++ b/core/lib/dal/.sqlx/query-58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status = 'ready_to_be_proven'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de" +} diff --git a/core/lib/dal/.sqlx/query-59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f.json b/core/lib/dal/.sqlx/query-59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f.json new file mode 100644 index 000000000000..8a0cb19b3904 --- /dev/null +++ b/core/lib/dal/.sqlx/query-59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE tokens\n SET\n usd_price = $2,\n usd_price_updated_at = $3,\n updated_at = NOW()\n WHERE\n l1_address = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Numeric", + "Timestamp" + ] + }, + "nullable": [] + }, + "hash": "59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f" +} diff --git a/core/lib/dal/.sqlx/query-5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b.json b/core/lib/dal/.sqlx/query-5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b.json new file mode 100644 index 000000000000..069cd1956397 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b.json @@ -0,0 +1,256 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b" +} diff --git a/core/lib/dal/.sqlx/query-5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13.json b/core/lib/dal/.sqlx/query-5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13.json new file mode 100644 index 000000000000..fd793cb9b83b --- /dev/null +++ b/core/lib/dal/.sqlx/query-5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13.json @@ -0,0 +1,88 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 6, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 7, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 8, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "virtual_blocks", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + false + ] + }, + "hash": "5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13" +} diff --git a/core/lib/dal/.sqlx/query-5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6.json b/core/lib/dal/.sqlx/query-5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6.json new file mode 100644 index 000000000000..eba36994fb34 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bootloader_code_hash,\n default_account_code_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "default_account_code_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6" +} diff --git a/core/lib/dal/.sqlx/query-5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f.json b/core/lib/dal/.sqlx/query-5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f.json new file mode 100644 index 000000000000..4958f38f5358 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f" +} diff --git a/core/lib/dal/.sqlx/query-5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c.json b/core/lib/dal/.sqlx/query-5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c.json new file mode 100644 index 000000000000..b57400c28f5e --- /dev/null +++ b/core/lib/dal/.sqlx/query-5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c.json @@ -0,0 +1,74 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id,\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system\n FROM\n contract_verification_requests\n WHERE\n status = 'successful'\n ORDER BY\n id\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "source_code", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "contract_name", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "zk_compiler_version", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "compiler_version", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "optimization_used", + "type_info": "Bool" + }, + { + "ordinal": 7, + "name": "optimizer_mode", + "type_info": "Text" + }, + { + "ordinal": 8, + "name": "constructor_arguments", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "is_system", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + true, + false, + false + ] + }, + "hash": "5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c" +} diff --git a/core/lib/dal/.sqlx/query-5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83.json b/core/lib/dal/.sqlx/query-5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83.json new file mode 100644 index 000000000000..4879c6095a12 --- /dev/null +++ b/core/lib/dal/.sqlx/query-5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n predicted_commit_gas_cost = $2,\n updated_at = NOW()\n WHERE\n number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83" +} diff --git a/core/lib/dal/.sqlx/query-61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513.json b/core/lib/dal/.sqlx/query-61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513.json new file mode 100644 index 000000000000..c713af9a210d --- /dev/null +++ b/core/lib/dal/.sqlx/query-61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n prover_jobs_fri\n WHERE\n status <> 'skipped'\n AND status <> 'successful'\n AND aggregation_round = $1\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int2" + ] + }, + "nullable": [ + false + ] + }, + "hash": "61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513" +} diff --git a/core/lib/dal/.sqlx/query-61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49.json b/core/lib/dal/.sqlx/query-61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49.json new file mode 100644 index 000000000000..2c0454b0dd8b --- /dev/null +++ b/core/lib/dal/.sqlx/query-61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(priority_op_id) AS \"op_id\"\n FROM\n transactions\n WHERE\n is_priority = TRUE\n AND miniblock_number IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "op_id", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49" +} diff --git a/core/lib/dal/.sqlx/query-6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363.json b/core/lib/dal/.sqlx/query-6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363.json new file mode 100644 index 000000000000..586cace76178 --- /dev/null +++ b/core/lib/dal/.sqlx/query-6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363.json @@ -0,0 +1,88 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "contract_address", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "tx_type", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "gas_used", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "has_failed", + "type_info": "Bool" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "confirmed_eth_tx_history_id", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "predicted_gas_cost", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false, + false, + false, + true, + true, + false + ] + }, + "hash": "6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363" +} diff --git a/core/lib/dal/.sqlx/query-66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104.json b/core/lib/dal/.sqlx/query-66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104.json new file mode 100644 index 000000000000..e07fbfbd70be --- /dev/null +++ b/core/lib/dal/.sqlx/query-66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104.json @@ -0,0 +1,92 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n eth_txs (\n raw_tx,\n nonce,\n tx_type,\n contract_address,\n predicted_gas_cost,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW(), NOW())\n RETURNING\n *\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "contract_address", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "tx_type", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "gas_used", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "has_failed", + "type_info": "Bool" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "confirmed_eth_tx_history_id", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "predicted_gas_cost", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Int8", + "Text", + "Text", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + true, + false, + false, + false, + true, + true, + false + ] + }, + "hash": "66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104" +} diff --git a/core/lib/dal/.sqlx/query-68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d.json b/core/lib/dal/.sqlx/query-68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d.json new file mode 100644 index 000000000000..69b24831d73f --- /dev/null +++ b/core/lib/dal/.sqlx/query-68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n WHERE\n number = $1\n AND hash = $2\n AND merkle_root_hash = $3\n AND parent_hash = $4\n AND l2_l1_merkle_root = $5\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Bytea", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + null + ] + }, + "hash": "68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d" +} diff --git a/core/lib/dal/.sqlx/query-68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7.json b/core/lib/dal/.sqlx/query-68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7.json new file mode 100644 index 000000000000..1d5336030a44 --- /dev/null +++ b/core/lib/dal/.sqlx/query-68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7.json @@ -0,0 +1,40 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n nonce,\n signature,\n gas_limit,\n max_fee_per_gas,\n max_priority_fee_per_gas,\n gas_per_pubdata_limit,\n input,\n data,\n tx_format,\n contract_address,\n value,\n paymaster,\n paymaster_input,\n execution_info,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n FALSE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n $19,\n NOW(),\n NOW()\n )\n ON CONFLICT (initiator_address, nonce) DO\n UPDATE\n SET\n hash = $1,\n signature = $4,\n gas_limit = $5,\n max_fee_per_gas = $6,\n max_priority_fee_per_gas = $7,\n gas_per_pubdata_limit = $8,\n input = $9,\n data = $10,\n tx_format = $11,\n contract_address = $12,\n value = $13,\n paymaster = $14,\n paymaster_input = $15,\n execution_info = JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n in_mempool = FALSE,\n received_at = $19,\n created_at = NOW(),\n updated_at = NOW(),\n error = NULL\n WHERE\n transactions.is_priority = FALSE\n AND transactions.miniblock_number IS NULL\n RETURNING\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n transactions.initiator_address = $2\n AND transactions.nonce = $3\n ) IS NOT NULL AS \"is_replaced!\"\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "is_replaced!", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Int8", + "Bytea", + "Numeric", + "Numeric", + "Numeric", + "Numeric", + "Bytea", + "Jsonb", + "Int4", + "Bytea", + "Numeric", + "Bytea", + "Bytea", + "Int8", + "Int4", + "Int4", + "Timestamp" + ] + }, + "nullable": [ + null + ] + }, + "hash": "68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7" +} diff --git a/core/lib/dal/.sqlx/query-6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38.json b/core/lib/dal/.sqlx/query-6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38.json new file mode 100644 index 000000000000..08dff439a7c8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38.json @@ -0,0 +1,17 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n gpu_prover_queue_fri (\n instance_host,\n instance_port,\n instance_status,\n specialized_prover_group_id,\n zone,\n created_at,\n updated_at\n )\n VALUES\n (CAST($1::TEXT AS inet), $2, 'available', $3, $4, NOW(), NOW())\n ON CONFLICT (instance_host, instance_port, zone) DO\n UPDATE\n SET\n instance_status = 'available',\n specialized_prover_group_id = $3,\n zone = $4,\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int4", + "Int2", + "Text" + ] + }, + "nullable": [] + }, + "hash": "6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38" +} diff --git a/core/lib/dal/.sqlx/query-6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20.json b/core/lib/dal/.sqlx/query-6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20.json new file mode 100644 index 000000000000..d00622a1f5fa --- /dev/null +++ b/core/lib/dal/.sqlx/query-6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20.json @@ -0,0 +1,12 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM eth_txs\n WHERE\n id >= (\n SELECT\n MIN(id)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n )\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [] + }, + "nullable": [] + }, + "hash": "6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20" +} diff --git a/core/lib/dal/.sqlx/query-6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914.json b/core/lib/dal/.sqlx/query-6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914.json new file mode 100644 index 000000000000..c90296e322ca --- /dev/null +++ b/core/lib/dal/.sqlx/query-6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914.json @@ -0,0 +1,12 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM basic_witness_input_producer_jobs\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [] + }, + "nullable": [] + }, + "hash": "6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914" +} diff --git a/core/lib/dal/.sqlx/query-6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0.json b/core/lib/dal/.sqlx/query-6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0.json new file mode 100644 index 000000000000..0ad799dd49d8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE miniblocks\n SET\n protocol_version = $1\n WHERE\n l1_batch_number IS NULL\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [] + }, + "hash": "6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0" +} diff --git a/core/lib/dal/.sqlx/query-6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6.json b/core/lib/dal/.sqlx/query-6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6.json new file mode 100644 index 000000000000..69cb14ce7816 --- /dev/null +++ b/core/lib/dal/.sqlx/query-6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6.json @@ -0,0 +1,86 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 6, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 7, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 8, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "virtual_blocks", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + false + ] + }, + "hash": "6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6" +} diff --git a/core/lib/dal/.sqlx/query-708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d.json b/core/lib/dal/.sqlx/query-708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d.json new file mode 100644 index 000000000000..a63bd3ebeeb9 --- /dev/null +++ b/core/lib/dal/.sqlx/query-708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n hash\n FROM\n miniblocks\n WHERE\n number >= $1\n ORDER BY\n number ASC\n LIMIT\n $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d" +} diff --git a/core/lib/dal/.sqlx/query-70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94.json b/core/lib/dal/.sqlx/query-70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94.json new file mode 100644 index 000000000000..45338f8e64ca --- /dev/null +++ b/core/lib/dal/.sqlx/query-70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94.json @@ -0,0 +1,38 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n l1_batches (\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n predicted_commit_gas_cost,\n predicted_prove_gas_cost,\n predicted_execute_gas_cost,\n initial_bootloader_heap_content,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n storage_refunds,\n pubdata_input,\n predicted_circuits,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n $19,\n $20,\n $21,\n $22,\n $23,\n $24,\n $25,\n NOW(),\n NOW()\n )\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Int4", + "Int8", + "Bool", + "Bytea", + "ByteaArray", + "ByteaArray", + "Bytea", + "ByteaArray", + "Int8", + "Int8", + "Int8", + "Jsonb", + "Jsonb", + "Numeric", + "Int8", + "Int8", + "Bytea", + "Bytea", + "Int4", + "ByteaArray", + "Int8Array", + "Bytea", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94" +} diff --git a/core/lib/dal/.sqlx/query-72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42.json b/core/lib/dal/.sqlx/query-72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42.json new file mode 100644 index 000000000000..707b7ce9e75c --- /dev/null +++ b/core/lib/dal/.sqlx/query-72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42.json @@ -0,0 +1,232 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42" +} diff --git a/core/lib/dal/.sqlx/query-72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345.json b/core/lib/dal/.sqlx/query-72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345.json new file mode 100644 index 000000000000..75f288ee14f6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345.json @@ -0,0 +1,12 @@ +{ + "db_name": "PostgreSQL", + "query": "\n VACUUM storage_logs\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [] + }, + "nullable": [] + }, + "hash": "72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345" +} diff --git a/core/lib/dal/.sqlx/query-73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665.json b/core/lib/dal/.sqlx/query-73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665.json new file mode 100644 index 000000000000..aa38e1c40357 --- /dev/null +++ b/core/lib/dal/.sqlx/query-73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665.json @@ -0,0 +1,30 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $3,\n input_blob_url = $4\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + }, + "Int8", + "Time", + "Text" + ] + }, + "nullable": [] + }, + "hash": "73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665" +} diff --git a/core/lib/dal/.sqlx/query-7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914.json b/core/lib/dal/.sqlx/query-7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914.json new file mode 100644 index 000000000000..9ff3ab86250d --- /dev/null +++ b/core/lib/dal/.sqlx/query-7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM eth_txs_history\n WHERE\n id = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [] + }, + "hash": "7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914" +} diff --git a/core/lib/dal/.sqlx/query-759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0.json b/core/lib/dal/.sqlx/query-759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0.json new file mode 100644 index 000000000000..d488293cf81f --- /dev/null +++ b/core/lib/dal/.sqlx/query-759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(operation_number) AS \"max?\"\n FROM\n storage_logs\n WHERE\n miniblock_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "max?", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + null + ] + }, + "hash": "759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0" +} diff --git a/core/lib/dal/.sqlx/query-75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3.json b/core/lib/dal/.sqlx/query-75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3.json new file mode 100644 index 000000000000..13f45b32225a --- /dev/null +++ b/core/lib/dal/.sqlx/query-75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3.json @@ -0,0 +1,256 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n eth_commit_tx_id IS NOT NULL\n AND eth_prove_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3" +} diff --git a/core/lib/dal/.sqlx/query-75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4.json b/core/lib/dal/.sqlx/query-75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4.json new file mode 100644 index 000000000000..71edd403d0b6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4.json @@ -0,0 +1,90 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $3\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number <= $1\n AND status = 'queued'\n AND protocol_version = ANY ($2)\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n witness_inputs_fri.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "merkle_tree_paths_blob_url", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "error", + "type_info": "Text" + }, + { + "ordinal": 5, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 6, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "processing_started_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "time_taken", + "type_info": "Time" + }, + { + "ordinal": 9, + "name": "is_blob_cleaned", + "type_info": "Bool" + }, + { + "ordinal": 10, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "picked_by", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + true, + false, + false, + true, + false, + false, + true, + true, + true, + true, + true + ] + }, + "hash": "75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4" +} diff --git a/core/lib/dal/.sqlx/query-75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60.json b/core/lib/dal/.sqlx/query-75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60.json new file mode 100644 index 000000000000..ff743f1028c2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n state AS \"state!\"\n FROM\n consensus_replica_state\n WHERE\n fake_key\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "state!", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60" +} diff --git a/core/lib/dal/.sqlx/query-76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b.json b/core/lib/dal/.sqlx/query-76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b.json new file mode 100644 index 000000000000..e5b4f3476c9f --- /dev/null +++ b/core/lib/dal/.sqlx/query-76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n call_traces (tx_hash, call_trace)\n SELECT\n u.tx_hash,\n u.call_trace\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (tx_hash, call_trace)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b" +} diff --git a/core/lib/dal/.sqlx/query-77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796.json b/core/lib/dal/.sqlx/query-77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796.json new file mode 100644 index 000000000000..acff9eeebeeb --- /dev/null +++ b/core/lib/dal/.sqlx/query-77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n hash = $1\n WHERE\n number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796" +} diff --git a/core/lib/dal/.sqlx/query-77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c.json b/core/lib/dal/.sqlx/query-77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c.json new file mode 100644 index 000000000000..30149eb79c9b --- /dev/null +++ b/core/lib/dal/.sqlx/query-77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n consensus_replica_state (fake_key, state)\n VALUES\n (TRUE, $1)\n ON CONFLICT (fake_key) DO\n UPDATE\n SET\n state = excluded.state\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Jsonb" + ] + }, + "nullable": [] + }, + "hash": "77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c" +} diff --git a/core/lib/dal/.sqlx/query-78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909.json b/core/lib/dal/.sqlx/query-78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909.json new file mode 100644 index 000000000000..f746bd5703be --- /dev/null +++ b/core/lib/dal/.sqlx/query-78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n call_traces\n WHERE\n tx_hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "call_trace", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909" +} diff --git a/core/lib/dal/.sqlx/query-7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776.json b/core/lib/dal/.sqlx/query-7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776.json new file mode 100644 index 000000000000..73a8c33695b1 --- /dev/null +++ b/core/lib/dal/.sqlx/query-7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776.json @@ -0,0 +1,18 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n aggregations_url = $1,\n number_of_dependent_jobs = $5,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n AND circuit_id = $3\n AND depth = $4\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8", + "Int2", + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776" +} diff --git a/core/lib/dal/.sqlx/query-7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6.json b/core/lib/dal/.sqlx/query-7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6.json new file mode 100644 index 000000000000..da78974f61a2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6.json @@ -0,0 +1,107 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n leaf_aggregation_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n leaf_aggregation_witness_jobs_fri.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "closed_form_inputs_blob_url", + "type_info": "Text" + }, + { + "ordinal": 4, + "name": "attempts", + "type_info": "Int2" + }, + { + "ordinal": 5, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "error", + "type_info": "Text" + }, + { + "ordinal": 7, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 9, + "name": "processing_started_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "time_taken", + "type_info": "Time" + }, + { + "ordinal": 11, + "name": "is_blob_cleaned", + "type_info": "Bool" + }, + { + "ordinal": 12, + "name": "number_of_basic_circuits", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "picked_by", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + true, + false, + false, + true, + false, + false, + true, + true, + true, + true, + true, + true + ] + }, + "hash": "7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6" +} diff --git a/core/lib/dal/.sqlx/query-7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283.json b/core/lib/dal/.sqlx/query-7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283.json new file mode 100644 index 000000000000..76a34db9699f --- /dev/null +++ b/core/lib/dal/.sqlx/query-7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n gas_limit,\n max_fee_per_gas,\n gas_per_pubdata_limit,\n data,\n upgrade_id,\n contract_address,\n l1_block_number,\n value,\n paymaster,\n paymaster_input,\n tx_format,\n l1_tx_mint,\n l1_tx_refund_recipient,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n TRUE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n NOW(),\n NOW()\n )\n ON CONFLICT (hash) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Numeric", + "Numeric", + "Numeric", + "Jsonb", + "Int4", + "Bytea", + "Int4", + "Numeric", + "Bytea", + "Bytea", + "Int4", + "Numeric", + "Bytea", + "Timestamp" + ] + }, + "nullable": [] + }, + "hash": "7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283" +} diff --git a/core/lib/dal/.sqlx/query-806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9.json b/core/lib/dal/.sqlx/query-806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9.json new file mode 100644 index 000000000000..c8e8a7aa6033 --- /dev/null +++ b/core/lib/dal/.sqlx/query-806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n node_aggregation_witness_jobs_fri\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9" +} diff --git a/core/lib/dal/.sqlx/query-8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe.json b/core/lib/dal/.sqlx/query-8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe.json new file mode 100644 index 000000000000..fac64c1ea3f9 --- /dev/null +++ b/core/lib/dal/.sqlx/query-8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe.json @@ -0,0 +1,32 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id, depth) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth\n FROM\n prover_jobs_fri\n JOIN node_aggregation_witness_jobs_fri nawj ON prover_jobs_fri.l1_batch_number = nawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = nawj.circuit_id\n AND prover_jobs_fri.depth = nawj.depth\n WHERE\n nawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 2\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth,\n nawj.number_of_dependent_jobs\n HAVING\n COUNT(*) = nawj.number_of_dependent_jobs\n )\n RETURNING\n l1_batch_number,\n circuit_id,\n depth;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 2, + "name": "depth", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe" +} diff --git a/core/lib/dal/.sqlx/query-81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9.json b/core/lib/dal/.sqlx/query-81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9.json new file mode 100644 index 000000000000..b8d80a904e48 --- /dev/null +++ b/core/lib/dal/.sqlx/query-81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9.json @@ -0,0 +1,18 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n storage (hashed_key, address, key, value, tx_hash, created_at, updated_at)\n SELECT\n u.hashed_key,\n u.address,\n u.key,\n u.value,\n u.tx_hash,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::bytea[], $3::bytea[], $4::bytea[], $5::bytea[]) AS u (hashed_key, address, key, value, tx_hash)\n ON CONFLICT (hashed_key) DO\n UPDATE\n SET\n tx_hash = excluded.tx_hash,\n value = excluded.value,\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "ByteaArray", + "ByteaArray", + "ByteaArray", + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9" +} diff --git a/core/lib/dal/.sqlx/query-83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8.json b/core/lib/dal/.sqlx/query-83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8.json new file mode 100644 index 000000000000..8d9458dce0a4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n base_fee_per_gas\n FROM\n miniblocks\n WHERE\n number <= $1\n ORDER BY\n number DESC\n LIMIT\n $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "base_fee_per_gas", + "type_info": "Numeric" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8" +} diff --git a/core/lib/dal/.sqlx/query-84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12.json b/core/lib/dal/.sqlx/query-84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12.json new file mode 100644 index 000000000000..a0a3cb3d63b2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12.json @@ -0,0 +1,256 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n system_logs,\n compressed_state_diffs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 36, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true + ] + }, + "hash": "84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12" +} diff --git a/core/lib/dal/.sqlx/query-852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8.json b/core/lib/dal/.sqlx/query-852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8.json new file mode 100644 index 000000000000..6e582aac6536 --- /dev/null +++ b/core/lib/dal/.sqlx/query-852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8.json @@ -0,0 +1,235 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n in_mempool = TRUE\n FROM\n (\n SELECT\n hash\n FROM\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n miniblock_number IS NULL\n AND in_mempool = FALSE\n AND error IS NULL\n AND (\n is_priority = TRUE\n OR (\n max_fee_per_gas >= $2\n AND gas_per_pubdata_limit >= $3\n )\n )\n AND tx_format != $4\n ORDER BY\n is_priority DESC,\n priority_op_id,\n received_at\n LIMIT\n $1\n ) AS subquery1\n ORDER BY\n hash\n ) AS subquery2\n WHERE\n transactions.hash = subquery2.hash\n RETURNING\n transactions.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8", + "Numeric", + "Numeric", + "Int4" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8" +} diff --git a/core/lib/dal/.sqlx/query-8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497.json b/core/lib/dal/.sqlx/query-8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497.json new file mode 100644 index 000000000000..f7906122f109 --- /dev/null +++ b/core/lib/dal/.sqlx/query-8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n tx_hash\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n AND confirmed_at IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "tx_hash", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false + ] + }, + "hash": "8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497" +} diff --git a/core/lib/dal/.sqlx/query-877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b.json b/core/lib/dal/.sqlx/query-877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b.json new file mode 100644 index 000000000000..3052b3a04d1a --- /dev/null +++ b/core/lib/dal/.sqlx/query-877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_execute_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b" +} diff --git a/core/lib/dal/.sqlx/query-878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4.json b/core/lib/dal/.sqlx/query-878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4.json new file mode 100644 index 000000000000..9dde4d74ed11 --- /dev/null +++ b/core/lib/dal/.sqlx/query-878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l2_address\n FROM\n tokens\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l2_address", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4" +} diff --git a/core/lib/dal/.sqlx/query-88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60.json b/core/lib/dal/.sqlx/query-88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60.json new file mode 100644 index 000000000000..16fbffd5b667 --- /dev/null +++ b/core/lib/dal/.sqlx/query-88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp,\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "virtual_blocks", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60" +} diff --git a/core/lib/dal/.sqlx/query-8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69.json b/core/lib/dal/.sqlx/query-8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69.json new file mode 100644 index 000000000000..3d47a756f3e6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode_hash,\n bytecode\n FROM\n factory_deps\n INNER JOIN miniblocks ON miniblocks.number = factory_deps.miniblock_number\n WHERE\n miniblocks.l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "bytecode", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69" +} diff --git a/core/lib/dal/.sqlx/query-894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2.json b/core/lib/dal/.sqlx/query-894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2.json new file mode 100644 index 000000000000..06d3461c3fa3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n protocol_version\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "protocol_version", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2" +} diff --git a/core/lib/dal/.sqlx/query-8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db.json b/core/lib/dal/.sqlx/query-8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db.json new file mode 100644 index 000000000000..ea6562d1a67f --- /dev/null +++ b/core/lib/dal/.sqlx/query-8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_block_number\n FROM\n transactions\n WHERE\n priority_op_id IS NOT NULL\n ORDER BY\n priority_op_id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_block_number", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + true + ] + }, + "hash": "8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db" +} diff --git a/core/lib/dal/.sqlx/query-8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895.json b/core/lib/dal/.sqlx/query-8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895.json new file mode 100644 index 000000000000..de369bccec54 --- /dev/null +++ b/core/lib/dal/.sqlx/query-8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895.json @@ -0,0 +1,36 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n storage_logs.hashed_key,\n storage_logs.value,\n initial_writes.index\n FROM\n storage_logs\n INNER JOIN initial_writes ON storage_logs.hashed_key = initial_writes.hashed_key\n WHERE\n storage_logs.miniblock_number = $1\n AND storage_logs.hashed_key >= $2::bytea\n AND storage_logs.hashed_key <= $3::bytea\n ORDER BY\n storage_logs.hashed_key\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "value", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895" +} diff --git a/core/lib/dal/.sqlx/query-8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f.json b/core/lib/dal/.sqlx/query-8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f.json new file mode 100644 index 000000000000..cf7822e8ec84 --- /dev/null +++ b/core/lib/dal/.sqlx/query-8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n protocol_version\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "protocol_version", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f" +} diff --git a/core/lib/dal/.sqlx/query-90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77.json b/core/lib/dal/.sqlx/query-90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77.json new file mode 100644 index 000000000000..dfd7cd9c5557 --- /dev/null +++ b/core/lib/dal/.sqlx/query-90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n nonce\n FROM\n eth_txs\n ORDER BY\n id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "nonce", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77" +} diff --git a/core/lib/dal/.sqlx/query-9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5.json b/core/lib/dal/.sqlx/query-9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5.json new file mode 100644 index 000000000000..92e74026bf57 --- /dev/null +++ b/core/lib/dal/.sqlx/query-9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n value\n FROM\n storage\n WHERE\n hashed_key = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "value", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5" +} diff --git a/core/lib/dal/.sqlx/query-95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300.json b/core/lib/dal/.sqlx/query-95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300.json new file mode 100644 index 000000000000..3c822fe50d17 --- /dev/null +++ b/core/lib/dal/.sqlx/query-95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE eth_txs\n SET\n gas_used = $1,\n confirmed_eth_tx_history_id = $2\n WHERE\n id = $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300" +} diff --git a/core/lib/dal/.sqlx/query-966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f.json b/core/lib/dal/.sqlx/query-966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f.json new file mode 100644 index 000000000000..bf4eb3f9462d --- /dev/null +++ b/core/lib/dal/.sqlx/query-966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f.json @@ -0,0 +1,40 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "recursion_scheduler_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "recursion_node_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "recursion_leaf_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "recursion_circuits_set_vks_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false + ] + }, + "hash": "966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f" +} diff --git a/core/lib/dal/.sqlx/query-9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47.json b/core/lib/dal/.sqlx/query-9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47.json new file mode 100644 index 000000000000..c05539164cee --- /dev/null +++ b/core/lib/dal/.sqlx/query-9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n factory_deps.bytecode,\n transactions.data AS \"data?\",\n transactions.contract_address AS \"contract_address?\"\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n ORDER BY\n miniblock_number DESC,\n operation_number DESC\n LIMIT\n 1\n ) storage_logs\n JOIN factory_deps ON factory_deps.bytecode_hash = storage_logs.value\n LEFT JOIN transactions ON transactions.hash = storage_logs.tx_hash\n WHERE\n storage_logs.value != $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "data?", + "type_info": "Jsonb" + }, + { + "ordinal": 2, + "name": "contract_address?", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false, + false, + true + ] + }, + "hash": "9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47" +} diff --git a/core/lib/dal/.sqlx/query-995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068.json b/core/lib/dal/.sqlx/query-995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068.json new file mode 100644 index 000000000000..49e0a4a8f07d --- /dev/null +++ b/core/lib/dal/.sqlx/query-995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068.json @@ -0,0 +1,32 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n ORDER BY\n l1_batch_number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "factory_deps_filepath", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "storage_logs_filepaths", + "type_info": "TextArray" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068" +} diff --git a/core/lib/dal/.sqlx/query-99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6.json b/core/lib/dal/.sqlx/query-99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6.json new file mode 100644 index 000000000000..2aa6a538125b --- /dev/null +++ b/core/lib/dal/.sqlx/query-99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6.json @@ -0,0 +1,82 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n protocol_versions\n WHERE\n id < $1\n ORDER BY\n id DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "recursion_scheduler_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "recursion_node_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "recursion_leaf_level_vk_hash", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "recursion_circuits_set_vks_hash", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "default_account_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "verifier_address", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "upgrade_tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "created_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + false + ] + }, + "hash": "99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6" +} diff --git a/core/lib/dal/.sqlx/query-99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005.json b/core/lib/dal/.sqlx/query-99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005.json new file mode 100644 index 000000000000..ab00c7b26ce3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hashed_key\n FROM\n initial_writes\n WHERE\n hashed_key = ANY ($1)\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [ + false + ] + }, + "hash": "99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005" +} diff --git a/core/lib/dal/.sqlx/query-99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc.json b/core/lib/dal/.sqlx/query-99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc.json new file mode 100644 index 000000000000..b8c14c534625 --- /dev/null +++ b/core/lib/dal/.sqlx/query-99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc.json @@ -0,0 +1,65 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n basic_witness_input_producer_jobs\n WHERE\n status = $2\n OR (\n status = $1\n AND processing_started_at < NOW() - $4::INTERVAL\n )\n OR (\n status = $3\n AND attempts < $5\n )\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n basic_witness_input_producer_jobs.l1_batch_number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + }, + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + }, + { + "Custom": { + "name": "basic_witness_input_producer_job_status", + "kind": { + "Enum": [ + "Queued", + "ManuallySkipped", + "InProgress", + "Successful", + "Failed" + ] + } + } + }, + "Interval", + "Int2" + ] + }, + "nullable": [ + false + ] + }, + "hash": "99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc" +} diff --git a/core/lib/dal/.sqlx/query-9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19.json b/core/lib/dal/.sqlx/query-9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19.json new file mode 100644 index 000000000000..a890a6ca07e0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n witness_inputs_fri (\n l1_batch_number,\n merkle_tree_paths_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19" +} diff --git a/core/lib/dal/.sqlx/query-9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86.json b/core/lib/dal/.sqlx/query-9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86.json new file mode 100644 index 000000000000..f9a53d707632 --- /dev/null +++ b/core/lib/dal/.sqlx/query-9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86" +} diff --git a/core/lib/dal/.sqlx/query-9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40.json b/core/lib/dal/.sqlx/query-9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40.json new file mode 100644 index 000000000000..c4beef961733 --- /dev/null +++ b/core/lib/dal/.sqlx/query-9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE contract_verification_requests\n SET\n status = 'successful',\n updated_at = NOW()\n WHERE\n id = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40" +} diff --git a/core/lib/dal/.sqlx/query-9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0.json b/core/lib/dal/.sqlx/query-9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0.json new file mode 100644 index 000000000000..674377635ced --- /dev/null +++ b/core/lib/dal/.sqlx/query-9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE snapshots\n SET\n storage_logs_filepaths[$2] = $3,\n updated_at = NOW()\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int4", + "Text" + ] + }, + "nullable": [] + }, + "hash": "9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0" +} diff --git a/core/lib/dal/.sqlx/query-9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03.json b/core/lib/dal/.sqlx/query-9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03.json new file mode 100644 index 000000000000..fd770071cf86 --- /dev/null +++ b/core/lib/dal/.sqlx/query-9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03" +} diff --git a/core/lib/dal/.sqlx/query-a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3.json b/core/lib/dal/.sqlx/query-a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3.json new file mode 100644 index 000000000000..7dc19564f7fa --- /dev/null +++ b/core/lib/dal/.sqlx/query-a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3" +} diff --git a/core/lib/dal/.sqlx/query-a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea.json b/core/lib/dal/.sqlx/query-a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea.json new file mode 100644 index 000000000000..fc36e47b54c8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n initial_bootloader_heap_content\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "initial_bootloader_heap_content", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea" +} diff --git a/core/lib/dal/.sqlx/query-a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601.json b/core/lib/dal/.sqlx/query-a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601.json new file mode 100644 index 000000000000..104a7fb25560 --- /dev/null +++ b/core/lib/dal/.sqlx/query-a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_dependency_tracker_fri\n SET\n status = 'queued'\n WHERE\n l1_batch_number = ANY ($1)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8Array" + ] + }, + "nullable": [] + }, + "hash": "a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601" +} diff --git a/core/lib/dal/.sqlx/query-a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a.json b/core/lib/dal/.sqlx/query-a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a.json new file mode 100644 index 000000000000..49e547e5564c --- /dev/null +++ b/core/lib/dal/.sqlx/query-a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(INDEX) AS \"max?\"\n FROM\n initial_writes\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "max?", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a" +} diff --git a/core/lib/dal/.sqlx/query-a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c.json b/core/lib/dal/.sqlx/query-a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c.json new file mode 100644 index 000000000000..f19add71350a --- /dev/null +++ b/core/lib/dal/.sqlx/query-a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hash\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c" +} diff --git a/core/lib/dal/.sqlx/query-a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4.json b/core/lib/dal/.sqlx/query-a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4.json new file mode 100644 index 000000000000..39b0c391ef59 --- /dev/null +++ b/core/lib/dal/.sqlx/query-a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n initial_writes (hashed_key, INDEX, l1_batch_number, created_at, updated_at)\n SELECT\n u.hashed_key,\n u.index,\n $3,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::BIGINT[]) AS u (hashed_key, INDEX)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray", + "Int8Array", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4" +} diff --git a/core/lib/dal/.sqlx/query-a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583.json b/core/lib/dal/.sqlx/query-a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583.json new file mode 100644 index 000000000000..c4f1f4bbcd0d --- /dev/null +++ b/core/lib/dal/.sqlx/query-a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_generation_details\n SET\n status = 'generated',\n proof_blob_url = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583" +} diff --git a/core/lib/dal/.sqlx/query-a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733.json b/core/lib/dal/.sqlx/query-a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733.json new file mode 100644 index 000000000000..0dab103fa24f --- /dev/null +++ b/core/lib/dal/.sqlx/query-a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n hash = $1,\n merkle_root_hash = $2,\n commitment = $3,\n default_aa_code_hash = $4,\n compressed_repeated_writes = $5,\n compressed_initial_writes = $6,\n l2_l1_compressed_messages = $7,\n l2_l1_merkle_root = $8,\n zkporter_is_available = $9,\n bootloader_code_hash = $10,\n rollup_last_leaf_index = $11,\n aux_data_hash = $12,\n pass_through_data_hash = $13,\n meta_parameters_hash = $14,\n compressed_state_diffs = $15,\n updated_at = NOW()\n WHERE\n number = $16\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bool", + "Bytea", + "Int8", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733" +} diff --git a/core/lib/dal/.sqlx/query-a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604.json b/core/lib/dal/.sqlx/query-a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604.json new file mode 100644 index 000000000000..3275df2a3d58 --- /dev/null +++ b/core/lib/dal/.sqlx/query-a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n l1_batch_number = $1\n AND status = 'successful'\n AND aggregation_round = $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int2" + ] + }, + "nullable": [ + false + ] + }, + "hash": "a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604" +} diff --git a/core/lib/dal/.sqlx/query-a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e.json b/core/lib/dal/.sqlx/query-a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e.json new file mode 100644 index 000000000000..7c757648b383 --- /dev/null +++ b/core/lib/dal/.sqlx/query-a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n upgrade_tx_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "upgrade_tx_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + true + ] + }, + "hash": "a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e" +} diff --git a/core/lib/dal/.sqlx/query-aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320.json b/core/lib/dal/.sqlx/query-aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320.json new file mode 100644 index 000000000000..27d482317286 --- /dev/null +++ b/core/lib/dal/.sqlx/query-aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320.json @@ -0,0 +1,32 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id, depth) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth\n FROM\n prover_jobs_fri\n JOIN node_aggregation_witness_jobs_fri nawj ON prover_jobs_fri.l1_batch_number = nawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = nawj.circuit_id\n AND prover_jobs_fri.depth = nawj.depth\n WHERE\n nawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 1\n AND prover_jobs_fri.depth = 0\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth,\n nawj.number_of_dependent_jobs\n HAVING\n COUNT(*) = nawj.number_of_dependent_jobs\n )\n RETURNING\n l1_batch_number,\n circuit_id,\n depth;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 2, + "name": "depth", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320" +} diff --git a/core/lib/dal/.sqlx/query-aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45.json b/core/lib/dal/.sqlx/query-aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45.json new file mode 100644 index 000000000000..614b853c6250 --- /dev/null +++ b/core/lib/dal/.sqlx/query-aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45.json @@ -0,0 +1,38 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\",\n circuit_id AS \"circuit_id!\",\n aggregation_round AS \"aggregation_round!\",\n status AS \"status!\"\n FROM\n prover_jobs_fri\n WHERE\n status <> 'skipped'\n AND status <> 'successful'\n GROUP BY\n circuit_id,\n aggregation_round,\n status\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "circuit_id!", + "type_info": "Int2" + }, + { + "ordinal": 2, + "name": "aggregation_round!", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "status!", + "type_info": "Text" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null, + false, + false, + false + ] + }, + "hash": "aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45" +} diff --git a/core/lib/dal/.sqlx/query-ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc.json b/core/lib/dal/.sqlx/query-ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc.json new file mode 100644 index 000000000000..2dad4563cc70 --- /dev/null +++ b/core/lib/dal/.sqlx/query-ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n id\n FROM\n protocol_versions\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc" +} diff --git a/core/lib/dal/.sqlx/query-ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820.json b/core/lib/dal/.sqlx/query-ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820.json new file mode 100644 index 000000000000..f96c6945623e --- /dev/null +++ b/core/lib/dal/.sqlx/query-ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n (\n SELECT\n l1_batch_number\n FROM\n miniblocks\n WHERE\n number = $1\n ) AS \"block_batch?\",\n (\n SELECT\n MAX(number) + 1\n FROM\n l1_batches\n ) AS \"max_batch?\"\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "block_batch?", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "max_batch?", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + null, + null + ] + }, + "hash": "ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820" +} diff --git a/core/lib/dal/.sqlx/query-ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2.json b/core/lib/dal/.sqlx/query-ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2.json new file mode 100644 index 000000000000..04fde45469fa --- /dev/null +++ b/core/lib/dal/.sqlx/query-ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE eth_txs_history\n SET\n updated_at = NOW(),\n confirmed_at = NOW()\n WHERE\n tx_hash = $1\n RETURNING\n id,\n eth_tx_id\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "eth_tx_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Text" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2" +} diff --git a/core/lib/dal/.sqlx/query-aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49.json b/core/lib/dal/.sqlx/query-aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49.json new file mode 100644 index 000000000000..b411d3ce8309 --- /dev/null +++ b/core/lib/dal/.sqlx/query-aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE eth_txs\n SET\n has_failed = TRUE\n WHERE\n id = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [] + }, + "hash": "aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49" +} diff --git a/core/lib/dal/.sqlx/query-aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781.json b/core/lib/dal/.sqlx/query-aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781.json new file mode 100644 index 000000000000..c82bed1169ca --- /dev/null +++ b/core/lib/dal/.sqlx/query-aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n miniblocks\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781" +} diff --git a/core/lib/dal/.sqlx/query-af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284.json b/core/lib/dal/.sqlx/query-af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284.json new file mode 100644 index 000000000000..6674fab59eab --- /dev/null +++ b/core/lib/dal/.sqlx/query-af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'sent_to_server',\n updated_at = NOW()\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284" +} diff --git a/core/lib/dal/.sqlx/query-afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4.json b/core/lib/dal/.sqlx/query-afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4.json new file mode 100644 index 000000000000..ede2995ff558 --- /dev/null +++ b/core/lib/dal/.sqlx/query-afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4" +} diff --git a/core/lib/dal/.sqlx/query-b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac.json b/core/lib/dal/.sqlx/query-b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac.json new file mode 100644 index 000000000000..82209e00b65a --- /dev/null +++ b/core/lib/dal/.sqlx/query-b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n id,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac" +} diff --git a/core/lib/dal/.sqlx/query-b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125.json b/core/lib/dal/.sqlx/query-b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125.json new file mode 100644 index 000000000000..fd8600d59aaf --- /dev/null +++ b/core/lib/dal/.sqlx/query-b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125.json @@ -0,0 +1,18 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n leaf_aggregation_witness_jobs_fri (\n l1_batch_number,\n circuit_id,\n closed_form_inputs_blob_url,\n number_of_basic_circuits,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number, circuit_id) DO\n UPDATE\n SET\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int2", + "Text", + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125" +} diff --git a/core/lib/dal/.sqlx/query-b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5.json b/core/lib/dal/.sqlx/query-b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5.json new file mode 100644 index 000000000000..bdd22927d386 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n scheduler_witness_jobs_fri\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5" +} diff --git a/core/lib/dal/.sqlx/query-b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2.json b/core/lib/dal/.sqlx/query-b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2.json new file mode 100644 index 000000000000..1ece82073712 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n value\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n AND storage_logs.miniblock_number <= $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "value", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2" +} diff --git a/core/lib/dal/.sqlx/query-b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6.json b/core/lib/dal/.sqlx/query-b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6.json new file mode 100644 index 000000000000..724c01ea6c53 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n proof_compression_jobs_fri\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6" +} diff --git a/core/lib/dal/.sqlx/query-b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040.json b/core/lib/dal/.sqlx/query-b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040.json new file mode 100644 index 000000000000..0ca284a3f57f --- /dev/null +++ b/core/lib/dal/.sqlx/query-b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040" +} diff --git a/core/lib/dal/.sqlx/query-b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3.json b/core/lib/dal/.sqlx/query-b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3.json new file mode 100644 index 000000000000..fa6f91edfb32 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3.json @@ -0,0 +1,83 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n scheduler_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n scheduler_witness_jobs_fri.*\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "scheduler_partial_input_blob_url", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "processing_started_at", + "type_info": "Timestamp" + }, + { + "ordinal": 4, + "name": "time_taken", + "type_info": "Time" + }, + { + "ordinal": 5, + "name": "error", + "type_info": "Text" + }, + { + "ordinal": 6, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 7, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "attempts", + "type_info": "Int2" + }, + { + "ordinal": 9, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "picked_by", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + true, + true, + true, + false, + false, + false, + true, + true + ] + }, + "hash": "b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3" +} diff --git a/core/lib/dal/.sqlx/query-b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9.json b/core/lib/dal/.sqlx/query-b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9.json new file mode 100644 index 000000000000..e87b9a2cddd6 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM factory_deps\n WHERE\n miniblock_number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9" +} diff --git a/core/lib/dal/.sqlx/query-b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4.json b/core/lib/dal/.sqlx/query-b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4.json new file mode 100644 index 000000000000..14b4115b30e8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4.json @@ -0,0 +1,17 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = $1,\n updated_at = NOW()\n WHERE\n instance_host = $2::TEXT::inet\n AND instance_port = $3\n AND zone = $4\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Text", + "Int4", + "Text" + ] + }, + "nullable": [] + }, + "hash": "b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4" +} diff --git a/core/lib/dal/.sqlx/query-b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671.json b/core/lib/dal/.sqlx/query-b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671.json new file mode 100644 index 000000000000..59a4d95f1f22 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n hash,\n number,\n timestamp\n FROM\n miniblocks\n WHERE\n number > $1\n ORDER BY\n number ASC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "timestamp", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671" +} diff --git a/core/lib/dal/.sqlx/query-b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2.json b/core/lib/dal/.sqlx/query-b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2.json new file mode 100644 index 000000000000..804f6c5ac332 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE eth_txs\n SET\n confirmed_eth_tx_history_id = $1\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2" +} diff --git a/core/lib/dal/.sqlx/query-b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361.json b/core/lib/dal/.sqlx/query-b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361.json new file mode 100644 index 000000000000..b8ba0465614a --- /dev/null +++ b/core/lib/dal/.sqlx/query-b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n miniblock_number = $1,\n index_in_block = data_table.index_in_block,\n error = NULLIF(data_table.error, ''),\n in_mempool = FALSE,\n execution_info = execution_info || data_table.new_execution_info,\n refunded_gas = data_table.refunded_gas,\n effective_gas_price = data_table.effective_gas_price,\n updated_at = NOW()\n FROM\n (\n SELECT\n UNNEST($2::bytea[]) AS hash,\n UNNEST($3::INTEGER[]) AS index_in_block,\n UNNEST($4::VARCHAR[]) AS error,\n UNNEST($5::jsonb[]) AS new_execution_info,\n UNNEST($6::BIGINT[]) AS refunded_gas,\n UNNEST($7::NUMERIC[]) AS effective_gas_price\n ) AS data_table\n WHERE\n transactions.hash = data_table.hash\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "ByteaArray", + "Int4Array", + "VarcharArray", + "JsonbArray", + "Int8Array", + "NumericArray" + ] + }, + "nullable": [] + }, + "hash": "b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361" +} diff --git a/core/lib/dal/.sqlx/query-b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329.json b/core/lib/dal/.sqlx/query-b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329.json new file mode 100644 index 000000000000..004d970d81ef --- /dev/null +++ b/core/lib/dal/.sqlx/query-b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n commitments (l1_batch_number, events_queue_commitment, bootloader_initial_content_commitment)\n VALUES\n ($1, $2, $3)\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Bytea" + ] + }, + "nullable": [] + }, + "hash": "b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329" +} diff --git a/core/lib/dal/.sqlx/query-b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488.json b/core/lib/dal/.sqlx/query-b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488.json new file mode 100644 index 000000000000..91d7b677959a --- /dev/null +++ b/core/lib/dal/.sqlx/query-b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n VERSION\n FROM\n compiler_versions\n WHERE\n compiler = $1\n ORDER BY\n VERSION\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "version", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Text" + ] + }, + "nullable": [ + false + ] + }, + "hash": "b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488" +} diff --git a/core/lib/dal/.sqlx/query-b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276.json b/core/lib/dal/.sqlx/query-b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276.json new file mode 100644 index 000000000000..cb8de87ca641 --- /dev/null +++ b/core/lib/dal/.sqlx/query-b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n l1_batch_number = $3,\n l1_batch_tx_index = data_table.l1_batch_tx_index,\n updated_at = NOW()\n FROM\n (\n SELECT\n UNNEST($1::INT[]) AS l1_batch_tx_index,\n UNNEST($2::bytea[]) AS hash\n ) AS data_table\n WHERE\n transactions.hash = data_table.hash\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4Array", + "ByteaArray", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276" +} diff --git a/core/lib/dal/.sqlx/query-bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98.json b/core/lib/dal/.sqlx/query-bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98.json new file mode 100644 index 000000000000..ddc5d583900a --- /dev/null +++ b/core/lib/dal/.sqlx/query-bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98.json @@ -0,0 +1,136 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n pubdata_input\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 7, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 10, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 11, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 12, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 15, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 16, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 17, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 18, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + false, + true + ] + }, + "hash": "bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98" +} diff --git a/core/lib/dal/.sqlx/query-bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd.json b/core/lib/dal/.sqlx/query-bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd.json new file mode 100644 index 000000000000..7f1fc9b176c5 --- /dev/null +++ b/core/lib/dal/.sqlx/query-bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS commit_tx ON (l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id)\n WHERE\n commit_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd" +} diff --git a/core/lib/dal/.sqlx/query-bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7.json b/core/lib/dal/.sqlx/query-bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7.json new file mode 100644 index 000000000000..e2386003538f --- /dev/null +++ b/core/lib/dal/.sqlx/query-bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7.json @@ -0,0 +1,54 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n storage_logs.key AS \"key!\",\n storage_logs.value AS \"value!\",\n storage_logs.address AS \"address!\",\n storage_logs.miniblock_number AS \"miniblock_number!\",\n initial_writes.l1_batch_number AS \"l1_batch_number!\",\n initial_writes.index\n FROM\n (\n SELECT\n hashed_key,\n MAX(ARRAY[miniblock_number, operation_number]::INT[]) AS op\n FROM\n storage_logs\n WHERE\n miniblock_number <= $1\n AND hashed_key >= $2\n AND hashed_key < $3\n GROUP BY\n hashed_key\n ORDER BY\n hashed_key\n ) AS keys\n INNER JOIN storage_logs ON keys.hashed_key = storage_logs.hashed_key\n AND storage_logs.miniblock_number = keys.op[1]\n AND storage_logs.operation_number = keys.op[2]\n INNER JOIN initial_writes ON keys.hashed_key = initial_writes.hashed_key;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "key!", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "value!", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "address!", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "miniblock_number!", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "l1_batch_number!", + "type_info": "Int8" + }, + { + "ordinal": 5, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Bytea" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false + ] + }, + "hash": "bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7" +} diff --git a/core/lib/dal/.sqlx/query-be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8.json b/core/lib/dal/.sqlx/query-be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8.json new file mode 100644 index 000000000000..695be9f2b8c4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8.json @@ -0,0 +1,94 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n miniblock_number,\n log_index_in_miniblock,\n log_index_in_tx,\n tx_hash,\n NULL::bytea AS \"block_hash\",\n NULL::BIGINT AS \"l1_batch_number?\",\n shard_id,\n is_service,\n tx_index_in_miniblock,\n tx_index_in_l1_batch,\n sender,\n key,\n value\n FROM\n l2_to_l1_logs\n WHERE\n tx_hash = $1\n ORDER BY\n log_index_in_tx ASC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "log_index_in_miniblock", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "log_index_in_tx", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "block_hash", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "l1_batch_number?", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "shard_id", + "type_info": "Int4" + }, + { + "ordinal": 7, + "name": "is_service", + "type_info": "Bool" + }, + { + "ordinal": 8, + "name": "tx_index_in_miniblock", + "type_info": "Int4" + }, + { + "ordinal": 9, + "name": "tx_index_in_l1_batch", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "sender", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "key", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "value", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false, + false, + false, + false, + null, + null, + false, + false, + false, + false, + false, + false, + false + ] + }, + "hash": "be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8" +} diff --git a/core/lib/dal/.sqlx/query-bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660.json b/core/lib/dal/.sqlx/query-bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660.json new file mode 100644 index 000000000000..550cb5ec7438 --- /dev/null +++ b/core/lib/dal/.sqlx/query-bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'in_gpu_proof'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660" +} diff --git a/core/lib/dal/.sqlx/query-bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9.json b/core/lib/dal/.sqlx/query-bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9.json new file mode 100644 index 000000000000..8079d52a7036 --- /dev/null +++ b/core/lib/dal/.sqlx/query-bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT DISTINCT\n hashed_key\n FROM\n storage_logs\n WHERE\n miniblock_number BETWEEN $1 AND $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9" +} diff --git a/core/lib/dal/.sqlx/query-c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2.json b/core/lib/dal/.sqlx/query-c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2.json new file mode 100644 index 000000000000..8161e8c1fc8c --- /dev/null +++ b/core/lib/dal/.sqlx/query-c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2.json @@ -0,0 +1,94 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n address,\n topic1,\n topic2,\n topic3,\n topic4,\n value,\n NULL::bytea AS \"block_hash\",\n NULL::BIGINT AS \"l1_batch_number?\",\n miniblock_number,\n tx_hash,\n tx_index_in_block,\n event_index_in_block,\n event_index_in_tx\n FROM\n events\n WHERE\n tx_hash = $1\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "address", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "topic1", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "topic2", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "topic3", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "topic4", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "value", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "block_hash", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "l1_batch_number?", + "type_info": "Int8" + }, + { + "ordinal": 8, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 9, + "name": "tx_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "tx_index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "event_index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 12, + "name": "event_index_in_tx", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + null, + null, + false, + false, + false, + false, + false + ] + }, + "hash": "c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2" +} diff --git a/core/lib/dal/.sqlx/query-c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77.json b/core/lib/dal/.sqlx/query-c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77.json new file mode 100644 index 000000000000..380a98bfabcf --- /dev/null +++ b/core/lib/dal/.sqlx/query-c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n contract_verification_requests\n WHERE\n status = 'queued'\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77" +} diff --git a/core/lib/dal/.sqlx/query-c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539.json b/core/lib/dal/.sqlx/query-c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539.json new file mode 100644 index 000000000000..b341120ad7f3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n scheduler_dependency_tracker_fri (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539" +} diff --git a/core/lib/dal/.sqlx/query-c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74.json b/core/lib/dal/.sqlx/query-c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74.json new file mode 100644 index 000000000000..b04fb829dd6b --- /dev/null +++ b/core/lib/dal/.sqlx/query-c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM storage_logs\n WHERE\n miniblock_number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74" +} diff --git a/core/lib/dal/.sqlx/query-c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61.json b/core/lib/dal/.sqlx/query-c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61.json new file mode 100644 index 000000000000..8cac9f31ac05 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM compiler_versions\n WHERE\n compiler = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text" + ] + }, + "nullable": [] + }, + "hash": "c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61" +} diff --git a/core/lib/dal/.sqlx/query-c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99.json b/core/lib/dal/.sqlx/query-c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99.json new file mode 100644 index 000000000000..58c336bb8328 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99.json @@ -0,0 +1,24 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n AND storage_logs.miniblock_number <= $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n ) t\n JOIN factory_deps ON value = factory_deps.bytecode_hash\n WHERE\n value != $3\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Int8", + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99" +} diff --git a/core/lib/dal/.sqlx/query-c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36.json b/core/lib/dal/.sqlx/query-c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36.json new file mode 100644 index 000000000000..8922816c7e15 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36" +} diff --git a/core/lib/dal/.sqlx/query-c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7.json b/core/lib/dal/.sqlx/query-c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7.json new file mode 100644 index 000000000000..ea4b266d8259 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n address,\n key,\n value\n FROM\n storage_logs\n WHERE\n miniblock_number BETWEEN (\n SELECT\n MIN(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n ) AND (\n SELECT\n MAX(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n )\n ORDER BY\n miniblock_number,\n operation_number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "address", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "key", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "value", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7" +} diff --git a/core/lib/dal/.sqlx/query-c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7.json b/core/lib/dal/.sqlx/query-c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7.json new file mode 100644 index 000000000000..4c24afad4f4c --- /dev/null +++ b/core/lib/dal/.sqlx/query-c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7.json @@ -0,0 +1,21 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n prover_jobs_fri (\n l1_batch_number,\n circuit_id,\n circuit_blob_url,\n aggregation_round,\n sequence_number,\n depth,\n is_node_final_proof,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number, aggregation_round, circuit_id, depth, sequence_number) DO\n UPDATE\n SET\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int2", + "Text", + "Int2", + "Int4", + "Int4", + "Bool", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7" +} diff --git a/core/lib/dal/.sqlx/query-c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a.json b/core/lib/dal/.sqlx/query-c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a.json new file mode 100644 index 000000000000..6a74606e484f --- /dev/null +++ b/core/lib/dal/.sqlx/query-c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_address,\n l2_address\n FROM\n tokens\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_address", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "l2_address", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false + ] + }, + "hash": "c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a" +} diff --git a/core/lib/dal/.sqlx/query-c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d.json b/core/lib/dal/.sqlx/query-c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d.json new file mode 100644 index 000000000000..34bff9031943 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COALESCE(MAX(number), 0) AS \"number!\"\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d" +} diff --git a/core/lib/dal/.sqlx/query-c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765.json b/core/lib/dal/.sqlx/query-c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765.json new file mode 100644 index 000000000000..6c16372b82d2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE scheduler_dependency_tracker_fri\n SET\n status = 'queuing'\n WHERE\n l1_batch_number IN (\n SELECT\n l1_batch_number\n FROM\n scheduler_dependency_tracker_fri\n WHERE\n status != 'queued'\n AND circuit_1_final_prover_job_id IS NOT NULL\n AND circuit_2_final_prover_job_id IS NOT NULL\n AND circuit_3_final_prover_job_id IS NOT NULL\n AND circuit_4_final_prover_job_id IS NOT NULL\n AND circuit_5_final_prover_job_id IS NOT NULL\n AND circuit_6_final_prover_job_id IS NOT NULL\n AND circuit_7_final_prover_job_id IS NOT NULL\n AND circuit_8_final_prover_job_id IS NOT NULL\n AND circuit_9_final_prover_job_id IS NOT NULL\n AND circuit_10_final_prover_job_id IS NOT NULL\n AND circuit_11_final_prover_job_id IS NOT NULL\n AND circuit_12_final_prover_job_id IS NOT NULL\n AND circuit_13_final_prover_job_id IS NOT NULL\n )\n RETURNING\n l1_batch_number;\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765" +} diff --git a/core/lib/dal/.sqlx/query-c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355.json b/core/lib/dal/.sqlx/query-c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355.json new file mode 100644 index 000000000000..ebae1f42fbbb --- /dev/null +++ b/core/lib/dal/.sqlx/query-c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(l1_batches.number)\n FROM\n l1_batches\n JOIN eth_txs ON (l1_batches.eth_commit_tx_id = eth_txs.id)\n JOIN eth_txs_history AS commit_tx ON (eth_txs.confirmed_eth_tx_history_id = commit_tx.id)\n WHERE\n commit_tx.confirmed_at IS NOT NULL\n AND eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n AND EXTRACT(\n epoch\n FROM\n commit_tx.confirmed_at\n ) < $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "max", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Numeric" + ] + }, + "nullable": [ + null + ] + }, + "hash": "c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355" +} diff --git a/core/lib/dal/.sqlx/query-c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d.json b/core/lib/dal/.sqlx/query-c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d.json new file mode 100644 index 000000000000..95ae04bed503 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d.json @@ -0,0 +1,17 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $2,\n l1_proof_blob_url = $3\n WHERE\n l1_batch_number = $4\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Time", + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d" +} diff --git a/core/lib/dal/.sqlx/query-c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3.json b/core/lib/dal/.sqlx/query-c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3.json new file mode 100644 index 000000000000..b85f4c542bf8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n ) sl\n WHERE\n sl.value != $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Bytea" + ] + }, + "nullable": [ + null + ] + }, + "hash": "c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3" +} diff --git a/core/lib/dal/.sqlx/query-ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a.json b/core/lib/dal/.sqlx/query-ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a.json new file mode 100644 index 000000000000..ff49f615ab50 --- /dev/null +++ b/core/lib/dal/.sqlx/query-ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status <> 'successful'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a" +} diff --git a/core/lib/dal/.sqlx/query-cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4.json b/core/lib/dal/.sqlx/query-cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4.json new file mode 100644 index 000000000000..732992595c73 --- /dev/null +++ b/core/lib/dal/.sqlx/query-cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp,\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + true + ] + }, + "hash": "cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4" +} diff --git a/core/lib/dal/.sqlx/query-cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c.json b/core/lib/dal/.sqlx/query-cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c.json new file mode 100644 index 000000000000..d2087e0a32b4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c.json @@ -0,0 +1,257 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n system_logs,\n compressed_state_diffs,\n protocol_version,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n (\n SELECT\n l1_batches.*,\n ROW_NUMBER() OVER (\n ORDER BY\n number ASC\n ) AS ROW_NUMBER\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id IS NOT NULL\n AND l1_batches.skip_proof = TRUE\n AND l1_batches.number > $1\n ORDER BY\n number\n LIMIT\n $2\n ) inn\n LEFT JOIN commitments ON commitments.l1_batch_number = inn.number\n WHERE\n number - ROW_NUMBER = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + false, + true, + true, + true, + true, + true + ] + }, + "hash": "cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c" +} diff --git a/core/lib/dal/.sqlx/query-ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef.json b/core/lib/dal/.sqlx/query-ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef.json new file mode 100644 index 000000000000..6f83fd55064d --- /dev/null +++ b/core/lib/dal/.sqlx/query-ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef.json @@ -0,0 +1,32 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MIN(l1_batch_number) AS \"l1_batch_number!\",\n circuit_id,\n aggregation_round\n FROM\n prover_jobs_fri\n WHERE\n status IN ('queued', 'in_gpu_proof', 'in_progress', 'failed')\n GROUP BY\n circuit_id,\n aggregation_round\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number!", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 2, + "name": "aggregation_round", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null, + false, + false + ] + }, + "hash": "ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef" +} diff --git a/core/lib/dal/.sqlx/query-cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970.json b/core/lib/dal/.sqlx/query-cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970.json new file mode 100644 index 000000000000..b1eae968a897 --- /dev/null +++ b/core/lib/dal/.sqlx/query-cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n u.hashed_key AS \"hashed_key!\",\n (\n SELECT\n value\n FROM\n storage_logs\n WHERE\n hashed_key = u.hashed_key\n AND miniblock_number <= $2\n ORDER BY\n miniblock_number DESC,\n operation_number DESC\n LIMIT\n 1\n ) AS \"value?\"\n FROM\n UNNEST($1::bytea[]) AS u (hashed_key)\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key!", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "value?", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "ByteaArray", + "Int8" + ] + }, + "nullable": [ + null, + null + ] + }, + "hash": "cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970" +} diff --git a/core/lib/dal/.sqlx/query-d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99.json b/core/lib/dal/.sqlx/query-d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99.json new file mode 100644 index 000000000000..0370a63d65e3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "timestamp", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99" +} diff --git a/core/lib/dal/.sqlx/query-d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283.json b/core/lib/dal/.sqlx/query-d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283.json new file mode 100644 index 000000000000..fd6ed893c23a --- /dev/null +++ b/core/lib/dal/.sqlx/query-d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283.json @@ -0,0 +1,254 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number = 0\n OR eth_commit_tx_id IS NOT NULL\n AND commitment IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 3, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 4, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "parent_hash", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "commitment", + "type_info": "Bytea" + }, + { + "ordinal": 11, + "name": "compressed_write_logs", + "type_info": "Bytea" + }, + { + "ordinal": 12, + "name": "compressed_contracts", + "type_info": "Bytea" + }, + { + "ordinal": 13, + "name": "eth_prove_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 14, + "name": "eth_commit_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 15, + "name": "eth_execute_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 16, + "name": "merkle_root_hash", + "type_info": "Bytea" + }, + { + "ordinal": 17, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 19, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 20, + "name": "compressed_initial_writes", + "type_info": "Bytea" + }, + { + "ordinal": 21, + "name": "compressed_repeated_writes", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "l2_l1_compressed_messages", + "type_info": "Bytea" + }, + { + "ordinal": 23, + "name": "l2_l1_merkle_root", + "type_info": "Bytea" + }, + { + "ordinal": 24, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 25, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 26, + "name": "rollup_last_leaf_index", + "type_info": "Int8" + }, + { + "ordinal": 27, + "name": "zkporter_is_available", + "type_info": "Bool" + }, + { + "ordinal": 28, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 29, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 30, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 31, + "name": "aux_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 32, + "name": "pass_through_data_hash", + "type_info": "Bytea" + }, + { + "ordinal": 33, + "name": "meta_parameters_hash", + "type_info": "Bytea" + }, + { + "ordinal": 34, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 35, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 36, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 37, + "name": "events_queue_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 38, + "name": "bootloader_initial_content_commitment", + "type_info": "Bytea" + }, + { + "ordinal": 39, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + true, + true, + true, + false, + false, + true, + true, + true, + true, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283" +} diff --git a/core/lib/dal/.sqlx/query-d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9.json b/core/lib/dal/.sqlx/query-d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9.json new file mode 100644 index 000000000000..8770a9b596ea --- /dev/null +++ b/core/lib/dal/.sqlx/query-d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS execute_tx ON (l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id)\n WHERE\n execute_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false + ] + }, + "hash": "d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9" +} diff --git a/core/lib/dal/.sqlx/query-d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76.json b/core/lib/dal/.sqlx/query-d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76.json new file mode 100644 index 000000000000..bff9c151373f --- /dev/null +++ b/core/lib/dal/.sqlx/query-d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76.json @@ -0,0 +1,94 @@ +{ + "db_name": "PostgreSQL", + "query": "\n WITH\n events_select AS (\n SELECT\n address,\n topic1,\n topic2,\n topic3,\n topic4,\n value,\n miniblock_number,\n tx_hash,\n tx_index_in_block,\n event_index_in_block,\n event_index_in_tx\n FROM\n events\n WHERE\n miniblock_number > $1\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n )\n SELECT\n miniblocks.hash AS \"block_hash?\",\n address AS \"address!\",\n topic1 AS \"topic1!\",\n topic2 AS \"topic2!\",\n topic3 AS \"topic3!\",\n topic4 AS \"topic4!\",\n value AS \"value!\",\n miniblock_number AS \"miniblock_number!\",\n miniblocks.l1_batch_number AS \"l1_batch_number?\",\n tx_hash AS \"tx_hash!\",\n tx_index_in_block AS \"tx_index_in_block!\",\n event_index_in_block AS \"event_index_in_block!\",\n event_index_in_tx AS \"event_index_in_tx!\"\n FROM\n events_select\n INNER JOIN miniblocks ON events_select.miniblock_number = miniblocks.number\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "block_hash?", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "address!", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "topic1!", + "type_info": "Bytea" + }, + { + "ordinal": 3, + "name": "topic2!", + "type_info": "Bytea" + }, + { + "ordinal": 4, + "name": "topic3!", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "topic4!", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "value!", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "miniblock_number!", + "type_info": "Int8" + }, + { + "ordinal": 8, + "name": "l1_batch_number?", + "type_info": "Int8" + }, + { + "ordinal": 9, + "name": "tx_hash!", + "type_info": "Bytea" + }, + { + "ordinal": 10, + "name": "tx_index_in_block!", + "type_info": "Int4" + }, + { + "ordinal": 11, + "name": "event_index_in_block!", + "type_info": "Int4" + }, + { + "ordinal": 12, + "name": "event_index_in_tx!", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + true, + false, + false, + false, + false + ] + }, + "hash": "d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76" +} diff --git a/core/lib/dal/.sqlx/query-d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42.json b/core/lib/dal/.sqlx/query-d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42.json new file mode 100644 index 000000000000..362a4a9b83da --- /dev/null +++ b/core/lib/dal/.sqlx/query-d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n address,\n key\n FROM\n protective_reads\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "address", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "key", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42" +} diff --git a/core/lib/dal/.sqlx/query-d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711.json b/core/lib/dal/.sqlx/query-d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711.json new file mode 100644 index 000000000000..a049d76c24b7 --- /dev/null +++ b/core/lib/dal/.sqlx/query-d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711.json @@ -0,0 +1,29 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number,\n status\n FROM\n proof_compression_jobs_fri\n WHERE\n l1_batch_number = (\n SELECT\n MIN(l1_batch_number)\n FROM\n proof_compression_jobs_fri\n WHERE\n status = $1\n OR status = $2\n )\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + } + ], + "parameters": { + "Left": [ + "Text", + "Text" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711" +} diff --git a/core/lib/dal/.sqlx/query-d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436.json b/core/lib/dal/.sqlx/query-d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436.json new file mode 100644 index 000000000000..c415e3d33ce4 --- /dev/null +++ b/core/lib/dal/.sqlx/query-d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n COUNT(*)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436" +} diff --git a/core/lib/dal/.sqlx/query-d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541.json b/core/lib/dal/.sqlx/query-d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541.json new file mode 100644 index 000000000000..f0ea745821f3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541.json @@ -0,0 +1,136 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n compressed_state_diffs,\n pubdata_input\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id = $1\n OR eth_prove_tx_id = $1\n OR eth_execute_tx_id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "l2_tx_count", + "type_info": "Int4" + }, + { + "ordinal": 3, + "name": "timestamp", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "is_finished", + "type_info": "Bool" + }, + { + "ordinal": 5, + "name": "fee_account_address", + "type_info": "Bytea" + }, + { + "ordinal": 6, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 7, + "name": "l2_to_l1_messages", + "type_info": "ByteaArray" + }, + { + "ordinal": 8, + "name": "bloom", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "priority_ops_onchain_data", + "type_info": "ByteaArray" + }, + { + "ordinal": 10, + "name": "used_contract_hashes", + "type_info": "Jsonb" + }, + { + "ordinal": 11, + "name": "base_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 12, + "name": "l1_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "l2_fair_gas_price", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "bootloader_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 15, + "name": "default_aa_code_hash", + "type_info": "Bytea" + }, + { + "ordinal": 16, + "name": "protocol_version", + "type_info": "Int4" + }, + { + "ordinal": 17, + "name": "system_logs", + "type_info": "ByteaArray" + }, + { + "ordinal": 18, + "name": "compressed_state_diffs", + "type_info": "Bytea" + }, + { + "ordinal": 19, + "name": "pubdata_input", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + false, + true, + true + ] + }, + "hash": "d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541" +} diff --git a/core/lib/dal/.sqlx/query-d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e.json b/core/lib/dal/.sqlx/query-d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e.json new file mode 100644 index 000000000000..ae7bcea1882f --- /dev/null +++ b/core/lib/dal/.sqlx/query-d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e.json @@ -0,0 +1,35 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n id,\n status,\n attempts\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Interval", + "Int2" + ] + }, + "nullable": [ + false, + false, + false + ] + }, + "hash": "d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e" +} diff --git a/core/lib/dal/.sqlx/query-d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d.json b/core/lib/dal/.sqlx/query-d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d.json new file mode 100644 index 000000000000..22f1d4d31bcf --- /dev/null +++ b/core/lib/dal/.sqlx/query-d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE miniblocks\n SET\n consensus = $2\n WHERE\n number = $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Jsonb" + ] + }, + "nullable": [] + }, + "hash": "d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d" +} diff --git a/core/lib/dal/.sqlx/query-da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b.json b/core/lib/dal/.sqlx/query-da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b.json new file mode 100644 index 000000000000..9c24f8706454 --- /dev/null +++ b/core/lib/dal/.sqlx/query-da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM storage\n WHERE\n hashed_key = ANY ($1)\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [] + }, + "hash": "da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b" +} diff --git a/core/lib/dal/.sqlx/query-db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6.json b/core/lib/dal/.sqlx/query-db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6.json new file mode 100644 index 000000000000..d9f7527dfa00 --- /dev/null +++ b/core/lib/dal/.sqlx/query-db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n id = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Time", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6" +} diff --git a/core/lib/dal/.sqlx/query-dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572.json b/core/lib/dal/.sqlx/query-dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572.json new file mode 100644 index 000000000000..9669622f5cf2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "SELECT COUNT(miniblocks.number) FROM miniblocks WHERE l1_batch_number IS NULL", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "count", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572" +} diff --git a/core/lib/dal/.sqlx/query-dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10.json b/core/lib/dal/.sqlx/query-dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10.json new file mode 100644 index 000000000000..77263c2a4dd7 --- /dev/null +++ b/core/lib/dal/.sqlx/query-dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n hash IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10" +} diff --git a/core/lib/dal/.sqlx/query-dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48.json b/core/lib/dal/.sqlx/query-dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48.json new file mode 100644 index 000000000000..b7320d0c3bb2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48.json @@ -0,0 +1,26 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n hash = $1,\n merkle_root_hash = $2,\n compressed_repeated_writes = $3,\n compressed_initial_writes = $4,\n l2_l1_compressed_messages = $5,\n l2_l1_merkle_root = $6,\n zkporter_is_available = $7,\n parent_hash = $8,\n rollup_last_leaf_index = $9,\n pass_through_data_hash = $10,\n meta_parameters_hash = $11,\n compressed_state_diffs = $12,\n updated_at = NOW()\n WHERE\n number = $13\n AND hash IS NULL\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bytea", + "Bool", + "Bytea", + "Int8", + "Bytea", + "Bytea", + "Bytea", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48" +} diff --git a/core/lib/dal/.sqlx/query-dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb.json b/core/lib/dal/.sqlx/query-dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb.json new file mode 100644 index 000000000000..70449a85ea4b --- /dev/null +++ b/core/lib/dal/.sqlx/query-dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb.json @@ -0,0 +1,34 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n status,\n error,\n compilation_errors\n FROM\n contract_verification_requests\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 1, + "name": "error", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "compilation_errors", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + true, + true + ] + }, + "hash": "dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb" +} diff --git a/core/lib/dal/.sqlx/query-dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31.json b/core/lib/dal/.sqlx/query-dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31.json new file mode 100644 index 000000000000..ef070554c2fd --- /dev/null +++ b/core/lib/dal/.sqlx/query-dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE l1_batches\n SET\n eth_execute_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int4", + "Int8", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31" +} diff --git a/core/lib/dal/.sqlx/query-df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f.json b/core/lib/dal/.sqlx/query-df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f.json new file mode 100644 index 000000000000..9ad3099d7765 --- /dev/null +++ b/core/lib/dal/.sqlx/query-df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n attempts\n FROM\n prover_jobs_fri\n WHERE\n id = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "attempts", + "type_info": "Int2" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f" +} diff --git a/core/lib/dal/.sqlx/query-df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386.json b/core/lib/dal/.sqlx/query-df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386.json new file mode 100644 index 000000000000..a04523bc07b8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386.json @@ -0,0 +1,19 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n snapshot_recovery (\n l1_batch_number,\n l1_batch_root_hash,\n miniblock_number,\n miniblock_root_hash,\n last_finished_chunk_id,\n total_chunk_count,\n updated_at,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n l1_batch_number = excluded.l1_batch_number,\n l1_batch_root_hash = excluded.l1_batch_root_hash,\n miniblock_number = excluded.miniblock_number,\n miniblock_root_hash = excluded.miniblock_root_hash,\n last_finished_chunk_id = excluded.last_finished_chunk_id,\n total_chunk_count = excluded.total_chunk_count,\n updated_at = excluded.updated_at\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Bytea", + "Int8", + "Bytea", + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386" +} diff --git a/core/lib/dal/.sqlx/query-e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0.json b/core/lib/dal/.sqlx/query-e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0.json new file mode 100644 index 000000000000..929e4de8c1b3 --- /dev/null +++ b/core/lib/dal/.sqlx/query-e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode,\n bytecode_hash\n FROM\n factory_deps\n WHERE\n bytecode_hash = ANY ($1)\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "bytecode_hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "ByteaArray" + ] + }, + "nullable": [ + false, + false + ] + }, + "hash": "e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0" +} diff --git a/core/lib/dal/.sqlx/query-e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e.json b/core/lib/dal/.sqlx/query-e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e.json new file mode 100644 index 000000000000..32cc15c206db --- /dev/null +++ b/core/lib/dal/.sqlx/query-e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e.json @@ -0,0 +1,118 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n scheduler_dependency_tracker_fri\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "status", + "type_info": "Text" + }, + { + "ordinal": 2, + "name": "circuit_1_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 3, + "name": "circuit_2_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 4, + "name": "circuit_3_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 5, + "name": "circuit_4_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "circuit_5_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 7, + "name": "circuit_6_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 8, + "name": "circuit_7_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 9, + "name": "circuit_8_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 10, + "name": "circuit_9_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "circuit_10_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "circuit_11_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 13, + "name": "circuit_12_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 14, + "name": "circuit_13_final_prover_job_id", + "type_info": "Int8" + }, + { + "ordinal": 15, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 16, + "name": "updated_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false + ] + }, + "hash": "e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e" +} diff --git a/core/lib/dal/.sqlx/query-e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b.json b/core/lib/dal/.sqlx/query-e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b.json new file mode 100644 index 000000000000..5606beb7123e --- /dev/null +++ b/core/lib/dal/.sqlx/query-e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n serialized_events_queue\n FROM\n events_queue\n WHERE\n l1_batch_number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "serialized_events_queue", + "type_info": "Jsonb" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b" +} diff --git a/core/lib/dal/.sqlx/query-e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6.json b/core/lib/dal/.sqlx/query-e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6.json new file mode 100644 index 000000000000..3176fb6ac3e8 --- /dev/null +++ b/core/lib/dal/.sqlx/query-e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM transactions\n WHERE\n miniblock_number IS NULL\n AND received_at < NOW() - $1::INTERVAL\n AND is_priority = FALSE\n AND error IS NULL\n RETURNING\n hash\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Interval" + ] + }, + "nullable": [ + false + ] + }, + "hash": "e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6" +} diff --git a/core/lib/dal/.sqlx/query-e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705.json b/core/lib/dal/.sqlx/query-e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705.json new file mode 100644 index 000000000000..b61fbc645a0f --- /dev/null +++ b/core/lib/dal/.sqlx/query-e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n usd_price,\n usd_price_updated_at\n FROM\n tokens\n WHERE\n l2_address = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "usd_price", + "type_info": "Numeric" + }, + { + "ordinal": 1, + "name": "usd_price_updated_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + true, + true + ] + }, + "hash": "e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705" +} diff --git a/core/lib/dal/.sqlx/query-e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b.json b/core/lib/dal/.sqlx/query-e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b.json new file mode 100644 index 000000000000..54ea6b6eb03a --- /dev/null +++ b/core/lib/dal/.sqlx/query-e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n proof_compression_jobs_fri (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text" + ] + }, + "nullable": [] + }, + "hash": "e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b" +} diff --git a/core/lib/dal/.sqlx/query-e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be.json b/core/lib/dal/.sqlx/query-e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be.json new file mode 100644 index 000000000000..831a67cbee99 --- /dev/null +++ b/core/lib/dal/.sqlx/query-e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be.json @@ -0,0 +1,12 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE transactions\n SET\n in_mempool = FALSE\n WHERE\n in_mempool = TRUE\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [] + }, + "nullable": [] + }, + "hash": "e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be" +} diff --git a/core/lib/dal/.sqlx/query-e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034.json b/core/lib/dal/.sqlx/query-e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034.json new file mode 100644 index 000000000000..975c061632ac --- /dev/null +++ b/core/lib/dal/.sqlx/query-e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034.json @@ -0,0 +1,61 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n processing_started_at = NOW(),\n updated_at = NOW(),\n picked_by = $4\n WHERE\n id = (\n SELECT\n pj.id\n FROM\n (\n SELECT\n *\n FROM\n UNNEST($1::SMALLINT[], $2::SMALLINT[])\n ) AS tuple (circuit_id, ROUND)\n JOIN LATERAL (\n SELECT\n *\n FROM\n prover_jobs_fri AS pj\n WHERE\n pj.status = 'queued'\n AND pj.protocol_version = ANY ($3)\n AND pj.circuit_id = tuple.circuit_id\n AND pj.aggregation_round = tuple.round\n ORDER BY\n pj.l1_batch_number ASC,\n pj.id ASC\n LIMIT\n 1\n ) AS pj ON TRUE\n ORDER BY\n pj.l1_batch_number ASC,\n pj.aggregation_round DESC,\n pj.id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "aggregation_round", + "type_info": "Int2" + }, + { + "ordinal": 4, + "name": "sequence_number", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "depth", + "type_info": "Int4" + }, + { + "ordinal": 6, + "name": "is_node_final_proof", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [ + "Int2Array", + "Int2Array", + "Int4Array", + "Text" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false + ] + }, + "hash": "e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034" +} diff --git a/core/lib/dal/.sqlx/query-e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744.json b/core/lib/dal/.sqlx/query-e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744.json new file mode 100644 index 000000000000..0bdcbb99add0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744.json @@ -0,0 +1,19 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n node_aggregation_witness_jobs_fri (\n l1_batch_number,\n circuit_id,\n depth,\n aggregations_url,\n number_of_dependent_jobs,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number, circuit_id, depth) DO\n UPDATE\n SET\n updated_at = NOW()\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Int2", + "Int4", + "Text", + "Int4", + "Int4" + ] + }, + "nullable": [] + }, + "hash": "e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744" +} diff --git a/core/lib/dal/.sqlx/query-ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30.json b/core/lib/dal/.sqlx/query-ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30.json new file mode 100644 index 000000000000..718b2a8f6872 --- /dev/null +++ b/core/lib/dal/.sqlx/query-ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n DELETE FROM miniblocks\n WHERE\n number > $1\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30" +} diff --git a/core/lib/dal/.sqlx/query-ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9.json b/core/lib/dal/.sqlx/query-ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9.json new file mode 100644 index 000000000000..7c0264b5646b --- /dev/null +++ b/core/lib/dal/.sqlx/query-ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9.json @@ -0,0 +1,60 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1,\n proof_blob_url = $2\n WHERE\n id = $3\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 2, + "name": "circuit_id", + "type_info": "Int2" + }, + { + "ordinal": 3, + "name": "aggregation_round", + "type_info": "Int2" + }, + { + "ordinal": 4, + "name": "sequence_number", + "type_info": "Int4" + }, + { + "ordinal": 5, + "name": "depth", + "type_info": "Int4" + }, + { + "ordinal": 6, + "name": "is_node_final_proof", + "type_info": "Bool" + } + ], + "parameters": { + "Left": [ + "Time", + "Text", + "Int8" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false + ] + }, + "hash": "ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9" +} diff --git a/core/lib/dal/.sqlx/query-edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4.json b/core/lib/dal/.sqlx/query-edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4.json new file mode 100644 index 000000000000..2c7d7f1da5f0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n l1_batch_number = $2\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Time", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4" +} diff --git a/core/lib/dal/.sqlx/query-ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f.json b/core/lib/dal/.sqlx/query-ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f.json new file mode 100644 index 000000000000..5732126a7ffd --- /dev/null +++ b/core/lib/dal/.sqlx/query-ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f.json @@ -0,0 +1,82 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n ORDER BY\n created_at DESC\n LIMIT\n 1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "id", + "type_info": "Int4" + }, + { + "ordinal": 1, + "name": "eth_tx_id", + "type_info": "Int4" + }, + { + "ordinal": 2, + "name": "tx_hash", + "type_info": "Text" + }, + { + "ordinal": 3, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 4, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 5, + "name": "base_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "priority_fee_per_gas", + "type_info": "Int8" + }, + { + "ordinal": 7, + "name": "confirmed_at", + "type_info": "Timestamp" + }, + { + "ordinal": 8, + "name": "signed_raw_tx", + "type_info": "Bytea" + }, + { + "ordinal": 9, + "name": "sent_at_block", + "type_info": "Int4" + }, + { + "ordinal": 10, + "name": "sent_at", + "type_info": "Timestamp" + } + ], + "parameters": { + "Left": [ + "Int4" + ] + }, + "nullable": [ + false, + false, + false, + false, + false, + false, + false, + true, + true, + true, + true + ] + }, + "hash": "ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f" +} diff --git a/core/lib/dal/.sqlx/query-ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea.json b/core/lib/dal/.sqlx/query-ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea.json new file mode 100644 index 000000000000..60d4a49d6281 --- /dev/null +++ b/core/lib/dal/.sqlx/query-ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea.json @@ -0,0 +1,28 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_batch_number,\n l1_batch_tx_index\n FROM\n transactions\n WHERE\n hash = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 1, + "name": "l1_batch_tx_index", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + true, + true + ] + }, + "hash": "ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea" +} diff --git a/core/lib/dal/.sqlx/query-ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737.json b/core/lib/dal/.sqlx/query-ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737.json new file mode 100644 index 000000000000..79b20fabb284 --- /dev/null +++ b/core/lib/dal/.sqlx/query-ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n proof_compression_jobs_fri (l1_batch_number, fri_proof_blob_url, status, created_at, updated_at)\n VALUES\n ($1, $2, $3, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8", + "Text", + "Text" + ] + }, + "nullable": [] + }, + "hash": "ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737" +} diff --git a/core/lib/dal/.sqlx/query-f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66.json b/core/lib/dal/.sqlx/query-f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66.json new file mode 100644 index 000000000000..9815b5d3895e --- /dev/null +++ b/core/lib/dal/.sqlx/query-f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66.json @@ -0,0 +1,36 @@ +{ + "db_name": "PostgreSQL", + "query": "\n WITH\n sl AS (\n SELECT\n (\n SELECT\n ARRAY[hashed_key, value] AS kv\n FROM\n storage_logs\n WHERE\n storage_logs.miniblock_number = $1\n AND storage_logs.hashed_key >= u.start_key\n AND storage_logs.hashed_key <= u.end_key\n ORDER BY\n storage_logs.hashed_key\n LIMIT\n 1\n )\n FROM\n UNNEST($2::bytea[], $3::bytea[]) AS u (start_key, end_key)\n )\n SELECT\n sl.kv[1] AS \"hashed_key?\",\n sl.kv[2] AS \"value?\",\n initial_writes.index\n FROM\n sl\n LEFT OUTER JOIN initial_writes ON initial_writes.hashed_key = sl.kv[1]\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hashed_key?", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "value?", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Int8", + "ByteaArray", + "ByteaArray" + ] + }, + "nullable": [ + null, + null, + true + ] + }, + "hash": "f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66" +} diff --git a/core/lib/dal/.sqlx/query-f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a.json b/core/lib/dal/.sqlx/query-f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a.json new file mode 100644 index 000000000000..616173355a94 --- /dev/null +++ b/core/lib/dal/.sqlx/query-f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n INDEX\n FROM\n initial_writes\n WHERE\n hashed_key = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "index", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea" + ] + }, + "nullable": [ + false + ] + }, + "hash": "f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a" +} diff --git a/core/lib/dal/.sqlx/query-f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593.json b/core/lib/dal/.sqlx/query-f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593.json new file mode 100644 index 000000000000..7ffda2c8a32e --- /dev/null +++ b/core/lib/dal/.sqlx/query-f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_address\n FROM\n tokens\n WHERE\n market_volume > $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_address", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Numeric" + ] + }, + "nullable": [ + false + ] + }, + "hash": "f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593" +} diff --git a/core/lib/dal/.sqlx/query-f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb.json b/core/lib/dal/.sqlx/query-f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb.json new file mode 100644 index 000000000000..9495f8f7c828 --- /dev/null +++ b/core/lib/dal/.sqlx/query-f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb.json @@ -0,0 +1,44 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l1_address,\n l2_address,\n NAME,\n symbol,\n decimals\n FROM\n tokens\n WHERE\n well_known = TRUE\n ORDER BY\n symbol\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_address", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "l2_address", + "type_info": "Bytea" + }, + { + "ordinal": 2, + "name": "name", + "type_info": "Varchar" + }, + { + "ordinal": 3, + "name": "symbol", + "type_info": "Varchar" + }, + { + "ordinal": 4, + "name": "decimals", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + false, + false, + false, + false, + false + ] + }, + "hash": "f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb" +} diff --git a/core/lib/dal/.sqlx/query-f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630.json b/core/lib/dal/.sqlx/query-f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630.json new file mode 100644 index 000000000000..59c28852a03e --- /dev/null +++ b/core/lib/dal/.sqlx/query-f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n error = $2,\n updated_at = NOW()\n WHERE\n l1_batch_number = $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Text", + "Int8" + ] + }, + "nullable": [] + }, + "hash": "f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630" +} diff --git a/core/lib/dal/.sqlx/query-f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a.json b/core/lib/dal/.sqlx/query-f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a.json new file mode 100644 index 000000000000..d62e213ef57b --- /dev/null +++ b/core/lib/dal/.sqlx/query-f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a.json @@ -0,0 +1,232 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n l1_batch_number = $1\n ORDER BY\n miniblock_number,\n index_in_block\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "hash", + "type_info": "Bytea" + }, + { + "ordinal": 1, + "name": "is_priority", + "type_info": "Bool" + }, + { + "ordinal": 2, + "name": "full_fee", + "type_info": "Numeric" + }, + { + "ordinal": 3, + "name": "layer_2_tip_fee", + "type_info": "Numeric" + }, + { + "ordinal": 4, + "name": "initiator_address", + "type_info": "Bytea" + }, + { + "ordinal": 5, + "name": "nonce", + "type_info": "Int8" + }, + { + "ordinal": 6, + "name": "signature", + "type_info": "Bytea" + }, + { + "ordinal": 7, + "name": "input", + "type_info": "Bytea" + }, + { + "ordinal": 8, + "name": "data", + "type_info": "Jsonb" + }, + { + "ordinal": 9, + "name": "received_at", + "type_info": "Timestamp" + }, + { + "ordinal": 10, + "name": "priority_op_id", + "type_info": "Int8" + }, + { + "ordinal": 11, + "name": "l1_batch_number", + "type_info": "Int8" + }, + { + "ordinal": 12, + "name": "index_in_block", + "type_info": "Int4" + }, + { + "ordinal": 13, + "name": "error", + "type_info": "Varchar" + }, + { + "ordinal": 14, + "name": "gas_limit", + "type_info": "Numeric" + }, + { + "ordinal": 15, + "name": "gas_per_storage_limit", + "type_info": "Numeric" + }, + { + "ordinal": 16, + "name": "gas_per_pubdata_limit", + "type_info": "Numeric" + }, + { + "ordinal": 17, + "name": "tx_format", + "type_info": "Int4" + }, + { + "ordinal": 18, + "name": "created_at", + "type_info": "Timestamp" + }, + { + "ordinal": 19, + "name": "updated_at", + "type_info": "Timestamp" + }, + { + "ordinal": 20, + "name": "execution_info", + "type_info": "Jsonb" + }, + { + "ordinal": 21, + "name": "contract_address", + "type_info": "Bytea" + }, + { + "ordinal": 22, + "name": "in_mempool", + "type_info": "Bool" + }, + { + "ordinal": 23, + "name": "l1_block_number", + "type_info": "Int4" + }, + { + "ordinal": 24, + "name": "value", + "type_info": "Numeric" + }, + { + "ordinal": 25, + "name": "paymaster", + "type_info": "Bytea" + }, + { + "ordinal": 26, + "name": "paymaster_input", + "type_info": "Bytea" + }, + { + "ordinal": 27, + "name": "max_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 28, + "name": "max_priority_fee_per_gas", + "type_info": "Numeric" + }, + { + "ordinal": 29, + "name": "effective_gas_price", + "type_info": "Numeric" + }, + { + "ordinal": 30, + "name": "miniblock_number", + "type_info": "Int8" + }, + { + "ordinal": 31, + "name": "l1_batch_tx_index", + "type_info": "Int4" + }, + { + "ordinal": 32, + "name": "refunded_gas", + "type_info": "Int8" + }, + { + "ordinal": 33, + "name": "l1_tx_mint", + "type_info": "Numeric" + }, + { + "ordinal": 34, + "name": "l1_tx_refund_recipient", + "type_info": "Bytea" + }, + { + "ordinal": 35, + "name": "upgrade_id", + "type_info": "Int4" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false, + false, + true, + true, + false, + true, + true, + true, + false, + false, + true, + true, + true, + true, + true, + true, + true, + true, + false, + false, + false, + true, + false, + true, + false, + false, + false, + true, + true, + true, + true, + true, + false, + true, + true, + true + ] + }, + "hash": "f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a" +} diff --git a/core/lib/dal/.sqlx/query-f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7.json b/core/lib/dal/.sqlx/query-f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7.json new file mode 100644 index 000000000000..e6e12748d0d2 --- /dev/null +++ b/core/lib/dal/.sqlx/query-f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7.json @@ -0,0 +1,24 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $3\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status = $2\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n proof_compression_jobs_fri.l1_batch_number\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l1_batch_number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Text", + "Text", + "Text" + ] + }, + "nullable": [ + false + ] + }, + "hash": "f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7" +} diff --git a/core/lib/dal/.sqlx/query-f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e.json b/core/lib/dal/.sqlx/query-f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e.json new file mode 100644 index 000000000000..cdf4b166270d --- /dev/null +++ b/core/lib/dal/.sqlx/query-f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e.json @@ -0,0 +1,20 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n MIN(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n hash IS NOT NULL\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "number", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [] + }, + "nullable": [ + null + ] + }, + "hash": "f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e" +} diff --git a/core/lib/dal/.sqlx/query-f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0.json b/core/lib/dal/.sqlx/query-f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0.json new file mode 100644 index 000000000000..c10268b73326 --- /dev/null +++ b/core/lib/dal/.sqlx/query-f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0.json @@ -0,0 +1,15 @@ +{ + "db_name": "PostgreSQL", + "query": "\n INSERT INTO\n transaction_traces (tx_hash, trace, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Bytea", + "Jsonb" + ] + }, + "nullable": [] + }, + "hash": "f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0" +} diff --git a/core/lib/dal/.sqlx/query-fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99.json b/core/lib/dal/.sqlx/query-fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99.json new file mode 100644 index 000000000000..3dd33855e0e0 --- /dev/null +++ b/core/lib/dal/.sqlx/query-fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n nonce AS \"nonce!\"\n FROM\n transactions\n WHERE\n initiator_address = $1\n AND nonce >= $2\n AND is_priority = FALSE\n AND (\n miniblock_number IS NOT NULL\n OR error IS NULL\n )\n ORDER BY\n nonce\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "nonce!", + "type_info": "Int8" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Int8" + ] + }, + "nullable": [ + true + ] + }, + "hash": "fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99" +} diff --git a/core/lib/dal/.sqlx/query-fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b.json b/core/lib/dal/.sqlx/query-fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b.json new file mode 100644 index 000000000000..effc22d6a43b --- /dev/null +++ b/core/lib/dal/.sqlx/query-fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b.json @@ -0,0 +1,16 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = 'available',\n updated_at = NOW()\n WHERE\n instance_host = $1::TEXT::inet\n AND instance_port = $2\n AND instance_status = 'full'\n AND zone = $3\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Text", + "Int4", + "Text" + ] + }, + "nullable": [] + }, + "hash": "fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b" +} diff --git a/core/lib/dal/.sqlx/query-fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b.json b/core/lib/dal/.sqlx/query-fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b.json new file mode 100644 index 000000000000..f8ad468d70db --- /dev/null +++ b/core/lib/dal/.sqlx/query-fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b.json @@ -0,0 +1,23 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n bytecode_hash = $1\n AND miniblock_number <= $2\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "bytecode", + "type_info": "Bytea" + } + ], + "parameters": { + "Left": [ + "Bytea", + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b" +} diff --git a/core/lib/dal/.sqlx/query-fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429.json b/core/lib/dal/.sqlx/query-fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429.json new file mode 100644 index 000000000000..bdcf7e5f0371 --- /dev/null +++ b/core/lib/dal/.sqlx/query-fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429.json @@ -0,0 +1,14 @@ +{ + "db_name": "PostgreSQL", + "query": "\n UPDATE miniblocks\n SET\n l1_batch_number = $1\n WHERE\n l1_batch_number IS NULL\n ", + "describe": { + "columns": [], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [] + }, + "hash": "fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429" +} diff --git a/core/lib/dal/.sqlx/query-fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084.json b/core/lib/dal/.sqlx/query-fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084.json new file mode 100644 index 000000000000..5573cdd99530 --- /dev/null +++ b/core/lib/dal/.sqlx/query-fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084.json @@ -0,0 +1,22 @@ +{ + "db_name": "PostgreSQL", + "query": "\n SELECT\n l2_to_l1_logs\n FROM\n l1_batches\n WHERE\n number = $1\n ", + "describe": { + "columns": [ + { + "ordinal": 0, + "name": "l2_to_l1_logs", + "type_info": "ByteaArray" + } + ], + "parameters": { + "Left": [ + "Int8" + ] + }, + "nullable": [ + false + ] + }, + "hash": "fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084" +} diff --git a/core/lib/dal/Cargo.toml b/core/lib/dal/Cargo.toml index a43516e7adbc..2442714f3cf8 100644 --- a/core/lib/dal/Cargo.toml +++ b/core/lib/dal/Cargo.toml @@ -29,22 +29,23 @@ url = "2" prost = "0.12.1" rand = "0.8" tokio = { version = "1", features = ["full"] } -sqlx = { version = "0.5.13", default-features = false, features = [ - "runtime-tokio-native-tls", +sqlx = { version = "0.7.3", default-features = false, features = [ + "runtime-tokio", + "tls-native-tls", "macros", "postgres", "bigdecimal", + "rust_decimal", "chrono", "json", - "offline", "migrate", "ipnetwork", ] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" -bigdecimal = "0.2.2" +bigdecimal = "0.3.0" bincode = "1" -num = "0.3.1" +num = "0.4.0" hex = "0.4" once_cell = "1.7" strum = { version = "0.24", features = ["derive"] } diff --git a/core/lib/dal/sqlx-data.json b/core/lib/dal/sqlx-data.json deleted file mode 100644 index b009e9d8028c..000000000000 --- a/core/lib/dal/sqlx-data.json +++ /dev/null @@ -1,11447 +0,0 @@ -{ - "db": "PostgreSQL", - "00b88ec7fcf40bb18e0018b7c76f6e1df560ab1e8935564355236e90b6147d2f": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Time", - "Int8" - ] - } - }, - "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n l1_batch_number = $2\n " - }, - "012bed5d34240ed28c331c8515c381d82925556a4801f678b8786235d525d784": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n eth_commit_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n " - }, - "015350f8d729ef490553550a68f07703b2581dda4fe3c00be6c5422c78980c4b": { - "describe": { - "columns": [ - { - "name": "max?", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(id) AS \"max?\"\n FROM\n protocol_versions\n " - }, - "01ac5343beb09ec5bd45b39d560e57a83f37da8999849377dfad60b44989be39": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "depth", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "status", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 5, - "type_info": "Int2" - }, - { - "name": "aggregations_url", - "ordinal": 6, - "type_info": "Text" - }, - { - "name": "processing_started_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 8, - "type_info": "Time" - }, - { - "name": "error", - "ordinal": 9, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 10, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 11, - "type_info": "Timestamp" - }, - { - "name": "number_of_dependent_jobs", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "protocol_version", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "picked_by", - "ordinal": 14, - "type_info": "Text" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n node_aggregation_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC,\n depth ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n node_aggregation_witness_jobs_fri.*\n " - }, - "01e4cde73867da612084c3f6fe882d56bbace9013f1d95ea0926eef1fb48039b": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "factory_deps_filepath", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "storage_logs_filepaths", - "ordinal": 2, - "type_info": "TextArray" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n WHERE\n l1_batch_number = $1\n " - }, - "01f72dfc1eee6360a8ef7809874a1b4ba7fe355ebc02ea49a054aa073ce324ba": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "ByteaArray" - ] - } - }, - "query": "\n UPDATE storage\n SET\n value = u.value\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (key, value)\n WHERE\n u.key = hashed_key\n " - }, - "02285b8d0bc76c8cfd259872ac24f3670813e5a5356ddcb7ac482a0201d045f7": { - "describe": { - "columns": [ - { - "name": "tx_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "index_in_block", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "block_number!", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "error", - "ordinal": 4, - "type_info": "Varchar" - }, - { - "name": "effective_gas_price", - "ordinal": 5, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "transfer_to?", - "ordinal": 7, - "type_info": "Jsonb" - }, - { - "name": "execute_contract_address?", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "tx_format?", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "gas_limit", - "ordinal": 11, - "type_info": "Numeric" - }, - { - "name": "block_hash", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "l1_batch_number?", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "contract_address?", - "ordinal": 14, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - true, - true, - true, - true, - true, - false, - null, - null, - true, - false, - true, - false, - true, - false - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n WITH\n sl AS (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.address = $1\n AND storage_logs.tx_hash = $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n )\n SELECT\n transactions.hash AS tx_hash,\n transactions.index_in_block AS index_in_block,\n transactions.l1_batch_tx_index AS l1_batch_tx_index,\n transactions.miniblock_number AS \"block_number!\",\n transactions.error AS error,\n transactions.effective_gas_price AS effective_gas_price,\n transactions.initiator_address AS initiator_address,\n transactions.data -> 'to' AS \"transfer_to?\",\n transactions.data -> 'contractAddress' AS \"execute_contract_address?\",\n transactions.tx_format AS \"tx_format?\",\n transactions.refunded_gas AS refunded_gas,\n transactions.gas_limit AS gas_limit,\n miniblocks.hash AS \"block_hash\",\n miniblocks.l1_batch_number AS \"l1_batch_number?\",\n sl.key AS \"contract_address?\"\n FROM\n transactions\n JOIN miniblocks ON miniblocks.number = transactions.miniblock_number\n LEFT JOIN sl ON sl.value != $3\n WHERE\n transactions.hash = $2\n " - }, - "026ab7dd7407f10074a2966b5eac2563a3e061bcc6505d8c295b1b2517f85f1b": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS prove_tx ON (l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id)\n WHERE\n prove_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "03c585c7e9f918e608757496088c7e3b6bdb2a08149d5f443310607d3c78988c": { - "describe": { - "columns": [ - { - "name": "storage_refunds", - "ordinal": 0, - "type_info": "Int8Array" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n storage_refunds\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "040eaa878c3473f5edc73b77e572b5ea100f59295cd693d14ee0d5ee089c7981": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n snapshots\n WHERE\n NOT (''::TEXT = ANY (storage_logs_filepaths))\n ORDER BY\n l1_batch_number DESC\n " - }, - "04fbbd198108d2614a3b29fa795994723ebe57b3ed209069bd3db906921ef1a3": { - "describe": { - "columns": [ - { - "name": "min?", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "max?", - "ordinal": 1, - "type_info": "Int8" - } - ], - "nullable": [ - null, - null - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n MIN(miniblocks.number) AS \"min?\",\n MAX(miniblocks.number) AS \"max?\"\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n " - }, - "05267e9774056bb0f984918ab861a2ee78eb59628d0429e89b27d185f83512be": { - "describe": { - "columns": [ - { - "name": "tx_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "call_trace", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n call_traces\n WHERE\n tx_hash IN (\n SELECT\n hash\n FROM\n transactions\n WHERE\n miniblock_number = $1\n )\n " - }, - "07310d96fc7e258154ad510684e33d196907ebd599e926d305e5ef9f26afa2fa": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int4", - "Text", - "Timestamp" - ] - } - }, - "query": "INSERT INTO eth_txs_history (eth_tx_id, base_fee_per_gas, priority_fee_per_gas, tx_hash, signed_raw_tx, created_at, updated_at, confirmed_at) VALUES ($1, 0, 0, $2, '\\x00', now(), now(), $3) RETURNING id" - }, - "083991abb3f1c2183d1bd1fb2ad4710daa723e2d9a23317c347f6081465c3643": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - }, - "Int8", - "Time", - "Text", - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - } - ] - } - }, - "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $3,\n error = $4\n WHERE\n l1_batch_number = $2\n AND status != $5\n RETURNING\n basic_witness_input_producer_jobs.attempts\n " - }, - "08e59ed8e2fd1a74e19d8bf0d131e4ee6682a89fb86f3b715a240805d44e6d87": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n proof_generation_details (l1_batch_number, status, proof_gen_data_blob_url, created_at, updated_at)\n VALUES\n ($1, 'ready_to_be_proven', $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "0914f0ad03d6a8c55d287f94917c6f03469d78bf4f45f5fd1eaf37171db2f04a": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status NOT IN ('generated', 'skipped')\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " - }, - "0a3c928a616b5ebc0b977bd773edcde721ca1c652ae2f8db41fb75cecdecb674": { - "describe": { - "columns": [ - { - "name": "count", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "SELECT COUNT(*) FROM storage_logs WHERE miniblock_number = $1" - }, - "0a3cb11f5bdcb8da31dbd4e3016fced141fb29dd8b6c32dd2dc3452dc294fe1f": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n protocol_versions (\n id,\n timestamp,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n bootloader_code_hash,\n default_account_code_hash,\n verifier_address,\n upgrade_tx_hash,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, NOW())\n " - }, - "0a53fc3c90a14038c9f3f32c3e2e5f7edcafa4fc6757264a96a46dbf7dd1f9cc": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Numeric", - "Numeric", - "Numeric", - "Jsonb", - "Int8", - "Numeric", - "Numeric", - "Bytea", - "Int4", - "Numeric", - "Bytea", - "Bytea", - "Int4", - "Numeric", - "Bytea", - "Timestamp" - ] - } - }, - "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n gas_limit,\n max_fee_per_gas,\n gas_per_pubdata_limit,\n data,\n priority_op_id,\n full_fee,\n layer_2_tip_fee,\n contract_address,\n l1_block_number,\n value,\n paymaster,\n paymaster_input,\n tx_format,\n l1_tx_mint,\n l1_tx_refund_recipient,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n TRUE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n NOW(),\n NOW()\n )\n ON CONFLICT (hash) DO NOTHING\n " - }, - "0aaefa9d5518ed1a2d8f735435e8048558243ff878b59586eb3a8b22794395d8": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Int4", - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n l1_batches.timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n JOIN protocol_versions ON protocol_versions.id = l1_batches.protocol_version\n WHERE\n eth_commit_tx_id IS NULL\n AND number != 0\n AND protocol_versions.bootloader_code_hash = $1\n AND protocol_versions.default_account_code_hash = $2\n AND commitment IS NOT NULL\n AND (\n protocol_versions.id = $3\n OR protocol_versions.upgrade_tx_hash IS NULL\n )\n ORDER BY\n number\n LIMIT\n $4\n " - }, - "0bdcf87f6910c7222b621f76f71bc6e326e15dca141050bc9d7dacae98a430e8": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "0c899c68886f76a232ffac0454cdfbf962636347864fc365fafa46c7a2da5f30": { - "describe": { - "columns": [ - { - "name": "virtual_blocks", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "0c95fbfb3a816bd49fd06e3a4f0a52daa202279bf612a9278f663deb78bc6e41": { - "describe": { - "columns": [ - { - "name": "protocol_version", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n protocol_version\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "0d13b8947b1bafa9e5bc6fdc70a986511265c541d81b1d21f0a751ae1399c626": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "instance_host", - "ordinal": 1, - "type_info": "Inet" - }, - { - "name": "instance_port", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "instance_status", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "specialized_prover_group_id", - "ordinal": 4, - "type_info": "Int2" - }, - { - "name": "zone", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "processing_started_at", - "ordinal": 8, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false, - false, - true - ], - "parameters": { - "Left": [ - "Interval", - "Int2", - "Text" - ] - } - }, - "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = 'reserved',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n gpu_prover_queue_fri\n WHERE\n specialized_prover_group_id = $2\n AND zone = $3\n AND (\n instance_status = 'available'\n OR (\n instance_status = 'reserved'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n )\n ORDER BY\n updated_at ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n gpu_prover_queue_fri.*\n " - }, - "10959c91f01ce0da196f4c6eaf0661a097308d9f81024fdfef24a14418202730": { - "describe": { - "columns": [ - { - "name": "verification_info", - "ordinal": 0, - "type_info": "Jsonb" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n verification_info\n FROM\n contracts_verification_info\n WHERE\n address = $1\n " - }, - "11af69fc254e54449b64c086667700a95e4c37a7a18531b3cdf120394cb055b9": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Interval" - ] - } - }, - "query": "\n UPDATE proof_generation_details\n SET\n status = 'picked_by_prover',\n updated_at = NOW(),\n prover_taken_at = NOW()\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status = 'ready_to_be_proven'\n OR (\n status = 'picked_by_prover'\n AND prover_taken_at < NOW() - $1::INTERVAL\n )\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n proof_generation_details.l1_batch_number\n " - }, - "12ab208f416e2875f89e558f0d4aff3a06b7a9c1866132d62e4449fa9436c7c4": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n " - }, - "12ab8ba692a42f528450f2adf8d263298abc0521734f807fbf45484158b167b2": { - "describe": { - "columns": [ - { - "name": "l1_address", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_address\n FROM\n tokens\n WHERE\n well_known = FALSE\n " - }, - "136569d7eb4037fd77e0fac2246c68e8e15a831f1a45dc3b2240d5c6809d5ef2": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "recursion_scheduler_level_vk_hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "recursion_node_level_vk_hash", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "recursion_leaf_level_vk_hash", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "recursion_circuits_set_vks_hash", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bootloader_code_hash", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "default_account_code_hash", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "verifier_address", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "upgrade_tx_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "created_at", - "ordinal": 10, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n protocol_versions\n WHERE\n id = $1\n " - }, - "15858168fea6808c6d59d0e6d8f28a20420763a3a22899ad0e5f4b953b615a9e": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n id\n FROM\n prover_fri_protocol_versions\n WHERE\n recursion_circuits_set_vks_hash = $1\n AND recursion_leaf_level_vk_hash = $2\n AND recursion_node_level_vk_hash = $3\n AND recursion_scheduler_level_vk_hash = $4\n " - }, - "15893d68429ba09662ee27935653c17c7a7939195dd2d9aa42512b1479d2ed20": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number!", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "last_batch_miniblock?", - "ordinal": 2, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "l1_gas_price", - "ordinal": 4, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "virtual_blocks", - "ordinal": 8, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "consensus", - "ordinal": 10, - "type_info": "Jsonb" - }, - { - "name": "protocol_version!", - "ordinal": 11, - "type_info": "Int4" - }, - { - "name": "fee_account_address?", - "ordinal": 12, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - null, - null, - false, - false, - false, - true, - true, - false, - false, - true, - true, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n miniblocks.number,\n COALESCE(\n miniblocks.l1_batch_number,\n (\n SELECT\n (MAX(number) + 1)\n FROM\n l1_batches\n )\n ) AS \"l1_batch_number!\",\n (\n SELECT\n MAX(m2.number)\n FROM\n miniblocks m2\n WHERE\n miniblocks.l1_batch_number = m2.l1_batch_number\n ) AS \"last_batch_miniblock?\",\n miniblocks.timestamp,\n miniblocks.l1_gas_price,\n miniblocks.l2_fair_gas_price,\n miniblocks.bootloader_code_hash,\n miniblocks.default_aa_code_hash,\n miniblocks.virtual_blocks,\n miniblocks.hash,\n miniblocks.consensus,\n miniblocks.protocol_version AS \"protocol_version!\",\n l1_batches.fee_account_address AS \"fee_account_address?\"\n FROM\n miniblocks\n LEFT JOIN l1_batches ON miniblocks.l1_batch_number = l1_batches.number\n WHERE\n miniblocks.number = $1\n " - }, - "1689c212d411ebd99a22210519ea2d505a1aabf52ff4136d2ed1b39c70dd1632": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number IS NOT NULL\n AND l1_batch_number IS NULL\n ORDER BY\n miniblock_number,\n index_in_block\n " - }, - "16e62660fd14f6d3731e69fa696a36408510bb05c15285dfa7708bc0b044d0c5": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Int4", - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n l1_batches.timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n JOIN protocol_versions ON protocol_versions.id = l1_batches.protocol_version\n WHERE\n eth_commit_tx_id IS NULL\n AND number != 0\n AND protocol_versions.bootloader_code_hash = $1\n AND protocol_versions.default_account_code_hash = $2\n AND commitment IS NOT NULL\n AND (\n protocol_versions.id = $3\n OR protocol_versions.upgrade_tx_hash IS NULL\n )\n AND events_queue_commitment IS NOT NULL\n AND bootloader_initial_content_commitment IS NOT NULL\n ORDER BY\n number\n LIMIT\n $4\n " - }, - "1766c0a21ba5918dd08f4babd8dbfdf10fb1cb43781219586c169fb976204331": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n initial_writes\n WHERE\n hashed_key = $1\n " - }, - "1862d3a78e4e9068df1b8ce3bbe9f3f0b5d629fdb5c36ea1bfb93ed246be968e": { - "describe": { - "columns": [ - { - "name": "is_priority", - "ordinal": 0, - "type_info": "Bool" - }, - { - "name": "initiator_address", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "gas_limit", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "received_at", - "ordinal": 4, - "type_info": "Timestamp" - }, - { - "name": "miniblock_number", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "error", - "ordinal": 6, - "type_info": "Varchar" - }, - { - "name": "effective_gas_price", - "ordinal": 7, - "type_info": "Numeric" - }, - { - "name": "refunded_gas", - "ordinal": 8, - "type_info": "Int8" - }, - { - "name": "eth_commit_tx_hash?", - "ordinal": 9, - "type_info": "Text" - }, - { - "name": "eth_prove_tx_hash?", - "ordinal": 10, - "type_info": "Text" - }, - { - "name": "eth_execute_tx_hash?", - "ordinal": 11, - "type_info": "Text" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n transactions.is_priority,\n transactions.initiator_address,\n transactions.gas_limit,\n transactions.gas_per_pubdata_limit,\n transactions.received_at,\n transactions.miniblock_number,\n transactions.error,\n transactions.effective_gas_price,\n transactions.refunded_gas,\n commit_tx.tx_hash AS \"eth_commit_tx_hash?\",\n prove_tx.tx_hash AS \"eth_prove_tx_hash?\",\n execute_tx.tx_hash AS \"eth_execute_tx_hash?\"\n FROM\n transactions\n LEFT JOIN miniblocks ON miniblocks.number = transactions.miniblock_number\n LEFT JOIN l1_batches ON l1_batches.number = miniblocks.l1_batch_number\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n transactions.hash = $1\n " - }, - "18820f4ab0c3d2cc9187c5660f9f50e423eb6134659fe52bcc2b27ad16740c96": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n DELETE FROM transactions\n WHERE\n in_mempool = TRUE\n AND initiator_address = ANY ($1)\n " - }, - "19314d74e94b610e2da6d728ca37ea964610e131d45f720f7a7b2a130fe9ed89": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text", - "Jsonb", - "Text" - ] - } - }, - "query": "\n UPDATE contract_verification_requests\n SET\n status = 'failed',\n updated_at = NOW(),\n error = $2,\n compilation_errors = $3,\n panic_message = $4\n WHERE\n id = $1\n " - }, - "19545806b8f772075096e69f8665d98a3d9f7df162ae22a98c3c7620fcd13bd2": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "recursion_scheduler_level_vk_hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "recursion_node_level_vk_hash", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "recursion_leaf_level_vk_hash", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "recursion_circuits_set_vks_hash", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bootloader_code_hash", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "default_account_code_hash", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "verifier_address", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "upgrade_tx_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "created_at", - "ordinal": 10, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n *\n FROM\n protocol_versions\n ORDER BY\n id DESC\n LIMIT\n 1\n " - }, - "19b89495be8aa735db039ccc8a262786c58e54f132588c48f07d9537cf21d3ed": { - "describe": { - "columns": [ - { - "name": "sent_at_block", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "SELECT sent_at_block FROM eth_txs_history WHERE eth_tx_id = $1 AND sent_at_block IS NOT NULL ORDER BY created_at ASC LIMIT 1" - }, - "1ad3bbd791f3ff0d31683bf59187b84c5fd52f0352f0f0e311d054cb9e45b07e": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT DISTINCT\n ON (hashed_key) hashed_key\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n miniblock_number > $1\n ) inn\n " - }, - "1b4ebbfc96b4fd66ecbe64a6be80a01a6c7cbe9297cbb55d42533fddc18719b6": { - "describe": { - "columns": [ - { - "name": "op_id", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(priority_op_id) AS \"op_id\"\n FROM\n transactions\n WHERE\n is_priority = TRUE\n " - }, - "1bc6597117db032b87df33040d61610ffa7f169d560e79e89b99eedf681c6773": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n scheduler_witness_jobs_fri (\n l1_batch_number,\n scheduler_partial_input_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n updated_at = NOW()\n " - }, - "1c60010ded4e79886890a745a050fa6d65c05d8144bdfd143480834ead4bd8d5": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "contract_address", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "source_code", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "contract_name", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "zk_compiler_version", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "compiler_version", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "optimization_used", - "ordinal": 6, - "type_info": "Bool" - }, - { - "name": "optimizer_mode", - "ordinal": 7, - "type_info": "Text" - }, - { - "name": "constructor_arguments", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "is_system", - "ordinal": 9, - "type_info": "Bool" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - true, - false, - false - ], - "parameters": { - "Left": [ - "Interval" - ] - } - }, - "query": "\n UPDATE contract_verification_requests\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id = (\n SELECT\n id\n FROM\n contract_verification_requests\n WHERE\n status = 'queued'\n OR (\n status = 'in_progress'\n AND processing_started_at < NOW() - $1::INTERVAL\n )\n ORDER BY\n created_at\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n id,\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system\n " - }, - "1c994d418ada78586de829fc2d34d26e48e968c79834858c98b7a7f9dfc81910": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM l2_to_l1_logs\n WHERE\n miniblock_number > $1\n " - }, - "1d2cc4b485536af350089cf7950be3b85419fde77038dd3de6c55aa9c55d375c": { - "describe": { - "columns": [ - { - "name": "value!", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "l1_address!", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "l2_address!", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "symbol!", - "ordinal": 3, - "type_info": "Varchar" - }, - { - "name": "name!", - "ordinal": 4, - "type_info": "Varchar" - }, - { - "name": "decimals!", - "ordinal": 5, - "type_info": "Int4" - }, - { - "name": "usd_price?", - "ordinal": 6, - "type_info": "Numeric" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - true - ], - "parameters": { - "Left": [ - "ByteaArray", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n storage.value AS \"value!\",\n tokens.l1_address AS \"l1_address!\",\n tokens.l2_address AS \"l2_address!\",\n tokens.symbol AS \"symbol!\",\n tokens.name AS \"name!\",\n tokens.decimals AS \"decimals!\",\n tokens.usd_price AS \"usd_price?\"\n FROM\n storage\n INNER JOIN tokens ON storage.address = tokens.l2_address\n OR (\n storage.address = $2\n AND tokens.l2_address = $3\n )\n WHERE\n storage.hashed_key = ANY ($1)\n AND storage.value != $4\n " - }, - "1d6b698b241cb6c5efd070a98165f6760cfeac185330d1d9c5cdb5b383ed8ed4": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Text", - "Text", - "Text", - "Text", - "Bool", - "Text", - "Bytea", - "Bool" - ] - } - }, - "query": "\n INSERT INTO\n contract_verification_requests (\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, 'queued', NOW(), NOW())\n RETURNING\n id\n " - }, - "1dcb3afb0c1947f92981f61d95c099c4591ce3f8d51f3df99db0165e086f96af": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n bytecode_hash = $1\n " - }, - "1e54aebf94d27244638f04d1d35a5a088ceebfef0228701fcbed8255b74b1050": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number = $1\n ORDER BY\n index_in_block\n " - }, - "1ea37ef1c3df72e5e9c50cfa1675fc7f60618209d0132e7937a1347b7e94b212": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n 1\n " - }, - "1ed2d7e5e98b15420a21650809d710ce910d0c9138d85cb55e16459c757dea03": { - "describe": { - "columns": [ - { - "name": "protocol_version", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n protocol_version\n FROM\n l1_batches\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "1f25016c41169aa4ab14db2faf7b2d0413d0f89c309de4b31254c309116ea60c": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Varchar", - "Varchar", - "Int4" - ] - } - }, - "query": "\n UPDATE tokens\n SET\n token_list_name = $2,\n token_list_symbol = $3,\n token_list_decimals = $4,\n well_known = TRUE,\n updated_at = NOW()\n WHERE\n l1_address = $1\n " - }, - "1f46524410ce0f193dc6547499bde995ddddc621ee2149f08f905af2d8aadd03": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "Int4Array", - "ByteaArray", - "ByteaArray", - "NumericArray", - "NumericArray", - "NumericArray", - "NumericArray", - "Int4Array", - "Int4Array", - "VarcharArray", - "NumericArray", - "JsonbArray", - "ByteaArray", - "JsonbArray", - "Int8Array", - "NumericArray", - "ByteaArray", - "ByteaArray", - "ByteaArray", - "Int8" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n hash = data_table.hash,\n signature = data_table.signature,\n gas_limit = data_table.gas_limit,\n max_fee_per_gas = data_table.max_fee_per_gas,\n max_priority_fee_per_gas = data_table.max_priority_fee_per_gas,\n gas_per_pubdata_limit = data_table.gas_per_pubdata_limit,\n input = data_table.input,\n data = data_table.data,\n tx_format = data_table.tx_format,\n miniblock_number = $21,\n index_in_block = data_table.index_in_block,\n error = NULLIF(data_table.error, ''),\n effective_gas_price = data_table.effective_gas_price,\n execution_info = data_table.new_execution_info,\n refunded_gas = data_table.refunded_gas,\n value = data_table.value,\n contract_address = data_table.contract_address,\n paymaster = data_table.paymaster,\n paymaster_input = data_table.paymaster_input,\n in_mempool = FALSE,\n updated_at = NOW()\n FROM\n (\n SELECT\n data_table_temp.*\n FROM\n (\n SELECT\n UNNEST($1::bytea[]) AS initiator_address,\n UNNEST($2::INT[]) AS nonce,\n UNNEST($3::bytea[]) AS hash,\n UNNEST($4::bytea[]) AS signature,\n UNNEST($5::NUMERIC[]) AS gas_limit,\n UNNEST($6::NUMERIC[]) AS max_fee_per_gas,\n UNNEST($7::NUMERIC[]) AS max_priority_fee_per_gas,\n UNNEST($8::NUMERIC[]) AS gas_per_pubdata_limit,\n UNNEST($9::INT[]) AS tx_format,\n UNNEST($10::INTEGER[]) AS index_in_block,\n UNNEST($11::VARCHAR[]) AS error,\n UNNEST($12::NUMERIC[]) AS effective_gas_price,\n UNNEST($13::jsonb[]) AS new_execution_info,\n UNNEST($14::bytea[]) AS input,\n UNNEST($15::jsonb[]) AS data,\n UNNEST($16::BIGINT[]) AS refunded_gas,\n UNNEST($17::NUMERIC[]) AS value,\n UNNEST($18::bytea[]) AS contract_address,\n UNNEST($19::bytea[]) AS paymaster,\n UNNEST($20::bytea[]) AS paymaster_input\n ) AS data_table_temp\n JOIN transactions ON transactions.initiator_address = data_table_temp.initiator_address\n AND transactions.nonce = data_table_temp.nonce\n ORDER BY\n transactions.hash\n ) AS data_table\n WHERE\n transactions.initiator_address = data_table.initiator_address\n AND transactions.nonce = data_table.nonce\n " - }, - "1f75f2d88c1d2496e48b02f374e492cf2545944291dd0d42b937c0d0c7eefd47": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "l1_tx_count", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "root_hash?", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "commit_tx_hash?", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "committed_at?", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "prove_tx_hash?", - "ordinal": 7, - "type_info": "Text" - }, - { - "name": "proven_at?", - "ordinal": 8, - "type_info": "Timestamp" - }, - { - "name": "execute_tx_hash?", - "ordinal": 9, - "type_info": "Text" - }, - { - "name": "executed_at?", - "ordinal": 10, - "type_info": "Timestamp" - }, - { - "name": "l1_gas_price", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 13, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 14, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - true, - false, - true, - false, - true, - false, - true, - false, - false, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n l1_batches.number,\n l1_batches.timestamp,\n l1_batches.l1_tx_count,\n l1_batches.l2_tx_count,\n l1_batches.hash AS \"root_hash?\",\n commit_tx.tx_hash AS \"commit_tx_hash?\",\n commit_tx.confirmed_at AS \"committed_at?\",\n prove_tx.tx_hash AS \"prove_tx_hash?\",\n prove_tx.confirmed_at AS \"proven_at?\",\n execute_tx.tx_hash AS \"execute_tx_hash?\",\n execute_tx.confirmed_at AS \"executed_at?\",\n l1_batches.l1_gas_price,\n l1_batches.l2_fair_gas_price,\n l1_batches.bootloader_code_hash,\n l1_batches.default_aa_code_hash\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n l1_batches.number = $1\n " - }, - "2003dcf7bc807c7d345368538accd9b0128f82306e27e4c7258116082a54ab95": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "received_at", - "ordinal": 1, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Timestamp", - "Int8" - ] - } - }, - "query": "\n SELECT\n transactions.hash,\n transactions.received_at\n FROM\n transactions\n LEFT JOIN miniblocks ON miniblocks.number = miniblock_number\n WHERE\n received_at > $1\n ORDER BY\n received_at ASC\n LIMIT\n $2\n " - }, - "2028ba507f3ccd474f0261e571eb19a3a7feec950cb3e503588cf55d954a493a": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n miniblock_number <= $1\n " - }, - "20f84f9ec21459d8c7ad53241758eeab159533211d2ddbef41e6ff0ba937d04a": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n skip_proof = TRUE\n WHERE\n number = $1\n " - }, - "23be43bf705d679ca751c89353716065fcad42c6b621efb3a135a16b477dcfd9": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "nonce", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "raw_tx", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "contract_address", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "tx_type", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "gas_used", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "has_failed", - "ordinal": 8, - "type_info": "Bool" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "confirmed_eth_tx_history_id", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "predicted_gas_cost", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false, - false, - false, - true, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n confirmed_eth_tx_history_id IS NULL\n AND id <= (\n SELECT\n COALESCE(MAX(eth_tx_id), 0)\n FROM\n eth_txs_history\n WHERE\n sent_at_block IS NOT NULL\n )\n ORDER BY\n id\n " - }, - "245dc5bb82cc82df38e4440a7746ca08324bc86a72e4ea85c9c7962a6c8c9e30": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n eth_prove_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n " - }, - "24722ee4ced7f03e60b1b5ecaaa5234d536b064951a67d826ac49b7a3a095a1a": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "index", - "ordinal": 1, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n hashed_key,\n INDEX\n FROM\n initial_writes\n WHERE\n l1_batch_number = $1\n ORDER BY\n INDEX\n " - }, - "249cb862d44196cb6dc3945e907717b0dd3cec64b0b29f59b273f1c6952e01da": { - "describe": { - "columns": [ - { - "name": "bytecode_hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n bytecode_hash\n FROM\n factory_deps\n WHERE\n miniblock_number > $1\n " - }, - "25aad4298d2459ef5aea7c4ea82eda1da000848ed4abf309b68989da33e1ce5a": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number!", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 2, - "type_info": "Int8" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "root_hash?", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "commit_tx_hash?", - "ordinal": 6, - "type_info": "Text" - }, - { - "name": "committed_at?", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "prove_tx_hash?", - "ordinal": 8, - "type_info": "Text" - }, - { - "name": "proven_at?", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "execute_tx_hash?", - "ordinal": 10, - "type_info": "Text" - }, - { - "name": "executed_at?", - "ordinal": 11, - "type_info": "Timestamp" - }, - { - "name": "l1_gas_price", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 14, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 15, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 16, - "type_info": "Int4" - }, - { - "name": "fee_account_address?", - "ordinal": 17, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - null, - false, - false, - false, - false, - false, - true, - false, - true, - false, - true, - false, - false, - true, - true, - true, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n miniblocks.number,\n COALESCE(\n miniblocks.l1_batch_number,\n (\n SELECT\n (MAX(number) + 1)\n FROM\n l1_batches\n )\n ) AS \"l1_batch_number!\",\n miniblocks.timestamp,\n miniblocks.l1_tx_count,\n miniblocks.l2_tx_count,\n miniblocks.hash AS \"root_hash?\",\n commit_tx.tx_hash AS \"commit_tx_hash?\",\n commit_tx.confirmed_at AS \"committed_at?\",\n prove_tx.tx_hash AS \"prove_tx_hash?\",\n prove_tx.confirmed_at AS \"proven_at?\",\n execute_tx.tx_hash AS \"execute_tx_hash?\",\n execute_tx.confirmed_at AS \"executed_at?\",\n miniblocks.l1_gas_price,\n miniblocks.l2_fair_gas_price,\n miniblocks.bootloader_code_hash,\n miniblocks.default_aa_code_hash,\n miniblocks.protocol_version,\n l1_batches.fee_account_address AS \"fee_account_address?\"\n FROM\n miniblocks\n LEFT JOIN l1_batches ON miniblocks.l1_batch_number = l1_batches.number\n LEFT JOIN eth_txs_history AS commit_tx ON (\n l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id\n AND commit_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS prove_tx ON (\n l1_batches.eth_prove_tx_id = prove_tx.eth_tx_id\n AND prove_tx.confirmed_at IS NOT NULL\n )\n LEFT JOIN eth_txs_history AS execute_tx ON (\n l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id\n AND execute_tx.confirmed_at IS NOT NULL\n )\n WHERE\n miniblocks.number = $1\n " - }, - "26cb272c2a46a267c47681e0f1f07997b7e24682da56f84d812da2b9aeb14ca2": { - "describe": { - "columns": [ - { - "name": "miniblock_number!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "index_in_block!", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "l1_batch_tx_index!", - "ordinal": 3, - "type_info": "Int4" - } - ], - "nullable": [ - true, - false, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n miniblock_number AS \"miniblock_number!\",\n hash,\n index_in_block AS \"index_in_block!\",\n l1_batch_tx_index AS \"l1_batch_tx_index!\"\n FROM\n transactions\n WHERE\n l1_batch_number = $1\n ORDER BY\n miniblock_number,\n index_in_block\n " - }, - "26e0b7eb1871d94ddc98254fece6381a9c4165e2727542eaeef3bbedd13a4f20": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE proof_generation_details\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "2737fea02599cdc163854b1395c42d4ef93ca238fd2fbc9155e6d012d0d1e113": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Varchar", - "Bytea" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n error = $1,\n updated_at = NOW()\n WHERE\n hash = $2\n " - }, - "2757b30c4641a346eb0226c706223efc18e51e6d4092188e081f4fafe92fe0ef": { - "describe": { - "columns": [ - { - "name": "bootloader_code_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "default_account_code_hash", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "id", - "ordinal": 2, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n bootloader_code_hash,\n default_account_code_hash,\n id\n FROM\n protocol_versions\n WHERE\n timestamp <= $1\n ORDER BY\n id DESC\n LIMIT\n 1\n " - }, - "280cf015e40353e2833c0a70b77095596297be0d728a0aa2d9b180fb72de222b": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n basic_witness_input_producer_jobs\n WHERE\n l1_batch_number = $1\n " - }, - "293258ecb299be5f5e81696d14883f115cd97586bd795ee31f58fc14e56d58cb": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM events\n WHERE\n miniblock_number > $1\n " - }, - "2955e976281f9cbd98b7378c5ab52964b268b93c32fd280c49bf9f932884300d": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n " - }, - "2a2469109033ba08591db3647b73595fe783b7b894748d07fed9735c58fb28fb": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number\n FROM\n miniblocks\n WHERE\n consensus IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "2b626262c8003817ee02978f77452554ccfb5b83f00efdc12bed0f60ef439785": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8", - "Int2", - "Int2", - "Int4" - ] - } - }, - "query": "\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n l1_batch_number = $1\n AND circuit_id = $2\n AND aggregation_round = $3\n AND depth = $4\n AND status = 'successful'\n ORDER BY\n sequence_number ASC;\n " - }, - "2c827c1c3cfa3552b90d4746c5df45d57f1f8b2558fdb374bf02e84d3c825a23": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n miniblocks\n " - }, - "2d0c2e9ec4187641baef8a33229bffc78d92adb3c1e3ca60b12163e38c67047e": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n contracts_verification_info\n WHERE\n address = $1\n " - }, - "2d1e0f2e043c193052c9cc20f9efeb5f094160627bc09db4bda2dda9a8c11c44": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Jsonb" - ] - } - }, - "query": "\n INSERT INTO\n contracts_verification_info (address, verification_info)\n VALUES\n ($1, $2)\n ON CONFLICT (address) DO\n UPDATE\n SET\n verification_info = $2\n " - }, - "2d31fcce581975a82d6156b52e35fb7a093b73727f75e0cb7db9cea480c95f5c": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n id IN (\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'in_gpu_proof'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n id,\n status,\n attempts\n " - }, - "2d862097cfae49a1fb28ec0a05176085385c3a79d72f49669b4215a9454323c2": { - "describe": { - "columns": [ - { - "name": "index", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n INDEX\n FROM\n initial_writes\n WHERE\n l1_batch_number <= $1\n ORDER BY\n l1_batch_number DESC,\n INDEX DESC\n LIMIT\n 1;\n " - }, - "2d87b294817859e42258136b1cb78f42a877039094c3d6354928a03dad29451a": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4Array" - ] - } - }, - "query": "\n DELETE FROM storage_logs\n WHERE\n miniblock_number = $1\n AND operation_number != ALL ($2)\n " - }, - "2dd7dbaeb2572404451e78a96f540e73a2778633bbf9d8e591ec912634639af9": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n miniblock_number = $1\n ORDER BY\n index_in_block\n " - }, - "2ddba807ac8ec5260bf92c77073eb89c728357c0744f209090824695a5d35fa3": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n l1_batch_number = NULL,\n miniblock_number = NULL,\n error = NULL,\n index_in_block = NULL,\n execution_info = '{}'\n WHERE\n miniblock_number > $1\n RETURNING\n hash\n " - }, - "2e0ea9434195270cc65cdca1f674d6b3b1d15b818974e4e403f4ac418ed40c2c": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Int8", - "Text", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n eth_txs_history (\n eth_tx_id,\n base_fee_per_gas,\n priority_fee_per_gas,\n tx_hash,\n signed_raw_tx,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW(), NOW())\n ON CONFLICT (tx_hash) DO NOTHING\n RETURNING\n id\n " - }, - "2e5b9ae1b81b0abfe7a962c93b3119a0a60dc9804175b2baf8b45939c74bd583": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "TextArray", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n compiler_versions (VERSION, compiler, created_at, updated_at)\n SELECT\n u.version,\n $2,\n NOW(),\n NOW()\n FROM\n UNNEST($1::TEXT[]) AS u (VERSION)\n ON CONFLICT (VERSION, compiler) DO NOTHING\n " - }, - "2eb25bfcfc1114de825dc4eeb0605d7d1c9e649663f6e9444c4425821d0a5b71": { - "describe": { - "columns": [ - { - "name": "eth_commit_tx_id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n eth_commit_tx_id\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "2eb617f3e34ac5b21f925053a45da2b4afc314a3b3e78b041b44c8a020a0ee12": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n in_mempool = FALSE\n FROM\n UNNEST($1::bytea[]) AS s (address)\n WHERE\n transactions.in_mempool = TRUE\n AND transactions.initiator_address = s.address\n " - }, - "31334f2878b1ac7d828d5bc22d65ef6676b2eac623c0f78634cae9072fe0498a": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n snapshots (\n l1_batch_number,\n storage_logs_filepaths,\n factory_deps_filepath,\n created_at,\n updated_at\n )\n VALUES\n ($1, ARRAY_FILL(''::TEXT, ARRAY[$2::INTEGER]), $3, NOW(), NOW())\n " - }, - "314f7e619a34efa89255a58c89f85d4402ff6005446bbded68c8d3dbca510f37": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int8", - "Bytea", - "Int4", - "Int4", - "Numeric", - "Int8", - "Int8", - "Int8", - "Bytea", - "Bytea", - "Int4", - "Int8" - ] - } - }, - "query": "\n INSERT INTO\n miniblocks (\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n gas_per_pubdata_limit,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13, NOW(), NOW())\n " - }, - "31f12a8c44124bb2ce31889ac5295f3823926f69cb1d54874878e6d6c301bfd8": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n " - }, - "322d919ff1ef4675623a58af2b0e9ebdda648667d48d6b27ddf155f2fe01d77a": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n commitment = $2,\n aux_data_hash = $3,\n updated_at = NOW()\n WHERE\n number = $1\n " - }, - "32792c6aee69cb8c8b928a209a3b04ba5868d1897553df85aac15b169ebb0732": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - } - ] - } - }, - "query": "\n INSERT INTO\n basic_witness_input_producer_jobs (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "33d6be45b246523ad76f9ae512322ff6372f63ecadb504a329499b02e7d3550e": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 1, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id\n FROM\n prover_jobs_fri\n JOIN leaf_aggregation_witness_jobs_fri lawj ON prover_jobs_fri.l1_batch_number = lawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = lawj.circuit_id\n WHERE\n lawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 0\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n lawj.number_of_basic_circuits\n HAVING\n COUNT(*) = lawj.number_of_basic_circuits\n )\n RETURNING\n l1_batch_number,\n circuit_id;\n " - }, - "3490fe0b778a03c73111bf8cbf426b0b3185a231bbf0b8b132a1a95bc157e827": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "index", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n SELECT\n hashed_key,\n l1_batch_number,\n INDEX\n FROM\n initial_writes\n WHERE\n hashed_key = ANY ($1::bytea[])\n " - }, - "35b87a3b7db0af87c6a95e9fe7ef9044ae85b579c7051301b40bd5f94df1f530": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n " - }, - "3671f23665664b8d6acf97e4f697e5afa28d855d87ea2f8c93e79c436749068a": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8", - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n LIMIT\n $3\n " - }, - "367ca58514762ffc26fd906c4c441a21691357494c2f9919bfcbcbb0e42315c2": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n miniblocks\n WHERE\n number = $1\n AND consensus IS NOT NULL\n " - }, - "38a8b00e320b16e99f6ea0e5954e2f7e49cd6600bd3d56cf41795c2c9e082e4c": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n " - }, - "3b0af308b0ce95a13a4eed40834279601234a489f73d843f2f314252ed4cb8b0": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "value!", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n SELECT\n hashed_key,\n value AS \"value!\"\n FROM\n storage\n WHERE\n hashed_key = ANY ($1)\n " - }, - "3b3fbcffd2702047045c2f358e8ac77b63879ab97a32eed8392b48cc46116a28": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n DELETE FROM call_traces\n WHERE\n tx_hash = ANY ($1)\n " - }, - "3b4d5009ec22f54cc7d305aa11d96ec397767a063dc21aa3add974cb9b070361": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "ByteaArray", - "Int8" - ] - } - }, - "query": "\n INSERT INTO\n factory_deps (bytecode_hash, bytecode, miniblock_number, created_at, updated_at)\n SELECT\n u.bytecode_hash,\n u.bytecode,\n $3,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (bytecode_hash, bytecode)\n ON CONFLICT (bytecode_hash) DO NOTHING\n " - }, - "3c1d5f985be7e378211aa339c2c6387f2f3eda07a630503324bd6576dbdf8231": { - "describe": { - "columns": [ - { - "name": "trace", - "ordinal": 0, - "type_info": "Jsonb" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n trace\n FROM\n transaction_traces\n WHERE\n tx_hash = $1\n " - }, - "3c3abbf689fa64c6da7de69fd916769dbb04d3a61cf232892236c974660ffe64": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n " - }, - "3e170eea3a5ea5c7389c15f76c6489745438eae73a07b577aa25bd08adf95354": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Int8", - "Bytea" - ] - } - }, - "query": "\n DELETE FROM tokens\n WHERE\n l2_address IN (\n SELECT\n SUBSTRING(key, 12, 20)\n FROM\n storage_logs\n WHERE\n storage_logs.address = $1\n AND miniblock_number > $2\n AND NOT EXISTS (\n SELECT\n 1\n FROM\n storage_logs AS s\n WHERE\n s.hashed_key = storage_logs.hashed_key\n AND (s.miniblock_number, s.operation_number) >= (storage_logs.miniblock_number, storage_logs.operation_number)\n AND s.value = $3\n )\n )\n " - }, - "3ec365c5c81f4678a905ae5bbd48b87ead36f593488437c6f67da629ca81e4fa": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n l1_batch_number = $1\n AND status != 'successful'\n AND status != 'in_progress'\n " - }, - "41c9f45d6eb727aafad0d8c18024cee5c602d275bb812022cc8fdabf0a60e151": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "eth_tx_id", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "tx_hash", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "base_fee_per_gas", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "priority_fee_per_gas", - "ordinal": 4, - "type_info": "Int8" - }, - { - "name": "signed_raw_tx", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 6, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n eth_txs_history.id,\n eth_txs_history.eth_tx_id,\n eth_txs_history.tx_hash,\n eth_txs_history.base_fee_per_gas,\n eth_txs_history.priority_fee_per_gas,\n eth_txs_history.signed_raw_tx,\n eth_txs.nonce\n FROM\n eth_txs_history\n JOIN eth_txs ON eth_txs.id = eth_txs_history.eth_tx_id\n WHERE\n eth_txs_history.sent_at_block IS NULL\n AND eth_txs.confirmed_eth_tx_history_id IS NULL\n ORDER BY\n eth_txs_history.id DESC\n " - }, - "43c7e352d09f69de1a182196aea4de79b67833f17d252b5b0e8e00cd6e75b5c1": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MIN(number) AS \"number\"\n FROM\n l1_batches\n " - }, - "46c4696fff5a4b8cc5cb46b05645da82065836fe17687ffad04126a6a8b2b27c": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Time", - "Int8" - ] - } - }, - "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n id = $2\n " - }, - "47c2f23d9209d155f3f32fd21ef7931a02fe5ffaf2c4dc2f1e7a48c0e932c060": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_root_hash", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "miniblock_number", - "ordinal": 2, - "type_info": "Int8" - }, - { - "name": "miniblock_root_hash", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "last_finished_chunk_id", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "total_chunk_count", - "ordinal": 5, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false, - false, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number,\n l1_batch_root_hash,\n miniblock_number,\n miniblock_root_hash,\n last_finished_chunk_id,\n total_chunk_count\n FROM\n snapshot_recovery\n " - }, - "481d3cdb6c9a90843b240dba84377cb8f1340b483faedbbc2b71055aa5451cae": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n is_finished = TRUE\n " - }, - "4cdc90ed409b37b3c1c57bbcca9f82918afa1b0ac410325e4d00cd1c4fdd1e8b": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_tx_count", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 4, - "type_info": "Bool" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 6, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "bloom", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 9, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 10, - "type_info": "Jsonb" - }, - { - "name": "base_fee_per_gas", - "ordinal": 11, - "type_info": "Numeric" - }, - { - "name": "l1_gas_price", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 14, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 15, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 16, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 17, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "pubdata_input", - "ordinal": 19, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - false, - true - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n pubdata_input\n FROM\n l1_batches\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "4d263992ed6d5abbd7d3ca43af9d772d8801b0ae673b7173ae08a1fa6cbf67b2": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "aggregation_round", - "ordinal": 3, - "type_info": "Int2" - }, - { - "name": "sequence_number", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "depth", - "ordinal": 5, - "type_info": "Int4" - }, - { - "name": "is_node_final_proof", - "ordinal": 6, - "type_info": "Bool" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n aggregation_round DESC,\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n " - }, - "4d50dabc25d392e6b9d0dbe0e386ea7ef2c1178b1b0394a17442185b79f2d77d": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Text" - ] - } - }, - "query": "SELECT eth_txs.id FROM eth_txs_history JOIN eth_txs ON eth_txs.confirmed_eth_tx_history_id = eth_txs_history.id WHERE eth_txs_history.tx_hash = $1" - }, - "4d84bb4e180b7267bee5e3c1f83c6d47e8e1b4b5124c82c1f35d405204fcf783": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "eth_tx_id", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "tx_hash", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 3, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 4, - "type_info": "Timestamp" - }, - { - "name": "base_fee_per_gas", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "priority_fee_per_gas", - "ordinal": 6, - "type_info": "Int8" - }, - { - "name": "confirmed_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "signed_raw_tx", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "sent_at", - "ordinal": 10, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n ORDER BY\n created_at DESC\n " - }, - "4d92a133a36afd682a84fbfd75aafca34d61347e0e2e29fb07ca3d1b8b1f309c": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n prover_fri_protocol_versions (\n id,\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW())\n ON CONFLICT (id) DO NOTHING\n " - }, - "525123d4ec2b427f1c171f30d0937d8d542b4f14cf560972c005ab3cc13d1f63": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n hash\n FROM\n miniblocks\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n " - }, - "532a80b0873871896dd318beba5ec427a099492905a1feee512dc43f39d10047": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int4" - ] - } - }, - "query": "\n UPDATE eth_txs_history\n SET\n sent_at_block = $2,\n sent_at = NOW()\n WHERE\n id = $1\n AND sent_at_block IS NULL\n " - }, - "534822a226068cde83ad8c30b569a8f447824a5ab466bb6eea1710e8aeaa2c56": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "53c04fd528752c0e0ef7ffa1f68a7ea81d8d10c76bbae540013667e13230e2ea": { - "describe": { - "columns": [ - { - "name": "fee_account_address", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n fee_account_address\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "53f78fdee39b113d2f55f6f951bd94f28b7b2b60d551d552a9b0bab1f1791e39": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n leaf_aggregation_witness_jobs_fri\n WHERE\n id = $1\n " - }, - "5503575d9377785894de6cf6139a8d4768c6a803a1a90889e5a1b8254c315231": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Text" - ] - } - }, - "query": "INSERT INTO eth_txs (raw_tx, nonce, tx_type, contract_address, predicted_gas_cost, created_at, updated_at) VALUES ('\\x00', 0, $1, '', 0, now(), now()) RETURNING id" - }, - "556f9b9e82d3a9399660dfa4bbf252f26335699a4e7f0347d7e894320245271d": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Jsonb" - ] - } - }, - "query": "\n INSERT INTO\n events_queue (l1_batch_number, serialized_events_queue)\n VALUES\n ($1, $2)\n " - }, - "55b0b4c569c0aaf9741afc85400ecd50a04799ffd36be0e17c56f47fcdbc8f60": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM l1_batches\n WHERE\n number > $1\n " - }, - "5659480e5d79dab3399e35539b240e7eb9f598999c28015a504605f88bf84b33": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "nonce", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "raw_tx", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "contract_address", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "tx_type", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "gas_used", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "has_failed", - "ordinal": 8, - "type_info": "Bool" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "confirmed_eth_tx_history_id", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "predicted_gas_cost", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false, - false, - false, - true, - true, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n id > (\n SELECT\n COALESCE(MAX(eth_tx_id), 0)\n FROM\n eth_txs_history\n )\n ORDER BY\n id\n LIMIT\n $1\n " - }, - "5821f1446983260168cec366af26009503182c300877e74a8539f231050e6f85": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE witness_inputs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "58aed39245c72d231b268ce83105bb2036d21f60d4c6934f9145730ac35c04de": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n proof_generation_details\n WHERE\n status = 'ready_to_be_proven'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " - }, - "59cb0dd78fadc121e2b1ebbc8a063f089c91aead2bc9abb284697e65840f1e8f": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Numeric", - "Timestamp" - ] - } - }, - "query": "\n UPDATE tokens\n SET\n usd_price = $2,\n usd_price_updated_at = $3,\n updated_at = NOW()\n WHERE\n l1_address = $1\n " - }, - "5aaed2a975042cc9b7b9d88e5fd5db07667280abef27cc73159d2fd9c95b209b": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n $1\n " - }, - "5c39f043c9b36693b0a845eb36549374a2d931e62615bc7e6ecd0af957b42a13": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "base_fee_per_gas", - "ordinal": 5, - "type_info": "Numeric" - }, - { - "name": "l1_gas_price", - "ordinal": 6, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 7, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "virtual_blocks", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "5d493cbce749cc5b56d4069423597b16599abaf51df0f19effe1a536376cf6a6": { - "describe": { - "columns": [ - { - "name": "bootloader_code_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "default_account_code_hash", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n bootloader_code_hash,\n default_account_code_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n " - }, - "5e781f84ec41edd0941fa84de837effac442434c6e734d977e6682a7484abe7f": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n " - }, - "5f6885b5457aaa78e10917ae5b8cd0bc0e8923a6bae64f22f09242766835ee0c": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "contract_address", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "source_code", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "contract_name", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "zk_compiler_version", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "compiler_version", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "optimization_used", - "ordinal": 6, - "type_info": "Bool" - }, - { - "name": "optimizer_mode", - "ordinal": 7, - "type_info": "Text" - }, - { - "name": "constructor_arguments", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "is_system", - "ordinal": 9, - "type_info": "Bool" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - true, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n id,\n contract_address,\n source_code,\n contract_name,\n zk_compiler_version,\n compiler_version,\n optimization_used,\n optimizer_mode,\n constructor_arguments,\n is_system\n FROM\n contract_verification_requests\n WHERE\n status = 'successful'\n ORDER BY\n id\n " - }, - "5f8fc05ae782846898295d210dd3d55ff2b1510868dfe80d14fffa3f5ff07b83": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n predicted_commit_gas_cost = $2,\n updated_at = NOW()\n WHERE\n number = $1\n " - }, - "61b2b858d4636809c21838635aa52aeb5f06c26f68d131dd242f6ed68816c513": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int2" - ] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n prover_jobs_fri\n WHERE\n status <> 'skipped'\n AND status <> 'successful'\n AND aggregation_round = $1\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " - }, - "61bc330d6d1b5fddec78342c1b0f00e82b0b3ad9ae36bf4fe44d7e85b74c6f49": { - "describe": { - "columns": [ - { - "name": "op_id", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(priority_op_id) AS \"op_id\"\n FROM\n transactions\n WHERE\n is_priority = TRUE\n AND miniblock_number IS NOT NULL\n " - }, - "6692ff6c0fbb2fc94f5cd2837a43ce80f9b2b27758651ccfc09df61a4ae8a363": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "nonce", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "raw_tx", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "contract_address", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "tx_type", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "gas_used", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "has_failed", - "ordinal": 8, - "type_info": "Bool" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "confirmed_eth_tx_history_id", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "predicted_gas_cost", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false, - false, - false, - true, - true, - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n eth_txs\n WHERE\n id = $1\n " - }, - "66e012ce974c38d9fe84cfc7eb28927f9e976319a305e0928ff366d535a97104": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "nonce", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "raw_tx", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "contract_address", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "tx_type", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "gas_used", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "has_failed", - "ordinal": 8, - "type_info": "Bool" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "confirmed_eth_tx_history_id", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "predicted_gas_cost", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - true, - false, - false, - false, - true, - true, - false - ], - "parameters": { - "Left": [ - "Bytea", - "Int8", - "Text", - "Text", - "Int8" - ] - } - }, - "query": "\n INSERT INTO\n eth_txs (\n raw_tx,\n nonce,\n tx_type,\n contract_address,\n predicted_gas_cost,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, NOW(), NOW())\n RETURNING\n *\n " - }, - "68936a53e5b80576f3f341523e6843eb48b5e26ee92cd8476f50251e8c32610d": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Bytea", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n l1_batches\n WHERE\n number = $1\n AND hash = $2\n AND merkle_root_hash = $3\n AND parent_hash = $4\n AND l2_l1_merkle_root = $5\n " - }, - "68c891ee9d71cffe709731f2804b734d5d255e36e48668b3bfc25a0f86ea52e7": { - "describe": { - "columns": [ - { - "name": "is_replaced!", - "ordinal": 0, - "type_info": "Bool" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Int8", - "Bytea", - "Numeric", - "Numeric", - "Numeric", - "Numeric", - "Bytea", - "Jsonb", - "Int4", - "Bytea", - "Numeric", - "Bytea", - "Bytea", - "Int8", - "Int4", - "Int4", - "Timestamp" - ] - } - }, - "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n nonce,\n signature,\n gas_limit,\n max_fee_per_gas,\n max_priority_fee_per_gas,\n gas_per_pubdata_limit,\n input,\n data,\n tx_format,\n contract_address,\n value,\n paymaster,\n paymaster_input,\n execution_info,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n FALSE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n $19,\n NOW(),\n NOW()\n )\n ON CONFLICT (initiator_address, nonce) DO\n UPDATE\n SET\n hash = $1,\n signature = $4,\n gas_limit = $5,\n max_fee_per_gas = $6,\n max_priority_fee_per_gas = $7,\n gas_per_pubdata_limit = $8,\n input = $9,\n data = $10,\n tx_format = $11,\n contract_address = $12,\n value = $13,\n paymaster = $14,\n paymaster_input = $15,\n execution_info = JSONB_BUILD_OBJECT('gas_used', $16::BIGINT, 'storage_writes', $17::INT, 'contracts_used', $18::INT),\n in_mempool = FALSE,\n received_at = $19,\n created_at = NOW(),\n updated_at = NOW(),\n error = NULL\n WHERE\n transactions.is_priority = FALSE\n AND transactions.miniblock_number IS NULL\n RETURNING\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n transactions.initiator_address = $2\n AND transactions.nonce = $3\n ) IS NOT NULL AS \"is_replaced!\"\n " - }, - "6ae2ed34230beae0e86c584e293e7ee767e4c98706246eb113498c0f817f5f38": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int4", - "Int2", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n gpu_prover_queue_fri (\n instance_host,\n instance_port,\n instance_status,\n specialized_prover_group_id,\n zone,\n created_at,\n updated_at\n )\n VALUES\n (CAST($1::TEXT AS inet), $2, 'available', $3, $4, NOW(), NOW())\n ON CONFLICT (instance_host, instance_port, zone) DO\n UPDATE\n SET\n instance_status = 'available',\n specialized_prover_group_id = $3,\n zone = $4,\n updated_at = NOW()\n " - }, - "6b327df84d2b3b31d02db35fd5d91a8d67abcdb743a619ed0d1b9c16206a3c20": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n DELETE FROM eth_txs\n WHERE\n id >= (\n SELECT\n MIN(id)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n )\n " - }, - "6bd3094be764e6378fe52b5bb533260b49ce42daaf9dbe8075daf0a8e0ad9914": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n DELETE FROM basic_witness_input_producer_jobs\n " - }, - "6c0d03b1fbe6f47546bc34c6b2eab01cb2c55bf86d2c8c99abb1b7ca21cf75c0": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n UPDATE miniblocks\n SET\n protocol_version = $1\n WHERE\n l1_batch_number IS NULL\n " - }, - "6ccb3beec0624153ef2e7bff61ba896e34b757421fca9682aecb3a98b54695a6": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "base_fee_per_gas", - "ordinal": 5, - "type_info": "Numeric" - }, - { - "name": "l1_gas_price", - "ordinal": 6, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 7, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "virtual_blocks", - "ordinal": 11, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n hash,\n l1_tx_count,\n l2_tx_count,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n virtual_blocks\n FROM\n miniblocks\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "708b2b3e40887e6d8d2d7aa20448a58479487686d774e6b2b1391347bdafe06d": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n hash\n FROM\n miniblocks\n WHERE\n number >= $1\n ORDER BY\n number ASC\n LIMIT\n $2\n " - }, - "70979db81f473950b2fae7816dbad7fe3464f2619cee2d583accaa829aa12b94": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4", - "Int4", - "Int8", - "Bool", - "Bytea", - "ByteaArray", - "ByteaArray", - "Bytea", - "ByteaArray", - "Int8", - "Int8", - "Int8", - "Jsonb", - "Jsonb", - "Numeric", - "Int8", - "Int8", - "Bytea", - "Bytea", - "Int4", - "ByteaArray", - "Int8Array", - "Bytea", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n l1_batches (\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n predicted_commit_gas_cost,\n predicted_prove_gas_cost,\n predicted_execute_gas_cost,\n initial_bootloader_heap_content,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n storage_refunds,\n pubdata_input,\n predicted_circuits,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n $17,\n $18,\n $19,\n $20,\n $21,\n $22,\n $23,\n $24,\n $25,\n NOW(),\n NOW()\n )\n " - }, - "72a4f50355324cce85ebaef9fa32826095e9290f0c1157094bd0c44e06012e42": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n hash = $1\n " - }, - "72ff9df79e78129cb96d14ece0198129b44534062f524823666ed432d2fcd345": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n VACUUM storage_logs\n " - }, - "73c4bf1e35d49faaab9f7828e80f396f9d193615d70184d4327378a7fc8a5665": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - }, - "Int8", - "Time", - "Text" - ] - } - }, - "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $3,\n input_blob_url = $4\n WHERE\n l1_batch_number = $2\n " - }, - "7560ba61643a8ec8eeefbe6034226313c255ce356a9a4e25c098484d3129c914": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n DELETE FROM eth_txs_history\n WHERE\n id = $1\n " - }, - "759b80414b5bcbfe03a0e1e15b37f92c4cfad9313b1461e12242d9becb59e0b0": { - "describe": { - "columns": [ - { - "name": "max?", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n MAX(operation_number) AS \"max?\"\n FROM\n storage_logs\n WHERE\n miniblock_number = $1\n " - }, - "75a3cf6f502ebb1a0e92b672dc6ce56b53cc4ca0a8c6ee7cac1b9a5863000be3": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n eth_commit_tx_id IS NOT NULL\n AND eth_prove_tx_id IS NULL\n ORDER BY\n number\n LIMIT\n $1\n " - }, - "75f6eaa518e7840374c4e44b0788bf92c7f2c55386c8208e3a82b30456abd5b4": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "merkle_tree_paths_blob_url", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "status", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "error", - "ordinal": 4, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 5, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "processing_started_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 8, - "type_info": "Time" - }, - { - "name": "is_blob_cleaned", - "ordinal": 9, - "type_info": "Bool" - }, - { - "name": "protocol_version", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "picked_by", - "ordinal": 11, - "type_info": "Text" - } - ], - "nullable": [ - false, - true, - false, - false, - true, - false, - false, - true, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8", - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $3\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number <= $1\n AND status = 'queued'\n AND protocol_version = ANY ($2)\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n witness_inputs_fri.*\n " - }, - "75fa24c29dc312cbfa89bf1f4a04a42b4ead6964edd17bfcacb4a828492bba60": { - "describe": { - "columns": [ - { - "name": "state!", - "ordinal": 0, - "type_info": "Jsonb" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n state AS \"state!\"\n FROM\n consensus_replica_state\n WHERE\n fake_key\n " - }, - "76cb9ad97b70d584b19af194576dcf2324f380932698386aa8f9751b1fa24a7b": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "ByteaArray" - ] - } - }, - "query": "\n INSERT INTO\n call_traces (tx_hash, call_trace)\n SELECT\n u.tx_hash,\n u.call_trace\n FROM\n UNNEST($1::bytea[], $2::bytea[]) AS u (tx_hash, call_trace)\n " - }, - "77a43830ca31eac85a3c03d87696bf94a013e49bf50ce23f4de4968781df0796": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n hash = $1\n WHERE\n number = $2\n " - }, - "77b35855fbb989f6314469b419726dc7bb98e0f7feaf14656307e20bd2bb0b6c": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Jsonb" - ] - } - }, - "query": "\n INSERT INTO\n consensus_replica_state (fake_key, state)\n VALUES\n (TRUE, $1)\n ON CONFLICT (fake_key) DO\n UPDATE\n SET\n state = excluded.state\n " - }, - "78978c19282961c5b3dc06352b41caa4cca66d6ad74b2cd1a34ea5f7bc1e6909": { - "describe": { - "columns": [ - { - "name": "tx_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "call_trace", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n call_traces\n WHERE\n tx_hash = $1\n " - }, - "7a2145e2234a7896031bbc1ce82715e903f3b399886c2c73e838bd924fed6776": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8", - "Int2", - "Int4", - "Int4" - ] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n aggregations_url = $1,\n number_of_dependent_jobs = $5,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n AND circuit_id = $3\n AND depth = $4\n " - }, - "7a8fffe8d4e3085e00c98f770d250d625f057acf1440b6550375ce5509a816a6": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "closed_form_inputs_blob_url", - "ordinal": 3, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 4, - "type_info": "Int2" - }, - { - "name": "status", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "error", - "ordinal": 6, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 8, - "type_info": "Timestamp" - }, - { - "name": "processing_started_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 10, - "type_info": "Time" - }, - { - "name": "is_blob_cleaned", - "ordinal": 11, - "type_info": "Bool" - }, - { - "name": "number_of_basic_circuits", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "protocol_version", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "picked_by", - "ordinal": 14, - "type_info": "Text" - } - ], - "nullable": [ - false, - false, - false, - true, - false, - false, - true, - false, - false, - true, - true, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n id = (\n SELECT\n id\n FROM\n leaf_aggregation_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC,\n id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n leaf_aggregation_witness_jobs_fri.*\n " - }, - "7fccc28bd829bce334f37197ee6b139e943f3ad2a41387b610606a42b7f03283": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Numeric", - "Numeric", - "Numeric", - "Jsonb", - "Int4", - "Bytea", - "Int4", - "Numeric", - "Bytea", - "Bytea", - "Int4", - "Numeric", - "Bytea", - "Timestamp" - ] - } - }, - "query": "\n INSERT INTO\n transactions (\n hash,\n is_priority,\n initiator_address,\n gas_limit,\n max_fee_per_gas,\n gas_per_pubdata_limit,\n data,\n upgrade_id,\n contract_address,\n l1_block_number,\n value,\n paymaster,\n paymaster_input,\n tx_format,\n l1_tx_mint,\n l1_tx_refund_recipient,\n received_at,\n created_at,\n updated_at\n )\n VALUES\n (\n $1,\n TRUE,\n $2,\n $3,\n $4,\n $5,\n $6,\n $7,\n $8,\n $9,\n $10,\n $11,\n $12,\n $13,\n $14,\n $15,\n $16,\n NOW(),\n NOW()\n )\n ON CONFLICT (hash) DO NOTHING\n " - }, - "806b82a9effd885ba537a2a1c7d7227120a8279db1875d26ccae5ee0785f46a9": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n node_aggregation_witness_jobs_fri\n WHERE\n id = $1\n " - }, - "8182690d0326b820d23fba49d391578db18c29cdca85b8b6aad86fe2a9bf6bbe": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 1, - "type_info": "Int2" - }, - { - "name": "depth", - "ordinal": 2, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id, depth) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth\n FROM\n prover_jobs_fri\n JOIN node_aggregation_witness_jobs_fri nawj ON prover_jobs_fri.l1_batch_number = nawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = nawj.circuit_id\n AND prover_jobs_fri.depth = nawj.depth\n WHERE\n nawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 2\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth,\n nawj.number_of_dependent_jobs\n HAVING\n COUNT(*) = nawj.number_of_dependent_jobs\n )\n RETURNING\n l1_batch_number,\n circuit_id,\n depth;\n " - }, - "81869cb392e9fcbb71ceaa857af77b39429d56072f63b3530c576fb31d7a56f9": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "ByteaArray", - "ByteaArray", - "ByteaArray", - "ByteaArray" - ] - } - }, - "query": "\n INSERT INTO\n storage (hashed_key, address, key, value, tx_hash, created_at, updated_at)\n SELECT\n u.hashed_key,\n u.address,\n u.key,\n u.value,\n u.tx_hash,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::bytea[], $3::bytea[], $4::bytea[], $5::bytea[]) AS u (hashed_key, address, key, value, tx_hash)\n ON CONFLICT (hashed_key) DO\n UPDATE\n SET\n tx_hash = excluded.tx_hash,\n value = excluded.value,\n updated_at = NOW()\n " - }, - "83a931ceddf34e1c760649d613f534014b9ab9ca7725e14fb17aa050d9f35eb8": { - "describe": { - "columns": [ - { - "name": "base_fee_per_gas", - "ordinal": 0, - "type_info": "Numeric" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n base_fee_per_gas\n FROM\n miniblocks\n WHERE\n number <= $1\n ORDER BY\n number DESC\n LIMIT\n $2\n " - }, - "84c804db9d60a4c1ebbce5e3dcdf03c0aad3ac30d85176e0a4e35f72bbb21b12": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "system_logs", - "ordinal": 35, - "type_info": "ByteaArray" - }, - { - "name": "compressed_state_diffs", - "ordinal": 36, - "type_info": "Bytea" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n system_logs,\n compressed_state_diffs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number = $1\n " - }, - "852aa5fe1c3b2dfe875cd4adf0d19a00c170cf7725d95dd6eb8b753fa5facec8": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8", - "Numeric", - "Numeric", - "Int4" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n in_mempool = TRUE\n FROM\n (\n SELECT\n hash\n FROM\n (\n SELECT\n hash\n FROM\n transactions\n WHERE\n miniblock_number IS NULL\n AND in_mempool = FALSE\n AND error IS NULL\n AND (\n is_priority = TRUE\n OR (\n max_fee_per_gas >= $2\n AND gas_per_pubdata_limit >= $3\n )\n )\n AND tx_format != $4\n ORDER BY\n is_priority DESC,\n priority_op_id,\n received_at\n LIMIT\n $1\n ) AS subquery1\n ORDER BY\n hash\n ) AS subquery2\n WHERE\n transactions.hash = subquery2.hash\n RETURNING\n transactions.*\n " - }, - "8625ca45ce76b8c8633d390e35e0c5f885240d99ea69140a4636b00469d08497": { - "describe": { - "columns": [ - { - "name": "tx_hash", - "ordinal": 0, - "type_info": "Text" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n tx_hash\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n AND confirmed_at IS NOT NULL\n " - }, - "877d20634068170326ab5801b69c70aff49e60b7def3d93b9206e650c259168b": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_execute_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n " - }, - "878c9cdfd69ad8988d049041edd63595237a0c54f67b8c669dfbb4fca32757e4": { - "describe": { - "columns": [ - { - "name": "l2_address", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l2_address\n FROM\n tokens\n " - }, - "88c629334e30bb9f5c81c858aa51af63b86e8da6d908d48998012231e1d66a60": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "virtual_blocks", - "ordinal": 1, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n timestamp,\n virtual_blocks\n FROM\n miniblocks\n WHERE\n number BETWEEN $1 AND $2\n ORDER BY\n number\n " - }, - "8903ba5db3f87851c12da133573b4207b69cc48b4ba648e797211631be612b69": { - "describe": { - "columns": [ - { - "name": "bytecode_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "bytecode", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n bytecode_hash,\n bytecode\n FROM\n factory_deps\n INNER JOIN miniblocks ON miniblocks.number = factory_deps.miniblock_number\n WHERE\n miniblocks.l1_batch_number = $1\n " - }, - "894665c2c467bd1aaeb331b112c567e2667c63a033baa6b427bd8a0898c08bf2": { - "describe": { - "columns": [ - { - "name": "protocol_version", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n protocol_version\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "8a7a57ca3d4d65da3e0877c003902c690c33686c889d318b1d64bdd7fa6374db": { - "describe": { - "columns": [ - { - "name": "l1_block_number", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_block_number\n FROM\n transactions\n WHERE\n priority_op_id IS NOT NULL\n ORDER BY\n priority_op_id DESC\n LIMIT\n 1\n " - }, - "8b9e5d525c026de97c0a732b1adc8dc4bd57e32dfefe1017acba9a15fc14b895": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "value", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "index", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n storage_logs.hashed_key,\n storage_logs.value,\n initial_writes.index\n FROM\n storage_logs\n INNER JOIN initial_writes ON storage_logs.hashed_key = initial_writes.hashed_key\n WHERE\n storage_logs.miniblock_number = $1\n AND storage_logs.hashed_key >= $2::bytea\n AND storage_logs.hashed_key <= $3::bytea\n ORDER BY\n storage_logs.hashed_key\n " - }, - "8f5e89ccadd4ea1da7bfe9793a1cbb724af0f0216433a70f19d784e3f2afbc9f": { - "describe": { - "columns": [ - { - "name": "protocol_version", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n protocol_version\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number = $1\n " - }, - "90f7657bae05c4bad6902c6bfb1b8ba0b771cb45573aca81db254f6bcfc17c77": { - "describe": { - "columns": [ - { - "name": "nonce", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n nonce\n FROM\n eth_txs\n ORDER BY\n id DESC\n LIMIT\n 1\n " - }, - "9334df89c9562d4b35611b8e5ffb17305343df99ebc55f240278b5c4e63f89f5": { - "describe": { - "columns": [ - { - "name": "value", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n value\n FROM\n storage\n WHERE\n hashed_key = $1\n " - }, - "95ea0522a3eff6c0d2d0b1c58fd2767e112b95f4d103c27acd6f7ede108bd300": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4", - "Int4" - ] - } - }, - "query": "\n UPDATE eth_txs\n SET\n gas_used = $1,\n confirmed_eth_tx_history_id = $2\n WHERE\n id = $3\n " - }, - "966dddc881bfe6fd94b56f587424125a2633ddb6abaa129f2b12389140d83c3f": { - "describe": { - "columns": [ - { - "name": "recursion_scheduler_level_vk_hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "recursion_node_level_vk_hash", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "recursion_leaf_level_vk_hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "recursion_circuits_set_vks_hash", - "ordinal": 3, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n recursion_scheduler_level_vk_hash,\n recursion_node_level_vk_hash,\n recursion_leaf_level_vk_hash,\n recursion_circuits_set_vks_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n " - }, - "9955b9215096f781442153518c4f0a9676e26f422506545ccc90b7e8a36c8d47": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "data?", - "ordinal": 1, - "type_info": "Jsonb" - }, - { - "name": "contract_address?", - "ordinal": 2, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - true - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n factory_deps.bytecode,\n transactions.data AS \"data?\",\n transactions.contract_address AS \"contract_address?\"\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n ORDER BY\n miniblock_number DESC,\n operation_number DESC\n LIMIT\n 1\n ) storage_logs\n JOIN factory_deps ON factory_deps.bytecode_hash = storage_logs.value\n LEFT JOIN transactions ON transactions.hash = storage_logs.tx_hash\n WHERE\n storage_logs.value != $2\n " - }, - "995cecd37a5235d1acc2e6fc418d9b6a1a6fe629f9a02c8e33330a0efda64068": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "factory_deps_filepath", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "storage_logs_filepaths", - "ordinal": 2, - "type_info": "TextArray" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number,\n factory_deps_filepath,\n storage_logs_filepaths\n FROM\n snapshots\n ORDER BY\n l1_batch_number DESC\n LIMIT\n 1\n " - }, - "99acb091650478fe0feb367b1d64561347b81f8931cc2addefa907c9aa9355e6": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "recursion_scheduler_level_vk_hash", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "recursion_node_level_vk_hash", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "recursion_leaf_level_vk_hash", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "recursion_circuits_set_vks_hash", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bootloader_code_hash", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "default_account_code_hash", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "verifier_address", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "upgrade_tx_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "created_at", - "ordinal": 10, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - false - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n protocol_versions\n WHERE\n id < $1\n ORDER BY\n id DESC\n LIMIT\n 1\n " - }, - "99d9ee2a0d0450acefa0d9b6c031e30606fddf6631c859ab03819ec476bcf005": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n SELECT\n hashed_key\n FROM\n initial_writes\n WHERE\n hashed_key = ANY ($1)\n " - }, - "99dd6f04e82585d81ac23bc4871578179e6269c6ff36877cedee264067ccdafc": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - }, - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - }, - { - "Custom": { - "kind": { - "Enum": [ - "Queued", - "ManuallySkipped", - "InProgress", - "Successful", - "Failed" - ] - }, - "name": "basic_witness_input_producer_job_status" - } - }, - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE basic_witness_input_producer_jobs\n SET\n status = $1,\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n basic_witness_input_producer_jobs\n WHERE\n status = $2\n OR (\n status = $1\n AND processing_started_at < NOW() - $4::INTERVAL\n )\n OR (\n status = $3\n AND attempts < $5\n )\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n basic_witness_input_producer_jobs.l1_batch_number\n " - }, - "9b90f7a7ffee3cd8439f90a6f79693831e2ab6d6d3c1805df5aa51d76994ec19": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n witness_inputs_fri (\n l1_batch_number,\n merkle_tree_paths_blob_url,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "9c2a5f32c627d3a5c6f1e87b31ce3b0fd67aa1f5f7ea0de673a2fbe1f742db86": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n timestamp\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "9cfcde703a48b110791d2ae1103c9317c01d6e35db3b07d0a31f436e7e3c7c40": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE contract_verification_requests\n SET\n status = 'successful',\n updated_at = NOW()\n WHERE\n id = $1\n " - }, - "9de5acb3de1b96ff8eb62a6324e8e221a8ef9014458cc7f1dbc60c056a0768a0": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int4", - "Text" - ] - } - }, - "query": "\n UPDATE snapshots\n SET\n storage_logs_filepaths[$2] = $3,\n updated_at = NOW()\n WHERE\n l1_batch_number = $1\n " - }, - "9ef2f43e6201cc00a0e1425a666a36532fee1450733849852dfd20e18ded1f03": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "a0e2b2c034cc5f668f0b3d43b94d2e2326d7ace079b095def52723a45b65d3f3": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "a2d02b71e3dcc29a2c0c20b44392cfbaf09164aecfa5eed8d7142518ad96abea": { - "describe": { - "columns": [ - { - "name": "initial_bootloader_heap_content", - "ordinal": 0, - "type_info": "Jsonb" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n initial_bootloader_heap_content\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "a4861c931e84d897c27f666de1c5ca679a0459a012899a373c67393d30d12601": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8Array" - ] - } - }, - "query": "\n UPDATE scheduler_dependency_tracker_fri\n SET\n status = 'queued'\n WHERE\n l1_batch_number = ANY ($1)\n " - }, - "a48c92f557e5e3a2674ce0dee9cd92f5a547150590b8c221c4065eab11175c7a": { - "describe": { - "columns": [ - { - "name": "max?", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(INDEX) AS \"max?\"\n FROM\n initial_writes\n " - }, - "a4a4b0bfbe05eac100c42a717e8d7cbb0bc526ebe61a07f735d4ab587058b22c": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n hash\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "a4fcd075b68467bb119e49e6b20a69138206dfeb41f3daff4a3eef1de0bed4e4": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray", - "Int8Array", - "Int8" - ] - } - }, - "query": "\n INSERT INTO\n initial_writes (hashed_key, INDEX, l1_batch_number, created_at, updated_at)\n SELECT\n u.hashed_key,\n u.index,\n $3,\n NOW(),\n NOW()\n FROM\n UNNEST($1::bytea[], $2::BIGINT[]) AS u (hashed_key, INDEX)\n " - }, - "a74d029f58801ec05d8d14a3b065d93e391600ab9da2e5fd4e8b139ab3d77583": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE proof_generation_details\n SET\n status = 'generated',\n proof_blob_url = $1,\n updated_at = NOW()\n WHERE\n l1_batch_number = $2\n " - }, - "a83f853b1d63365e88975a926816c6e7b4595f3e7c3dca1d1590de5437187733": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bool", - "Bytea", - "Int8", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n hash = $1,\n merkle_root_hash = $2,\n commitment = $3,\n default_aa_code_hash = $4,\n compressed_repeated_writes = $5,\n compressed_initial_writes = $6,\n l2_l1_compressed_messages = $7,\n l2_l1_merkle_root = $8,\n zkporter_is_available = $9,\n bootloader_code_hash = $10,\n rollup_last_leaf_index = $11,\n aux_data_hash = $12,\n pass_through_data_hash = $13,\n meta_parameters_hash = $14,\n compressed_state_diffs = $15,\n updated_at = NOW()\n WHERE\n number = $16\n " - }, - "a84ee70bec8c03bd51e1c6bad44c9a64904026506914abae2946e5d353d6a604": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8", - "Int2" - ] - } - }, - "query": "\n SELECT\n id\n FROM\n prover_jobs_fri\n WHERE\n l1_batch_number = $1\n AND status = 'successful'\n AND aggregation_round = $2\n " - }, - "a91c23c4d33771122cec2589c6fe2757dbc13be6b30f5840744e5e0569adc66e": { - "describe": { - "columns": [ - { - "name": "upgrade_tx_hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n upgrade_tx_hash\n FROM\n protocol_versions\n WHERE\n id = $1\n " - }, - "aa91697157517322b0dbb53dca99f41220c51f58a03c61d6b7789eab0504e320": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 1, - "type_info": "Int2" - }, - { - "name": "depth", - "ordinal": 2, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued'\n WHERE\n (l1_batch_number, circuit_id, depth) IN (\n SELECT\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth\n FROM\n prover_jobs_fri\n JOIN node_aggregation_witness_jobs_fri nawj ON prover_jobs_fri.l1_batch_number = nawj.l1_batch_number\n AND prover_jobs_fri.circuit_id = nawj.circuit_id\n AND prover_jobs_fri.depth = nawj.depth\n WHERE\n nawj.status = 'waiting_for_proofs'\n AND prover_jobs_fri.status = 'successful'\n AND prover_jobs_fri.aggregation_round = 1\n AND prover_jobs_fri.depth = 0\n GROUP BY\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.depth,\n nawj.number_of_dependent_jobs\n HAVING\n COUNT(*) = nawj.number_of_dependent_jobs\n )\n RETURNING\n l1_batch_number,\n circuit_id,\n depth;\n " - }, - "aaf4fb97c95a5290fb1620cd868477dcf21955e0921ba648ba2e751dbfc3cb45": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_id!", - "ordinal": 1, - "type_info": "Int2" - }, - { - "name": "aggregation_round!", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "status!", - "ordinal": 3, - "type_info": "Text" - } - ], - "nullable": [ - null, - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\",\n circuit_id AS \"circuit_id!\",\n aggregation_round AS \"aggregation_round!\",\n status AS \"status!\"\n FROM\n prover_jobs_fri\n WHERE\n status <> 'skipped'\n AND status <> 'successful'\n GROUP BY\n circuit_id,\n aggregation_round,\n status\n " - }, - "ac505ae6cfc744b07b52997db789bdc9efc6b89fc0444caf8271edd7dfe4a3bc": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n id\n FROM\n protocol_versions\n " - }, - "ac673a122962b57b0272df2d82a1788feea2fbb5682de09120dd109899510820": { - "describe": { - "columns": [ - { - "name": "block_batch?", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "max_batch?", - "ordinal": 1, - "type_info": "Int8" - } - ], - "nullable": [ - null, - null - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n (\n SELECT\n l1_batch_number\n FROM\n miniblocks\n WHERE\n number = $1\n ) AS \"block_batch?\",\n (\n SELECT\n MAX(number) + 1\n FROM\n l1_batches\n ) AS \"max_batch?\"\n " - }, - "ada54322a28012b1b761f3631c4cd6ca26aa2fa565fcf208b6985f461c1868f2": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "eth_tx_id", - "ordinal": 1, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Text" - ] - } - }, - "query": "\n UPDATE eth_txs_history\n SET\n updated_at = NOW(),\n confirmed_at = NOW()\n WHERE\n tx_hash = $1\n RETURNING\n id,\n eth_tx_id\n " - }, - "aeda34b1beadca72e3e600ea9ae63f436a4f16dbeb784d0d28be392ad96b1c49": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n UPDATE eth_txs\n SET\n has_failed = TRUE\n WHERE\n id = $1\n " - }, - "aefea1f3e87f28791cc547f193a895006e23ec73018f4b4e0a364a741f5c9781": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n miniblocks\n WHERE\n number = $1\n " - }, - "af72fabd90eb43fb315f46d7fe9f724216807ffd481cd6f7f19968e42e52b284": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'sent_to_server',\n updated_at = NOW()\n WHERE\n l1_batch_number = $1\n " - }, - "afc24bd1407dba82cd3dc9e7ee71ac4ab2d73bda6022700aeb0a630a2563a4b4": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'failed',\n error = $1,\n updated_at = NOW()\n WHERE\n id = $2\n " - }, - "b17c71983da060f08616e001b42f8dcbcb014b4f808c6232abd9a83354c995ac": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n id,\n status,\n attempts\n " - }, - "b23ddb16513d69331056b94d466663a9c5ea62ea7c99a77941eb8f05d4454125": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int2", - "Text", - "Int4", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n leaf_aggregation_witness_jobs_fri (\n l1_batch_number,\n circuit_id,\n closed_form_inputs_blob_url,\n number_of_basic_circuits,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number, circuit_id) DO\n UPDATE\n SET\n updated_at = NOW()\n " - }, - "b321c5ba22358cbb1fd9c627f1e7b56187686173327498ac75424593547c19c5": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n scheduler_witness_jobs_fri\n WHERE\n l1_batch_number = $1\n " - }, - "b33e8da69281efe7750043e409d9871731c41cef01da3d6aaf2c53f7b17c47b2": { - "describe": { - "columns": [ - { - "name": "value", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Int8" - ] - } - }, - "query": "\n SELECT\n value\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n AND storage_logs.miniblock_number <= $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n " - }, - "b367ecb1ebee86ec598c4079591f8c12deeca6b8843fe3869cc2b02b30da5de6": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n proof_compression_jobs_fri\n WHERE\n l1_batch_number = $1\n " - }, - "b3d71dbe14bcd94131b29b64dcb49b6370c211a7fc24ad03a5f0e327f9d18040": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n witness_inputs_fri\n WHERE\n l1_batch_number = $1\n " - }, - "b4304b9afb9f838eee1fe95af5fd964d4bb39b9dcd18fb03bc11ce2fb32b7fb3": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "scheduler_partial_input_blob_url", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "status", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "processing_started_at", - "ordinal": 3, - "type_info": "Timestamp" - }, - { - "name": "time_taken", - "ordinal": 4, - "type_info": "Time" - }, - { - "name": "error", - "ordinal": 5, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 6, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "attempts", - "ordinal": 8, - "type_info": "Int2" - }, - { - "name": "protocol_version", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "picked_by", - "ordinal": 10, - "type_info": "Text" - } - ], - "nullable": [ - false, - false, - false, - true, - true, - true, - false, - false, - false, - true, - true - ], - "parameters": { - "Left": [ - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE scheduler_witness_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $2\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n scheduler_witness_jobs_fri\n WHERE\n status = 'queued'\n AND protocol_version = ANY ($1)\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n scheduler_witness_jobs_fri.*\n " - }, - "b452354c888bfc19b5f4012582061b86b1abd915739533f9982fea9d8e21b9e9": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM factory_deps\n WHERE\n miniblock_number > $1\n " - }, - "b4794e6a0c2366d5d95ab373c310103263af3ff5cb6c9dc5df59d3cd2a5e56b4": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Text", - "Int4", - "Text" - ] - } - }, - "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = $1,\n updated_at = NOW()\n WHERE\n instance_host = $2::TEXT::inet\n AND instance_port = $3\n AND zone = $4\n " - }, - "b49478150dbc8731c531ef3eddc0c2cfff08e6fef3c3824d20dfdf2d0f73e671": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n hash,\n number,\n timestamp\n FROM\n miniblocks\n WHERE\n number > $1\n ORDER BY\n number ASC\n " - }, - "b4a0444897b60c7061363a48b2b5386a2fd53492f3df05545edbfb0ec0f059d2": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int4" - ] - } - }, - "query": "\n UPDATE eth_txs\n SET\n confirmed_eth_tx_history_id = $1\n WHERE\n id = $2\n " - }, - "b5fd77f515fe168908cc90e44d0697e36b3c2a997038c30553f7727cdfa17361": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "ByteaArray", - "Int4Array", - "VarcharArray", - "JsonbArray", - "Int8Array", - "NumericArray" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n miniblock_number = $1,\n index_in_block = data_table.index_in_block,\n error = NULLIF(data_table.error, ''),\n in_mempool = FALSE,\n execution_info = execution_info || data_table.new_execution_info,\n refunded_gas = data_table.refunded_gas,\n effective_gas_price = data_table.effective_gas_price,\n updated_at = NOW()\n FROM\n (\n SELECT\n UNNEST($2::bytea[]) AS hash,\n UNNEST($3::INTEGER[]) AS index_in_block,\n UNNEST($4::VARCHAR[]) AS error,\n UNNEST($5::jsonb[]) AS new_execution_info,\n UNNEST($6::BIGINT[]) AS refunded_gas,\n UNNEST($7::NUMERIC[]) AS effective_gas_price\n ) AS data_table\n WHERE\n transactions.hash = data_table.hash\n " - }, - "b678edd9f6ea97b8f086566811f651aa072f030c70a5e6de38843a1d9afdf329": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n INSERT INTO\n commitments (l1_batch_number, events_queue_commitment, bootloader_initial_content_commitment)\n VALUES\n ($1, $2, $3)\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "b75e3d2fecbf5d85e93848b7a35180abbd76956e073432af8d8500327b74e488": { - "describe": { - "columns": [ - { - "name": "version", - "ordinal": 0, - "type_info": "Text" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Text" - ] - } - }, - "query": "\n SELECT\n VERSION\n FROM\n compiler_versions\n WHERE\n compiler = $1\n ORDER BY\n VERSION\n " - }, - "b7bf6999002dd89dc1224468ca79c9a85e3c24fca1bf87905f7fc68fe2ce3276": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4Array", - "ByteaArray", - "Int8" - ] - } - }, - "query": "\n UPDATE transactions\n SET\n l1_batch_number = $3,\n l1_batch_tx_index = data_table.l1_batch_tx_index,\n updated_at = NOW()\n FROM\n (\n SELECT\n UNNEST($1::INT[]) AS l1_batch_tx_index,\n UNNEST($2::bytea[]) AS hash\n ) AS data_table\n WHERE\n transactions.hash = data_table.hash\n " - }, - "bb1904a01a3860b5440ae23763d6d5ee4341edadb8a86b459a07427b7e265e98": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_tx_count", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 4, - "type_info": "Bool" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 6, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "bloom", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 9, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 10, - "type_info": "Jsonb" - }, - { - "name": "base_fee_per_gas", - "ordinal": 11, - "type_info": "Numeric" - }, - { - "name": "l1_gas_price", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 14, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 15, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 16, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 17, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "pubdata_input", - "ordinal": 19, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - false, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n pubdata_input\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "bd51c9d93b103292f5acbdb266ba4b4e2af48907fa9321064ddb24ac02ab17cd": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS commit_tx ON (l1_batches.eth_commit_tx_id = commit_tx.eth_tx_id)\n WHERE\n commit_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "bd74435dc6dba3f4173858682ee5661d1df4ec053797d75cfd32272be4f485e7": { - "describe": { - "columns": [ - { - "name": "key!", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "value!", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "address!", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "miniblock_number!", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "l1_batch_number!", - "ordinal": 4, - "type_info": "Int8" - }, - { - "name": "index", - "ordinal": 5, - "type_info": "Int8" - } - ], - "nullable": [ - true, - true, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n storage_logs.key AS \"key!\",\n storage_logs.value AS \"value!\",\n storage_logs.address AS \"address!\",\n storage_logs.miniblock_number AS \"miniblock_number!\",\n initial_writes.l1_batch_number AS \"l1_batch_number!\",\n initial_writes.index\n FROM\n (\n SELECT\n hashed_key,\n MAX(ARRAY[miniblock_number, operation_number]::INT[]) AS op\n FROM\n storage_logs\n WHERE\n miniblock_number <= $1\n AND hashed_key >= $2\n AND hashed_key < $3\n GROUP BY\n hashed_key\n ORDER BY\n hashed_key\n ) AS keys\n INNER JOIN storage_logs ON keys.hashed_key = storage_logs.hashed_key\n AND storage_logs.miniblock_number = keys.op[1]\n AND storage_logs.operation_number = keys.op[2]\n INNER JOIN initial_writes ON keys.hashed_key = initial_writes.hashed_key;\n " - }, - "be16d820c124dba9f4a272f54f0b742349e78e6e4ce3e7c9a0dcf6447eedc6d8": { - "describe": { - "columns": [ - { - "name": "miniblock_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "log_index_in_miniblock", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "log_index_in_tx", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "tx_hash", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "block_hash", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "l1_batch_number?", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "shard_id", - "ordinal": 6, - "type_info": "Int4" - }, - { - "name": "is_service", - "ordinal": 7, - "type_info": "Bool" - }, - { - "name": "tx_index_in_miniblock", - "ordinal": 8, - "type_info": "Int4" - }, - { - "name": "tx_index_in_l1_batch", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "sender", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "key", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "value", - "ordinal": 12, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - null, - null, - false, - false, - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n miniblock_number,\n log_index_in_miniblock,\n log_index_in_tx,\n tx_hash,\n NULL::bytea AS \"block_hash\",\n NULL::BIGINT AS \"l1_batch_number?\",\n shard_id,\n is_service,\n tx_index_in_miniblock,\n tx_index_in_l1_batch,\n sender,\n key,\n value\n FROM\n l2_to_l1_logs\n WHERE\n tx_hash = $1\n ORDER BY\n log_index_in_tx ASC\n " - }, - "bfb80956a18eabf266f5b5a9d62912d57f8eb2a38bdb7884fc812a2897a3a660": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'in_gpu_proof'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n l1_batch_number,\n status,\n attempts\n " - }, - "bfc84bcf0985446b337467dd1da709dbee508ad6d1cae43e477cf1bef8cb4aa9": { - "describe": { - "columns": [ - { - "name": "hashed_key", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT DISTINCT\n hashed_key\n FROM\n storage_logs\n WHERE\n miniblock_number BETWEEN $1 AND $2\n " - }, - "c038cecd8184e5e8d9f498116bff995b654adfe328cb825a44ad36b4bf9ec8f2": { - "describe": { - "columns": [ - { - "name": "address", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "topic1", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "topic2", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "topic3", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "topic4", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "value", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "block_hash", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "l1_batch_number?", - "ordinal": 7, - "type_info": "Int8" - }, - { - "name": "miniblock_number", - "ordinal": 8, - "type_info": "Int8" - }, - { - "name": "tx_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "tx_index_in_block", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "event_index_in_block", - "ordinal": 11, - "type_info": "Int4" - }, - { - "name": "event_index_in_tx", - "ordinal": 12, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - null, - null, - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n address,\n topic1,\n topic2,\n topic3,\n topic4,\n value,\n NULL::bytea AS \"block_hash\",\n NULL::BIGINT AS \"l1_batch_number?\",\n miniblock_number,\n tx_hash,\n tx_index_in_block,\n event_index_in_block,\n event_index_in_tx\n FROM\n events\n WHERE\n tx_hash = $1\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n " - }, - "c03df29f4661fa47c1412bd82ba379f3b2e9ff1bc6e8e38f473fb4950c8e4b77": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n contract_verification_requests\n WHERE\n status = 'queued'\n " - }, - "c10cf20825de4d24300c7ec50d4a653852f7e43670076eb2ebcd49542a870539": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n INSERT INTO\n scheduler_dependency_tracker_fri (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n updated_at = NOW()\n " - }, - "c139df45a977290d1c2c7987fb9c1d66aeaeb6e2d36fddcf96775f01716a8a74": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM storage_logs\n WHERE\n miniblock_number > $1\n " - }, - "c14837e92dbb02f2fde7109f524432d865852afe0c60e11a2c1800d30599aa61": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text" - ] - } - }, - "query": "\n DELETE FROM compiler_versions\n WHERE\n compiler = $1\n " - }, - "c192377c08abab9306c5b0844368aa0f8525832cb4075e831c0d4b23c5675b99": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Int8", - "Bytea" - ] - } - }, - "query": "\n SELECT\n bytecode\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n AND storage_logs.miniblock_number <= $2\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n ) t\n JOIN factory_deps ON value = factory_deps.bytecode_hash\n WHERE\n value != $3\n " - }, - "c23d5ff919ade5898c6a912780ae899e360650afccb34f5cc301b5cbac4a3d36": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = $1,\n updated_at = NOW()\n WHERE\n id = $2\n " - }, - "c36abacc705a2244d423599779e38d60d6e93bcb34fd20422e227714fccbf6b7": { - "describe": { - "columns": [ - { - "name": "address", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "key", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "value", - "ordinal": 2, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n address,\n key,\n value\n FROM\n storage_logs\n WHERE\n miniblock_number BETWEEN (\n SELECT\n MIN(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n ) AND (\n SELECT\n MAX(number)\n FROM\n miniblocks\n WHERE\n l1_batch_number = $1\n )\n ORDER BY\n miniblock_number,\n operation_number\n " - }, - "c41312e01aa66897552e8be9acc8d43c31ec7441a7f6c5040e120810ebbb72f7": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int2", - "Text", - "Int2", - "Int4", - "Int4", - "Bool", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n prover_jobs_fri (\n l1_batch_number,\n circuit_id,\n circuit_blob_url,\n aggregation_round,\n sequence_number,\n depth,\n is_node_final_proof,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, $7, $8, 'queued', NOW(), NOW())\n ON CONFLICT (l1_batch_number, aggregation_round, circuit_id, depth, sequence_number) DO\n UPDATE\n SET\n updated_at = NOW()\n " - }, - "c4ea7812861a283448095acbb1164420a25eef488de2b67e91ed39657667bd4a": { - "describe": { - "columns": [ - { - "name": "l1_address", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "l2_address", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_address,\n l2_address\n FROM\n tokens\n " - }, - "c5656667e5610ffb33e7b977ac92b7c4d79cbd404e0267794ec203df0cbb169d": { - "describe": { - "columns": [ - { - "name": "number!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COALESCE(MAX(number), 0) AS \"number!\"\n FROM\n l1_batches\n WHERE\n eth_prove_tx_id IS NOT NULL\n " - }, - "c5d6e1d5d834409bd793c8ce1fb2c212918b31dabebf08a84efdfe1feee85765": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n UPDATE scheduler_dependency_tracker_fri\n SET\n status = 'queuing'\n WHERE\n l1_batch_number IN (\n SELECT\n l1_batch_number\n FROM\n scheduler_dependency_tracker_fri\n WHERE\n status != 'queued'\n AND circuit_1_final_prover_job_id IS NOT NULL\n AND circuit_2_final_prover_job_id IS NOT NULL\n AND circuit_3_final_prover_job_id IS NOT NULL\n AND circuit_4_final_prover_job_id IS NOT NULL\n AND circuit_5_final_prover_job_id IS NOT NULL\n AND circuit_6_final_prover_job_id IS NOT NULL\n AND circuit_7_final_prover_job_id IS NOT NULL\n AND circuit_8_final_prover_job_id IS NOT NULL\n AND circuit_9_final_prover_job_id IS NOT NULL\n AND circuit_10_final_prover_job_id IS NOT NULL\n AND circuit_11_final_prover_job_id IS NOT NULL\n AND circuit_12_final_prover_job_id IS NOT NULL\n AND circuit_13_final_prover_job_id IS NOT NULL\n )\n RETURNING\n l1_batch_number;\n " - }, - "c6d523c6ae857022318350a2f210d7eaeeb4549ed59b58f8d984be2a22a80355": { - "describe": { - "columns": [ - { - "name": "max", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Numeric" - ] - } - }, - "query": "\n SELECT\n MAX(l1_batches.number)\n FROM\n l1_batches\n JOIN eth_txs ON (l1_batches.eth_commit_tx_id = eth_txs.id)\n JOIN eth_txs_history AS commit_tx ON (eth_txs.confirmed_eth_tx_history_id = commit_tx.id)\n WHERE\n commit_tx.confirmed_at IS NOT NULL\n AND eth_prove_tx_id IS NOT NULL\n AND eth_execute_tx_id IS NULL\n AND EXTRACT(\n epoch\n FROM\n commit_tx.confirmed_at\n ) < $1\n " - }, - "c706a49ff54f6b424e24d061fe7ac429aac3c030f7e226a1264243d8cdae038d": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Time", - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n updated_at = NOW(),\n time_taken = $2,\n l1_proof_blob_url = $3\n WHERE\n l1_batch_number = $4\n " - }, - "c809f42a221b18a767e9dd0286503d8bd356f2f9cc249cd8b90caa5a8b5918e3": { - "describe": { - "columns": [ - { - "name": "count!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [ - "Bytea", - "Bytea" - ] - } - }, - "query": "\n SELECT\n COUNT(*) AS \"count!\"\n FROM\n (\n SELECT\n *\n FROM\n storage_logs\n WHERE\n storage_logs.hashed_key = $1\n ORDER BY\n storage_logs.miniblock_number DESC,\n storage_logs.operation_number DESC\n LIMIT\n 1\n ) sl\n WHERE\n sl.value != $2\n " - }, - "ca9d06141265b8524ee28c55569cb21a635037d89ce24dd3ad58ffaadb59594a": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status <> 'successful'\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n " - }, - "cb98d84fc34af1e4a4c2f427c5bb4afd384063ae394a847b26304dd18d490ab4": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "hash", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n timestamp,\n hash\n FROM\n l1_batches\n WHERE\n number = $1\n " - }, - "cddf48514aa2aa249d0530d44c741368993009bb4bd90c2ad177ce56317aa04c": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 34, - "type_info": "ByteaArray" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 36, - "type_info": "Int4" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - false, - true, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8", - "Int8" - ] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n system_logs,\n compressed_state_diffs,\n protocol_version,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n (\n SELECT\n l1_batches.*,\n ROW_NUMBER() OVER (\n ORDER BY\n number ASC\n ) AS ROW_NUMBER\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id IS NOT NULL\n AND l1_batches.skip_proof = TRUE\n AND l1_batches.number > $1\n ORDER BY\n number\n LIMIT\n $2\n ) inn\n LEFT JOIN commitments ON commitments.l1_batch_number = inn.number\n WHERE\n number - ROW_NUMBER = $1\n " - }, - "ce5779092feb8a3d3e2c5e395783e67f08f2ead5f55bfb6594e50346bf9cf2ef": { - "describe": { - "columns": [ - { - "name": "l1_batch_number!", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 1, - "type_info": "Int2" - }, - { - "name": "aggregation_round", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - null, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MIN(l1_batch_number) AS \"l1_batch_number!\",\n circuit_id,\n aggregation_round\n FROM\n prover_jobs_fri\n WHERE\n status IN ('queued', 'in_gpu_proof', 'in_progress', 'failed')\n GROUP BY\n circuit_id,\n aggregation_round\n " - }, - "cea9fe027a6a0ada827f23b48ac32432295b2f7ee40bf13522a6edbd236f1970": { - "describe": { - "columns": [ - { - "name": "hashed_key!", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "value?", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - null, - null - ], - "parameters": { - "Left": [ - "ByteaArray", - "Int8" - ] - } - }, - "query": "\n SELECT\n u.hashed_key AS \"hashed_key!\",\n (\n SELECT\n value\n FROM\n storage_logs\n WHERE\n hashed_key = u.hashed_key\n AND miniblock_number <= $2\n ORDER BY\n miniblock_number DESC,\n operation_number DESC\n LIMIT\n 1\n ) AS \"value?\"\n FROM\n UNNEST($1::bytea[]) AS u (hashed_key)\n " - }, - "d14b52df2cd9f9e484c60ba00383b438f14b68535111cf2cedd363fc646aac99": { - "describe": { - "columns": [ - { - "name": "timestamp", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n timestamp\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id IS NULL\n AND number > 0\n ORDER BY\n number\n LIMIT\n 1\n " - }, - "d1b261f4057e4113b96eb87c9e20015eeb3ef2643ceda3024504a471b24d1283": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "timestamp", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 2, - "type_info": "Bool" - }, - { - "name": "l1_tx_count", - "ordinal": 3, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "bloom", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "hash", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "parent_hash", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "commitment", - "ordinal": 10, - "type_info": "Bytea" - }, - { - "name": "compressed_write_logs", - "ordinal": 11, - "type_info": "Bytea" - }, - { - "name": "compressed_contracts", - "ordinal": 12, - "type_info": "Bytea" - }, - { - "name": "eth_prove_tx_id", - "ordinal": 13, - "type_info": "Int4" - }, - { - "name": "eth_commit_tx_id", - "ordinal": 14, - "type_info": "Int4" - }, - { - "name": "eth_execute_tx_id", - "ordinal": 15, - "type_info": "Int4" - }, - { - "name": "merkle_root_hash", - "ordinal": 16, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 18, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 19, - "type_info": "Jsonb" - }, - { - "name": "compressed_initial_writes", - "ordinal": 20, - "type_info": "Bytea" - }, - { - "name": "compressed_repeated_writes", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "l2_l1_compressed_messages", - "ordinal": 22, - "type_info": "Bytea" - }, - { - "name": "l2_l1_merkle_root", - "ordinal": 23, - "type_info": "Bytea" - }, - { - "name": "l1_gas_price", - "ordinal": 24, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 25, - "type_info": "Int8" - }, - { - "name": "rollup_last_leaf_index", - "ordinal": 26, - "type_info": "Int8" - }, - { - "name": "zkporter_is_available", - "ordinal": 27, - "type_info": "Bool" - }, - { - "name": "bootloader_code_hash", - "ordinal": 28, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 29, - "type_info": "Bytea" - }, - { - "name": "base_fee_per_gas", - "ordinal": 30, - "type_info": "Numeric" - }, - { - "name": "aux_data_hash", - "ordinal": 31, - "type_info": "Bytea" - }, - { - "name": "pass_through_data_hash", - "ordinal": 32, - "type_info": "Bytea" - }, - { - "name": "meta_parameters_hash", - "ordinal": 33, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 34, - "type_info": "Int4" - }, - { - "name": "compressed_state_diffs", - "ordinal": 35, - "type_info": "Bytea" - }, - { - "name": "system_logs", - "ordinal": 36, - "type_info": "ByteaArray" - }, - { - "name": "events_queue_commitment", - "ordinal": 37, - "type_info": "Bytea" - }, - { - "name": "bootloader_initial_content_commitment", - "ordinal": 38, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 39, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - true, - true, - true, - false, - false, - true, - true, - true, - true, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number,\n timestamp,\n is_finished,\n l1_tx_count,\n l2_tx_count,\n fee_account_address,\n bloom,\n priority_ops_onchain_data,\n hash,\n parent_hash,\n commitment,\n compressed_write_logs,\n compressed_contracts,\n eth_prove_tx_id,\n eth_commit_tx_id,\n eth_execute_tx_id,\n merkle_root_hash,\n l2_to_l1_logs,\n l2_to_l1_messages,\n used_contract_hashes,\n compressed_initial_writes,\n compressed_repeated_writes,\n l2_l1_compressed_messages,\n l2_l1_merkle_root,\n l1_gas_price,\n l2_fair_gas_price,\n rollup_last_leaf_index,\n zkporter_is_available,\n bootloader_code_hash,\n default_aa_code_hash,\n base_fee_per_gas,\n aux_data_hash,\n pass_through_data_hash,\n meta_parameters_hash,\n protocol_version,\n compressed_state_diffs,\n system_logs,\n events_queue_commitment,\n bootloader_initial_content_commitment,\n pubdata_input\n FROM\n l1_batches\n LEFT JOIN commitments ON commitments.l1_batch_number = l1_batches.number\n WHERE\n number = 0\n OR eth_commit_tx_id IS NOT NULL\n AND commitment IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "d3b09cbcddf6238b358d32d57678242aad3e9a47400f6d6837a35f4c54a216b9": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n number\n FROM\n l1_batches\n LEFT JOIN eth_txs_history AS execute_tx ON (l1_batches.eth_execute_tx_id = execute_tx.eth_tx_id)\n WHERE\n execute_tx.confirmed_at IS NOT NULL\n ORDER BY\n number DESC\n LIMIT\n 1\n " - }, - "d70cfc158e31dd2d5c942d24f81fd17f833fb15b58b0110c7cc566946db98e76": { - "describe": { - "columns": [ - { - "name": "block_hash?", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "address!", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "topic1!", - "ordinal": 2, - "type_info": "Bytea" - }, - { - "name": "topic2!", - "ordinal": 3, - "type_info": "Bytea" - }, - { - "name": "topic3!", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "topic4!", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "value!", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "miniblock_number!", - "ordinal": 7, - "type_info": "Int8" - }, - { - "name": "l1_batch_number?", - "ordinal": 8, - "type_info": "Int8" - }, - { - "name": "tx_hash!", - "ordinal": 9, - "type_info": "Bytea" - }, - { - "name": "tx_index_in_block!", - "ordinal": 10, - "type_info": "Int4" - }, - { - "name": "event_index_in_block!", - "ordinal": 11, - "type_info": "Int4" - }, - { - "name": "event_index_in_tx!", - "ordinal": 12, - "type_info": "Int4" - } - ], - "nullable": [ - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n WITH\n events_select AS (\n SELECT\n address,\n topic1,\n topic2,\n topic3,\n topic4,\n value,\n miniblock_number,\n tx_hash,\n tx_index_in_block,\n event_index_in_block,\n event_index_in_tx\n FROM\n events\n WHERE\n miniblock_number > $1\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n )\n SELECT\n miniblocks.hash AS \"block_hash?\",\n address AS \"address!\",\n topic1 AS \"topic1!\",\n topic2 AS \"topic2!\",\n topic3 AS \"topic3!\",\n topic4 AS \"topic4!\",\n value AS \"value!\",\n miniblock_number AS \"miniblock_number!\",\n miniblocks.l1_batch_number AS \"l1_batch_number?\",\n tx_hash AS \"tx_hash!\",\n tx_index_in_block AS \"tx_index_in_block!\",\n event_index_in_block AS \"event_index_in_block!\",\n event_index_in_tx AS \"event_index_in_tx!\"\n FROM\n events_select\n INNER JOIN miniblocks ON events_select.miniblock_number = miniblocks.number\n ORDER BY\n miniblock_number ASC,\n event_index_in_block ASC\n " - }, - "d712707e47e143c52330ea6e0513d2839f0f928c06b8020eecec38e895f99b42": { - "describe": { - "columns": [ - { - "name": "address", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "key", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n address,\n key\n FROM\n protective_reads\n WHERE\n l1_batch_number = $1\n " - }, - "d7e8eabd7b43ff62838fbc847e4813d2b2d411bd5faf8306cd48db500532b711": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "Text", - "Text" - ] - } - }, - "query": "\n SELECT\n l1_batch_number,\n status\n FROM\n proof_compression_jobs_fri\n WHERE\n l1_batch_number = (\n SELECT\n MIN(l1_batch_number)\n FROM\n proof_compression_jobs_fri\n WHERE\n status = $1\n OR status = $2\n )\n " - }, - "d7ed82f0d012f72374edb2ebcec33c83477d65a6f8cb2673f67b3148cd95b436": { - "describe": { - "columns": [ - { - "name": "count", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n COUNT(*)\n FROM\n eth_txs\n WHERE\n has_failed = TRUE\n " - }, - "d8e0f98a67ffb53a1caa6820f8475da2787332deca5708d1d08730cdbfc73541": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_tx_count", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "l2_tx_count", - "ordinal": 2, - "type_info": "Int4" - }, - { - "name": "timestamp", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "is_finished", - "ordinal": 4, - "type_info": "Bool" - }, - { - "name": "fee_account_address", - "ordinal": 5, - "type_info": "Bytea" - }, - { - "name": "l2_to_l1_logs", - "ordinal": 6, - "type_info": "ByteaArray" - }, - { - "name": "l2_to_l1_messages", - "ordinal": 7, - "type_info": "ByteaArray" - }, - { - "name": "bloom", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "priority_ops_onchain_data", - "ordinal": 9, - "type_info": "ByteaArray" - }, - { - "name": "used_contract_hashes", - "ordinal": 10, - "type_info": "Jsonb" - }, - { - "name": "base_fee_per_gas", - "ordinal": 11, - "type_info": "Numeric" - }, - { - "name": "l1_gas_price", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "l2_fair_gas_price", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "bootloader_code_hash", - "ordinal": 14, - "type_info": "Bytea" - }, - { - "name": "default_aa_code_hash", - "ordinal": 15, - "type_info": "Bytea" - }, - { - "name": "protocol_version", - "ordinal": 16, - "type_info": "Int4" - }, - { - "name": "system_logs", - "ordinal": 17, - "type_info": "ByteaArray" - }, - { - "name": "compressed_state_diffs", - "ordinal": 18, - "type_info": "Bytea" - }, - { - "name": "pubdata_input", - "ordinal": 19, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - false, - true, - true - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n number,\n l1_tx_count,\n l2_tx_count,\n timestamp,\n is_finished,\n fee_account_address,\n l2_to_l1_logs,\n l2_to_l1_messages,\n bloom,\n priority_ops_onchain_data,\n used_contract_hashes,\n base_fee_per_gas,\n l1_gas_price,\n l2_fair_gas_price,\n bootloader_code_hash,\n default_aa_code_hash,\n protocol_version,\n system_logs,\n compressed_state_diffs,\n pubdata_input\n FROM\n l1_batches\n WHERE\n eth_commit_tx_id = $1\n OR eth_prove_tx_id = $1\n OR eth_execute_tx_id = $1\n " - }, - "d8e3ee346375e4b6a8b2c73a3827e88abd0f8164c2413dc83c91c29665ca645e": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "attempts", - "ordinal": 2, - "type_info": "Int2" - } - ], - "nullable": [ - false, - false, - false - ], - "parameters": { - "Left": [ - "Interval", - "Int2" - ] - } - }, - "query": "\n UPDATE leaf_aggregation_witness_jobs_fri\n SET\n status = 'queued',\n updated_at = NOW(),\n processing_started_at = NOW()\n WHERE\n (\n status = 'in_progress'\n AND processing_started_at <= NOW() - $1::INTERVAL\n AND attempts < $2\n )\n OR (\n status = 'failed'\n AND attempts < $2\n )\n RETURNING\n id,\n status,\n attempts\n " - }, - "d90ed4c0f67c1826f9be90bb5566aba34bfab67494fee578613b03ef7255324d": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Jsonb" - ] - } - }, - "query": "\n UPDATE miniblocks\n SET\n consensus = $2\n WHERE\n number = $1\n " - }, - "da51a5220c2b964303292592c34e8ee5e54b170de9da863bbdbc79e3f206640b": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n DELETE FROM storage\n WHERE\n hashed_key = ANY ($1)\n " - }, - "db3e74f0e83ffbf84a6d61e560f2060fbea775dc185f639139fbfd23e4d5f3c6": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Time", - "Int8" - ] - } - }, - "query": "\n UPDATE node_aggregation_witness_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n id = $2\n " - }, - "dc16d0fac093a52480b66dfcb5976fb01e6629e8c982c265f2af1d5000090572": { - "describe": { - "columns": [ - { - "name": "count", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "SELECT COUNT(miniblocks.number) FROM miniblocks WHERE l1_batch_number IS NULL" - }, - "dc481f59aae632ff6f5fa23f5c5c82627a936f7ea9f6c354eca4bea76fac6b10": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MAX(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n hash IS NOT NULL\n " - }, - "dc764e1636c4e958753c1fd54562e2ca92fdfdf01cfd0b11f5ce24f0458a5e48": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bytea", - "Bool", - "Bytea", - "Int8", - "Bytea", - "Bytea", - "Bytea", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n hash = $1,\n merkle_root_hash = $2,\n compressed_repeated_writes = $3,\n compressed_initial_writes = $4,\n l2_l1_compressed_messages = $5,\n l2_l1_merkle_root = $6,\n zkporter_is_available = $7,\n parent_hash = $8,\n rollup_last_leaf_index = $9,\n pass_through_data_hash = $10,\n meta_parameters_hash = $11,\n compressed_state_diffs = $12,\n updated_at = NOW()\n WHERE\n number = $13\n AND hash IS NULL\n " - }, - "dd55e46dfa5ba3692d9620088a3550b8db817630d1a9341db4a1f453f12e64fb": { - "describe": { - "columns": [ - { - "name": "status", - "ordinal": 0, - "type_info": "Text" - }, - { - "name": "error", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "compilation_errors", - "ordinal": 2, - "type_info": "Jsonb" - } - ], - "nullable": [ - false, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n status,\n error,\n compilation_errors\n FROM\n contract_verification_requests\n WHERE\n id = $1\n " - }, - "dea22358feed1418430505767d03aa4239d3a8be71b47178b4b8fb11fe898b31": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int4", - "Int8", - "Int8" - ] - } - }, - "query": "\n UPDATE l1_batches\n SET\n eth_execute_tx_id = $1,\n updated_at = NOW()\n WHERE\n number BETWEEN $2 AND $3\n " - }, - "df00e33809768120e395d8f740770a4e629b2a1cde641e74e4e55bb100df809f": { - "describe": { - "columns": [ - { - "name": "attempts", - "ordinal": 0, - "type_info": "Int2" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n attempts\n FROM\n prover_jobs_fri\n WHERE\n id = $1\n " - }, - "df3b08549a11729fb475341b8f38f8af02aa297d85a2695c5f448ed14b2d7386": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Bytea", - "Int8", - "Bytea", - "Int4", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n snapshot_recovery (\n l1_batch_number,\n l1_batch_root_hash,\n miniblock_number,\n miniblock_root_hash,\n last_finished_chunk_id,\n total_chunk_count,\n updated_at,\n created_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO\n UPDATE\n SET\n l1_batch_number = excluded.l1_batch_number,\n l1_batch_root_hash = excluded.l1_batch_root_hash,\n miniblock_number = excluded.miniblock_number,\n miniblock_root_hash = excluded.miniblock_root_hash,\n last_finished_chunk_id = excluded.last_finished_chunk_id,\n total_chunk_count = excluded.total_chunk_count,\n updated_at = excluded.updated_at\n " - }, - "e073cfdc7a00559994ce04eca15f35d55901fb1e6805f23413ea43e3637540a0": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "bytecode_hash", - "ordinal": 1, - "type_info": "Bytea" - } - ], - "nullable": [ - false, - false - ], - "parameters": { - "Left": [ - "ByteaArray" - ] - } - }, - "query": "\n SELECT\n bytecode,\n bytecode_hash\n FROM\n factory_deps\n WHERE\n bytecode_hash = ANY ($1)\n " - }, - "e3479d12d9dc97001cf03dc42d9b957e92cd375ec33fe16f855f319ffc0b208e": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "status", - "ordinal": 1, - "type_info": "Text" - }, - { - "name": "circuit_1_final_prover_job_id", - "ordinal": 2, - "type_info": "Int8" - }, - { - "name": "circuit_2_final_prover_job_id", - "ordinal": 3, - "type_info": "Int8" - }, - { - "name": "circuit_3_final_prover_job_id", - "ordinal": 4, - "type_info": "Int8" - }, - { - "name": "circuit_4_final_prover_job_id", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "circuit_5_final_prover_job_id", - "ordinal": 6, - "type_info": "Int8" - }, - { - "name": "circuit_6_final_prover_job_id", - "ordinal": 7, - "type_info": "Int8" - }, - { - "name": "circuit_7_final_prover_job_id", - "ordinal": 8, - "type_info": "Int8" - }, - { - "name": "circuit_8_final_prover_job_id", - "ordinal": 9, - "type_info": "Int8" - }, - { - "name": "circuit_9_final_prover_job_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "circuit_10_final_prover_job_id", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "circuit_11_final_prover_job_id", - "ordinal": 12, - "type_info": "Int8" - }, - { - "name": "circuit_12_final_prover_job_id", - "ordinal": 13, - "type_info": "Int8" - }, - { - "name": "circuit_13_final_prover_job_id", - "ordinal": 14, - "type_info": "Int8" - }, - { - "name": "created_at", - "ordinal": 15, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 16, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n scheduler_dependency_tracker_fri\n WHERE\n l1_batch_number = $1\n " - }, - "e5a90d17b2c25744df4585b53678c7ffd9a04eae27afbdf37a6ba8ff7ac85f3b": { - "describe": { - "columns": [ - { - "name": "serialized_events_queue", - "ordinal": 0, - "type_info": "Jsonb" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n serialized_events_queue\n FROM\n events_queue\n WHERE\n l1_batch_number = $1\n " - }, - "e63cc86a8d527dae2905b2af6a66bc6419ba51514519652e055c769b096015f6": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Interval" - ] - } - }, - "query": "\n DELETE FROM transactions\n WHERE\n miniblock_number IS NULL\n AND received_at < NOW() - $1::INTERVAL\n AND is_priority = FALSE\n AND error IS NULL\n RETURNING\n hash\n " - }, - "e71c39b93ceba5416ff3d988290cb35d4d07d47f33fe1a5b9e9fe1f0ae09b705": { - "describe": { - "columns": [ - { - "name": "usd_price", - "ordinal": 0, - "type_info": "Numeric" - }, - { - "name": "usd_price_updated_at", - "ordinal": 1, - "type_info": "Timestamp" - } - ], - "nullable": [ - true, - true - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n usd_price,\n usd_price_updated_at\n FROM\n tokens\n WHERE\n l2_address = $1\n " - }, - "e74a34a59e6afda689b0ec9e19071ababa66e4a443fbefbfffca72b7540b075b": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n proof_compression_jobs_fri (l1_batch_number, status, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "e76217231b4d896118e9630de9485b19e1294b3aa6e084d2051bb532408672be": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [] - } - }, - "query": "\n UPDATE transactions\n SET\n in_mempool = FALSE\n WHERE\n in_mempool = TRUE\n " - }, - "e9adf5b5a1ab84c20a514a7775f91a9984685eaaaa0a8b223410d560a15a3034": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "aggregation_round", - "ordinal": 3, - "type_info": "Int2" - }, - { - "name": "sequence_number", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "depth", - "ordinal": 5, - "type_info": "Int4" - }, - { - "name": "is_node_final_proof", - "ordinal": 6, - "type_info": "Bool" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Int2Array", - "Int2Array", - "Int4Array", - "Text" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'in_progress',\n attempts = attempts + 1,\n processing_started_at = NOW(),\n updated_at = NOW(),\n picked_by = $4\n WHERE\n id = (\n SELECT\n pj.id\n FROM\n (\n SELECT\n *\n FROM\n UNNEST($1::SMALLINT[], $2::SMALLINT[])\n ) AS tuple (circuit_id, ROUND)\n JOIN LATERAL (\n SELECT\n *\n FROM\n prover_jobs_fri AS pj\n WHERE\n pj.status = 'queued'\n AND pj.protocol_version = ANY ($3)\n AND pj.circuit_id = tuple.circuit_id\n AND pj.aggregation_round = tuple.round\n ORDER BY\n pj.l1_batch_number ASC,\n pj.id ASC\n LIMIT\n 1\n ) AS pj ON TRUE\n ORDER BY\n pj.l1_batch_number ASC,\n pj.aggregation_round DESC,\n pj.id ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n " - }, - "e9ca863d6e77edd39a9fc55700a6686e655206601854799139c22c017a214744": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Int2", - "Int4", - "Text", - "Int4", - "Int4" - ] - } - }, - "query": "\n INSERT INTO\n node_aggregation_witness_jobs_fri (\n l1_batch_number,\n circuit_id,\n depth,\n aggregations_url,\n number_of_dependent_jobs,\n protocol_version,\n status,\n created_at,\n updated_at\n )\n VALUES\n ($1, $2, $3, $4, $5, $6, 'waiting_for_proofs', NOW(), NOW())\n ON CONFLICT (l1_batch_number, circuit_id, depth) DO\n UPDATE\n SET\n updated_at = NOW()\n " - }, - "ea904aa930d602d33b6fbc1bf1178a8a0ec739f4ddec8ffeb3a87253aeb18d30": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n DELETE FROM miniblocks\n WHERE\n number > $1\n " - }, - "ec04b89218111a5dc8d5ade506ac3465e2211ef3013386feb12d4cc04e0eade9": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 1, - "type_info": "Int8" - }, - { - "name": "circuit_id", - "ordinal": 2, - "type_info": "Int2" - }, - { - "name": "aggregation_round", - "ordinal": 3, - "type_info": "Int2" - }, - { - "name": "sequence_number", - "ordinal": 4, - "type_info": "Int4" - }, - { - "name": "depth", - "ordinal": 5, - "type_info": "Int4" - }, - { - "name": "is_node_final_proof", - "ordinal": 6, - "type_info": "Bool" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [ - "Time", - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE prover_jobs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1,\n proof_blob_url = $2\n WHERE\n id = $3\n RETURNING\n prover_jobs_fri.id,\n prover_jobs_fri.l1_batch_number,\n prover_jobs_fri.circuit_id,\n prover_jobs_fri.aggregation_round,\n prover_jobs_fri.sequence_number,\n prover_jobs_fri.depth,\n prover_jobs_fri.is_node_final_proof\n " - }, - "edc61e1285bf6d3837acc67af4f15aaade450980719933089824eb8c494d64a4": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Time", - "Int8" - ] - } - }, - "query": "\n UPDATE witness_inputs_fri\n SET\n status = 'successful',\n updated_at = NOW(),\n time_taken = $1\n WHERE\n l1_batch_number = $2\n " - }, - "ee17d2b3edfe705d14811e3938d4312b2b780563a9fde48bae5e51650475670f": { - "describe": { - "columns": [ - { - "name": "id", - "ordinal": 0, - "type_info": "Int4" - }, - { - "name": "eth_tx_id", - "ordinal": 1, - "type_info": "Int4" - }, - { - "name": "tx_hash", - "ordinal": 2, - "type_info": "Text" - }, - { - "name": "created_at", - "ordinal": 3, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 4, - "type_info": "Timestamp" - }, - { - "name": "base_fee_per_gas", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "priority_fee_per_gas", - "ordinal": 6, - "type_info": "Int8" - }, - { - "name": "confirmed_at", - "ordinal": 7, - "type_info": "Timestamp" - }, - { - "name": "signed_raw_tx", - "ordinal": 8, - "type_info": "Bytea" - }, - { - "name": "sent_at_block", - "ordinal": 9, - "type_info": "Int4" - }, - { - "name": "sent_at", - "ordinal": 10, - "type_info": "Timestamp" - } - ], - "nullable": [ - false, - false, - false, - false, - false, - false, - false, - true, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int4" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n eth_txs_history\n WHERE\n eth_tx_id = $1\n ORDER BY\n created_at DESC\n LIMIT\n 1\n " - }, - "ef331469f78c6ff68a254a15b55d056cc9bae25bc070c5de8424f88fab20e5ea": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 1, - "type_info": "Int4" - } - ], - "nullable": [ - true, - true - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n l1_batch_number,\n l1_batch_tx_index\n FROM\n transactions\n WHERE\n hash = $1\n " - }, - "ef687be83e496d6647e4dfef9eabae63443c51deb818dd0affd1a0949b161737": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8", - "Text", - "Text" - ] - } - }, - "query": "\n INSERT INTO\n proof_compression_jobs_fri (l1_batch_number, fri_proof_blob_url, status, created_at, updated_at)\n VALUES\n ($1, $2, $3, NOW(), NOW())\n ON CONFLICT (l1_batch_number) DO NOTHING\n " - }, - "f012d0922265269746396dac8f25ff66f2c3b2b83d45360818a8782e56aa3d66": { - "describe": { - "columns": [ - { - "name": "hashed_key?", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "value?", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "index", - "ordinal": 2, - "type_info": "Int8" - } - ], - "nullable": [ - null, - null, - true - ], - "parameters": { - "Left": [ - "Int8", - "ByteaArray", - "ByteaArray" - ] - } - }, - "query": "\n WITH\n sl AS (\n SELECT\n (\n SELECT\n ARRAY[hashed_key, value] AS kv\n FROM\n storage_logs\n WHERE\n storage_logs.miniblock_number = $1\n AND storage_logs.hashed_key >= u.start_key\n AND storage_logs.hashed_key <= u.end_key\n ORDER BY\n storage_logs.hashed_key\n LIMIT\n 1\n )\n FROM\n UNNEST($2::bytea[], $3::bytea[]) AS u (start_key, end_key)\n )\n SELECT\n sl.kv[1] AS \"hashed_key?\",\n sl.kv[2] AS \"value?\",\n initial_writes.index\n FROM\n sl\n LEFT OUTER JOIN initial_writes ON initial_writes.hashed_key = sl.kv[1]\n " - }, - "f1a90090c192d68367e799188356efe8d41759bbdcdd6d39db93208f2664f03a": { - "describe": { - "columns": [ - { - "name": "index", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea" - ] - } - }, - "query": "\n SELECT\n INDEX\n FROM\n initial_writes\n WHERE\n hashed_key = $1\n " - }, - "f22c5d136fe68bbfcee60beb304cfdc050b85e6d773b13f9699f15c335d42593": { - "describe": { - "columns": [ - { - "name": "l1_address", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Numeric" - ] - } - }, - "query": "\n SELECT\n l1_address\n FROM\n tokens\n WHERE\n market_volume > $1\n " - }, - "f39372e37160df4897f62a800694867ed765dcb9dc60754df9df8700d4244bfb": { - "describe": { - "columns": [ - { - "name": "l1_address", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "l2_address", - "ordinal": 1, - "type_info": "Bytea" - }, - { - "name": "name", - "ordinal": 2, - "type_info": "Varchar" - }, - { - "name": "symbol", - "ordinal": 3, - "type_info": "Varchar" - }, - { - "name": "decimals", - "ordinal": 4, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - false, - false, - false - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n l1_address,\n l2_address,\n NAME,\n symbol,\n decimals\n FROM\n tokens\n WHERE\n well_known = TRUE\n ORDER BY\n symbol\n " - }, - "f4362a61ab05af3d71a3232d2f017db60405a887f9f7fa0ca60aa7fc879ce630": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Text", - "Int8" - ] - } - }, - "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n error = $2,\n updated_at = NOW()\n WHERE\n l1_batch_number = $3\n " - }, - "f63586d59264eab7388ad1de823227ecaa45d76d1ba260074898fe57c059a15a": { - "describe": { - "columns": [ - { - "name": "hash", - "ordinal": 0, - "type_info": "Bytea" - }, - { - "name": "is_priority", - "ordinal": 1, - "type_info": "Bool" - }, - { - "name": "full_fee", - "ordinal": 2, - "type_info": "Numeric" - }, - { - "name": "layer_2_tip_fee", - "ordinal": 3, - "type_info": "Numeric" - }, - { - "name": "initiator_address", - "ordinal": 4, - "type_info": "Bytea" - }, - { - "name": "nonce", - "ordinal": 5, - "type_info": "Int8" - }, - { - "name": "signature", - "ordinal": 6, - "type_info": "Bytea" - }, - { - "name": "input", - "ordinal": 7, - "type_info": "Bytea" - }, - { - "name": "data", - "ordinal": 8, - "type_info": "Jsonb" - }, - { - "name": "received_at", - "ordinal": 9, - "type_info": "Timestamp" - }, - { - "name": "priority_op_id", - "ordinal": 10, - "type_info": "Int8" - }, - { - "name": "l1_batch_number", - "ordinal": 11, - "type_info": "Int8" - }, - { - "name": "index_in_block", - "ordinal": 12, - "type_info": "Int4" - }, - { - "name": "error", - "ordinal": 13, - "type_info": "Varchar" - }, - { - "name": "gas_limit", - "ordinal": 14, - "type_info": "Numeric" - }, - { - "name": "gas_per_storage_limit", - "ordinal": 15, - "type_info": "Numeric" - }, - { - "name": "gas_per_pubdata_limit", - "ordinal": 16, - "type_info": "Numeric" - }, - { - "name": "tx_format", - "ordinal": 17, - "type_info": "Int4" - }, - { - "name": "created_at", - "ordinal": 18, - "type_info": "Timestamp" - }, - { - "name": "updated_at", - "ordinal": 19, - "type_info": "Timestamp" - }, - { - "name": "execution_info", - "ordinal": 20, - "type_info": "Jsonb" - }, - { - "name": "contract_address", - "ordinal": 21, - "type_info": "Bytea" - }, - { - "name": "in_mempool", - "ordinal": 22, - "type_info": "Bool" - }, - { - "name": "l1_block_number", - "ordinal": 23, - "type_info": "Int4" - }, - { - "name": "value", - "ordinal": 24, - "type_info": "Numeric" - }, - { - "name": "paymaster", - "ordinal": 25, - "type_info": "Bytea" - }, - { - "name": "paymaster_input", - "ordinal": 26, - "type_info": "Bytea" - }, - { - "name": "max_fee_per_gas", - "ordinal": 27, - "type_info": "Numeric" - }, - { - "name": "max_priority_fee_per_gas", - "ordinal": 28, - "type_info": "Numeric" - }, - { - "name": "effective_gas_price", - "ordinal": 29, - "type_info": "Numeric" - }, - { - "name": "miniblock_number", - "ordinal": 30, - "type_info": "Int8" - }, - { - "name": "l1_batch_tx_index", - "ordinal": 31, - "type_info": "Int4" - }, - { - "name": "refunded_gas", - "ordinal": 32, - "type_info": "Int8" - }, - { - "name": "l1_tx_mint", - "ordinal": 33, - "type_info": "Numeric" - }, - { - "name": "l1_tx_refund_recipient", - "ordinal": 34, - "type_info": "Bytea" - }, - { - "name": "upgrade_id", - "ordinal": 35, - "type_info": "Int4" - } - ], - "nullable": [ - false, - false, - true, - true, - false, - true, - true, - true, - false, - false, - true, - true, - true, - true, - true, - true, - true, - true, - false, - false, - false, - true, - false, - true, - false, - false, - false, - true, - true, - true, - true, - true, - false, - true, - true, - true - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n *\n FROM\n transactions\n WHERE\n l1_batch_number = $1\n ORDER BY\n miniblock_number,\n index_in_block\n " - }, - "f717ca5d0890759496739a678955e6f8b7f88a0894a7f9e27fc26f93997d37c7": { - "describe": { - "columns": [ - { - "name": "l1_batch_number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Text", - "Text", - "Text" - ] - } - }, - "query": "\n UPDATE proof_compression_jobs_fri\n SET\n status = $1,\n attempts = attempts + 1,\n updated_at = NOW(),\n processing_started_at = NOW(),\n picked_by = $3\n WHERE\n l1_batch_number = (\n SELECT\n l1_batch_number\n FROM\n proof_compression_jobs_fri\n WHERE\n status = $2\n ORDER BY\n l1_batch_number ASC\n LIMIT\n 1\n FOR UPDATE\n SKIP LOCKED\n )\n RETURNING\n proof_compression_jobs_fri.l1_batch_number\n " - }, - "f91790ae5cc4b087bf942ba52dd63a1e89945f8d5e0f4da42ecf6313c4f5967e": { - "describe": { - "columns": [ - { - "name": "number", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - null - ], - "parameters": { - "Left": [] - } - }, - "query": "\n SELECT\n MIN(number) AS \"number\"\n FROM\n l1_batches\n WHERE\n hash IS NOT NULL\n " - }, - "f922c0718c9dda2f285f09cbabad425bac8ed3d2780c60c9b63afbcea131f9a0": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Bytea", - "Jsonb" - ] - } - }, - "query": "\n INSERT INTO\n transaction_traces (tx_hash, trace, created_at, updated_at)\n VALUES\n ($1, $2, NOW(), NOW())\n " - }, - "fcc108fd59203644ff86ded0505c7dfb7aad7261e5fc402d845aedc3b91a4e99": { - "describe": { - "columns": [ - { - "name": "nonce!", - "ordinal": 0, - "type_info": "Int8" - } - ], - "nullable": [ - true - ], - "parameters": { - "Left": [ - "Bytea", - "Int8" - ] - } - }, - "query": "\n SELECT\n nonce AS \"nonce!\"\n FROM\n transactions\n WHERE\n initiator_address = $1\n AND nonce >= $2\n AND is_priority = FALSE\n AND (\n miniblock_number IS NOT NULL\n OR error IS NULL\n )\n ORDER BY\n nonce\n " - }, - "fcddeb96dcd1611dedb2091c1be304e8a35fd65bf37e976b7106f57c57e70b9b": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Text", - "Int4", - "Text" - ] - } - }, - "query": "\n UPDATE gpu_prover_queue_fri\n SET\n instance_status = 'available',\n updated_at = NOW()\n WHERE\n instance_host = $1::TEXT::inet\n AND instance_port = $2\n AND instance_status = 'full'\n AND zone = $3\n " - }, - "fde16cd2d3de03f4b61625fa453a58f82acd817932415f04bcbd05442ad80c2b": { - "describe": { - "columns": [ - { - "name": "bytecode", - "ordinal": 0, - "type_info": "Bytea" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Bytea", - "Int8" - ] - } - }, - "query": "\n SELECT\n bytecode\n FROM\n factory_deps\n WHERE\n bytecode_hash = $1\n AND miniblock_number <= $2\n " - }, - "fdffa5841554286a924b217b5885d9ec9b3f628c3a4cf5e10580ea6e5e3a2429": { - "describe": { - "columns": [], - "nullable": [], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n UPDATE miniblocks\n SET\n l1_batch_number = $1\n WHERE\n l1_batch_number IS NULL\n " - }, - "fe501f86f4bf6c5b8ccc2e039a4eb09b538a67d1c39fda052c4f4ddb23ce0084": { - "describe": { - "columns": [ - { - "name": "l2_to_l1_logs", - "ordinal": 0, - "type_info": "ByteaArray" - } - ], - "nullable": [ - false - ], - "parameters": { - "Left": [ - "Int8" - ] - } - }, - "query": "\n SELECT\n l2_to_l1_logs\n FROM\n l1_batches\n WHERE\n number = $1\n " - } -} \ No newline at end of file diff --git a/core/lib/dal/src/blocks_dal.rs b/core/lib/dal/src/blocks_dal.rs index 239f9074800e..07992d1e937c 100644 --- a/core/lib/dal/src/blocks_dal.rs +++ b/core/lib/dal/src/blocks_dal.rs @@ -2234,7 +2234,7 @@ impl BlocksDal<'_, '_> { WHERE number = $1 "#, - l1_batch_number.0 as u32 + l1_batch_number.0 as i32 ) .fetch_optional(self.storage.conn()) .await? @@ -2254,7 +2254,7 @@ impl BlocksDal<'_, '_> { WHERE number = $1 "#, - miniblock_number.0 as u32 + miniblock_number.0 as i32 ) .fetch_optional(self.storage.conn()) .await? diff --git a/core/lib/dal/src/eth_sender_dal.rs b/core/lib/dal/src/eth_sender_dal.rs index a524ae5694c7..5e490e6590a2 100644 --- a/core/lib/dal/src/eth_sender_dal.rs +++ b/core/lib/dal/src/eth_sender_dal.rs @@ -237,7 +237,7 @@ impl EthSenderDal<'_, '_> { RETURNING id "#, - eth_tx_id as u32, + eth_tx_id as i32, base_fee_per_gas, priority_fee_per_gas, tx_hash, diff --git a/core/lib/dal/src/snapshots_creator_dal.rs b/core/lib/dal/src/snapshots_creator_dal.rs index 15e7f0aae525..9267470878e1 100644 --- a/core/lib/dal/src/snapshots_creator_dal.rs +++ b/core/lib/dal/src/snapshots_creator_dal.rs @@ -93,7 +93,7 @@ impl SnapshotsCreatorDal<'_, '_> { ), value: H256::from_slice(&row.value), l1_batch_number_of_initial_write: L1BatchNumber(row.l1_batch_number as u32), - enumeration_index: row.index.unwrap() as u64, + enumeration_index: row.index as u64, }) .collect(); Ok(storage_logs) diff --git a/core/lib/types/Cargo.toml b/core/lib/types/Cargo.toml index 594160c27287..92c04a413765 100644 --- a/core/lib/types/Cargo.toml +++ b/core/lib/types/Cargo.toml @@ -26,7 +26,7 @@ zksync_protobuf = { version = "0.1.0", git = "https://github.com/matter-labs/era anyhow = "1.0.75" chrono = { version = "0.4", features = ["serde"] } -num = { version = "0.3.1", features = ["serde"] } +num = { version = "0.4.0", features = ["serde"] } once_cell = "1.7" rlp = "0.5" serde = "1.0.90" diff --git a/core/lib/utils/Cargo.toml b/core/lib/utils/Cargo.toml index 64b100d257ec..5561f9b36da0 100644 --- a/core/lib/utils/Cargo.toml +++ b/core/lib/utils/Cargo.toml @@ -14,8 +14,8 @@ zksync_basic_types = { path = "../../lib/basic_types" } zk_evm = { git = "https://github.com/matter-labs/era-zk_evm.git", tag = "v1.3.3-rc2" } vlog = { path = "../../lib/vlog" } -num = { version = "0.3.1", features = ["serde"] } -bigdecimal = { version = "0.2.2", features = ["serde"] } +bigdecimal = { version = "0.3.0", features = ["serde"] } +num = { version = "0.4.0", features = ["serde"] } serde = { version = "1.0", features = ["derive"] } tokio = { version = "1", features = ["time"] } tracing = "0.1" diff --git a/core/lib/web3_decl/Cargo.toml b/core/lib/web3_decl/Cargo.toml index f88ae989fecf..df82880d3c85 100644 --- a/core/lib/web3_decl/Cargo.toml +++ b/core/lib/web3_decl/Cargo.toml @@ -15,7 +15,7 @@ serde = "1.0" serde_json = "1.0" rlp = "0.5.0" thiserror = "1.0" -bigdecimal = { version = "0.2.2", features = ["serde"] } +bigdecimal = { version = "0.3.0", features = ["serde"] } jsonrpsee = { version = "0.21.0", default-features = false, features = [ "macros", ] } diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 9a86c4e1a1c9..67a26f67e5bc 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -62,7 +62,7 @@ async-trait = "0.1" bitflags = "1.3.2" num = { version = "0.3.1", features = ["serde"] } -bigdecimal = { version = "0.2.2", features = ["serde"] } +bigdecimal = { version = "0.3.0", features = ["serde"] } reqwest = { version = "0.11", features = ["blocking", "json"] } hex = "0.4" lru = { version = "0.12.1", default-features = false } 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 895563f1001e..877714fb3ee1 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/mod.rs @@ -433,7 +433,7 @@ impl FullApiParams { if matches!(transport, ApiTransport::WebSocket(_)) && self.namespaces.contains(&Namespace::Pubsub) { - let mut pub_sub = EthSubscribe::new(); + let mut pub_sub = EthSubscribe::new(runtime.handle().clone()); if let Some(sender) = &self.optional.pub_sub_events_sender { pub_sub.set_events_sender(sender.clone()); } diff --git a/core/lib/zksync_core/src/api_server/web3/pubsub.rs b/core/lib/zksync_core/src/api_server/web3/pubsub.rs index 07a5eeb64af2..2e5adfe5e1cc 100644 --- a/core/lib/zksync_core/src/api_server/web3/pubsub.rs +++ b/core/lib/zksync_core/src/api_server/web3/pubsub.rs @@ -200,10 +200,11 @@ pub(super) struct EthSubscribe { transactions: broadcast::Sender>, logs: broadcast::Sender>, events_sender: Option>, + handle: tokio::runtime::Handle, } impl EthSubscribe { - pub fn new() -> Self { + pub fn new(handle: tokio::runtime::Handle) -> Self { let (blocks, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); let (transactions, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); let (logs, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); @@ -213,6 +214,7 @@ impl EthSubscribe { transactions, logs, events_sender: None, + handle, } } @@ -321,7 +323,7 @@ impl EthSubscribe { return; }; let blocks_rx = self.blocks.subscribe(); - tokio::spawn(Self::run_subscriber( + self.handle.spawn(Self::run_subscriber( sink, SubscriptionType::Blocks, blocks_rx, @@ -335,7 +337,7 @@ impl EthSubscribe { return; }; let transactions_rx = self.transactions.subscribe(); - tokio::spawn(Self::run_subscriber( + self.handle.spawn(Self::run_subscriber( sink, SubscriptionType::Txs, transactions_rx, @@ -355,7 +357,7 @@ impl EthSubscribe { return; }; let logs_rx = self.logs.subscribe(); - tokio::spawn(Self::run_subscriber( + self.handle.spawn(Self::run_subscriber( sink, SubscriptionType::Logs, logs_rx, @@ -369,7 +371,7 @@ impl EthSubscribe { return; }; - tokio::spawn(async move { + self.handle.spawn(async move { sink.send_timeout( SubscriptionMessage::from_json(&PubSubResult::Syncing(false)).unwrap(), SUBSCRIPTION_SINK_SEND_TIMEOUT, @@ -406,7 +408,9 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = tokio::spawn(notifier.notify_blocks(stop_receiver.clone())); + let notifier_task = self + .handle + .spawn(notifier.notify_blocks(stop_receiver.clone())); notifier_tasks.push(notifier_task); let notifier = PubSubNotifier { @@ -415,7 +419,9 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = tokio::spawn(notifier.notify_txs(stop_receiver.clone())); + let notifier_task = self + .handle + .spawn(notifier.notify_txs(stop_receiver.clone())); notifier_tasks.push(notifier_task); let notifier = PubSubNotifier { @@ -424,7 +430,7 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = tokio::spawn(notifier.notify_logs(stop_receiver)); + let notifier_task = self.handle.spawn(notifier.notify_logs(stop_receiver)); notifier_tasks.push(notifier_task); notifier_tasks diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs index 57664629c722..0c46064637c2 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs @@ -65,16 +65,19 @@ impl ApiServerHandles { pub(crate) async fn shutdown(self) { let stop_server = async { for task in self.tasks { - // FIXME(PLA-481): avoid these errors (by spawning notifier tasks on server runtime?) - if let Err(err) = task.await.expect("Server panicked") { - let err = err.root_cause().to_string(); - assert!(err.contains("Tokio 1.x context was found")); - } + let task_result = task.await.unwrap_or_else(|err| { + if err.is_cancelled() { + Ok(()) + } else { + panic!("Server panicked: {err:?}"); + } + }); + task_result.expect("Server task returned an error"); } }; tokio::time::timeout(TEST_TIMEOUT, stop_server) .await - .unwrap(); + .expect(format!("panicking at {}", chrono::Utc::now()).as_str()); } } diff --git a/docker/external-node/Dockerfile b/docker/external-node/Dockerfile index fd1ece786d43..c21f00daad2c 100644 --- a/docker/external-node/Dockerfile +++ b/docker/external-node/Dockerfile @@ -18,7 +18,7 @@ WORKDIR /usr/src/zksync COPY . . RUN cargo build --release -RUN cargo install sqlx-cli --version 0.5.13 +RUN cargo install sqlx-cli --version 0.7.3 FROM debian:bookworm-slim diff --git a/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile b/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile index 9aa7a2b00679..a6e7d6fc3f28 100644 --- a/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile +++ b/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile @@ -72,7 +72,7 @@ RUN echo "deb http://packages.cloud.google.com/apt cloud-sdk main" > /etc/apt/so RUN wget -c -O - https://sh.rustup.rs | bash -s -- -y RUN rustup install nightly-2023-08-21 RUN rustup default stable -RUN cargo install --version=0.5.13 sqlx-cli +RUN cargo install --version=0.7.3 sqlx-cli RUN cargo install cargo-nextest # Copy compiler (both solc and zksolc) binaries diff --git a/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile b/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile index ed10b2529740..9efe6e579c39 100644 --- a/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile +++ b/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile @@ -70,7 +70,7 @@ RUN echo "deb http://packages.cloud.google.com/apt cloud-sdk main" > /etc/apt/so RUN wget -c -O - https://sh.rustup.rs | bash -s -- -y RUN rustup install nightly-2023-08-21 RUN rustup default stable -RUN cargo install --version=0.5.13 sqlx-cli +RUN cargo install --version=0.7.3 sqlx-cli RUN cargo install cargo-nextest # Copy compiler (both solc and zksolc) binaries diff --git a/docker/zk-environment/Dockerfile b/docker/zk-environment/Dockerfile index f86aeaddb110..b0d3ef13dcf0 100644 --- a/docker/zk-environment/Dockerfile +++ b/docker/zk-environment/Dockerfile @@ -103,7 +103,7 @@ RUN echo "deb [arch=${ARCH}] http://packages.cloud.google.com/apt cloud-sdk main RUN wget -c -O - https://sh.rustup.rs | bash -s -- -y && \ rustup default stable -RUN cargo install --version=0.5.13 sqlx-cli +RUN cargo install --version=0.7.3 sqlx-cli RUN cargo install cargo-nextest # Copy compiler (both solc and zksolc) binaries diff --git a/docs/guides/setup-dev.md b/docs/guides/setup-dev.md index f17e267ac53e..7fadbf37eec6 100644 --- a/docs/guides/setup-dev.md +++ b/docs/guides/setup-dev.md @@ -22,7 +22,7 @@ npm install -g yarn yarn set version 1.22.19 # SQL tools -cargo install sqlx-cli --version 0.5.13 +cargo install sqlx-cli --version 0.7.3 # Stop default postgres (as we'll use the docker one) sudo systemctl stop postgresql # Start docker. @@ -217,7 +217,7 @@ SQLx is a Rust library we use to interact with Postgres, and its CLI is used to features of the library. ```bash -cargo install sqlx-cli --version 0.5.13 +cargo install sqlx-cli --version 0.7.3 ``` ## Solidity compiler `solc` From c00fc48c9276dc00277c7518466ddeb5f52e7a34 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Mon, 8 Jan 2024 04:23:56 -0600 Subject: [PATCH 31/49] fix: fixes markdown table formatting in `docs/specs/zk_evm/vm_specification/compiler/overview.md#instruction-set` (#829) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Fixes markdown table formatting ## Why ❔ - The formatting of the table was unreadable - The prettier ignore is used to prevent zk fmt from disrupting markdown **Before:** ![Screenshot 2024-01-07 at 6 09 40 PM](https://github.com/matter-labs/zksync-era/assets/29983536/8e6a3bae-6aff-4c57-a4b4-62e9fc6c4a6a) **After:** ![Screenshot 2024-01-07 at 6 10 16 PM](https://github.com/matter-labs/zksync-era/assets/29983536/b091a048-14cb-4e91-a33f-cd9f183ab429) ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .../vm_specification/compiler/overview.md | 163 +++++++++--------- 1 file changed, 82 insertions(+), 81 deletions(-) diff --git a/docs/specs/zk_evm/vm_specification/compiler/overview.md b/docs/specs/zk_evm/vm_specification/compiler/overview.md index ec949c297f3d..62652e9447c5 100644 --- a/docs/specs/zk_evm/vm_specification/compiler/overview.md +++ b/docs/specs/zk_evm/vm_specification/compiler/overview.md @@ -38,87 +38,88 @@ The table below describes the scheme of translation Yul and EVMLA to EraVM bytec At the moment it does not explain much of the LLVM IR and assembly aspects, but mainly focus on the EVM-equivalence for the sake of assisting the upcoming audit. -| Yul name | EVMLA name | Descriptive keywords | Input/output | System contract usage | Front end notes | -| ---------------------- | ----------------- | -------------------------- | ------------------------------------------------------------------------------------------------------------ | ---------------------------------------------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------- | -| add | ADD | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | -| sub | SUB | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | -| mul | MUL | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | -| div | DIV | arithmetic, binary | Stack: 2 inputs, 1 output | - | x / 0. Division by 0 returns 0. | -| sdiv | SDIV | arithmetic, binary, signed | Stack: 2 inputs, 1 output | - | x / 0. Division by 0 returns 0. -(2^255) / (-1). In case of overflow the first operand is returned. | -| mod | MOD | arithmetic, binary | Stack: 2 inputs, 1 output | - | x % 0. Remainder of division by 0 returns 0. | -| smod | SMOD | arithmetic, binary, signed | Stack: 2 inputs, 1 output | - | x % 0. Remainder of division by 0 returns 0. | -| exp | EXP | arithmetic, binary | Stack: 2 inputs, 1 output | - | Unfolds into the binary exponentiation algorithm. | -| lt | LT | logical, binary | Stack: 2 inputs, 1 output | - | - | -| slt | SLT | logical, binary, signed | Stack: 2 inputs, 1 output | - | - | -| gt | GT | logical, binary | Stack: 2 inputs, 1 output | - | - | -| sgt | SGT | logical, binary, signed | Stack: 2 inputs, 1 output | - | - | -| eq | EQ | logical, binary | Stack: 2 inputs, 1 output | - | - | -| iszero | ISZERO | logical, unary | Stack: 1 input, 1 output | - | - | -| or | OR | bitwise, binary | Stack: 2 inputs, 1 output | - | - | -| xor | XOR | bitwise, binary | Stack: 2 inputs, 1 output | - | - | -| and | AND | bitwise, binary | Stack: 2 inputs, 1 output | - | - | -| not | NOT | bitwise, unary | Stack: 1 input, 1 output | - | - | -| shl | SHL | bitwise, binary | Stack: 2 inputs, 1 output | - | x << N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero is returned. | -| shr | SHR | bitwise, binary | Stack: 2 inputs, 1 output | - | x >> N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero is returned. | -| sar | SAR | bitwise, binary, signed | Stack: 2 inputs, 1 output | - | x >> N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero or minus one is returned, depending on the sign bit. | -| signextend | SIGNEXTEND | bitwise, binary | Stack: 2 inputs, 1 output | - | See the LLVM runtime section below. | -| byte | BYTE | bitwise, binary | Stack: 2 inputs, 1 output | - | - | -| addmod | ADDMOD | modular, ternary | Stack: 3 inputs, 1 output | - | - | -| mulmod | MULMOD | modular, ternary | Stack: 3 inputs, 1 output | - | - | -| - | PUSH | stack | Stack: 1 output | - | - | -| - | PUSH{1..32} | stack | Stack: 1 output | - | - | -| - | PUSHSIZE | stack | Stack: 1 output | - | Pushes 0. | -| - | PUSH [tag] | stack | Stack: 1 output (compile time) | - | - | -| - | PUSH data | stack | Stack: 1 output | - | Unfolded into several cells if the length is more than 32 bytes. | -| - | Tag | stack | - | - | - | -| pop | POP | stack | Stack: 1 input | - | - | -| - | DUP{1..16} | stack | Stack: 1 output | - | - | -| - | SWAP{1..16} | stack | Stack: 1 swap | - | - | -| - | JUMP | stack | Stack: 1 input (compile time) | - | Expects a compile-time known tag and generates an unconditional jump to the statically known block of LLVM IR. | -| - | JUMPI | stack | Stack: 2 inputs (1 in compile time) | - | Expects a compile-time known tag and generates a conditional jump to the statically known block of LLVM IR. | -| - | JUMPDEST | stack | - | - | Unused by the static analyzer and totally discarded. | -| mload` | MLOAD | heap | Stack: 1 input. Heap: read 32 bytes. Stack: 1 output | - | - | -| mstore | MSTORE | heap | Stack: 2 inputs. Heap: write 32 bytes | - | - | -| mstore8 | MSTORE8 | heap | Stack: 2 inputs. Heap: write 1 byte | - | - | -| msize | MSIZE | heap, context | Context: 1 request. Stack: 1 output | - | - | -| memoryguard | | heap | Stack: 1 output | - | - | -| sload | SLOAD | storage | Stack: 1 input. Storage: read 1 slot. Stack: 1 output | - | - | -| sstore | SSTORE | storage | Stack: 2 inputs. Storage: write 1 slot | - | - | -| loadimmutable | PUSHIMMUTABLE | immutable, heap | Stack: 1 input (compile time). Heap: read 32 bytes (deploy code). System contracts: 1 request (runtime code) | ImmutableSimulator | - | -| setimmutable | ASSIGNIMMUTABLE | immutable, heap | Stack: 3 inputs (1 in compile time). Heap: write 32 bytes (deploy code) | ImmutableSimulator | No-op in the runtime code. | -| log0 | LOG0 | event | Stack: 2 inputs. Heap: read N bytes. VM: write 1 event (0 topics, N bytes) | - | - | -| log1 | LOG1 | event | Stack: 3 inputs. Heap: read N bytes. VM: write 1 event (1 topics, N bytes) | - | - | -| log2 | LOG2 | event | Stack: 4 inputs. Heap: read N bytes. VM: write 1 event (2 topics, N bytes) | - | - | -| log3 | LOG3 | event | Stack: 5 inputs. Heap: read N bytes. VM: write 1 event (3 topics, N bytes) | - | - | -| log4 | LOG4 | event | Stack: 6 inputs. Heap: read N bytes. VM: write 1 event (4 topics, N bytes) | - | - | -| calldataload | CALLDATALOAD | calldata | Stack: 1 input. Calldata: read 32 bytes. Stack: 1 output | - | 0 in deploy code. | -| calldatacopy | CALLDATACOPY | calldata, heap | Stack: 3 inputs. Calldata: read N bytes. Heap: write N bytes | - | Generated by solc in the runtime code only. Copies 0 in deploy code. | -| calldatasize | CALLDATASIZE | calldata | Stack: 1 output | - | 0 in deploy code. | -| codecopy | CODECOPY | calldata | Stack: 3 inputs. Calldata: read N bytes. Heap: write N bytes | - | Generated by solc in the deploy code only, but is treated as CALLDATACOPY, since the constructor arguments are calldata in zkSync 2.0. Compile time error in Yul runtime code. Copies 0 in EVMLA runtime code. | -| codesize | CODESIZE | calldata | Stack: 1 output | - | - | -| returndatacopy | RETURNDATACOPY | return data, heap | Stack: 3 inputs. Return data: read N bytes. Heap: write N bytes | - | - | -| returndatasize | RETURNDATASIZE | return data | Stack: 1 output | - | - | -| keccak256 | KECCAK256 | SHA3 | hash | Heap: read N bytes. System contracts: 1 request. Stack: 1 output | Keccak256 | Calls a system contract which may revert if the input is too long. In this case the error is bubbled-up and the calling contract also reverts. | -| call | CALL | call | Stack: 7 inputs, 1 output. Heap: read N bytes, write M bytes | MsgValueSimulator | - | -| staticcall | STATICCALL | call | Stack: 6 inputs, 1 output. Heap: read N bytes, write M bytes | Ecrecover, SHA256 | - | -| delegatecall | DELEGATECALL | call | Stack: 6 inputs, 1 output. Heap: read N bytes, write M bytes | - | - | -| linkersymbol | PUSHLIB | library | Stack: 1 output | - | - | -| - | PUSHDEPLOYADDRESS | library | Context: 1 request. Stack: 1 output | - | - | -| create | CREATE | create | | ContractDeployer, MsgValueSimulator | - | -| create2 | CREATE2 | create | | ContractDeployer, MsgValueSimulator | - | -| datasize | PUSH #[$] | create | Stack: 1 output | - | - | -| dataoffset | PUSH [$] | create | Stack: 1 output | - | - | -| datacopy | CODECOPY | create, heap | Stack: 3 inputs. Heap: write 32 bytes | - | - | -| return | RETURN | return, positive | Stack: 2 inputs | - | In the deploy code the auxiliary heap is used to avoid conflicts with memory allocated by the Yul generator. | -| stop | STOP | return, positive | - | - | - | -| revert | REVERT | return, negative | Stack: 2 inputs | - | - | -| invalid | INVALID | return, negative | - | - | - | -| address | ADDRESS | context, transaction | Context: 1 request. Stack: 1 output | - | - | -| caller | CALLER | context, transaction | Context: 1 request. Stack: 1 output | - | - | -| gas | GAS | context, transaction | Context: 1 request. Stack: 1 output | - | - | -| chainid | CHAINID | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | -| gasprice | GASPRICE | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | -| origin | ORIGIN | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | + +| Yul name | EVMLA name | Descriptive keywords | Input/output | System contract usage | Front end notes | +|------------------|-----------------|------------------------------|----------------------------------------------------|--------------------------|----------------------------------------------------------------------------------------------------------| +| add | ADD | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | +| sub | SUB | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | +| mul | MUL | arithmetic, binary | Stack: 2 inputs, 1 output | - | - | +| div | DIV | arithmetic, binary | Stack: 2 inputs, 1 output | - | x / 0. Division by 0 returns 0. | +| sdiv | SDIV | arithmetic, binary, signed | Stack: 2 inputs, 1 output | - | x / 0. Division by 0 returns 0. -(2^255) / (-1). In case of overflow the first operand is returned. | +| mod | MOD | arithmetic, binary | Stack: 2 inputs, 1 output | - | x % 0. Remainder of division by 0 returns 0. | +| smod | SMOD | arithmetic, binary, signed | Stack: 2 inputs, 1 output | - | x % 0. Remainder of division by 0 returns 0. | +| exp | EXP | arithmetic, binary | Stack: 2 inputs, 1 output | - | Unfolds into the binary exponentiation algorithm. | +| lt | LT | logical, binary | Stack: 2 inputs, 1 output | - | - | +| slt | SLT | logical, binary, signed | Stack: 2 inputs, 1 output | - | - | +| gt | GT | logical, binary | Stack: 2 inputs, 1 output | - | - | +| sgt | SGT | logical, binary, signed | Stack: 2 inputs, 1 output | - | - | +| eq | EQ | logical, binary | Stack: 2 inputs, 1 output | - | - | +| iszero | ISZERO | logical, unary | Stack: 1 input, 1 output | - | - | +| or | OR | bitwise, binary | Stack: 2 inputs, 1 output | - | - | +| xor | XOR | bitwise, binary | Stack: 2 inputs, 1 output | - | - | +| and | AND | bitwise, binary | Stack: 2 inputs, 1 output | - | - | +| not | NOT | bitwise, unary | Stack: 1 input, 1 output | - | - | +| shl | SHL | bitwise, binary | Stack: 2 inputs, 1 output | - | x << N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero is returned. | +| shr | SHR | bitwise, binary | Stack: 2 inputs, 1 output | - | x >> N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero is returned. | +| sar | SAR | bitwise, binary, signed | Stack: 2 inputs, 1 output | - | x >> N, N > 255. Shifting by more than a word size is a UB in LLVM. This case is checked explicitly and zero or minus one is returned, depending on the sign bit. | +| signextend | SIGNEXTEND | bitwise, binary | Stack: 2 inputs, 1 output | - | See the LLVM runtime section below. | +| byte | BYTE | bitwise, binary | Stack: 2 inputs, 1 output | - | - | +| addmod | ADDMOD | modular, ternary | Stack: 3 inputs, 1 output | - | - | +| mulmod | MULMOD | modular, ternary | Stack: 3 inputs, 1 output | - | - | +| - | PUSH | stack | Stack: 1 output | - | - | +| - | PUSH{1..32} | stack | Stack: 1 output | - | - | +| - | PUSHSIZE | stack | Stack: 1 output | - | Pushes 0. | +| - | PUSH [tag] | stack | Stack: 1 output (compile time) | - | - | +| - | PUSH data | stack | Stack: 1 output | - | Unfolded into several cells if the length is more than 32 bytes. | +| - | Tag | stack | - | - | - | +| pop | POP | stack | Stack: 1 input | - | - | +| - | DUP{1..16} | stack | Stack: 1 output | - | - | +| - | SWAP{1..16} | stack | Stack: 1 swap | - | - | +| - | JUMP | stack | Stack: 1 input (compile time) | - | Expects a compile-time known tag and generates an unconditional jump to the statically known block of LLVM IR. | +| - | JUMPI | stack | Stack: 2 inputs (1 in compile time) | - | Expects a compile-time known tag and generates a conditional jump to the statically known block of LLVM IR. | +| - | JUMPDEST | stack | - | - | Unused by the static analyzer and totally discarded. | +| mload` | MLOAD | heap | Stack: 1 input. Heap: read 32 bytes. Stack: 1 output | - | - | +| mstore | MSTORE | heap | Stack: 2 inputs. Heap: write 32 bytes | - | - | +| mstore8 | MSTORE8 | heap | Stack: 2 inputs. Heap: write 1 byte | - | - | +| msize | MSIZE | heap, context | Context: 1 request. Stack: 1 output | - | - | +| memoryguard | | heap | Stack: 1 output | - | - | +| sload | SLOAD | storage | Stack: 1 input. Storage: read 1 slot. Stack: 1 output | - | - | +| sstore | SSTORE | storage | Stack: 2 inputs. Storage: write 1 slot | - | - | +| loadimmutable | PUSHIMMUTABLE | immutable, heap | Stack: 1 input (compile time). Heap: read 32 bytes (deploy code). System contracts: 1 request (runtime code) | ImmutableSimulator | - | +| setimmutable | ASSIGNIMMUTABLE | immutable, heap | Stack: 3 inputs (1 in compile time). Heap: write 32 bytes (deploy code) | ImmutableSimulator | No-op in the runtime code. | +| log0 | LOG0 | event | Stack: 2 inputs. Heap: read N bytes. VM: write 1 event (0 topics, N bytes) | - | - | +| log1 | LOG1 | event | Stack: 3 inputs. Heap: read N bytes. VM: write 1 event (1 topics, N bytes) | - | - | +| log2 | LOG2 | event | Stack: 4 inputs. Heap: read N bytes. VM: write 1 event (2 topics, N bytes) | - | - | +| log3 | LOG3 | event | Stack: 5 inputs. Heap: read N bytes. VM: write 1 event (3 topics, N bytes) | - | - | +| log4 | LOG4 | event | Stack: 6 inputs. Heap: read N bytes. VM: write 1 event (4 topics, N bytes) | - | - | +| calldataload | CALLDATALOAD | calldata | Stack: 1 input. Calldata: read 32 bytes. Stack: 1 output | - | 0 in deploy code. | +| calldatacopy | CALLDATACOPY | calldata, heap | Stack: 3 inputs. Calldata: read N bytes. Heap: write N bytes | - | Generated by solc in the runtime code only. Copies 0 in deploy code. | +| calldatasize | CALLDATASIZE | calldata | Stack: 1 output | - | 0 in deploy code. | +| codecopy | CODECOPY | calldata | Stack: 3 inputs. Calldata: read N bytes. Heap: write N bytes | - | Generated by solc in the deploy code only, but is treated as CALLDATACOPY, since the constructor arguments are calldata in zkSync 2.0. Compile time error in Yul runtime code. Copies 0 in EVMLA runtime code. | +| codesize | CODESIZE | calldata | Stack: 1 output | - | - | +| returndatacopy | RETURNDATACOPY | return data, heap | Stack: 3 inputs. Return data: read N bytes. Heap: write N bytes | - | - | +| returndatasize | RETURNDATASIZE | return data | Stack: 1 output | - | - | +| keccak256 | KECCAK256 | SHA3 | hash | Heap: read N bytes. System contracts: 1 request. Stack: 1 output | Keccak256 Calls a system contract which may revert if the input is too long. In this case the error is bubbled-up and the calling contract also reverts. | +| call | CALL | call | Stack: 7 inputs, 1 output. Heap: read N bytes, write M bytes | MsgValueSimulator | - | +| staticcall | STATICCALL | call | Stack: 6 inputs, 1 output. Heap: read N bytes, write M bytes | Ecrecover, SHA256 | - | +| delegatecall | DELEGATECALL | call | Stack: 6 inputs, 1 output. Heap: read N bytes, write M bytes | - | - | +| linkersymbol | PUSHLIB | library | Stack: 1 output | - | - | +| - | PUSHDEPLOYADDRESS | library | Context: 1 request. Stack: 1 output | - | - | +| create | CREATE | create | - | ContractDeployer, MsgValueSimulator | - | +| create2 | CREATE2 | create | - | ContractDeployer, MsgValueSimulator | - | +| datasize | PUSH #[$] | create | Stack: 1 output | - | - | +| dataoffset | PUSH [$] | create | Stack: 1 output | - | - | +| datacopy | CODECOPY | create, heap | Stack: 3 inputs. Heap: write 32 bytes | - | - | +| return | RETURN | return, positive | Stack: 2 inputs | - | In the deploy code the auxiliary heap is used to avoid conflicts with memory allocated by the Yul generator. | +| stop | STOP | return, positive | - | - | - | +| revert | REVERT | return, negative | Stack: 2 inputs | - | - | +| invalid | INVALID | return, negative | - | - | - | +| address | ADDRESS | context, transaction | Context: 1 request. Stack: 1 output | - | - | +| caller | CALLER | context, transaction | Context: 1 request. Stack: 1 output | - | - | +| gas | GAS | context, transaction | Context: 1 request. Stack: 1 output | - | - | +| chainid | CHAINID | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | +| gasprice | GASPRICE | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | +| origin | ORIGIN | context, transaction | System contracts: 1 request. Stack: 1 output | ContractContext | - | | | callvalue | CALLVALUE | context, transaction | Context: 1 request. Stack: 1 output | - | - | | blockhash | BLOCKHASH | context, block | Stack: 1 input. System contracts: 1 request. Stack: 1 output | ContractContext | - | | gaslimit | GASLIMIT | context, block | System contracts: 1 request. Stack: 1 output | ContractContext | - | From be99118e23083803870fe3b4560fca5e7b9f6dd2 Mon Sep 17 00:00:00 2001 From: Aleksandr Stepanov Date: Mon, 8 Jan 2024 13:26:45 +0200 Subject: [PATCH 32/49] ci: Fixed zk-environment workflow (#831) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ ## Why ❔ ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .github/workflows/zk-environment-publish.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/zk-environment-publish.yml b/.github/workflows/zk-environment-publish.yml index 44768241ccda..fc162e78714c 100644 --- a/.github/workflows/zk-environment-publish.yml +++ b/.github/workflows/zk-environment-publish.yml @@ -5,14 +5,14 @@ on: branches: - main paths: - - "docker/zk-environment/*" - - ".github/workflows/zk-environment.publish.yml" + - "docker/zk-environment/**" + - ".github/workflows/zk-environment-publish.yml" pull_request: branches: - main paths: - - "docker/zk-environment/*" - - ".github/workflows/zk-environment.publish.yml" + - "docker/zk-environment/**" + - ".github/workflows/zk-environment-publish.yml" concurrency: group: ${{ github.workflow }}-${{ github.event_name == 'pull_request' && github.event.pull_request.number || github.sha }} From 1b579848130c1566ec74b6910ab4925873af5051 Mon Sep 17 00:00:00 2001 From: Aleksandr Stepanov Date: Mon, 8 Jan 2024 16:07:30 +0200 Subject: [PATCH 33/49] ci: Fixed zk-environment workflow typos and npm issues. (#833) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ ## Why ❔ ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .github/workflows/zk-environment-publish.yml | 8 ++++---- .../zk-environment/20.04_amd64_cuda_11_8.Dockerfile | 11 +++++++---- .../zk-environment/20.04_amd64_cuda_12_0.Dockerfile | 9 ++++++--- docker/zk-environment/Dockerfile | 2 +- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/.github/workflows/zk-environment-publish.yml b/.github/workflows/zk-environment-publish.yml index fc162e78714c..0551b15aac50 100644 --- a/.github/workflows/zk-environment-publish.yml +++ b/.github/workflows/zk-environment-publish.yml @@ -23,7 +23,7 @@ jobs: outputs: zk_environment: ${{ steps.changed-files-yaml.outputs.zk_env_any_changed }} zk_environment_cuda_11_8: ${{ steps.changed-files-yaml.outputs.zk_env_cuda_11_8_any_changed }} - zk_environment_cuda_12: ${{ steps.changed-files-yaml.outputs.zk_env_cuda_12_any_changed }} + zk_environment_cuda_12_0: ${{ steps.changed-files-yaml.outputs.zk_env_cuda_12_any_changed }} runs-on: ubuntu-latest steps: - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 @@ -37,13 +37,13 @@ jobs: files_yaml: | zk_env: - docker/zk-environment/Dockerfile - - .github/workflows/zk-environment.publish.yml + - .github/workflows/zk-environment-publish.yml zk_env_cuda_11_8: - docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile - - .github/workflows/zk-environment.publish.yml + - .github/workflows/zk-environment-publish.yml zk_env_cuda_12: - docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile - - .github/workflows/zk-environment.publish.yml + - .github/workflows/zk-environment-publish.yml get_short_sha: if: needs.changed_files.outputs.zk_environment == 'true' diff --git a/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile b/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile index a6e7d6fc3f28..bd77e680d5fc 100644 --- a/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile +++ b/docker/zk-environment/20.04_amd64_cuda_11_8.Dockerfile @@ -50,10 +50,13 @@ RUN apt update; apt install -y docker-ce-cli # Configurate git to fetch submodules correctly (https://stackoverflow.com/questions/38378914/how-to-fix-git-error-rpc-failed-curl-56-gnutls) RUN git config --global http.postBuffer 1048576000 -# Install node and yarn -RUN wget -c -O - https://deb.nodesource.com/setup_18.x | bash - -RUN apt-get install -y nodejs -RUN npm install -g yarn +# Install Node and yarn +ENV NODE_MAJOR=18 +RUN mkdir -p /etc/apt/keyrings && \ + wget -c -O - https://deb.nodesource.com/gpgkey/nodesource-repo.gpg.key | gpg --dearmor -o /etc/apt/keyrings/nodesource.gpg && \ + echo "deb [signed-by=/etc/apt/keyrings/nodesource.gpg] https://deb.nodesource.com/node_$NODE_MAJOR.x nodistro main" | tee /etc/apt/sources.list.d/nodesource.list && \ + apt-get update && apt-get install nodejs -y && \ + npm install -g yarn # Install Rust and required cargo packages ENV RUSTUP_HOME=/usr/local/rustup \ diff --git a/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile b/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile index 9efe6e579c39..d0bb05fed16e 100644 --- a/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile +++ b/docker/zk-environment/20.04_amd64_cuda_12_0.Dockerfile @@ -49,9 +49,12 @@ RUN apt update; apt install -y docker-ce-cli RUN git config --global http.postBuffer 1048576000 # Install node and yarn -RUN wget -c -O - https://deb.nodesource.com/setup_18.x | bash - -RUN apt-get install -y nodejs -RUN npm install -g yarn +ENV NODE_MAJOR=18 +RUN mkdir -p /etc/apt/keyrings && \ + wget -c -O - https://deb.nodesource.com/gpgkey/nodesource-repo.gpg.key | gpg --dearmor -o /etc/apt/keyrings/nodesource.gpg && \ + echo "deb [signed-by=/etc/apt/keyrings/nodesource.gpg] https://deb.nodesource.com/node_$NODE_MAJOR.x nodistro main" | tee /etc/apt/sources.list.d/nodesource.list && \ + apt-get update && apt-get install nodejs -y && \ + npm install -g yarn # Install Rust and required cargo packages ENV RUSTUP_HOME=/usr/local/rustup \ diff --git a/docker/zk-environment/Dockerfile b/docker/zk-environment/Dockerfile index b0d3ef13dcf0..d942fab7ece1 100644 --- a/docker/zk-environment/Dockerfile +++ b/docker/zk-environment/Dockerfile @@ -83,7 +83,7 @@ ENV NODE_MAJOR=18 RUN mkdir -p /etc/apt/keyrings && \ wget -c -O - https://deb.nodesource.com/gpgkey/nodesource-repo.gpg.key | gpg --dearmor -o /etc/apt/keyrings/nodesource.gpg && \ echo "deb [signed-by=/etc/apt/keyrings/nodesource.gpg] https://deb.nodesource.com/node_$NODE_MAJOR.x nodistro main" | tee /etc/apt/sources.list.d/nodesource.list && \ - apt-get update && apt-get install nodejs -y && \ + apt-get update && apt-get install nodejs npm -y && \ npm install -g yarn # Install Rust and required cargo packages From 0a010f0a6f6682cedc49cb12ab9f9dfcdbccf68e Mon Sep 17 00:00:00 2001 From: perekopskiy <53865202+perekopskiy@users.noreply.github.com> Date: Mon, 8 Jan 2024 17:52:41 +0200 Subject: [PATCH 34/49] feat(state-keeper): Reject transactions that fail to publish bytecodes (#832) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ State keeper and API reject transactions that fail to publish bytecodes ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/main.rs | 1 + core/lib/multivm/src/interface/traits/vm.rs | 10 ++- .../src/interface/types/errors/halt.rs | 4 ++ core/lib/multivm/src/versions/vm_1_3_2/vm.rs | 12 +++- .../src/versions/vm_boojum_integration/vm.rs | 12 +++- core/lib/multivm/src/versions/vm_latest/vm.rs | 12 +++- core/lib/multivm/src/versions/vm_m5/vm.rs | 7 +- core/lib/multivm/src/versions/vm_m6/vm.rs | 12 +++- .../src/versions/vm_refunds_enhancement/vm.rs | 12 +++- .../src/versions/vm_virtual_blocks/vm.rs | 12 +++- core/lib/multivm/src/vm_instance.rs | 14 ++-- .../src/api_server/execution_sandbox/error.rs | 3 + .../api_server/execution_sandbox/execute.rs | 17 +++-- .../src/api_server/tx_sender/mod.rs | 8 ++- .../src/api_server/tx_sender/result.rs | 3 + .../vm_interactions.rs | 2 + .../src/state_keeper/batch_executor/mod.rs | 69 ++++++++++++++++--- .../batch_executor/tests/tester.rs | 1 + .../src/state_keeper/io/mempool.rs | 2 +- core/lib/zksync_core/src/state_keeper/mod.rs | 1 + .../ts-integration/tests/contracts.test.ts | 24 +++++++ 21 files changed, 192 insertions(+), 46 deletions(-) diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index b20ab0a9fb81..7098ddcd1a6a 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -74,6 +74,7 @@ async fn build_state_keeper( save_call_traces, false, config.optional.enum_index_migration_chunk_size, + true, )); let main_node_url = config.required.main_node_url().unwrap(); diff --git a/core/lib/multivm/src/interface/traits/vm.rs b/core/lib/multivm/src/interface/traits/vm.rs index cfee13d2031b..1158588f8499 100644 --- a/core/lib/multivm/src/interface/traits/vm.rs +++ b/core/lib/multivm/src/interface/traits/vm.rs @@ -104,7 +104,10 @@ pub trait VmInterface { &mut self, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.inspect_transaction_with_bytecode_compression( Self::TracerDispatcher::default(), tx, @@ -118,7 +121,10 @@ pub trait VmInterface { tracer: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result; + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ); /// Record VM memory metrics. fn record_vm_memory_metrics(&self) -> VmMemoryMetrics; diff --git a/core/lib/multivm/src/interface/types/errors/halt.rs b/core/lib/multivm/src/interface/types/errors/halt.rs index c302467dacee..70de7548f14e 100644 --- a/core/lib/multivm/src/interface/types/errors/halt.rs +++ b/core/lib/multivm/src/interface/types/errors/halt.rs @@ -41,6 +41,7 @@ pub enum Halt { FailedToAppendTransactionToL2Block(String), VMPanic, TracerCustom(String), + FailedToPublishCompressedBytecodes, } impl Display for Halt { @@ -112,6 +113,9 @@ impl Display for Halt { Halt::ValidationOutOfGas => { write!(f, "Validation run out of gas") } + Halt::FailedToPublishCompressedBytecodes => { + write!(f, "Failed to publish compressed bytecodes") + } } } } diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm.rs index f0cf5d9c1aa2..50fa44380f58 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm.rs @@ -162,7 +162,10 @@ impl VmInterface for Vm { _tracer: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.last_tx_compressed_bytecodes = vec![]; let bytecodes = if with_compression { let deps = tx.execute.factory_deps.as_deref().unwrap_or_default(); @@ -209,9 +212,12 @@ impl VmInterface for Vm { .iter() .any(|info| !self.vm.is_bytecode_known(info)) { - Err(crate::interface::BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result.glue_into(), + ) } else { - Ok(result.glue_into()) + (Ok(()), result.glue_into()) } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs index a602ebbb9d71..425833bd910f 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/vm.rs @@ -127,13 +127,19 @@ impl VmInterface for Vm { tracer: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.push_transaction_with_compression(tx, with_compression); let result = self.inspect_inner(tracer, VmExecutionMode::OneTx); if self.has_unpublished_bytecodes() { - Err(BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result, + ) } else { - Ok(result) + (Ok(()), result) } } diff --git a/core/lib/multivm/src/versions/vm_latest/vm.rs b/core/lib/multivm/src/versions/vm_latest/vm.rs index 920c4fefaab6..6b571cd25c40 100644 --- a/core/lib/multivm/src/versions/vm_latest/vm.rs +++ b/core/lib/multivm/src/versions/vm_latest/vm.rs @@ -127,13 +127,19 @@ impl VmInterface for Vm { tracer: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.push_transaction_with_compression(tx, with_compression); let result = self.inspect_inner(tracer, VmExecutionMode::OneTx); if self.has_unpublished_bytecodes() { - Err(BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result, + ) } else { - Ok(result) + (Ok(()), result) } } diff --git a/core/lib/multivm/src/versions/vm_m5/vm.rs b/core/lib/multivm/src/versions/vm_m5/vm.rs index 037388d10612..08fa783cbdbe 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm.rs @@ -172,13 +172,16 @@ impl VmInterface for Vm { _tracer: Self::TracerDispatcher, tx: Transaction, _with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { crate::vm_m5::vm_with_bootloader::push_transaction_to_bootloader_memory( &mut self.vm, &tx, self.system_env.execution_mode.glue_into(), ); - Ok(self.execute(VmExecutionMode::OneTx)) + (Ok(()), self.execute(VmExecutionMode::OneTx)) } fn record_vm_memory_metrics(&self) -> VmMemoryMetrics { diff --git a/core/lib/multivm/src/versions/vm_m6/vm.rs b/core/lib/multivm/src/versions/vm_m6/vm.rs index 938b47784de9..420b5c8f9bf7 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm.rs @@ -179,7 +179,10 @@ impl VmInterface for Vm { _tracer: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.last_tx_compressed_bytecodes = vec![]; let bytecodes = if with_compression { let deps = tx.execute.factory_deps.as_deref().unwrap_or_default(); @@ -226,9 +229,12 @@ impl VmInterface for Vm { .iter() .any(|info| !self.vm.is_bytecode_exists(info)) { - Err(crate::interface::BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result.glue_into(), + ) } else { - Ok(result.glue_into()) + (Ok(()), result.glue_into()) } } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/vm.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/vm.rs index 678a467d4474..f1554ee17615 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/vm.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/vm.rs @@ -118,13 +118,19 @@ impl VmInterface for Vm { dispatcher: Self::TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.push_transaction_with_compression(tx, with_compression); let result = self.inspect(dispatcher, VmExecutionMode::OneTx); if self.has_unpublished_bytecodes() { - Err(BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result, + ) } else { - Ok(result) + (Ok(()), result) } } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/vm.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/vm.rs index ed05e9514753..3bb43669f008 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/vm.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/vm.rs @@ -118,13 +118,19 @@ impl VmInterface for Vm { tracer: TracerDispatcher, tx: Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { self.push_transaction_with_compression(tx, with_compression); let result = self.inspect_inner(tracer, VmExecutionMode::OneTx); if self.has_unpublished_bytecodes() { - Err(BytecodeCompressionError::BytecodeCompressionFailed) + ( + Err(BytecodeCompressionError::BytecodeCompressionFailed), + result, + ) } else { - Ok(result) + (Ok(()), result) } } diff --git a/core/lib/multivm/src/vm_instance.rs b/core/lib/multivm/src/vm_instance.rs index 88b6a5275122..ea3ad74f5302 100644 --- a/core/lib/multivm/src/vm_instance.rs +++ b/core/lib/multivm/src/vm_instance.rs @@ -5,8 +5,8 @@ use zksync_utils::bytecode::CompressedBytecodeInfo; use crate::{ glue::history_mode::HistoryMode, interface::{ - BootloaderMemory, CurrentExecutionState, FinishedL1Batch, L1BatchEnv, L2BlockEnv, - SystemEnv, VmExecutionMode, VmExecutionResultAndLogs, VmInterface, + BootloaderMemory, BytecodeCompressionError, CurrentExecutionState, FinishedL1Batch, + L1BatchEnv, L2BlockEnv, SystemEnv, VmExecutionMode, VmExecutionResultAndLogs, VmInterface, VmInterfaceHistoryEnabled, VmMemoryMetrics, }, tracers::TracerDispatcher, @@ -86,7 +86,10 @@ impl VmInterface for VmInstance { &mut self, tx: zksync_types::Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { dispatch_vm!(self.execute_transaction_with_bytecode_compression(tx, with_compression)) } @@ -96,7 +99,10 @@ impl VmInterface for VmInstance { dispatcher: Self::TracerDispatcher, tx: zksync_types::Transaction, with_compression: bool, - ) -> Result { + ) -> ( + Result<(), BytecodeCompressionError>, + VmExecutionResultAndLogs, + ) { dispatch_vm!(self.inspect_transaction_with_bytecode_compression( dispatcher.into(), tx, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/error.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/error.rs index c5928cfd8470..9d6d635a344c 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/error.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/error.rs @@ -64,6 +64,9 @@ impl From for SandboxExecutionError { Halt::ValidationOutOfGas => Self::AccountValidationFailed( "The validation of the transaction ran out of gas".to_string(), ), + Halt::FailedToPublishCompressedBytecodes => { + Self::UnexpectedVMBehavior("Failed to publish compressed bytecodes".to_string()) + } } } } diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs index 7e65cd9a8931..cbff25698cf7 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/execute.rs @@ -1,7 +1,7 @@ //! Implementation of "executing" methods, e.g. `eth_call`. use multivm::{ - interface::{TxExecutionMode, VmExecutionMode, VmExecutionResultAndLogs, VmInterface}, + interface::{TxExecutionMode, VmExecutionResultAndLogs, VmInterface}, tracers::StorageInvocations, vm_latest::constants::ETH_CALL_GAS_LIMIT, MultiVMTracer, @@ -94,7 +94,7 @@ pub(crate) async fn execute_tx_eth_call( // limiting the amount of gas the call can use. // We can't use `BLOCK_ERGS_LIMIT` here since the VM itself has some overhead. tx.common_data.fee.gas_limit = ETH_CALL_GAS_LIMIT.into(); - let (vm_result, _) = execute_tx_in_sandbox( + let (vm_result, _, _) = execute_tx_in_sandbox( vm_permit, shared_args, false, @@ -125,14 +125,14 @@ pub(crate) async fn execute_tx_in_sandbox( tx: Transaction, block_args: BlockArgs, custom_tracers: Vec, -) -> (VmExecutionResultAndLogs, TransactionExecutionMetrics) { +) -> (VmExecutionResultAndLogs, TransactionExecutionMetrics, bool) { let total_factory_deps = tx .execute .factory_deps .as_ref() .map_or(0, |deps| deps.len() as u16); - let execution_result = tokio::task::spawn_blocking(move || { + let (published_bytecodes, execution_result) = tokio::task::spawn_blocking(move || { let span = span!(Level::DEBUG, "execute_in_sandbox").entered(); let result = apply::apply_vm_in_sandbox( vm_permit, @@ -143,7 +143,6 @@ pub(crate) async fn execute_tx_in_sandbox( tx, block_args, |vm, tx| { - vm.push_transaction(tx); let storage_invocation_tracer = StorageInvocations::new(execution_args.missed_storage_invocation_limit); let custom_tracers: Vec<_> = custom_tracers @@ -151,7 +150,7 @@ pub(crate) async fn execute_tx_in_sandbox( .map(|tracer| tracer.into_boxed()) .chain(vec![storage_invocation_tracer.into_tracer_pointer()]) .collect(); - vm.inspect(custom_tracers.into(), VmExecutionMode::OneTx) + vm.inspect_transaction_with_bytecode_compression(custom_tracers.into(), tx, true) }, ); span.exit(); @@ -162,5 +161,9 @@ pub(crate) async fn execute_tx_in_sandbox( let tx_execution_metrics = vm_metrics::collect_tx_execution_metrics(total_factory_deps, &execution_result); - (execution_result, tx_execution_metrics) + ( + execution_result, + tx_execution_metrics, + published_bytecodes.is_ok(), + ) } 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 d79daf8f2575..c2fc8beb5761 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 @@ -283,7 +283,7 @@ impl TxSender { let block_args = BlockArgs::pending(&mut connection).await; drop(connection); - let (_, tx_metrics) = execute_tx_in_sandbox( + let (_, tx_metrics, published_bytecodes) = execute_tx_in_sandbox( vm_permit.clone(), shared_args.clone(), true, @@ -319,6 +319,10 @@ impl TxSender { return Err(err.into()); } + if !published_bytecodes { + return Err(SubmitTxError::FailedToPublishCompressedBytecodes); + } + let stage_started_at = Instant::now(); self.ensure_tx_executable(tx.clone().into(), &tx_metrics, true)?; @@ -588,7 +592,7 @@ impl TxSender { let vm_execution_cache_misses_limit = self.0.sender_config.vm_execution_cache_misses_limit; let execution_args = TxExecutionArgs::for_gas_estimate(vm_execution_cache_misses_limit, &tx, base_fee); - let (exec_result, tx_metrics) = execute_tx_in_sandbox( + let (exec_result, tx_metrics, _) = execute_tx_in_sandbox( vm_permit, shared_args, true, diff --git a/core/lib/zksync_core/src/api_server/tx_sender/result.rs b/core/lib/zksync_core/src/api_server/tx_sender/result.rs index 2749e2a13c35..a8183c5e8ac4 100644 --- a/core/lib/zksync_core/src/api_server/tx_sender/result.rs +++ b/core/lib/zksync_core/src/api_server/tx_sender/result.rs @@ -69,6 +69,8 @@ pub enum SubmitTxError { /// Error returned from main node #[error("{0}")] ProxyError(#[from] zksync_web3_decl::jsonrpsee::core::ClientError), + #[error("not enough gas to publish compressed bytecodes")] + FailedToPublishCompressedBytecodes, } impl SubmitTxError { @@ -99,6 +101,7 @@ impl SubmitTxError { Self::InsufficientFundsForTransfer => "insufficient-funds-for-transfer", Self::IntrinsicGas => "intrinsic-gas", Self::ProxyError(_) => "proxy-error", + Self::FailedToPublishCompressedBytecodes => "failed-to-publish-compressed-bytecodes", } } diff --git a/core/lib/zksync_core/src/basic_witness_input_producer/vm_interactions.rs b/core/lib/zksync_core/src/basic_witness_input_producer/vm_interactions.rs index e655112fade2..8ad2a66155de 100644 --- a/core/lib/zksync_core/src/basic_witness_input_producer/vm_interactions.rs +++ b/core/lib/zksync_core/src/basic_witness_input_producer/vm_interactions.rs @@ -74,6 +74,7 @@ pub(super) fn execute_tx( vm.make_snapshot(); if vm .execute_transaction_with_bytecode_compression(tx.clone(), true) + .0 .is_ok() { vm.pop_snapshot_no_rollback(); @@ -84,6 +85,7 @@ pub(super) fn execute_tx( vm.rollback_to_the_latest_snapshot(); if vm .execute_transaction_with_bytecode_compression(tx.clone(), false) + .0 .is_err() { return Err(anyhow!("compression can't fail if we don't apply it")); diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs index dc8a6b548b9f..8b8749c13adb 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/mod.rs @@ -89,6 +89,7 @@ pub struct MainBatchExecutorBuilder { max_allowed_tx_gas_limit: U256, upload_witness_inputs_to_gcs: bool, enum_index_migration_chunk_size: usize, + optional_bytecode_compression: bool, } impl MainBatchExecutorBuilder { @@ -99,6 +100,7 @@ impl MainBatchExecutorBuilder { save_call_traces: bool, upload_witness_inputs_to_gcs: bool, enum_index_migration_chunk_size: usize, + optional_bytecode_compression: bool, ) -> Self { Self { state_keeper_db_path, @@ -107,6 +109,7 @@ impl MainBatchExecutorBuilder { max_allowed_tx_gas_limit, upload_witness_inputs_to_gcs, enum_index_migration_chunk_size, + optional_bytecode_compression, } } } @@ -135,6 +138,7 @@ impl L1BatchExecutorBuilder for MainBatchExecutorBuilder { l1_batch_params, system_env, self.upload_witness_inputs_to_gcs, + self.optional_bytecode_compression, ) } } @@ -158,6 +162,7 @@ impl BatchExecutorHandle { l1_batch_env: L1BatchEnv, system_env: SystemEnv, upload_witness_inputs_to_gcs: bool, + optional_bytecode_compression: bool, ) -> Self { // Since we process `BatchExecutor` commands one-by-one (the next command is never enqueued // until a previous command is processed), capacity 1 is enough for the commands channel. @@ -165,6 +170,7 @@ impl BatchExecutorHandle { let executor = BatchExecutor { save_call_traces, max_allowed_tx_gas_limit, + optional_bytecode_compression, commands: commands_receiver, }; @@ -285,6 +291,7 @@ pub(super) enum Command { pub(super) struct BatchExecutor { save_call_traces: bool, max_allowed_tx_gas_limit: U256, + optional_bytecode_compression: bool, commands: mpsc::Receiver, } @@ -364,7 +371,12 @@ impl BatchExecutor { // Execute the transaction. let latency = KEEPER_METRICS.tx_execution_time[&TxExecutionStage::Execution].start(); - let (tx_result, compressed_bytecodes, call_tracer_result) = self.execute_tx_in_vm(tx, vm); + let (tx_result, compressed_bytecodes, call_tracer_result) = + if self.optional_bytecode_compression { + self.execute_tx_in_vm_with_optional_compression(tx, vm) + } else { + self.execute_tx_in_vm(tx, vm) + }; latency.observe(); APP_METRICS.processed_txs[&TxStage::StateKeeper].inc(); APP_METRICS.processed_l1_txs[&TxStage::StateKeeper].inc_by(tx.is_l1().into()); @@ -432,11 +444,7 @@ impl BatchExecutor { result } - // Err when transaction is rejected. - // `Ok(TxExecutionStatus::Success)` when the transaction succeeded - // `Ok(TxExecutionStatus::Failure)` when the transaction failed. - // Note that failed transactions are considered properly processed and are included in blocks - fn execute_tx_in_vm( + fn execute_tx_in_vm_with_optional_compression( &self, tx: &Transaction, vm: &mut VmInstance, @@ -464,7 +472,7 @@ impl BatchExecutor { vec![] }; - if let Ok(result) = + if let (Ok(()), result) = vm.inspect_transaction_with_bytecode_compression(tracer.into(), tx.clone(), true) { let compressed_bytecodes = vm.get_last_tx_compressed_bytecodes(); @@ -485,8 +493,10 @@ impl BatchExecutor { vec![] }; - let result = vm - .inspect_transaction_with_bytecode_compression(tracer.into(), tx.clone(), false) + let result = + vm.inspect_transaction_with_bytecode_compression(tracer.into(), tx.clone(), false); + result + .0 .expect("Compression can't fail if we don't apply it"); let compressed_bytecodes = vm.get_last_tx_compressed_bytecodes(); @@ -496,7 +506,46 @@ impl BatchExecutor { .unwrap() .take() .unwrap_or_default(); - (result, compressed_bytecodes, trace) + (result.1, compressed_bytecodes, trace) + } + + // Err when transaction is rejected. + // `Ok(TxExecutionStatus::Success)` when the transaction succeeded + // `Ok(TxExecutionStatus::Failure)` when the transaction failed. + // Note that failed transactions are considered properly processed and are included in blocks + fn execute_tx_in_vm( + &self, + tx: &Transaction, + vm: &mut VmInstance, + ) -> ( + VmExecutionResultAndLogs, + Vec, + Vec, + ) { + let call_tracer_result = Arc::new(OnceCell::default()); + let tracer = if self.save_call_traces { + vec![CallTracer::new(call_tracer_result.clone()).into_tracer_pointer()] + } else { + vec![] + }; + + let (published_bytecodes, mut result) = + vm.inspect_transaction_with_bytecode_compression(tracer.into(), tx.clone(), true); + if published_bytecodes.is_ok() { + let compressed_bytecodes = vm.get_last_tx_compressed_bytecodes(); + + let trace = Arc::try_unwrap(call_tracer_result) + .unwrap() + .take() + .unwrap_or_default(); + (result, compressed_bytecodes, trace) + } else { + // Transaction failed to publish bytecodes, we reject it so initiator doesn't pay fee. + result.result = ExecutionResult::Halt { + reason: Halt::FailedToPublishCompressedBytecodes, + }; + (result, Default::default(), Default::default()) + } } fn dryrun_block_tip( diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs index 6417c65a5f8f..6e7e5681b330 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/tester.rs @@ -112,6 +112,7 @@ impl Tester { l1_batch, system_env, self.config.upload_witness_inputs_to_gcs, + false, ) } diff --git a/core/lib/zksync_core/src/state_keeper/io/mempool.rs b/core/lib/zksync_core/src/state_keeper/io/mempool.rs index 45484f645b5e..f80b57358253 100644 --- a/core/lib/zksync_core/src/state_keeper/io/mempool.rs +++ b/core/lib/zksync_core/src/state_keeper/io/mempool.rs @@ -151,7 +151,7 @@ impl StateKeeperIO for MempoolIO { ); let current_timestamp = current_timestamp.await.ok()?; - tracing::info!( + tracing::trace!( "(l1_gas_price, fair_l2_gas_price) for L1 batch #{} is ({}, {})", self.current_l1_batch_number.0, self.filter.l1_gas_price, diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index a17d169bc6ea..dc5016772a7b 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -52,6 +52,7 @@ pub(crate) async fn create_state_keeper( state_keeper_config.save_call_traces, state_keeper_config.upload_witness_inputs_to_gcs, state_keeper_config.enum_index_migration_chunk_size(), + false, ); let io = MempoolIO::new( diff --git a/core/tests/ts-integration/tests/contracts.test.ts b/core/tests/ts-integration/tests/contracts.test.ts index fea1b15845a5..e7d5fcf3a239 100644 --- a/core/tests/ts-integration/tests/contracts.test.ts +++ b/core/tests/ts-integration/tests/contracts.test.ts @@ -306,6 +306,30 @@ describe('Smart contract behavior checks', () => { ).toBeAccepted([]); }); + test('Should reject tx with not enough gas for publishing bytecode', async () => { + // Send a transaction with big unique factory dep and provide gas enough for validation but not for bytecode publishing. + // Transaction should be rejected by API. + + const BYTECODE_LEN = 50016; + const bytecode = ethers.utils.hexlify(ethers.utils.randomBytes(BYTECODE_LEN)); + + // Estimate gas for "no-op". It's a good estimate for validation gas. + const gasLimit = await alice.estimateGas({ + to: alice.address, + data: '0x' + }); + + await expect( + alice.sendTransaction({ + to: alice.address, + gasLimit, + customData: { + factoryDeps: [bytecode] + } + }) + ).toBeRejected('not enough gas to publish compressed bytecodes'); + }); + afterAll(async () => { await testMaster.deinitialize(); }); From 7d4a878da4cf9147be1c2294344b1e7c4a434765 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Mon, 8 Jan 2024 10:22:43 -0600 Subject: [PATCH 35/49] feat: adds cargo spellcheck and cspell to Dockerfile for zk env (#835) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Adds cargo-spellcheck to Dockerfile for zk env - Adds cspell installation to Dockerfile for zk env ## Why ❔ - Needed as dep for zk spellcheck tool ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- docker/zk-environment/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/zk-environment/Dockerfile b/docker/zk-environment/Dockerfile index d942fab7ece1..b074bc4bbc69 100644 --- a/docker/zk-environment/Dockerfile +++ b/docker/zk-environment/Dockerfile @@ -84,7 +84,7 @@ RUN mkdir -p /etc/apt/keyrings && \ wget -c -O - https://deb.nodesource.com/gpgkey/nodesource-repo.gpg.key | gpg --dearmor -o /etc/apt/keyrings/nodesource.gpg && \ echo "deb [signed-by=/etc/apt/keyrings/nodesource.gpg] https://deb.nodesource.com/node_$NODE_MAJOR.x nodistro main" | tee /etc/apt/sources.list.d/nodesource.list && \ apt-get update && apt-get install nodejs npm -y && \ - npm install -g yarn + npm install -g yarn && npm install -g cspell@latest # Install Rust and required cargo packages ENV RUSTUP_HOME=/usr/local/rustup \ @@ -105,6 +105,7 @@ RUN wget -c -O - https://sh.rustup.rs | bash -s -- -y && \ RUN cargo install --version=0.7.3 sqlx-cli RUN cargo install cargo-nextest +RUN cargo install cargo-spellcheck # Copy compiler (both solc and zksolc) binaries # Obtain `solc` 0.8.20. From 83c1546b66791267d538775a2773ae74adb72e0c Mon Sep 17 00:00:00 2001 From: Artem Makhortov <13339874+artmakh@users.noreply.github.com> Date: Tue, 9 Jan 2024 00:00:38 +0700 Subject: [PATCH 36/49] chore(docs): setup-dev improvements (#836) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Small improvements to setup-dev docs ## Why ❔ Eliminate steps which can be resulted in wrong versions of packages installed ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- docs/guides/setup-dev.md | 32 +++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/docs/guides/setup-dev.md b/docs/guides/setup-dev.md index 7fadbf37eec6..a833fa10f455 100644 --- a/docs/guides/setup-dev.md +++ b/docs/guides/setup-dev.md @@ -10,14 +10,14 @@ curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh # NVM curl -o- https://raw.githubusercontent.com/nvm-sh/nvm/v0.39.5/install.sh | bash # All necessary stuff -sudo apt-get install build-essential pkg-config cmake clang lldb lld libssl-dev postgresql docker-compose +sudo apt-get install build-essential pkg-config cmake clang lldb lld libssl-dev postgresql # Docker sudo usermod -aG docker YOUR_USER ## You might need to re-connect (due to usermod change). # Node & yarn -nvm install node +nvm install 18 npm install -g yarn yarn set version 1.22.19 @@ -55,9 +55,9 @@ want to only have CLI tool, you need the `docker-ce` package and you can follow Installing `docker` via `snap` or from the default repository can cause troubles. -You need to install both `docker` and `docker-compose`. +You need to install both `docker` and `docker compose`. -**Note:** `docker-compose` is installed automatically with `Docker Desktop`. +**Note:** `docker compose` is installed automatically with `Docker Desktop`. **Note:** On linux you may encounter the following error when you’ll try to work with `zksync`: @@ -90,10 +90,10 @@ If logging out does not resolve the issue, restarting the computer should. `Node.js`, we suggest you to install [nvm](https://github.com/nvm-sh/nvm). It will allow you to update `Node.js` version easily in the future (by running `nvm use` in the root of the repository) 2. Install `yarn` (make sure to get version 1.22.19 - you can change the version by running `yarn set version 1.22.19`). - Instructions can be found on the [official site](https://classic.yarnpkg.com/en/docs/install/). - Check if `yarn` is installed by running `yarn -v`. If you face any problems when installing `yarn`, it might be the - case that your package manager installed the wrong package.Make sure to thoroughly follow the instructions above on - the official website. It contains a lot of troubleshooting guides in it. + Instructions can be found on the [official site](https://classic.yarnpkg.com/en/docs/install/). Check if `yarn` is + installed by running `yarn -v`. If you face any problems when installing `yarn`, it might be the case that your + package manager installed the wrong package.Make sure to thoroughly follow the instructions above on the official + website. It contains a lot of troubleshooting guides in it. ## `Axel` @@ -123,7 +123,7 @@ Make sure the version is higher than `2.17.10`. In order to compile RocksDB, you must have LLVM available. On debian-based linux it can be installed as follows: -On linux: +On debian-based linux: ```bash sudo apt-get install build-essential pkg-config cmake clang lldb lld @@ -144,7 +144,7 @@ On mac: brew install openssl ``` -On linux: +On debian-based linux: ```bash sudo apt-get install libssl-dev @@ -196,7 +196,7 @@ On mac: brew install postgresql@14 ``` -On linux: +On debian-based linux: ```bash sudo apt-get install postgresql @@ -224,10 +224,20 @@ cargo install sqlx-cli --version 0.7.3 Install the latest solidity compiler. +On mac: + ```bash brew install solidity ``` +On debian-based linux: + +```bash +sudo add-apt-repository ppa:ethereum/ethereum +sudo apt-get update +sudo apt-get install solc +``` + Alternatively, download a [precompiled version](https://github.com/ethereum/solc-bin) and add it to your PATH. ## Python From 53defbab3477894f554fe3bef9e214f3c954ad1c Mon Sep 17 00:00:00 2001 From: Fedor Sakharov Date: Mon, 8 Jan 2024 18:13:53 +0100 Subject: [PATCH 37/49] chore(ci): return the lint checks of sqlx (#830) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .github/workflows/ci-core-lint-reusable.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/ci-core-lint-reusable.yml b/.github/workflows/ci-core-lint-reusable.yml index 365d477c64bb..541049cdeb0c 100644 --- a/.github/workflows/ci-core-lint-reusable.yml +++ b/.github/workflows/ci-core-lint-reusable.yml @@ -36,5 +36,4 @@ jobs: ci_run zk lint js --check ci_run zk lint ts --check ci_run zk lint md --check - - # ci_run zk db check-sqlx-data + ci_run zk db check-sqlx-data From b68518042a02d62dfa4ad26de3c806e2fa198dd3 Mon Sep 17 00:00:00 2001 From: Maksym Date: Mon, 8 Jan 2024 22:10:09 +0200 Subject: [PATCH 38/49] ci: fix merge manifest job (#828) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Fix job for merge docker manifests ## Why ❔ In case of multi platform builds ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .github/workflows/build-core-template.yml | 47 ++++++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/.github/workflows/build-core-template.yml b/.github/workflows/build-core-template.yml index c27829968f59..181a1d7a4166 100644 --- a/.github/workflows/build-core-template.yml +++ b/.github/workflows/build-core-template.yml @@ -34,6 +34,9 @@ jobs: - snapshots-creator platforms: - linux/amd64 + include: + - components: external-node + platforms: linux/arm64 steps: - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 @@ -84,9 +87,51 @@ jobs: PLATFORM: ${{ matrix.platforms }} run: | ci_run rustup default nightly-2023-08-21 - ci_run zk docker $DOCKER_ACTION $COMPONENT + ci_run zk docker $DOCKER_ACTION --platform=${PLATFORM} $COMPONENT - name: Show sccache stats if: always() run: | ci_run sccache --show-stats ci_run cat /tmp/sccache_log.txt + + create_manifest: + name: Create release manifest + runs-on: matterlabs-ci-runner + needs: build-images + if: ${{ inputs.action == 'push' }} + strategy: + matrix: + component: + - name: server-v2 + platform: linux/amd64 + - name: external-node + platform: linux/amd64,linux/arm64 + - name: contract-verifier + platform: linux/amd64 + - name: cross-external-nodes-checker + platform: linux/amd64 + - name: snapshots-creator + platform: linux/amd64 + env: + IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} + DOCKER_CLI_EXPERIMENTAL: enabled + steps: + - name: login to Docker registries + run: | + docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} + gcloud auth configure-docker us-docker.pkg.dev -q + + - name: Create Docker manifest + run: | + docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.component.name }}") + platforms=${{ matrix.component.platform }} + for repo in "${docker_repositories[@]}"; do + repo_tags="" + for platform in ${platforms//,/ }; do + platform=$(echo $platform | tr '/' '-') + repo_tags+=" --amend ${repo}:${IMAGE_TAG_SUFFIX}-${platform}" + done + echo $repo_tags + docker manifest create ${repo}:${IMAGE_TAG_SUFFIX} ${repo_tags} + docker manifest push ${repo}:${IMAGE_TAG_SUFFIX} + done From 83fc7be3cb9f4d3082b8b9fa8b8f568330bf744f Mon Sep 17 00:00:00 2001 From: perekopskiy <53865202+perekopskiy@users.noreply.github.com> Date: Mon, 8 Jan 2024 22:20:15 +0200 Subject: [PATCH 39/49] fix(vm): fix circuit tracer (#837) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ `finish_cycle` method of circuit tracer wasn't called and there were bugs in it. ## Why ❔ Fix tracer ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- .../old_vm/oracles/precompile.rs | 2 +- .../vm_boojum_integration/tests/circuits.rs | 7 +-- .../tracers/circuits_tracer.rs | 47 ++++++++----------- .../tracers/default_tracers.rs | 6 +++ .../vm_latest/old_vm/oracles/precompile.rs | 2 +- .../src/versions/vm_latest/tests/circuits.rs | 7 +-- .../vm_latest/tracers/circuits_tracer.rs | 47 ++++++++----------- .../vm_latest/tracers/default_tracers.rs | 6 +++ 8 files changed, 62 insertions(+), 62 deletions(-) diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs index b23c72669d70..0fc1108ead85 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/precompile.rs @@ -19,7 +19,7 @@ use crate::vm_boojum_integration::old_vm::history_recorder::{ #[derive(Debug, Clone)] pub struct PrecompilesProcessorWithHistory { pub timestamp_history: HistoryRecorder, H>, - pub precompile_cycles_history: HistoryRecorder, HistoryEnabled>, + pub precompile_cycles_history: HistoryRecorder, H>, } impl Default for PrecompilesProcessorWithHistory { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs index 2630c913e02a..7c5b8ee2a2d4 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tests/circuits.rs @@ -30,14 +30,15 @@ fn test_circuits() { vm.vm.push_transaction(tx); let res = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); - const EXPECTED_CIRCUITS_USED: f32 = 1.5521; + const EXPECTED_CIRCUITS_USED: f32 = 4.8685; let delta = (res.statistics.estimated_circuits_used - EXPECTED_CIRCUITS_USED) / EXPECTED_CIRCUITS_USED; if delta.abs() > 0.1 { panic!( - "Estimation differs from expected result by too much: {}%", - delta * 100.0 + "Estimation differs from expected result by too much: {}%, expected value: {}", + delta * 100.0, + res.statistics.estimated_circuits_used ); } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs index 9f26cd057ff0..e6b52221e02b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/circuits_tracer.rs @@ -12,10 +12,7 @@ use crate::{ interface::{dyn_tracers::vm_1_4_0::DynTracer, tracer::TracerExecutionStatus}, vm_boojum_integration::{ bootloader_state::BootloaderState, - old_vm::{ - history_recorder::{HistoryMode, VectorHistoryEvent}, - memory::SimpleMemory, - }, + old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, tracers::traits::VmTracer, types::internals::ZkSyncVmState, }, @@ -28,7 +25,7 @@ pub(crate) struct CircuitsTracer { last_decommitment_history_entry_checked: Option, last_written_keys_history_entry_checked: Option, last_read_keys_history_entry_checked: Option, - last_precompile_history_entry_checked: Option, + last_precompile_inner_entry_checked: Option, _phantom_data: PhantomData, } @@ -39,7 +36,7 @@ impl CircuitsTracer { last_decommitment_history_entry_checked: None, last_written_keys_history_entry_checked: None, last_read_keys_history_entry_checked: None, - last_precompile_history_entry_checked: None, + last_precompile_inner_entry_checked: None, _phantom_data: Default::default(), } } @@ -97,11 +94,11 @@ impl VmTracer for CircuitsTracer { self.last_read_keys_history_entry_checked = Some(state.storage.read_keys.history().len()); - self.last_precompile_history_entry_checked = Some( + self.last_precompile_inner_entry_checked = Some( state .precompiles_processor .precompile_cycles_history - .history() + .inner() .len(), ); } @@ -121,7 +118,7 @@ impl VmTracer for CircuitsTracer { .history(); for (_, history_event) in &history[last_decommitment_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); let bytecode_len = state .decommittment_processor .known_bytecodes @@ -145,7 +142,7 @@ impl VmTracer for CircuitsTracer { let history = state.storage.written_keys.history(); for (_, history_event) in &history[last_writes_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); self.estimated_circuits_used += 2.0 * STORAGE_APPLICATION_CYCLE_FRACTION; } @@ -158,7 +155,7 @@ impl VmTracer for CircuitsTracer { let history = state.storage.read_keys.history(); for (_, history_event) in &history[last_reads_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); // If the slot is already written to, then we've already taken 2 cycles into account. if !state @@ -173,26 +170,22 @@ impl VmTracer for CircuitsTracer { self.last_read_keys_history_entry_checked = Some(history.len()); // Process precompiles. - let last_precompile_history_entry_checked = self - .last_precompile_history_entry_checked + let last_precompile_inner_entry_checked = self + .last_precompile_inner_entry_checked .expect("Value must be set during init"); - let history = state + let inner = state .precompiles_processor .precompile_cycles_history - .history(); - for (_, history_event) in &history[last_precompile_history_entry_checked..] { - if let VectorHistoryEvent::Push((precompile, cycles)) = history_event { - let fraction = match precompile { - PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, - PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, - PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, - }; - self.estimated_circuits_used += (*cycles as f32) * fraction; - } else { - panic!("Precompile calls should not be rolled back"); - } + .inner(); + for (precompile, cycles) in &inner[last_precompile_inner_entry_checked..] { + let fraction = match precompile { + PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, + PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, + PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, + }; + self.estimated_circuits_used += (*cycles as f32) * fraction; } - self.last_precompile_history_entry_checked = Some(history.len()); + self.last_precompile_inner_entry_checked = Some(inner.len()); TracerExecutionStatus::Continue } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs index e41451ae8ef4..422463d29216 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/default_tracers.rs @@ -284,6 +284,12 @@ impl DefaultExecutionTracer { .finish_cycle(state, bootloader_state) .stricter(&result); } + + result = self + .circuits_tracer + .finish_cycle(state, bootloader_state) + .stricter(&result); + result.stricter(&self.should_stop_execution()) } diff --git a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs index 0e750fe69356..fed5c1b6c0e5 100644 --- a/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs +++ b/core/lib/multivm/src/versions/vm_latest/old_vm/oracles/precompile.rs @@ -17,7 +17,7 @@ use crate::vm_latest::old_vm::history_recorder::{HistoryEnabled, HistoryMode, Hi #[derive(Debug, Clone)] pub struct PrecompilesProcessorWithHistory { pub timestamp_history: HistoryRecorder, H>, - pub precompile_cycles_history: HistoryRecorder, HistoryEnabled>, + pub precompile_cycles_history: HistoryRecorder, H>, } impl Default for PrecompilesProcessorWithHistory { diff --git a/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs b/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs index c76f3dc1c729..bc19fc8793a7 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/circuits.rs @@ -30,14 +30,15 @@ fn test_circuits() { vm.vm.push_transaction(tx); let res = vm.vm.inspect(Default::default(), VmExecutionMode::OneTx); - const EXPECTED_CIRCUITS_USED: f32 = 1.5521; + const EXPECTED_CIRCUITS_USED: f32 = 4.8685; let delta = (res.statistics.estimated_circuits_used - EXPECTED_CIRCUITS_USED) / EXPECTED_CIRCUITS_USED; if delta.abs() > 0.1 { panic!( - "Estimation differs from expected result by too much: {}%", - delta * 100.0 + "Estimation differs from expected result by too much: {}%, expected value: {}", + delta * 100.0, + res.statistics.estimated_circuits_used ); } } diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs index c2383b11561f..e33b15e569ae 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/circuits_tracer.rs @@ -12,10 +12,7 @@ use crate::{ interface::{dyn_tracers::vm_1_4_0::DynTracer, tracer::TracerExecutionStatus}, vm_latest::{ bootloader_state::BootloaderState, - old_vm::{ - history_recorder::{HistoryMode, VectorHistoryEvent}, - memory::SimpleMemory, - }, + old_vm::{history_recorder::HistoryMode, memory::SimpleMemory}, tracers::traits::VmTracer, types::internals::ZkSyncVmState, }, @@ -28,7 +25,7 @@ pub(crate) struct CircuitsTracer { last_decommitment_history_entry_checked: Option, last_written_keys_history_entry_checked: Option, last_read_keys_history_entry_checked: Option, - last_precompile_history_entry_checked: Option, + last_precompile_inner_entry_checked: Option, _phantom_data: PhantomData, } @@ -39,7 +36,7 @@ impl CircuitsTracer { last_decommitment_history_entry_checked: None, last_written_keys_history_entry_checked: None, last_read_keys_history_entry_checked: None, - last_precompile_history_entry_checked: None, + last_precompile_inner_entry_checked: None, _phantom_data: Default::default(), } } @@ -97,11 +94,11 @@ impl VmTracer for CircuitsTracer { self.last_read_keys_history_entry_checked = Some(state.storage.read_keys.history().len()); - self.last_precompile_history_entry_checked = Some( + self.last_precompile_inner_entry_checked = Some( state .precompiles_processor .precompile_cycles_history - .history() + .inner() .len(), ); } @@ -121,7 +118,7 @@ impl VmTracer for CircuitsTracer { .history(); for (_, history_event) in &history[last_decommitment_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); let bytecode_len = state .decommittment_processor .known_bytecodes @@ -145,7 +142,7 @@ impl VmTracer for CircuitsTracer { let history = state.storage.written_keys.history(); for (_, history_event) in &history[last_writes_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); self.estimated_circuits_used += 2.0 * STORAGE_APPLICATION_CYCLE_FRACTION; } @@ -158,7 +155,7 @@ impl VmTracer for CircuitsTracer { let history = state.storage.read_keys.history(); for (_, history_event) in &history[last_reads_history_entry_checked..] { // We assume that only insertions may happen during a single VM inspection. - assert!(history_event.value.is_some()); + assert!(history_event.value.is_none()); // If the slot is already written to, then we've already taken 2 cycles into account. if !state @@ -173,26 +170,22 @@ impl VmTracer for CircuitsTracer { self.last_read_keys_history_entry_checked = Some(history.len()); // Process precompiles. - let last_precompile_history_entry_checked = self - .last_precompile_history_entry_checked + let last_precompile_inner_entry_checked = self + .last_precompile_inner_entry_checked .expect("Value must be set during init"); - let history = state + let inner = state .precompiles_processor .precompile_cycles_history - .history(); - for (_, history_event) in &history[last_precompile_history_entry_checked..] { - if let VectorHistoryEvent::Push((precompile, cycles)) = history_event { - let fraction = match precompile { - PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, - PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, - PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, - }; - self.estimated_circuits_used += (*cycles as f32) * fraction; - } else { - panic!("Precompile calls should not be rolled back"); - } + .inner(); + for (precompile, cycles) in &inner[last_precompile_inner_entry_checked..] { + let fraction = match precompile { + PrecompileAddress::Ecrecover => ECRECOVER_CYCLE_FRACTION, + PrecompileAddress::SHA256 => SHA256_CYCLE_FRACTION, + PrecompileAddress::Keccak256 => KECCAK256_CYCLE_FRACTION, + }; + self.estimated_circuits_used += (*cycles as f32) * fraction; } - self.last_precompile_history_entry_checked = Some(history.len()); + self.last_precompile_inner_entry_checked = Some(inner.len()); TracerExecutionStatus::Continue } diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs index cba23d14d23a..5d5ecfad0dbb 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/default_tracers.rs @@ -284,6 +284,12 @@ impl DefaultExecutionTracer { .finish_cycle(state, bootloader_state) .stricter(&result); } + + result = self + .circuits_tracer + .finish_cycle(state, bootloader_state) + .stricter(&result); + result.stricter(&self.should_stop_execution()) } From a488dd374cdd981b7895f0ebe8822a75a14fe21e Mon Sep 17 00:00:00 2001 From: Aleksandr Stepanov Date: Mon, 8 Jan 2024 23:43:52 +0200 Subject: [PATCH 40/49] ci: Revert fix merge manifest job (#839) Reverts matter-labs/zksync-era#828 --- .github/workflows/build-core-template.yml | 47 +---------------------- 1 file changed, 1 insertion(+), 46 deletions(-) diff --git a/.github/workflows/build-core-template.yml b/.github/workflows/build-core-template.yml index 181a1d7a4166..c27829968f59 100644 --- a/.github/workflows/build-core-template.yml +++ b/.github/workflows/build-core-template.yml @@ -34,9 +34,6 @@ jobs: - snapshots-creator platforms: - linux/amd64 - include: - - components: external-node - platforms: linux/arm64 steps: - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 @@ -87,51 +84,9 @@ jobs: PLATFORM: ${{ matrix.platforms }} run: | ci_run rustup default nightly-2023-08-21 - ci_run zk docker $DOCKER_ACTION --platform=${PLATFORM} $COMPONENT + ci_run zk docker $DOCKER_ACTION $COMPONENT - name: Show sccache stats if: always() run: | ci_run sccache --show-stats ci_run cat /tmp/sccache_log.txt - - create_manifest: - name: Create release manifest - runs-on: matterlabs-ci-runner - needs: build-images - if: ${{ inputs.action == 'push' }} - strategy: - matrix: - component: - - name: server-v2 - platform: linux/amd64 - - name: external-node - platform: linux/amd64,linux/arm64 - - name: contract-verifier - platform: linux/amd64 - - name: cross-external-nodes-checker - platform: linux/amd64 - - name: snapshots-creator - platform: linux/amd64 - env: - IMAGE_TAG_SUFFIX: ${{ inputs.image_tag_suffix }} - DOCKER_CLI_EXPERIMENTAL: enabled - steps: - - name: login to Docker registries - run: | - docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} - gcloud auth configure-docker us-docker.pkg.dev -q - - - name: Create Docker manifest - run: | - docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.component.name }}") - platforms=${{ matrix.component.platform }} - for repo in "${docker_repositories[@]}"; do - repo_tags="" - for platform in ${platforms//,/ }; do - platform=$(echo $platform | tr '/' '-') - repo_tags+=" --amend ${repo}:${IMAGE_TAG_SUFFIX}-${platform}" - done - echo $repo_tags - docker manifest create ${repo}:${IMAGE_TAG_SUFFIX} ${repo_tags} - docker manifest push ${repo}:${IMAGE_TAG_SUFFIX} - done From 6cfe39d790de5915f5fb8342ed67f605b1ab9b46 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Tue, 9 Jan 2024 03:14:44 -0600 Subject: [PATCH 41/49] feat: Integrate `cspell` checker in CI for `/docs` and update PR template (#681) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Series of PRs: This is the first in a series PRs aimed at enhancing spelling accuracy in this repository.** See: - [ ] https://github.com/matter-labs/zksync-era/pull/682 - [ ] https://github.com/matter-labs/zksync-era/pull/683 - [ ] https://github.com/matter-labs/zksync-era/pull/684 - [ ] https://github.com/matter-labs/zksync-era/pull/685 - [ ] https://github.com/matter-labs/zksync-era/pull/827 Once merged, a final PR will enable the `dev_comments: true` in the cargo-spellcheck config file. ## What ❔ - **Introduction of cspell checker:** This PR integrates the cspell checker into the CI, specifically targeting the `/docs` directory to identify and correct spelling errors. - Updates PR template to include `cspell` command - Updates dictionary ## Why ❔ - **Improving Documentation Quality:** The `/docs` directory currently contains numerous spelling errors. By resolving these and integrating a spellcheck step in our CI, we aim to maintain high-quality, error-free documentation. - **Preventing Future Spelling Mistakes:** The cspell checker will help in automatically detecting and preventing spelling errors in future contributions. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Oleksandr Stepanov --- .github/pull_request_template.md | 2 +- .github/workflows/check-spelling.yml | 30 +- .../guides/advanced/0_alternative_vm_intro.md | 8 +- docs/guides/advanced/compression.md | 2 +- docs/guides/advanced/contracts.md | 6 +- docs/guides/advanced/deeper_overview.md | 6 +- docs/guides/advanced/pubdata.md | 2 +- docs/guides/advanced/zk_intuition.md | 2 +- docs/guides/repositories.md | 2 +- docs/specs/README.md | 4 +- docs/specs/data_availability/compression.md | 6 +- .../specs/data_availability/reconstruction.md | 2 +- docs/specs/l1_l2_communication/l1_to_l2.md | 2 +- docs/specs/l1_smart_contracts.md | 2 +- .../boojum_function_check_if_satisfied.md | 2 +- docs/specs/prover/boojum_gadgets.md | 4 +- docs/specs/prover/circuit_testing.md | 2 +- docs/specs/prover/circuits/README.md | 2 +- .../specs/prover/circuits/code_decommitter.md | 12 +- docs/specs/prover/circuits/demux_log_queue.md | 2 +- docs/specs/prover/circuits/log_sorter.md | 12 +- docs/specs/prover/circuits/main_vm.md | 14 +- docs/specs/prover/circuits/overview.md | 2 +- .../prover/circuits/sha256_round_function.md | 4 +- .../prover/circuits/sort_decommitments.md | 18 +- .../prover/circuits/storage_application.md | 2 +- docs/specs/prover/circuits/storage_sorter.md | 4 +- docs/specs/the_hyperchain/shared_bridge.md | 4 +- docs/specs/zk_evm/bootloader.md | 8 +- docs/specs/zk_evm/fee_model.md | 2 +- docs/specs/zk_evm/vm_overview.md | 2 +- .../compiler/instructions/extensions/call.md | 4 +- infrastructure/zk/package.json | 2 +- infrastructure/zk/src/spellcheck.ts | 20 +- spellcheck/era.dic | 11 +- yarn.lock | 650 +++++++++++++++++- 36 files changed, 756 insertions(+), 103 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 7b828d1ca895..764b85baccaf 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -18,4 +18,4 @@ - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. -- [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. +- [ ] Spellcheck has been run via `zk spellcheck`. diff --git a/.github/workflows/check-spelling.yml b/.github/workflows/check-spelling.yml index 9073401eab9e..0a3bce24cb7e 100644 --- a/.github/workflows/check-spelling.yml +++ b/.github/workflows/check-spelling.yml @@ -12,14 +12,30 @@ env: jobs: spellcheck: - runs-on: ubuntu-latest + runs-on: [matterlabs-ci-runner] steps: - - name: Install cargo-spellcheck - uses: taiki-e/install-action@v2 + - uses: actions/checkout@ac593985615ec2ede58e132d2e21d2b1cbd6127c # v3 with: - tool: cargo-spellcheck + submodules: "recursive" + - name: Use Node.js + uses: actions/setup-node@v3 + with: + node-version: 18 + + - name: Setup environment + run: | + echo ZKSYNC_HOME=$(pwd) >> $GITHUB_ENV + echo $(pwd)/bin >> $GITHUB_PATH + echo IN_DOCKER=1 >> .env + + - name: Start services + run: | + docker compose up -d zk - - uses: actions/checkout@3df4ab11eba7bda6032a0b82a6bb43b11571feac # v4 + - name: Build zk + run: | + ci_run zk - - name: Run cargo-spellcheck - run: cargo spellcheck --cfg=./spellcheck/era.cfg --code 1 + - name: Run spellcheck + run: | + ci_run zk spellcheck diff --git a/docs/guides/advanced/0_alternative_vm_intro.md b/docs/guides/advanced/0_alternative_vm_intro.md index bce8c48a60aa..b3acf810b467 100644 --- a/docs/guides/advanced/0_alternative_vm_intro.md +++ b/docs/guides/advanced/0_alternative_vm_intro.md @@ -48,9 +48,9 @@ _Pointers_ are separate type in the VM. It is only possible to: - Read some value within a pointer. - Shrink the pointer by reducing the slice to which pointer points to. -- Receive the pointer to the returndata/as a calldata. -- Pointers can be stored only on stack/registers to make sure that the other contracts can not read memory/returndata of - contracts they are not supposed to. +- Receive the pointer to the `returndata` as a calldata. +- Pointers can be stored only on stack/registers to make sure that the other contracts can not read `memory/returndata` + of contracts they are not supposed to. - A pointer can be converted to the u256 integer representing it, but an integer can not be converted to a pointer to prevent unallowed memory access. - It is not possible to return a pointer that points to a memory page with id smaller than the one for the current page. @@ -63,7 +63,7 @@ For each frame, the following memory areas are allocated: - _Heap_ (plays the same role as `memory` on Ethereum). - _AuxHeap_ (auxiliary heap). It has the same properties as Heap, but it is used for the compiler to encode - calldata/copy the returndata from the calls to system contracts to not interfere with the standard Solidity memory + calldata/copy the `returndata` from the calls to system contracts to not interfere with the standard Solidity memory alignment. - _Stack_. Unlike Ethereum, stack is not the primary place to get arguments for opcodes. The biggest difference between stack on zkEVM and EVM is that on zkSync stack can be accessed at any location (just like memory). While users do not diff --git a/docs/guides/advanced/compression.md b/docs/guides/advanced/compression.md index de257cc4c2be..12071e79891c 100644 --- a/docs/guides/advanced/compression.md +++ b/docs/guides/advanced/compression.md @@ -31,7 +31,7 @@ Dictionary would be: 3 -> 0xC (count: 1) ``` -Note that '1' maps to '0xD', as it occurs twice, and first occurrence is earlier than first occurence of 0xB, that also +Note that '1' maps to '0xD', as it occurs twice, and first occurrence is earlier than first occurrence of 0xB, that also occurs twice. Compressed bytecode: diff --git a/docs/guides/advanced/contracts.md b/docs/guides/advanced/contracts.md index cb42a945da6c..502d9b04cad7 100644 --- a/docs/guides/advanced/contracts.md +++ b/docs/guides/advanced/contracts.md @@ -86,9 +86,9 @@ changed - so updating the same slot multiple times doesn't increase the amount o ### Account abstraction and some method calls -As `zkSync` has a built-in AccountAbstraction (more on this in a separate article) - you shouldn't depend on some of the -solidity functions (like `ecrecover` - that checks the keys, or `tx.origin`) - in all the cases, the compiler will try -to warn you. +As `zkSync` has a built-in Account Abstraction (more on this in a separate article) - you shouldn't depend on some of +the solidity functions (like `ecrecover` - that checks the keys, or `tx.origin`) - in all the cases, the compiler will +try to warn you. ## Summary diff --git a/docs/guides/advanced/deeper_overview.md b/docs/guides/advanced/deeper_overview.md index fef5e8bb6add..7fa4a009a920 100644 --- a/docs/guides/advanced/deeper_overview.md +++ b/docs/guides/advanced/deeper_overview.md @@ -69,8 +69,8 @@ pub struct SelectionGate { } ``` -Internaly the `Variable` object is `pub struct Variable(pub(crate) u64);` - so it is an index to the position within the -constraint system object. +Internally the `Variable` object is `pub struct Variable(pub(crate) u64);` - so it is an index to the position within +the constraint system object. And now let's see how we can add this gate into the system. @@ -211,7 +211,7 @@ how we were operating on regular 'Variables' above). ### CSSelectable -Implements the `Selectable` trait - that allows this struct to participage in operations like conditionally select (so +Implements the `Selectable` trait - that allows this struct to participate in operations like conditionally select (so it can be used as 'a' or 'b' in the Select gate example above). ### CSVarLengthEncodable diff --git a/docs/guides/advanced/pubdata.md b/docs/guides/advanced/pubdata.md index 9b8fd4346c42..b5cbe6b49b51 100644 --- a/docs/guides/advanced/pubdata.md +++ b/docs/guides/advanced/pubdata.md @@ -79,7 +79,7 @@ The 4 main fields to look at here are: 1. Structure: `num entries as u32 || for each entry: (8 byte id, 32 bytes final value)` 3. `factoryDeps`: An array of uncompressed bytecodes 4. `l2ArbitraryLengthMessages` : L2 → L1 Messages - 1. We don’t need them all, we are just concerned with messages sent from the `Compressor/BytcodeCompressor` contract + 1. We don’t need them all, we are just concerned with messages sent from the `Compressor/BytecodeCompressor` contract 2. These messages will follow the compression algorithm outline [here](./bytecode_compression.md) For the ids on the repeated writes, they are generated as we process the first time keys. For example: if we see diff --git a/docs/guides/advanced/zk_intuition.md b/docs/guides/advanced/zk_intuition.md index 97b0d26920a0..f3a9ac1fd190 100644 --- a/docs/guides/advanced/zk_intuition.md +++ b/docs/guides/advanced/zk_intuition.md @@ -62,7 +62,7 @@ pub fn compute_decommitter_circuit_snapshots< ... ) -> ( Vec>, - CodeDecommittmentsDeduplicatorInstanceWitness, + CodeDecommitmentsDeduplicatorInstanceWitness, ) ``` diff --git a/docs/guides/repositories.md b/docs/guides/repositories.md index b2b1a92d373b..a22e46ec11ac 100644 --- a/docs/guides/repositories.md +++ b/docs/guides/repositories.md @@ -46,7 +46,7 @@ | ----------------------------------------------------------------------------- | ------------------------------------------------------------------- | | [era-bellman-cuda](https://github.com/matter-labs/era-bellman-cuda) | Cuda implementations for cryptographic functions used by the prover | | [era-heavy-ops-service](https://github.com/matter-labs/era-heavy-ops-service) | Main circuit prover that requires GPU to run | -| [era-cicruit_testing](https://github.com/matter-labs/era-circuit_testing) | ?? | +| [era-circuit_testing](https://github.com/matter-labs/era-circuit_testing) | ?? | ### Tools & contract developers diff --git a/docs/specs/README.md b/docs/specs/README.md index 724d7b923e84..34103584e547 100644 --- a/docs/specs/README.md +++ b/docs/specs/README.md @@ -5,7 +5,7 @@ 1. [L1 Contracts](./l1_smart_contracts.md) 1. [zkEVM](./zk_evm/README.md) - [VM Overview](./zk_evm/vm_overview.md) - - [VM Specificiation](./zk_evm/vm_specification/README.md) + - [VM Specification](./zk_evm/vm_specification/README.md) - [Bootloader](./zk_evm/bootloader.md) - [System Contracts](./zk_evm/system_contracts.md) - [Precompiles](./zk_evm/precompiles.md) @@ -33,4 +33,4 @@ 1. [The Hyperchain](./the_hyperchain/README.md) - [Overview](./the_hyperchain/overview.md) - [Shared Bridge](./the_hyperchain/shared_bridge.md) - - [Hyperbrdges](./the_hyperchain/hyperbridges.md) + - [Hyperbridges](./the_hyperchain/hyperbridges.md) diff --git a/docs/specs/data_availability/compression.md b/docs/specs/data_availability/compression.md index ef29ece4a8bb..0a0057ac9faf 100644 --- a/docs/specs/data_availability/compression.md +++ b/docs/specs/data_availability/compression.md @@ -111,15 +111,15 @@ There are two reasons for it: - A minor reason: sometimes it is less efficient in case the packing is used for very few slots (since for correct unpacking we need to provide the number of slots for each packing type). -- A fundamental reason: currently enum indeces are stored directly in the merkle tree & have very strict order of +- A fundamental reason: currently enum indices are stored directly in the merkle tree & have very strict order of incrementing enforced by the circuits and (they are given in order by pairs `(address, key)`), which are generally not accessible from pubdata. All this means that we are not allowed to change the order of “first writes” above, so indexes for them are directly recoverable from their order, and so we can not permute them. If we were to reorder keys without supplying the new -enumeration indeces for them, the state would be unrecoverable. Always supplying the new enum index may add additional 5 +enumeration indices for them, the state would be unrecoverable. Always supplying the new enum index may add additional 5 bytes for each key, which might negate the compression benefits in a lot of cases. Even if the compression will still be beneficial, the added complexity may not be worth it. -That being said, we _could_ rearange those for _repeated_ writes, but for now we stick to the same value compression +That being said, we _could_ rearrange those for _repeated_ writes, but for now we stick to the same value compression format for simplicity. diff --git a/docs/specs/data_availability/reconstruction.md b/docs/specs/data_availability/reconstruction.md index 4b86b4325a22..a97f22298e01 100644 --- a/docs/specs/data_availability/reconstruction.md +++ b/docs/specs/data_availability/reconstruction.md @@ -1,4 +1,4 @@ -# L2 State Recosntruction Tool +# L2 State Reconstruction Tool Given that we post all data to L1, there is a tool, created by the [Equilibrium Team](https://equilibrium.co/) that solely uses L1 pubdata for reconstructing the state and verifying that the state root on L1 can be created using diff --git a/docs/specs/l1_l2_communication/l1_to_l2.md b/docs/specs/l1_l2_communication/l1_to_l2.md index 12a20292ef64..ed1605a039a6 100644 --- a/docs/specs/l1_l2_communication/l1_to_l2.md +++ b/docs/specs/l1_l2_communication/l1_to_l2.md @@ -108,7 +108,7 @@ In a very rare event when the team needs to revert the batch with the upgrade on [reset](https://github.com/code-423n4/2023-10-zksync/blob/ef99273a8fdb19f5912ca38ba46d6bd02071363d/code/contracts/ethereum/contracts/zksync/facets/Executor.sol#L412). Note, however, that we do not “remember” that certain batches had a version before the upgrade, i.e. if the reverted -batches will have to be reexecuted, the upgrade transaction must still be present there, even if some of the deleted +batches will have to be re-executed, the upgrade transaction must still be present there, even if some of the deleted batches were committed before the upgrade and thus didn’t contain the transaction. ### Execute diff --git a/docs/specs/l1_smart_contracts.md b/docs/specs/l1_smart_contracts.md index 3bcd6a1ac4fe..b5b0a484559c 100644 --- a/docs/specs/l1_smart_contracts.md +++ b/docs/specs/l1_smart_contracts.md @@ -23,7 +23,7 @@ implementation that is inspired by the [mudgen reference implementation](https:/ external functions, only the fallback that delegates a call to one of the facets (target/implementation contract). So even an upgrade system is a separate facet that can be replaced. -One of the differences from the reference implementation is access freezability. Each of the facets has an associated +One of the differences from the reference implementation is access freezable. Each of the facets has an associated parameter that indicates if it is possible to freeze access to the facet. Privileged actors can freeze the **diamond** (not a specific facet!) and all facets with the marker `isFreezable` should be inaccessible until the governor or admin unfreezes the diamond. Note that it is a very dangerous thing since the diamond proxy can freeze the upgrade system and diff --git a/docs/specs/prover/boojum_function_check_if_satisfied.md b/docs/specs/prover/boojum_function_check_if_satisfied.md index 31b3a88a59d6..922889b90d49 100644 --- a/docs/specs/prover/boojum_function_check_if_satisfied.md +++ b/docs/specs/prover/boojum_function_check_if_satisfied.md @@ -70,7 +70,7 @@ we iterate over each gate_type_id and evaluator. ![Check_if_satisfied(9).png](<./img/boojum_function_check_if_satisfied/Check_if_satisfied(9).png>) If gate_type_id is a LookupFormalGate, we don’t need to do anything in this loop because it is handled by the lookup -table. For all other cases, we need to check the evaluator’s total_quotient_terms_over_all_repititions is non-zero. +table. For all other cases, we need to check the evaluator’s total_quotient_terms_over_all_repetitions is non-zero. ![Check_if_satisfied(11).png](<./img/boojum_function_check_if_satisfied/Check_if_satisfied(11).png>) diff --git a/docs/specs/prover/boojum_gadgets.md b/docs/specs/prover/boojum_gadgets.md index 666422e53f45..22fdbcb89a97 100644 --- a/docs/specs/prover/boojum_gadgets.md +++ b/docs/specs/prover/boojum_gadgets.md @@ -86,7 +86,7 @@ pub struct Variable(pub(crate) u64); ``` which is represented in the current Field. Variable is quite diverse, and to have "good" alignment and size we manually -do encoding management to be able to represent it as both copiable variable or witness. +do encoding management to be able to represent it as both copyable variable or witness. The implementation of this circuit type itself is similar. We can also divide them into classes as main and dependent: Such type like U8-U512 decoding inside functions to Num for using them in logical operations. As mentioned above, the @@ -151,7 +151,7 @@ And here is the main functions: ```rust fn push(&mut self, value: Element) { - // increment lenght + // increment length // head - hash(head, value) // witness.push_back(value.witness) } diff --git a/docs/specs/prover/circuit_testing.md b/docs/specs/prover/circuit_testing.md index 16f1e7b8209f..e0e59caf819f 100644 --- a/docs/specs/prover/circuit_testing.md +++ b/docs/specs/prover/circuit_testing.md @@ -49,7 +49,7 @@ We have to use special integer types because we are working in a finite field. The constants here are specific to the curve used, and are described in detail by code comments in the ecrecover_precompile_inner_routine. -Finally we can call the ecrecover_precompile_innner_routine: +Finally we can call the ecrecover_precompile_inner_routine: ![Contest(11).png](<./img/circuit_testing/Contest(11).png>) diff --git a/docs/specs/prover/circuits/README.md b/docs/specs/prover/circuits/README.md index fbec506d8f91..f0da38ab3aeb 100644 --- a/docs/specs/prover/circuits/README.md +++ b/docs/specs/prover/circuits/README.md @@ -1,7 +1,7 @@ # Circuits - [Overview](./overview.md) -- [Code decommiter](./code_decommitter.md) +- [Code decommitter](./code_decommitter.md) - [Demux log queue](./demux_log_queue.md) - [ECRecover](./ecrecover.md) - [Keccak round function](./keccak_round_function.md) diff --git a/docs/specs/prover/circuits/code_decommitter.md b/docs/specs/prover/circuits/code_decommitter.md index 91ff102afcb2..2e5b9609de10 100644 --- a/docs/specs/prover/circuits/code_decommitter.md +++ b/docs/specs/prover/circuits/code_decommitter.md @@ -23,12 +23,12 @@ pub struct CodeDecommitterOutputData { ```rust pub struct CodeDecommitterFSMInputOutput { - pub internal_fsm: CodeDecommittmentFSM, - pub decommittment_requests_queue_state: QueueState, + pub internal_fsm: CodeDecommitmentFSM, + pub decommitment_requests_queue_state: QueueState, pub memory_queue_state: QueueState, } -pub struct CodeDecommittmentFSM { +pub struct CodeDecommitmentFSM { pub sha256_inner_state: [UInt32; 8], // 8 uint32 words of internal sha256 state pub hash_to_compare_against: UInt256, pub current_index: UInt32, @@ -64,7 +64,7 @@ let mut structured_input = CodeDecommitterCycleInputOutput::alloc_ignoring_outputs(cs, closed_form_input.clone()); ``` -We chose what `memory_queue` state and `deccomitments_queue` state to continue to work with. +We chose what `memory_queue` state and `decommitments_queue` state to continue to work with. ```rust let requests_queue_state = QueueState::conditionally_select( @@ -75,7 +75,7 @@ let requests_queue_state = QueueState::conditionally_select( .sorted_requests_queue_initial_state, &structured_input .hidden_fsm_input - .decommittment_requests_queue_state, + .decommitment_requests_queue_state, ); let memory_queue_state = QueueState::conditionally_select( @@ -89,7 +89,7 @@ let memory_queue_state = QueueState::conditionally_select( We do the same with inner FSM part. ```rust -let initial_state = CodeDecommittmentFSM::conditionally_select( +let initial_state = CodeDecommitmentFSM::conditionally_select( cs, structured_input.start_flag, &starting_fsm_state, diff --git a/docs/specs/prover/circuits/demux_log_queue.md b/docs/specs/prover/circuits/demux_log_queue.md index 27bc596bfcf2..f84c8d1ea1ed 100644 --- a/docs/specs/prover/circuits/demux_log_queue.md +++ b/docs/specs/prover/circuits/demux_log_queue.md @@ -60,7 +60,7 @@ structured_input .enforce_trivial_head(cs); ``` -So long as `tail` is some equivalent of the merkel tree root and `head` is an equivalent of the current node hash, we +So long as `tail` is some equivalent of the merkle tree root and `head` is an equivalent of the current node hash, we provide some path witness when we pop elements and require that we properly end up in the root. So we must prove that element of head is zero: diff --git a/docs/specs/prover/circuits/log_sorter.md b/docs/specs/prover/circuits/log_sorter.md index 05294b6f63c4..87edeb2057c0 100644 --- a/docs/specs/prover/circuits/log_sorter.md +++ b/docs/specs/prover/circuits/log_sorter.md @@ -197,14 +197,14 @@ Additional checks for length. We should always check whether the sorted queue an length. ```rust -let unsorted_queue_lenght = Num::from_variable(unsorted_queue.length.get_variable()); -let intermediate_sorted_queue_lenght = +let unsorted_queue_length = Num::from_variable(unsorted_queue.length.get_variable()); +let intermediate_sorted_queue_length = Num::from_variable(intermediate_sorted_queue.length.get_variable()); Num::enforce_equal( cs, - &unsorted_queue_lenght, - &intermediate_sorted_queue_lenght, + &unsorted_queue_length, + &intermediate_sorted_queue_length, ); ``` @@ -266,7 +266,7 @@ values_are_equal.conditionally_enforce_true(cs, should_enforce); let this_item_is_non_trivial_rollback = Boolean::multi_and(cs, &[sorted_item.rollback, should_pop]); let negate_previous_item_rollback = previous_item.rollback.negated(cs); -let prevous_item_is_non_trivial_write = Boolean::multi_and( +let previous_item_is_non_trivial_write = Boolean::multi_and( cs, &[negate_previous_item_rollback, negate_previous_is_trivial], ); @@ -274,7 +274,7 @@ let is_sequential_rollback = Boolean::multi_and( cs, &[ this_item_is_non_trivial_rollback, - prevous_item_is_non_trivial_write, + previous_item_is_non_trivial_write, ], ); same_log.conditionally_enforce_true(cs, is_sequential_rollback); diff --git a/docs/specs/prover/circuits/main_vm.md b/docs/specs/prover/circuits/main_vm.md index cc18f7b56610..048ed9ea3ca9 100644 --- a/docs/specs/prover/circuits/main_vm.md +++ b/docs/specs/prover/circuits/main_vm.md @@ -40,8 +40,8 @@ pub struct VmLocalState { pub callstack: Callstack, pub memory_queue_state: [Num; FULL_SPONGE_QUEUE_STATE_WIDTH], pub memory_queue_length: UInt32, - pub code_decommittment_queue_state: [Num; FULL_SPONGE_QUEUE_STATE_WIDTH], - pub code_decommittment_queue_length: UInt32, + pub code_decommitment_queue_state: [Num; FULL_SPONGE_QUEUE_STATE_WIDTH], + pub code_decommitment_queue_length: UInt32, pub context_composite_u128: [UInt32; 4], } ``` @@ -120,7 +120,7 @@ for _cycle_idx in 0..limit { The VM runs in cycles. For each cycle, 1. Start in a prestate - perform all common operations for every opcode, namely deal with exceptions, resources, edge - cases like end of execution, select opcods, compute common values. Within the zkEVM framework, numerous entities + cases like end of execution, select opcodes, compute common values. Within the zkEVM framework, numerous entities identified as "opcodes" in the EVM paradigm are elegantly manifested as mere function calls. This modification is rooted in the succinct observation that, from the perspective of an external caller, an inlined function (analogous to an opcode) is inherently indistinguishable from an internal function call. @@ -185,7 +185,7 @@ pub struct StateDiffsAccumulator { // other meta parameters of VM pub new_tx_number: Option<(Boolean, UInt32)>, pub new_ergs_per_pubdata: Option<(Boolean, UInt32)>, - // memory bouds + // memory bounds pub new_heap_bounds: Vec<(Boolean, UInt32)>, pub new_aux_heap_bounds: Vec<(Boolean, UInt32)>, // u128 special register, one from context, another from call/ret @@ -194,7 +194,7 @@ pub struct StateDiffsAccumulator { pub callstacks: Vec<(Boolean, Callstack)>, // memory page counter pub memory_page_counters: Option>, - // decommittment queue + // decommitment queue pub decommitment_queue_candidates: Option<( Boolean, UInt32, @@ -271,8 +271,8 @@ let memory_queue_final_tail = QueueTailState::conditionally_select( ```rust let decommitment_queue_current_tail = QueueTailState { - tail: final_state.code_decommittment_queue_state, - length: final_state.code_decommittment_queue_length, + tail: final_state.code_decommitment_queue_state, + length: final_state.code_decommitment_queue_length, }; let decommitment_queue_final_tail = QueueTailState::conditionally_select( cs, diff --git a/docs/specs/prover/circuits/overview.md b/docs/specs/prover/circuits/overview.md index 73b164230e55..4e0a8ea19b41 100644 --- a/docs/specs/prover/circuits/overview.md +++ b/docs/specs/prover/circuits/overview.md @@ -38,7 +38,7 @@ That’s how we can make our prover structure more optimized and flexible. For now, we have 13 base layer circuits: - [MainVM](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/Main%20Vm.md) -- [CodeDecommittmentsSorter](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/SortDecommitments.md) +- [CodeDecommitmentsSorter](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/SortDecommitments.md) - [CodeDecommitter](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/CodeDecommitter.md) - [LogDemuxer](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/DemuxLogQueue.md) - [KeccakRoundFunction](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Circuits%20Section/Circuits/KeccakRoundFunction.md) diff --git a/docs/specs/prover/circuits/sha256_round_function.md b/docs/specs/prover/circuits/sha256_round_function.md index 055d45a50fd7..959c9b979ccc 100644 --- a/docs/specs/prover/circuits/sha256_round_function.md +++ b/docs/specs/prover/circuits/sha256_round_function.md @@ -127,7 +127,7 @@ Form the final state (depending on flag we choose between states): ); structured_input.hidden_fsm_output.internal_fsm = final_state; - structured_input.hidden_fsm_output.log_queue_state = final_requets_state; + structured_input.hidden_fsm_output.log_queue_state = final_request_state; structured_input.hidden_fsm_output.memory_queue_state = final_memory_state; ``` @@ -166,7 +166,7 @@ progress: ```rust let input_queue_is_empty = precompile_calls_queue.is_empty(cs); -let can_finish_immediatelly = +let can_finish_immediately = Boolean::multi_and(cs, &[state.read_precompile_call, input_queue_is_empty]); ``` diff --git a/docs/specs/prover/circuits/sort_decommitments.md b/docs/specs/prover/circuits/sort_decommitments.md index 8fd4ea11b281..622f91054c4b 100644 --- a/docs/specs/prover/circuits/sort_decommitments.md +++ b/docs/specs/prover/circuits/sort_decommitments.md @@ -5,7 +5,7 @@ ### [Input](https://github.com/matter-labs/era-zkevm_circuits/blob/main/src/sort_decommittment_requests/input.rs#L62) ```rust -pub struct CodeDecommittmentsDeduplicatorInputData { +pub struct CodeDecommitmentsDeduplicatorInputData { pub initial_queue_state: QueueState, pub sorted_queue_initial_state: QueueState, } @@ -14,7 +14,7 @@ pub struct CodeDecommittmentsDeduplicatorInputData { ### [Output](https://github.com/matter-labs/era-zkevm_circuits/blob/main/src/sort_decommittment_requests/input.rs#L81) ```rust -pub struct CodeDecommittmentsDeduplicatorOutputData { +pub struct CodeDecommitmentsDeduplicatorOutputData { pub final_queue_state: QueueState, } ``` @@ -22,7 +22,7 @@ pub struct CodeDecommittmentsDeduplicatorOutputData { ### [FSM Input and FSM Output](https://github.com/matter-labs/era-zkevm_circuits/blob/main/src/sort_decommittment_requests/input.rs#L26) ```rust -pub struct CodeDecommittmentsDeduplicatorFSMInputOutput { +pub struct CodeDecommitmentsDeduplicatorFSMInputOutput { pub initial_queue_state: QueueState, pub sorted_queue_state: QueueState, pub final_queue_state: QueueState, @@ -39,10 +39,10 @@ pub struct CodeDecommittmentsDeduplicatorFSMInputOutput { ## Main circuit logic This circuit handles the sorting and deduplication of code cancellation requests. Before starting, during the pre-start -phase, the first decommiter queue is generated. To decommiter a code, the input will receive the hash root of the code, -the length of the code, the code hash of the opcode, the number of opcodes and the code of the page. Next, it sorts the -queue and, in the process, identifies and removes identical requests, serving as a filtering mechanism in case the same -contract is called several times. +phase, the first decommitter queue is generated. To decommitter a code, the input will receive the hash root of the +code, the length of the code, the code hash of the opcode, the number of opcodes and the code of the page. Next, it +sorts the queue and, in the process, identifies and removes identical requests, serving as a filtering mechanism in case +the same contract is called several times. The detailed explanation of sorting and deduplicating can be found [here](https://github.com/code-423n4/2023-10-zksync/blob/c3ff020df5d11fe91209bd99d7fb0ec1272dc387/docs/Circuits%20Section/Circuits/Sorting.md). @@ -52,13 +52,13 @@ The detailed explanation of sorting and deduplicating can be found The circuit begins with allocating input part of the PI. ```rust -let CodeDecommittmentsDeduplicatorInstanceWitness { +let CodeDecommitmentsDeduplicatorInstanceWitness { closed_form_input, initial_queue_witness, sorted_queue_witness, } = witness; -let mut structured_input = CodeDecommittmentsDeduplicatorInputOutput::alloc_ignoring_outputs( +let mut structured_input = CodeDecommitmentsDeduplicatorInputOutput::alloc_ignoring_outputs( cs, closed_form_input.clone(), ); diff --git a/docs/specs/prover/circuits/storage_application.md b/docs/specs/prover/circuits/storage_application.md index 862f0c262b53..05f013de888f 100644 --- a/docs/specs/prover/circuits/storage_application.md +++ b/docs/specs/prover/circuits/storage_application.md @@ -133,7 +133,7 @@ state_diff_data.derived_key = UInt8::parallel_select( ... ``` -Finally, we compute a new merkel path. +Finally, we compute a new Merkle path. ```rust let mut current_hash = blake2s(cs, &leaf_bytes); diff --git a/docs/specs/prover/circuits/storage_sorter.md b/docs/specs/prover/circuits/storage_sorter.md index 3a6757af5c5e..d7f82cd9acbe 100644 --- a/docs/specs/prover/circuits/storage_sorter.md +++ b/docs/specs/prover/circuits/storage_sorter.md @@ -273,9 +273,9 @@ new_lhs.iter().zip(new_rhs).for_each(|(l, r)| { Now we update PI output parts and compute a commitment. Then we allocate it as public variables. ```rust -let input_committment = +let input_commitment = commit_variable_length_encodable_item(cs, &compact_form, round_function); -for el in input_committment.iter() { +for el in input_commitment.iter() { let gate = PublicInputGate::new(el.get_variable()); gate.add_to_cs(cs); } diff --git a/docs/specs/the_hyperchain/shared_bridge.md b/docs/specs/the_hyperchain/shared_bridge.md index a57c1ff7dd7b..72b4970dd356 100644 --- a/docs/specs/the_hyperchain/shared_bridge.md +++ b/docs/specs/the_hyperchain/shared_bridge.md @@ -119,8 +119,8 @@ be able to leverage them when available). - Some assets have to be natively supported (ETH, WETH) and it also makes sense to support some generally accepted token standards (ERC20 tokens), as this makes it easy to bridge those tokens (and ensures a single version of them exists on - the hyperchain). These cannonical asset contracts are deployed from L1 by a bridge shared by all hyperchains. This is - where assets are locked on L1. These bridges use the Bridgehub to communicate with all hyperchains. Currently, these + the hyperchain). These canonical asset contracts are deployed from L1 by a bridge shared by all hyperchains. This is + where assets are locked on L1. These bridges use the BridgeHub to communicate with all hyperchains. Currently, these bridges are the `WETH` and `ERC20` bridges. - The pair on L2 is deployed from L1. The hash of the factory dependencies is stored on L1, and when a hyperchain diff --git a/docs/specs/zk_evm/bootloader.md b/docs/specs/zk_evm/bootloader.md index bdadd1d3fce1..9a4b3f81b155 100644 --- a/docs/specs/zk_evm/bootloader.md +++ b/docs/specs/zk_evm/bootloader.md @@ -76,7 +76,7 @@ supported: transactions. The L1 contract ensures that the hash did indeed match the [hashes of the priority transactions on L1](https://github.com/code-423n4/2023-10-zksync/blob/ef99273a8fdb19f5912ca38ba46d6bd02071363d/code/contracts/ethereum/contracts/zksync/facets/Executor.sol#L282). -You can also read more on L1->L2 transactions and upgrade transacitons +You can also read more on L1->L2 transactions and upgrade transactions [here](https://github.com/code-423n4/2023-10-zksync/blob/main/docs/Smart%20contract%20Section/Handling%20L1%E2%86%92L2%20ops%20on%20zkSync.md). However, as already stated, the bootloader’s memory is not deterministic and the operator is free to put anything it @@ -129,7 +129,7 @@ Once read, these slots can be used for temporary data. - `[7283..40050]` – slots used for compressed bytecodes each in the following format: - 32 bytecode hash - 32 zeroes (but then it will be modified by the bootloader to contain 28 zeroes and then the 4-byte selector of the - `publishCompressedBytecode` function of the `BytecodeCompresor`) + `publishCompressedBytecode` function of the `BytecodeCompressor`) - The calldata to the bytecode compressor (without the selector). - `[40051..40052]` – slots where the hash and the number of current priority ops is stored. More on it in the priority operations @@ -152,9 +152,9 @@ entries. These will not be published to L1, but will be used to verify the corre worst-case number of bytes that may be needed for this scratch space is if all the pubdata consists of repeated writes (i.e. we’ll need only 4 bytes to include key) that turn into 0 (i.e. they’ll need only 1 byte to describe it). However, each of these writes in the uncompressed form will be represented as 272 byte state diff entry and so we get the number -of diffs is `120k / 5 = 24k`. This means that they will have accoomdate `24k * 272 = 6528000` bytes of calldata for the +of diffs is `120k / 5 = 24k`. This means that they will have accommodate `24k * 272 = 6528000` bytes of calldata for the uncompressed state diffs. Adding 120k on top leaves us with roughly `6650000` bytes needed for calldata. `207813` slots -are needed to accomodate this amount of data. We round up to `208000` slots to give space for constant-size factors for +are needed to accommodate this amount of data. We round up to `208000` slots to give space for constant-size factors for ABI-encoding, like offsets, lengths, etc. In theory though much more calldata could be used (if for instance 1 byte is used for enum index). It is the diff --git a/docs/specs/zk_evm/fee_model.md b/docs/specs/zk_evm/fee_model.md index 2298529595a1..a75d45a737b9 100644 --- a/docs/specs/zk_evm/fee_model.md +++ b/docs/specs/zk_evm/fee_model.md @@ -478,7 +478,7 @@ Note, that the computational overhead is proportional to the `tx.gasLimit` and t pay is proportional to the L1 gas price (recall the formula of `B_O`). We can roughly express the transaction overhead from computation as `tx.gasLimit * L1_GAS_PRICE * C` where `C` is just some constant. Note, that since a transaction typically contains some storage writes, and its -`tx.gasLimit = gasSpentOnCompute + pubdataPublished * gasPricePerPubdata`, `tx.gasLimit` is roughly proprtional to +`tx.gasLimit = gasSpentOnCompute + pubdataPublished * gasPricePerPubdata`, `tx.gasLimit` is roughly proportional to `gasPricePerPubdata` and so it is also proportional to `L1_GAS_PRICE`. This means that formula `tx.gasLimit * L1_GAS_PRICE * C` becomes _quadratic_ to the `L1_GAS_PRICE`. diff --git a/docs/specs/zk_evm/vm_overview.md b/docs/specs/zk_evm/vm_overview.md index a664352c54ef..3d7062cfb1f1 100644 --- a/docs/specs/zk_evm/vm_overview.md +++ b/docs/specs/zk_evm/vm_overview.md @@ -15,7 +15,7 @@ The zkEVM has a lot of special features compared to the EVM that are needed for precompiles etc. These functions are either built into the zkEVM, or there are special [system contracts](./system_contracts.md) for them. The system contracts are deployed at predefined addresses, they are called by the Bootloader, and they have special permissions compared to normal user contracts. These are not to be -confused with the [precompiles](./precompiles.md), which are also predeloyed contracts with special support from the +confused with the [precompiles](./precompiles.md), which are also pre-deployed contracts with special support from the zkEVM, but these contract do not have special permissions and are called by the users and not the Bootloader. The zkEVM also has user-facing features. For the best possible UX the ZK Stack supports native diff --git a/docs/specs/zk_evm/vm_specification/compiler/instructions/extensions/call.md b/docs/specs/zk_evm/vm_specification/compiler/instructions/extensions/call.md index 9833205e20ba..20033e8500aa 100644 --- a/docs/specs/zk_evm/vm_specification/compiler/instructions/extensions/call.md +++ b/docs/specs/zk_evm/vm_specification/compiler/instructions/extensions/call.md @@ -32,8 +32,8 @@ manipulations, since one can not easily load pointer value into Solidity’s var | system_mimic_call_byref(to, ACTIVE_PTR, implicit r3, r4, r5 = who to mimic) | who_to_call | 0xFFF8 | 0 | 0 | who_to_mimic | value_to_put_into_r3 | value_to_put_into_r4 | WILL mess up the registers and WILL use r1-r4 for our standard ABI convention and r5 for the extra who_to_mimic argument | any in the code; mimic call in the bytecode | Runtime *{i256, i1} \_\_mimiccall(*i8 addrspace(3), i256, i256, \*{i256, i1}) | Same as one above, but takes ABI data from ACTIVE_PTR | | raw_far_call | who_to_call | 0xFFF7 | 0 | 0 | abi_data (CAN be with “to system = true”) | output_offset | output_length | Same as for EVM call | call | static | delegate (the call type is preserved) | | It’s very similar to “system_call” described below, but for the cases when we only need to have to_system = true set in ABI (responsibility of the user, NOT the compiler), but we do not actually need to pass anything through r3 and r4 (so we can save on setting them or zeroing them, whatever) | | raw_far_call_byref | who_to_call | 0xFFF6 | 0 | 0 | 0xFFFF to prevent optimizing out by Yul | output_offset | output_length | Same as for EVM call | call | static | delegate (the call type is preserved) | | Same as one above, but takes ABI data from ACTIVE_PTR | -| system_call | who_to_call | 0xFFF5 | value_to_put_into_r3 (only for call with 7 arguments) | value_to_put_into_r4 | abi_data (MUST have “to system” set) | value_to_put_into_r5 | value_to_put_into_r6 | Same as for EVM call | call | static | delegate (the call type is preserved) | to call system contracts, like MSG_VALUE_SIMUALTOR. We may need 4 different formal definitions for cases when we would want to have integer/ptr in r3 and r4 | | -| system_call_byref | who_to_call | 0xFFF4 | value_to_put_into_r3 (only for call with 7 arguments) | value_to_put_into_r4 | 0xFFFF to prevent optimizing out by Yul | value_to_put_into_r5 | value_to_put_into_r6 | Same as for EVM call | call | static | delegate (the call type is preserved) | to call system contracts, like MSG_VALUE_SIMUALTOR. We may need 4 different formal definitions for cases when we would want to have integer/ptr in r3 and r4 | Same as one above, but takes ABI data from ACTIVE_PTR | +| system_call | who_to_call | 0xFFF5 | value_to_put_into_r3 (only for call with 7 arguments) | value_to_put_into_r4 | abi_data (MUST have “to system” set) | value_to_put_into_r5 | value_to_put_into_r6 | Same as for EVM call | call | static | delegate (the call type is preserved) | to call system contracts, like MSG_VALUE_SIMULATOR. We may need 4 different formal definitions for cases when we would want to have integer/ptr in r3 and r4 | | +| system_call_byref | who_to_call | 0xFFF4 | value_to_put_into_r3 (only for call with 7 arguments) | value_to_put_into_r4 | 0xFFFF to prevent optimizing out by Yul | value_to_put_into_r5 | value_to_put_into_r6 | Same as for EVM call | call | static | delegate (the call type is preserved) | to call system contracts, like MSG_VALUE_SIMULATOR. We may need 4 different formal definitions for cases when we would want to have integer/ptr in r3 and r4 | Same as one above, but takes ABI data from ACTIVE_PTR | | set_context_u128 | 0 | 0xFFF3 | value | 0 | 0xFFFF to prevent optimizing out by Yul | 0 | 0 | - | call | | | | set_pubdata_price | in0 | 0xFFF2 | 0 | 0 | 0xFFFF to prevent optimizing out by Yul | 0 | 0 | - | call | context.set_ergs_per_pubdata in0 in assembly | | | increment_tx_counter | 0 | 0xFFF1 | 0 | 0 | 0xFFFF to prevent optimizing out by Yul | 0 | 0 | - | call | context.inc_tx_num in assembly | | diff --git a/infrastructure/zk/package.json b/infrastructure/zk/package.json index c8ed6d63e50f..c3f0c33ade2d 100644 --- a/infrastructure/zk/package.json +++ b/infrastructure/zk/package.json @@ -32,6 +32,6 @@ "@types/tabtab": "^3.0.1", "hardhat": "=2.16.0", "typescript": "^4.3.5", - "cspell": "^8.2.3" + "cspell": "*" } } diff --git a/infrastructure/zk/src/spellcheck.ts b/infrastructure/zk/src/spellcheck.ts index dfa387921bcb..a47683732300 100644 --- a/infrastructure/zk/src/spellcheck.ts +++ b/infrastructure/zk/src/spellcheck.ts @@ -1,28 +1,34 @@ import { Command } from 'commander'; import * as utils from './utils'; -export async function runSpellCheck(pattern: string, config: string, useCargo: boolean, useCSpell: boolean) { +export async function runSpellCheck(pattern: string, useCargo: boolean, useCSpell: boolean) { const cSpellCommand = `cspell ${pattern} --config=./spellcheck/cspell.json`; const cargoCommand = `cargo spellcheck --cfg=./spellcheck/era.cfg`; try { + let results = []; if (!useCargo && !useCSpell) { - await Promise.all([utils.spawn(cSpellCommand), utils.spawn(cargoCommand)]); + results = await Promise.all([utils.spawn(cSpellCommand), utils.spawn(cargoCommand)]); } else { // Run cspell if specified - // zk spellcheck --use-cspell if (useCSpell) { - await utils.spawn(cSpellCommand); + results.push(await utils.spawn(cSpellCommand)); } // Run cargo spellcheck if specified - // zk spellcheck --use-cargo if (useCargo) { - await utils.spawn(cargoCommand); + results.push(await utils.spawn(cargoCommand)); } } + + // Check results and exit with error code if any command failed + if (results.some((code) => code !== 0)) { + console.error('Spell check failed'); + process.exit(1); + } } catch (error) { console.error('Error occurred during spell checking:', error); + process.exit(1); } } @@ -33,5 +39,5 @@ export const command = new Command('spellcheck') .option('--use-cspell', 'Use cspell') .description('Run spell check on specified files') .action((cmd) => { - runSpellCheck(cmd.pattern, cmd.config, cmd.useCargo, cmd.useCSpell); + runSpellCheck(cmd.pattern, cmd.useCargo, cmd.useCSpell); }); diff --git a/spellcheck/era.dic b/spellcheck/era.dic index 9bc9025729cd..57bb17a27e2f 100644 --- a/spellcheck/era.dic +++ b/spellcheck/era.dic @@ -514,6 +514,12 @@ porco rosso insize MLOAD +sload +sload +uadd +nocallback +nosync +swrite // ETC gitter @@ -817,6 +823,7 @@ compactions M6 compiler_common noop +tokenized rustc sqlx zkevm @@ -863,4 +870,6 @@ tokenized Aggregator DecommittmentProcessor decommitment -hardcoded \ No newline at end of file +hardcoded +plookup +shivini diff --git a/yarn.lock b/yarn.lock index d701e81aaa07..6b91505e3f7c 100644 --- a/yarn.lock +++ b/yarn.lock @@ -368,6 +368,348 @@ resolved "https://registry.yarnpkg.com/@colors/colors/-/colors-1.5.0.tgz#bb504579c1cae923e6576a4f5da43d25f97bdbd9" integrity sha512-ooWCrlZP11i8GImSjTHYHLkvFDP48nS4+204nGb1RiX/WXYHmJA2III9/e2DWVabCESdW7hBAEzHRqUn9OUVvQ== +"@cspell/cspell-bundled-dicts@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-bundled-dicts/-/cspell-bundled-dicts-8.1.3.tgz#39d92ebeb9eeeeb943cc7e05525731a06b2f388a" + integrity sha512-TwLyL2bCtetXGhMudjOIgFPAsWF2UkT0E7T+DAZG8aUBfHoC/eco/sTmR6UJVpi6Crjs0YOQkFUBGrQ2pxJPcA== + dependencies: + "@cspell/dict-ada" "^4.0.2" + "@cspell/dict-aws" "^4.0.0" + "@cspell/dict-bash" "^4.1.3" + "@cspell/dict-companies" "^3.0.28" + "@cspell/dict-cpp" "^5.0.10" + "@cspell/dict-cryptocurrencies" "^4.0.0" + "@cspell/dict-csharp" "^4.0.2" + "@cspell/dict-css" "^4.0.12" + "@cspell/dict-dart" "^2.0.3" + "@cspell/dict-django" "^4.1.0" + "@cspell/dict-docker" "^1.1.7" + "@cspell/dict-dotnet" "^5.0.0" + "@cspell/dict-elixir" "^4.0.3" + "@cspell/dict-en-common-misspellings" "^1.0.2" + "@cspell/dict-en-gb" "1.1.33" + "@cspell/dict-en_us" "^4.3.12" + "@cspell/dict-filetypes" "^3.0.3" + "@cspell/dict-fonts" "^4.0.0" + "@cspell/dict-fsharp" "^1.0.1" + "@cspell/dict-fullstack" "^3.1.5" + "@cspell/dict-gaming-terms" "^1.0.4" + "@cspell/dict-git" "^2.0.0" + "@cspell/dict-golang" "^6.0.5" + "@cspell/dict-haskell" "^4.0.1" + "@cspell/dict-html" "^4.0.5" + "@cspell/dict-html-symbol-entities" "^4.0.0" + "@cspell/dict-java" "^5.0.6" + "@cspell/dict-k8s" "^1.0.2" + "@cspell/dict-latex" "^4.0.0" + "@cspell/dict-lorem-ipsum" "^4.0.0" + "@cspell/dict-lua" "^4.0.3" + "@cspell/dict-makefile" "^1.0.0" + "@cspell/dict-node" "^4.0.3" + "@cspell/dict-npm" "^5.0.13" + "@cspell/dict-php" "^4.0.4" + "@cspell/dict-powershell" "^5.0.2" + "@cspell/dict-public-licenses" "^2.0.5" + "@cspell/dict-python" "^4.1.10" + "@cspell/dict-r" "^2.0.1" + "@cspell/dict-ruby" "^5.0.1" + "@cspell/dict-rust" "^4.0.1" + "@cspell/dict-scala" "^5.0.0" + "@cspell/dict-software-terms" "^3.3.11" + "@cspell/dict-sql" "^2.1.2" + "@cspell/dict-svelte" "^1.0.2" + "@cspell/dict-swift" "^2.0.1" + "@cspell/dict-typescript" "^3.1.2" + "@cspell/dict-vue" "^3.0.0" + +"@cspell/cspell-json-reporter@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-json-reporter/-/cspell-json-reporter-8.1.3.tgz#5645bead06de8bbacedc654b61663b05c57bd8d0" + integrity sha512-9iOU0Y733XuF0cqC7xwzJkOKFdJ65rYGnHFdUHzr5lxEqeG9X/jhlkzyHuGGOhPxkUeFP1x9XoLhXo1isMDbKA== + dependencies: + "@cspell/cspell-types" "8.1.3" + +"@cspell/cspell-pipe@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-pipe/-/cspell-pipe-8.1.3.tgz#728567a69d7c315377c5a1178bbe9151da82048a" + integrity sha512-/dcnyLDeyFuoX4seZv7VsDQyRpt3ZY0vjZiDpqFul8hPydM8czLyRPPMD6Za+Gqg6dZmh9+VsQWK52hVsqc0QA== + +"@cspell/cspell-resolver@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-resolver/-/cspell-resolver-8.1.3.tgz#e54bbb0b5d06813907c92cdead6d1dea12a54263" + integrity sha512-bGyJYqkHRilqhyKGL/NvODN5U+UvCuQo7kxgt0i3Vd7m7k6XYLsSLYZ4w6r1S5IQ/ybU8I5lh6/6fNqKwvo9eg== + dependencies: + global-directory "^4.0.1" + +"@cspell/cspell-service-bus@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-service-bus/-/cspell-service-bus-8.1.3.tgz#2b34012d7905a596adf4a8a8daaead86c0d32666" + integrity sha512-8E5ZveQKneNfK+cuFMy0y6tDsho71UPppEHNoLZsEFDbIxDdtQcAfs0pk4nwEzxPBt+dBB+Yl8KExQ6x2FAYQw== + +"@cspell/cspell-types@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/cspell-types/-/cspell-types-8.1.3.tgz#aaba55f4a8f6496a30ccf2235675ec6c21b9e33a" + integrity sha512-j14FENj+DzWu6JjzTl+0X5/OJv9AEckpEp6Jaw9YglxirrBBzTkZGfoLePe/AWo/MlIYp0asl92C1UHEjgz+FQ== + +"@cspell/dict-ada@^4.0.2": + version "4.0.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-ada/-/dict-ada-4.0.2.tgz#8da2216660aeb831a0d9055399a364a01db5805a" + integrity sha512-0kENOWQeHjUlfyId/aCM/mKXtkEgV0Zu2RhUXCBr4hHo9F9vph+Uu8Ww2b0i5a4ZixoIkudGA+eJvyxrG1jUpA== + +"@cspell/dict-aws@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-aws/-/dict-aws-4.0.0.tgz#ab71fe0c05d9ad662d27495e74361bdcb5b470eb" + integrity sha512-1YkCMWuna/EGIDN/zKkW+j98/55mxigftrSFgsehXhPld+ZMJM5J9UuBA88YfL7+/ETvBdd7mwW6IwWsC+/ltQ== + +"@cspell/dict-bash@^4.1.3": + version "4.1.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-bash/-/dict-bash-4.1.3.tgz#25fba40825ac10083676ab2c777e471c3f71b36e" + integrity sha512-tOdI3QVJDbQSwPjUkOiQFhYcu2eedmX/PtEpVWg0aFps/r6AyjUQINtTgpqMYnYuq8O1QUIQqnpx21aovcgZCw== + +"@cspell/dict-companies@^3.0.28": + version "3.0.28" + resolved "https://registry.yarnpkg.com/@cspell/dict-companies/-/dict-companies-3.0.28.tgz#d617be3e036955d2f656d568f0cc6d1bdf198819" + integrity sha512-UinHkMYB/1pUkLKm1PGIm9PBFYxeAa6YvbB1Rq/RAAlrs0WDwiDBr3BAYdxydukG1IqqwT5z9WtU+8D/yV/5lw== + +"@cspell/dict-cpp@^5.0.10": + version "5.0.10" + resolved "https://registry.yarnpkg.com/@cspell/dict-cpp/-/dict-cpp-5.0.10.tgz#08c3eb438b631dd3f0fc04f5a6d4b6cab87c8d9b" + integrity sha512-WCRuDrkFdpmeIR6uXQYKU9loMQKNFS4bUhtHdv5fu4qVyJSh3k/kgmtTm1h1BDTj8EwPRc/RGxS+9Z3b2mnabA== + +"@cspell/dict-cryptocurrencies@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-cryptocurrencies/-/dict-cryptocurrencies-4.0.0.tgz#6517a7e1b0ed184cf3fc18f03230c82508369dec" + integrity sha512-EiZp91ATyRxTmauIQfOX9adLYCunKjHEh092rrM7o2eMXP9n7zpXAL9BK7LviL+LbB8VDOm21q+s83cKrrRrsg== + +"@cspell/dict-csharp@^4.0.2": + version "4.0.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-csharp/-/dict-csharp-4.0.2.tgz#e55659dbe594e744d86b1baf0f3397fe57b1e283" + integrity sha512-1JMofhLK+4p4KairF75D3A924m5ERMgd1GvzhwK2geuYgd2ZKuGW72gvXpIV7aGf52E3Uu1kDXxxGAiZ5uVG7g== + +"@cspell/dict-css@^4.0.12": + version "4.0.12" + resolved "https://registry.yarnpkg.com/@cspell/dict-css/-/dict-css-4.0.12.tgz#59abf3512ae729835c933c38f64a3d8a5f09ce3d" + integrity sha512-vGBgPM92MkHQF5/2jsWcnaahOZ+C6OE/fPvd5ScBP72oFY9tn5GLuomcyO0z8vWCr2e0nUSX1OGimPtcQAlvSw== + +"@cspell/dict-dart@^2.0.3": + version "2.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-dart/-/dict-dart-2.0.3.tgz#75e7ffe47d5889c2c831af35acdd92ebdbd4cf12" + integrity sha512-cLkwo1KT5CJY5N5RJVHks2genFkNCl/WLfj+0fFjqNR+tk3tBI1LY7ldr9piCtSFSm4x9pO1x6IV3kRUY1lLiw== + +"@cspell/dict-data-science@^1.0.11": + version "1.0.11" + resolved "https://registry.yarnpkg.com/@cspell/dict-data-science/-/dict-data-science-1.0.11.tgz#4eabba75c21d27253c1114b4fbbade0ead739ffc" + integrity sha512-TaHAZRVe0Zlcc3C23StZqqbzC0NrodRwoSAc8dis+5qLeLLnOCtagYQeROQvDlcDg3X/VVEO9Whh4W/z4PAmYQ== + +"@cspell/dict-django@^4.1.0": + version "4.1.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-django/-/dict-django-4.1.0.tgz#2d4b765daf3c83e733ef3e06887ea34403a4de7a" + integrity sha512-bKJ4gPyrf+1c78Z0Oc4trEB9MuhcB+Yg+uTTWsvhY6O2ncFYbB/LbEZfqhfmmuK/XJJixXfI1laF2zicyf+l0w== + +"@cspell/dict-docker@^1.1.7": + version "1.1.7" + resolved "https://registry.yarnpkg.com/@cspell/dict-docker/-/dict-docker-1.1.7.tgz#bcf933283fbdfef19c71a642e7e8c38baf9014f2" + integrity sha512-XlXHAr822euV36GGsl2J1CkBIVg3fZ6879ZOg5dxTIssuhUOCiV2BuzKZmt6aIFmcdPmR14+9i9Xq+3zuxeX0A== + +"@cspell/dict-dotnet@^5.0.0": + version "5.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-dotnet/-/dict-dotnet-5.0.0.tgz#13690aafe14b240ad17a30225ac1ec29a5a6a510" + integrity sha512-EOwGd533v47aP5QYV8GlSSKkmM9Eq8P3G/eBzSpH3Nl2+IneDOYOBLEUraHuiCtnOkNsz0xtZHArYhAB2bHWAw== + +"@cspell/dict-elixir@^4.0.3": + version "4.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-elixir/-/dict-elixir-4.0.3.tgz#57c25843e46cf3463f97da72d9ef8e37c818296f" + integrity sha512-g+uKLWvOp9IEZvrIvBPTr/oaO6619uH/wyqypqvwpmnmpjcfi8+/hqZH8YNKt15oviK8k4CkINIqNhyndG9d9Q== + +"@cspell/dict-en-common-misspellings@^1.0.2": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-en-common-misspellings/-/dict-en-common-misspellings-1.0.2.tgz#3c4ebab8e9e906d66d60f53c8f8c2e77b7f108e7" + integrity sha512-jg7ZQZpZH7+aAxNBlcAG4tGhYF6Ksy+QS5Df73Oo+XyckBjC9QS+PrRwLTeYoFIgXy5j3ICParK5r3MSSoL4gw== + +"@cspell/dict-en-gb@1.1.33": + version "1.1.33" + resolved "https://registry.yarnpkg.com/@cspell/dict-en-gb/-/dict-en-gb-1.1.33.tgz#7f1fd90fc364a5cb77111b5438fc9fcf9cc6da0e" + integrity sha512-tKSSUf9BJEV+GJQAYGw5e+ouhEe2ZXE620S7BLKe3ZmpnjlNG9JqlnaBhkIMxKnNFkLY2BP/EARzw31AZnOv4g== + +"@cspell/dict-en_us@^4.3.12": + version "4.3.12" + resolved "https://registry.yarnpkg.com/@cspell/dict-en_us/-/dict-en_us-4.3.12.tgz#3b0ceaf5ed3cf30b225834ca7d528e4dc96e9605" + integrity sha512-1bsUxFjgxF30FTzcU5uvmCvH3lyqVKR9dbwsJhomBlUM97f0edrd6590SiYBXDm7ruE68m3lJd4vs0Ev2D6FtQ== + +"@cspell/dict-filetypes@^3.0.3": + version "3.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-filetypes/-/dict-filetypes-3.0.3.tgz#ab0723ca2f4d3d5674e9c9745efc9f144e49c905" + integrity sha512-J9UP+qwwBLfOQ8Qg9tAsKtSY/WWmjj21uj6zXTI9hRLD1eG1uUOLcfVovAmtmVqUWziPSKMr87F6SXI3xmJXgw== + +"@cspell/dict-fonts@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-fonts/-/dict-fonts-4.0.0.tgz#9bc8beb2a7b068b4fdb45cb994b36fd184316327" + integrity sha512-t9V4GeN/m517UZn63kZPUYP3OQg5f0OBLSd3Md5CU3eH1IFogSvTzHHnz4Wqqbv8NNRiBZ3HfdY/pqREZ6br3Q== + +"@cspell/dict-fsharp@^1.0.1": + version "1.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-fsharp/-/dict-fsharp-1.0.1.tgz#d62c699550a39174f182f23c8c1330a795ab5f53" + integrity sha512-23xyPcD+j+NnqOjRHgW3IU7Li912SX9wmeefcY0QxukbAxJ/vAN4rBpjSwwYZeQPAn3fxdfdNZs03fg+UM+4yQ== + +"@cspell/dict-fullstack@^3.1.5": + version "3.1.5" + resolved "https://registry.yarnpkg.com/@cspell/dict-fullstack/-/dict-fullstack-3.1.5.tgz#35d18678161f214575cc613dd95564e05422a19c" + integrity sha512-6ppvo1dkXUZ3fbYn/wwzERxCa76RtDDl5Afzv2lijLoijGGUw5yYdLBKJnx8PJBGNLh829X352ftE7BElG4leA== + +"@cspell/dict-gaming-terms@^1.0.4": + version "1.0.4" + resolved "https://registry.yarnpkg.com/@cspell/dict-gaming-terms/-/dict-gaming-terms-1.0.4.tgz#b67d89d014d865da6cb40de4269d4c162a00658e" + integrity sha512-hbDduNXlk4AOY0wFxcDMWBPpm34rpqJBeqaySeoUH70eKxpxm+dvjpoRLJgyu0TmymEICCQSl6lAHTHSDiWKZg== + +"@cspell/dict-git@^2.0.0": + version "2.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-git/-/dict-git-2.0.0.tgz#fa5cb298845da9c69efc01c6af07a99097718dc9" + integrity sha512-n1AxyX5Kgxij/sZFkxFJlzn3K9y/sCcgVPg/vz4WNJ4K9YeTsUmyGLA2OQI7d10GJeiuAo2AP1iZf2A8j9aj2w== + +"@cspell/dict-golang@^6.0.5": + version "6.0.5" + resolved "https://registry.yarnpkg.com/@cspell/dict-golang/-/dict-golang-6.0.5.tgz#4dd2e2fda419730a21fb77ade3b90241ad4a5bcc" + integrity sha512-w4mEqGz4/wV+BBljLxduFNkMrd3rstBNDXmoX5kD4UTzIb4Sy0QybWCtg2iVT+R0KWiRRA56QKOvBsgXiddksA== + +"@cspell/dict-haskell@^4.0.1": + version "4.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-haskell/-/dict-haskell-4.0.1.tgz#e9fca7c452411ff11926e23ffed2b50bb9b95e47" + integrity sha512-uRrl65mGrOmwT7NxspB4xKXFUenNC7IikmpRZW8Uzqbqcu7ZRCUfstuVH7T1rmjRgRkjcIjE4PC11luDou4wEQ== + +"@cspell/dict-html-symbol-entities@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-html-symbol-entities/-/dict-html-symbol-entities-4.0.0.tgz#4d86ac18a4a11fdb61dfb6f5929acd768a52564f" + integrity sha512-HGRu+48ErJjoweR5IbcixxETRewrBb0uxQBd6xFGcxbEYCX8CnQFTAmKI5xNaIt2PKaZiJH3ijodGSqbKdsxhw== + +"@cspell/dict-html@^4.0.5": + version "4.0.5" + resolved "https://registry.yarnpkg.com/@cspell/dict-html/-/dict-html-4.0.5.tgz#03a5182148d80e6c25f71339dbb2b7c5b9894ef8" + integrity sha512-p0brEnRybzSSWi8sGbuVEf7jSTDmXPx7XhQUb5bgG6b54uj+Z0Qf0V2n8b/LWwIPJNd1GygaO9l8k3HTCy1h4w== + +"@cspell/dict-java@^5.0.6": + version "5.0.6" + resolved "https://registry.yarnpkg.com/@cspell/dict-java/-/dict-java-5.0.6.tgz#2462d6fc15f79ec15eb88ecf875b6ad2a7bf7a6a" + integrity sha512-kdE4AHHHrixyZ5p6zyms1SLoYpaJarPxrz8Tveo6gddszBVVwIUZ+JkQE1bWNLK740GWzIXdkznpUfw1hP9nXw== + +"@cspell/dict-k8s@^1.0.2": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-k8s/-/dict-k8s-1.0.2.tgz#b19e66f4ac8a4264c0f3981ac6e23e88a60f1c91" + integrity sha512-tLT7gZpNPnGa+IIFvK9SP1LrSpPpJ94a/DulzAPOb1Q2UBFwdpFd82UWhio0RNShduvKG/WiMZf/wGl98pn+VQ== + +"@cspell/dict-latex@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-latex/-/dict-latex-4.0.0.tgz#85054903db834ea867174795d162e2a8f0e9c51e" + integrity sha512-LPY4y6D5oI7D3d+5JMJHK/wxYTQa2lJMSNxps2JtuF8hbAnBQb3igoWEjEbIbRRH1XBM0X8dQqemnjQNCiAtxQ== + +"@cspell/dict-lorem-ipsum@^4.0.0": + version "4.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-lorem-ipsum/-/dict-lorem-ipsum-4.0.0.tgz#2793a5dbfde474a546b0caecc40c38fdf076306e" + integrity sha512-1l3yjfNvMzZPibW8A7mQU4kTozwVZVw0AvFEdy+NcqtbxH+TvbSkNMqROOFWrkD2PjnKG0+Ea0tHI2Pi6Gchnw== + +"@cspell/dict-lua@^4.0.3": + version "4.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-lua/-/dict-lua-4.0.3.tgz#2d23c8f7e74b4e62000678d80e7d1ebb10b003e0" + integrity sha512-lDHKjsrrbqPaea13+G9s0rtXjMO06gPXPYRjRYawbNmo4E/e3XFfVzeci3OQDQNDmf2cPOwt9Ef5lu2lDmwfJg== + +"@cspell/dict-makefile@^1.0.0": + version "1.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-makefile/-/dict-makefile-1.0.0.tgz#5afb2910873ebbc01ab8d9c38661c4c93d0e5a40" + integrity sha512-3W9tHPcSbJa6s0bcqWo6VisEDTSN5zOtDbnPabF7rbyjRpNo0uHXHRJQF8gAbFzoTzBBhgkTmrfSiuyQm7vBUQ== + +"@cspell/dict-node@^4.0.3": + version "4.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-node/-/dict-node-4.0.3.tgz#5ae0222d72871e82978049f8e11ea627ca42fca3" + integrity sha512-sFlUNI5kOogy49KtPg8SMQYirDGIAoKBO3+cDLIwD4MLdsWy1q0upc7pzGht3mrjuyMiPRUV14Bb0rkVLrxOhg== + +"@cspell/dict-npm@^5.0.13": + version "5.0.13" + resolved "https://registry.yarnpkg.com/@cspell/dict-npm/-/dict-npm-5.0.13.tgz#81051f791ee29563430145b360947f711316ccd1" + integrity sha512-uPb3DlQA/FvlmzT5RjZoy7fy91mxMRZW1B+K3atVM5A/cmP1QlDaSW/iCtde5kHET1MOV7uxz+vy0Yha2OI5pQ== + +"@cspell/dict-php@^4.0.4": + version "4.0.4" + resolved "https://registry.yarnpkg.com/@cspell/dict-php/-/dict-php-4.0.4.tgz#7510c0fe4bdbb049c143eb3c471820d1e681bbb9" + integrity sha512-fRlLV730fJbulDsLIouZxXoxHt3KIH6hcLFwxaupHL+iTXDg0lo7neRpbqD5MScr/J3idEr7i9G8XWzIikKFug== + +"@cspell/dict-powershell@^5.0.2": + version "5.0.3" + resolved "https://registry.yarnpkg.com/@cspell/dict-powershell/-/dict-powershell-5.0.3.tgz#7bceb4e7db39f87479a6d2af3a033ce26796ae49" + integrity sha512-lEdzrcyau6mgzu1ie98GjOEegwVHvoaWtzQnm1ie4DyZgMr+N6D0Iyj1lzvtmt0snvsDFa5F2bsYzf3IMKcpcA== + +"@cspell/dict-public-licenses@^2.0.5": + version "2.0.5" + resolved "https://registry.yarnpkg.com/@cspell/dict-public-licenses/-/dict-public-licenses-2.0.5.tgz#86948b29bd36184943955eaa80bf594488c4dd8a" + integrity sha512-91HK4dSRri/HqzAypHgduRMarJAleOX5NugoI8SjDLPzWYkwZ1ftuCXSk+fy8DLc3wK7iOaFcZAvbjmnLhVs4A== + +"@cspell/dict-python@^4.1.10": + version "4.1.10" + resolved "https://registry.yarnpkg.com/@cspell/dict-python/-/dict-python-4.1.10.tgz#bae6557e7b828a1701d3733b7766c4d95f279175" + integrity sha512-ErF/Ohcu6Xk4QVNzFgo8p7CxkxvAKAmFszvso41qOOhu8CVpB35ikBRpGVDw9gsCUtZzi15Yl0izi4do6WcLkA== + dependencies: + "@cspell/dict-data-science" "^1.0.11" + +"@cspell/dict-r@^2.0.1": + version "2.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-r/-/dict-r-2.0.1.tgz#73474fb7cce45deb9094ebf61083fbf5913f440a" + integrity sha512-KCmKaeYMLm2Ip79mlYPc8p+B2uzwBp4KMkzeLd5E6jUlCL93Y5Nvq68wV5fRLDRTf7N1LvofkVFWfDcednFOgA== + +"@cspell/dict-ruby@^5.0.1": + version "5.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-ruby/-/dict-ruby-5.0.1.tgz#a59df952d66781d811e7aac9208c145680e8cdf9" + integrity sha512-rruTm7Emhty/BSYavSm8ZxRuVw0OBqzJkwIFXcV0cX7To8D1qbmS9HFHRuRg8IL11+/nJvtdDz+lMFBSmPUagQ== + +"@cspell/dict-rust@^4.0.1": + version "4.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-rust/-/dict-rust-4.0.1.tgz#ef0b88cb3a45265824e2c9ce31b0baa4e1050351" + integrity sha512-xJSSzHDK2z6lSVaOmMxl3PTOtfoffaxMo7fTcbZUF+SCJzfKbO6vnN9TCGX2sx1RHFDz66Js6goz6SAZQdOwaw== + +"@cspell/dict-scala@^5.0.0": + version "5.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-scala/-/dict-scala-5.0.0.tgz#b64365ad559110a36d44ccd90edf7151ea648022" + integrity sha512-ph0twaRoV+ylui022clEO1dZ35QbeEQaKTaV2sPOsdwIokABPIiK09oWwGK9qg7jRGQwVaRPEq0Vp+IG1GpqSQ== + +"@cspell/dict-software-terms@^3.3.11": + version "3.3.12" + resolved "https://registry.yarnpkg.com/@cspell/dict-software-terms/-/dict-software-terms-3.3.12.tgz#49db44c470925c759465b6fdbd078f2a831e09c8" + integrity sha512-6aa4T9VqOMc0SFNBt6gxp0CWjvRqMg/uxvgpRbil+ToHWcU+Q+As0WKhPLaOniuTdCM85WWzRouD0O1XUGqg5Q== + +"@cspell/dict-sql@^2.1.2": + version "2.1.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-sql/-/dict-sql-2.1.2.tgz#80492b887e7986dd8bc39a9c5ea513ede2b17cb1" + integrity sha512-Pi0hAcvsSGtZZeyyAN1VfGtQJbrXos5x2QjJU0niAQKhmITSOrXU/1II1Gogk+FYDjWyV9wP2De0U2f7EWs6oQ== + +"@cspell/dict-svelte@^1.0.2": + version "1.0.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-svelte/-/dict-svelte-1.0.2.tgz#0c866b08a7a6b33bbc1a3bdbe6a1b484ca15cdaa" + integrity sha512-rPJmnn/GsDs0btNvrRBciOhngKV98yZ9SHmg8qI6HLS8hZKvcXc0LMsf9LLuMK1TmS2+WQFAan6qeqg6bBxL2Q== + +"@cspell/dict-swift@^2.0.1": + version "2.0.1" + resolved "https://registry.yarnpkg.com/@cspell/dict-swift/-/dict-swift-2.0.1.tgz#06ec86e52e9630c441d3c19605657457e33d7bb6" + integrity sha512-gxrCMUOndOk7xZFmXNtkCEeroZRnS2VbeaIPiymGRHj5H+qfTAzAKxtv7jJbVA3YYvEzWcVE2oKDP4wcbhIERw== + +"@cspell/dict-typescript@^3.1.2": + version "3.1.2" + resolved "https://registry.yarnpkg.com/@cspell/dict-typescript/-/dict-typescript-3.1.2.tgz#14d05f54db2984feaa24ea133b583d19c04cc104" + integrity sha512-lcNOYWjLUvDZdLa0UMNd/LwfVdxhE9rKA+agZBGjL3lTA3uNvH7IUqSJM/IXhJoBpLLMVEOk8v1N9xi+vDuCdA== + +"@cspell/dict-vue@^3.0.0": + version "3.0.0" + resolved "https://registry.yarnpkg.com/@cspell/dict-vue/-/dict-vue-3.0.0.tgz#68ccb432ad93fcb0fd665352d075ae9a64ea9250" + integrity sha512-niiEMPWPV9IeRBRzZ0TBZmNnkK3olkOPYxC1Ny2AX4TGlYRajcW0WUtoSHmvvjZNfWLSg2L6ruiBeuPSbjnG6A== + +"@cspell/dynamic-import@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/dynamic-import/-/dynamic-import-8.1.3.tgz#7bfa0dc1dbbd44bced42677ff7129932bf325d3e" + integrity sha512-/lXFLa92v4oOcZ2PbdRpOqBvnqWlYmGaV7iCy8+QhIWlMdzi+7tBX3LVTm9Jzvt/rJseVHQQ6RvfTsSmhbUMFQ== + dependencies: + import-meta-resolve "^4.0.0" + +"@cspell/strong-weak-map@8.1.3": + version "8.1.3" + resolved "https://registry.yarnpkg.com/@cspell/strong-weak-map/-/strong-weak-map-8.1.3.tgz#8c782d32bea999999c761a46b4010ee1569d07c1" + integrity sha512-GhWyximzk8tumo0zhrDV3+nFYiETYefiTBWAEVbXJMibuvitFocVZwddqN85J0UdZ2M7q6tvBleEaI9ME/16gA== + "@cspotcode/source-map-support@^0.8.0": version "0.8.1" resolved "https://registry.yarnpkg.com/@cspotcode/source-map-support/-/source-map-support-0.8.1.tgz#00629c35a688e05a88b1cda684fb9d5e73f000a1" @@ -3146,6 +3488,11 @@ ansi-regex@^5.0.1: resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-5.0.1.tgz#082cb2c89c9fe8659a311a53bd6a4dc5301db304" integrity sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ== +ansi-regex@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/ansi-regex/-/ansi-regex-6.0.1.tgz#3183e38fae9a65d7cb5e53945cd5897d0260a06a" + integrity sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA== + ansi-styles@^2.2.1: version "2.2.1" resolved "https://registry.yarnpkg.com/ansi-styles/-/ansi-styles-2.2.1.tgz#b432dd3358b634cf75e1e4664368240533c1ddbe" @@ -3268,6 +3615,11 @@ array-includes@^3.1.7: get-intrinsic "^1.2.1" is-string "^1.0.7" +array-timsort@^1.0.3: + version "1.0.3" + resolved "https://registry.yarnpkg.com/array-timsort/-/array-timsort-1.0.3.tgz#3c9e4199e54fb2b9c3fe5976396a21614ef0d926" + integrity sha512-/+3GRL7dDAGEfM6TseQk/U+mi18TU2Ms9I3UlLdUMhz2hbvGNTKdj9xniwXfUqgYhHxRx0+8UnKkvlNwVU+cWQ== + array-union@^2.1.0: version "2.1.0" resolved "https://registry.yarnpkg.com/array-union/-/array-union-2.1.0.tgz#b798420adbeb1de828d84acd8a2e23d3efe85e8d" @@ -4559,7 +4911,7 @@ call-bind@^1.0.0, call-bind@^1.0.2, call-bind@^1.0.4, call-bind@^1.0.5, call-bin get-intrinsic "^1.2.1" set-function-length "^1.1.1" -callsites@^3.0.0: +callsites@^3.0.0, callsites@^3.1.0: version "3.1.0" resolved "https://registry.yarnpkg.com/callsites/-/callsites-3.1.0.tgz#b3630abd8943432f54b3f0519238e33cd7df2f73" integrity sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ== @@ -4631,6 +4983,13 @@ chai@^4.3.4: pathval "^1.1.1" type-detect "^4.0.8" +chalk-template@^1.1.0: + version "1.1.0" + resolved "https://registry.yarnpkg.com/chalk-template/-/chalk-template-1.1.0.tgz#ffc55db6dd745e9394b85327c8ac8466edb7a7b1" + integrity sha512-T2VJbcDuZQ0Tb2EWwSotMPJjgpy1/tGee1BTpUNsGZ/qgNjV2t7Mvu+d4600U564nbLesN1x2dPL+xii174Ekg== + dependencies: + chalk "^5.2.0" + chalk@4.1.2, chalk@^4.0.0, chalk@^4.1.0, chalk@^4.1.2: version "4.1.2" resolved "https://registry.yarnpkg.com/chalk/-/chalk-4.1.2.tgz#aac4e2b7734a740867aeb16bf02aad556a1e7a01" @@ -4659,6 +5018,11 @@ chalk@^2.4.1, chalk@^2.4.2: escape-string-regexp "^1.0.5" supports-color "^5.3.0" +chalk@^5.2.0, chalk@^5.3.0: + version "5.3.0" + resolved "https://registry.yarnpkg.com/chalk/-/chalk-5.3.0.tgz#67c20a7ebef70e7f3970a01f90fa210cb6860385" + integrity sha512-dLitG79d+GV1Nb/VYcCDFivJeK1hiukt9QjRNVOsUtTy1rR1YJsmpGGTZ3qJos+uw7WmWF4wUwBd9jxjocFC2w== + char-regex@^1.0.2: version "1.0.2" resolved "https://registry.yarnpkg.com/char-regex/-/char-regex-1.0.2.tgz#d744358226217f981ed58f479b1d6bcc29545dcf" @@ -4773,6 +5137,14 @@ clean-stack@^2.0.0: resolved "https://registry.yarnpkg.com/clean-stack/-/clean-stack-2.2.0.tgz#ee8472dbb129e727b31e8a10a427dee9dfe4008b" integrity sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A== +clear-module@^4.1.2: + version "4.1.2" + resolved "https://registry.yarnpkg.com/clear-module/-/clear-module-4.1.2.tgz#5a58a5c9f8dccf363545ad7284cad3c887352a80" + integrity sha512-LWAxzHqdHsAZlPlEyJ2Poz6AIs384mPeqLVCru2p0BrP9G/kVGuhNyZYClLO6cXlnuJjzC8xtsJIuMjKqLXoAw== + dependencies: + parent-module "^2.0.0" + resolve-from "^5.0.0" + cli-cursor@^2.1.0: version "2.1.0" resolved "https://registry.yarnpkg.com/cli-cursor/-/cli-cursor-2.1.0.tgz#b35dac376479facc3e94747d41d0d0f5238ffcb5" @@ -4938,6 +5310,11 @@ commander@^10.0.0: resolved "https://registry.yarnpkg.com/commander/-/commander-10.0.1.tgz#881ee46b4f77d1c1dccc5823433aa39b022cbe06" integrity sha512-y4Mg2tXshplEbSGzx7amzPwKKOCGuoSRP/CjEdwwk0FOGlUbq6lKuoyDZTNZkmxHdJtp54hdfY/JUrdL7Xfdug== +commander@^11.1.0: + version "11.1.0" + resolved "https://registry.yarnpkg.com/commander/-/commander-11.1.0.tgz#62fdce76006a68e5c1ab3314dc92e800eb83d906" + integrity sha512-yPVavfyCcRhmorC7rWlkHn15b4wDVgVmBA7kV4QVBsF7kv/9TKJAbAXVTxvTnwP8HHKjRCJDClKbciiYS7p0DQ== + commander@^2.19.0: version "2.20.3" resolved "https://registry.yarnpkg.com/commander/-/commander-2.20.3.tgz#fd485e84c03eb4881c20722ba48035e8531aeb33" @@ -4970,6 +5347,17 @@ commander@~9.4.1: resolved "https://registry.yarnpkg.com/commander/-/commander-9.4.1.tgz#d1dd8f2ce6faf93147295c0df13c7c21141cfbdd" integrity sha512-5EEkTNyHNGFPD2H+c/dXXfQZYa/scCKasxWcXJaWnNJ99pnQN9Vnmqow+p+PlFPE63Q6mThaZws1T+HxfpgtPw== +comment-json@^4.2.3: + version "4.2.3" + resolved "https://registry.yarnpkg.com/comment-json/-/comment-json-4.2.3.tgz#50b487ebbf43abe44431f575ebda07d30d015365" + integrity sha512-SsxdiOf064DWoZLH799Ata6u7iV658A11PlWtZATDlXPpKGJnbJZ5Z24ybixAi+LUUqJ/GKowAejtC5GFUG7Tw== + dependencies: + array-timsort "^1.0.3" + core-util-is "^1.0.3" + esprima "^4.0.1" + has-own-prop "^2.0.0" + repeat-string "^1.6.1" + component-emitter@^1.2.1: version "1.3.1" resolved "https://registry.yarnpkg.com/component-emitter/-/component-emitter-1.3.1.tgz#ef1d5796f7d93f135ee6fb684340b26403c97d17" @@ -4990,6 +5378,17 @@ concat-stream@^1.5.1, concat-stream@^1.6.0, concat-stream@^1.6.2, concat-stream@ readable-stream "^2.2.2" typedarray "^0.0.6" +configstore@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/configstore/-/configstore-6.0.0.tgz#49eca2ebc80983f77e09394a1a56e0aca8235566" + integrity sha512-cD31W1v3GqUlQvbBCGcXmd2Nj9SvLDOP1oQ0YFuLETufzSPaKp11rYBsSOm7rCsW3OnIRAFM3OxRhceaXNYHkA== + dependencies: + dot-prop "^6.0.1" + graceful-fs "^4.2.6" + unique-string "^3.0.0" + write-file-atomic "^3.0.3" + xdg-basedir "^5.0.1" + content-disposition@0.5.4: version "0.5.4" resolved "https://registry.yarnpkg.com/content-disposition/-/content-disposition-0.5.4.tgz#8b82b4efac82512a02bb0b1dcec9d2c5e8eb5bfe" @@ -5061,7 +5460,7 @@ core-util-is@1.0.2: resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.2.tgz#b5fd54220aa2bc5ab57aab7140c940754503c1a7" integrity sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ== -core-util-is@~1.0.0: +core-util-is@^1.0.3, core-util-is@~1.0.0: version "1.0.3" resolved "https://registry.yarnpkg.com/core-util-is/-/core-util-is-1.0.3.tgz#a6042d3634c2b27e9328f837b965fac83808db85" integrity sha512-ZQBvi1DcpJ4GDqanjucZ2Hj3wEO5pZDS89BWbkcrvdxksJorwUDDZamX9ldFkp9aw2lmBDLgkObEA4DWNJ9FYQ== @@ -5201,6 +5600,123 @@ crypto-js@^3.1.9-1: resolved "https://registry.yarnpkg.com/crypto-js/-/crypto-js-3.3.0.tgz#846dd1cce2f68aacfa156c8578f926a609b7976b" integrity sha512-DIT51nX0dCfKltpRiXV+/TVZq+Qq2NgF4644+K7Ttnla7zEzqc+kjJyiB96BHNyUTBxyjzRcZYpUdZa+QAqi6Q== +crypto-random-string@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/crypto-random-string/-/crypto-random-string-4.0.0.tgz#5a3cc53d7dd86183df5da0312816ceeeb5bb1fc2" + integrity sha512-x8dy3RnvYdlUcPOjkEHqozhiwzKNSq7GcPuXFbnyMOCHxX8V3OgIg/pYuabl2sbUPfIJaeAQB7PMOK8DFIdoRA== + dependencies: + type-fest "^1.0.1" + +cspell-config-lib@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-config-lib/-/cspell-config-lib-8.1.3.tgz#5cd4a8e4d844e9b5737825bda5b445ed50795f88" + integrity sha512-whzJYxcxos3vnywn0alCFZ+Myc0K/C62pUurfOGhgvIba7ArmlXhNRaL2r5noBxWARtpBOtzz3vrzSBK7Lq6jg== + dependencies: + "@cspell/cspell-types" "8.1.3" + comment-json "^4.2.3" + yaml "^2.3.4" + +cspell-dictionary@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-dictionary/-/cspell-dictionary-8.1.3.tgz#7294ade35ceb388cb841a6ac8435b631aa1cac68" + integrity sha512-nkRQDPNnA6tw+hJFBqq26M0nK306q5rtyv/AUIWa8ZHhQkwzACnpMSpuJA7/DV5GVvPKltMK5M4A6vgfpoaFHw== + dependencies: + "@cspell/cspell-pipe" "8.1.3" + "@cspell/cspell-types" "8.1.3" + cspell-trie-lib "8.1.3" + fast-equals "^5.0.1" + gensequence "^6.0.0" + +cspell-gitignore@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-gitignore/-/cspell-gitignore-8.1.3.tgz#1d2ec9dc8d97ea8291a197e5acf6d6b6c9634193" + integrity sha512-NHx5lg44eCKb6yJmUPOCz4prcuYowzoo5GJ5hOcCfbk7ZEBWV1E2/kDRuQMOK2W0y1hNGr45CSxO3UxWJlYg7w== + dependencies: + cspell-glob "8.1.3" + find-up-simple "^1.0.0" + +cspell-glob@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-glob/-/cspell-glob-8.1.3.tgz#3eeecbf6ecbfab184107021fdf8691ce4e8fa6c5" + integrity sha512-Likr7UVUXBpthQnM5r6yao3X0YBNRbJ9AHWXTC2RJfzwZOFKF+pKPfeo3FU+Px8My96M4RC2bVMbrbZUwN5NJw== + dependencies: + micromatch "^4.0.5" + +cspell-grammar@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-grammar/-/cspell-grammar-8.1.3.tgz#8ee7d4cef92053c53b320ae105e64b771c73d21e" + integrity sha512-dTOwNq6a5wcVzOsi4xY5/tq2r2w/+wLVU+WfyySTsPe66Rjqx/QceFl4OinImks/ZMKF7Zyjd3WGyQ5TcSsJFQ== + dependencies: + "@cspell/cspell-pipe" "8.1.3" + "@cspell/cspell-types" "8.1.3" + +cspell-io@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-io/-/cspell-io-8.1.3.tgz#57e99a78d998d5ab6a7a09468892174f1233b62b" + integrity sha512-QkcFeYd79oIl7PgSqFSZyvwXnZQhXmdCI733n54IN2+iXDcf7W0mwptxoC/cE19RkEwAwEFLG81UAy6L/BXI6A== + dependencies: + "@cspell/cspell-service-bus" "8.1.3" + +cspell-lib@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-lib/-/cspell-lib-8.1.3.tgz#5c1f87c4adb2643baa510cf1bec083bd9145b5ab" + integrity sha512-Kk8bpHVkDZO4MEiPkDvRf/LgJ0h5mufbKLTWModq6k0Ca8EkZ/qgQlZ0ve0rIivbleSqebuWjpJHKDM+IHmzHA== + dependencies: + "@cspell/cspell-bundled-dicts" "8.1.3" + "@cspell/cspell-pipe" "8.1.3" + "@cspell/cspell-resolver" "8.1.3" + "@cspell/cspell-types" "8.1.3" + "@cspell/dynamic-import" "8.1.3" + "@cspell/strong-weak-map" "8.1.3" + clear-module "^4.1.2" + comment-json "^4.2.3" + configstore "^6.0.0" + cspell-config-lib "8.1.3" + cspell-dictionary "8.1.3" + cspell-glob "8.1.3" + cspell-grammar "8.1.3" + cspell-io "8.1.3" + cspell-trie-lib "8.1.3" + fast-equals "^5.0.1" + gensequence "^6.0.0" + import-fresh "^3.3.0" + resolve-from "^5.0.0" + vscode-languageserver-textdocument "^1.0.11" + vscode-uri "^3.0.8" + +cspell-trie-lib@8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell-trie-lib/-/cspell-trie-lib-8.1.3.tgz#0831429f898f816c301ac440dc1f2b199d51f7ba" + integrity sha512-EDSYU9MCtzPSJDrfvDrTKmc0rzl50Ehjg1c5rUCqn33p2LCRe/G8hW0FxXe0mxrZxrMO2b8l0PVSGlrCXCQ8RQ== + dependencies: + "@cspell/cspell-pipe" "8.1.3" + "@cspell/cspell-types" "8.1.3" + gensequence "^6.0.0" + +cspell@^8.1.3: + version "8.1.3" + resolved "https://registry.yarnpkg.com/cspell/-/cspell-8.1.3.tgz#c8643b86d7e06b69e7032bf2ddf3b27563f85c43" + integrity sha512-SU4Su6002bPoJYaiMeNV4wwLoS8TwaOgIwaTxhys3GDbJIxZV6CrDgwksezHcG7TZrC4yrveDVsdpnrzmQ7T5Q== + dependencies: + "@cspell/cspell-json-reporter" "8.1.3" + "@cspell/cspell-pipe" "8.1.3" + "@cspell/cspell-types" "8.1.3" + "@cspell/dynamic-import" "8.1.3" + chalk "^5.3.0" + chalk-template "^1.1.0" + commander "^11.1.0" + cspell-gitignore "8.1.3" + cspell-glob "8.1.3" + cspell-io "8.1.3" + cspell-lib "8.1.3" + fast-glob "^3.3.2" + fast-json-stable-stringify "^2.1.0" + file-entry-cache "^7.0.2" + get-stdin "^9.0.0" + semver "^7.5.4" + strip-ansi "^7.1.0" + vscode-uri "^3.0.8" + d@1, d@^1.0.1: version "1.0.1" resolved "https://registry.yarnpkg.com/d/-/d-1.0.1.tgz#8698095372d58dbee346ffd0c7093f99f8f9eb5a" @@ -5589,6 +6105,13 @@ dom-walk@^0.1.0: resolved "https://registry.yarnpkg.com/dom-walk/-/dom-walk-0.1.2.tgz#0c548bef048f4d1f2a97249002236060daa3fd84" integrity sha512-6QvTW9mrGeIegrFXdtQi9pk7O/nSK6lSdXW2eqUspN5LWD7UTji2Fqw5V2YLjBpHEoU9Xl/eUWNpDeZvoyOv2w== +dot-prop@^6.0.1: + version "6.0.1" + resolved "https://registry.yarnpkg.com/dot-prop/-/dot-prop-6.0.1.tgz#fc26b3cf142b9e59b74dbd39ed66ce620c681083" + integrity sha512-tE7ztYzXHIeyvc7N+hR3oi7FIbf/NIjVP9hmAt3yMXzrQ072/fpjGLx2GxNxGxUl5V73MEqYzioOMoVhGMJ5cA== + dependencies: + is-obj "^2.0.0" + dotenv@^16.0.3: version "16.3.1" resolved "https://registry.yarnpkg.com/dotenv/-/dotenv-16.3.1.tgz#369034de7d7e5b120972693352a3bf112172cc3e" @@ -6103,7 +6626,7 @@ esprima@2.7.x, esprima@^2.7.1: resolved "https://registry.yarnpkg.com/esprima/-/esprima-2.7.3.tgz#96e3b70d5779f6ad49cd032673d1c312767ba581" integrity sha512-OarPfz0lFCiW4/AV2Oy1Rp9qu0iusTKqykwTspGCZtPxmF81JR4MmIebvF1F9+UOKth2ZubLQ4XGGaU+hSn99A== -esprima@^4.0.0: +esprima@^4.0.0, esprima@^4.0.1: version "4.0.1" resolved "https://registry.yarnpkg.com/esprima/-/esprima-4.0.1.tgz#13b04cdb3e6c5d19df91ab6987a8695619b0aa71" integrity sha512-eGuFFw7Upda+g4p+QHvnW0RyTX/SVeJBDM/gCtMARO0cLuT2HcEKnTPvhjV6aGeqrCB/sbNop0Kszm0jsaWU4A== @@ -7016,6 +7539,13 @@ file-entry-cache@^6.0.1: dependencies: flat-cache "^3.0.4" +file-entry-cache@^7.0.2: + version "7.0.2" + resolved "https://registry.yarnpkg.com/file-entry-cache/-/file-entry-cache-7.0.2.tgz#2d61bb70ba89b9548e3035b7c9173fe91deafff0" + integrity sha512-TfW7/1iI4Cy7Y8L6iqNdZQVvdXn0f8B4QcIXmkIbtTIe/Okm/nSlHb4IwGzRVOd3WfSieCgvf5cMzEfySAIl0g== + dependencies: + flat-cache "^3.2.0" + fill-range@^4.0.0: version "4.0.0" resolved "https://registry.yarnpkg.com/fill-range/-/fill-range-4.0.0.tgz#d544811d428f98eb06a63dc402d2403c328c38f7" @@ -7109,6 +7639,15 @@ flat-cache@^3.0.4: keyv "^4.5.3" rimraf "^3.0.2" +flat-cache@^3.2.0: + version "3.2.0" + resolved "https://registry.yarnpkg.com/flat-cache/-/flat-cache-3.2.0.tgz#2c0c2d5040c99b1632771a9d105725c0115363ee" + integrity sha512-CYcENa+FtcUKLmhhqyctpclsq7QF38pKjZHsGNiSQF5r4FtoKDWabFDl3hzaEQMvT1LHEysw5twgLvpYYb4vbw== + dependencies: + flatted "^3.2.9" + keyv "^4.5.3" + rimraf "^3.0.2" + flat@^5.0.2: version "5.0.2" resolved "https://registry.yarnpkg.com/flat/-/flat-5.0.2.tgz#8ca6fe332069ffa9d324c327198c598259ceb241" @@ -7351,6 +7890,11 @@ ganache-core@^2.13.2: ethereumjs-wallet "0.6.5" web3 "1.2.11" +gensequence@^6.0.0: + version "6.0.0" + resolved "https://registry.yarnpkg.com/gensequence/-/gensequence-6.0.0.tgz#ae46a0f89ebd7cc334e45cfb8f1c99a65248694e" + integrity sha512-8WwuywE9pokJRAcg2QFR/plk3cVPebSUqRPzpGQh3WQ0wIiHAw+HyOQj5IuHyUTQBHpBKFoB2JUMu9zT3vJ16Q== + gensync@^1.0.0-beta.2: version "1.0.0-beta.2" resolved "https://registry.yarnpkg.com/gensync/-/gensync-1.0.0-beta.2.tgz#32a6ee76c3d7f52d46b2b1ae5d93fea8580a25e0" @@ -7391,21 +7935,16 @@ get-port@^3.1.0: resolved "https://registry.yarnpkg.com/get-port/-/get-port-3.2.0.tgz#dd7ce7de187c06c8bf353796ac71e099f0980ebc" integrity sha512-x5UJKlgeUiNT8nyo/AcnwLnZuZNcSjSw0kogRB+Whd1fjjFq4B1hySFxSFWWSn4mIBzg3sRNUDFYc4g5gjPoLg== -get-stdin@=8.0.0: - version "8.0.0" - resolved "https://registry.yarnpkg.com/get-stdin/-/get-stdin-8.0.0.tgz#cbad6a73feb75f6eeb22ba9e01f89aa28aa97a53" - integrity sha512-sY22aA6xchAzprjyqmSEQv4UbAAzRN0L2dQB0NlN5acTTK9Don6nhoc3eAbUnpZiCANAMfd/+40kVdKfFygohg== +get-stdin@^9.0.0, get-stdin@~9.0.0: + version "9.0.0" + resolved "https://registry.yarnpkg.com/get-stdin/-/get-stdin-9.0.0.tgz#3983ff82e03d56f1b2ea0d3e60325f39d703a575" + integrity sha512-dVKBjfWisLAicarI2Sf+JuBE/DghV4UzNAVe9yhEJuzeREd3JhOTE9cUaJTeSa77fsbQUK3pcOpJfM59+VKZaA== get-stdin@~5.0.1: version "5.0.1" resolved "https://registry.yarnpkg.com/get-stdin/-/get-stdin-5.0.1.tgz#122e161591e21ff4c52530305693f20e6393a398" integrity sha512-jZV7n6jGE3Gt7fgSTJoz91Ak5MuTLwMwkoYdjxuJ/AmjIsE1UC03y/IWkZCQGEvVNS9qoRNwy5BCqxImv0FVeA== -get-stdin@~9.0.0: - version "9.0.0" - resolved "https://registry.yarnpkg.com/get-stdin/-/get-stdin-9.0.0.tgz#3983ff82e03d56f1b2ea0d3e60325f39d703a575" - integrity sha512-dVKBjfWisLAicarI2Sf+JuBE/DghV4UzNAVe9yhEJuzeREd3JhOTE9cUaJTeSa77fsbQUK3pcOpJfM59+VKZaA== - get-stream@^4.1.0: version "4.1.0" resolved "https://registry.yarnpkg.com/get-stream/-/get-stream-4.1.0.tgz#c1b255575f3dc21d59bfc79cd3d2b46b1c3a54b5" @@ -7543,6 +8082,13 @@ glob@~8.0.3: minimatch "^5.0.1" once "^1.3.0" +global-directory@^4.0.1: + version "4.0.1" + resolved "https://registry.yarnpkg.com/global-directory/-/global-directory-4.0.1.tgz#4d7ac7cfd2cb73f304c53b8810891748df5e361e" + integrity sha512-wHTUcDUoZ1H5/0iVqEudYW4/kAlN5cZ3j/bXn0Dpbizl9iaUVeWSHqiOjsgk6OW2bkLclbBjzewBz6weQ1zA2Q== + dependencies: + ini "4.1.1" + global-modules@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/global-modules/-/global-modules-2.0.0.tgz#997605ad2345f27f51539bea26574421215c7780" @@ -7665,7 +8211,7 @@ got@^11.8.5: p-cancelable "^2.0.0" responselike "^2.0.0" -graceful-fs@^4.1.11, graceful-fs@^4.1.2, graceful-fs@^4.1.6, graceful-fs@^4.1.9, graceful-fs@^4.2.0, graceful-fs@^4.2.4, graceful-fs@^4.2.9: +graceful-fs@^4.1.11, graceful-fs@^4.1.2, graceful-fs@^4.1.6, graceful-fs@^4.1.9, graceful-fs@^4.2.0, graceful-fs@^4.2.4, graceful-fs@^4.2.6, graceful-fs@^4.2.9: version "4.2.11" resolved "https://registry.yarnpkg.com/graceful-fs/-/graceful-fs-4.2.11.tgz#4183e4e8bf08bb6e05bbb2f7d2e0c8f712ca40e3" integrity sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ== @@ -7869,6 +8415,11 @@ has-flag@^4.0.0: resolved "https://registry.yarnpkg.com/has-flag/-/has-flag-4.0.0.tgz#944771fd9c81c81265c4d6941860da06bb59479b" integrity sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ== +has-own-prop@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/has-own-prop/-/has-own-prop-2.0.0.tgz#f0f95d58f65804f5d218db32563bb85b8e0417af" + integrity sha512-Pq0h+hvsVm6dDEa8x82GnLSYHOzNDt7f0ddFa3FqcQlgzEiptPqL+XrOJNavjOzSYiYWIrgeVYYgGlLmnxwilQ== + has-property-descriptors@^1.0.0: version "1.0.1" resolved "https://registry.yarnpkg.com/has-property-descriptors/-/has-property-descriptors-1.0.1.tgz#52ba30b6c5ec87fd89fa574bc1c39125c6f65340" @@ -8145,6 +8696,11 @@ import-local@^3.0.2: pkg-dir "^4.2.0" resolve-cwd "^3.0.0" +import-meta-resolve@^4.0.0: + version "4.0.0" + resolved "https://registry.yarnpkg.com/import-meta-resolve/-/import-meta-resolve-4.0.0.tgz#0b1195915689f60ab00f830af0f15cc841e8919e" + integrity sha512-okYUR7ZQPH+efeuMJGlq4f8ubUgO50kByRPyt/Cy1Io4PSRsPjxME+YlVaCOx+NIToW7hCsZNFJyTPFFKepRSA== + imurmurhash@^0.1.4: version "0.1.4" resolved "https://registry.yarnpkg.com/imurmurhash/-/imurmurhash-0.1.4.tgz#9218b9b2b928a238b13dc4fb6b6d576f231453ea" @@ -8173,6 +8729,11 @@ inherits@2.0.3: resolved "https://registry.yarnpkg.com/inherits/-/inherits-2.0.3.tgz#633c2c83e3da42a502f52466022480f4208261de" integrity sha512-x00IRNXNy63jwGkJmzPigoySHbaqpNuzKbBOmzK+g2OdZpQ9w+sxCN+VSB3ja7IAge2OP2qpfxTjeNcyjmW1uw== +ini@4.1.1: + version "4.1.1" + resolved "https://registry.yarnpkg.com/ini/-/ini-4.1.1.tgz#d95b3d843b1e906e56d6747d5447904ff50ce7a1" + integrity sha512-QQnnxNyfvmHFIsj7gkPcYymR8Jdw/o7mp5ZFihxn6h8Ci6fh3Dx4E1gPjpQEpIuPo9XVNY/ZUwh4BPMjGyL01g== + ini@^1.3.5, ini@~1.3.0: version "1.3.8" resolved "https://registry.yarnpkg.com/ini/-/ini-1.3.8.tgz#a29da425b48806f34767a4efce397269af28432c" @@ -8457,6 +9018,11 @@ is-number@^7.0.0: resolved "https://registry.yarnpkg.com/is-number/-/is-number-7.0.0.tgz#7535345b896734d5f80c4d06c50955527a14f12b" integrity sha512-41Cifkg6e8TylSpdtTpeLVMqvSBEVzTttHvERD741+pnZ8ANv0004MRL43QKPDlK9cGvNp6NZWZUBlbGXYxxng== +is-obj@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/is-obj/-/is-obj-2.0.0.tgz#473fb05d973705e3fd9620545018ca8e22ef4982" + integrity sha512-drqDG3cbczxxEJRoOXcOjtdp1J/lyp1mNn0xaznRs8+muBhgQcrnbspox5X5fOw0HnMnbfDzvnEMEtqDEJEo8w== + is-path-inside@^3.0.3: version "3.0.3" resolved "https://registry.yarnpkg.com/is-path-inside/-/is-path-inside-3.0.3.tgz#d231362e53a07ff2b0e0ea7fed049161ffd16283" @@ -10028,7 +10594,7 @@ micromatch@^3.1.4: snapdragon "^0.8.1" to-regex "^3.0.2" -micromatch@^4.0.2, micromatch@^4.0.4: +micromatch@^4.0.2, micromatch@^4.0.4, micromatch@^4.0.5: version "4.0.5" resolved "https://registry.yarnpkg.com/micromatch/-/micromatch-4.0.5.tgz#bc8999a7cbbf77cdc89f132f6e467051b49090c6" integrity sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA== @@ -10884,6 +11450,13 @@ parent-module@^1.0.0: dependencies: callsites "^3.0.0" +parent-module@^2.0.0: + version "2.0.0" + resolved "https://registry.yarnpkg.com/parent-module/-/parent-module-2.0.0.tgz#fa71f88ff1a50c27e15d8ff74e0e3a9523bf8708" + integrity sha512-uo0Z9JJeWzv8BG+tRcapBKNJ0dro9cLyczGzulS6EfeyAdeC9sbojtW6XwvYxJkEne9En+J2XEl4zyglVeIwFg== + dependencies: + callsites "^3.1.0" + parse-asn1@^5.0.0, parse-asn1@^5.1.6: version "5.1.6" resolved "https://registry.yarnpkg.com/parse-asn1/-/parse-asn1-5.1.6.tgz#385080a3ec13cb62a62d39409cb3e88844cdaed4" @@ -12854,6 +13427,13 @@ strip-ansi@^6.0.0, strip-ansi@^6.0.1: dependencies: ansi-regex "^5.0.1" +strip-ansi@^7.1.0: + version "7.1.0" + resolved "https://registry.yarnpkg.com/strip-ansi/-/strip-ansi-7.1.0.tgz#d5b6568ca689d8561370b0707685d22434faff45" + integrity sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ== + dependencies: + ansi-regex "^6.0.1" + strip-bom@^2.0.0: version "2.0.0" resolved "https://registry.yarnpkg.com/strip-bom/-/strip-bom-2.0.0.tgz#6219a85616520491f35788bdbf1447a99c7e6b0e" @@ -13450,6 +14030,11 @@ type-fest@^0.7.1: resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-0.7.1.tgz#8dda65feaf03ed78f0a3f9678f1869147f7c5c48" integrity sha512-Ne2YiiGN8bmrmJJEuTWTLJR32nh/JdL1+PSicowtNb0WFpn59GK8/lfD61bVtzguz7b3PBt74nxpv/Pw5po5Rg== +type-fest@^1.0.1: + version "1.4.0" + resolved "https://registry.yarnpkg.com/type-fest/-/type-fest-1.4.0.tgz#e9fb813fe3bf1744ec359d55d1affefa76f14be1" + integrity sha512-yGSza74xk0UG8k+pLh5oeoYirvIiWo5t0/o3zHHAO2tRDiZcxWP7fywNlXhqb6/r6sWvwi+RsyQMWhVLe4BVuA== + type-is@~1.6.18: version "1.6.18" resolved "https://registry.yarnpkg.com/type-is/-/type-is-1.6.18.tgz#4e552cd05df09467dcbc4ef739de89f2cf37c131" @@ -13629,6 +14214,13 @@ union-value@^1.0.0: is-extendable "^0.1.1" set-value "^2.0.1" +unique-string@^3.0.0: + version "3.0.0" + resolved "https://registry.yarnpkg.com/unique-string/-/unique-string-3.0.0.tgz#84a1c377aff5fd7a8bc6b55d8244b2bd90d75b9a" + integrity sha512-VGXBUVwxKMBUznyffQweQABPRRW1vHZAbadFZud4pLFAqRGvv/96vafgjWFqzourzr8YonlQiPgH0YCJfawoGQ== + dependencies: + crypto-random-string "^4.0.0" + universalify@^0.1.0: version "0.1.2" resolved "https://registry.yarnpkg.com/universalify/-/universalify-0.1.2.tgz#b646f69be3942dabcecc9d6639c80dc105efaa66" @@ -13815,6 +14407,16 @@ verror@1.10.0: core-util-is "1.0.2" extsprintf "^1.2.0" +vscode-languageserver-textdocument@^1.0.11: + version "1.0.11" + resolved "https://registry.yarnpkg.com/vscode-languageserver-textdocument/-/vscode-languageserver-textdocument-1.0.11.tgz#0822a000e7d4dc083312580d7575fe9e3ba2e2bf" + integrity sha512-X+8T3GoiwTVlJbicx/sIAF+yuJAqz8VvwJyoMVhwEMoEKE/fkDmrqUgDMyBECcM2A2frVZIUj5HI/ErRXCfOeA== + +vscode-uri@^3.0.8: + version "3.0.8" + resolved "https://registry.yarnpkg.com/vscode-uri/-/vscode-uri-3.0.8.tgz#1770938d3e72588659a172d0fd4642780083ff9f" + integrity sha512-AyFQ0EVmsOZOlAnxoFOGOq1SQDWAB7C6aqMGS23svWAllfOaxbuFvcT8D1i8z3Gyn8fraVeZNNmN6e9bxxXkKw== + walker@^1.0.8: version "1.0.8" resolved "https://registry.yarnpkg.com/walker/-/walker-1.0.8.tgz#bd498db477afe573dc04185f011d3ab8a8d7653f" @@ -14236,6 +14838,16 @@ wrappy@1: resolved "https://registry.yarnpkg.com/wrappy/-/wrappy-1.0.2.tgz#b5243d8f3ec1aa35f1364605bc0d1036e30ab69f" integrity sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ== +write-file-atomic@^3.0.3: + version "3.0.3" + resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-3.0.3.tgz#56bd5c5a5c70481cd19c571bd39ab965a5de56e8" + integrity sha512-AvHcyZ5JnSfq3ioSyjrBkH9yW4m7Ayk8/9My/DD9onKeu/94fwrMocemO2QAJFAlnnDN+ZDS+ZjAR5ua1/PV/Q== + dependencies: + imurmurhash "^0.1.4" + is-typedarray "^1.0.0" + signal-exit "^3.0.2" + typedarray-to-buffer "^3.1.5" + write-file-atomic@^4.0.2: version "4.0.2" resolved "https://registry.yarnpkg.com/write-file-atomic/-/write-file-atomic-4.0.2.tgz#a9df01ae5b77858a027fd2e80768ee433555fcfd" @@ -14270,6 +14882,11 @@ ws@^7.4.6: resolved "https://registry.yarnpkg.com/ws/-/ws-7.5.9.tgz#54fa7db29f4c7cec68b1ddd3a89de099942bb591" integrity sha512-F+P9Jil7UiSKSkppIiD94dN07AwvFixvLIj1Og1Rl9GGMuNipJnV9JzjD6XuqmAeiswGvUmNLjr5cFuXwNS77Q== +xdg-basedir@^5.0.1: + version "5.1.0" + resolved "https://registry.yarnpkg.com/xdg-basedir/-/xdg-basedir-5.1.0.tgz#1efba19425e73be1bc6f2a6ceb52a3d2c884c0c9" + integrity sha512-GCPAHLvrIH13+c0SuacwvRYj2SxJXQ4kaVTT5xgL3kPrz56XxkF21IGhjSE1+W0aw7gpBWRGXLCPnPby6lSpmQ== + xhr-request-promise@^0.1.2: version "0.1.3" resolved "https://registry.yarnpkg.com/xhr-request-promise/-/xhr-request-promise-0.1.3.tgz#2d5f4b16d8c6c893be97f1a62b0ed4cf3ca5f96c" @@ -14359,6 +14976,11 @@ yaml@^1.10.2: resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b" integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg== +yaml@^2.3.4: + version "2.3.4" + resolved "https://registry.yarnpkg.com/yaml/-/yaml-2.3.4.tgz#53fc1d514be80aabf386dc6001eb29bf3b7523b2" + integrity sha512-8aAvwVUSHpfEqTQ4w/KMlf3HcRdt50E5ODIQJBw1fQ5RL34xabzxtUlzTXVqc4rkZsPbvrXKWnABCD7kWSmocA== + yargs-parser@20.2.4: version "20.2.4" resolved "https://registry.yarnpkg.com/yargs-parser/-/yargs-parser-20.2.4.tgz#b42890f14566796f85ae8e3a25290d205f154a54" From 2e84b6956464063674204d75a547d8175a5411ec Mon Sep 17 00:00:00 2001 From: CrytoInsight <150222426+CrytoInsight@users.noreply.github.com> Date: Tue, 9 Jan 2024 17:57:45 +0800 Subject: [PATCH 42/49] chore: fix link (#784) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ image ## Why ❔ ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- docs/guides/advanced/zk_intuition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/guides/advanced/zk_intuition.md b/docs/guides/advanced/zk_intuition.md index f3a9ac1fd190..47f6e6d43b13 100644 --- a/docs/guides/advanced/zk_intuition.md +++ b/docs/guides/advanced/zk_intuition.md @@ -143,7 +143,7 @@ version 1.4.0. [bellman repo]: https://github.com/matter-labs/bellman [bellman cuda repo]: https://github.com/matter-labs/era-bellman-cuda [example ecrecover circuit]: - https://github.com/matter-labs/sync_vm/blob/683ade0bbb445f3e2ceb82dd3f4346a0c5d16a78/src/glue/ecrecover_circuit/mod.rs#L157 + https://github.com/matter-labs/era-sync_vm/blob/v1.3.2/src/glue/ecrecover_circuit/mod.rs#L157 [zksync core witness]: https://github.com/matter-labs/zksync-era/blob/main/core/lib/zksync_core/src/witness_generator/mod.rs [separate witness binary]: https://github.com/matter-labs/zksync-era/blob/main/prover/witness_generator/src/main.rs From af4e554fb8a3b526757d2d5adbbedccee6b9ec81 Mon Sep 17 00:00:00 2001 From: web3-Jack <148616461+web3jacker@users.noreply.github.com> Date: Tue, 9 Jan 2024 18:09:12 +0800 Subject: [PATCH 43/49] chore(docs): spelling and grammar fixes (#789) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Igor Aleksanov --- core/tests/vm-benchmark/README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/tests/vm-benchmark/README.md b/core/tests/vm-benchmark/README.md index 4d66f287a707..62d65b4dd80a 100644 --- a/core/tests/vm-benchmark/README.md +++ b/core/tests/vm-benchmark/README.md @@ -1,7 +1,8 @@ # Benchmarking the VM -Currently all benchmarking happens on contract deployment bytecodes. These can execute arbitrary code, so that is -surprisingly useful. This library can be used to build more complex benchmarks, however. +Currently all benchmarking happens on contract deployment bytecodes. Since contract deployment bytecodes can execute +arbitrary code, they are surprisingly useful for benchmarking. This library can be used to build more complex +benchmarks, however. ## Benchmarking From 1a7b4fecca40d1dc3537942c9391c6ad1aaffdaa Mon Sep 17 00:00:00 2001 From: JW Date: Tue, 9 Jan 2024 02:15:21 -0800 Subject: [PATCH 44/49] chore: change fri_prover witness vector receiver port (#772) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ This PR sets default witness_vector_receiver_port=3316 ## Why ❔ 4000 is the beacon chain RPC port https://github.com/matter-labs/zksync-era/blob/3a0ec355f4ade8641d52ccdcc9ed03753fcb7882/docker-compose.yml#L96 `TcpListener::bind` in [prover_fri](https://github.com/matter-labs/zksync-era/blob/3a0ec355f4ade8641d52ccdcc9ed03753fcb7882/prover/prover_fri/src/socket_listener.rs#L58) will fail if a beacon chain node is running. Let's define the port as 3316 consistent with the env_config below https://github.com/matter-labs/zksync-era/blob/3a0ec355f4ade8641d52ccdcc9ed03753fcb7882/core/lib/env_config/src/fri_prover.rs#L32 ## Checklist - [ x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ x] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ x] Code has been formatted via `zk fmt` and `zk lint`. - [ x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. Co-authored-by: perekopskiy <53865202+perekopskiy@users.noreply.github.com> --- etc/env/base/fri_prover.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/etc/env/base/fri_prover.toml b/etc/env/base/fri_prover.toml index ec21e35e1f00..94af27417ae0 100644 --- a/etc/env/base/fri_prover.toml +++ b/etc/env/base/fri_prover.toml @@ -9,6 +9,6 @@ setup_load_mode="FromDisk" specialized_group_id=100 witness_vector_generator_thread_count=5 queue_capacity=10 -witness_vector_receiver_port=4000 +witness_vector_receiver_port=3316 zone_read_url="http://metadata.google.internal/computeMetadata/v1/instance/zone" shall_save_to_public_bucket=true From 998c455199d715d3a338f7641e241415dd3f1776 Mon Sep 17 00:00:00 2001 From: Mattew <149158367+MattewGraham@users.noreply.github.com> Date: Tue, 9 Jan 2024 18:18:53 +0800 Subject: [PATCH 45/49] chore: fix error link (#814) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - fix error link ## Why ❔ - The link is no longer accessible ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: Fedor Sakharov Co-authored-by: perekopskiy <53865202+perekopskiy@users.noreply.github.com> --- docs/guides/advanced/prover_keys.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/guides/advanced/prover_keys.md b/docs/guides/advanced/prover_keys.md index 8cf59067cf89..34660492bf21 100644 --- a/docs/guides/advanced/prover_keys.md +++ b/docs/guides/advanced/prover_keys.md @@ -124,4 +124,4 @@ friendly hash function (currently Poseidon2). [prover_setup_data]: https://github.com/matter-labs/zksync-era/blob/d2ca29bf20b4ec2d9ec9e327b4ba6b281d9793de/prover/vk_setup_data_generator_server_fri/src/lib.rs#L61 [verifier_computation]: - https://github.com/matter-labs/era-contracts/blob/dev/ethereum/contracts/zksync/Verifier.sol#L268 + https://github.com/matter-labs/era-contracts/blob/dev/l1-contracts/contracts/zksync/Verifier.sol#268 From 3e14cf580e0482fe77edb6b8b7cafd952527726d Mon Sep 17 00:00:00 2001 From: 0xWizar <150222244+0xWizar@users.noreply.github.com> Date: Tue, 9 Jan 2024 18:27:57 +0800 Subject: [PATCH 46/49] chore: fix unreachable link (#840) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Unreachable link ## Why ❔ image ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --------- Co-authored-by: perekopskiy <53865202+perekopskiy@users.noreply.github.com> --- docs/guides/advanced/how_l2_messaging_works.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/guides/advanced/how_l2_messaging_works.md b/docs/guides/advanced/how_l2_messaging_works.md index 85410dfbadc0..3bae11fc59dd 100644 --- a/docs/guides/advanced/how_l2_messaging_works.md +++ b/docs/guides/advanced/how_l2_messaging_works.md @@ -201,7 +201,7 @@ transmitted to L1 for final verification. [log_writing_in_vm]: https://github.com/matter-labs/era-zk_evm/blob/v1.3.2/src/opcodes/execution/log.rs [log_opcode]: https://github.com/matter-labs/era-zkevm_opcode_defs/blob/v1.3.2/src/definitions/log.rs#L16 [zkevm_assembly_parse]: - https://github.com/matter-labs/zkEVM-assembly/blob/fcfeb51e45544a629d4279b3455def847dcc2505/src/assembly/instruction/log.rs#L32 + https://github.com/matter-labs/era-zkEVM-assembly/blob/v1.3.2/src/assembly/instruction/log.rs#L32 [executor_sol]: https://github.com/matter-labs/era-contracts/blob/3a4506522aaef81485d8abb96f5a6394bd2ba69e/ethereum/contracts/zksync/facets/Executor.sol#L26 [mainet_executor]: https://etherscan.io/address/0x389a081BCf20e5803288183b929F08458F1d863D From 997db872455351a484c3161d0a733a4bc59dd684 Mon Sep 17 00:00:00 2001 From: Stanislav Bezkorovainyi Date: Tue, 9 Jan 2024 12:23:27 +0100 Subject: [PATCH 47/49] feat(vm): Add batch input abstraction (#817) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ The new fee model will allow us to have separate L1 gas price and pubdata price (and it is needed for EIP4844). That's why the `BatchFeeInput` struct is needed as it provides the needed fields for both the VM version. The version that is provided in this PR will require not additional changes on the server side (i.e. whatever config params that we have right now is enough, the full integration of the config params for the new fee model will be added to the main PR: https://github.com/matter-labs/zksync-era/pull/791/. While the defined enum allows for both l1-gas-price pegged and independent pubdata pricing, in this PR only L1Pegged is ever instantiated ❗ A new JSON-RPC method has been added: `zks_getMainNodeFeeParams`. It will be used by the EN and it will serve the same purpose as the `zks_getL1GasPrice`, i.e. retrieving the fee info from the main node, but now it will be extended to provide the fair l2 gas price as well as other config params that for now will be defined on the main node. The `zks_getL1GasPrice` will continue working as usual to not interrupt the existing external nodes, but may be removed in the long run. ## Why ❔ Splitting the integration of the 1.4.1 & the new fee model (https://github.com/matter-labs/zksync-era/pull/791/) into separate smaller PRs ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/main.rs | 11 +- core/bin/snapshots_creator/src/tests.rs | 3 +- .../system-constants-generator/src/utils.rs | 8 +- core/lib/dal/src/blocks_dal.rs | 4 +- core/lib/dal/src/models/storage_block.rs | 7 +- core/lib/dal/src/sync_dal.rs | 10 +- core/lib/dal/src/tests/mod.rs | 4 +- core/lib/mempool/src/tests.rs | 8 +- core/lib/mempool/src/types.rs | 22 +- .../src/glue/types/vm/block_context_mode.rs | 21 +- .../interface/types/inputs/l1_batch_env.rs | 20 +- core/lib/multivm/src/utils.rs | 66 +-- .../versions/vm_1_3_2/vm_with_bootloader.rs | 58 ++- .../implementation/tx.rs | 6 +- .../vm_boojum_integration/tracers/refunds.rs | 10 +- .../vm_boojum_integration/types/l1_batch.rs | 13 +- .../vm_boojum_integration/utils/fee.rs | 27 +- .../versions/vm_latest/implementation/tx.rs | 6 +- .../versions/vm_latest/tests/default_aa.rs | 6 +- .../src/versions/vm_latest/tests/refunds.rs | 3 +- .../vm_latest/tests/tester/vm_tester.rs | 7 +- .../src/versions/vm_latest/tracers/refunds.rs | 10 +- .../src/versions/vm_latest/types/l1_batch.rs | 13 +- .../src/versions/vm_latest/utils/fee.rs | 69 +++- .../src/versions/vm_m5/vm_with_bootloader.rs | 52 ++- .../src/versions/vm_m6/vm_with_bootloader.rs | 58 ++- .../implementation/tx.rs | 6 +- .../vm_refunds_enhancement/tracers/refunds.rs | 10 +- .../vm_refunds_enhancement/types/l1_batch.rs | 13 +- .../vm_refunds_enhancement/utils/fee.rs | 27 +- .../vm_virtual_blocks/implementation/tx.rs | 6 +- .../vm_virtual_blocks/tracers/refunds.rs | 10 +- .../vm_virtual_blocks/types/l1_batch_env.rs | 14 +- .../versions/vm_virtual_blocks/utils/fee.rs | 26 +- core/lib/state/src/test_utils.rs | 3 +- core/lib/types/src/block.rs | 6 +- core/lib/types/src/fee_model.rs | 185 +++++++++ core/lib/types/src/lib.rs | 1 + core/lib/web3_decl/src/namespaces/zks.rs | 4 + .../src/api_server/execution_sandbox/apply.rs | 17 +- .../src/api_server/execution_sandbox/mod.rs | 5 +- .../src/api_server/tx_sender/mod.rs | 62 ++- .../web3/backend_jsonrpsee/namespaces/zks.rs | 5 + .../src/api_server/web3/namespaces/debug.rs | 4 +- .../src/api_server/web3/namespaces/zks.rs | 23 +- .../src/api_server/web3/tests/mod.rs | 5 + core/lib/zksync_core/src/fee_model.rs | 386 ++++++++++++++++++ core/lib/zksync_core/src/genesis.rs | 4 +- .../src/l1_gas_price/gas_adjuster/mod.rs | 6 + .../src/l1_gas_price/main_node_fetcher.rs | 30 +- core/lib/zksync_core/src/l1_gas_price/mod.rs | 6 +- core/lib/zksync_core/src/lib.rs | 25 +- .../src/metadata_calculator/recovery/tests.rs | 4 +- .../zksync_core/src/state_keeper/io/common.rs | 13 +- .../src/state_keeper/io/mempool.rs | 30 +- .../src/state_keeper/io/seal_logic.rs | 11 +- .../src/state_keeper/io/tests/mod.rs | 37 +- .../src/state_keeper/io/tests/tester.rs | 36 +- .../src/state_keeper/mempool_actor.rs | 24 +- core/lib/zksync_core/src/state_keeper/mod.rs | 6 +- .../zksync_core/src/state_keeper/tests/mod.rs | 4 +- .../src/state_keeper/tests/tester.rs | 5 +- .../src/state_keeper/updates/mod.rs | 11 +- .../zksync_core/src/sync_layer/external_io.rs | 8 +- core/lib/zksync_core/src/sync_layer/tests.rs | 4 +- core/lib/zksync_core/src/utils/testonly.rs | 4 +- core/tests/vm-benchmark/harness/src/lib.rs | 7 +- 67 files changed, 1220 insertions(+), 395 deletions(-) create mode 100644 core/lib/types/src/fee_model.rs create mode 100644 core/lib/zksync_core/src/fee_model.rs diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 7098ddcd1a6a..863deacdb589 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -17,7 +17,7 @@ use zksync_core::{ }, block_reverter::{BlockReverter, BlockReverterFlags, L1ExecutedBatchesRevert}, consistency_checker::ConsistencyChecker, - l1_gas_price::MainNodeGasPriceFetcher, + l1_gas_price::MainNodeFeeParamsFetcher, metadata_calculator::{MetadataCalculator, MetadataCalculatorConfig}, reorg_detector::ReorgDetector, setup_sigint_handler, @@ -124,7 +124,7 @@ async fn init_tasks( let (stop_sender, stop_receiver) = watch::channel(false); let mut healthchecks: Vec> = Vec::new(); // Create components. - let gas_adjuster = Arc::new(MainNodeGasPriceFetcher::new(&main_node_url)); + let fee_params_fetcher = Arc::new(MainNodeFeeParamsFetcher::new(&main_node_url)); let sync_state = SyncState::new(); let (action_queue_sender, action_queue) = ActionQueue::new(); @@ -233,7 +233,8 @@ async fn init_tasks( let updater_handle = task::spawn(batch_status_updater.run(stop_receiver.clone())); let sk_handle = task::spawn(state_keeper.run()); let fetcher_handle = tokio::spawn(fetcher.run()); - let gas_adjuster_handle = tokio::spawn(gas_adjuster.clone().run(stop_receiver.clone())); + let fee_params_fetcher_handle = + tokio::spawn(fee_params_fetcher.clone().run(stop_receiver.clone())); let (tx_sender, vm_barrier, cache_update_handle) = { let tx_sender_builder = @@ -262,7 +263,7 @@ async fn init_tasks( let tx_sender = tx_sender_builder .build( - gas_adjuster, + fee_params_fetcher, Arc::new(vm_concurrency_limiter), ApiContracts::load_from_disk(), // TODO (BFT-138): Allow to dynamically reload API contracts storage_caches, @@ -321,7 +322,7 @@ async fn init_tasks( fetcher_handle, updater_handle, tree_handle, - gas_adjuster_handle, + fee_params_fetcher_handle, ]); task_handles.push(consistency_checker_handle); diff --git a/core/bin/snapshots_creator/src/tests.rs b/core/bin/snapshots_creator/src/tests.rs index d344f453e6c9..e3f4179f5410 100644 --- a/core/bin/snapshots_creator/src/tests.rs +++ b/core/bin/snapshots_creator/src/tests.rs @@ -143,8 +143,7 @@ async fn create_miniblock( l1_tx_count: 0, l2_tx_count: 0, base_fee_per_gas: 0, - l1_gas_price: 0, - l2_fair_gas_price: 0, + batch_fee_input: Default::default(), base_system_contracts_hashes: Default::default(), protocol_version: Some(Default::default()), virtual_blocks: 0, diff --git a/core/bin/system-constants-generator/src/utils.rs b/core/bin/system-constants-generator/src/utils.rs index 87d8eaaf9e33..c594f3065bea 100644 --- a/core/bin/system-constants-generator/src/utils.rs +++ b/core/bin/system-constants-generator/src/utils.rs @@ -18,7 +18,7 @@ use zksync_contracts::{ }; use zksync_state::{InMemoryStorage, StorageView, WriteStorage}; use zksync_types::{ - block::MiniblockHasher, ethabi::Token, fee::Fee, l1::L1Tx, l2::L2Tx, + block::MiniblockHasher, ethabi::Token, fee::Fee, fee_model::BatchFeeInput, l1::L1Tx, l2::L2Tx, utils::storage_key_for_eth_balance, AccountTreeId, Address, Execute, L1BatchNumber, L1TxCommonData, L2ChainId, MiniblockNumber, Nonce, ProtocolVersionId, StorageKey, Timestamp, Transaction, BOOTLOADER_ADDRESS, H256, SYSTEM_CONTEXT_ADDRESS, @@ -174,8 +174,10 @@ fn default_l1_batch() -> L1BatchEnv { previous_batch_hash: None, number: L1BatchNumber(1), timestamp: 100, - l1_gas_price: 50_000_000_000, // 50 gwei - fair_l2_gas_price: 250_000_000, // 0.25 gwei + fee_input: BatchFeeInput::l1_pegged( + 50_000_000_000, // 50 gwei + 250_000_000, // 0.25 gwei + ), fee_account: Address::random(), enforced_base_fee: None, first_l2_block: L2BlockEnv { diff --git a/core/lib/dal/src/blocks_dal.rs b/core/lib/dal/src/blocks_dal.rs index 07992d1e937c..255533ba1cf9 100644 --- a/core/lib/dal/src/blocks_dal.rs +++ b/core/lib/dal/src/blocks_dal.rs @@ -626,8 +626,8 @@ impl BlocksDal<'_, '_> { miniblock_header.l1_tx_count as i32, miniblock_header.l2_tx_count as i32, base_fee_per_gas, - miniblock_header.l1_gas_price as i64, - miniblock_header.l2_fair_gas_price as i64, + miniblock_header.batch_fee_input.l1_gas_price() as i64, + miniblock_header.batch_fee_input.fair_l2_gas_price() as i64, MAX_GAS_PER_PUBDATA_BYTE as i64, miniblock_header .base_system_contracts_hashes diff --git a/core/lib/dal/src/models/storage_block.rs b/core/lib/dal/src/models/storage_block.rs index 4c5db47566a8..2ff98815abf4 100644 --- a/core/lib/dal/src/models/storage_block.rs +++ b/core/lib/dal/src/models/storage_block.rs @@ -529,8 +529,11 @@ impl From for MiniblockHeader { l1_tx_count: row.l1_tx_count as u16, l2_tx_count: row.l2_tx_count as u16, base_fee_per_gas: row.base_fee_per_gas.to_u64().unwrap(), - l1_gas_price: row.l1_gas_price as u64, - l2_fair_gas_price: row.l2_fair_gas_price as u64, + // For now, only L1 pegged fee model is supported. + batch_fee_input: zksync_types::fee_model::BatchFeeInput::l1_pegged( + row.l1_gas_price as u64, + row.l2_fair_gas_price as u64, + ), base_system_contracts_hashes: convert_base_system_contracts_hashes( row.bootloader_code_hash, row.default_aa_code_hash, diff --git a/core/lib/dal/src/sync_dal.rs b/core/lib/dal/src/sync_dal.rs index c9a737f581d0..a80e810525e4 100644 --- a/core/lib/dal/src/sync_dal.rs +++ b/core/lib/dal/src/sync_dal.rs @@ -188,8 +188,14 @@ mod tests { block.virtual_blocks.unwrap(), miniblock_header.virtual_blocks ); - assert_eq!(block.l1_gas_price, miniblock_header.l1_gas_price); - assert_eq!(block.l2_fair_gas_price, miniblock_header.l2_fair_gas_price); + assert_eq!( + block.l1_gas_price, + miniblock_header.batch_fee_input.l1_gas_price() + ); + assert_eq!( + block.l2_fair_gas_price, + miniblock_header.batch_fee_input.fair_l2_gas_price() + ); assert_eq!(block.operator_address, operator_address); assert!(block.transactions.is_none()); diff --git a/core/lib/dal/src/tests/mod.rs b/core/lib/dal/src/tests/mod.rs index 7a4a85b5edb3..5830ee14110d 100644 --- a/core/lib/dal/src/tests/mod.rs +++ b/core/lib/dal/src/tests/mod.rs @@ -4,6 +4,7 @@ use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{ block::{MiniblockHasher, MiniblockHeader}, fee::{Fee, TransactionExecutionMetrics}, + fee_model::BatchFeeInput, helpers::unix_timestamp_ms, l1::{L1Tx, OpProcessingType, PriorityQueueType}, l2::L2Tx, @@ -36,8 +37,7 @@ pub(crate) fn create_miniblock_header(number: u32) -> MiniblockHeader { l1_tx_count: 0, l2_tx_count: 0, base_fee_per_gas: 100, - l1_gas_price: 100, - l2_fair_gas_price: 100, + batch_fee_input: BatchFeeInput::l1_pegged(100, 100), base_system_contracts_hashes: BaseSystemContractsHashes::default(), protocol_version: Some(protocol_version), virtual_blocks: 1, diff --git a/core/lib/mempool/src/tests.rs b/core/lib/mempool/src/tests.rs index 38d4b62aafaf..656d90c63d14 100644 --- a/core/lib/mempool/src/tests.rs +++ b/core/lib/mempool/src/tests.rs @@ -244,13 +244,13 @@ fn mempool_size() { fn filtering() { // Filter to find transactions with non-zero `gas_per_pubdata` values. let filter_non_zero = L2TxFilter { - l1_gas_price: 0u64, + fee_input: Default::default(), fee_per_gas: 0u64, gas_per_pubdata: 1u32, }; // No-op filter that fetches any transaction. let filter_zero = L2TxFilter { - l1_gas_price: 0u64, + fee_input: Default::default(), fee_per_gas: 0u64, gas_per_pubdata: 0u32, }; @@ -288,13 +288,13 @@ fn filtering() { #[test] fn stashed_accounts() { let filter_non_zero = L2TxFilter { - l1_gas_price: 0u64, + fee_input: Default::default(), fee_per_gas: 0u64, gas_per_pubdata: 1u32, }; // No-op filter that fetches any transaction. let filter_zero = L2TxFilter { - l1_gas_price: 0u64, + fee_input: Default::default(), fee_per_gas: 0u64, gas_per_pubdata: 0u32, }; diff --git a/core/lib/mempool/src/types.rs b/core/lib/mempool/src/types.rs index 9bc58a4e2cea..99a63ffd08e2 100644 --- a/core/lib/mempool/src/types.rs +++ b/core/lib/mempool/src/types.rs @@ -1,6 +1,8 @@ use std::{cmp::Ordering, collections::HashMap}; -use zksync_types::{fee::Fee, l2::L2Tx, Address, Nonce, Transaction, U256}; +use zksync_types::{ + fee::Fee, fee_model::BatchFeeInput, l2::L2Tx, Address, Nonce, Transaction, U256, +}; /// Pending mempool transactions of account #[derive(Debug)] @@ -128,8 +130,8 @@ pub(crate) struct InsertionMetadata { /// criteria for transaction it wants to fetch. #[derive(Debug, Default, PartialEq, Eq)] pub struct L2TxFilter { - /// L1 gas price. - pub l1_gas_price: u64, + /// Batch fee model input. It typically includes things like L1 gas price, L2 fair fee, etc. + pub fee_input: BatchFeeInput, /// Effective fee price for the transaction. The price of 1 gas in wei. pub fee_per_gas: u64, /// Effective pubdata price in gas for transaction. The number of gas per 1 pubdata byte. @@ -143,9 +145,9 @@ mod tests { /// Checks the filter logic. #[test] fn filter() { - fn filter(l1_gas_price: u64, fee_per_gas: u64, gas_per_pubdata: u32) -> L2TxFilter { + fn filter(fee_per_gas: u64, gas_per_pubdata: u32) -> L2TxFilter { L2TxFilter { - l1_gas_price, + fee_input: BatchFeeInput::sensible_l1_pegged_default(), fee_per_gas, gas_per_pubdata, } @@ -166,31 +168,31 @@ mod tests { }, }; - let noop_filter = filter(0, 0, 0); + let noop_filter = filter(0, 0); assert!( score.matches_filter(&noop_filter), "Noop filter should always match" ); - let max_gas_filter = filter(0, MAX_FEE_PER_GAS, 0); + let max_gas_filter = filter(MAX_FEE_PER_GAS, 0); assert!( score.matches_filter(&max_gas_filter), "Correct max gas should be accepted" ); - let pubdata_filter = filter(0, 0, GAS_PER_PUBDATA_LIMIT); + let pubdata_filter = filter(0, GAS_PER_PUBDATA_LIMIT); assert!( score.matches_filter(&pubdata_filter), "Correct pubdata price should be accepted" ); - let decline_gas_filter = filter(0, MAX_FEE_PER_GAS + 1, 0); + let decline_gas_filter = filter(MAX_FEE_PER_GAS + 1, 0); assert!( !score.matches_filter(&decline_gas_filter), "Incorrect max gas should be rejected" ); - let decline_pubdata_filter = filter(0, 0, GAS_PER_PUBDATA_LIMIT + 1); + let decline_pubdata_filter = filter(0, GAS_PER_PUBDATA_LIMIT + 1); assert!( !score.matches_filter(&decline_pubdata_filter), "Incorrect pubdata price should be rejected" diff --git a/core/lib/multivm/src/glue/types/vm/block_context_mode.rs b/core/lib/multivm/src/glue/types/vm/block_context_mode.rs index 0cbbcbf33e3a..094339705e14 100644 --- a/core/lib/multivm/src/glue/types/vm/block_context_mode.rs +++ b/core/lib/multivm/src/glue/types/vm/block_context_mode.rs @@ -4,15 +4,16 @@ use crate::glue::GlueFrom; impl GlueFrom for crate::vm_m5::vm_with_bootloader::BlockContextMode { fn glue_from(value: crate::interface::L1BatchEnv) -> Self { + let fee_input = value.fee_input.into_l1_pegged(); let derived = crate::vm_m5::vm_with_bootloader::DerivedBlockContext { context: crate::vm_m5::vm_with_bootloader::BlockContext { block_number: value.number.0, block_timestamp: value.timestamp, operator_address: value.fee_account, - l1_gas_price: value.l1_gas_price, - fair_l2_gas_price: value.fair_l2_gas_price, + l1_gas_price: fee_input.l1_gas_price, + fair_l2_gas_price: fee_input.fair_l2_gas_price, }, - base_fee: value.base_fee(), + base_fee: crate::vm_m5::vm_with_bootloader::get_batch_base_fee(&value), }; match value.previous_batch_hash { Some(hash) => Self::NewBlock(derived, h256_to_u256(hash)), @@ -23,15 +24,16 @@ impl GlueFrom for crate::vm_m5::vm_with_bootloader impl GlueFrom for crate::vm_m6::vm_with_bootloader::BlockContextMode { fn glue_from(value: crate::interface::L1BatchEnv) -> Self { + let fee_input = value.fee_input.into_l1_pegged(); let derived = crate::vm_m6::vm_with_bootloader::DerivedBlockContext { context: crate::vm_m6::vm_with_bootloader::BlockContext { block_number: value.number.0, block_timestamp: value.timestamp, operator_address: value.fee_account, - l1_gas_price: value.l1_gas_price, - fair_l2_gas_price: value.fair_l2_gas_price, + l1_gas_price: fee_input.l1_gas_price, + fair_l2_gas_price: fee_input.fair_l2_gas_price, }, - base_fee: value.base_fee(), + base_fee: crate::vm_m6::vm_with_bootloader::get_batch_base_fee(&value), }; match value.previous_batch_hash { Some(hash) => Self::NewBlock(derived, h256_to_u256(hash)), @@ -44,15 +46,16 @@ impl GlueFrom for crate::vm_1_3_2::vm_with_bootloader::BlockContextMode { fn glue_from(value: crate::interface::L1BatchEnv) -> Self { + let fee_input = value.fee_input.into_l1_pegged(); let derived = crate::vm_1_3_2::vm_with_bootloader::DerivedBlockContext { context: crate::vm_1_3_2::vm_with_bootloader::BlockContext { block_number: value.number.0, block_timestamp: value.timestamp, operator_address: value.fee_account, - l1_gas_price: value.l1_gas_price, - fair_l2_gas_price: value.fair_l2_gas_price, + l1_gas_price: fee_input.l1_gas_price, + fair_l2_gas_price: fee_input.fair_l2_gas_price, }, - base_fee: value.base_fee(), + base_fee: crate::vm_1_3_2::vm_with_bootloader::get_batch_base_fee(&value), }; match value.previous_batch_hash { Some(hash) => Self::NewBlock(derived, h256_to_u256(hash)), diff --git a/core/lib/multivm/src/interface/types/inputs/l1_batch_env.rs b/core/lib/multivm/src/interface/types/inputs/l1_batch_env.rs index b5cb0cbe5e8f..fe0577c6a471 100644 --- a/core/lib/multivm/src/interface/types/inputs/l1_batch_env.rs +++ b/core/lib/multivm/src/interface/types/inputs/l1_batch_env.rs @@ -1,7 +1,6 @@ -use zksync_types::{Address, L1BatchNumber, H256}; +use zksync_types::{fee_model::BatchFeeInput, Address, L1BatchNumber, H256}; use super::L2BlockEnv; -use crate::vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata; /// Unique params for each block #[derive(Debug, Clone)] @@ -10,23 +9,8 @@ pub struct L1BatchEnv { pub previous_batch_hash: Option, pub number: L1BatchNumber, pub timestamp: u64, - pub l1_gas_price: u64, - pub fair_l2_gas_price: u64, + pub fee_input: BatchFeeInput, pub fee_account: Address, pub enforced_base_fee: Option, pub first_l2_block: L2BlockEnv, } - -impl L1BatchEnv { - pub fn base_fee(&self) -> u64 { - if let Some(base_fee) = self.enforced_base_fee { - return base_fee; - } - let (base_fee, _) = - derive_base_fee_and_gas_per_pubdata(self.l1_gas_price, self.fair_l2_gas_price); - base_fee - } - pub(crate) fn block_gas_price_per_pubdata(&self) -> u64 { - derive_base_fee_and_gas_per_pubdata(self.l1_gas_price, self.fair_l2_gas_price).1 - } -} diff --git a/core/lib/multivm/src/utils.rs b/core/lib/multivm/src/utils.rs index 4bab29f14b56..1f639124b94b 100644 --- a/core/lib/multivm/src/utils.rs +++ b/core/lib/multivm/src/utils.rs @@ -1,68 +1,82 @@ -use zksync_types::{VmVersion, U256}; +use zksync_types::{fee_model::BatchFeeInput, VmVersion, U256}; + +use crate::vm_latest::L1BatchEnv; /// Calculates the base fee and gas per pubdata for the given L1 gas price. pub fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + batch_fee_input: BatchFeeInput, vm_version: VmVersion, ) -> (u64, u64) { match vm_version { VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { crate::vm_m5::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { crate::vm_m6::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } VmVersion::Vm1_3_2 => { crate::vm_1_3_2::vm_with_bootloader::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } VmVersion::VmVirtualBlocks => { crate::vm_virtual_blocks::utils::fee::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } VmVersion::VmVirtualBlocksRefundsEnhancement => { crate::vm_refunds_enhancement::utils::fee::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } VmVersion::VmBoojumIntegration => { crate::vm_boojum_integration::utils::fee::derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - fair_gas_price, + batch_fee_input.into_l1_pegged(), ) } } } -/// Changes the fee model output so that the expected gas per pubdata is smaller than or the `tx_gas_per_pubdata_limit`. -pub fn adjust_l1_gas_price_for_tx( - l1_gas_price: u64, - fair_l2_gas_price: u64, +pub fn get_batch_base_fee(l1_batch_env: &L1BatchEnv, vm_version: VmVersion) -> u64 { + match vm_version { + VmVersion::M5WithRefunds | VmVersion::M5WithoutRefunds => { + crate::vm_m5::vm_with_bootloader::get_batch_base_fee(l1_batch_env) + } + VmVersion::M6Initial | VmVersion::M6BugWithCompressionFixed => { + crate::vm_m6::vm_with_bootloader::get_batch_base_fee(l1_batch_env) + } + VmVersion::Vm1_3_2 => crate::vm_1_3_2::vm_with_bootloader::get_batch_base_fee(l1_batch_env), + VmVersion::VmVirtualBlocks => { + crate::vm_virtual_blocks::utils::fee::get_batch_base_fee(l1_batch_env) + } + VmVersion::VmVirtualBlocksRefundsEnhancement => { + crate::vm_refunds_enhancement::utils::fee::get_batch_base_fee(l1_batch_env) + } + VmVersion::VmBoojumIntegration => { + crate::vm_boojum_integration::utils::fee::get_batch_base_fee(l1_batch_env) + } + } +} + +/// Changes the batch fee input so that the expected gas per pubdata is smaller than or the `tx_gas_per_pubdata_limit`. +pub fn adjust_pubdata_price_for_tx( + batch_fee_input: BatchFeeInput, tx_gas_per_pubdata_limit: U256, vm_version: VmVersion, -) -> u64 { - if U256::from( - derive_base_fee_and_gas_per_pubdata(l1_gas_price, fair_l2_gas_price, vm_version).1, - ) <= tx_gas_per_pubdata_limit +) -> BatchFeeInput { + if U256::from(derive_base_fee_and_gas_per_pubdata(batch_fee_input, vm_version).1) + <= tx_gas_per_pubdata_limit { - return l1_gas_price; + return batch_fee_input; } // The latest VM supports adjusting the pubdata price for all the types of the fee models. - crate::vm_latest::utils::fee::adjust_l1_gas_price_for_tx( - fair_l2_gas_price, + crate::vm_latest::utils::fee::adjust_pubdata_price_for_tx( + batch_fee_input, tx_gas_per_pubdata_limit, ) } diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs index 5ecbea6ff150..8bf8c78235d4 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs @@ -14,9 +14,10 @@ use zk_evm_1_3_3::{ use zksync_contracts::BaseSystemContracts; use zksync_state::WriteStorage; use zksync_types::{ - l1::is_l1_tx_type, zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, - BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, - U256, USED_BOOTLOADER_MEMORY_WORDS, + fee_model::L1PeggedBatchFeeModelInput, l1::is_l1_tx_type, + zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, + L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + USED_BOOTLOADER_MEMORY_WORDS, }; use zksync_utils::{ address_to_u256, @@ -25,15 +26,18 @@ use zksync_utils::{ misc::ceil_div, }; -use crate::vm_1_3_2::{ - bootloader_state::BootloaderState, - history_recorder::HistoryMode, - transaction_data::TransactionData, - utils::{ - code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, +use crate::{ + vm_1_3_2::{ + bootloader_state::BootloaderState, + history_recorder::HistoryMode, + transaction_data::TransactionData, + utils::{ + code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, + }, + vm_instance::ZkSyncVmState, + OracleTools, VmInstance, }, - vm_instance::ZkSyncVmState, - OracleTools, VmInstance, + vm_latest::L1BatchEnv, }; // TODO (SMA-1703): move these to config and make them programmatically generable. @@ -59,7 +63,11 @@ pub struct BlockContext { impl BlockContext { pub fn block_gas_price_per_pubdata(&self) -> u64 { - derive_base_fee_and_gas_per_pubdata(self.l1_gas_price, self.fair_l2_gas_price).1 + derive_base_fee_and_gas_per_pubdata(L1PeggedBatchFeeModelInput { + l1_gas_price: self.l1_gas_price, + fair_l2_gas_price: self.fair_l2_gas_price, + }) + .1 } } @@ -84,15 +92,19 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -102,10 +114,22 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) } +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + impl From for DerivedBlockContext { fn from(context: BlockContext) -> Self { - let base_fee = - derive_base_fee_and_gas_per_pubdata(context.l1_gas_price, context.fair_l2_gas_price).0; + let base_fee = derive_base_fee_and_gas_per_pubdata(L1PeggedBatchFeeModelInput { + l1_gas_price: context.l1_gas_price, + fair_l2_gas_price: context.fair_l2_gas_price, + }) + .0; DerivedBlockContext { context, base_fee } } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs index e341f79d1d46..9eac3e749837 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/tx.rs @@ -7,6 +7,7 @@ use crate::{ constants::BOOTLOADER_HEAP_PAGE, implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, types::internals::TransactionData, + utils::fee::get_batch_gas_per_pubdata, vm::Vm, }, HistoryMode, @@ -38,8 +39,7 @@ impl Vm { .decommittment_processor .populate(codes_for_decommiter, timestamp); - let trusted_ergs_limit = - tx.trusted_ergs_limit(self.batch_env.block_gas_price_per_pubdata()); + let trusted_ergs_limit = tx.trusted_ergs_limit(get_batch_gas_per_pubdata(&self.batch_env)); let memory = self.bootloader_state.push_tx( tx, @@ -61,7 +61,7 @@ impl Vm { with_compression: bool, ) { let tx: TransactionData = tx.into(); - let block_gas_per_pubdata_byte = self.batch_env.block_gas_price_per_pubdata(); + let block_gas_per_pubdata_byte = get_batch_gas_per_pubdata(&self.batch_env); let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); self.push_raw_transaction(tx, overhead, 0, with_compression); } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs index 9f2d6837030c..a4e7295eca81 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs @@ -34,6 +34,7 @@ use crate::{ }, }, types::internals::ZkSyncVmState, + utils::fee::get_batch_base_fee, }, }; @@ -116,13 +117,14 @@ impl RefundsTracer { }); // For now, bootloader charges only for base fee. - let effective_gas_price = self.l1_batch.base_fee(); + let effective_gas_price = get_batch_base_fee(&self.l1_batch); let bootloader_eth_price_per_pubdata_byte = U256::from(effective_gas_price) * U256::from(current_ergs_per_pubdata_byte); - let fair_eth_price_per_pubdata_byte = - U256::from(eth_price_per_pubdata_byte(self.l1_batch.l1_gas_price)); + let fair_eth_price_per_pubdata_byte = U256::from(eth_price_per_pubdata_byte( + self.l1_batch.fee_input.l1_gas_price(), + )); // For now, L1 originated transactions are allowed to pay less than fair fee per pubdata, // so we should take it into account. @@ -132,7 +134,7 @@ impl RefundsTracer { ); let fair_fee_eth = U256::from(gas_spent_on_computation) - * U256::from(self.l1_batch.fair_l2_gas_price) + * U256::from(self.l1_batch.fee_input.fair_l2_gas_price()) + U256::from(pubdata_published) * eth_price_per_pubdata_byte_for_calculation; let pre_paid_eth = U256::from(tx_gas_limit) * U256::from(effective_gas_price); let refund_eth = pre_paid_eth.checked_sub(fair_fee_eth).unwrap_or_else(|| { diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs index 6f16e95f8d77..386dc040099b 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/l1_batch.rs @@ -1,7 +1,7 @@ use zksync_types::U256; use zksync_utils::{address_to_u256, h256_to_u256}; -use crate::interface::L1BatchEnv; +use crate::{interface::L1BatchEnv, vm_boojum_integration::utils::fee::get_batch_base_fee}; const OPERATOR_ADDRESS_SLOT: usize = 0; const PREV_BLOCK_HASH_SLOT: usize = 1; @@ -19,6 +19,8 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 .map(|prev_block_hash| (h256_to_u256(prev_block_hash), U256::one())) .unwrap_or_default(); + let fee_input = l1_batch.fee_input.into_l1_pegged(); + vec![ ( OPERATOR_ADDRESS_SLOT, @@ -27,12 +29,15 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 (PREV_BLOCK_HASH_SLOT, prev_block_hash), (NEW_BLOCK_TIMESTAMP_SLOT, U256::from(l1_batch.timestamp)), (NEW_BLOCK_NUMBER_SLOT, U256::from(l1_batch.number.0)), - (L1_GAS_PRICE_SLOT, U256::from(l1_batch.l1_gas_price)), + (L1_GAS_PRICE_SLOT, U256::from(fee_input.l1_gas_price)), ( FAIR_L2_GAS_PRICE_SLOT, - U256::from(l1_batch.fair_l2_gas_price), + U256::from(fee_input.fair_l2_gas_price), + ), + ( + EXPECTED_BASE_FEE_SLOT, + U256::from(get_batch_base_fee(l1_batch)), ), - (EXPECTED_BASE_FEE_SLOT, U256::from(l1_batch.base_fee())), (SHOULD_SET_NEW_BLOCK_SLOT, should_set_new_block), ] } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs index b8d5bd1a38f2..511dc7bccd57 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/utils/fee.rs @@ -1,8 +1,11 @@ //! Utility functions for vm use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; +use zksync_types::fee_model::L1PeggedBatchFeeModelInput; use zksync_utils::ceil_div; -use crate::vm_boojum_integration::old_vm::utils::eth_price_per_pubdata_byte; +use crate::{ + vm_boojum_integration::old_vm::utils::eth_price_per_pubdata_byte, vm_latest::L1BatchEnv, +}; /// Calculates the amount of gas required to publish one byte of pubdata pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { @@ -13,15 +16,18 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { /// Calculates the base fee and gas per pubdata for the given L1 gas price. pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -30,3 +36,16 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( base_fee_to_gas_per_pubdata(l1_gas_price, base_fee), ) } + +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + +pub(crate) fn get_batch_gas_per_pubdata(l1_batch_env: &L1BatchEnv) -> u64 { + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()).1 +} diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/tx.rs b/core/lib/multivm/src/versions/vm_latest/implementation/tx.rs index 326be41c5ee7..6293ea13617e 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/tx.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/tx.rs @@ -7,6 +7,7 @@ use crate::{ constants::BOOTLOADER_HEAP_PAGE, implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, types::internals::TransactionData, + utils::fee::get_batch_gas_per_pubdata, vm::Vm, }, HistoryMode, @@ -38,8 +39,7 @@ impl Vm { .decommittment_processor .populate(codes_for_decommiter, timestamp); - let trusted_ergs_limit = - tx.trusted_ergs_limit(self.batch_env.block_gas_price_per_pubdata()); + let trusted_ergs_limit = tx.trusted_ergs_limit(get_batch_gas_per_pubdata(&self.batch_env)); let memory = self.bootloader_state.push_tx( tx, @@ -61,7 +61,7 @@ impl Vm { with_compression: bool, ) { let tx: TransactionData = tx.into(); - let block_gas_per_pubdata_byte = self.batch_env.block_gas_price_per_pubdata(); + let block_gas_per_pubdata_byte = get_batch_gas_per_pubdata(&self.batch_env); let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); self.push_raw_transaction(tx, overhead, 0, with_compression); } diff --git a/core/lib/multivm/src/versions/vm_latest/tests/default_aa.rs b/core/lib/multivm/src/versions/vm_latest/tests/default_aa.rs index 7c951e313210..05e3e64f9c9a 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/default_aa.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/default_aa.rs @@ -13,6 +13,7 @@ use crate::{ tester::{DeployContractsTx, TxType, VmTesterBuilder}, utils::{get_balance, read_test_contract, verify_required_storage}, }, + utils::fee::get_batch_base_fee, HistoryEnabled, }, }; @@ -34,7 +35,7 @@ fn test_default_aa_interaction() { bytecode_hash, address, } = account.get_deploy_tx(&counter, None, TxType::L2); - let maximal_fee = tx.gas_limit() * vm.vm.batch_env.base_fee(); + let maximal_fee = tx.gas_limit() * get_batch_base_fee(&vm.vm.batch_env); vm.vm.push_transaction(tx); let result = vm.vm.execute(VmExecutionMode::OneTx); @@ -62,7 +63,8 @@ fn test_default_aa_interaction() { verify_required_storage(&vm.vm.state, expected_slots); let expected_fee = maximal_fee - - U256::from(result.refunds.gas_refunded) * U256::from(vm.vm.batch_env.base_fee()); + - U256::from(result.refunds.gas_refunded) + * U256::from(get_batch_base_fee(&vm.vm.batch_env)); let operator_balance = get_balance( AccountTreeId::new(L2_ETH_TOKEN_ADDRESS), &vm.fee_account, diff --git a/core/lib/multivm/src/versions/vm_latest/tests/refunds.rs b/core/lib/multivm/src/versions/vm_latest/tests/refunds.rs index dc1f4fe55bca..7746121c1cff 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/refunds.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/refunds.rs @@ -6,6 +6,7 @@ use crate::{ utils::read_test_contract, }, types::internals::TransactionData, + utils::fee::get_batch_gas_per_pubdata, HistoryEnabled, }, }; @@ -60,7 +61,7 @@ fn test_predetermined_refunded_gas() { .build(); let tx: TransactionData = tx.into(); - let block_gas_per_pubdata_byte = vm.vm.batch_env.block_gas_price_per_pubdata(); + let block_gas_per_pubdata_byte = get_batch_gas_per_pubdata(&vm.vm.batch_env); // Overhead let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); vm.vm diff --git a/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs b/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs index 9574c434062d..0220ba4fc4d5 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/tester/vm_tester.rs @@ -4,6 +4,7 @@ use zksync_contracts::BaseSystemContracts; use zksync_state::{InMemoryStorage, StoragePtr, StorageView, WriteStorage}; use zksync_types::{ block::MiniblockHasher, + fee_model::BatchFeeInput, get_code_key, get_is_account_key, helpers::unix_timestamp_ms, utils::{deployed_address_create, storage_key_for_eth_balance}, @@ -251,8 +252,10 @@ pub(crate) fn default_l1_batch(number: L1BatchNumber) -> L1BatchEnv { previous_batch_hash: None, number, timestamp, - l1_gas_price: 50_000_000_000, // 50 gwei - fair_l2_gas_price: 250_000_000, // 0.25 gwei + fee_input: BatchFeeInput::l1_pegged( + 50_000_000_000, // 50 gwei + 250_000_000, // 0.25 gwei + ), fee_account: Address::random(), enforced_base_fee: None, first_l2_block: L2BlockEnv { diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs index c67f086dc119..76d3e4851c45 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/refunds.rs @@ -34,6 +34,7 @@ use crate::{ }, }, types::internals::ZkSyncVmState, + utils::fee::get_batch_base_fee, }, }; @@ -116,13 +117,14 @@ impl RefundsTracer { }); // For now, bootloader charges only for base fee. - let effective_gas_price = self.l1_batch.base_fee(); + let effective_gas_price = get_batch_base_fee(&self.l1_batch); let bootloader_eth_price_per_pubdata_byte = U256::from(effective_gas_price) * U256::from(current_ergs_per_pubdata_byte); - let fair_eth_price_per_pubdata_byte = - U256::from(eth_price_per_pubdata_byte(self.l1_batch.l1_gas_price)); + let fair_eth_price_per_pubdata_byte = U256::from(eth_price_per_pubdata_byte( + self.l1_batch.fee_input.l1_gas_price(), + )); // For now, L1 originated transactions are allowed to pay less than fair fee per pubdata, // so we should take it into account. @@ -132,7 +134,7 @@ impl RefundsTracer { ); let fair_fee_eth = U256::from(gas_spent_on_computation) - * U256::from(self.l1_batch.fair_l2_gas_price) + * U256::from(self.l1_batch.fee_input.fair_l2_gas_price()) + U256::from(pubdata_published) * eth_price_per_pubdata_byte_for_calculation; let pre_paid_eth = U256::from(tx_gas_limit) * U256::from(effective_gas_price); let refund_eth = pre_paid_eth.checked_sub(fair_fee_eth).unwrap_or_else(|| { diff --git a/core/lib/multivm/src/versions/vm_latest/types/l1_batch.rs b/core/lib/multivm/src/versions/vm_latest/types/l1_batch.rs index 6f16e95f8d77..dad16e6a16a0 100644 --- a/core/lib/multivm/src/versions/vm_latest/types/l1_batch.rs +++ b/core/lib/multivm/src/versions/vm_latest/types/l1_batch.rs @@ -1,7 +1,7 @@ use zksync_types::U256; use zksync_utils::{address_to_u256, h256_to_u256}; -use crate::interface::L1BatchEnv; +use crate::{interface::L1BatchEnv, vm_latest::utils::fee::get_batch_base_fee}; const OPERATOR_ADDRESS_SLOT: usize = 0; const PREV_BLOCK_HASH_SLOT: usize = 1; @@ -19,6 +19,8 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 .map(|prev_block_hash| (h256_to_u256(prev_block_hash), U256::one())) .unwrap_or_default(); + let fee_input = l1_batch.fee_input.into_l1_pegged(); + vec![ ( OPERATOR_ADDRESS_SLOT, @@ -27,12 +29,15 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 (PREV_BLOCK_HASH_SLOT, prev_block_hash), (NEW_BLOCK_TIMESTAMP_SLOT, U256::from(l1_batch.timestamp)), (NEW_BLOCK_NUMBER_SLOT, U256::from(l1_batch.number.0)), - (L1_GAS_PRICE_SLOT, U256::from(l1_batch.l1_gas_price)), + (L1_GAS_PRICE_SLOT, U256::from(fee_input.l1_gas_price)), ( FAIR_L2_GAS_PRICE_SLOT, - U256::from(l1_batch.fair_l2_gas_price), + U256::from(fee_input.fair_l2_gas_price), + ), + ( + EXPECTED_BASE_FEE_SLOT, + U256::from(get_batch_base_fee(l1_batch)), ), - (EXPECTED_BASE_FEE_SLOT, U256::from(l1_batch.base_fee())), (SHOULD_SET_NEW_BLOCK_SLOT, should_set_new_block), ] } diff --git a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs index 6efd9274523d..1157a1eb57bd 100644 --- a/core/lib/multivm/src/versions/vm_latest/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_latest/utils/fee.rs @@ -1,9 +1,12 @@ //! Utility functions for vm use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; -use zksync_types::U256; +use zksync_types::{ + fee_model::{BatchFeeInput, L1PeggedBatchFeeModelInput}, + U256, +}; use zksync_utils::ceil_div; -use crate::vm_latest::old_vm::utils::eth_price_per_pubdata_byte; +use crate::vm_latest::{old_vm::utils::eth_price_per_pubdata_byte, L1BatchEnv}; /// Calculates the amount of gas required to publish one byte of pubdata pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { @@ -14,15 +17,19 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { /// Calculates the base fee and gas per pubdata for the given L1 gas price. pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -32,20 +39,46 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) } +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + +pub(crate) fn get_batch_gas_per_pubdata(l1_batch_env: &L1BatchEnv) -> u64 { + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()).1 +} + /// Changes the fee model output so that the expected gas per pubdata is smaller than or the `tx_gas_per_pubdata_limit`. /// This function expects that the currently expected gas per pubdata is greater than the `tx_gas_per_pubdata_limit`. -pub(crate) fn adjust_l1_gas_price_for_tx( - fair_l2_gas_price: u64, +pub(crate) fn adjust_pubdata_price_for_tx( + mut batch_fee_input: BatchFeeInput, tx_gas_per_pubdata_limit: U256, -) -> u64 { - //``` - // gasPerPubdata = ceil(17 * l1gasprice / fair_l2_gas_price) - // gasPerPubdata <= 17 * l1gasprice / fair_l2_gas_price + 1 - // fair_l2_gas_price(gasPerPubdata - 1) / 17 <= l1gasprice - //``` - let l1_gas_price = U256::from(fair_l2_gas_price) - * (tx_gas_per_pubdata_limit - U256::from(1u32)) - / U256::from(17); - - l1_gas_price.as_u64() +) -> BatchFeeInput { + match &mut batch_fee_input { + BatchFeeInput::L1Pegged(fee_input) => { + // `gasPerPubdata = ceil(17 * l1gasprice / fair_l2_gas_price)` + // `gasPerPubdata <= 17 * l1gasprice / fair_l2_gas_price + 1` + // `fair_l2_gas_price(gasPerPubdata - 1) / 17 <= l1gasprice` + let new_l1_gas_price = U256::from(fee_input.fair_l2_gas_price) + * (tx_gas_per_pubdata_limit - U256::from(1u32)) + / U256::from(17); + + fee_input.l1_gas_price = new_l1_gas_price.as_u64(); + } + BatchFeeInput::PubdataIndependent(fee_input) => { + // `gasPerPubdata = ceil(fair_pubdata_price / fair_l2_gas_price)` + // `gasPerPubdata <= fair_pubdata_price / fair_l2_gas_price + 1` + // `fair_l2_gas_price(gasPerPubdata - 1) <= fair_pubdata_price` + let new_fair_pubdata_price = U256::from(fee_input.fair_l2_gas_price) + * (tx_gas_per_pubdata_limit - U256::from(1u32)); + + fee_input.fair_pubdata_price = new_fair_pubdata_price.as_u64(); + } + } + + batch_fee_input } diff --git a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs index 766103fee4fb..13737681f31d 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs @@ -12,23 +12,27 @@ use zk_evm_1_3_1::{ }; use zksync_contracts::BaseSystemContracts; use zksync_types::{ - zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, - L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + fee_model::L1PeggedBatchFeeModelInput, zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, + Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, + MAX_NEW_FACTORY_DEPS, U256, }; use zksync_utils::{ address_to_u256, bytecode::hash_bytecode, bytes_to_be_words, h256_to_u256, misc::ceil_div, }; -use crate::vm_m5::{ - bootloader_state::BootloaderState, - oracles::OracleWithHistory, - storage::Storage, - transaction_data::TransactionData, - utils::{ - code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, +use crate::{ + vm_latest::L1BatchEnv, + vm_m5::{ + bootloader_state::BootloaderState, + oracles::OracleWithHistory, + storage::Storage, + transaction_data::TransactionData, + utils::{ + code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, + }, + vm_instance::{MultiVMSubversion, VmInstance, ZkSyncVmState}, + OracleTools, }, - vm_instance::{MultiVMSubversion, VmInstance, ZkSyncVmState}, - OracleTools, }; // TODO (SMA-1703): move these to config and make them programmatically generable. @@ -73,15 +77,19 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible to a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -91,10 +99,22 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) } +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + impl From for DerivedBlockContext { fn from(context: BlockContext) -> Self { - let base_fee = - derive_base_fee_and_gas_per_pubdata(context.l1_gas_price, context.fair_l2_gas_price).0; + let base_fee = derive_base_fee_and_gas_per_pubdata(L1PeggedBatchFeeModelInput { + l1_gas_price: context.l1_gas_price, + fair_l2_gas_price: context.fair_l2_gas_price, + }) + .0; DerivedBlockContext { context, base_fee } } diff --git a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs index 02dfc0f313a4..a4735c5a6175 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs @@ -12,8 +12,9 @@ use zk_evm_1_3_1::{ }; use zksync_contracts::BaseSystemContracts; use zksync_types::{ - zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, Address, Transaction, BOOTLOADER_ADDRESS, - L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + fee_model::L1PeggedBatchFeeModelInput, zkevm_test_harness::INITIAL_MONOTONIC_CYCLE_COUNTER, + Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_GAS_PER_PUBDATA_BYTE, + MAX_NEW_FACTORY_DEPS, U256, }; use zksync_utils::{ address_to_u256, @@ -22,16 +23,19 @@ use zksync_utils::{ misc::ceil_div, }; -use crate::vm_m6::{ - bootloader_state::BootloaderState, - history_recorder::HistoryMode, - storage::Storage, - transaction_data::{TransactionData, L1_TX_TYPE}, - utils::{ - code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, +use crate::{ + vm_latest::L1BatchEnv, + vm_m6::{ + bootloader_state::BootloaderState, + history_recorder::HistoryMode, + storage::Storage, + transaction_data::{TransactionData, L1_TX_TYPE}, + utils::{ + code_page_candidate_from_base, heap_page_from_base, BLOCK_GAS_LIMIT, INITIAL_BASE_PAGE, + }, + vm_instance::{MultiVMSubversion, ZkSyncVmState}, + OracleTools, VmInstance, }, - vm_instance::{MultiVMSubversion, ZkSyncVmState}, - OracleTools, VmInstance, }; // TODO (SMA-1703): move these to config and make them programmatically generable. @@ -57,7 +61,11 @@ pub struct BlockContext { impl BlockContext { pub fn block_gas_price_per_pubdata(&self) -> u64 { - derive_base_fee_and_gas_per_pubdata(self.l1_gas_price, self.fair_l2_gas_price).1 + derive_base_fee_and_gas_per_pubdata(L1PeggedBatchFeeModelInput { + l1_gas_price: self.l1_gas_price, + fair_l2_gas_price: self.fair_l2_gas_price, + }) + .1 } } @@ -82,15 +90,19 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { } pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -100,10 +112,22 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( ) } +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + impl From for DerivedBlockContext { fn from(context: BlockContext) -> Self { - let base_fee = - derive_base_fee_and_gas_per_pubdata(context.l1_gas_price, context.fair_l2_gas_price).0; + let base_fee = derive_base_fee_and_gas_per_pubdata(L1PeggedBatchFeeModelInput { + l1_gas_price: context.l1_gas_price, + fair_l2_gas_price: context.fair_l2_gas_price, + }) + .0; DerivedBlockContext { context, base_fee } } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/tx.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/tx.rs index a786e9b0ad7e..6dc4772d095a 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/tx.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/tx.rs @@ -7,6 +7,7 @@ use crate::{ constants::BOOTLOADER_HEAP_PAGE, implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, types::internals::TransactionData, + utils::fee::get_batch_gas_per_pubdata, vm::Vm, }, HistoryMode, @@ -38,8 +39,7 @@ impl Vm { .decommittment_processor .populate(codes_for_decommiter, timestamp); - let trusted_ergs_limit = - tx.trusted_ergs_limit(self.batch_env.block_gas_price_per_pubdata()); + let trusted_ergs_limit = tx.trusted_ergs_limit(get_batch_gas_per_pubdata(&self.batch_env)); let memory = self.bootloader_state.push_tx( tx, @@ -61,7 +61,7 @@ impl Vm { with_compression: bool, ) { let tx: TransactionData = tx.into(); - let block_gas_per_pubdata_byte = self.batch_env.block_gas_price_per_pubdata(); + let block_gas_per_pubdata_byte = get_batch_gas_per_pubdata(&self.batch_env); let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); self.push_raw_transaction(tx, overhead, 0, with_compression); } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs index b66918b476b1..20e799a3883d 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs @@ -31,6 +31,7 @@ use crate::{ }, }, types::internals::ZkSyncVmState, + utils::fee::get_batch_base_fee, }, }; @@ -111,13 +112,14 @@ impl RefundsTracer { }); // For now, bootloader charges only for base fee. - let effective_gas_price = self.l1_batch.base_fee(); + let effective_gas_price = get_batch_base_fee(&self.l1_batch); let bootloader_eth_price_per_pubdata_byte = U256::from(effective_gas_price) * U256::from(current_ergs_per_pubdata_byte); - let fair_eth_price_per_pubdata_byte = - U256::from(eth_price_per_pubdata_byte(self.l1_batch.l1_gas_price)); + let fair_eth_price_per_pubdata_byte = U256::from(eth_price_per_pubdata_byte( + self.l1_batch.fee_input.l1_gas_price(), + )); // For now, L1 originated transactions are allowed to pay less than fair fee per pubdata, // so we should take it into account. @@ -127,7 +129,7 @@ impl RefundsTracer { ); let fair_fee_eth = U256::from(gas_spent_on_computation) - * U256::from(self.l1_batch.fair_l2_gas_price) + * U256::from(self.l1_batch.fee_input.fair_l2_gas_price()) + U256::from(pubdata_published) * eth_price_per_pubdata_byte_for_calculation; let pre_paid_eth = U256::from(tx_gas_limit) * U256::from(effective_gas_price); let refund_eth = pre_paid_eth.checked_sub(fair_fee_eth).unwrap_or_else(|| { diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/l1_batch.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/l1_batch.rs index 6f16e95f8d77..b449165be348 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/l1_batch.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/l1_batch.rs @@ -1,7 +1,7 @@ use zksync_types::U256; use zksync_utils::{address_to_u256, h256_to_u256}; -use crate::interface::L1BatchEnv; +use crate::{interface::L1BatchEnv, vm_refunds_enhancement::utils::fee::get_batch_base_fee}; const OPERATOR_ADDRESS_SLOT: usize = 0; const PREV_BLOCK_HASH_SLOT: usize = 1; @@ -19,6 +19,8 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 .map(|prev_block_hash| (h256_to_u256(prev_block_hash), U256::one())) .unwrap_or_default(); + let fee_input = l1_batch.fee_input.into_l1_pegged(); + vec![ ( OPERATOR_ADDRESS_SLOT, @@ -27,12 +29,15 @@ pub(crate) fn bootloader_initial_memory(l1_batch: &L1BatchEnv) -> Vec<(usize, U2 (PREV_BLOCK_HASH_SLOT, prev_block_hash), (NEW_BLOCK_TIMESTAMP_SLOT, U256::from(l1_batch.timestamp)), (NEW_BLOCK_NUMBER_SLOT, U256::from(l1_batch.number.0)), - (L1_GAS_PRICE_SLOT, U256::from(l1_batch.l1_gas_price)), + (L1_GAS_PRICE_SLOT, U256::from(fee_input.l1_gas_price)), ( FAIR_L2_GAS_PRICE_SLOT, - U256::from(l1_batch.fair_l2_gas_price), + U256::from(fee_input.fair_l2_gas_price), + ), + ( + EXPECTED_BASE_FEE_SLOT, + U256::from(get_batch_base_fee(l1_batch)), ), - (EXPECTED_BASE_FEE_SLOT, U256::from(l1_batch.base_fee())), (SHOULD_SET_NEW_BLOCK_SLOT, should_set_new_block), ] } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs index e48f4e8c9654..773ae3cb4096 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/utils/fee.rs @@ -1,8 +1,11 @@ //! Utility functions for vm use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; +use zksync_types::fee_model::L1PeggedBatchFeeModelInput; use zksync_utils::ceil_div; -use crate::vm_refunds_enhancement::old_vm::utils::eth_price_per_pubdata_byte; +use crate::{ + vm_latest::L1BatchEnv, vm_refunds_enhancement::old_vm::utils::eth_price_per_pubdata_byte, +}; /// Calculates the amount of gas required to publish one byte of pubdata pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { @@ -13,15 +16,18 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { /// Calculates the base fee and gas per pubdata for the given L1 gas price. pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -30,3 +36,16 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( base_fee_to_gas_per_pubdata(l1_gas_price, base_fee), ) } + +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + +pub(crate) fn get_batch_gas_per_pubdata(l1_batch_env: &L1BatchEnv) -> u64 { + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()).1 +} diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/tx.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/tx.rs index 72a7dbc65de0..0f4705a633f2 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/tx.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/tx.rs @@ -7,6 +7,7 @@ use crate::{ constants::BOOTLOADER_HEAP_PAGE, implementation::bytecode::{bytecode_to_factory_dep, compress_bytecodes}, types::internals::TransactionData, + utils::fee::get_batch_gas_per_pubdata, vm::Vm, }, HistoryMode, @@ -38,8 +39,7 @@ impl Vm { .decommittment_processor .populate(codes_for_decommiter, timestamp); - let trusted_ergs_limit = - tx.trusted_ergs_limit(self.batch_env.block_gas_price_per_pubdata()); + let trusted_ergs_limit = tx.trusted_ergs_limit(get_batch_gas_per_pubdata(&self.batch_env)); let memory = self.bootloader_state.push_tx( tx, @@ -61,7 +61,7 @@ impl Vm { with_compression: bool, ) { let tx: TransactionData = tx.into(); - let block_gas_per_pubdata_byte = self.batch_env.block_gas_price_per_pubdata(); + let block_gas_per_pubdata_byte = get_batch_gas_per_pubdata(&self.batch_env); let overhead = tx.overhead_gas(block_gas_per_pubdata_byte as u32); self.push_raw_transaction(tx, overhead, 0, with_compression); } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs index 9cdf30da9ce1..106a431edbaf 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs @@ -32,6 +32,7 @@ use crate::{ }, }, types::internals::ZkSyncVmState, + utils::fee::get_batch_base_fee, }, }; @@ -109,13 +110,14 @@ impl RefundsTracer { }); // For now, bootloader charges only for base fee. - let effective_gas_price = self.l1_batch.base_fee(); + let effective_gas_price = get_batch_base_fee(&self.l1_batch); let bootloader_eth_price_per_pubdata_byte = U256::from(effective_gas_price) * U256::from(current_ergs_per_pubdata_byte); - let fair_eth_price_per_pubdata_byte = - U256::from(eth_price_per_pubdata_byte(self.l1_batch.l1_gas_price)); + let fair_eth_price_per_pubdata_byte = U256::from(eth_price_per_pubdata_byte( + self.l1_batch.fee_input.l1_gas_price(), + )); // For now, L1 originated transactions are allowed to pay less than fair fee per pubdata, // so we should take it into account. @@ -125,7 +127,7 @@ impl RefundsTracer { ); let fair_fee_eth = U256::from(gas_spent_on_computation) - * U256::from(self.l1_batch.fair_l2_gas_price) + * U256::from(self.l1_batch.fee_input.fair_l2_gas_price()) + U256::from(pubdata_published) * eth_price_per_pubdata_byte_for_calculation; let pre_paid_eth = U256::from(tx_gas_limit) * U256::from(effective_gas_price); let refund_eth = pre_paid_eth.checked_sub(fair_fee_eth).unwrap_or_else(|| { diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/types/l1_batch_env.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/types/l1_batch_env.rs index 0e43863b1965..f86d8749c9ed 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/types/l1_batch_env.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/types/l1_batch_env.rs @@ -1,7 +1,7 @@ use zksync_types::U256; use zksync_utils::{address_to_u256, h256_to_u256}; -use crate::interface::L1BatchEnv; +use crate::{interface::L1BatchEnv, vm_virtual_blocks::utils::fee::get_batch_base_fee}; const OPERATOR_ADDRESS_SLOT: usize = 0; const PREV_BLOCK_HASH_SLOT: usize = 1; @@ -27,12 +27,18 @@ pub(crate) fn bootloader_initial_memory(l1_batch_env: &L1BatchEnv) -> Vec<(usize (PREV_BLOCK_HASH_SLOT, prev_block_hash), (NEW_BLOCK_TIMESTAMP_SLOT, U256::from(l1_batch_env.timestamp)), (NEW_BLOCK_NUMBER_SLOT, U256::from(l1_batch_env.number.0)), - (L1_GAS_PRICE_SLOT, U256::from(l1_batch_env.l1_gas_price)), + ( + L1_GAS_PRICE_SLOT, + U256::from(l1_batch_env.fee_input.l1_gas_price()), + ), ( FAIR_L2_GAS_PRICE_SLOT, - U256::from(l1_batch_env.fair_l2_gas_price), + U256::from(l1_batch_env.fee_input.fair_l2_gas_price()), + ), + ( + EXPECTED_BASE_FEE_SLOT, + U256::from(get_batch_base_fee(l1_batch_env)), ), - (EXPECTED_BASE_FEE_SLOT, U256::from(l1_batch_env.base_fee())), (SHOULD_SET_NEW_BLOCK_SLOT, should_set_new_block), ] } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs index 64bf1462c777..5af37aba11eb 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/utils/fee.rs @@ -1,8 +1,9 @@ //! Utility functions for vm use zksync_system_constants::MAX_GAS_PER_PUBDATA_BYTE; +use zksync_types::fee_model::L1PeggedBatchFeeModelInput; use zksync_utils::ceil_div; -use crate::vm_virtual_blocks::old_vm::utils::eth_price_per_pubdata_byte; +use crate::{vm_latest::L1BatchEnv, vm_virtual_blocks::old_vm::utils::eth_price_per_pubdata_byte}; /// Calculates the amount of gas required to publish one byte of pubdata pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { @@ -13,15 +14,19 @@ pub fn base_fee_to_gas_per_pubdata(l1_gas_price: u64, base_fee: u64) -> u64 { /// Calculates the base fee and gas per pubdata for the given L1 gas price. pub(crate) fn derive_base_fee_and_gas_per_pubdata( - l1_gas_price: u64, - fair_gas_price: u64, + fee_input: L1PeggedBatchFeeModelInput, ) -> (u64, u64) { + let L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + } = fee_input; + let eth_price_per_pubdata_byte = eth_price_per_pubdata_byte(l1_gas_price); // The `baseFee` is set in such a way that it is always possible for a transaction to // publish enough public data while compensating us for it. let base_fee = std::cmp::max( - fair_gas_price, + fair_l2_gas_price, ceil_div(eth_price_per_pubdata_byte, MAX_GAS_PER_PUBDATA_BYTE), ); @@ -30,3 +35,16 @@ pub(crate) fn derive_base_fee_and_gas_per_pubdata( base_fee_to_gas_per_pubdata(l1_gas_price, base_fee), ) } + +pub(crate) fn get_batch_base_fee(l1_batch_env: &L1BatchEnv) -> u64 { + if let Some(base_fee) = l1_batch_env.enforced_base_fee { + return base_fee; + } + let (base_fee, _) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()); + base_fee +} + +pub(crate) fn get_batch_gas_per_pubdata(l1_batch_env: &L1BatchEnv) -> u64 { + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input.into_l1_pegged()).1 +} diff --git a/core/lib/state/src/test_utils.rs b/core/lib/state/src/test_utils.rs index 21c56678586a..d24571860272 100644 --- a/core/lib/state/src/test_utils.rs +++ b/core/lib/state/src/test_utils.rs @@ -73,8 +73,7 @@ pub(crate) async fn create_miniblock( l1_tx_count: 0, l2_tx_count: 0, base_fee_per_gas: 0, - l1_gas_price: 0, - l2_fair_gas_price: 0, + batch_fee_input: Default::default(), base_system_contracts_hashes: Default::default(), protocol_version: Some(Default::default()), virtual_blocks: 0, diff --git a/core/lib/types/src/block.rs b/core/lib/types/src/block.rs index 485f34196450..2de8abd2512f 100644 --- a/core/lib/types/src/block.rs +++ b/core/lib/types/src/block.rs @@ -7,6 +7,7 @@ use zksync_system_constants::SYSTEM_BLOCK_INFO_BLOCK_NUMBER_MULTIPLIER; use zksync_utils::concat_and_hash; use crate::{ + fee_model::BatchFeeInput, l2_to_l1_log::{SystemL2ToL1Log, UserL2ToL1Log}, priority_op_onchain_data::PriorityOpOnchainData, web3::signing::keccak256, @@ -69,7 +70,7 @@ pub struct L1BatchHeader { } /// Holder for the miniblock metadata that is not available from transactions themselves. -#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +#[derive(Debug, Clone, PartialEq)] pub struct MiniblockHeader { pub number: MiniblockNumber, pub timestamp: u64, @@ -78,8 +79,7 @@ pub struct MiniblockHeader { pub l2_tx_count: u16, pub base_fee_per_gas: u64, // Min wei per gas that txs in this miniblock need to have. - pub l1_gas_price: u64, // L1 gas price assumed in the corresponding batch - pub l2_fair_gas_price: u64, // L2 gas price assumed in the corresponding batch + pub batch_fee_input: BatchFeeInput, pub base_system_contracts_hashes: BaseSystemContractsHashes, pub protocol_version: Option, /// The maximal number of virtual blocks to be created in the miniblock. diff --git a/core/lib/types/src/fee_model.rs b/core/lib/types/src/fee_model.rs new file mode 100644 index 000000000000..f0403d9a71c5 --- /dev/null +++ b/core/lib/types/src/fee_model.rs @@ -0,0 +1,185 @@ +use serde::{Deserialize, Serialize}; +use zksync_system_constants::L1_GAS_PER_PUBDATA_BYTE; + +/// Fee input to be provided into the VM. It contains two options: +/// - `L1Pegged`: L1 gas price is provided to the VM, and the pubdata price is derived from it. Using this option is required for the +/// versions of Era prior to 1.4.1 integration. +/// - `PubdataIndependent`: L1 gas price and pubdata price are not necessarily dependent on one another. This options is more suitable for the +/// versions of Era after the 1.4.1 integration. It is expected that if a VM supports `PubdataIndependent` version, then it should also support `L1Pegged` version, but converting it into `PubdataIndependentBatchFeeModelInput` in-place. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum BatchFeeInput { + L1Pegged(L1PeggedBatchFeeModelInput), + PubdataIndependent(PubdataIndependentBatchFeeModelInput), +} + +impl BatchFeeInput { + // Sometimes for temporary usage or tests a "sensible" default, i.e. the one consisting of non-zero values is needed. + pub fn sensible_l1_pegged_default() -> Self { + Self::L1Pegged(L1PeggedBatchFeeModelInput { + l1_gas_price: 1_000_000_000, + fair_l2_gas_price: 100_000_000, + }) + } + + pub fn l1_pegged(l1_gas_price: u64, fair_l2_gas_price: u64) -> Self { + Self::L1Pegged(L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + }) + } +} + +impl Default for BatchFeeInput { + fn default() -> Self { + Self::L1Pegged(L1PeggedBatchFeeModelInput { + l1_gas_price: 0, + fair_l2_gas_price: 0, + }) + } +} + +impl BatchFeeInput { + pub fn into_l1_pegged(self) -> L1PeggedBatchFeeModelInput { + match self { + BatchFeeInput::L1Pegged(input) => input, + _ => panic!("Can not convert PubdataIndependentBatchFeeModelInput into L1PeggedBatchFeeModelInput"), + } + } + + pub fn fair_pubdata_price(&self) -> u64 { + match self { + BatchFeeInput::L1Pegged(input) => input.l1_gas_price * L1_GAS_PER_PUBDATA_BYTE as u64, + BatchFeeInput::PubdataIndependent(input) => input.fair_pubdata_price, + } + } + + pub fn fair_l2_gas_price(&self) -> u64 { + match self { + BatchFeeInput::L1Pegged(input) => input.fair_l2_gas_price, + BatchFeeInput::PubdataIndependent(input) => input.fair_l2_gas_price, + } + } + + pub fn l1_gas_price(&self) -> u64 { + match self { + BatchFeeInput::L1Pegged(input) => input.l1_gas_price, + BatchFeeInput::PubdataIndependent(input) => input.l1_gas_price, + } + } + + pub fn into_pubdata_independent(self) -> PubdataIndependentBatchFeeModelInput { + match self { + BatchFeeInput::PubdataIndependent(input) => input, + BatchFeeInput::L1Pegged(input) => PubdataIndependentBatchFeeModelInput { + fair_l2_gas_price: input.fair_l2_gas_price, + fair_pubdata_price: input.l1_gas_price * L1_GAS_PER_PUBDATA_BYTE as u64, + l1_gas_price: input.l1_gas_price, + }, + } + } +} + +/// Pubdata is only published via calldata and so its price is pegged to the L1 gas price. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct L1PeggedBatchFeeModelInput { + /// Fair L2 gas price to provide + pub fair_l2_gas_price: u64, + /// The L1 gas price to provide to the VM. + pub l1_gas_price: u64, +} + +/// Pubdata price may be independent from L1 gas price. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct PubdataIndependentBatchFeeModelInput { + /// Fair L2 gas price to provide + pub fair_l2_gas_price: u64, + /// Fair pubdata price to provide. + pub fair_pubdata_price: u64, + /// The L1 gas price to provide to the VM. Even if some of the VM versions may not use this value, it is still maintained for backward compatibility. + pub l1_gas_price: u64, +} + +/// The enum which represents the version of the fee model. It is used to determine which fee model should be used for the batch. +/// - `V1`, the first model that was used in zkSync Era. In this fee model, the pubdata price must be pegged to the L1 gas price. +/// Also, the fair L2 gas price is expected to only include the proving/computation price for the operator and not the costs that come from +/// processing the batch on L1. +/// - `V2`, the second model that was used in zkSync Era. There the pubdata price might be independent from the L1 gas price. Also, +/// The fair L2 gas price is expected to both the proving/computation price for the operator and the costs that come from +/// processing the batch on L1. +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub enum FeeModelConfig { + V1(FeeModelConfigV1), + V2(FeeModelConfigV2), +} + +/// Config params for the first version of the fee model. Here, the pubdata price is pegged to the L1 gas price and +/// neither fair L2 gas price nor the pubdata price include the overhead for closing the batch +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub struct FeeModelConfigV1 { + /// The minimal acceptable L2 gas price, i.e. the price that should include the cost of computation/proving as well + /// as potentially premium for congestion. + /// Unlike the `V2`, this price will be directly used as the `fair_l2_gas_price` in the bootloader. + pub minimal_l2_gas_price: u64, +} + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub struct FeeModelConfigV2 { + /// The minimal acceptable L2 gas price, i.e. the price that should include the cost of computation/proving as well + /// as potentially premium for congestion. + pub minimal_l2_gas_price: u64, + /// The constant that represents the possibility that a batch can be sealed because of overuse of computation resources. + /// It has range from 0 to 1. If it is 0, the compute will not depend on the cost for closing the batch. + /// If it is 1, the gas limit per batch will have to cover the entire cost of closing the batch. + pub compute_overhead_part: f64, + /// The constant that represents the possibility that a batch can be sealed because of overuse of pubdata. + /// It has range from 0 to 1. If it is 0, the pubdata will not depend on the cost for closing the batch. + /// If it is 1, the pubdata limit per batch will have to cover the entire cost of closing the batch. + pub pubdata_overhead_part: f64, + /// The constant amount of L1 gas that is used as the overhead for the batch. It includes the price for batch verification, etc. + pub batch_overhead_l1_gas: u64, + /// The maximum amount of gas that can be used by the batch. This value is derived from the circuits limitation per batch. + pub max_gas_per_batch: u64, + /// The maximum amount of pubdata that can be used by the batch. Note that if the calldata is used as pubdata, this variable should not exceed 128kb. + pub max_pubdata_per_batch: u64, +} + +impl Default for FeeModelConfig { + /// Config with all zeroes is not a valid config (since for instance having 0 max gas per batch may incur division by zero), + /// so we implement a sensible default config here. + fn default() -> Self { + Self::V1(FeeModelConfigV1 { + minimal_l2_gas_price: 100_000_000, + }) + } +} + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub struct FeeParamsV1 { + pub config: FeeModelConfigV1, + pub l1_gas_price: u64, +} + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub struct FeeParamsV2 { + pub config: FeeModelConfigV2, + pub l1_gas_price: u64, + pub l1_pubdata_price: u64, +} + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub enum FeeParams { + V1(FeeParamsV1), + V2(FeeParamsV2), +} + +impl FeeParams { + // Sometimes for temporary usage or tests a "sensible" default, i.e. the one consisting of non-zero values is needed. + pub fn sensible_v1_default() -> Self { + Self::V1(FeeParamsV1 { + config: FeeModelConfigV1 { + minimal_l2_gas_price: 100_000_000, + }, + l1_gas_price: 1_000_000_000, + }) + } +} diff --git a/core/lib/types/src/lib.rs b/core/lib/types/src/lib.rs index a7d7e71e0eff..6d9017e3310c 100644 --- a/core/lib/types/src/lib.rs +++ b/core/lib/types/src/lib.rs @@ -39,6 +39,7 @@ pub mod contract_verification_api; pub mod contracts; pub mod event; pub mod fee; +pub mod fee_model; pub mod l1; pub mod l2; pub mod l2_to_l1_log; diff --git a/core/lib/web3_decl/src/namespaces/zks.rs b/core/lib/web3_decl/src/namespaces/zks.rs index 1a8922dcb28b..a9eddaa03122 100644 --- a/core/lib/web3_decl/src/namespaces/zks.rs +++ b/core/lib/web3_decl/src/namespaces/zks.rs @@ -8,6 +8,7 @@ use zksync_types::{ TransactionDetails, }, fee::Fee, + fee_model::FeeParams, transaction_request::CallRequest, Address, L1BatchNumber, MiniblockNumber, H256, U256, U64, }; @@ -101,6 +102,9 @@ pub trait ZksNamespace { #[method(name = "getL1GasPrice")] async fn get_l1_gas_price(&self) -> RpcResult; + #[method(name = "getFeeParams")] + async fn get_fee_params(&self) -> RpcResult; + #[method(name = "getProtocolVersion")] async fn get_protocol_version( &self, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs index 33e8e8f8bfc7..ba40531875da 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/apply.rs @@ -10,7 +10,7 @@ use std::time::{Duration, Instant}; use multivm::{ interface::{L1BatchEnv, L2BlockEnv, SystemEnv, VmInterface}, - utils::adjust_l1_gas_price_for_tx, + utils::adjust_pubdata_price_for_tx, vm_latest::{constants::BLOCK_GAS_LIMIT, HistoryDisabled}, VmInstance, }; @@ -180,23 +180,21 @@ pub(super) fn apply_vm_in_sandbox( let TxSharedArgs { operator_account, - l1_gas_price, - fair_l2_gas_price, + fee_input, base_system_contracts, validation_computational_gas_limit, chain_id, .. } = shared_args; - let l1_gas_price = if adjust_pubdata_price { - adjust_l1_gas_price_for_tx( - l1_gas_price, - fair_l2_gas_price, + let fee_input = if adjust_pubdata_price { + adjust_pubdata_price_for_tx( + fee_input, tx.gas_per_pubdata_byte_limit(), protocol_version.into(), ) } else { - l1_gas_price + fee_input }; let system_env = SystemEnv { @@ -214,8 +212,7 @@ pub(super) fn apply_vm_in_sandbox( previous_batch_hash: None, number: vm_l1_batch_number, timestamp: l1_batch_timestamp, - l1_gas_price, - fair_l2_gas_price, + fee_input, fee_account: *operator_account.address(), enforced_base_fee: execution_args.enforced_base_fee, first_l2_block: next_l2_block_info, diff --git a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs b/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs index 8c62bd3f498d..900bb28886f1 100644 --- a/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs +++ b/core/lib/zksync_core/src/api_server/execution_sandbox/mod.rs @@ -4,7 +4,7 @@ use tokio::runtime::Handle; use zksync_dal::{ConnectionPool, SqlxError, StorageProcessor}; use zksync_state::{PostgresStorage, PostgresStorageCaches, ReadStorage, StorageView}; use zksync_system_constants::PUBLISH_BYTECODE_OVERHEAD; -use zksync_types::{api, AccountTreeId, L2ChainId, MiniblockNumber}; +use zksync_types::{api, fee_model::BatchFeeInput, AccountTreeId, L2ChainId, MiniblockNumber}; use zksync_utils::bytecode::{compress_bytecode, hash_bytecode}; use self::vm_metrics::SandboxStage; @@ -201,8 +201,7 @@ pub(super) async fn get_pubdata_for_factory_deps( #[derive(Debug, Clone)] pub(crate) struct TxSharedArgs { pub operator_account: AccountTreeId, - pub l1_gas_price: u64, - pub fair_l2_gas_price: u64, + pub fee_input: BatchFeeInput, pub base_system_contracts: MultiVMBaseSystemContracts, pub caches: PostgresStorageCaches, pub validation_computational_gas_limit: u32, 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 c2fc8beb5761..efefb021bb2f 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 @@ -4,7 +4,7 @@ use std::{cmp, sync::Arc, time::Instant}; use multivm::{ interface::VmExecutionResultAndLogs, - utils::{adjust_l1_gas_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead}, + utils::{adjust_pubdata_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead}, vm_latest::constants::{BLOCK_GAS_LIMIT, MAX_PUBDATA_PER_BLOCK}, }; use zksync_config::configs::{api::Web3JsonRpcConfig, chain::StateKeeperConfig}; @@ -13,6 +13,7 @@ use zksync_dal::{transactions_dal::L2TxSubmissionResult, ConnectionPool}; use zksync_state::PostgresStorageCaches; use zksync_types::{ fee::{Fee, TransactionExecutionMetrics}, + fee_model::BatchFeeInput, get_code_key, get_intrinsic_constants, l2::{error::TxCheckError::TxDuplication, L2Tx}, utils::storage_key_for_eth_balance, @@ -32,7 +33,7 @@ use crate::{ }, tx_sender::result::ApiCallResult, }, - l1_gas_price::L1GasPriceProvider, + fee_model::BatchFeeModelInputProvider, metrics::{TxStage, APP_METRICS}, state_keeper::seal_criteria::{ConditionalSealer, NoopSealer, SealData}, }; @@ -166,7 +167,7 @@ impl TxSenderBuilder { pub async fn build( self, - l1_gas_price_source: Arc, + batch_fee_input_provider: Arc, vm_concurrency_limiter: Arc, api_contracts: ApiContracts, storage_caches: PostgresStorageCaches, @@ -183,7 +184,7 @@ impl TxSenderBuilder { sender_config: self.config, master_connection_pool: self.master_connection_pool, replica_connection_pool: self.replica_connection_pool, - l1_gas_price_source, + batch_fee_input_provider, api_contracts, proxy: self.proxy, vm_concurrency_limiter, @@ -234,7 +235,7 @@ pub struct TxSenderInner { pub master_connection_pool: Option, pub replica_connection_pool: ConnectionPool, // Used to keep track of gas prices for the fee ticker. - pub l1_gas_price_source: Arc, + pub batch_fee_input_provider: Arc, pub(super) api_contracts: ApiContracts, /// Optional transaction proxy to be used for transaction submission. pub(super) proxy: Option, @@ -381,8 +382,7 @@ impl TxSender { fn shared_args(&self) -> TxSharedArgs { TxSharedArgs { operator_account: AccountTreeId::new(self.0.sender_config.fee_account_addr), - l1_gas_price: self.0.l1_gas_price_source.estimate_effective_gas_price(), - fair_l2_gas_price: self.0.sender_config.fair_l2_gas_price, + fee_input: self.0.batch_fee_input_provider.get_batch_fee_input(), base_system_contracts: self.0.api_contracts.eth_call.clone(), caches: self.storage_caches(), validation_computational_gas_limit: self @@ -554,7 +554,7 @@ impl TxSender { mut tx: Transaction, gas_per_pubdata_byte: u64, tx_gas_limit: u32, - l1_gas_price: u64, + fee_input: BatchFeeInput, block_args: BlockArgs, base_fee: u64, vm_version: VmVersion, @@ -588,7 +588,7 @@ impl TxSender { } } - let shared_args = self.shared_args_for_gas_estimate(l1_gas_price); + let shared_args = self.shared_args_for_gas_estimate(fee_input); let vm_execution_cache_misses_limit = self.0.sender_config.vm_execution_cache_misses_limit; let execution_args = TxExecutionArgs::for_gas_estimate(vm_execution_cache_misses_limit, &tx, base_fee); @@ -607,12 +607,11 @@ impl TxSender { (exec_result, tx_metrics) } - fn shared_args_for_gas_estimate(&self, l1_gas_price: u64) -> TxSharedArgs { + fn shared_args_for_gas_estimate(&self, fee_input: BatchFeeInput) -> TxSharedArgs { let config = &self.0.sender_config; TxSharedArgs { operator_account: AccountTreeId::new(config.fee_account_addr), - l1_gas_price, - fair_l2_gas_price: config.fair_l2_gas_price, + fee_input, // We want to bypass the computation gas limit check for gas estimation validation_computational_gas_limit: BLOCK_GAS_LIMIT, base_system_contracts: self.0.api_contracts.estimate_gas.clone(), @@ -645,26 +644,21 @@ impl TxSender { .unwrap_or(ProtocolVersionId::last_pre_boojum()); drop(connection); - let l1_gas_price = { - let effective_gas_price = self.0.l1_gas_price_source.estimate_effective_gas_price(); - let current_l1_gas_price = - ((effective_gas_price as f64) * self.0.sender_config.gas_price_scale_factor) as u64; - - // In order for execution to pass smoothly, we need to ensure that block's required `gasPerPubdata` will be - // <= to the one in the transaction itself. - adjust_l1_gas_price_for_tx( - current_l1_gas_price, - self.0.sender_config.fair_l2_gas_price, + let fee_input = { + // For now, both L1 gas price and pubdata price are scaled with the same coefficient + let fee_input = self.0.batch_fee_input_provider.get_batch_fee_input_scaled( + self.0.sender_config.gas_price_scale_factor, + self.0.sender_config.gas_price_scale_factor, + ); + adjust_pubdata_price_for_tx( + fee_input, tx.gas_per_pubdata_byte_limit(), protocol_version.into(), ) }; - let (base_fee, gas_per_pubdata_byte) = derive_base_fee_and_gas_per_pubdata( - l1_gas_price, - self.0.sender_config.fair_l2_gas_price, - protocol_version.into(), - ); + let (base_fee, gas_per_pubdata_byte) = + derive_base_fee_and_gas_per_pubdata(fee_input, protocol_version.into()); match &mut tx.common_data { ExecuteTransactionCommon::L2(common_data) => { common_data.fee.max_fee_per_gas = base_fee.into(); @@ -770,7 +764,7 @@ impl TxSender { tx.clone(), gas_per_pubdata_byte, try_gas_limit, - l1_gas_price, + fee_input, block_args, base_fee, protocol_version.into(), @@ -809,7 +803,7 @@ impl TxSender { tx.clone(), gas_per_pubdata_byte, suggested_gas_limit, - l1_gas_price, + fee_input, block_args, base_fee, protocol_version.into(), @@ -869,9 +863,6 @@ impl TxSender { } pub async fn gas_price(&self) -> u64 { - let gas_price = self.0.l1_gas_price_source.estimate_effective_gas_price(); - let l1_gas_price = (gas_price as f64 * self.0.sender_config.gas_price_scale_factor).round(); - let mut connection = self .0 .replica_connection_pool @@ -889,8 +880,11 @@ impl TxSender { drop(connection); let (base_fee, _) = derive_base_fee_and_gas_per_pubdata( - l1_gas_price as u64, - self.0.sender_config.fair_l2_gas_price, + // For now, both the L1 gas price and the L1 pubdata price are scaled with the same coefficient + self.0.batch_fee_input_provider.get_batch_fee_input_scaled( + self.0.sender_config.gas_price_scale_factor, + self.0.sender_config.gas_price_scale_factor, + ), protocol_version.into(), ); base_fee diff --git a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs b/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs index 1f70f85f265f..e7500968f42e 100644 --- a/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs +++ b/core/lib/zksync_core/src/api_server/web3/backend_jsonrpsee/namespaces/zks.rs @@ -7,6 +7,7 @@ use zksync_types::{ TransactionDetails, }, fee::Fee, + fee_model::FeeParams, transaction_request::CallRequest, Address, L1BatchNumber, MiniblockNumber, H256, U256, U64, }; @@ -142,6 +143,10 @@ impl ZksNamespaceServer for ZksNamespace { Ok(self.get_l1_gas_price_impl()) } + async fn get_fee_params(&self) -> RpcResult { + Ok(self.get_fee_params_impl()) + } + async fn get_protocol_version( &self, version_id: Option, diff --git a/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs b/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs index 20be1340bada..46640d63648e 100644 --- a/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs +++ b/core/lib/zksync_core/src/api_server/web3/namespaces/debug.rs @@ -6,6 +6,7 @@ use zksync_dal::ConnectionPool; use zksync_state::PostgresStorageCaches; use zksync_types::{ api::{BlockId, BlockNumber, DebugCall, ResultDebugCall, TracerConfig}, + fee_model::BatchFeeInput, l2::L2Tx, transaction_request::CallRequest, vm_trace::Call, @@ -205,8 +206,7 @@ impl DebugNamespace { fn shared_args(&self) -> TxSharedArgs { TxSharedArgs { operator_account: AccountTreeId::default(), - l1_gas_price: 100_000, - fair_l2_gas_price: self.fair_l2_gas_price, + fee_input: BatchFeeInput::l1_pegged(100_000, self.fair_l2_gas_price), base_system_contracts: self.api_contracts.eth_call.clone(), caches: self.storage_caches.clone(), validation_computational_gas_limit: BLOCK_GAS_LIMIT, 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 5017d81c2954..b0c966679b64 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 @@ -9,6 +9,7 @@ use zksync_types::{ ProtocolVersion, StorageProof, TransactionDetails, }, fee::Fee, + fee_model::FeeParams, l1::L1Tx, l2::L2Tx, l2_to_l1_log::L2ToL1Log, @@ -582,13 +583,31 @@ impl ZksNamespace { .state .tx_sender .0 - .l1_gas_price_source - .estimate_effective_gas_price(); + .batch_fee_input_provider + .get_batch_fee_input() + .l1_gas_price(); method_latency.observe(); gas_price.into() } + #[tracing::instrument(skip(self))] + pub fn get_fee_params_impl(&self) -> FeeParams { + const METHOD_NAME: &str = "get_fee_params"; + + let method_latency = API_METRICS.start_call(METHOD_NAME); + let fee_model_params = self + .state + .tx_sender + .0 + .batch_fee_input_provider + .get_fee_model_params(); + + method_latency.observe(); + + fee_model_params + } + #[tracing::instrument(skip(self))] pub async fn get_protocol_version_impl( &self, diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs index 0c46064637c2..92252f35bb76 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs @@ -11,6 +11,7 @@ use zksync_config::configs::{ use zksync_dal::{transactions_dal::L2TxSubmissionResult, ConnectionPool}; use zksync_health_check::CheckHealth; use zksync_state::PostgresStorageCaches; +use zksync_system_constants::L1_GAS_PER_PUBDATA_BYTE; use zksync_types::{ block::MiniblockHeader, fee::TransactionExecutionMetrics, tx::IncludedTxLocation, Address, L1BatchNumber, VmEvent, H256, U64, @@ -43,6 +44,10 @@ impl L1GasPriceProvider for MockL1GasPriceProvider { fn estimate_effective_gas_price(&self) -> u64 { self.0 } + + fn estimate_effective_pubdata_price(&self) -> u64 { + self.0 * L1_GAS_PER_PUBDATA_BYTE as u64 + } } impl ApiServerHandles { diff --git a/core/lib/zksync_core/src/fee_model.rs b/core/lib/zksync_core/src/fee_model.rs new file mode 100644 index 000000000000..9798beffd80f --- /dev/null +++ b/core/lib/zksync_core/src/fee_model.rs @@ -0,0 +1,386 @@ +use std::{fmt, sync::Arc}; + +use zksync_types::{ + fee_model::{ + BatchFeeInput, FeeModelConfig, FeeModelConfigV2, FeeParams, FeeParamsV1, FeeParamsV2, + L1PeggedBatchFeeModelInput, PubdataIndependentBatchFeeModelInput, + }, + U256, +}; +use zksync_utils::ceil_div_u256; + +use crate::l1_gas_price::L1GasPriceProvider; + +/// Trait responsible for providing fee info for a batch +pub trait BatchFeeModelInputProvider: fmt::Debug + 'static + Send + Sync { + /// Returns the batch fee with scaling applied. This may be used to account for the fact that the L1 gas and pubdata prices may fluctuate, esp. + /// in API methods that should return values that are valid for some period of time after the estimation was done. + fn get_batch_fee_input_scaled( + &self, + l1_gas_price_scale_factor: f64, + l1_pubdata_price_scale_factor: f64, + ) -> BatchFeeInput { + let params = self.get_fee_model_params(); + + match params { + FeeParams::V1(params) => BatchFeeInput::L1Pegged(compute_batch_fee_model_input_v1( + params, + l1_gas_price_scale_factor, + )), + FeeParams::V2(params) => { + BatchFeeInput::PubdataIndependent(compute_batch_fee_model_input_v2( + params, + l1_gas_price_scale_factor, + l1_pubdata_price_scale_factor, + )) + } + } + } + + /// Returns the batch fee input as-is, i.e. without any scaling for the L1 gas and pubdata prices. + fn get_batch_fee_input(&self) -> BatchFeeInput { + self.get_batch_fee_input_scaled(1.0, 1.0) + } + + /// Returns the fee model parameters. + fn get_fee_model_params(&self) -> FeeParams; +} + +/// The struct that represents the batch fee input provider to be used in the main node of the server, i.e. +/// it explicitly gets the L1 gas price from the provider and uses it to calculate the batch fee input instead of getting +/// it from other node. +#[derive(Debug)] +pub(crate) struct MainNodeFeeInputProvider { + provider: Arc, + config: FeeModelConfig, +} + +impl BatchFeeModelInputProvider for MainNodeFeeInputProvider { + fn get_fee_model_params(&self) -> FeeParams { + match self.config { + FeeModelConfig::V1(config) => FeeParams::V1(FeeParamsV1 { + config, + l1_gas_price: self.provider.estimate_effective_gas_price(), + }), + FeeModelConfig::V2(config) => FeeParams::V2(FeeParamsV2 { + config, + l1_gas_price: self.provider.estimate_effective_gas_price(), + l1_pubdata_price: self.provider.estimate_effective_pubdata_price(), + }), + } + } +} + +impl MainNodeFeeInputProvider { + pub(crate) fn new(provider: Arc, config: FeeModelConfig) -> Self { + Self { provider, config } + } +} + +/// Calculates the batch fee input based on the main node parameters. +/// This function uses the `V1` fee model, i.e. where the pubdata price does not include the proving costs. +fn compute_batch_fee_model_input_v1( + params: FeeParamsV1, + l1_gas_price_scale_factor: f64, +) -> L1PeggedBatchFeeModelInput { + let l1_gas_price = (params.l1_gas_price as f64 * l1_gas_price_scale_factor) as u64; + + L1PeggedBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price: params.config.minimal_l2_gas_price, + } +} + +/// Calculates the batch fee input based on the main node parameters. +/// This function uses the `V2` fee model, i.e. where the pubdata price does not include the proving costs. +fn compute_batch_fee_model_input_v2( + params: FeeParamsV2, + l1_gas_price_scale_factor: f64, + l1_pubdata_price_scale_factor: f64, +) -> PubdataIndependentBatchFeeModelInput { + let FeeParamsV2 { + config, + l1_gas_price, + l1_pubdata_price, + } = params; + + let FeeModelConfigV2 { + minimal_l2_gas_price, + compute_overhead_part, + pubdata_overhead_part, + batch_overhead_l1_gas, + max_gas_per_batch, + max_pubdata_per_batch, + } = config; + + // Firstly, we scale the gas price and pubdata price in case it is needed. + let l1_gas_price = (l1_gas_price as f64 * l1_gas_price_scale_factor) as u64; + let l1_pubdata_price = (l1_pubdata_price as f64 * l1_pubdata_price_scale_factor) as u64; + + // While the final results of the calculations are not expected to have any overflows, the intermediate computations + // might, so we use U256 for them. + let l1_batch_overhead_wei = U256::from(l1_gas_price) * U256::from(batch_overhead_l1_gas); + + let fair_l2_gas_price = { + // Firstly, we calculate which part of the overall overhead overhead each unit of L2 gas should cover. + let l1_batch_overhead_per_gas = + ceil_div_u256(l1_batch_overhead_wei, U256::from(max_gas_per_batch)); + + // Then, we multiply by the `compute_overhead_part` to get the overhead for the computation for each gas. + // Also, this means that if we almost never close batches because of compute, the `compute_overhead_part` should be zero and so + // it is possible that the computation costs include for no overhead. + let gas_overhead_wei = + (l1_batch_overhead_per_gas.as_u64() as f64 * compute_overhead_part) as u64; + + // We sum up the minimal L2 gas price (i.e. the raw prover/compute cost of a single L2 gas) and the overhead for batch being closed. + minimal_l2_gas_price + gas_overhead_wei + }; + + let fair_pubdata_price = { + // Firstly, we calculate which part of the overall overhead overhead each pubdata byte should cover. + let l1_batch_overhead_per_pubdata = + ceil_div_u256(l1_batch_overhead_wei, U256::from(max_pubdata_per_batch)); + + // Then, we multiply by the `pubdata_overhead_part` to get the overhead for each pubdata byte. + // Also, this means that if we almost never close batches because of pubdata, the `pubdata_overhead_part` should be zero and so + // it is possible that the pubdata costs include no overhead. + let pubdata_overhead_wei = + (l1_batch_overhead_per_pubdata.as_u64() as f64 * pubdata_overhead_part) as u64; + + // We sum up the raw L1 pubdata price (i.e. the expected price of publishing a single pubdata byte) and the overhead for batch being closed. + l1_pubdata_price + pubdata_overhead_wei + }; + + PubdataIndependentBatchFeeModelInput { + l1_gas_price, + fair_l2_gas_price, + fair_pubdata_price, + } +} + +#[cfg(test)] +mod tests { + use super::*; + + // To test that overflow never happens, we'll use giant L1 gas price, i.e. + // almost realistic very large value of 100k gwei. Since it is so large, we'll also + // use it for the L1 pubdata price. + const GIANT_L1_GAS_PRICE: u64 = 100_000_000_000_000; + + // As a small small L2 gas price we'll use the value of 1 wei. + const SMALL_L1_GAS_PRICE: u64 = 1; + + #[test] + fn test_compute_batch_fee_model_input_v2_giant_numbers() { + let config = FeeModelConfigV2 { + minimal_l2_gas_price: GIANT_L1_GAS_PRICE, + // We generally don't expect those values to be larger than 1. Still, in theory the operator + // may need to set higher values in extreme cases. + compute_overhead_part: 5.0, + pubdata_overhead_part: 5.0, + // The batch overhead would likely never grow beyond that + batch_overhead_l1_gas: 1_000_000, + // Let's imagine that for some reason the limit is relatively small + max_gas_per_batch: 50_000_000, + // The pubdata will likely never go below that + max_pubdata_per_batch: 100_000, + }; + + let params = FeeParamsV2 { + config, + l1_gas_price: GIANT_L1_GAS_PRICE, + l1_pubdata_price: GIANT_L1_GAS_PRICE, + }; + + // We'll use scale factor of 3.0 + let input = compute_batch_fee_model_input_v2(params, 3.0, 3.0); + + assert_eq!(input.l1_gas_price, GIANT_L1_GAS_PRICE * 3); + assert_eq!(input.fair_l2_gas_price, 130_000_000_000_000); + assert_eq!(input.fair_pubdata_price, 15_300_000_000_000_000); + } + + #[test] + fn test_compute_batch_fee_model_input_v2_small_numbers() { + // Here we assume that the operator wants to make the lives of users as cheap as possible. + let config = FeeModelConfigV2 { + minimal_l2_gas_price: SMALL_L1_GAS_PRICE, + compute_overhead_part: 0.0, + pubdata_overhead_part: 0.0, + batch_overhead_l1_gas: 0, + max_gas_per_batch: 50_000_000, + max_pubdata_per_batch: 100_000, + }; + + let params = FeeParamsV2 { + config, + l1_gas_price: SMALL_L1_GAS_PRICE, + l1_pubdata_price: SMALL_L1_GAS_PRICE, + }; + + let input = compute_batch_fee_model_input_v2(params, 1.0, 1.0); + + assert_eq!(input.l1_gas_price, SMALL_L1_GAS_PRICE); + assert_eq!(input.fair_l2_gas_price, SMALL_L1_GAS_PRICE); + assert_eq!(input.fair_pubdata_price, SMALL_L1_GAS_PRICE); + } + + #[test] + fn test_compute_batch_fee_model_input_v2_only_pubdata_overhead() { + // Here we use sensible config, but when only pubdata is used to close the batch + let config = FeeModelConfigV2 { + minimal_l2_gas_price: 100_000_000_000, + compute_overhead_part: 0.0, + pubdata_overhead_part: 1.0, + batch_overhead_l1_gas: 700_000, + max_gas_per_batch: 500_000_000, + max_pubdata_per_batch: 100_000, + }; + + let params = FeeParamsV2 { + config, + l1_gas_price: GIANT_L1_GAS_PRICE, + l1_pubdata_price: GIANT_L1_GAS_PRICE, + }; + + let input = compute_batch_fee_model_input_v2(params, 1.0, 1.0); + assert_eq!(input.l1_gas_price, GIANT_L1_GAS_PRICE); + // The fair L2 gas price is identical to the minimal one. + assert_eq!(input.fair_l2_gas_price, 100_000_000_000); + // The fair pubdata price is the minimal one plus the overhead. + assert_eq!(input.fair_pubdata_price, 800_000_000_000_000); + } + + #[test] + fn test_compute_batch_fee_model_input_v2_only_compute_overhead() { + // Here we use sensible config, but when only compute is used to close the batch + let config = FeeModelConfigV2 { + minimal_l2_gas_price: 100_000_000_000, + compute_overhead_part: 1.0, + pubdata_overhead_part: 0.0, + batch_overhead_l1_gas: 700_000, + max_gas_per_batch: 500_000_000, + max_pubdata_per_batch: 100_000, + }; + + let params = FeeParamsV2 { + config, + l1_gas_price: GIANT_L1_GAS_PRICE, + l1_pubdata_price: GIANT_L1_GAS_PRICE, + }; + + let input = compute_batch_fee_model_input_v2(params, 1.0, 1.0); + assert_eq!(input.l1_gas_price, GIANT_L1_GAS_PRICE); + // The fair L2 gas price is identical to the minimal one, plus the overhead + assert_eq!(input.fair_l2_gas_price, 240_000_000_000); + // The fair pubdata price is equal to the original one. + assert_eq!(input.fair_pubdata_price, GIANT_L1_GAS_PRICE); + } + + #[test] + fn test_compute_batch_fee_model_input_v2_param_tweaking() { + // In this test we generally checking that each param behaves as expected + let base_config = FeeModelConfigV2 { + minimal_l2_gas_price: 100_000_000_000, + compute_overhead_part: 0.5, + pubdata_overhead_part: 0.5, + batch_overhead_l1_gas: 700_000, + max_gas_per_batch: 500_000_000, + max_pubdata_per_batch: 100_000, + }; + + let base_params = FeeParamsV2 { + config: base_config, + l1_gas_price: 1_000_000_000, + l1_pubdata_price: 1_000_000_000, + }; + + let base_input = compute_batch_fee_model_input_v2(base_params, 1.0, 1.0); + + let base_input_larger_l1_gas_price = compute_batch_fee_model_input_v2( + FeeParamsV2 { + l1_gas_price: base_params.l1_gas_price * 2, + ..base_params + }, + 1.0, + 1.0, + ); + let base_input_scaled_l1_gas_price = + compute_batch_fee_model_input_v2(base_params, 2.0, 1.0); + assert_eq!( + base_input_larger_l1_gas_price, base_input_scaled_l1_gas_price, + "Scaling has the correct effect for the L1 gas price" + ); + assert!( + base_input.fair_l2_gas_price < base_input_larger_l1_gas_price.fair_l2_gas_price, + "L1 gas price increase raises L2 gas price" + ); + assert!( + base_input.fair_pubdata_price < base_input_larger_l1_gas_price.fair_pubdata_price, + "L1 gas price increase raises pubdata price" + ); + + let base_input_larger_pubdata_price = compute_batch_fee_model_input_v2( + FeeParamsV2 { + l1_pubdata_price: base_params.l1_pubdata_price * 2, + ..base_params + }, + 1.0, + 1.0, + ); + let base_input_scaled_pubdata_price = + compute_batch_fee_model_input_v2(base_params, 1.0, 2.0); + assert_eq!( + base_input_larger_pubdata_price, base_input_scaled_pubdata_price, + "Scaling has the correct effect for the pubdata price" + ); + assert_eq!( + base_input.fair_l2_gas_price, base_input_larger_pubdata_price.fair_l2_gas_price, + "L1 pubdata increase has no effect on L2 gas price" + ); + assert!( + base_input.fair_pubdata_price < base_input_larger_pubdata_price.fair_pubdata_price, + "Pubdata price increase raises pubdata price" + ); + + let base_input_larger_max_gas = compute_batch_fee_model_input_v2( + FeeParamsV2 { + config: FeeModelConfigV2 { + max_gas_per_batch: base_config.max_gas_per_batch * 2, + ..base_config + }, + ..base_params + }, + 1.0, + 1.0, + ); + assert!( + base_input.fair_l2_gas_price > base_input_larger_max_gas.fair_l2_gas_price, + "Max gas increase lowers L2 gas price" + ); + assert_eq!( + base_input.fair_pubdata_price, base_input_larger_max_gas.fair_pubdata_price, + "Max gas increase has no effect on pubdata price" + ); + + let base_input_larger_max_pubdata = compute_batch_fee_model_input_v2( + FeeParamsV2 { + config: FeeModelConfigV2 { + max_pubdata_per_batch: base_config.max_pubdata_per_batch * 2, + ..base_config + }, + ..base_params + }, + 1.0, + 1.0, + ); + assert_eq!( + base_input.fair_l2_gas_price, base_input_larger_max_pubdata.fair_l2_gas_price, + "Max pubdata increase has no effect on L2 gas price" + ); + assert!( + base_input.fair_pubdata_price > base_input_larger_max_pubdata.fair_pubdata_price, + "Max pubdata increase lowers pubdata price" + ); + } +} diff --git a/core/lib/zksync_core/src/genesis.rs b/core/lib/zksync_core/src/genesis.rs index 7172968e0ef3..efc872cd995a 100644 --- a/core/lib/zksync_core/src/genesis.rs +++ b/core/lib/zksync_core/src/genesis.rs @@ -9,6 +9,7 @@ use zksync_merkle_tree::domain::ZkSyncTree; use zksync_types::{ block::{BlockGasCount, DeployedContract, L1BatchHeader, MiniblockHasher, MiniblockHeader}, commitment::{L1BatchCommitment, L1BatchMetadata}, + fee_model::BatchFeeInput, get_code_key, get_system_context_init_logs, protocol_version::{L1VerifierConfig, ProtocolVersion}, tokens::{TokenInfo, TokenMetadata, ETHEREUM_ADDRESS}, @@ -294,8 +295,7 @@ pub(crate) async fn create_genesis_l1_batch( l1_tx_count: 0, l2_tx_count: 0, base_fee_per_gas: 0, - l1_gas_price: 0, - l2_fair_gas_price: 0, + batch_fee_input: BatchFeeInput::l1_pegged(0, 0), base_system_contracts_hashes: base_system_contracts.hashes(), protocol_version: Some(protocol_version), virtual_blocks: 0, diff --git a/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs b/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs index 2adcd852c323..ac5d04134384 100644 --- a/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs +++ b/core/lib/zksync_core/src/l1_gas_price/gas_adjuster/mod.rs @@ -8,6 +8,7 @@ use std::{ use tokio::sync::watch; use zksync_config::GasAdjusterConfig; use zksync_eth_client::{Error, EthInterface}; +use zksync_system_constants::L1_GAS_PER_PUBDATA_BYTE; use self::metrics::METRICS; use super::{L1GasPriceProvider, L1TxParamsProvider}; @@ -126,6 +127,11 @@ impl L1GasPriceProvider for GasAdjuster { // Bound the price if it's too high. self.bound_gas_price(calculated_price) } + + fn estimate_effective_pubdata_price(&self) -> u64 { + // For now, pubdata is only sent via calldata, so its price is pegged to the L1 gas price. + self.estimate_effective_gas_price() * L1_GAS_PER_PUBDATA_BYTE as u64 + } } impl L1TxParamsProvider for GasAdjuster { diff --git a/core/lib/zksync_core/src/l1_gas_price/main_node_fetcher.rs b/core/lib/zksync_core/src/l1_gas_price/main_node_fetcher.rs index a3f7b92f0e46..04555728241c 100644 --- a/core/lib/zksync_core/src/l1_gas_price/main_node_fetcher.rs +++ b/core/lib/zksync_core/src/l1_gas_price/main_node_fetcher.rs @@ -1,18 +1,16 @@ use std::{ - sync::{ - atomic::{AtomicU64, Ordering}, - Arc, - }, + sync::{Arc, RwLock}, time::Duration, }; use tokio::sync::watch::Receiver; +use zksync_types::fee_model::FeeParams; use zksync_web3_decl::{ jsonrpsee::http_client::{HttpClient, HttpClientBuilder}, namespaces::ZksNamespaceClient, }; -use super::L1GasPriceProvider; +use crate::fee_model::BatchFeeModelInputProvider; const SLEEP_INTERVAL: Duration = Duration::from_secs(5); @@ -23,16 +21,16 @@ const SLEEP_INTERVAL: Duration = Duration::from_secs(5); /// The same algorithm cannot be consistently replicated on the external node side, /// since it relies on the configuration, which may change. #[derive(Debug)] -pub struct MainNodeGasPriceFetcher { +pub struct MainNodeFeeParamsFetcher { client: HttpClient, - gas_price: AtomicU64, + main_node_fee_params: RwLock, } -impl MainNodeGasPriceFetcher { +impl MainNodeFeeParamsFetcher { pub fn new(main_node_url: &str) -> Self { Self { client: Self::build_client(main_node_url), - gas_price: AtomicU64::new(1u64), // Start with 1 wei until the first update. + main_node_fee_params: RwLock::new(FeeParams::sensible_v1_default()), } } @@ -45,11 +43,11 @@ impl MainNodeGasPriceFetcher { pub async fn run(self: Arc, stop_receiver: Receiver) -> anyhow::Result<()> { loop { if *stop_receiver.borrow() { - tracing::info!("Stop signal received, MainNodeGasPriceFetcher is shutting down"); + tracing::info!("Stop signal received, MainNodeFeeParamsFetcher is shutting down"); break; } - let main_node_gas_price = match self.client.get_l1_gas_price().await { + let main_node_fee_params = match self.client.get_fee_params().await { Ok(price) => price, Err(err) => { tracing::warn!("Unable to get the gas price: {}", err); @@ -58,16 +56,16 @@ impl MainNodeGasPriceFetcher { continue; } }; - self.gas_price - .store(main_node_gas_price.as_u64(), Ordering::Relaxed); + *self.main_node_fee_params.write().unwrap() = main_node_fee_params; + tokio::time::sleep(SLEEP_INTERVAL).await; } Ok(()) } } -impl L1GasPriceProvider for MainNodeGasPriceFetcher { - fn estimate_effective_gas_price(&self) -> u64 { - self.gas_price.load(Ordering::Relaxed) +impl BatchFeeModelInputProvider for MainNodeFeeParamsFetcher { + fn get_fee_model_params(&self) -> FeeParams { + *self.main_node_fee_params.read().unwrap() } } diff --git a/core/lib/zksync_core/src/l1_gas_price/mod.rs b/core/lib/zksync_core/src/l1_gas_price/mod.rs index 57b0e51d4b43..fac91d02559e 100644 --- a/core/lib/zksync_core/src/l1_gas_price/mod.rs +++ b/core/lib/zksync_core/src/l1_gas_price/mod.rs @@ -3,7 +3,7 @@ use std::fmt; pub use gas_adjuster::GasAdjuster; -pub use main_node_fetcher::MainNodeGasPriceFetcher; +pub use main_node_fetcher::MainNodeFeeParamsFetcher; pub use singleton::GasAdjusterSingleton; mod gas_adjuster; @@ -16,6 +16,10 @@ pub trait L1GasPriceProvider: fmt::Debug + 'static + Send + Sync { /// Returns a best guess of a realistic value for the L1 gas price. /// Return value is in wei. fn estimate_effective_gas_price(&self) -> u64; + + /// Returns a best guess of a realistic value for the L1 pubdata price. + /// Return value is in wei. + fn estimate_effective_pubdata_price(&self) -> u64; } /// Extended version of `L1GasPriceProvider` that can provide parameters diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index fb3b8bd51955..3b953eb91e84 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -3,6 +3,7 @@ use std::{net::Ipv4Addr, str::FromStr, sync::Arc, time::Instant}; use anyhow::Context as _; +use fee_model::MainNodeFeeInputProvider; use futures::channel::oneshot; use prometheus_exporter::PrometheusExporterConfig; use temp_config_store::TempConfigStore; @@ -34,6 +35,7 @@ use zksync_object_store::{ObjectStore, ObjectStoreFactory}; use zksync_queued_job_processor::JobProcessor; use zksync_state::PostgresStorageCaches; use zksync_types::{ + fee_model::FeeModelConfigV1, protocol_version::{L1VerifierConfig, VerifierParams}, system_contracts::get_system_smart_contracts, web3::contract::tokens::Detokenize, @@ -79,6 +81,7 @@ mod consensus; pub mod consistency_checker; pub mod eth_sender; pub mod eth_watch; +mod fee_model; pub mod gas_tracker; pub mod genesis; pub mod house_keeper; @@ -677,7 +680,6 @@ async fn add_state_keeper_to_task_futures, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { - let fair_l2_gas_price = state_keeper_config.fair_l2_gas_price; let pool_builder = ConnectionPool::singleton(postgres_config.master_url()?); let state_keeper_pool = pool_builder .build() @@ -693,6 +695,13 @@ async fn add_state_keeper_to_task_futures, + batch_fee_input_provider: Arc, l2_erc20_bridge_addr: Address, chain_id: L2ChainId, @@ -107,13 +106,10 @@ impl StateKeeperIO for MempoolIO { .await?; // Initialize the filter for the transactions that come after the pending batch. // We use values from the pending block to match the filter with one used before the restart. - let (base_fee, gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata( - l1_batch_env.l1_gas_price, - l1_batch_env.fair_l2_gas_price, - system_env.version.into(), - ); + let (base_fee, gas_per_pubdata) = + derive_base_fee_and_gas_per_pubdata(l1_batch_env.fee_input, system_env.version.into()); self.filter = L2TxFilter { - l1_gas_price: l1_batch_env.l1_gas_price, + fee_input: l1_batch_env.fee_input, fee_per_gas: base_fee, gas_per_pubdata: gas_per_pubdata as u32, }; @@ -152,10 +148,9 @@ impl StateKeeperIO for MempoolIO { let current_timestamp = current_timestamp.await.ok()?; tracing::trace!( - "(l1_gas_price, fair_l2_gas_price) for L1 batch #{} is ({}, {})", + "Fee input for L1 batch #{} is {:#?}", self.current_l1_batch_number.0, - self.filter.l1_gas_price, - self.fair_l2_gas_price + self.filter.fee_input ); let mut storage = self.pool.access_storage().await.unwrap(); let (base_system_contracts, protocol_version) = storage @@ -166,8 +161,7 @@ impl StateKeeperIO for MempoolIO { // We create a new filter each time, since parameters may change and a previously // ignored transaction in the mempool may be scheduled for the execution. self.filter = l2_tx_filter( - self.l1_gas_price_provider.as_ref(), - self.fair_l2_gas_price, + self.batch_fee_input_provider.as_ref(), protocol_version.into(), ); // We only need to get the root hash when we're certain that we have a new transaction. @@ -181,8 +175,7 @@ impl StateKeeperIO for MempoolIO { self.fee_account, current_timestamp, prev_l1_batch_hash, - self.filter.l1_gas_price, - self.fair_l2_gas_price, + self.filter.fee_input, self.current_miniblock_number, prev_miniblock_hash, base_system_contracts, @@ -406,7 +399,7 @@ impl MempoolIO { mempool: MempoolGuard, object_store: Arc, miniblock_sealer_handle: MiniblockSealerHandle, - l1_gas_price_provider: Arc, + batch_fee_input_provider: Arc, pool: ConnectionPool, config: &StateKeeperConfig, delay_interval: Duration, @@ -448,10 +441,9 @@ impl MempoolIO { miniblock_sealer_handle, current_miniblock_number: last_miniblock_number + 1, fee_account: config.fee_account_addr, - fair_l2_gas_price: config.fair_l2_gas_price, validation_computational_gas_limit, delay_interval, - l1_gas_price_provider, + batch_fee_input_provider, l2_erc20_bridge_addr, chain_id, virtual_blocks_interval: config.virtual_blocks_interval, diff --git a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs index 908e24fd2d5f..2157c6c320fb 100644 --- a/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs +++ b/core/lib/zksync_core/src/state_keeper/io/seal_logic.rs @@ -7,12 +7,16 @@ use std::{ }; use itertools::Itertools; -use multivm::interface::{FinishedL1Batch, L1BatchEnv}; +use multivm::{ + interface::{FinishedL1Batch, L1BatchEnv}, + utils::get_batch_base_fee, +}; use zksync_dal::{blocks_dal::ConsensusBlockFields, StorageProcessor}; use zksync_system_constants::ACCOUNT_CODE_STORAGE_ADDRESS; use zksync_types::{ block::{unpack_block_info, L1BatchHeader, MiniblockHeader}, event::{extract_added_tokens, extract_long_l2_to_l1_messages}, + fee_model::BatchFeeInput, l1::L1Tx, l2::L2Tx, l2_to_l1_log::{SystemL2ToL1Log, UserL2ToL1Log}, @@ -128,7 +132,7 @@ impl UpdatesManager { l2_to_l1_messages, bloom: Default::default(), used_contract_hashes: finished_batch.final_execution_state.used_contract_hashes, - base_fee_per_gas: l1_batch_env.base_fee(), + base_fee_per_gas: get_batch_base_fee(l1_batch_env, self.protocol_version().into()), l1_gas_price: self.l1_gas_price(), l2_fair_gas_price: self.fair_l2_gas_price(), base_system_contracts_hashes: self.base_system_contract_hashes(), @@ -336,8 +340,7 @@ impl MiniblockSealCommand { l1_tx_count: l1_tx_count as u16, l2_tx_count: l2_tx_count as u16, base_fee_per_gas: self.base_fee_per_gas, - l1_gas_price: self.l1_gas_price, - l2_fair_gas_price: self.fair_l2_gas_price, + batch_fee_input: BatchFeeInput::l1_pegged(self.l1_gas_price, self.fair_l2_gas_price), base_system_contracts_hashes: self.base_system_contracts_hashes, protocol_version: self.protocol_version, virtual_blocks: self.miniblock.virtual_blocks, diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index 2135187362c9..13dd82611520 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -6,8 +6,8 @@ use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::ConnectionPool; use zksync_mempool::L2TxFilter; use zksync_types::{ - block::BlockGasCount, tx::ExecutionMetrics, AccountTreeId, Address, L1BatchNumber, - MiniblockNumber, ProtocolVersionId, StorageKey, VmEvent, H256, U256, + block::BlockGasCount, fee_model::BatchFeeInput, tx::ExecutionMetrics, AccountTreeId, Address, + L1BatchNumber, MiniblockNumber, ProtocolVersionId, StorageKey, VmEvent, H256, U256, }; use zksync_utils::time::seconds_since_epoch; @@ -53,21 +53,17 @@ async fn test_filter_with_pending_batch() { // These gas values are random and don't matter for filter calculation as there will be a // pending batch the filter will be based off of. tester - .insert_miniblock(&connection_pool, 1, 5, 55, 555) + .insert_miniblock(&connection_pool, 1, 5, BatchFeeInput::l1_pegged(55, 555)) .await; tester.insert_sealed_batch(&connection_pool, 1).await; // Inserting a pending miniblock that isn't included in a sealed batch means there is a pending batch. // The gas values are randomly chosen but so affect filter values calculation. - let (give_l1_gas_price, give_fair_l2_gas_price) = (100, 1000); + + let fee_input = BatchFeeInput::l1_pegged(100, 1000); + tester - .insert_miniblock( - &connection_pool, - 2, - 10, - give_l1_gas_price, - give_fair_l2_gas_price, - ) + .insert_miniblock(&connection_pool, 2, 10, fee_input) .await; let (mut mempool, _) = tester.create_test_mempool_io(connection_pool, 1).await; @@ -75,13 +71,10 @@ async fn test_filter_with_pending_batch() { assert_eq!(mempool.filter(), &L2TxFilter::default()); mempool.load_pending_batch().await; - let (want_base_fee, want_gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata( - give_l1_gas_price, - give_fair_l2_gas_price, - ProtocolVersionId::latest().into(), - ); + let (want_base_fee, want_gas_per_pubdata) = + derive_base_fee_and_gas_per_pubdata(fee_input, ProtocolVersionId::latest().into()); let want_filter = L2TxFilter { - l1_gas_price: give_l1_gas_price, + fee_input, fee_per_gas: want_base_fee, gas_per_pubdata: want_gas_per_pubdata as u32, }; @@ -98,14 +91,13 @@ async fn test_filter_with_no_pending_batch() { // Insert a sealed batch so there will be a `prev_l1_batch_state_root`. // These gas values are random and don't matter for filter calculation. tester - .insert_miniblock(&connection_pool, 1, 5, 55, 555) + .insert_miniblock(&connection_pool, 1, 5, BatchFeeInput::l1_pegged(55, 555)) .await; tester.insert_sealed_batch(&connection_pool, 1).await; // Create a copy of the tx filter that the mempool will use. let want_filter = l2_tx_filter( - &tester.create_gas_adjuster().await, - tester.fair_l2_gas_price(), + &tester.create_batch_fee_input_provider().await, ProtocolVersionId::latest().into(), ); @@ -139,7 +131,7 @@ async fn test_timestamps_are_distinct( tester.set_timestamp(prev_miniblock_timestamp); tester - .insert_miniblock(&connection_pool, 1, 5, 55, 555) + .insert_miniblock(&connection_pool, 1, 5, BatchFeeInput::l1_pegged(55, 555)) .await; if delay_prev_miniblock_compared_to_batch { tester.set_timestamp(prev_miniblock_timestamp - 1); @@ -149,8 +141,7 @@ async fn test_timestamps_are_distinct( let (mut mempool, mut guard) = tester.create_test_mempool_io(connection_pool, 1).await; // Insert a transaction to trigger L1 batch creation. let tx_filter = l2_tx_filter( - &tester.create_gas_adjuster().await, - tester.fair_l2_gas_price(), + &tester.create_batch_fee_input_provider().await, ProtocolVersionId::latest().into(), ); tester.insert_tx(&mut guard, tx_filter.fee_per_gas, tx_filter.gas_per_pubdata); diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs index 3adf1f3bff06..626924a9c5e5 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/tester.rs @@ -9,12 +9,15 @@ use zksync_dal::ConnectionPool; use zksync_eth_client::clients::MockEthereum; use zksync_object_store::ObjectStoreFactory; use zksync_types::{ - block::MiniblockHeader, protocol_version::L1VerifierConfig, - system_contracts::get_system_smart_contracts, Address, L2ChainId, PriorityOpId, - ProtocolVersionId, H256, + block::MiniblockHeader, + fee_model::{BatchFeeInput, FeeModelConfig, FeeModelConfigV1}, + protocol_version::L1VerifierConfig, + system_contracts::get_system_smart_contracts, + Address, L2ChainId, PriorityOpId, ProtocolVersionId, H256, }; use crate::{ + fee_model::MainNodeFeeInputProvider, genesis::create_genesis_l1_batch, l1_gas_price::GasAdjuster, state_keeper::{io::MiniblockSealer, tests::create_transaction, MempoolGuard, MempoolIO}, @@ -36,7 +39,7 @@ impl Tester { } } - pub(super) async fn create_gas_adjuster(&self) -> GasAdjuster { + async fn create_gas_adjuster(&self) -> GasAdjuster { let eth_client = MockEthereum::default().with_fee_history(vec![0, 4, 6, 8, 7, 5, 5, 8, 10, 9]); @@ -56,6 +59,16 @@ impl Tester { .unwrap() } + pub(super) async fn create_batch_fee_input_provider(&self) -> MainNodeFeeInputProvider { + let gas_adjuster = Arc::new(self.create_gas_adjuster().await); + MainNodeFeeInputProvider::new( + gas_adjuster, + FeeModelConfig::V1(FeeModelConfigV1 { + minimal_l2_gas_price: self.fair_l2_gas_price(), + }), + ) + } + // Constant value to be used both in tests and inside of the IO. pub(super) fn fair_l2_gas_price(&self) -> u64 { 100 @@ -67,6 +80,13 @@ impl Tester { miniblock_sealer_capacity: usize, ) -> (MempoolIO, MempoolGuard) { let gas_adjuster = Arc::new(self.create_gas_adjuster().await); + let batch_fee_input_provider = MainNodeFeeInputProvider::new( + gas_adjuster, + FeeModelConfig::V1(FeeModelConfigV1 { + minimal_l2_gas_price: self.fair_l2_gas_price(), + }), + ); + let mempool = MempoolGuard::new(PriorityOpId(0), 100); let (miniblock_sealer, miniblock_sealer_handle) = MiniblockSealer::new(pool.clone(), miniblock_sealer_capacity); @@ -84,7 +104,7 @@ impl Tester { mempool.clone(), object_store, miniblock_sealer_handle, - gas_adjuster, + Arc::new(batch_fee_input_provider), pool, &config, Duration::from_secs(1), @@ -123,8 +143,7 @@ impl Tester { pool: &ConnectionPool, number: u32, base_fee_per_gas: u64, - l1_gas_price: u64, - l2_fair_gas_price: u64, + fee_input: BatchFeeInput, ) { let mut storage = pool.access_storage_tagged("state_keeper").await.unwrap(); storage @@ -132,8 +151,7 @@ impl Tester { .insert_miniblock(&MiniblockHeader { timestamp: self.current_timestamp, base_fee_per_gas, - l1_gas_price, - l2_fair_gas_price, + batch_fee_input: fee_input, base_system_contracts_hashes: self.base_system_contracts.hashes(), ..create_miniblock(number) }) diff --git a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs b/core/lib/zksync_core/src/state_keeper/mempool_actor.rs index 5ad962b112e9..f452c87a2df9 100644 --- a/core/lib/zksync_core/src/state_keeper/mempool_actor.rs +++ b/core/lib/zksync_core/src/state_keeper/mempool_actor.rs @@ -8,22 +8,20 @@ use zksync_mempool::L2TxFilter; use zksync_types::{ProtocolVersionId, VmVersion}; use super::{metrics::KEEPER_METRICS, types::MempoolGuard}; -use crate::{api_server::execution_sandbox::BlockArgs, l1_gas_price::L1GasPriceProvider}; +use crate::{api_server::execution_sandbox::BlockArgs, fee_model::BatchFeeModelInputProvider}; /// 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 /// to process them. pub fn l2_tx_filter( - gas_price_provider: &dyn L1GasPriceProvider, - fair_l2_gas_price: u64, + batch_fee_input_provider: &dyn BatchFeeModelInputProvider, vm_version: VmVersion, ) -> L2TxFilter { - let effective_gas_price = gas_price_provider.estimate_effective_gas_price(); + let fee_input = batch_fee_input_provider.get_batch_fee_input(); - let (base_fee, gas_per_pubdata) = - derive_base_fee_and_gas_per_pubdata(effective_gas_price, fair_l2_gas_price, vm_version); + let (base_fee, gas_per_pubdata) = derive_base_fee_and_gas_per_pubdata(fee_input, vm_version); L2TxFilter { - l1_gas_price: effective_gas_price, + fee_input, fee_per_gas: base_fee, gas_per_pubdata: gas_per_pubdata as u32, } @@ -32,20 +30,20 @@ pub fn l2_tx_filter( #[derive(Debug)] pub struct MempoolFetcher { mempool: MempoolGuard, - l1_gas_price_provider: Arc, + batch_fee_input_provider: Arc, sync_interval: Duration, sync_batch_size: usize, } -impl MempoolFetcher { +impl MempoolFetcher { pub fn new( mempool: MempoolGuard, - l1_gas_price_provider: Arc, + batch_fee_input_provider: Arc, config: &MempoolConfig, ) -> Self { Self { mempool, - l1_gas_price_provider, + batch_fee_input_provider, sync_interval: config.sync_interval(), sync_batch_size: config.sync_batch_size, } @@ -56,7 +54,6 @@ impl MempoolFetcher { pool: ConnectionPool, remove_stuck_txs: bool, stuck_tx_timeout: Duration, - fair_l2_gas_price: u64, stop_receiver: watch::Receiver, ) -> anyhow::Result<()> { { @@ -90,8 +87,7 @@ impl MempoolFetcher { .unwrap_or_else(ProtocolVersionId::latest); let l2_tx_filter = l2_tx_filter( - self.l1_gas_price_provider.as_ref(), - fair_l2_gas_price, + self.batch_fee_input_provider.as_ref(), protocol_version.into(), ); diff --git a/core/lib/zksync_core/src/state_keeper/mod.rs b/core/lib/zksync_core/src/state_keeper/mod.rs index dc5016772a7b..b1534d9612f0 100644 --- a/core/lib/zksync_core/src/state_keeper/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/mod.rs @@ -17,7 +17,7 @@ pub use self::{ pub(crate) use self::{ mempool_actor::MempoolFetcher, seal_criteria::SequencerSealer, types::MempoolGuard, }; -use crate::l1_gas_price::L1GasPriceProvider; +use crate::fee_model::BatchFeeModelInputProvider; mod batch_executor; pub(crate) mod extractors; @@ -40,7 +40,7 @@ pub(crate) async fn create_state_keeper( mempool_config: &MempoolConfig, pool: ConnectionPool, mempool: MempoolGuard, - l1_gas_price_provider: Arc, + batch_fee_input_provider: Arc, miniblock_sealer_handle: MiniblockSealerHandle, object_store: Arc, stop_receiver: watch::Receiver, @@ -59,7 +59,7 @@ pub(crate) async fn create_state_keeper( mempool, object_store, miniblock_sealer_handle, - l1_gas_price_provider, + batch_fee_input_provider, pool, &state_keeper_config, mempool_config.delay_interval(), diff --git a/core/lib/zksync_core/src/state_keeper/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/tests/mod.rs index 7d0c5fbd93da..06e3321ce483 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/mod.rs @@ -20,6 +20,7 @@ use zksync_system_constants::ZKPORTER_IS_AVAILABLE; use zksync_types::{ aggregated_operations::AggregatedActionType, block::{BlockGasCount, MiniblockExecutionData, MiniblockHasher}, + fee_model::BatchFeeInput, tx::tx_execution_info::ExecutionMetrics, Address, L1BatchNumber, L2ChainId, LogQuery, MiniblockNumber, ProtocolVersionId, StorageLogQuery, StorageLogQueryType, Timestamp, Transaction, H256, U256, @@ -70,8 +71,7 @@ pub(super) fn default_l1_batch_env( previous_batch_hash: None, number: L1BatchNumber(number), timestamp, - l1_gas_price: 1, - fair_l2_gas_price: 1, + fee_input: BatchFeeInput::l1_pegged(1, 1), fee_account, enforced_base_fee: None, first_l2_block: L2BlockEnv { diff --git a/core/lib/zksync_core/src/state_keeper/tests/tester.rs b/core/lib/zksync_core/src/state_keeper/tests/tester.rs index 83ac7f6b6e88..340a650eb5eb 100644 --- a/core/lib/zksync_core/src/state_keeper/tests/tester.rs +++ b/core/lib/zksync_core/src/state_keeper/tests/tester.rs @@ -15,7 +15,7 @@ use multivm::{ }; use tokio::sync::{mpsc, watch}; use zksync_types::{ - block::MiniblockExecutionData, protocol_version::ProtocolUpgradeTx, + block::MiniblockExecutionData, fee_model::BatchFeeInput, protocol_version::ProtocolUpgradeTx, witness_block_state::WitnessBlockState, Address, L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, Transaction, H256, }; @@ -651,8 +651,7 @@ impl StateKeeperIO for TestIO { previous_batch_hash: Some(H256::zero()), number: self.batch_number, timestamp: self.timestamp, - l1_gas_price: self.l1_gas_price, - fair_l2_gas_price: self.fair_l2_gas_price, + fee_input: BatchFeeInput::l1_pegged(self.l1_gas_price, self.fair_l2_gas_price), fee_account: self.fee_account, enforced_base_fee: None, first_l2_block: first_miniblock_info, diff --git a/core/lib/zksync_core/src/state_keeper/updates/mod.rs b/core/lib/zksync_core/src/state_keeper/updates/mod.rs index ad5e9e432d43..5584557edc34 100644 --- a/core/lib/zksync_core/src/state_keeper/updates/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/updates/mod.rs @@ -1,4 +1,7 @@ -use multivm::interface::{L1BatchEnv, VmExecutionResultAndLogs}; +use multivm::{ + interface::{L1BatchEnv, VmExecutionResultAndLogs}, + utils::get_batch_base_fee, +}; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::blocks_dal::ConsensusBlockFields; use zksync_types::{ @@ -41,9 +44,9 @@ impl UpdatesManager { ) -> Self { Self { batch_timestamp: l1_batch_env.timestamp, - l1_gas_price: l1_batch_env.l1_gas_price, - fair_l2_gas_price: l1_batch_env.fair_l2_gas_price, - base_fee_per_gas: l1_batch_env.base_fee(), + l1_gas_price: l1_batch_env.fee_input.l1_gas_price(), + fair_l2_gas_price: l1_batch_env.fee_input.fair_l2_gas_price(), + base_fee_per_gas: get_batch_base_fee(&l1_batch_env, protocol_version.into()), protocol_version, base_system_contract_hashes, l1_batch: L1BatchUpdates::new(), 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 d751cdc8d014..0c5440a38ed4 100644 --- a/core/lib/zksync_core/src/sync_layer/external_io.rs +++ b/core/lib/zksync_core/src/sync_layer/external_io.rs @@ -6,8 +6,9 @@ use multivm::interface::{FinishedL1Batch, L1BatchEnv, SystemEnv}; use zksync_contracts::{BaseSystemContracts, SystemContractCode}; use zksync_dal::ConnectionPool; use zksync_types::{ - ethabi::Address, protocol_version::ProtocolUpgradeTx, witness_block_state::WitnessBlockState, - L1BatchNumber, L2ChainId, MiniblockNumber, ProtocolVersionId, Transaction, H256, U256, + ethabi::Address, fee_model::BatchFeeInput, protocol_version::ProtocolUpgradeTx, + witness_block_state::WitnessBlockState, L1BatchNumber, L2ChainId, MiniblockNumber, + ProtocolVersionId, Transaction, H256, U256, }; use zksync_utils::{be_words_to_bytes, bytes_to_be_words}; @@ -339,8 +340,7 @@ impl StateKeeperIO for ExternalIO { operator_address, timestamp, previous_l1_batch_hash, - l1_gas_price, - l2_fair_gas_price, + BatchFeeInput::l1_pegged(l1_gas_price, l2_fair_gas_price), miniblock_number, previous_miniblock_hash, base_system_contracts, diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 826f4e343e92..011f8b608ff5 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -165,8 +165,8 @@ async fn external_io_basics() { .unwrap() .expect("Miniblock #1 is not persisted"); assert_eq!(miniblock.timestamp, 1); - assert_eq!(miniblock.l1_gas_price, 2); - assert_eq!(miniblock.l2_fair_gas_price, 3); + assert_eq!(miniblock.batch_fee_input.l1_gas_price(), 2); + assert_eq!(miniblock.batch_fee_input.fair_l2_gas_price(), 3); assert_eq!(miniblock.l1_tx_count, 0); assert_eq!(miniblock.l2_tx_count, 1); diff --git a/core/lib/zksync_core/src/utils/testonly.rs b/core/lib/zksync_core/src/utils/testonly.rs index b026c951ef19..7e868a15c1c4 100644 --- a/core/lib/zksync_core/src/utils/testonly.rs +++ b/core/lib/zksync_core/src/utils/testonly.rs @@ -6,6 +6,7 @@ use zksync_types::{ block::{L1BatchHeader, MiniblockHeader}, commitment::{L1BatchMetaParameters, L1BatchMetadata}, fee::Fee, + fee_model::BatchFeeInput, l2::L2Tx, transaction_request::PaymasterParams, Address, L1BatchNumber, L2ChainId, MiniblockNumber, Nonce, ProtocolVersionId, H256, U256, @@ -20,8 +21,7 @@ pub(crate) fn create_miniblock(number: u32) -> MiniblockHeader { l1_tx_count: 0, l2_tx_count: 0, base_fee_per_gas: 100, - l1_gas_price: 100, - l2_fair_gas_price: 100, + batch_fee_input: BatchFeeInput::l1_pegged(100, 100), base_system_contracts_hashes: BaseSystemContractsHashes::default(), protocol_version: Some(ProtocolVersionId::latest()), virtual_blocks: 1, diff --git a/core/tests/vm-benchmark/harness/src/lib.rs b/core/tests/vm-benchmark/harness/src/lib.rs index 08d52dd67a69..9b4081dc69ae 100644 --- a/core/tests/vm-benchmark/harness/src/lib.rs +++ b/core/tests/vm-benchmark/harness/src/lib.rs @@ -14,6 +14,7 @@ use zksync_types::{ block::MiniblockHasher, ethabi::{encode, Token}, fee::Fee, + fee_model::BatchFeeInput, helpers::unix_timestamp_ms, l2::L2Tx, utils::storage_key_for_eth_balance, @@ -69,8 +70,10 @@ impl BenchmarkingVm { previous_batch_hash: None, number: L1BatchNumber(1), timestamp, - l1_gas_price: 50_000_000_000, // 50 gwei - fair_l2_gas_price: 250_000_000, // 0.25 gwei + fee_input: BatchFeeInput::l1_pegged( + 50_000_000_000, // 50 gwei + 250_000_000, // 0.25 gwei + ), fee_account: Address::random(), enforced_base_fee: None, first_l2_block: L2BlockEnv { From b8b190f886f1d13602a0b2cc8a2b8525e68b1033 Mon Sep 17 00:00:00 2001 From: Fedor Sakharov Date: Tue, 9 Jan 2024 12:26:47 +0100 Subject: [PATCH 48/49] feat(core): removes multiple tokio runtimes and worker number setting. (#826) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ ## Why ❔ ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zk fmt` and `zk lint`. - [ ] Spellcheck has been run via `cargo spellcheck --cfg=./spellcheck/era.cfg --code 1`. --- core/bin/external_node/src/config/mod.rs | 2 - core/bin/external_node/src/main.rs | 10 +--- core/lib/config/src/configs/api.rs | 21 -------- core/lib/env_config/src/api.rs | 8 --- .../api_server/contract_verification/mod.rs | 6 +-- .../zksync_core/src/api_server/web3/mod.rs | 54 ++++++------------- .../zksync_core/src/api_server/web3/pubsub.rs | 22 +++----- .../src/api_server/web3/tests/mod.rs | 1 - core/lib/zksync_core/src/lib.rs | 2 - .../prepared_configs/mainnet-config.env | 2 - .../testnet-goerli-config-deprecated.env | 2 - .../testnet-sepolia-config.env | 2 - 12 files changed, 27 insertions(+), 105 deletions(-) diff --git a/core/bin/external_node/src/config/mod.rs b/core/bin/external_node/src/config/mod.rs index e25b5390b0ff..cc923e1eb3c3 100644 --- a/core/bin/external_node/src/config/mod.rs +++ b/core/bin/external_node/src/config/mod.rs @@ -355,8 +355,6 @@ pub struct RequiredENConfig { pub ws_port: u16, /// Port on which the healthcheck REST server is listening. pub healthcheck_port: u16, - /// Number of threads per API server - pub threads_per_server: usize, /// Address of the Ethereum node API. /// Intentionally private: use getter method as it manages the missing port. eth_client_url: String, diff --git a/core/bin/external_node/src/main.rs b/core/bin/external_node/src/main.rs index 863deacdb589..1cf17bc84001 100644 --- a/core/bin/external_node/src/main.rs +++ b/core/bin/external_node/src/main.rs @@ -278,7 +278,6 @@ async fn init_tasks( .with_filter_limit(config.optional.filters_limit) .with_batch_request_size_limit(config.optional.max_batch_request_size) .with_response_body_size_limit(config.optional.max_response_body_size()) - .with_threads(config.required.threads_per_server) .with_tx_sender(tx_sender.clone(), vm_barrier.clone()) .with_sync_state(sync_state.clone()) .enable_api_namespaces(config.optional.api_namespaces()) @@ -294,7 +293,6 @@ async fn init_tasks( .with_batch_request_size_limit(config.optional.max_batch_request_size) .with_response_body_size_limit(config.optional.max_response_body_size()) .with_polling_interval(config.optional.polling_interval()) - .with_threads(config.required.threads_per_server) .with_tx_sender(tx_sender, vm_barrier) .with_sync_state(sync_state) .enable_api_namespaces(config.optional.api_namespaces()) @@ -485,17 +483,11 @@ async fn main() -> anyhow::Result<()> { if let Some(last_correct_batch) = reorg_detector_last_correct_batch { tracing::info!("Performing rollback to L1 batch #{last_correct_batch}"); - let block_reverter_connection_pool = - ConnectionPool::builder(&config.postgres.database_url, 1) - .build() - .await - .context("failed to build a block reverter connection pool")?; - let reverter = BlockReverter::new( config.required.state_cache_path, config.required.merkle_tree_path, None, - block_reverter_connection_pool, + connection_pool, L1ExecutedBatchesRevert::Allowed, ); reverter diff --git a/core/lib/config/src/configs/api.rs b/core/lib/config/src/configs/api.rs index 6b2d727da442..609634d6f34a 100644 --- a/core/lib/config/src/configs/api.rs +++ b/core/lib/config/src/configs/api.rs @@ -38,8 +38,6 @@ pub struct Web3JsonRpcConfig { pub subscriptions_limit: Option, /// Interval between polling db for pubsub (in ms). pub pubsub_polling_interval: Option, - /// number of threads per server - pub threads_per_server: u32, /// Tx nonce: how far ahead from the committed nonce can it be. pub max_nonce_ahead: u32, /// The multiplier to use when suggesting gas price. Should be higher than one, @@ -69,12 +67,6 @@ pub struct Web3JsonRpcConfig { /// Latest values cache size in MiBs. The default value is 128 MiB. If set to 0, the latest /// values cache will be disabled. pub latest_values_cache_size_mb: Option, - /// Override value for the amount of threads used for HTTP RPC server. - /// If not set, the value from `threads_per_server` is used. - pub http_threads: Option, - /// Override value for the amount of threads used for WebSocket RPC server. - /// If not set, the value from `threads_per_server` is used. - pub ws_threads: Option, /// Limit for fee history block range. pub fee_history_limit: Option, /// Maximum number of requests in a single batch JSON RPC request. Default is 500. @@ -103,7 +95,6 @@ impl Web3JsonRpcConfig { filters_limit: Some(10000), subscriptions_limit: Some(10000), pubsub_polling_interval: Some(200), - threads_per_server: 1, max_nonce_ahead: 50, gas_price_scale_factor: 1.2, request_timeout: Default::default(), @@ -116,8 +107,6 @@ impl Web3JsonRpcConfig { factory_deps_cache_size_mb: Default::default(), initial_writes_cache_size_mb: Default::default(), latest_values_cache_size_mb: Default::default(), - http_threads: Default::default(), - ws_threads: Default::default(), fee_history_limit: Default::default(), max_batch_request_size: Default::default(), max_response_body_size_mb: Default::default(), @@ -180,14 +169,6 @@ impl Web3JsonRpcConfig { self.latest_values_cache_size_mb.unwrap_or(128) * super::BYTES_IN_MEGABYTE } - pub fn http_server_threads(&self) -> usize { - self.http_threads.unwrap_or(self.threads_per_server) as usize - } - - pub fn ws_server_threads(&self) -> usize { - self.ws_threads.unwrap_or(self.threads_per_server) as usize - } - pub fn fee_history_limit(&self) -> u64 { self.fee_history_limit.unwrap_or(1024) } @@ -230,8 +211,6 @@ pub struct ContractVerificationApiConfig { pub port: u16, /// URL to access REST server. pub url: String, - /// number of threads per server - pub threads_per_server: u32, } impl ContractVerificationApiConfig { diff --git a/core/lib/env_config/src/api.rs b/core/lib/env_config/src/api.rs index 990f9c40b9e2..3bffb281ad5c 100644 --- a/core/lib/env_config/src/api.rs +++ b/core/lib/env_config/src/api.rs @@ -66,7 +66,6 @@ mod tests { filters_limit: Some(10000), subscriptions_limit: Some(10000), pubsub_polling_interval: Some(200), - threads_per_server: 128, max_nonce_ahead: 5, request_timeout: Some(10), account_pks: Some(vec![ @@ -82,8 +81,6 @@ mod tests { factory_deps_cache_size_mb: Some(128), initial_writes_cache_size_mb: Some(32), latest_values_cache_size_mb: Some(256), - http_threads: Some(128), - ws_threads: Some(256), fee_history_limit: Some(100), max_batch_request_size: Some(200), max_response_body_size_mb: Some(10), @@ -93,7 +90,6 @@ mod tests { contract_verification: ContractVerificationApiConfig { port: 3070, url: "http://127.0.0.1:3070".into(), - threads_per_server: 128, }, prometheus: PrometheusConfig { listener_port: 3312, @@ -117,7 +113,6 @@ mod tests { API_WEB3_JSON_RPC_FILTERS_LIMIT=10000 API_WEB3_JSON_RPC_SUBSCRIPTIONS_LIMIT=10000 API_WEB3_JSON_RPC_PUBSUB_POLLING_INTERVAL=200 - API_WEB3_JSON_RPC_THREADS_PER_SERVER=128 API_WEB3_JSON_RPC_MAX_NONCE_AHEAD=5 API_WEB3_JSON_RPC_GAS_PRICE_SCALE_FACTOR=1.2 API_WEB3_JSON_RPC_REQUEST_TIMEOUT=10 @@ -129,14 +124,11 @@ mod tests { API_WEB3_JSON_RPC_FACTORY_DEPS_CACHE_SIZE_MB=128 API_WEB3_JSON_RPC_INITIAL_WRITES_CACHE_SIZE_MB=32 API_WEB3_JSON_RPC_LATEST_VALUES_CACHE_SIZE_MB=256 - API_WEB3_JSON_RPC_HTTP_THREADS=128 - API_WEB3_JSON_RPC_WS_THREADS=256 API_WEB3_JSON_RPC_FEE_HISTORY_LIMIT=100 API_WEB3_JSON_RPC_MAX_BATCH_REQUEST_SIZE=200 API_WEB3_JSON_RPC_WEBSOCKET_REQUESTS_PER_MINUTE_LIMIT=10 API_CONTRACT_VERIFICATION_PORT="3070" API_CONTRACT_VERIFICATION_URL="http://127.0.0.1:3070" - API_CONTRACT_VERIFICATION_THREADS_PER_SERVER=128 API_WEB3_JSON_RPC_MAX_RESPONSE_BODY_SIZE_MB=10 API_PROMETHEUS_LISTENER_PORT="3312" API_PROMETHEUS_PUSHGATEWAY_URL="http://127.0.0.1:9091" diff --git a/core/lib/zksync_core/src/api_server/contract_verification/mod.rs b/core/lib/zksync_core/src/api_server/contract_verification/mod.rs index 3c10b91ff713..27d36429d035 100644 --- a/core/lib/zksync_core/src/api_server/contract_verification/mod.rs +++ b/core/lib/zksync_core/src/api_server/contract_verification/mod.rs @@ -13,7 +13,7 @@ mod api_decl; mod api_impl; mod metrics; -fn start_server(api: RestApi, bind_to: SocketAddr, threads: usize) -> Server { +fn start_server(api: RestApi, bind_to: SocketAddr) -> Server { HttpServer::new(move || { let api = api.clone(); App::new() @@ -32,7 +32,6 @@ fn start_server(api: RestApi, bind_to: SocketAddr, threads: usize) -> Server { web::get().to(|| async { HttpResponse::Ok().finish() }), ) }) - .workers(threads) .bind(bind_to) .unwrap() .shutdown_timeout(60) @@ -58,10 +57,9 @@ pub fn start_server_thread_detached( actix_rt::System::new().block_on(async move { let bind_address = api_config.bind_addr(); - let threads = api_config.threads_per_server as usize; let api = RestApi::new(master_connection_pool, replica_connection_pool); - let server = start_server(api, bind_address, threads); + let server = start_server(api, bind_address); let close_handle = server.handle(); actix_rt::spawn(async move { if stop_receiver.changed().await.is_ok() { 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 877714fb3ee1..06a4b0d545f5 100644 --- a/core/lib/zksync_core/src/api_server/web3/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/mod.rs @@ -125,7 +125,6 @@ struct FullApiParams { transport: ApiTransport, tx_sender: TxSender, vm_barrier: VmConcurrencyBarrier, - threads: usize, polling_interval: Duration, namespaces: Vec, optional: OptionalApiParams, @@ -141,7 +140,6 @@ pub struct ApiBuilder { transport: Option, tx_sender: Option, vm_barrier: Option, - threads: Option, // Optional params that may or may not be set using builder methods. We treat `namespaces` // specially because we want to output a warning if they are not set. namespaces: Option>, @@ -160,7 +158,6 @@ impl ApiBuilder { transport: None, tx_sender: None, vm_barrier: None, - threads: None, namespaces: None, optional: OptionalApiParams::default(), } @@ -224,11 +221,6 @@ impl ApiBuilder { self } - pub fn with_threads(mut self, threads: usize) -> Self { - self.threads = Some(threads); - self - } - pub fn with_polling_interval(mut self, polling_interval: Duration) -> Self { self.polling_interval = polling_interval; self @@ -258,7 +250,6 @@ impl ApiBuilder { transport: self.transport.context("API transport not set")?, tx_sender: self.tx_sender.context("Transaction sender not set")?, vm_barrier: self.vm_barrier.context("VM barrier not set")?, - threads: self.threads.context("Number of server threads not set")?, polling_interval: self.polling_interval, namespaces: self.namespaces.unwrap_or_else(|| { tracing::warn!( @@ -399,9 +390,9 @@ impl FullApiParams { stop_receiver: watch::Receiver, ) -> anyhow::Result { let transport = self.transport; - let (runtime_thread_name, health_check_name) = match transport { - ApiTransport::Http(_) => ("jsonrpsee-http-worker", "http_api"), - ApiTransport::WebSocket(_) => ("jsonrpsee-ws-worker", "ws_api"), + let health_check_name = match transport { + ApiTransport::Http(_) => "http_api", + ApiTransport::WebSocket(_) => "ws_api", }; let (health_check, health_updater) = ReactiveHealthCheck::new(health_check_name); let vm_barrier = self.vm_barrier.clone(); @@ -419,21 +410,12 @@ impl FullApiParams { let websocket_requests_per_minute_limit = self.optional.websocket_requests_per_minute_limit; let subscriptions_limit = self.optional.subscriptions_limit; - let runtime = tokio::runtime::Builder::new_multi_thread() - .enable_all() - .thread_name(runtime_thread_name) - .worker_threads(self.threads) - .build() - .with_context(|| { - format!("Failed creating Tokio runtime for {health_check_name} jsonrpsee server") - })?; - let mut tasks = vec![]; let mut pubsub = None; if matches!(transport, ApiTransport::WebSocket(_)) && self.namespaces.contains(&Namespace::Pubsub) { - let mut pub_sub = EthSubscribe::new(runtime.handle().clone()); + let mut pub_sub = EthSubscribe::new(); if let Some(sender) = &self.optional.pub_sub_events_sender { pub_sub.set_events_sender(sender.clone()); } @@ -449,22 +431,18 @@ impl FullApiParams { let rpc = self.build_rpc_module(pubsub).await; // Start the server in a separate tokio runtime from a dedicated thread. let (local_addr_sender, local_addr) = oneshot::channel(); - let server_task = tokio::task::spawn_blocking(move || { - let res = runtime.block_on(Self::run_jsonrpsee_server( - rpc, - transport, - stop_receiver, - local_addr_sender, - health_updater, - vm_barrier, - batch_request_config, - response_body_size_limit, - subscriptions_limit, - websocket_requests_per_minute_limit, - )); - runtime.shutdown_timeout(GRACEFUL_SHUTDOWN_TIMEOUT); - res - }); + let server_task = tokio::spawn(Self::run_jsonrpsee_server( + rpc, + transport, + stop_receiver, + local_addr_sender, + health_updater, + vm_barrier, + batch_request_config, + response_body_size_limit, + subscriptions_limit, + websocket_requests_per_minute_limit, + )); let local_addr = match local_addr.await { Ok(addr) => addr, diff --git a/core/lib/zksync_core/src/api_server/web3/pubsub.rs b/core/lib/zksync_core/src/api_server/web3/pubsub.rs index 2e5adfe5e1cc..07a5eeb64af2 100644 --- a/core/lib/zksync_core/src/api_server/web3/pubsub.rs +++ b/core/lib/zksync_core/src/api_server/web3/pubsub.rs @@ -200,11 +200,10 @@ pub(super) struct EthSubscribe { transactions: broadcast::Sender>, logs: broadcast::Sender>, events_sender: Option>, - handle: tokio::runtime::Handle, } impl EthSubscribe { - pub fn new(handle: tokio::runtime::Handle) -> Self { + pub fn new() -> Self { let (blocks, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); let (transactions, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); let (logs, _) = broadcast::channel(BROADCAST_CHANNEL_CAPACITY); @@ -214,7 +213,6 @@ impl EthSubscribe { transactions, logs, events_sender: None, - handle, } } @@ -323,7 +321,7 @@ impl EthSubscribe { return; }; let blocks_rx = self.blocks.subscribe(); - self.handle.spawn(Self::run_subscriber( + tokio::spawn(Self::run_subscriber( sink, SubscriptionType::Blocks, blocks_rx, @@ -337,7 +335,7 @@ impl EthSubscribe { return; }; let transactions_rx = self.transactions.subscribe(); - self.handle.spawn(Self::run_subscriber( + tokio::spawn(Self::run_subscriber( sink, SubscriptionType::Txs, transactions_rx, @@ -357,7 +355,7 @@ impl EthSubscribe { return; }; let logs_rx = self.logs.subscribe(); - self.handle.spawn(Self::run_subscriber( + tokio::spawn(Self::run_subscriber( sink, SubscriptionType::Logs, logs_rx, @@ -371,7 +369,7 @@ impl EthSubscribe { return; }; - self.handle.spawn(async move { + tokio::spawn(async move { sink.send_timeout( SubscriptionMessage::from_json(&PubSubResult::Syncing(false)).unwrap(), SUBSCRIPTION_SINK_SEND_TIMEOUT, @@ -408,9 +406,7 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = self - .handle - .spawn(notifier.notify_blocks(stop_receiver.clone())); + let notifier_task = tokio::spawn(notifier.notify_blocks(stop_receiver.clone())); notifier_tasks.push(notifier_task); let notifier = PubSubNotifier { @@ -419,9 +415,7 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = self - .handle - .spawn(notifier.notify_txs(stop_receiver.clone())); + let notifier_task = tokio::spawn(notifier.notify_txs(stop_receiver.clone())); notifier_tasks.push(notifier_task); let notifier = PubSubNotifier { @@ -430,7 +424,7 @@ impl EthSubscribe { polling_interval, events_sender: self.events_sender.clone(), }; - let notifier_task = self.handle.spawn(notifier.notify_logs(stop_receiver)); + let notifier_task = tokio::spawn(notifier.notify_logs(stop_receiver)); notifier_tasks.push(notifier_task); notifier_tasks diff --git a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs index 92252f35bb76..50f3f69996f4 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests/mod.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests/mod.rs @@ -163,7 +163,6 @@ async fn spawn_server( } }; let server_handles = server_builder - .with_threads(1) .with_polling_interval(POLL_INTERVAL) .with_tx_sender(tx_sender, vm_barrier) .with_pub_sub_events(pub_sub_events_sender) diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 3b953eb91e84..7b5a9f933ee2 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -1074,7 +1074,6 @@ async fn run_http_api( .http(api_config.web3_json_rpc.http_port) .with_last_miniblock_pool(last_miniblock_pool) .with_filter_limit(api_config.web3_json_rpc.filters_limit()) - .with_threads(api_config.web3_json_rpc.http_server_threads()) .with_tree_api(api_config.web3_json_rpc.tree_api_url()) .with_batch_request_size_limit(api_config.web3_json_rpc.max_batch_request_size()) .with_response_body_size_limit(api_config.web3_json_rpc.max_response_body_size()) @@ -1128,7 +1127,6 @@ async fn run_ws_api( .websocket_requests_per_minute_limit(), ) .with_polling_interval(api_config.web3_json_rpc.pubsub_interval()) - .with_threads(api_config.web3_json_rpc.ws_server_threads()) .with_tree_api(api_config.web3_json_rpc.tree_api_url()) .with_tx_sender(tx_sender, vm_barrier) .enable_api_namespaces(namespaces); diff --git a/docs/guides/external-node/prepared_configs/mainnet-config.env b/docs/guides/external-node/prepared_configs/mainnet-config.env index 67679777fbd2..546e572f6cbc 100644 --- a/docs/guides/external-node/prepared_configs/mainnet-config.env +++ b/docs/guides/external-node/prepared_configs/mainnet-config.env @@ -41,8 +41,6 @@ EN_FILTERS_LIMIT=10000 EN_SUBSCRIPTIONS_LIMIT=10000 # Interval for polling the DB for pubsub (in ms). EN_PUBSUB_POLLING_INTERVAL=200 -# Number of threads per API server. -EN_THREADS_PER_SERVER=128 # Tx nonce: how far ahead from the committed nonce can it be. # This shouldn't be larger than the value on the main node (50). EN_MAX_NONCE_AHEAD=50 diff --git a/docs/guides/external-node/prepared_configs/testnet-goerli-config-deprecated.env b/docs/guides/external-node/prepared_configs/testnet-goerli-config-deprecated.env index b2e7e43c528b..e5c0ac947df8 100644 --- a/docs/guides/external-node/prepared_configs/testnet-goerli-config-deprecated.env +++ b/docs/guides/external-node/prepared_configs/testnet-goerli-config-deprecated.env @@ -41,8 +41,6 @@ EN_FILTERS_LIMIT=10000 EN_SUBSCRIPTIONS_LIMIT=10000 # Interval for polling the DB for pubsub (in ms). EN_PUBSUB_POLLING_INTERVAL=200 -# Number of threads per API server. -EN_THREADS_PER_SERVER=128 # Tx nonce: how far ahead from the committed nonce can it be. # This shouldn't be larger than the value on the main node (50). EN_MAX_NONCE_AHEAD=50 diff --git a/docs/guides/external-node/prepared_configs/testnet-sepolia-config.env b/docs/guides/external-node/prepared_configs/testnet-sepolia-config.env index 9019b847fb47..bc1898f89be3 100644 --- a/docs/guides/external-node/prepared_configs/testnet-sepolia-config.env +++ b/docs/guides/external-node/prepared_configs/testnet-sepolia-config.env @@ -41,8 +41,6 @@ EN_FILTERS_LIMIT=10000 EN_SUBSCRIPTIONS_LIMIT=10000 # Interval for polling the DB for pubsub (in ms). EN_PUBSUB_POLLING_INTERVAL=200 -# Number of threads per API server. -EN_THREADS_PER_SERVER=128 # Tx nonce: how far ahead from the committed nonce can it be. # This shouldn't be larger than the value on the main node (50). EN_MAX_NONCE_AHEAD=50 From 604fdb97507b1a93eedec10852e6487864024036 Mon Sep 17 00:00:00 2001 From: Dustin Brickwood Date: Tue, 9 Jan 2024 11:05:53 -0600 Subject: [PATCH 49/49] chore: adds development guide for zk spellcheck (#847) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Adds relevant docs related to spellchecker ## Why ❔ - Provides devs with info related to the tool and how it is used ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zk fmt` and `zk lint`. - [x] Spellcheck has been run via `zk spellcheck`. --- docs/guides/development.md | 51 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/docs/guides/development.md b/docs/guides/development.md index 955189cb3ab4..aa382d58d2f5 100644 --- a/docs/guides/development.md +++ b/docs/guides/development.md @@ -89,6 +89,57 @@ Currently the following criteria are checked: - Other code should always be formatted via `zk fmt`. - Dummy Prover should not be staged for commit (see below for the explanation). +## Spell Checking + +In our development workflow, we utilize a spell checking process to ensure the quality and accuracy of our documentation +and code comments. This is achieved using two primary tools: `cspell` and `cargo-spellcheck`. This section outlines how +to use these tools and configure them for your needs. + +### Using the Spellcheck Command + +The spell check command `zk spellcheck` is designed to check for spelling errors in our documentation and code. To run +the spell check, use the following command: + +``` +zk spellcheck +Options: +--pattern : Specifies the glob pattern for files to check. Default is docs/**/*. +--config : Path to the configuration file. Default is ./spellcheck/cspell.json. +--use-cargo: Utilize cargo spellcheck. +--use-cspell: Utilize cspell. +``` + +### General Rules + +**Code References in Comments**: When referring to code elements within development comments, they should be wrapped in +backticks. For example, reference a variable as `block_number`. + +**Code Blocks in Comments**: For larger blocks of pseudocode or commented-out code, use code blocks formatted as +follows: + +```` +// ``` +// let overhead_for_pubdata = { +// let numerator: U256 = overhead_for_block_gas * total_gas_limit +// + gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK); +// let denominator = +// gas_per_pubdata_byte_limit * U256::from(MAX_PUBDATA_PER_BLOCK) + overhead_for_block_gas; +// ``` +```` + +**Language Settings**: We use the Hunspell language setting of `en_US`. + +**CSpell Usage**: For spell checking within the `docs/` directory, we use `cspell`. The configuration for this tool is +found in `cspell.json`. It's tailored to check our documentation for spelling errors. + +**Cargo-Spellcheck for Rust and Dev Comments**: For Rust code and development comments, `cargo-spellcheck` is used. Its +configuration is maintained in `era.cfg`. + +### Adding Words to the Dictionary + +To add a new word to the spell checker dictionary, navigate to `/spellcheck/era.dic` and include the word. Ensure that +the word is relevant and necessary to be included in the dictionary to maintain the integrity of our documentation. + ## Using Dummy Prover By default, the chosen prover is a "dummy" one, meaning that it doesn't actually compute proofs but rather uses mocks to