From cd621f0dffaa4e9c83189b9bfad89d46700787b6 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Thu, 19 Sep 2024 15:10:07 +0300 Subject: [PATCH 01/14] Extract gas estimation into separate module ...and make it more efficient --- .../src/tx_sender/gas_estimation.rs | 406 ++++++++++++++++++ core/node/api_server/src/tx_sender/mod.rs | 345 +-------------- 2 files changed, 413 insertions(+), 338 deletions(-) create mode 100644 core/node/api_server/src/tx_sender/gas_estimation.rs diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs new file mode 100644 index 00000000000..c4239bb09d3 --- /dev/null +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -0,0 +1,406 @@ +use std::time::Instant; + +use anyhow::Context; +use zksync_dal::CoreDal; +use zksync_multivm::{ + interface::{ + OneshotTracingParams, TransactionExecutionMetrics, TxExecutionArgs, TxExecutionMode, + VmExecutionResultAndLogs, + }, + utils::{ + adjust_pubdata_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead, + get_max_batch_gas_limit, + }, + vm_latest::constants::BATCH_COMPUTATIONAL_GAS_LIMIT, + zk_evm_latest::ethereum_types::H256, + VmVersion, +}; +use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; +use zksync_types::{ + api::state_override::StateOverride, fee::Fee, fee_model::BatchFeeInput, get_code_key, + AccountTreeId, ExecuteTransactionCommon, PackedEthSignature, Transaction, +}; + +use super::{result::ApiCallResult, SubmitTxError, TxSender}; +use crate::execution_sandbox::{BlockArgs, TxSetupArgs, VmPermit, SANDBOX_METRICS}; + +impl TxSender { + /// Given the gas_limit to be used for the body of the transaction, + /// returns the result for executing the transaction with such gas_limit + #[allow(clippy::too_many_arguments)] + async fn estimate_gas_step( + &self, + vm_permit: VmPermit, + mut tx: Transaction, + tx_gas_limit: u64, + gas_price_per_pubdata: u32, + fee_model_params: BatchFeeInput, + block_args: BlockArgs, + base_fee: u64, + vm_version: VmVersion, + state_override: Option, + ) -> anyhow::Result<(VmExecutionResultAndLogs, TransactionExecutionMetrics)> { + let overhead = derive_overhead( + tx_gas_limit, + gas_price_per_pubdata, + tx.encoding_len(), + tx.tx_format() as u8, + vm_version, + ); + let gas_limit_with_overhead = tx_gas_limit + overhead as u64; + // We need to ensure that we never use a gas limit that is higher than the maximum allowed + let forced_gas_limit = gas_limit_with_overhead.min(get_max_batch_gas_limit(vm_version)); + + match &mut tx.common_data { + ExecuteTransactionCommon::L1(l1_common_data) => { + l1_common_data.gas_limit = forced_gas_limit.into(); + let required_funds = + l1_common_data.gas_limit * l1_common_data.max_fee_per_gas + tx.execute.value; + l1_common_data.to_mint = required_funds; + } + ExecuteTransactionCommon::L2(l2_common_data) => { + l2_common_data.fee.gas_limit = forced_gas_limit.into(); + } + ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { + common_data.gas_limit = forced_gas_limit.into(); + let required_funds = + common_data.gas_limit * common_data.max_fee_per_gas + tx.execute.value; + common_data.to_mint = required_funds; + } + } + + let setup_args = self.args_for_gas_estimate(fee_model_params, base_fee).await; + let execution_args = TxExecutionArgs::for_gas_estimate(tx); + let connection = self.acquire_replica_connection().await?; + let execution_output = self + .0 + .executor + .execute_tx_in_sandbox( + vm_permit, + setup_args, + execution_args, + connection, + block_args, + state_override, + OneshotTracingParams::default(), + ) + .await?; + Ok((execution_output.vm, execution_output.metrics)) + } + + async fn args_for_gas_estimate(&self, fee_input: BatchFeeInput, base_fee: u64) -> TxSetupArgs { + let config = &self.0.sender_config; + TxSetupArgs { + execution_mode: TxExecutionMode::EstimateFee, + operator_account: AccountTreeId::new(config.fee_account_addr), + fee_input, + // We want to bypass the computation gas limit check for gas estimation + validation_computational_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, + base_system_contracts: self.0.api_contracts.estimate_gas.clone(), + caches: self.storage_caches(), + chain_id: config.chain_id, + whitelisted_tokens_for_aa: self.read_whitelisted_tokens_for_aa_cache().await, + enforced_base_fee: Some(base_fee), + } + } + + #[tracing::instrument(level = "debug", skip_all, fields( + initiator = ?tx.initiator_account(), + nonce = ?tx.nonce(), + ))] + pub async fn get_txs_fee_in_wei( + &self, + mut tx: Transaction, + estimated_fee_scale_factor: f64, + acceptable_overestimation: u64, + state_override: Option, + ) -> Result { + let estimation_started_at = Instant::now(); + + let mut connection = self.acquire_replica_connection().await?; + let block_args = BlockArgs::pending(&mut connection).await?; + let protocol_version = connection + .blocks_dal() + .pending_protocol_version() + .await + .context("failed getting pending protocol version")?; + let max_gas_limit = get_max_batch_gas_limit(protocol_version.into()); + drop(connection); + + let fee_input = adjust_pubdata_price_for_tx( + self.scaled_batch_fee_input().await?, + tx.gas_per_pubdata_byte_limit(), + // We do not have to adjust the params to the `gasPrice` of the transaction, since + // its gas price will be amended later on to suit the `fee_input` + None, + 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(); + common_data.fee.max_priority_fee_per_gas = base_fee.into(); + } + ExecuteTransactionCommon::L1(common_data) => { + common_data.max_fee_per_gas = base_fee.into(); + } + ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { + common_data.max_fee_per_gas = base_fee.into(); + } + } + + self.ensure_sufficient_balance(&tx, state_override.as_ref()) + .await?; + + // For L2 transactions we need a properly formatted signature + if let ExecuteTransactionCommon::L2(l2_common_data) = &mut tx.common_data { + if l2_common_data.signature.is_empty() { + l2_common_data.signature = PackedEthSignature::default().serialize_packed().into(); + } + } + + // Acquire the vm token for the whole duration of the binary search. + let vm_permit = self.0.vm_concurrency_limiter.acquire().await; + let vm_permit = vm_permit.ok_or(SubmitTxError::ServerShuttingDown)?; + + // When the pubdata cost grows very high, the total gas limit required may become very high as well. If + // we do binary search over any possible gas limit naively, we may end up with a very high number of iterations, + // which affects performance. + // + // To optimize for this case, we first calculate the amount of gas needed to cover for the pubdata. After that, we + // need to do a smaller binary search that is focused on computational gas limit only. + let (gas_used, additional_gas_for_pubdata) = if tx.is_l1() { + // For L1 transactions the pubdata priced in such a way that the maximal computational + // gas limit should be enough to cover for the pubdata as well, so no additional gas is provided there. + (None, 0_u64) + } else { + // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. + // And getting how much pubdata it used. + + // In theory, if the transaction has failed with such large gas limit, we could have returned an API error here right away, + // but doing it later on keeps the code more lean. + let (result, _) = self + .estimate_gas_step( + vm_permit.clone(), + tx.clone(), + max_gas_limit, + gas_per_pubdata_byte as u32, + fee_input, + block_args, + base_fee, + protocol_version.into(), + state_override.clone(), + ) + .await + .context("estimate_gas step failed")?; + + // It is assumed that there is no overflow here + let gas_for_pubdata = + (result.statistics.pubdata_published as u64) * gas_per_pubdata_byte; + (Some(result.statistics.gas_used), gas_for_pubdata) + }; + + // We are using binary search to find the minimal values of gas_limit under which + // the transaction succeeds + let mut lower_bound = gas_used.unwrap_or(0); + let mut upper_bound = MAX_L2_TX_GAS_LIMIT; + tracing::trace!( + "preparation took {:?}, starting binary search", + estimation_started_at.elapsed() + ); + let mut number_of_iterations = 0_usize; + + if let Some(gas_used) = gas_used { + // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. + // If the transaction succeeds, it will discard most of the search space at once. + let iteration_started_at = Instant::now(); + let optimistic_gas_limit = + (gas_used * 6 / 5).min(upper_bound) + additional_gas_for_pubdata; + + let (result, _) = self + .estimate_gas_step( + vm_permit.clone(), + tx.clone(), + optimistic_gas_limit, + gas_per_pubdata_byte as u32, + fee_input, + block_args, + base_fee, + protocol_version.into(), + state_override.clone(), + ) + .await + .context("estimate_gas step failed")?; + Self::adjust_search_bounds( + &mut lower_bound, + &mut upper_bound, + optimistic_gas_limit, + &result, + ); + + tracing::trace!( + "iteration {number_of_iterations} took {:?}. lower_bound: {lower_bound}, upper_bound: {upper_bound}", + iteration_started_at.elapsed() + ); + number_of_iterations += 1; + } + + 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 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 = additional_gas_for_pubdata + mid; + let (result, _) = self + .estimate_gas_step( + vm_permit.clone(), + tx.clone(), + try_gas_limit, + gas_per_pubdata_byte as u32, + fee_input, + block_args, + base_fee, + protocol_version.into(), + state_override.clone(), + ) + .await + .context("estimate_gas step failed")?; + Self::adjust_search_bounds(&mut lower_bound, &mut upper_bound, try_gas_limit, &result); + + tracing::trace!( + "iteration {number_of_iterations} took {:?}. lower_bound: {lower_bound}, upper_bound: {upper_bound}", + iteration_started_at.elapsed() + ); + number_of_iterations += 1; + } + SANDBOX_METRICS + .estimate_gas_binary_search_iterations + .observe(number_of_iterations); + + let suggested_gas_limit = + ((upper_bound + additional_gas_for_pubdata) as f64 * estimated_fee_scale_factor) as u64; + let (result, tx_metrics) = self + .estimate_gas_step( + vm_permit, + tx.clone(), + suggested_gas_limit, + gas_per_pubdata_byte as u32, + fee_input, + block_args, + base_fee, + protocol_version.into(), + state_override, + ) + .await + .context("final estimate_gas step failed")?; + + result.into_api_call_result()?; + self.ensure_tx_executable(&tx, &tx_metrics, false)?; + + // Now, we need to calculate the final overhead for the transaction. + let overhead = derive_overhead( + suggested_gas_limit, + gas_per_pubdata_byte as u32, + tx.encoding_len(), + tx.tx_format() as u8, + protocol_version.into(), + ); + + let full_gas_limit = match suggested_gas_limit.overflowing_add(overhead.into()) { + (value, false) => { + if value > max_gas_limit { + return Err(SubmitTxError::ExecutionReverted( + "exceeds block gas limit".to_string(), + vec![], + )); + } + + value + } + (_, true) => { + return Err(SubmitTxError::ExecutionReverted( + "exceeds block gas limit".to_string(), + vec![], + )); + } + }; + + let gas_for_pubdata = u64::from(tx_metrics.pubdata_published) * gas_per_pubdata_byte; + let estimated_gas_for_pubdata = + (gas_for_pubdata as f64 * estimated_fee_scale_factor) as u64; + + tracing::debug!( + "gas for pubdata: {estimated_gas_for_pubdata}, computational gas: {}, overhead gas: {overhead} \ + (with params base_fee: {base_fee}, gas_per_pubdata_byte: {gas_per_pubdata_byte}) \ + estimated_fee_scale_factor: {estimated_fee_scale_factor}", + suggested_gas_limit - estimated_gas_for_pubdata, + ); + + Ok(Fee { + max_fee_per_gas: base_fee.into(), + max_priority_fee_per_gas: 0u32.into(), + gas_limit: full_gas_limit.into(), + gas_per_pubdata_limit: gas_per_pubdata_byte.into(), + }) + } + + async fn ensure_sufficient_balance( + &self, + tx: &Transaction, + state_override: Option<&StateOverride>, + ) -> Result<(), SubmitTxError> { + let hashed_key = get_code_key(&tx.initiator_account()); + // If the default account does not have enough funds for transferring `tx.value`, without taking into account the fee, + // there is no sense to estimate the fee. + let account_code_hash = self + .acquire_replica_connection() + .await? + .storage_web3_dal() + .get_value(&hashed_key) + .await + .with_context(|| { + format!( + "failed getting code hash for account {:?}", + tx.initiator_account() + ) + })?; + + if !tx.is_l1() && account_code_hash == H256::zero() { + let balance = match state_override + .and_then(|overrides| overrides.get(&tx.initiator_account())) + .and_then(|account| account.balance) + { + Some(balance) => balance, + None => self.get_balance(&tx.initiator_account()).await?, + }; + + if tx.execute.value > balance { + tracing::info!( + "fee estimation failed on validation step. + account: {} does not have enough funds for for transferring tx.value: {}.", + tx.initiator_account(), + tx.execute.value + ); + return Err(SubmitTxError::InsufficientFundsForTransfer); + } + } + Ok(()) + } + + fn adjust_search_bounds( + lower_bound: &mut u64, + upper_bound: &mut u64, + pivot: u64, + result: &VmExecutionResultAndLogs, + ) { + // For now, we don't discern between "out of gas" and other failure reasons since it's difficult in the general case. + if result.result.is_failed() { + *lower_bound = pivot + 1; + } else { + *upper_bound = pivot; + } + } +} diff --git a/core/node/api_server/src/tx_sender/mod.rs b/core/node/api_server/src/tx_sender/mod.rs index 44eaae2e3ee..15dabac7d3d 100644 --- a/core/node/api_server/src/tx_sender/mod.rs +++ b/core/node/api_server/src/tx_sender/mod.rs @@ -1,6 +1,6 @@ //! Helper module to submit transactions into the ZKsync Network. -use std::{sync::Arc, time::Instant}; +use std::sync::Arc; use anyhow::Context as _; use tokio::sync::RwLock; @@ -14,11 +14,7 @@ use zksync_multivm::{ OneshotTracingParams, TransactionExecutionMetrics, TxExecutionArgs, TxExecutionMode, VmExecutionResultAndLogs, }, - utils::{ - adjust_pubdata_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead, - get_max_batch_gas_limit, - }, - vm_latest::constants::BATCH_COMPUTATIONAL_GAS_LIMIT, + utils::{derive_base_fee_and_gas_per_pubdata, get_max_batch_gas_limit}, }; use zksync_node_fee_model::{ApiFeeInputProvider, BatchFeeModelInputProvider}; use zksync_state::PostgresStorageCaches; @@ -28,15 +24,13 @@ use zksync_state_keeper::{ }; use zksync_types::{ api::state_override::StateOverride, - fee::Fee, fee_model::BatchFeeInput, - get_code_key, get_intrinsic_constants, + get_intrinsic_constants, l2::{error::TxCheckError::TxDuplication, L2Tx}, transaction_request::CallOverrides, utils::storage_key_for_eth_balance, - vm::VmVersion, - AccountTreeId, Address, ExecuteTransactionCommon, L2ChainId, Nonce, PackedEthSignature, - ProtocolVersionId, Transaction, H160, H256, MAX_L2_TX_GAS_LIMIT, MAX_NEW_FACTORY_DEPS, U256, + AccountTreeId, Address, L2ChainId, Nonce, ProtocolVersionId, Transaction, H160, H256, + MAX_NEW_FACTORY_DEPS, U256, }; use zksync_utils::h256_to_u256; @@ -45,11 +39,12 @@ use self::{master_pool_sink::MasterPoolSink, tx_sink::TxSink}; use crate::{ execution_sandbox::{ BlockArgs, SubmitTxStage, TransactionExecutor, TxSetupArgs, VmConcurrencyBarrier, - VmConcurrencyLimiter, VmPermit, SANDBOX_METRICS, + VmConcurrencyLimiter, SANDBOX_METRICS, }, tx_sender::result::ApiCallResult, }; +mod gas_estimation; pub mod master_pool_sink; pub mod proxy; mod result; @@ -677,332 +672,6 @@ impl TxSender { Ok(h256_to_u256(balance)) } - /// Given the gas_limit to be used for the body of the transaction, - /// returns the result for executing the transaction with such gas_limit - #[allow(clippy::too_many_arguments)] - async fn estimate_gas_step( - &self, - vm_permit: VmPermit, - mut tx: Transaction, - tx_gas_limit: u64, - gas_price_per_pubdata: u32, - fee_model_params: BatchFeeInput, - block_args: BlockArgs, - base_fee: u64, - vm_version: VmVersion, - state_override: Option, - ) -> anyhow::Result<(VmExecutionResultAndLogs, TransactionExecutionMetrics)> { - let gas_limit_with_overhead = tx_gas_limit - + derive_overhead( - tx_gas_limit, - gas_price_per_pubdata, - tx.encoding_len(), - tx.tx_format() as u8, - vm_version, - ) as u64; - // We need to ensure that we never use a gas limit that is higher than the maximum allowed - let forced_gas_limit = gas_limit_with_overhead.min(get_max_batch_gas_limit(vm_version)); - - match &mut tx.common_data { - ExecuteTransactionCommon::L1(l1_common_data) => { - l1_common_data.gas_limit = forced_gas_limit.into(); - let required_funds = - l1_common_data.gas_limit * l1_common_data.max_fee_per_gas + tx.execute.value; - l1_common_data.to_mint = required_funds; - } - ExecuteTransactionCommon::L2(l2_common_data) => { - l2_common_data.fee.gas_limit = forced_gas_limit.into(); - } - ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { - common_data.gas_limit = forced_gas_limit.into(); - - let required_funds = - common_data.gas_limit * common_data.max_fee_per_gas + tx.execute.value; - - common_data.to_mint = required_funds; - } - } - - let setup_args = self.args_for_gas_estimate(fee_model_params, base_fee).await; - let execution_args = TxExecutionArgs::for_gas_estimate(tx); - let connection = self.acquire_replica_connection().await?; - let execution_output = self - .0 - .executor - .execute_tx_in_sandbox( - vm_permit, - setup_args, - execution_args, - connection, - block_args, - state_override, - OneshotTracingParams::default(), - ) - .await?; - Ok((execution_output.vm, execution_output.metrics)) - } - - async fn args_for_gas_estimate(&self, fee_input: BatchFeeInput, base_fee: u64) -> TxSetupArgs { - let config = &self.0.sender_config; - TxSetupArgs { - execution_mode: TxExecutionMode::EstimateFee, - operator_account: AccountTreeId::new(config.fee_account_addr), - fee_input, - // We want to bypass the computation gas limit check for gas estimation - validation_computational_gas_limit: BATCH_COMPUTATIONAL_GAS_LIMIT, - base_system_contracts: self.0.api_contracts.estimate_gas.clone(), - caches: self.storage_caches(), - chain_id: config.chain_id, - whitelisted_tokens_for_aa: self.read_whitelisted_tokens_for_aa_cache().await, - enforced_base_fee: Some(base_fee), - } - } - - #[tracing::instrument(level = "debug", skip_all, fields( - initiator = ?tx.initiator_account(), - nonce = ?tx.nonce(), - ))] - pub async fn get_txs_fee_in_wei( - &self, - mut tx: Transaction, - estimated_fee_scale_factor: f64, - acceptable_overestimation: u64, - state_override: Option, - ) -> Result { - let estimation_started_at = Instant::now(); - - let mut connection = self.acquire_replica_connection().await?; - let block_args = BlockArgs::pending(&mut connection).await?; - let protocol_version = connection - .blocks_dal() - .pending_protocol_version() - .await - .context("failed getting pending protocol version")?; - let max_gas_limit = get_max_batch_gas_limit(protocol_version.into()); - drop(connection); - - let fee_input = adjust_pubdata_price_for_tx( - self.scaled_batch_fee_input().await?, - tx.gas_per_pubdata_byte_limit(), - // We do not have to adjust the params to the `gasPrice` of the transaction, since - // its gas price will be amended later on to suit the `fee_input` - None, - 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(); - common_data.fee.max_priority_fee_per_gas = base_fee.into(); - } - ExecuteTransactionCommon::L1(common_data) => { - common_data.max_fee_per_gas = base_fee.into(); - } - ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { - common_data.max_fee_per_gas = base_fee.into(); - } - } - - let hashed_key = get_code_key(&tx.initiator_account()); - // If the default account does not have enough funds for transferring `tx.value`, without taking into account the fee, - // there is no sense to estimate the fee. - let account_code_hash = self - .acquire_replica_connection() - .await? - .storage_web3_dal() - .get_value(&hashed_key) - .await - .with_context(|| { - format!( - "failed getting code hash for account {:?}", - tx.initiator_account() - ) - })?; - - if !tx.is_l1() && account_code_hash == H256::zero() { - let balance = match state_override - .as_ref() - .and_then(|overrides| overrides.get(&tx.initiator_account())) - .and_then(|account| account.balance) - { - Some(balance) => balance, - None => self.get_balance(&tx.initiator_account()).await?, - }; - - if tx.execute.value > balance { - tracing::info!( - "fee estimation failed on validation step. - account: {} does not have enough funds for for transferring tx.value: {}.", - tx.initiator_account(), - tx.execute.value - ); - return Err(SubmitTxError::InsufficientFundsForTransfer); - } - } - - // For L2 transactions we need a properly formatted signature - if let ExecuteTransactionCommon::L2(l2_common_data) = &mut tx.common_data { - if l2_common_data.signature.is_empty() { - l2_common_data.signature = PackedEthSignature::default().serialize_packed().into(); - } - } - - // Acquire the vm token for the whole duration of the binary search. - let vm_permit = self.0.vm_concurrency_limiter.acquire().await; - let vm_permit = vm_permit.ok_or(SubmitTxError::ServerShuttingDown)?; - - // When the pubdata cost grows very high, the total gas limit required may become very high as well. If - // we do binary search over any possible gas limit naively, we may end up with a very high number of iterations, - // which affects performance. - // - // To optimize for this case, we first calculate the amount of gas needed to cover for the pubdata. After that, we - // need to do a smaller binary search that is focused on computational gas limit only. - let additional_gas_for_pubdata = if tx.is_l1() { - // For L1 transactions the pubdata priced in such a way that the maximal computational - // gas limit should be enough to cover for the pubdata as well, so no additional gas is provided there. - 0u64 - } else { - // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. - // And getting how much pubdata it used. - - // In theory, if the transaction has failed with such large gas limit, we could have returned an API error here right away, - // but doing it later on keeps the code more lean. - let (result, _) = self - .estimate_gas_step( - vm_permit.clone(), - tx.clone(), - max_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override.clone(), - ) - .await - .context("estimate_gas step failed")?; - - // It is assumed that there is no overflow here - (result.statistics.pubdata_published as u64) * gas_per_pubdata_byte - }; - - // We are using binary search to find the minimal values of gas_limit under which - // the transaction succeeds - let mut lower_bound = 0; - let mut upper_bound = MAX_L2_TX_GAS_LIMIT; - tracing::trace!( - "preparation took {:?}, starting binary search", - estimation_started_at.elapsed() - ); - - let mut number_of_iterations = 0usize; - 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 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 = additional_gas_for_pubdata + mid; - let (result, _) = self - .estimate_gas_step( - vm_permit.clone(), - tx.clone(), - try_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override.clone(), - ) - .await - .context("estimate_gas step failed")?; - - if result.result.is_failed() { - lower_bound = mid + 1; - } else { - upper_bound = mid; - } - - tracing::trace!( - "iteration {number_of_iterations} took {:?}. lower_bound: {lower_bound}, upper_bound: {upper_bound}", - iteration_started_at.elapsed() - ); - number_of_iterations += 1; - } - SANDBOX_METRICS - .estimate_gas_binary_search_iterations - .observe(number_of_iterations); - - let suggested_gas_limit = - ((upper_bound + additional_gas_for_pubdata) as f64 * estimated_fee_scale_factor) as u64; - let (result, tx_metrics) = self - .estimate_gas_step( - vm_permit, - tx.clone(), - suggested_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override, - ) - .await - .context("final estimate_gas step failed")?; - - result.into_api_call_result()?; - self.ensure_tx_executable(&tx, &tx_metrics, false)?; - - // Now, we need to calculate the final overhead for the transaction. - let overhead = derive_overhead( - suggested_gas_limit, - gas_per_pubdata_byte as u32, - tx.encoding_len(), - tx.tx_format() as u8, - protocol_version.into(), - ) as u64; - - let full_gas_limit = match suggested_gas_limit.overflowing_add(overhead) { - (value, false) => { - if value > max_gas_limit { - return Err(SubmitTxError::ExecutionReverted( - "exceeds block gas limit".to_string(), - vec![], - )); - } - - value - } - (_, true) => { - return Err(SubmitTxError::ExecutionReverted( - "exceeds block gas limit".to_string(), - vec![], - )); - } - }; - - let gas_for_pubdata = (tx_metrics.pubdata_published as u64) * gas_per_pubdata_byte; - let estimated_gas_for_pubdata = - (gas_for_pubdata as f64 * estimated_fee_scale_factor) as u64; - - tracing::debug!( - "gas for pubdata: {estimated_gas_for_pubdata}, computational gas: {}, overhead gas: {overhead} \ - (with params base_fee: {base_fee}, gas_per_pubdata_byte: {gas_per_pubdata_byte}) \ - estimated_fee_scale_factor: {estimated_fee_scale_factor}", - suggested_gas_limit - estimated_gas_for_pubdata, - ); - - Ok(Fee { - max_fee_per_gas: base_fee.into(), - max_priority_fee_per_gas: 0u32.into(), - gas_limit: full_gas_limit.into(), - gas_per_pubdata_limit: gas_per_pubdata_byte.into(), - }) - } - // For now, both L1 gas price and pubdata price are scaled with the same coefficient async fn scaled_batch_fee_input(&self) -> anyhow::Result { self.0 From 5f760b96a56cf339bea073c199c011512299f412 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Thu, 19 Sep 2024 16:11:38 +0300 Subject: [PATCH 02/14] Sketch fee estimation test --- .../api_server/src/execution_sandbox/tests.rs | 32 ++------- core/node/api_server/src/lib.rs | 2 + core/node/api_server/src/testonly.rs | 28 ++++++++ .../src/tx_sender/gas_estimation.rs | 25 ++++--- core/node/api_server/src/tx_sender/tests.rs | 70 ++++++++++++++++++- 5 files changed, 115 insertions(+), 42 deletions(-) create mode 100644 core/node/api_server/src/testonly.rs diff --git a/core/node/api_server/src/execution_sandbox/tests.rs b/core/node/api_server/src/execution_sandbox/tests.rs index 79c5a733038..70eda6cfab4 100644 --- a/core/node/api_server/src/execution_sandbox/tests.rs +++ b/core/node/api_server/src/execution_sandbox/tests.rs @@ -17,15 +17,12 @@ use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, prepare_recovery_snapshot}; use zksync_types::{ api::state_override::{OverrideAccount, StateOverride}, - fee::Fee, - l2::L2Tx, - transaction_request::PaymasterParams, - K256PrivateKey, Nonce, ProtocolVersionId, Transaction, U256, + ProtocolVersionId, Transaction, U256, }; use zksync_vm_executor::oneshot::MainOneshotExecutor; use super::{storage::StorageWithOverrides, *}; -use crate::tx_sender::ApiContracts; +use crate::{testonly::create_transfer, tx_sender::ApiContracts}; #[tokio::test] async fn creating_block_args() { @@ -210,7 +207,7 @@ async fn test_instantiating_vm(connection: Connection<'static, Core>, block_args ProtocolVersionId::latest().into(), ); setup_args.enforced_base_fee = Some(base_fee); - let transaction = Transaction::from(create_transfer(base_fee, gas_per_pubdata)); + let transaction = Transaction::from(create_transfer(0.into(), base_fee, gas_per_pubdata)); let execution_args = TxExecutionArgs::for_gas_estimate(transaction.clone()); let (env, storage) = apply::prepare_env_and_storage(connection, setup_args, &block_args) @@ -228,27 +225,6 @@ async fn test_instantiating_vm(connection: Connection<'static, Core>, block_args assert!(!tx_result.result.is_failed(), "{tx_result:#?}"); } -fn create_transfer(fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { - let fee = Fee { - gas_limit: 200_000.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(), - }; - L2Tx::new_signed( - Some(Address::random()), - vec![], - Nonce(0), - fee, - U256::zero(), - L2ChainId::default(), - &K256PrivateKey::random(), - vec![], - PaymasterParams::default(), - ) - .unwrap() -} - #[test_casing(2, [false, true])] #[tokio::test] async fn validating_transaction(set_balance: bool) { @@ -267,7 +243,7 @@ async fn validating_transaction(set_balance: bool) { ProtocolVersionId::latest().into(), ); setup_args.enforced_base_fee = Some(base_fee); - let transaction = create_transfer(base_fee, gas_per_pubdata); + let transaction = create_transfer(0.into(), base_fee, gas_per_pubdata); let validation_params = validate::get_validation_params(&mut connection, &transaction, u32::MAX, &[]) diff --git a/core/node/api_server/src/lib.rs b/core/node/api_server/src/lib.rs index 00a3a5632ce..c735bd50447 100644 --- a/core/node/api_server/src/lib.rs +++ b/core/node/api_server/src/lib.rs @@ -4,5 +4,7 @@ mod utils; pub mod execution_sandbox; pub mod healthcheck; +#[cfg(test)] +mod testonly; pub mod tx_sender; pub mod web3; diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs new file mode 100644 index 00000000000..c8fe78adab6 --- /dev/null +++ b/core/node/api_server/src/testonly.rs @@ -0,0 +1,28 @@ +//! Test utils shared among multiple modules. + +use zksync_multivm::zk_evm_latest::ethereum_types::{Address, U256}; +use zksync_types::{ + fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, K256PrivateKey, L2ChainId, Nonce, +}; + +/// Creates a correctly signed transfer transaction. +pub(crate) fn create_transfer(value: U256, fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { + let fee = Fee { + gas_limit: 200_000.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(), + }; + L2Tx::new_signed( + Some(Address::random()), + vec![], + Nonce(0), + fee, + value, + L2ChainId::default(), + &K256PrivateKey::random(), + vec![], + PaymasterParams::default(), + ) + .unwrap() +} diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index c4239bb09d3..c872c0315b9 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -198,12 +198,12 @@ impl TxSender { // It is assumed that there is no overflow here let gas_for_pubdata = - (result.statistics.pubdata_published as u64) * gas_per_pubdata_byte; + u64::from(result.statistics.pubdata_published) * gas_per_pubdata_byte; (Some(result.statistics.gas_used), gas_for_pubdata) }; - // We are using binary search to find the minimal values of gas_limit under which - // the transaction succeeds + // We are using binary search to find the minimal values of gas_limit under which the transaction succeeds. + // **Important.** Lower and upper bound do not include `additional_gas_for_pubdata`; it needs to be added when calling `estimate_gas_step()`. let mut lower_bound = gas_used.unwrap_or(0); let mut upper_bound = MAX_L2_TX_GAS_LIMIT; tracing::trace!( @@ -212,18 +212,21 @@ impl TxSender { ); let mut number_of_iterations = 0_usize; - if let Some(gas_used) = gas_used { - // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. - // If the transaction succeeds, it will discard most of the search space at once. - let iteration_started_at = Instant::now(); - let optimistic_gas_limit = - (gas_used * 6 / 5).min(upper_bound) + additional_gas_for_pubdata; + // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. + // If the transaction succeeds, it will discard most of the search space at once. + let optimistic_gas_limit = gas_used.and_then(|gas_used| { + let optimistic_gas_limit = gas_used * 6 / 5; + // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. + (optimistic_gas_limit < (lower_bound + upper_bound) / 2).then_some(optimistic_gas_limit) + }); + if let Some(optimistic_gas_limit) = optimistic_gas_limit { + let iteration_started_at = Instant::now(); let (result, _) = self .estimate_gas_step( vm_permit.clone(), tx.clone(), - optimistic_gas_limit, + optimistic_gas_limit + additional_gas_for_pubdata, gas_per_pubdata_byte as u32, fee_input, block_args, @@ -268,7 +271,7 @@ impl TxSender { ) .await .context("estimate_gas step failed")?; - Self::adjust_search_bounds(&mut lower_bound, &mut upper_bound, try_gas_limit, &result); + Self::adjust_search_bounds(&mut lower_bound, &mut upper_bound, mid, &result); tracing::trace!( "iteration {number_of_iterations} took {:?}. lower_bound: {lower_bound}, upper_bound: {upper_bound}", diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index 0ac3eb0b4f3..178688aa31f 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -1,18 +1,24 @@ //! Tests for the transaction sender. -use std::time::Duration; +use std::{collections::HashMap, time::Duration}; use assert_matches::assert_matches; use zksync_multivm::interface::ExecutionResult; use zksync_node_fee_model::MockBatchFeeParamsProvider; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; -use zksync_types::{api, get_nonce_key, L1BatchNumber, L2BlockNumber, StorageLog}; +use zksync_types::{ + api, api::state_override::OverrideAccount, get_nonce_key, L1BatchNumber, L2BlockNumber, + StorageLog, +}; use zksync_utils::u256_to_h256; use zksync_vm_executor::oneshot::MockOneshotExecutor; use super::*; -use crate::{execution_sandbox::BlockStartInfo, web3::testonly::create_test_tx_sender}; +use crate::{ + execution_sandbox::BlockStartInfo, testonly::create_transfer, + web3::testonly::create_test_tx_sender, +}; #[tokio::test] async fn getting_nonce_for_account() { @@ -200,3 +206,61 @@ async fn eth_call_requires_single_connection() { .unwrap(); assert_eq!(output, b"success!"); } + +#[tokio::test] +async fn estimating_gas_for_transfer() { + let pool = ConnectionPool::::constrained_test_pool(1).await; + let mut storage = pool.connection().await.unwrap(); + let genesis_params = GenesisParams::mock(); + insert_genesis_batch(&mut storage, &genesis_params) + .await + .unwrap(); + drop(storage); + + let tx_executor = TransactionExecutor::real(usize::MAX); + let (tx_sender, _) = create_test_tx_sender( + pool.clone(), + genesis_params.config().l2_chain_id, + tx_executor, + ) + .await; + + let value = 1_000_000_000.into(); + let tx = create_transfer(value, 55, 555); // fee params don't matter; they should be overwritten by the estimation logic + let fee_scale_factor = 1.0; + let acceptable_overestimation = 1_000; + // Without overrides, the transaction should fail because of insufficient balance. + let err = tx_sender + .get_txs_fee_in_wei( + tx.clone().into(), + fee_scale_factor, + acceptable_overestimation, + None, + ) + .await + .unwrap_err(); + assert_matches!(err, SubmitTxError::InsufficientFundsForTransfer); + + let account_overrides = OverrideAccount { + balance: Some(value * 2), + ..OverrideAccount::default() + }; + let call_overrides = + StateOverride::new(HashMap::from([(tx.initiator_account(), account_overrides)])); + let fee = tx_sender + .get_txs_fee_in_wei( + tx.into(), + fee_scale_factor, + acceptable_overestimation, + Some(call_overrides), + ) + .await + .unwrap(); + // Sanity-check gas limit + assert!( + fee.gas_limit > 1_000.into() && fee.gas_limit < 1_000_000.into(), + "{fee:?}" + ); + + // FIXME: check that transaction with this limit doesn't fail & fails with a lower one +} From 8fa763c8113ae3c0bccdfdfe5b4a70bfa387b1f9 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Fri, 20 Sep 2024 16:37:23 +0300 Subject: [PATCH 03/14] Modularize gas estimation --- .../api_server/src/execution_sandbox/tests.rs | 10 +- core/node/api_server/src/testonly.rs | 53 +- .../src/tx_sender/gas_estimation.rs | 625 ++++++++++-------- core/node/api_server/src/tx_sender/mod.rs | 13 +- core/node/api_server/src/tx_sender/tests.rs | 116 +++- .../api_server/src/web3/namespaces/eth.rs | 2 + .../api_server/src/web3/namespaces/zks.rs | 2 + 7 files changed, 511 insertions(+), 310 deletions(-) diff --git a/core/node/api_server/src/execution_sandbox/tests.rs b/core/node/api_server/src/execution_sandbox/tests.rs index 70eda6cfab4..e4c5a757e2c 100644 --- a/core/node/api_server/src/execution_sandbox/tests.rs +++ b/core/node/api_server/src/execution_sandbox/tests.rs @@ -17,12 +17,12 @@ use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, prepare_recovery_snapshot}; use zksync_types::{ api::state_override::{OverrideAccount, StateOverride}, - ProtocolVersionId, Transaction, U256, + K256PrivateKey, ProtocolVersionId, Transaction, U256, }; use zksync_vm_executor::oneshot::MainOneshotExecutor; use super::{storage::StorageWithOverrides, *}; -use crate::{testonly::create_transfer, tx_sender::ApiContracts}; +use crate::{testonly::TestAccount, tx_sender::ApiContracts}; #[tokio::test] async fn creating_block_args() { @@ -207,7 +207,8 @@ async fn test_instantiating_vm(connection: Connection<'static, Core>, block_args ProtocolVersionId::latest().into(), ); setup_args.enforced_base_fee = Some(base_fee); - let transaction = Transaction::from(create_transfer(0.into(), base_fee, gas_per_pubdata)); + let alice = K256PrivateKey::random(); + let transaction = Transaction::from(alice.create_transfer(0.into(), base_fee, gas_per_pubdata)); let execution_args = TxExecutionArgs::for_gas_estimate(transaction.clone()); let (env, storage) = apply::prepare_env_and_storage(connection, setup_args, &block_args) @@ -243,7 +244,8 @@ async fn validating_transaction(set_balance: bool) { ProtocolVersionId::latest().into(), ); setup_args.enforced_base_fee = Some(base_fee); - let transaction = create_transfer(0.into(), base_fee, gas_per_pubdata); + let alice = K256PrivateKey::random(); + let transaction = alice.create_transfer(0.into(), base_fee, gas_per_pubdata); let validation_params = validate::get_validation_params(&mut connection, &transaction, u32::MAX, &[]) diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index c8fe78adab6..c74c127eb10 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -1,28 +1,37 @@ //! Test utils shared among multiple modules. -use zksync_multivm::zk_evm_latest::ethereum_types::{Address, U256}; use zksync_types::{ - fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, K256PrivateKey, L2ChainId, Nonce, + fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, Address, K256PrivateKey, L2ChainId, + Nonce, U256, }; -/// Creates a correctly signed transfer transaction. -pub(crate) fn create_transfer(value: U256, fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { - let fee = Fee { - gas_limit: 200_000.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(), - }; - L2Tx::new_signed( - Some(Address::random()), - vec![], - Nonce(0), - fee, - value, - L2ChainId::default(), - &K256PrivateKey::random(), - vec![], - PaymasterParams::default(), - ) - .unwrap() +pub(crate) trait TestAccount { + fn create_transfer(&self, value: U256, fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { + let fee = Fee { + gas_limit: 200_000.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(), + }; + self.create_transfer_with_fee(value, fee) + } + + fn create_transfer_with_fee(&self, value: U256, fee: Fee) -> L2Tx; +} + +impl TestAccount for K256PrivateKey { + fn create_transfer_with_fee(&self, value: U256, fee: Fee) -> L2Tx { + L2Tx::new_signed( + Some(Address::random()), + vec![], + Nonce(0), + fee, + value, + L2ChainId::default(), + self, + vec![], + PaymasterParams::default(), + ) + .unwrap() + } } diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index c872c0315b9..984801d0826 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -1,4 +1,4 @@ -use std::time::Instant; +use std::{ops, time::Instant}; use anyhow::Context; use zksync_dal::CoreDal; @@ -13,81 +13,26 @@ use zksync_multivm::{ }, vm_latest::constants::BATCH_COMPUTATIONAL_GAS_LIMIT, zk_evm_latest::ethereum_types::H256, - VmVersion, }; use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{ api::state_override::StateOverride, fee::Fee, fee_model::BatchFeeInput, get_code_key, - AccountTreeId, ExecuteTransactionCommon, PackedEthSignature, Transaction, + AccountTreeId, ExecuteTransactionCommon, PackedEthSignature, ProtocolVersionId, Transaction, }; use super::{result::ApiCallResult, SubmitTxError, TxSender}; use crate::execution_sandbox::{BlockArgs, TxSetupArgs, VmPermit, SANDBOX_METRICS}; -impl TxSender { - /// Given the gas_limit to be used for the body of the transaction, - /// returns the result for executing the transaction with such gas_limit - #[allow(clippy::too_many_arguments)] - async fn estimate_gas_step( - &self, - vm_permit: VmPermit, - mut tx: Transaction, - tx_gas_limit: u64, - gas_price_per_pubdata: u32, - fee_model_params: BatchFeeInput, - block_args: BlockArgs, - base_fee: u64, - vm_version: VmVersion, - state_override: Option, - ) -> anyhow::Result<(VmExecutionResultAndLogs, TransactionExecutionMetrics)> { - let overhead = derive_overhead( - tx_gas_limit, - gas_price_per_pubdata, - tx.encoding_len(), - tx.tx_format() as u8, - vm_version, - ); - let gas_limit_with_overhead = tx_gas_limit + overhead as u64; - // We need to ensure that we never use a gas limit that is higher than the maximum allowed - let forced_gas_limit = gas_limit_with_overhead.min(get_max_batch_gas_limit(vm_version)); - - match &mut tx.common_data { - ExecuteTransactionCommon::L1(l1_common_data) => { - l1_common_data.gas_limit = forced_gas_limit.into(); - let required_funds = - l1_common_data.gas_limit * l1_common_data.max_fee_per_gas + tx.execute.value; - l1_common_data.to_mint = required_funds; - } - ExecuteTransactionCommon::L2(l2_common_data) => { - l2_common_data.fee.gas_limit = forced_gas_limit.into(); - } - ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { - common_data.gas_limit = forced_gas_limit.into(); - let required_funds = - common_data.gas_limit * common_data.max_fee_per_gas + tx.execute.value; - common_data.to_mint = required_funds; - } - } - - let setup_args = self.args_for_gas_estimate(fee_model_params, base_fee).await; - let execution_args = TxExecutionArgs::for_gas_estimate(tx); - let connection = self.acquire_replica_connection().await?; - let execution_output = self - .0 - .executor - .execute_tx_in_sandbox( - vm_permit, - setup_args, - execution_args, - connection, - block_args, - state_override, - OneshotTracingParams::default(), - ) - .await?; - Ok((execution_output.vm, execution_output.metrics)) - } +#[derive(Debug, Clone, Copy)] +pub(crate) enum BinarySearchKind { + /// Full binary search. + Full, + /// Binary search with an optimized initial pivot. + #[allow(dead_code)] // FIXME + Optimized, +} +impl TxSender { async fn args_for_gas_estimate(&self, fee_input: BatchFeeInput, base_fee: u64) -> TxSetupArgs { let config = &self.0.sender_config; TxSetupArgs { @@ -110,138 +55,73 @@ impl TxSender { ))] pub async fn get_txs_fee_in_wei( &self, - mut tx: Transaction, + tx: Transaction, estimated_fee_scale_factor: f64, acceptable_overestimation: u64, state_override: Option, + kind: BinarySearchKind, ) -> Result { let estimation_started_at = Instant::now(); + let mut estimator = GasEstimator::new(self, tx, state_override).await?; + estimator.adjust_transaction_fee(); - let mut connection = self.acquire_replica_connection().await?; - let block_args = BlockArgs::pending(&mut connection).await?; - let protocol_version = connection - .blocks_dal() - .pending_protocol_version() - .await - .context("failed getting pending protocol version")?; - let max_gas_limit = get_max_batch_gas_limit(protocol_version.into()); - drop(connection); - - let fee_input = adjust_pubdata_price_for_tx( - self.scaled_batch_fee_input().await?, - tx.gas_per_pubdata_byte_limit(), - // We do not have to adjust the params to the `gasPrice` of the transaction, since - // its gas price will be amended later on to suit the `fee_input` - None, - protocol_version.into(), + let initial_estimate = estimator.initialize().await?; + tracing::trace!( + "preparation took {:?}, starting binary search", + estimation_started_at.elapsed() ); - 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(); - common_data.fee.max_priority_fee_per_gas = base_fee.into(); - } - ExecuteTransactionCommon::L1(common_data) => { - common_data.max_fee_per_gas = base_fee.into(); - } - ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { - common_data.max_fee_per_gas = base_fee.into(); + let (bounds, initial_pivot) = match kind { + BinarySearchKind::Full => { + let lower_bound = initial_estimate.gas_charged_for_pubdata; + let upper_bound = MAX_L2_TX_GAS_LIMIT; + (lower_bound..=upper_bound, None) } - } - - self.ensure_sufficient_balance(&tx, state_override.as_ref()) - .await?; - - // For L2 transactions we need a properly formatted signature - if let ExecuteTransactionCommon::L2(l2_common_data) = &mut tx.common_data { - if l2_common_data.signature.is_empty() { - l2_common_data.signature = PackedEthSignature::default().serialize_packed().into(); + BinarySearchKind::Optimized => { + let lower_bound = initial_estimate + .lower_gas_bound_without_overhead() + .unwrap_or(0); + let upper_bound = MAX_L2_TX_GAS_LIMIT; + + // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. + // If the transaction succeeds, it will discard most of the search space at once. + let optimistic_gas_limit = initial_estimate + .optimistic_gas_limit_without_overhead() + .filter(|&gas| { + // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. + gas < (lower_bound + upper_bound) / 2 + }); + (lower_bound..=upper_bound, optimistic_gas_limit) } - } - - // Acquire the vm token for the whole duration of the binary search. - let vm_permit = self.0.vm_concurrency_limiter.acquire().await; - let vm_permit = vm_permit.ok_or(SubmitTxError::ServerShuttingDown)?; - - // When the pubdata cost grows very high, the total gas limit required may become very high as well. If - // we do binary search over any possible gas limit naively, we may end up with a very high number of iterations, - // which affects performance. - // - // To optimize for this case, we first calculate the amount of gas needed to cover for the pubdata. After that, we - // need to do a smaller binary search that is focused on computational gas limit only. - let (gas_used, additional_gas_for_pubdata) = if tx.is_l1() { - // For L1 transactions the pubdata priced in such a way that the maximal computational - // gas limit should be enough to cover for the pubdata as well, so no additional gas is provided there. - (None, 0_u64) - } else { - // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. - // And getting how much pubdata it used. - - // In theory, if the transaction has failed with such large gas limit, we could have returned an API error here right away, - // but doing it later on keeps the code more lean. - let (result, _) = self - .estimate_gas_step( - vm_permit.clone(), - tx.clone(), - max_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override.clone(), - ) - .await - .context("estimate_gas step failed")?; - - // It is assumed that there is no overflow here - let gas_for_pubdata = - u64::from(result.statistics.pubdata_published) * gas_per_pubdata_byte; - (Some(result.statistics.gas_used), gas_for_pubdata) }; - // We are using binary search to find the minimal values of gas_limit under which the transaction succeeds. - // **Important.** Lower and upper bound do not include `additional_gas_for_pubdata`; it needs to be added when calling `estimate_gas_step()`. - let mut lower_bound = gas_used.unwrap_or(0); - let mut upper_bound = MAX_L2_TX_GAS_LIMIT; - tracing::trace!( - "preparation took {:?}, starting binary search", - estimation_started_at.elapsed() - ); - let mut number_of_iterations = 0_usize; + let unscaled_gas_limit = + Self::binary_search(&estimator, bounds, initial_pivot, acceptable_overestimation) + .await?; + + let suggested_gas_limit = (unscaled_gas_limit as f64 * estimated_fee_scale_factor) as u64; + estimator + .finalize(suggested_gas_limit, estimated_fee_scale_factor) + .await + } - // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. - // If the transaction succeeds, it will discard most of the search space at once. - let optimistic_gas_limit = gas_used.and_then(|gas_used| { - let optimistic_gas_limit = gas_used * 6 / 5; - // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. - (optimistic_gas_limit < (lower_bound + upper_bound) / 2).then_some(optimistic_gas_limit) - }); + async fn binary_search( + estimator: &GasEstimator<'_>, + bounds: ops::RangeInclusive, + initial_pivot: Option, + acceptable_overestimation: u64, + ) -> Result { + let mut number_of_iterations = 0; + let mut lower_bound = *bounds.start(); + let mut upper_bound = *bounds.end(); - if let Some(optimistic_gas_limit) = optimistic_gas_limit { + if let Some(pivot) = initial_pivot { let iteration_started_at = Instant::now(); - let (result, _) = self - .estimate_gas_step( - vm_permit.clone(), - tx.clone(), - optimistic_gas_limit + additional_gas_for_pubdata, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override.clone(), - ) + let (result, _) = estimator + .step(pivot) .await .context("estimate_gas step failed")?; - Self::adjust_search_bounds( - &mut lower_bound, - &mut upper_bound, - optimistic_gas_limit, - &result, - ); + Self::adjust_search_bounds(&mut lower_bound, &mut upper_bound, pivot, &result); tracing::trace!( "iteration {number_of_iterations} took {:?}. lower_bound: {lower_bound}, upper_bound: {upper_bound}", @@ -250,25 +130,15 @@ impl TxSender { number_of_iterations += 1; } + // We are using binary search to find the minimal values of gas_limit under which the transaction succeeds. 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 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 = additional_gas_for_pubdata + mid; - let (result, _) = self - .estimate_gas_step( - vm_permit.clone(), - tx.clone(), - try_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override.clone(), - ) + let (result, _) = estimator + .step(mid) .await .context("estimate_gas step failed")?; Self::adjust_search_bounds(&mut lower_bound, &mut upper_bound, mid, &result); @@ -282,72 +152,7 @@ impl TxSender { SANDBOX_METRICS .estimate_gas_binary_search_iterations .observe(number_of_iterations); - - let suggested_gas_limit = - ((upper_bound + additional_gas_for_pubdata) as f64 * estimated_fee_scale_factor) as u64; - let (result, tx_metrics) = self - .estimate_gas_step( - vm_permit, - tx.clone(), - suggested_gas_limit, - gas_per_pubdata_byte as u32, - fee_input, - block_args, - base_fee, - protocol_version.into(), - state_override, - ) - .await - .context("final estimate_gas step failed")?; - - result.into_api_call_result()?; - self.ensure_tx_executable(&tx, &tx_metrics, false)?; - - // Now, we need to calculate the final overhead for the transaction. - let overhead = derive_overhead( - suggested_gas_limit, - gas_per_pubdata_byte as u32, - tx.encoding_len(), - tx.tx_format() as u8, - protocol_version.into(), - ); - - let full_gas_limit = match suggested_gas_limit.overflowing_add(overhead.into()) { - (value, false) => { - if value > max_gas_limit { - return Err(SubmitTxError::ExecutionReverted( - "exceeds block gas limit".to_string(), - vec![], - )); - } - - value - } - (_, true) => { - return Err(SubmitTxError::ExecutionReverted( - "exceeds block gas limit".to_string(), - vec![], - )); - } - }; - - let gas_for_pubdata = u64::from(tx_metrics.pubdata_published) * gas_per_pubdata_byte; - let estimated_gas_for_pubdata = - (gas_for_pubdata as f64 * estimated_fee_scale_factor) as u64; - - tracing::debug!( - "gas for pubdata: {estimated_gas_for_pubdata}, computational gas: {}, overhead gas: {overhead} \ - (with params base_fee: {base_fee}, gas_per_pubdata_byte: {gas_per_pubdata_byte}) \ - estimated_fee_scale_factor: {estimated_fee_scale_factor}", - suggested_gas_limit - estimated_gas_for_pubdata, - ); - - Ok(Fee { - max_fee_per_gas: base_fee.into(), - max_priority_fee_per_gas: 0u32.into(), - gas_limit: full_gas_limit.into(), - gas_per_pubdata_limit: gas_per_pubdata_byte.into(), - }) + Ok(upper_bound) } async fn ensure_sufficient_balance( @@ -407,3 +212,303 @@ impl TxSender { } } } + +/// Initial gas estimate with effectively infinite gas limit. +#[derive(Debug)] +pub(super) struct InitialGasEstimate { + /// Set to `None` if not estimated (e.g., for L1 transactions), or if the VM returned bogus refund stats. + pub total_gas_charged: Option, + /// Set to `None` if not estimated (e.g., for L1 transactions). + pub computational_gas_used: Option, + pub operator_overhead: u64, + pub gas_charged_for_pubdata: u64, +} + +impl InitialGasEstimate { + /// Total gas charged for a transaction consists of: + /// + /// - Operator-set overhead (`operator_overhead`) + /// - Intrinsic bootloader overhead + /// - Gas used during validation / execution + /// - Gas charged for pubdata at the end of execution (`gas_for_pubdata`) + /// + /// We add `operator_overhead` manually to the binary search argument at each `step()` because it depends on the gas limit in the general case, + /// so this value corresponds to the other 3 terms. + /// + /// If the value cannot be computed, it is set to `None`. + pub fn lower_gas_bound_without_overhead(&self) -> Option { + let mut total_gas_bound = self.computational_gas_used? + self.gas_charged_for_pubdata; + if let Some(gas_charged) = self.total_gas_charged { + total_gas_bound = total_gas_bound.min(gas_charged); + } + total_gas_bound.checked_sub(self.operator_overhead) + } + + pub fn optimistic_gas_limit_without_overhead(&self) -> Option { + let gas_charged_without_overhead = self + .total_gas_charged? + .checked_sub(self.operator_overhead)?; + // FIXME: use a tighter multiplier? + Some(gas_charged_without_overhead * 6 / 5) + } +} + +// Public for testing purposes +#[derive(Debug)] +pub(super) struct GasEstimator<'a> { + sender: &'a TxSender, + transaction: Transaction, + state_override: Option, + vm_permit: VmPermit, + fee_input: BatchFeeInput, + base_fee: u64, + gas_per_pubdata_byte: u64, + max_gas_limit: u64, + block_args: BlockArgs, + protocol_version: ProtocolVersionId, +} + +impl<'a> GasEstimator<'a> { + pub(super) async fn new( + sender: &'a TxSender, + mut transaction: Transaction, + state_override: Option, + ) -> Result { + let mut connection = sender.acquire_replica_connection().await?; + let block_args = BlockArgs::pending(&mut connection).await?; + let protocol_version = connection + .blocks_dal() + .pending_protocol_version() + .await + .context("failed getting pending protocol version")?; + drop(connection); + + let max_gas_limit = get_max_batch_gas_limit(protocol_version.into()); + let fee_input = adjust_pubdata_price_for_tx( + sender.scaled_batch_fee_input().await?, + transaction.gas_per_pubdata_byte_limit(), + // We do not have to adjust the params to the `gasPrice` of the transaction, since + // its gas price will be amended later on to suit the `fee_input` + None, + protocol_version.into(), + ); + let (base_fee, gas_per_pubdata_byte) = + derive_base_fee_and_gas_per_pubdata(fee_input, protocol_version.into()); + + sender + .ensure_sufficient_balance(&transaction, state_override.as_ref()) + .await?; + + // For L2 transactions we need a properly formatted signature + if let ExecuteTransactionCommon::L2(l2_common_data) = &mut transaction.common_data { + if l2_common_data.signature.is_empty() { + l2_common_data.signature = PackedEthSignature::default().serialize_packed().into(); + } + } + + // Acquire the vm token for the whole duration of the binary search. + let vm_permit = sender.0.vm_concurrency_limiter.acquire().await; + let vm_permit = vm_permit.ok_or(SubmitTxError::ServerShuttingDown)?; + + Ok(Self { + sender, + transaction, + state_override, + vm_permit, + fee_input, + base_fee, + gas_per_pubdata_byte, + max_gas_limit, + block_args, + protocol_version, + }) + } + + pub(super) fn adjust_transaction_fee(&mut self) { + match &mut self.transaction.common_data { + ExecuteTransactionCommon::L2(common_data) => { + common_data.fee.max_fee_per_gas = self.base_fee.into(); + common_data.fee.max_priority_fee_per_gas = self.base_fee.into(); + } + ExecuteTransactionCommon::L1(common_data) => { + common_data.max_fee_per_gas = self.base_fee.into(); + } + ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { + common_data.max_fee_per_gas = self.base_fee.into(); + } + } + } + + pub(super) async fn initialize(&self) -> anyhow::Result { + let operator_overhead = self.tx_overhead(self.max_gas_limit); + + // When the pubdata cost grows very high, the total gas limit required may become very high as well. If + // we do binary search over any possible gas limit naively, we may end up with a very high number of iterations, + // which affects performance. + // + // To optimize for this case, we first calculate the amount of gas needed to cover for the pubdata. After that, we + // need to do a smaller binary search that is focused on computational gas limit only. + if self.transaction.is_l1() { + // For L1 transactions the pubdata priced in such a way that the maximal computational + // gas limit should be enough to cover for the pubdata as well, so no additional gas is provided there. + Ok(InitialGasEstimate { + total_gas_charged: None, + computational_gas_used: None, + operator_overhead, + gas_charged_for_pubdata: 0, + }) + } else { + // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. + // And getting how much pubdata it used. + + // In theory, if the transaction has failed with such large gas limit, we could have returned an API error here right away, + // but doing it later on keeps the code more lean. + // FIXME: reconsider the choice above? + let (result, _) = self + .unadjusted_step(self.max_gas_limit) + .await + .context("estimate_gas step failed")?; + + // It is assumed that there is no overflow here + let gas_charged_for_pubdata = + u64::from(result.statistics.pubdata_published) * self.gas_per_pubdata_byte; + + let total_gas_charged = self.max_gas_limit.checked_sub(result.refunds.gas_refunded); + Ok(InitialGasEstimate { + total_gas_charged, + computational_gas_used: Some(result.statistics.computational_gas_used.into()), + operator_overhead, + gas_charged_for_pubdata, + }) + } + } + + /// Derives operator overhead for a transaction given its gas limit. + fn tx_overhead(&self, tx_gas_limit: u64) -> u64 { + derive_overhead( + tx_gas_limit, + self.gas_per_pubdata_byte as u32, + self.transaction.encoding_len(), + self.transaction.tx_format() as u8, + self.protocol_version.into(), + ) + .into() + } + + async fn step( + &self, + tx_gas_limit: u64, + ) -> anyhow::Result<(VmExecutionResultAndLogs, TransactionExecutionMetrics)> { + let gas_limit_with_overhead = tx_gas_limit + self.tx_overhead(tx_gas_limit); + // We need to ensure that we never use a gas limit that is higher than the maximum allowed + let forced_gas_limit = + gas_limit_with_overhead.min(get_max_batch_gas_limit(self.protocol_version.into())); + self.unadjusted_step(forced_gas_limit).await + } + + pub(super) async fn unadjusted_step( + &self, + forced_gas_limit: u64, + ) -> anyhow::Result<(VmExecutionResultAndLogs, TransactionExecutionMetrics)> { + let mut tx = self.transaction.clone(); + match &mut tx.common_data { + ExecuteTransactionCommon::L1(l1_common_data) => { + l1_common_data.gas_limit = forced_gas_limit.into(); + let required_funds = + l1_common_data.gas_limit * l1_common_data.max_fee_per_gas + tx.execute.value; + l1_common_data.to_mint = required_funds; + } + ExecuteTransactionCommon::L2(l2_common_data) => { + l2_common_data.fee.gas_limit = forced_gas_limit.into(); + } + ExecuteTransactionCommon::ProtocolUpgrade(common_data) => { + common_data.gas_limit = forced_gas_limit.into(); + let required_funds = + common_data.gas_limit * common_data.max_fee_per_gas + tx.execute.value; + common_data.to_mint = required_funds; + } + } + + let setup_args = self + .sender + .args_for_gas_estimate(self.fee_input, self.base_fee) + .await; + let execution_args = TxExecutionArgs::for_gas_estimate(tx); + let connection = self.sender.acquire_replica_connection().await?; + let executor = &self.sender.0.executor; + let execution_output = executor + .execute_tx_in_sandbox( + self.vm_permit.clone(), + setup_args, + execution_args, + connection, + self.block_args, + self.state_override.clone(), + OneshotTracingParams::default(), + ) + .await?; + Ok((execution_output.vm, execution_output.metrics)) + } + + async fn finalize( + self, + suggested_gas_limit: u64, + estimated_fee_scale_factor: f64, + ) -> Result { + let (result, tx_metrics) = self + .step(suggested_gas_limit) + .await + .context("final estimate_gas step failed")?; + result.into_api_call_result()?; + self.sender + .ensure_tx_executable(&self.transaction, &tx_metrics, false)?; + + // Now, we need to calculate the final overhead for the transaction. + let overhead = derive_overhead( + suggested_gas_limit, + self.gas_per_pubdata_byte as u32, + self.transaction.encoding_len(), + self.transaction.tx_format() as u8, + self.protocol_version.into(), + ); + + let full_gas_limit = match suggested_gas_limit.overflowing_add(overhead.into()) { + (value, false) => { + if value > self.max_gas_limit { + return Err(SubmitTxError::ExecutionReverted( + "exceeds block gas limit".to_string(), + vec![], + )); + } + + value + } + (_, true) => { + return Err(SubmitTxError::ExecutionReverted( + "exceeds block gas limit".to_string(), + vec![], + )); + } + }; + + let gas_for_pubdata = u64::from(tx_metrics.pubdata_published) * self.gas_per_pubdata_byte; + let estimated_gas_for_pubdata = + (gas_for_pubdata as f64 * estimated_fee_scale_factor) as u64; + + tracing::debug!( + "gas for pubdata: {estimated_gas_for_pubdata}, computational gas: {comp_gas}, overhead gas: {overhead} \ + (with params base_fee: {base_fee}, gas_per_pubdata_byte: {gas_per_pubdata_byte}) \ + estimated_fee_scale_factor: {estimated_fee_scale_factor}", + comp_gas = suggested_gas_limit - estimated_gas_for_pubdata, + base_fee = self.base_fee, + gas_per_pubdata_byte = self.gas_per_pubdata_byte + ); + + Ok(Fee { + max_fee_per_gas: self.base_fee.into(), + max_priority_fee_per_gas: 0u32.into(), + gas_limit: full_gas_limit.into(), + gas_per_pubdata_limit: self.gas_per_pubdata_byte.into(), + }) + } +} diff --git a/core/node/api_server/src/tx_sender/mod.rs b/core/node/api_server/src/tx_sender/mod.rs index 15dabac7d3d..802c3222f58 100644 --- a/core/node/api_server/src/tx_sender/mod.rs +++ b/core/node/api_server/src/tx_sender/mod.rs @@ -34,14 +34,11 @@ use zksync_types::{ }; use zksync_utils::h256_to_u256; -pub(super) use self::result::SubmitTxError; -use self::{master_pool_sink::MasterPoolSink, tx_sink::TxSink}; -use crate::{ - execution_sandbox::{ - BlockArgs, SubmitTxStage, TransactionExecutor, TxSetupArgs, VmConcurrencyBarrier, - VmConcurrencyLimiter, SANDBOX_METRICS, - }, - tx_sender::result::ApiCallResult, +pub(super) use self::{gas_estimation::BinarySearchKind, result::SubmitTxError}; +use self::{master_pool_sink::MasterPoolSink, result::ApiCallResult, tx_sink::TxSink}; +use crate::execution_sandbox::{ + BlockArgs, SubmitTxStage, TransactionExecutor, TxSetupArgs, VmConcurrencyBarrier, + VmConcurrencyLimiter, SANDBOX_METRICS, }; mod gas_estimation; diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index 178688aa31f..249b9a74eef 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -3,21 +3,21 @@ use std::{collections::HashMap, time::Duration}; use assert_matches::assert_matches; +use test_casing::test_casing; use zksync_multivm::interface::ExecutionResult; use zksync_node_fee_model::MockBatchFeeParamsProvider; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; use zksync_types::{ - api, api::state_override::OverrideAccount, get_nonce_key, L1BatchNumber, L2BlockNumber, - StorageLog, + api, api::state_override::OverrideAccount, get_nonce_key, K256PrivateKey, L1BatchNumber, + L2BlockNumber, StorageLog, }; use zksync_utils::u256_to_h256; use zksync_vm_executor::oneshot::MockOneshotExecutor; -use super::*; +use super::{gas_estimation::GasEstimator, *}; use crate::{ - execution_sandbox::BlockStartInfo, testonly::create_transfer, - web3::testonly::create_test_tx_sender, + execution_sandbox::BlockStartInfo, testonly::TestAccount, web3::testonly::create_test_tx_sender, }; #[tokio::test] @@ -208,7 +208,7 @@ async fn eth_call_requires_single_connection() { } #[tokio::test] -async fn estimating_gas_for_transfer() { +async fn initial_gas_estimation_is_somewhat_accurate() { let pool = ConnectionPool::::constrained_test_pool(1).await; let mut storage = pool.connection().await.unwrap(); let genesis_params = GenesisParams::mock(); @@ -225,10 +225,69 @@ async fn estimating_gas_for_transfer() { ) .await; - let value = 1_000_000_000.into(); - let tx = create_transfer(value, 55, 555); // fee params don't matter; they should be overwritten by the estimation logic + let alice = K256PrivateKey::random(); + let transfer_value = U256::from(1_000_000_000); + let account_overrides = OverrideAccount { + balance: Some(transfer_value * 2), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([(alice.address(), account_overrides)])); + // fee params don't matter; we adjust via `adjust_transaction_fee()` + let tx = alice.create_transfer(transfer_value, 55, 555); + + let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) + .await + .unwrap(); + estimator.adjust_transaction_fee(); + let initial_estimate = estimator.initialize().await.unwrap(); + assert!(initial_estimate.gas_charged_for_pubdata > 0); + assert!(initial_estimate.operator_overhead > 0); + let total_gas_charged = initial_estimate.total_gas_charged.unwrap(); + assert!( + total_gas_charged + > initial_estimate.gas_charged_for_pubdata + initial_estimate.operator_overhead, + "{initial_estimate:?}" + ); + + // Check that a transaction fails if supplied with the lower bound. + let lower_bound = initial_estimate.lower_gas_bound_without_overhead().unwrap() + + initial_estimate.operator_overhead; + assert!(lower_bound < total_gas_charged, "{initial_estimate:?}"); + let (vm_result, _) = estimator.unadjusted_step(lower_bound).await.unwrap(); + assert!(vm_result.result.is_failed(), "{:?}", vm_result.result); + + // A slightly larger limit should work. + let initial_pivot = total_gas_charged * 64 / 63; + let (vm_result, _) = estimator.unadjusted_step(initial_pivot).await.unwrap(); + assert!(!vm_result.result.is_failed(), "{:?}", vm_result.result); +} + +// FIXME: also test far call recursion + +#[test_casing(3, [1, 100, 1_000])] +#[tokio::test] +async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { + let pool = ConnectionPool::::constrained_test_pool(1).await; + let mut storage = pool.connection().await.unwrap(); + let genesis_params = GenesisParams::mock(); + insert_genesis_batch(&mut storage, &genesis_params) + .await + .unwrap(); + drop(storage); + + let tx_executor = TransactionExecutor::real(usize::MAX); + let (tx_sender, _) = create_test_tx_sender( + pool.clone(), + genesis_params.config().l2_chain_id, + tx_executor, + ) + .await; + + let alice = K256PrivateKey::random(); + let transfer_value = 1_000_000_000.into(); + // fee params don't matter; they should be overwritten by the estimation logic + let tx = alice.create_transfer(transfer_value, 55, 555); let fee_scale_factor = 1.0; - let acceptable_overestimation = 1_000; // Without overrides, the transaction should fail because of insufficient balance. let err = tx_sender .get_txs_fee_in_wei( @@ -236,31 +295,56 @@ async fn estimating_gas_for_transfer() { fee_scale_factor, acceptable_overestimation, None, + BinarySearchKind::Full, ) .await .unwrap_err(); assert_matches!(err, SubmitTxError::InsufficientFundsForTransfer); let account_overrides = OverrideAccount { - balance: Some(value * 2), + balance: Some(transfer_value * 2), ..OverrideAccount::default() }; - let call_overrides = - StateOverride::new(HashMap::from([(tx.initiator_account(), account_overrides)])); + let state_override = StateOverride::new(HashMap::from([(alice.address(), account_overrides)])); let fee = tx_sender .get_txs_fee_in_wei( - tx.into(), + tx.clone().into(), fee_scale_factor, acceptable_overestimation, - Some(call_overrides), + Some(state_override.clone()), + BinarySearchKind::Full, ) .await .unwrap(); // Sanity-check gas limit + let gas_limit_after_full_search = u64::try_from(fee.gas_limit).unwrap(); assert!( - fee.gas_limit > 1_000.into() && fee.gas_limit < 1_000_000.into(), + (10_000..1_000_000).contains(&gas_limit_after_full_search), "{fee:?}" ); - // FIXME: check that transaction with this limit doesn't fail & fails with a lower one + let fee = tx_sender + .get_txs_fee_in_wei( + tx.into(), + fee_scale_factor, + acceptable_overestimation, + Some(state_override.clone()), + BinarySearchKind::Optimized, + ) + .await + .unwrap(); + let gas_limit_after_optimized_search = u64::try_from(fee.gas_limit).unwrap(); + + if acceptable_overestimation == 1 { + assert_eq!( + gas_limit_after_full_search, + gas_limit_after_optimized_search + ); + } else { + let diff = gas_limit_after_full_search.abs_diff(gas_limit_after_optimized_search); + assert!( + diff <= acceptable_overestimation, + "full={gas_limit_after_full_search}, optimized={gas_limit_after_optimized_search}" + ); + } } diff --git a/core/node/api_server/src/web3/namespaces/eth.rs b/core/node/api_server/src/web3/namespaces/eth.rs index fda5ff6f06b..d76d0ee1b90 100644 --- a/core/node/api_server/src/web3/namespaces/eth.rs +++ b/core/node/api_server/src/web3/namespaces/eth.rs @@ -19,6 +19,7 @@ use zksync_web3_decl::{ }; use crate::{ + tx_sender::BinarySearchKind, utils::open_readonly_transaction, web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, state::RpcState, TypedFilter}, }; @@ -138,6 +139,7 @@ impl EthNamespace { scale_factor, acceptable_overestimation as u64, state_override, + BinarySearchKind::Full, ) .await?; Ok(fee.gas_limit) diff --git a/core/node/api_server/src/web3/namespaces/zks.rs b/core/node/api_server/src/web3/namespaces/zks.rs index 4f88eb17e23..b27f553a262 100644 --- a/core/node/api_server/src/web3/namespaces/zks.rs +++ b/core/node/api_server/src/web3/namespaces/zks.rs @@ -30,6 +30,7 @@ use zksync_web3_decl::{ }; use crate::{ + tx_sender::BinarySearchKind, utils::open_readonly_transaction, web3::{backend_jsonrpsee::MethodTracer, metrics::API_METRICS, RpcState}, }; @@ -113,6 +114,7 @@ impl ZksNamespace { scale_factor, acceptable_overestimation as u64, state_override, + BinarySearchKind::Full, ) .await?) } From 54d8705821b8f78c31e37b564b498c8a01f6ac35 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 10:42:56 +0300 Subject: [PATCH 04/14] Test gas estimation with load test txs --- core/node/api_server/src/testonly.rs | 27 +++ core/node/api_server/src/tx_sender/tests.rs | 178 +++++++++++++++++--- 2 files changed, 179 insertions(+), 26 deletions(-) diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index c74c127eb10..d53cbbb979d 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -1,10 +1,13 @@ //! Test utils shared among multiple modules. +use zksync_contracts::{get_loadnext_contract, test_contracts::LoadnextContractExecutionParams}; use zksync_types::{ fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, Address, K256PrivateKey, L2ChainId, Nonce, U256, }; +pub(crate) const LOAD_TEST_ADDRESS: Address = Address::repeat_byte(1); + pub(crate) trait TestAccount { fn create_transfer(&self, value: U256, fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { let fee = Fee { @@ -17,6 +20,8 @@ pub(crate) trait TestAccount { } fn create_transfer_with_fee(&self, value: U256, fee: Fee) -> L2Tx; + + fn create_load_test_tx(&self, params: LoadnextContractExecutionParams) -> L2Tx; } impl TestAccount for K256PrivateKey { @@ -34,4 +39,26 @@ impl TestAccount for K256PrivateKey { ) .unwrap() } + + fn create_load_test_tx(&self, params: LoadnextContractExecutionParams) -> L2Tx { + let fee = Fee { + gas_limit: 200_000.into(), + max_fee_per_gas: 55.into(), + max_priority_fee_per_gas: 0_u64.into(), + gas_per_pubdata_limit: 555.into(), + }; + + L2Tx::new_signed( + Some(LOAD_TEST_ADDRESS), + params.to_bytes(), + Nonce(0), + fee, + 0.into(), + L2ChainId::default(), + self, + get_loadnext_contract().factory_deps, + PaymasterParams::default(), + ) + .unwrap() + } } diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index 249b9a74eef..e6ee77137c9 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -3,21 +3,25 @@ use std::{collections::HashMap, time::Duration}; use assert_matches::assert_matches; -use test_casing::test_casing; +use test_casing::{test_casing, Product, TestCases}; +use zksync_contracts::{get_loadnext_contract, test_contracts::LoadnextContractExecutionParams}; use zksync_multivm::interface::ExecutionResult; use zksync_node_fee_model::MockBatchFeeParamsProvider; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; use zksync_types::{ - api, api::state_override::OverrideAccount, get_nonce_key, K256PrivateKey, L1BatchNumber, - L2BlockNumber, StorageLog, + api, + api::state_override::{Bytecode, OverrideAccount, OverrideState}, + get_nonce_key, K256PrivateKey, L1BatchNumber, L2BlockNumber, StorageLog, }; use zksync_utils::u256_to_h256; use zksync_vm_executor::oneshot::MockOneshotExecutor; use super::{gas_estimation::GasEstimator, *}; use crate::{ - execution_sandbox::BlockStartInfo, testonly::TestAccount, web3::testonly::create_test_tx_sender, + execution_sandbox::BlockStartInfo, + testonly::{TestAccount, LOAD_TEST_ADDRESS}, + web3::testonly::create_test_tx_sender, }; #[tokio::test] @@ -262,11 +266,83 @@ async fn initial_gas_estimation_is_somewhat_accurate() { assert!(!vm_result.result.is_failed(), "{:?}", vm_result.result); } -// FIXME: also test far call recursion +const LOAD_TEST_CASES: TestCases = test_casing::cases! {[ + LoadnextContractExecutionParams::default(), + // No storage modification + LoadnextContractExecutionParams { + writes: 0, + events: 0, + ..LoadnextContractExecutionParams::default() + }, + // Deep recursion + LoadnextContractExecutionParams { + recursive_calls: 10, + ..LoadnextContractExecutionParams::default() + }, + // No deploys + LoadnextContractExecutionParams { + deploys: 0, + ..LoadnextContractExecutionParams::default() + }, + // Lots of deploys + LoadnextContractExecutionParams { + deploys: 10, + ..LoadnextContractExecutionParams::default() + }, +]}; + +#[test_casing(5, LOAD_TEST_CASES)] +#[tokio::test] +async fn estimating_load_test_transaction(tx_params: LoadnextContractExecutionParams) { + let pool = ConnectionPool::::constrained_test_pool(1).await; + let mut storage = pool.connection().await.unwrap(); + let genesis_params = GenesisParams::mock(); + insert_genesis_batch(&mut storage, &genesis_params) + .await + .unwrap(); + drop(storage); + + let tx_executor = TransactionExecutor::real(usize::MAX); + let (tx_sender, _) = create_test_tx_sender( + pool.clone(), + genesis_params.config().l2_chain_id, + tx_executor, + ) + .await; + + let alice = K256PrivateKey::random(); + // Set the array length in the load test contract to 100, so that reads don't fail. + let load_test_state = HashMap::from([(H256::zero(), H256::from_low_u64_be(100))]); + let load_test_overrides = OverrideAccount { + code: Some(Bytecode::new(get_loadnext_contract().bytecode).unwrap()), + state: Some(OverrideState::State(load_test_state)), + ..OverrideAccount::default() + }; + let state_override = + StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); + let tx = alice.create_load_test_tx(tx_params); + + let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) + .await + .unwrap(); + estimator.adjust_transaction_fee(); + let initial_estimate = estimator.initialize().await.unwrap(); + dbg!(&initial_estimate); + + let lower_bound = initial_estimate.lower_gas_bound_without_overhead().unwrap() + + initial_estimate.operator_overhead; + dbg!(lower_bound); + let (vm_result, _) = estimator.unadjusted_step(lower_bound).await.unwrap(); + assert!(vm_result.result.is_failed(), "{:?}", vm_result.result); + + // A slightly larger limit should work. + let initial_pivot = initial_estimate.total_gas_charged.unwrap() * 64 / 63; + let (vm_result, _) = estimator.unadjusted_step(initial_pivot).await.unwrap(); + assert!(!vm_result.result.is_failed(), "{:?}", vm_result.result); +} -#[test_casing(3, [1, 100, 1_000])] #[tokio::test] -async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { +async fn insufficient_funds_error_for_transfer() { let pool = ConnectionPool::::constrained_test_pool(1).await; let mut storage = pool.connection().await.unwrap(); let genesis_params = GenesisParams::mock(); @@ -293,19 +369,37 @@ async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { .get_txs_fee_in_wei( tx.clone().into(), fee_scale_factor, - acceptable_overestimation, + 1_000, None, BinarySearchKind::Full, ) .await .unwrap_err(); assert_matches!(err, SubmitTxError::InsufficientFundsForTransfer); +} - let account_overrides = OverrideAccount { - balance: Some(transfer_value * 2), - ..OverrideAccount::default() - }; - let state_override = StateOverride::new(HashMap::from([(alice.address(), account_overrides)])); +async fn test_estimating_gas( + state_override: StateOverride, + tx: L2Tx, + acceptable_overestimation: u64, +) { + let pool = ConnectionPool::::constrained_test_pool(1).await; + let mut storage = pool.connection().await.unwrap(); + let genesis_params = GenesisParams::mock(); + insert_genesis_batch(&mut storage, &genesis_params) + .await + .unwrap(); + drop(storage); + + let tx_executor = TransactionExecutor::real(usize::MAX); + let (tx_sender, _) = create_test_tx_sender( + pool.clone(), + genesis_params.config().l2_chain_id, + tx_executor, + ) + .await; + + let fee_scale_factor = 1.0; let fee = tx_sender .get_txs_fee_in_wei( tx.clone().into(), @@ -319,7 +413,7 @@ async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { // Sanity-check gas limit let gas_limit_after_full_search = u64::try_from(fee.gas_limit).unwrap(); assert!( - (10_000..1_000_000).contains(&gas_limit_after_full_search), + (10_000..10_000_000).contains(&gas_limit_after_full_search), "{fee:?}" ); @@ -335,16 +429,48 @@ async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { .unwrap(); let gas_limit_after_optimized_search = u64::try_from(fee.gas_limit).unwrap(); - if acceptable_overestimation == 1 { - assert_eq!( - gas_limit_after_full_search, - gas_limit_after_optimized_search - ); - } else { - let diff = gas_limit_after_full_search.abs_diff(gas_limit_after_optimized_search); - assert!( - diff <= acceptable_overestimation, - "full={gas_limit_after_full_search}, optimized={gas_limit_after_optimized_search}" - ); - } + let diff = gas_limit_after_full_search.abs_diff(gas_limit_after_optimized_search); + assert!( + diff <= acceptable_overestimation, + "full={gas_limit_after_full_search}, optimized={gas_limit_after_optimized_search}" + ); +} + +#[test_casing(3, [0, 100, 1_000])] +#[tokio::test] +async fn estimating_gas_for_transfer(acceptable_overestimation: u64) { + let alice = K256PrivateKey::random(); + let transfer_value = 1_000_000_000.into(); + let account_overrides = OverrideAccount { + balance: Some(transfer_value * 2), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([(alice.address(), account_overrides)])); + // fee params don't matter; they should be overwritten by the estimation logic + let tx = alice.create_transfer(transfer_value, 55, 555); + + test_estimating_gas(state_override, tx, acceptable_overestimation).await; +} + +#[test_casing(10, Product((LOAD_TEST_CASES, [0, 100])))] +#[tokio::test] +async fn estimating_gas_for_load_test_tx( + tx_params: LoadnextContractExecutionParams, + acceptable_overestimation: u64, +) { + let alice = K256PrivateKey::random(); + // Set the array length in the load test contract to 100, so that reads don't fail. + let load_test_state = HashMap::from([(H256::zero(), H256::from_low_u64_be(100))]); + let load_test_overrides = OverrideAccount { + code: Some(Bytecode::new(get_loadnext_contract().bytecode).unwrap()), + state: Some(OverrideState::State(load_test_state)), + ..OverrideAccount::default() + }; + let state_override = + StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); + let tx = alice.create_load_test_tx(tx_params); + + test_estimating_gas(state_override, tx, acceptable_overestimation).await; } + +// FIXME: more test contracts: expensive, precompiles -> code oracle, failures (counter, infinite) From 47239f48a9430740cc9565fe1fa1f8ee3f84ff85 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 13:03:06 +0300 Subject: [PATCH 05/14] Test gas estimation with more tx types --- .../src/execution_sandbox/storage.rs | 6 +- core/node/api_server/src/testonly.rs | 112 +++++++++- .../src/tx_sender/gas_estimation.rs | 1 + core/node/api_server/src/tx_sender/tests.rs | 195 ++++++++++++++++-- 4 files changed, 283 insertions(+), 31 deletions(-) diff --git a/core/node/api_server/src/execution_sandbox/storage.rs b/core/node/api_server/src/execution_sandbox/storage.rs index 0d4c88d4a0a..a6c5d7afcf1 100644 --- a/core/node/api_server/src/execution_sandbox/storage.rs +++ b/core/node/api_server/src/execution_sandbox/storage.rs @@ -8,7 +8,7 @@ use std::{ use zksync_multivm::interface::storage::ReadStorage; use zksync_types::{ api::state_override::{OverrideState, StateOverride}, - get_code_key, get_nonce_key, + get_code_key, get_known_code_key, get_nonce_key, utils::{decompose_full_nonce, nonces_to_full_nonce, storage_key_for_eth_balance}, AccountTreeId, StorageKey, StorageValue, H256, }; @@ -56,6 +56,10 @@ impl StorageWithOverrides { let code_key = get_code_key(account); let code_hash = code.hash(); self.overridden_slots.insert(code_key, code_hash); + // FIXME: is this addition correct? + let known_code_key = get_known_code_key(&code_hash); + self.overridden_slots + .insert(known_code_key, H256::from_low_u64_be(1)); self.store_factory_dep(code_hash, code.clone().into_bytes()); } diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index d53cbbb979d..0f7b96c82ae 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -1,13 +1,41 @@ //! Test utils shared among multiple modules. -use zksync_contracts::{get_loadnext_contract, test_contracts::LoadnextContractExecutionParams}; +use zksync_contracts::{ + get_loadnext_contract, load_contract, read_bytecode, + test_contracts::LoadnextContractExecutionParams, +}; use zksync_types::{ - fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, Address, K256PrivateKey, L2ChainId, - Nonce, U256, + ethabi::Token, fee::Fee, l2::L2Tx, transaction_request::PaymasterParams, Address, + K256PrivateKey, L2ChainId, Nonce, H256, U256, }; pub(crate) const LOAD_TEST_ADDRESS: Address = Address::repeat_byte(1); +const EXPENSIVE_CONTRACT_PATH: &str = + "etc/contracts-test-data/artifacts-zk/contracts/expensive/expensive.sol/Expensive.json"; +pub(crate) const EXPENSIVE_CONTRACT_ADDRESS: Address = Address::repeat_byte(2); + +const PRECOMPILES_CONTRACT_PATH: &str = + "etc/contracts-test-data/artifacts-zk/contracts/precompiles/precompiles.sol/Precompiles.json"; +pub(crate) const PRECOMPILES_CONTRACT_ADDRESS: Address = Address::repeat_byte(3); + +pub(crate) fn read_expensive_contract_bytecode() -> Vec { + read_bytecode(EXPENSIVE_CONTRACT_PATH) +} + +pub(crate) fn read_precompiles_contract_bytecode() -> Vec { + read_bytecode(PRECOMPILES_CONTRACT_PATH) +} + +fn default_fee() -> Fee { + Fee { + gas_limit: 200_000.into(), + max_fee_per_gas: 55.into(), + max_priority_fee_per_gas: 0_u64.into(), + gas_per_pubdata_limit: 555.into(), + } +} + pub(crate) trait TestAccount { fn create_transfer(&self, value: U256, fee_per_gas: u64, gas_per_pubdata: u64) -> L2Tx { let fee = Fee { @@ -22,6 +50,12 @@ pub(crate) trait TestAccount { fn create_transfer_with_fee(&self, value: U256, fee: Fee) -> L2Tx; fn create_load_test_tx(&self, params: LoadnextContractExecutionParams) -> L2Tx; + + fn create_expensive_tx(&self, write_count: usize) -> L2Tx; + + fn create_expensive_cleanup_tx(&self) -> L2Tx; + + fn create_decommitting_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx; } impl TestAccount for K256PrivateKey { @@ -41,18 +75,74 @@ impl TestAccount for K256PrivateKey { } fn create_load_test_tx(&self, params: LoadnextContractExecutionParams) -> L2Tx { - let fee = Fee { - gas_limit: 200_000.into(), - max_fee_per_gas: 55.into(), - max_priority_fee_per_gas: 0_u64.into(), - gas_per_pubdata_limit: 555.into(), - }; - L2Tx::new_signed( Some(LOAD_TEST_ADDRESS), params.to_bytes(), Nonce(0), - fee, + default_fee(), + 0.into(), + L2ChainId::default(), + self, + get_loadnext_contract().factory_deps, + PaymasterParams::default(), + ) + .unwrap() + } + + fn create_expensive_tx(&self, write_count: usize) -> L2Tx { + let calldata = load_contract(EXPENSIVE_CONTRACT_PATH) + .function("expensive") + .expect("no `expensive` function in contract") + .encode_input(&[Token::Uint(write_count.into())]) + .expect("failed encoding `expensive` function"); + L2Tx::new_signed( + Some(EXPENSIVE_CONTRACT_ADDRESS), + calldata, + Nonce(0), + default_fee(), + 0.into(), + L2ChainId::default(), + self, + get_loadnext_contract().factory_deps, + PaymasterParams::default(), + ) + .unwrap() + } + + fn create_expensive_cleanup_tx(&self) -> L2Tx { + let calldata = load_contract(EXPENSIVE_CONTRACT_PATH) + .function("cleanUp") + .expect("no `cleanUp` function in contract") + .encode_input(&[]) + .expect("failed encoding `cleanUp` input"); + L2Tx::new_signed( + Some(EXPENSIVE_CONTRACT_ADDRESS), + calldata, + Nonce(0), + default_fee(), + 0.into(), + L2ChainId::default(), + self, + get_loadnext_contract().factory_deps, + PaymasterParams::default(), + ) + .unwrap() + } + + fn create_decommitting_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx { + let calldata = load_contract(PRECOMPILES_CONTRACT_PATH) + .function("callCodeOracle") + .expect("no `callCodeOracle` function") + .encode_input(&[ + Token::FixedBytes(bytecode_hash.0.to_vec()), + Token::FixedBytes(expected_keccak_hash.0.to_vec()), + ]) + .expect("failed encoding `callCodeOracle` input"); + L2Tx::new_signed( + Some(PRECOMPILES_CONTRACT_ADDRESS), + calldata, + Nonce(0), + default_fee(), 0.into(), L2ChainId::default(), self, diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 984801d0826..371ede7fa6d 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -368,6 +368,7 @@ impl<'a> GasEstimator<'a> { .unadjusted_step(self.max_gas_limit) .await .context("estimate_gas step failed")?; + assert!(!result.result.is_failed(), "{:?}", result.result); // FIXME: remove // It is assumed that there is no overflow here let gas_charged_for_pubdata = diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index e6ee77137c9..cf2d017dfdc 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -9,18 +9,24 @@ use zksync_multivm::interface::ExecutionResult; use zksync_node_fee_model::MockBatchFeeParamsProvider; use zksync_node_genesis::{insert_genesis_batch, GenesisParams}; use zksync_node_test_utils::{create_l2_block, create_l2_transaction, prepare_recovery_snapshot}; +use zksync_system_constants::CODE_ORACLE_ADDRESS; use zksync_types::{ api, api::state_override::{Bytecode, OverrideAccount, OverrideState}, - get_nonce_key, K256PrivateKey, L1BatchNumber, L2BlockNumber, StorageLog, + get_nonce_key, + web3::keccak256, + K256PrivateKey, L1BatchNumber, L2BlockNumber, StorageLog, }; -use zksync_utils::u256_to_h256; +use zksync_utils::{bytecode::hash_bytecode, u256_to_h256}; use zksync_vm_executor::oneshot::MockOneshotExecutor; use super::{gas_estimation::GasEstimator, *}; use crate::{ execution_sandbox::BlockStartInfo, - testonly::{TestAccount, LOAD_TEST_ADDRESS}, + testonly::{ + read_expensive_contract_bytecode, read_precompiles_contract_bytecode, TestAccount, + EXPENSIVE_CONTRACT_ADDRESS, LOAD_TEST_ADDRESS, PRECOMPILES_CONTRACT_ADDRESS, + }, web3::testonly::create_test_tx_sender, }; @@ -293,7 +299,28 @@ const LOAD_TEST_CASES: TestCases = test_casing: #[test_casing(5, LOAD_TEST_CASES)] #[tokio::test] -async fn estimating_load_test_transaction(tx_params: LoadnextContractExecutionParams) { +async fn initial_estimate_for_load_test_transaction(tx_params: LoadnextContractExecutionParams) { + let alice = K256PrivateKey::random(); + // Set the array length in the load test contract to 100, so that reads don't fail. + let load_test_state = HashMap::from([(H256::zero(), H256::from_low_u64_be(100))]); + let load_test_overrides = OverrideAccount { + code: Some(Bytecode::new(get_loadnext_contract().bytecode).unwrap()), + state: Some(OverrideState::State(load_test_state)), + ..OverrideAccount::default() + }; + let state_override = + StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); + let tx = alice.create_load_test_tx(tx_params); + + test_initial_estimate(state_override, tx).await; +} + +/// Tests the lower bound and initial pivot extracted from the initial estimate (one with effectively infinite gas amount). +/// Returns the VM result for a VM run with the initial pivot. +async fn test_initial_estimate( + state_override: StateOverride, + tx: L2Tx, +) -> VmExecutionResultAndLogs { let pool = ConnectionPool::::constrained_test_pool(1).await; let mut storage = pool.connection().await.unwrap(); let genesis_params = GenesisParams::mock(); @@ -310,28 +337,14 @@ async fn estimating_load_test_transaction(tx_params: LoadnextContractExecutionPa ) .await; - let alice = K256PrivateKey::random(); - // Set the array length in the load test contract to 100, so that reads don't fail. - let load_test_state = HashMap::from([(H256::zero(), H256::from_low_u64_be(100))]); - let load_test_overrides = OverrideAccount { - code: Some(Bytecode::new(get_loadnext_contract().bytecode).unwrap()), - state: Some(OverrideState::State(load_test_state)), - ..OverrideAccount::default() - }; - let state_override = - StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); - let tx = alice.create_load_test_tx(tx_params); - let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) .await .unwrap(); estimator.adjust_transaction_fee(); let initial_estimate = estimator.initialize().await.unwrap(); - dbg!(&initial_estimate); let lower_bound = initial_estimate.lower_gas_bound_without_overhead().unwrap() + initial_estimate.operator_overhead; - dbg!(lower_bound); let (vm_result, _) = estimator.unadjusted_step(lower_bound).await.unwrap(); assert!(vm_result.result.is_failed(), "{:?}", vm_result.result); @@ -339,6 +352,103 @@ async fn estimating_load_test_transaction(tx_params: LoadnextContractExecutionPa let initial_pivot = initial_estimate.total_gas_charged.unwrap() * 64 / 63; let (vm_result, _) = estimator.unadjusted_step(initial_pivot).await.unwrap(); assert!(!vm_result.result.is_failed(), "{:?}", vm_result.result); + vm_result +} + +/// Estimates both transactions with initial writes and cleanup. +#[test_casing(4, [10, 50, 200, 1_000])] +#[tokio::test] +async fn initial_estimate_for_expensive_contract(write_count: usize) { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_expensive_contract_bytecode(); + let mut contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + EXPENSIVE_CONTRACT_ADDRESS, + contract_overrides.clone(), + )])); + let tx = alice.create_expensive_tx(write_count); + + let vm_result = test_initial_estimate(state_override, tx).await; + + let contract_logs = vm_result.logs.storage_logs.into_iter().filter_map(|log| { + (*log.log.key.address() == EXPENSIVE_CONTRACT_ADDRESS) + .then_some((*log.log.key.key(), log.log.value)) + }); + let contract_logs: HashMap<_, _> = contract_logs.collect(); + assert!(contract_logs.len() >= write_count, "{contract_logs:?}"); + contract_overrides.state = Some(OverrideState::StateDiff(contract_logs)); + + let state_override = StateOverride::new(HashMap::from([( + EXPENSIVE_CONTRACT_ADDRESS, + contract_overrides, + )])); + let tx = alice.create_expensive_cleanup_tx(); + + test_initial_estimate(state_override, tx).await; +} + +#[tokio::test] +async fn initial_estimate_for_code_oracle_tx() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_precompiles_contract_bytecode(); + let contract_bytecode_hash = hash_bytecode(&contract_bytecode); + let contract_keccak_hash = H256(keccak256(&contract_bytecode)); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + + // Add another contract that is never executed, but has a large bytecode. + let huge_contact_address = Address::repeat_byte(23); + let huge_contract_bytecode = vec![0_u8; 10_001 * 32]; + let huge_contract_bytecode_hash = hash_bytecode(&huge_contract_bytecode); + let huge_contract_keccak_hash = H256(keccak256(&huge_contract_bytecode)); + let huge_contract_overrides = OverrideAccount { + code: Some(Bytecode::new(huge_contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + + let state_override = StateOverride::new(HashMap::from([ + (PRECOMPILES_CONTRACT_ADDRESS, contract_overrides), + (huge_contact_address, huge_contract_overrides), + ])); + + // Test contracts that are already decommitted when requested from the precompiles test contract. + let genesis_params = GenesisParams::mock(); + let code_oracle_bytecode = genesis_params + .system_contracts() + .iter() + .find_map(|contract| { + (*contract.account_id.address() == CODE_ORACLE_ADDRESS).then_some(&contract.bytecode) + }) + .expect("no code oracle"); + let code_oracle_bytecode_hash = hash_bytecode(code_oracle_bytecode); + let code_oracle_keccak_hash = H256(keccak256(code_oracle_bytecode)); + + let warm_bytecode_hashes = [ + (code_oracle_bytecode_hash, code_oracle_keccak_hash), + (contract_bytecode_hash, contract_keccak_hash), + ]; + let mut decomitter_stats = 0.0; + for (hash, keccak_hash) in warm_bytecode_hashes { + let tx = alice.create_decommitting_tx(hash, keccak_hash); + let vm_result = test_initial_estimate(state_override.clone(), tx).await; + let stats = &vm_result.statistics.circuit_statistic; + decomitter_stats = stats.code_decommitter.max(decomitter_stats); + } + assert!(decomitter_stats > 0.0); + + let tx = alice.create_decommitting_tx(huge_contract_bytecode_hash, huge_contract_keccak_hash); + let vm_result = test_initial_estimate(state_override, tx).await; + // Sanity check: the transaction should spend significantly more on decommitment compared to previous ones + let new_decomitter_stats = vm_result.statistics.circuit_statistic.code_decommitter; + assert!( + new_decomitter_stats > decomitter_stats * 1.5, + "old={decomitter_stats}, new={new_decomitter_stats}" + ); } #[tokio::test] @@ -473,4 +583,51 @@ async fn estimating_gas_for_load_test_tx( test_estimating_gas(state_override, tx, acceptable_overestimation).await; } -// FIXME: more test contracts: expensive, precompiles -> code oracle, failures (counter, infinite) +#[test_casing(4, [10, 50, 100, 200])] +#[tokio::test] +async fn estimating_gas_for_expensive_txs(write_count: usize) { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_expensive_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + EXPENSIVE_CONTRACT_ADDRESS, + contract_overrides.clone(), + )])); + let tx = alice.create_expensive_tx(write_count); + + test_estimating_gas(state_override, tx, 0).await; +} + +#[tokio::test] +async fn estimating_gas_for_code_oracle_tx() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_precompiles_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + + // Add another contract that is never executed, but has a large bytecode. + let huge_contact_address = Address::repeat_byte(23); + let huge_contract_bytecode = vec![0_u8; 10_001 * 32]; + let huge_contract_bytecode_hash = hash_bytecode(&huge_contract_bytecode); + let huge_contract_keccak_hash = keccak256(&huge_contract_bytecode); + let huge_contract_overrides = OverrideAccount { + code: Some(Bytecode::new(huge_contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + + let state_override = StateOverride::new(HashMap::from([ + (PRECOMPILES_CONTRACT_ADDRESS, contract_overrides), + (huge_contact_address, huge_contract_overrides), + ])); + let tx = + alice.create_decommitting_tx(huge_contract_bytecode_hash, H256(huge_contract_keccak_hash)); + + test_estimating_gas(state_override, tx, 0).await; +} + +// FIXME: more test contracts: failures (counter, infinite) From 6c93c7dba8f8a18ddacdd95893d45d3280e69710 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 13:12:26 +0300 Subject: [PATCH 06/14] Reduce multiplier for initial pivot --- core/node/api_server/src/tx_sender/gas_estimation.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 371ede7fa6d..ca6065339e6 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -98,7 +98,6 @@ impl TxSender { let unscaled_gas_limit = Self::binary_search(&estimator, bounds, initial_pivot, acceptable_overestimation) .await?; - let suggested_gas_limit = (unscaled_gas_limit as f64 * estimated_fee_scale_factor) as u64; estimator .finalize(suggested_gas_limit, estimated_fee_scale_factor) @@ -248,8 +247,10 @@ impl InitialGasEstimate { let gas_charged_without_overhead = self .total_gas_charged? .checked_sub(self.operator_overhead)?; - // FIXME: use a tighter multiplier? - Some(gas_charged_without_overhead * 6 / 5) + // 21/20 is an empirical multiplier. It is higher than what empirically suffices for some common transactions; + // one can argue that using 64/63 multiplier would be more accurate due to the 63/64 rule for far calls + // (however, far calls are not the only source of gas overhead in Era). + Some(gas_charged_without_overhead * 21 / 20) } } @@ -368,7 +369,6 @@ impl<'a> GasEstimator<'a> { .unadjusted_step(self.max_gas_limit) .await .context("estimate_gas step failed")?; - assert!(!result.result.is_failed(), "{:?}", result.result); // FIXME: remove // It is assumed that there is no overflow here let gas_charged_for_pubdata = From d76a0d72e1cf4b7fe1663f0f328bdee3f41d690a Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 13:54:46 +0300 Subject: [PATCH 07/14] Return early if initial estimate fails --- core/node/api_server/src/testonly.rs | 30 +++++ .../src/tx_sender/gas_estimation.rs | 9 +- core/node/api_server/src/tx_sender/result.rs | 18 ++- core/node/api_server/src/tx_sender/tests.rs | 118 ++++++++++++------ 4 files changed, 129 insertions(+), 46 deletions(-) diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index 0f7b96c82ae..c92f7d7ce7c 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -19,6 +19,10 @@ const PRECOMPILES_CONTRACT_PATH: &str = "etc/contracts-test-data/artifacts-zk/contracts/precompiles/precompiles.sol/Precompiles.json"; pub(crate) const PRECOMPILES_CONTRACT_ADDRESS: Address = Address::repeat_byte(3); +const COUNTER_CONTRACT_PATH: &str = + "etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json"; +pub(crate) const COUNTER_CONTRACT_ADDRESS: Address = Address::repeat_byte(4); + pub(crate) fn read_expensive_contract_bytecode() -> Vec { read_bytecode(EXPENSIVE_CONTRACT_PATH) } @@ -27,6 +31,10 @@ pub(crate) fn read_precompiles_contract_bytecode() -> Vec { read_bytecode(PRECOMPILES_CONTRACT_PATH) } +pub(crate) fn read_counter_contract_bytecode() -> Vec { + read_bytecode(COUNTER_CONTRACT_PATH) +} + fn default_fee() -> Fee { Fee { gas_limit: 200_000.into(), @@ -56,6 +64,8 @@ pub(crate) trait TestAccount { fn create_expensive_cleanup_tx(&self) -> L2Tx; fn create_decommitting_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx; + + fn create_reverting_counter_tx(&self) -> L2Tx; } impl TestAccount for K256PrivateKey { @@ -151,4 +161,24 @@ impl TestAccount for K256PrivateKey { ) .unwrap() } + + fn create_reverting_counter_tx(&self) -> L2Tx { + let calldata = load_contract(COUNTER_CONTRACT_PATH) + .function("incrementWithRevert") + .expect("no `incrementWithRevert` function") + .encode_input(&[Token::Uint(1.into()), Token::Bool(true)]) + .expect("failed encoding `incrementWithRevert` input"); + L2Tx::new_signed( + Some(COUNTER_CONTRACT_ADDRESS), + calldata, + Nonce(0), + default_fee(), + 0.into(), + L2ChainId::default(), + self, + get_loadnext_contract().factory_deps, + PaymasterParams::default(), + ) + .unwrap() + } } diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index ca6065339e6..13d8e3b5359 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -340,7 +340,7 @@ impl<'a> GasEstimator<'a> { } } - pub(super) async fn initialize(&self) -> anyhow::Result { + pub(super) async fn initialize(&self) -> Result { let operator_overhead = self.tx_overhead(self.max_gas_limit); // When the pubdata cost grows very high, the total gas limit required may become very high as well. If @@ -362,13 +362,14 @@ impl<'a> GasEstimator<'a> { // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. // And getting how much pubdata it used. - // In theory, if the transaction has failed with such large gas limit, we could have returned an API error here right away, - // but doing it later on keeps the code more lean. - // FIXME: reconsider the choice above? let (result, _) = self .unadjusted_step(self.max_gas_limit) .await .context("estimate_gas step failed")?; + // If the transaction has failed with such a large gas limit, we return an API error here right away, + // since the inferred gas bounds are be unreliable in this case. + // FIXME: double-check that returning early is OK + result.check_api_call_result()?; // It is assumed that there is no overflow here let gas_charged_for_pubdata = diff --git a/core/node/api_server/src/tx_sender/result.rs b/core/node/api_server/src/tx_sender/result.rs index f4bda54efc6..a49313f0dd6 100644 --- a/core/node/api_server/src/tx_sender/result.rs +++ b/core/node/api_server/src/tx_sender/result.rs @@ -158,11 +158,27 @@ impl From for SubmitTxError { } } -pub(crate) trait ApiCallResult { +pub(crate) trait ApiCallResult: Sized { + fn check_api_call_result(&self) -> Result<(), SubmitTxError>; + fn into_api_call_result(self) -> Result, SubmitTxError>; } impl ApiCallResult for VmExecutionResultAndLogs { + fn check_api_call_result(&self) -> Result<(), SubmitTxError> { + match &self.result { + ExecutionResult::Success { .. } => Ok(()), + ExecutionResult::Revert { output } => Err(SubmitTxError::ExecutionReverted( + output.to_user_friendly_string(), + output.encoded_data(), + )), + ExecutionResult::Halt { reason } => { + let output: SandboxExecutionError = reason.clone().into(); + Err(output.into()) + } + } + } + fn into_api_call_result(self) -> Result, SubmitTxError> { match self.result { ExecutionResult::Success { output } => Ok(output), diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index cf2d017dfdc..4ddc6755d7e 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -24,7 +24,8 @@ use super::{gas_estimation::GasEstimator, *}; use crate::{ execution_sandbox::BlockStartInfo, testonly::{ - read_expensive_contract_bytecode, read_precompiles_contract_bytecode, TestAccount, + read_counter_contract_bytecode, read_expensive_contract_bytecode, + read_precompiles_contract_bytecode, TestAccount, COUNTER_CONTRACT_ADDRESS, EXPENSIVE_CONTRACT_ADDRESS, LOAD_TEST_ADDRESS, PRECOMPILES_CONTRACT_ADDRESS, }, web3::testonly::create_test_tx_sender, @@ -217,8 +218,7 @@ async fn eth_call_requires_single_connection() { assert_eq!(output, b"success!"); } -#[tokio::test] -async fn initial_gas_estimation_is_somewhat_accurate() { +async fn create_real_tx_sender() -> TxSender { let pool = ConnectionPool::::constrained_test_pool(1).await; let mut storage = pool.connection().await.unwrap(); let genesis_params = GenesisParams::mock(); @@ -228,12 +228,14 @@ async fn initial_gas_estimation_is_somewhat_accurate() { drop(storage); let tx_executor = TransactionExecutor::real(usize::MAX); - let (tx_sender, _) = create_test_tx_sender( - pool.clone(), - genesis_params.config().l2_chain_id, - tx_executor, - ) - .await; + create_test_tx_sender(pool, genesis_params.config().l2_chain_id, tx_executor) + .await + .0 +} + +#[tokio::test] +async fn initial_gas_estimation_is_somewhat_accurate() { + let tx_sender = create_real_tx_sender().await; let alice = K256PrivateKey::random(); let transfer_value = U256::from(1_000_000_000); @@ -321,22 +323,7 @@ async fn test_initial_estimate( state_override: StateOverride, tx: L2Tx, ) -> VmExecutionResultAndLogs { - let pool = ConnectionPool::::constrained_test_pool(1).await; - let mut storage = pool.connection().await.unwrap(); - let genesis_params = GenesisParams::mock(); - insert_genesis_batch(&mut storage, &genesis_params) - .await - .unwrap(); - drop(storage); - - let tx_executor = TransactionExecutor::real(usize::MAX); - let (tx_sender, _) = create_test_tx_sender( - pool.clone(), - genesis_params.config().l2_chain_id, - tx_executor, - ) - .await; - + let tx_sender = create_real_tx_sender().await; let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) .await .unwrap(); @@ -355,6 +342,15 @@ async fn test_initial_estimate( vm_result } +async fn test_initial_estimate_error(state_override: StateOverride, tx: L2Tx) -> SubmitTxError { + let tx_sender = create_real_tx_sender().await; + let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) + .await + .unwrap(); + estimator.adjust_transaction_fee(); + estimator.initialize().await.unwrap_err() +} + /// Estimates both transactions with initial writes and cleanup. #[test_casing(4, [10, 50, 200, 1_000])] #[tokio::test] @@ -451,6 +447,27 @@ async fn initial_estimate_for_code_oracle_tx() { ); } +#[tokio::test] +async fn revert_during_initial_estimate() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_counter_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + COUNTER_CONTRACT_ADDRESS, + contract_overrides, + )])); + + let tx = alice.create_reverting_counter_tx(); + let err = test_initial_estimate_error(state_override, tx).await; + let SubmitTxError::ExecutionReverted(err, _) = err else { + panic!("Unexpected error: {err:?}"); + }; + assert_eq!(err, "This method always reverts"); +} + #[tokio::test] async fn insufficient_funds_error_for_transfer() { let pool = ConnectionPool::::constrained_test_pool(1).await; @@ -493,21 +510,7 @@ async fn test_estimating_gas( tx: L2Tx, acceptable_overestimation: u64, ) { - let pool = ConnectionPool::::constrained_test_pool(1).await; - let mut storage = pool.connection().await.unwrap(); - let genesis_params = GenesisParams::mock(); - insert_genesis_batch(&mut storage, &genesis_params) - .await - .unwrap(); - drop(storage); - - let tx_executor = TransactionExecutor::real(usize::MAX); - let (tx_sender, _) = create_test_tx_sender( - pool.clone(), - genesis_params.config().l2_chain_id, - tx_executor, - ) - .await; + let tx_sender = create_real_tx_sender().await; let fee_scale_factor = 1.0; let fee = tx_sender @@ -630,4 +633,37 @@ async fn estimating_gas_for_code_oracle_tx() { test_estimating_gas(state_override, tx, 0).await; } -// FIXME: more test contracts: failures (counter, infinite) +#[tokio::test] +async fn estimating_gas_for_reverting_tx() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_counter_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + COUNTER_CONTRACT_ADDRESS, + contract_overrides, + )])); + + let tx = alice.create_reverting_counter_tx(); + let tx_sender = create_real_tx_sender().await; + + let fee_scale_factor = 1.0; + let acceptable_overestimation = 0; + for binary_search_kind in [BinarySearchKind::Full, BinarySearchKind::Optimized] { + let err = tx_sender + .get_txs_fee_in_wei( + tx.clone().into(), + fee_scale_factor, + acceptable_overestimation, + Some(state_override.clone()), + binary_search_kind, + ) + .await + .unwrap_err(); + assert_matches!(err, SubmitTxError::ExecutionReverted(..)); + } +} + +// FIXME: more test contracts: failures (infinite) From f6749231c079af60ee2dbfc4a9315beefe069fb3 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 14:23:57 +0300 Subject: [PATCH 08/14] Test infinite loop txs; fix factory deps for test txs --- core/node/api_server/src/testonly.rs | 42 +++++++-- .../src/tx_sender/gas_estimation.rs | 6 +- core/node/api_server/src/tx_sender/tests.rs | 85 ++++++++++++++++--- 3 files changed, 110 insertions(+), 23 deletions(-) diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index c92f7d7ce7c..78adac9af47 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -23,6 +23,10 @@ const COUNTER_CONTRACT_PATH: &str = "etc/contracts-test-data/artifacts-zk/contracts/counter/counter.sol/Counter.json"; pub(crate) const COUNTER_CONTRACT_ADDRESS: Address = Address::repeat_byte(4); +const INFINITE_LOOP_CONTRACT_PATH: &str = + "etc/contracts-test-data/artifacts-zk/contracts/infinite/infinite.sol/InfiniteLoop.json"; +pub(crate) const INFINITE_LOOP_CONTRACT_ADDRESS: Address = Address::repeat_byte(5); + pub(crate) fn read_expensive_contract_bytecode() -> Vec { read_bytecode(EXPENSIVE_CONTRACT_PATH) } @@ -35,6 +39,10 @@ pub(crate) fn read_counter_contract_bytecode() -> Vec { read_bytecode(COUNTER_CONTRACT_PATH) } +pub(crate) fn read_infinite_loop_contract_bytecode() -> Vec { + read_bytecode(INFINITE_LOOP_CONTRACT_PATH) +} + fn default_fee() -> Fee { Fee { gas_limit: 200_000.into(), @@ -63,9 +71,11 @@ pub(crate) trait TestAccount { fn create_expensive_cleanup_tx(&self) -> L2Tx; - fn create_decommitting_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx; + fn create_code_oracle_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx; fn create_reverting_counter_tx(&self) -> L2Tx; + + fn create_infinite_loop_tx(&self) -> L2Tx; } impl TestAccount for K256PrivateKey { @@ -113,7 +123,7 @@ impl TestAccount for K256PrivateKey { 0.into(), L2ChainId::default(), self, - get_loadnext_contract().factory_deps, + vec![], PaymasterParams::default(), ) .unwrap() @@ -133,13 +143,13 @@ impl TestAccount for K256PrivateKey { 0.into(), L2ChainId::default(), self, - get_loadnext_contract().factory_deps, + vec![], PaymasterParams::default(), ) .unwrap() } - fn create_decommitting_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx { + fn create_code_oracle_tx(&self, bytecode_hash: H256, expected_keccak_hash: H256) -> L2Tx { let calldata = load_contract(PRECOMPILES_CONTRACT_PATH) .function("callCodeOracle") .expect("no `callCodeOracle` function") @@ -156,7 +166,7 @@ impl TestAccount for K256PrivateKey { 0.into(), L2ChainId::default(), self, - get_loadnext_contract().factory_deps, + vec![], PaymasterParams::default(), ) .unwrap() @@ -176,7 +186,27 @@ impl TestAccount for K256PrivateKey { 0.into(), L2ChainId::default(), self, - get_loadnext_contract().factory_deps, + vec![], + PaymasterParams::default(), + ) + .unwrap() + } + + fn create_infinite_loop_tx(&self) -> L2Tx { + let calldata = load_contract(INFINITE_LOOP_CONTRACT_PATH) + .function("infiniteLoop") + .expect("no `infiniteLoop` function") + .encode_input(&[]) + .expect("failed encoding `infiniteLoop` input"); + L2Tx::new_signed( + Some(INFINITE_LOOP_CONTRACT_ADDRESS), + calldata, + Nonce(0), + default_fee(), + 0.into(), + L2ChainId::default(), + self, + vec![], PaymasterParams::default(), ) .unwrap() diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 13d8e3b5359..34775c2d9a2 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -359,15 +359,15 @@ impl<'a> GasEstimator<'a> { gas_charged_for_pubdata: 0, }) } else { - // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit. - // And getting how much pubdata it used. + // For L2 transactions, we estimate the amount of gas needed to cover for the pubdata by creating a transaction with infinite gas limit, + // and getting how much pubdata it used. let (result, _) = self .unadjusted_step(self.max_gas_limit) .await .context("estimate_gas step failed")?; // If the transaction has failed with such a large gas limit, we return an API error here right away, - // since the inferred gas bounds are be unreliable in this case. + // since the inferred gas bounds would be unreliable in this case. // FIXME: double-check that returning early is OK result.check_api_call_result()?; diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index 4ddc6755d7e..b40d6105004 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -25,12 +25,16 @@ use crate::{ execution_sandbox::BlockStartInfo, testonly::{ read_counter_contract_bytecode, read_expensive_contract_bytecode, - read_precompiles_contract_bytecode, TestAccount, COUNTER_CONTRACT_ADDRESS, - EXPENSIVE_CONTRACT_ADDRESS, LOAD_TEST_ADDRESS, PRECOMPILES_CONTRACT_ADDRESS, + read_infinite_loop_contract_bytecode, read_precompiles_contract_bytecode, TestAccount, + COUNTER_CONTRACT_ADDRESS, EXPENSIVE_CONTRACT_ADDRESS, INFINITE_LOOP_CONTRACT_ADDRESS, + LOAD_TEST_ADDRESS, PRECOMPILES_CONTRACT_ADDRESS, }, web3::testonly::create_test_tx_sender, }; +/// Initial pivot multiplier empirically sufficient for most tx types. +const DEFAULT_MULTIPLIER: f64 = 64.0 / 63.0; + #[tokio::test] async fn getting_nonce_for_account() { let l2_chain_id = L2ChainId::default(); @@ -314,7 +318,7 @@ async fn initial_estimate_for_load_test_transaction(tx_params: LoadnextContractE StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); let tx = alice.create_load_test_tx(tx_params); - test_initial_estimate(state_override, tx).await; + test_initial_estimate(state_override, tx, DEFAULT_MULTIPLIER).await; } /// Tests the lower bound and initial pivot extracted from the initial estimate (one with effectively infinite gas amount). @@ -322,6 +326,7 @@ async fn initial_estimate_for_load_test_transaction(tx_params: LoadnextContractE async fn test_initial_estimate( state_override: StateOverride, tx: L2Tx, + initial_pivot_multiplier: f64, ) -> VmExecutionResultAndLogs { let tx_sender = create_real_tx_sender().await; let mut estimator = GasEstimator::new(&tx_sender, tx.into(), Some(state_override)) @@ -336,7 +341,8 @@ async fn test_initial_estimate( assert!(vm_result.result.is_failed(), "{:?}", vm_result.result); // A slightly larger limit should work. - let initial_pivot = initial_estimate.total_gas_charged.unwrap() * 64 / 63; + let initial_pivot = + (initial_estimate.total_gas_charged.unwrap() as f64 * initial_pivot_multiplier) as u64; let (vm_result, _) = estimator.unadjusted_step(initial_pivot).await.unwrap(); assert!(!vm_result.result.is_failed(), "{:?}", vm_result.result); vm_result @@ -367,7 +373,7 @@ async fn initial_estimate_for_expensive_contract(write_count: usize) { )])); let tx = alice.create_expensive_tx(write_count); - let vm_result = test_initial_estimate(state_override, tx).await; + let vm_result = test_initial_estimate(state_override, tx, DEFAULT_MULTIPLIER).await; let contract_logs = vm_result.logs.storage_logs.into_iter().filter_map(|log| { (*log.log.key.address() == EXPENSIVE_CONTRACT_ADDRESS) @@ -383,7 +389,7 @@ async fn initial_estimate_for_expensive_contract(write_count: usize) { )])); let tx = alice.create_expensive_cleanup_tx(); - test_initial_estimate(state_override, tx).await; + test_initial_estimate(state_override, tx, DEFAULT_MULTIPLIER).await; } #[tokio::test] @@ -430,15 +436,17 @@ async fn initial_estimate_for_code_oracle_tx() { ]; let mut decomitter_stats = 0.0; for (hash, keccak_hash) in warm_bytecode_hashes { - let tx = alice.create_decommitting_tx(hash, keccak_hash); - let vm_result = test_initial_estimate(state_override.clone(), tx).await; + println!("Testing bytecode: {hash:?}"); + let tx = alice.create_code_oracle_tx(hash, keccak_hash); + let vm_result = test_initial_estimate(state_override.clone(), tx, DEFAULT_MULTIPLIER).await; let stats = &vm_result.statistics.circuit_statistic; decomitter_stats = stats.code_decommitter.max(decomitter_stats); } assert!(decomitter_stats > 0.0); - let tx = alice.create_decommitting_tx(huge_contract_bytecode_hash, huge_contract_keccak_hash); - let vm_result = test_initial_estimate(state_override, tx).await; + println!("Testing large bytecode"); + let tx = alice.create_code_oracle_tx(huge_contract_bytecode_hash, huge_contract_keccak_hash); + let vm_result = test_initial_estimate(state_override, tx, 1.05).await; // Sanity check: the transaction should spend significantly more on decommitment compared to previous ones let new_decomitter_stats = vm_result.statistics.circuit_statistic.code_decommitter; assert!( @@ -468,6 +476,25 @@ async fn revert_during_initial_estimate() { assert_eq!(err, "This method always reverts"); } +#[tokio::test] +async fn out_of_gas_during_initial_estimate() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_infinite_loop_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + INFINITE_LOOP_CONTRACT_ADDRESS, + contract_overrides, + )])); + + let tx = alice.create_infinite_loop_tx(); + let err = test_initial_estimate_error(state_override, tx).await; + // Unfortunately, we don't provide human-readable out-of-gas errors at the time + assert_matches!(err, SubmitTxError::ExecutionReverted(msg, _) if msg.is_empty()); +} + #[tokio::test] async fn insufficient_funds_error_for_transfer() { let pool = ConnectionPool::::constrained_test_pool(1).await; @@ -617,7 +644,7 @@ async fn estimating_gas_for_code_oracle_tx() { let huge_contact_address = Address::repeat_byte(23); let huge_contract_bytecode = vec![0_u8; 10_001 * 32]; let huge_contract_bytecode_hash = hash_bytecode(&huge_contract_bytecode); - let huge_contract_keccak_hash = keccak256(&huge_contract_bytecode); + let huge_contract_keccak_hash = H256(keccak256(&huge_contract_bytecode)); let huge_contract_overrides = OverrideAccount { code: Some(Bytecode::new(huge_contract_bytecode).unwrap()), ..OverrideAccount::default() @@ -627,8 +654,7 @@ async fn estimating_gas_for_code_oracle_tx() { (PRECOMPILES_CONTRACT_ADDRESS, contract_overrides), (huge_contact_address, huge_contract_overrides), ])); - let tx = - alice.create_decommitting_tx(huge_contract_bytecode_hash, H256(huge_contract_keccak_hash)); + let tx = alice.create_code_oracle_tx(huge_contract_bytecode_hash, huge_contract_keccak_hash); test_estimating_gas(state_override, tx, 0).await; } @@ -666,4 +692,35 @@ async fn estimating_gas_for_reverting_tx() { } } -// FIXME: more test contracts: failures (infinite) +#[tokio::test] +async fn estimating_gas_for_infinite_loop_tx() { + let alice = K256PrivateKey::random(); + let contract_bytecode = read_infinite_loop_contract_bytecode(); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = StateOverride::new(HashMap::from([( + INFINITE_LOOP_CONTRACT_ADDRESS, + contract_overrides, + )])); + + let tx = alice.create_infinite_loop_tx(); + let tx_sender = create_real_tx_sender().await; + + let fee_scale_factor = 1.0; + let acceptable_overestimation = 0; + for binary_search_kind in [BinarySearchKind::Full, BinarySearchKind::Optimized] { + let err = tx_sender + .get_txs_fee_in_wei( + tx.clone().into(), + fee_scale_factor, + acceptable_overestimation, + Some(state_override.clone()), + binary_search_kind, + ) + .await + .unwrap_err(); + assert_matches!(err, SubmitTxError::ExecutionReverted(msg, _) if msg.is_empty()); + } +} From 9b4c7b56c9c4b2f55d4c5e1e63ce4a9a6be09752 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 15:13:55 +0300 Subject: [PATCH 09/14] Add metrics for gas estimation --- .../src/execution_sandbox/vm_metrics.rs | 10 +++++ .../src/tx_sender/gas_estimation.rs | 43 +++++++++++++------ 2 files changed, 40 insertions(+), 13 deletions(-) diff --git a/core/node/api_server/src/execution_sandbox/vm_metrics.rs b/core/node/api_server/src/execution_sandbox/vm_metrics.rs index cbfe7e90bd0..613475b6ef9 100644 --- a/core/node/api_server/src/execution_sandbox/vm_metrics.rs +++ b/core/node/api_server/src/execution_sandbox/vm_metrics.rs @@ -89,8 +89,18 @@ pub(crate) struct SandboxMetrics { pub(super) sandbox_execution_permits: Histogram, #[metrics(buckets = Buckets::LATENCIES)] submit_tx: Family>, + + /// Number of iterations necessary to estimate gas for a transaction. #[metrics(buckets = Buckets::linear(0.0..=30.0, 3.0))] pub estimate_gas_binary_search_iterations: Histogram, + /// Relative difference between the unscaled final gas estimate and the optimized lower bound. Positive if the lower bound + /// is (as expected) lower than the final gas estimate. + #[metrics(buckets = Buckets::linear(-0.05..=0.15, 0.01))] + pub estimate_gas_lower_bound_relative_diff: Histogram, + /// Relative difference between the optimistic gas limit and the unscaled final gas estimate. Positive if the optimistic gas limit + /// is (as expected) greater than the final gas estimate. + #[metrics(buckets = Buckets::linear(-0.05..=0.15, 0.01))] + pub estimate_gas_optimistic_gas_limit_relative_diff: Histogram, } impl SandboxMetrics { diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 34775c2d9a2..69c940a13d9 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -71,6 +71,11 @@ impl TxSender { estimation_started_at.elapsed() ); + let optimized_lower_bound = initial_estimate.lower_gas_bound_without_overhead(); + // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. + // If the transaction succeeds, it will discard most of the search space at once. + let optimistic_gas_limit = initial_estimate.optimistic_gas_limit_without_overhead(); + let (bounds, initial_pivot) = match kind { BinarySearchKind::Full => { let lower_bound = initial_estimate.gas_charged_for_pubdata; @@ -78,26 +83,38 @@ impl TxSender { (lower_bound..=upper_bound, None) } BinarySearchKind::Optimized => { - let lower_bound = initial_estimate - .lower_gas_bound_without_overhead() - .unwrap_or(0); + let lower_bound = optimized_lower_bound.unwrap_or(0); let upper_bound = MAX_L2_TX_GAS_LIMIT; - - // Perform an initial search iteration with the pivot slightly greater than `gas_used` to account for 63/64 rule for far calls etc. - // If the transaction succeeds, it will discard most of the search space at once. - let optimistic_gas_limit = initial_estimate - .optimistic_gas_limit_without_overhead() - .filter(|&gas| { - // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. - gas < (lower_bound + upper_bound) / 2 - }); - (lower_bound..=upper_bound, optimistic_gas_limit) + let initial_pivot = optimistic_gas_limit.filter(|&gas| { + // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. + gas < (lower_bound + upper_bound) / 2 + }); + (lower_bound..=upper_bound, initial_pivot) } }; let unscaled_gas_limit = Self::binary_search(&estimator, bounds, initial_pivot, acceptable_overestimation) .await?; + // Metrics are intentionally reported regardless of the binary search mode, so that the collected stats can be used to adjust + // optimized binary search params (e.g., the initial pivot multiplier). + if let Some(lower_bound) = optimized_lower_bound { + let tx_overhead = estimator.tx_overhead(unscaled_gas_limit); + let diff = (unscaled_gas_limit as f64 - lower_bound as f64) + / (unscaled_gas_limit + tx_overhead) as f64; + SANDBOX_METRICS + .estimate_gas_lower_bound_relative_diff + .observe(diff); + } + if let Some(optimistic_gas_limit) = optimistic_gas_limit { + let tx_overhead = estimator.tx_overhead(unscaled_gas_limit); + let diff = (optimistic_gas_limit as f64 - unscaled_gas_limit as f64) + / (unscaled_gas_limit + tx_overhead) as f64; + SANDBOX_METRICS + .estimate_gas_optimistic_gas_limit_relative_diff + .observe(diff); + } + let suggested_gas_limit = (unscaled_gas_limit as f64 * estimated_fee_scale_factor) as u64; estimator .finalize(suggested_gas_limit, estimated_fee_scale_factor) From a2b822d58e5b9a62bdde3eee63295b9dcd037ea7 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Mon, 23 Sep 2024 21:17:45 +0300 Subject: [PATCH 10/14] Document gas estimation --- .../src/tx_sender/gas_estimation.rs | 36 +++++++++++++++---- 1 file changed, 29 insertions(+), 7 deletions(-) diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 5b64ce8f48e..2af7621b021 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -216,23 +216,40 @@ pub(super) struct InitialGasEstimate { pub total_gas_charged: Option, /// Set to `None` if not estimated (e.g., for L1 transactions). pub computational_gas_used: Option, + /// Operator-defined overhead for the estimated transaction. For recent VM versions, the overhead only depends + /// on the transaction encoding size. pub operator_overhead: u64, pub gas_charged_for_pubdata: u64, } impl InitialGasEstimate { + /// Returns the lower gas limit bound, i.e., gas limit that is guaranteed to be lower than the minimum passing gas limit, + /// but is reasonably close to it. + /// + /// # Background + /// /// Total gas charged for a transaction consists of: /// - /// - Operator-set overhead (`operator_overhead`) + /// - Operator-set overhead (`self.operator_overhead`) /// - Intrinsic bootloader overhead - /// - Gas used during validation / execution - /// - Gas charged for pubdata at the end of execution (`gas_for_pubdata`) + /// - Gas used during validation / execution (`self.computational_gas_used`) + /// - Gas charged for pubdata at the end of execution (`self.gas_charged_for_pubdata`) /// /// We add `operator_overhead` manually to the binary search argument at each `step()` because it depends on the gas limit in the general case, - /// so this value corresponds to the other 3 terms. + /// so the returned value corresponds to the other 3 terms. /// /// If the value cannot be computed, it is set to `None`. pub fn lower_gas_bound_without_overhead(&self) -> Option { + // The two ways to compute the used gas (by `computational_gas_used` and by the charged gas) don't return the identical values + // due to various factors: + // + // - `computational_gas_used` tracks gas usage for the entire VM execution, while the transaction initiator (or a paymaster) is only charged + // for a part of it. + // - The bootloader is somewhat lenient in the case pubdata costs are approximately equal to the amount of gas left + // (i.e., for some transaction types, such as base token transfers, there exists an entire range of gas limit values + // which all lead to a successful execution with 0 refund). + // + // We use the lesser of these two estimates as the lower bound. let mut total_gas_bound = self.computational_gas_used? + self.gas_charged_for_pubdata; if let Some(gas_charged) = self.total_gas_charged { total_gas_bound = total_gas_bound.min(gas_charged); @@ -240,18 +257,23 @@ impl InitialGasEstimate { total_gas_bound.checked_sub(self.operator_overhead) } + /// Returns heuristically chosen gas limit without operator overhead that should be sufficient for most transactions. + /// This value is reasonably close to the lower gas limit bound, so that when used as the initial binary search pivot, + /// it will discard most of the search space in the average case. pub fn optimistic_gas_limit_without_overhead(&self) -> Option { let gas_charged_without_overhead = self .total_gas_charged? .checked_sub(self.operator_overhead)?; // 21/20 is an empirical multiplier. It is higher than what empirically suffices for some common transactions; - // one can argue that using 64/63 multiplier would be more accurate due to the 63/64 rule for far calls - // (however, far calls are not the only source of gas overhead in Era). + // one can argue that using 64/63 multiplier would be more accurate due to the 63/64 rule for far calls. + // However, far calls are not the only source of gas overhead in Era; another one are decommit operations. Some(gas_charged_without_overhead * 21 / 20) } } -// Public for testing purposes +/// Encapsulates gas estimation process for a specific transaction. +/// +/// Public for testing purposes. #[derive(Debug)] pub(super) struct GasEstimator<'a> { sender: &'a TxSender, From da627675984e82d84c1a9e4205d17a706d777e09 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Tue, 24 Sep 2024 10:33:42 +0300 Subject: [PATCH 11/14] Test deep recursion --- Cargo.lock | 1 + core/node/api_server/Cargo.toml | 1 + core/node/api_server/src/testonly.rs | 18 ++++- core/node/api_server/src/tx_sender/tests.rs | 85 ++++++++++++++++++++- 4 files changed, 102 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c2d01881fcd..fb55e9ddc03 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10196,6 +10196,7 @@ dependencies = [ "tower-http", "tracing", "vise", + "zk_evm 0.150.5", "zksync_config", "zksync_consensus_roles", "zksync_contracts", diff --git a/core/node/api_server/Cargo.toml b/core/node/api_server/Cargo.toml index 040e2a94a11..d0723a9d23e 100644 --- a/core/node/api_server/Cargo.toml +++ b/core/node/api_server/Cargo.toml @@ -56,6 +56,7 @@ tower-http = { workspace = true, features = ["cors", "metrics"] } lru.workspace = true [dev-dependencies] +zk_evm_1_5_0.workspace = true zksync_node_genesis.workspace = true zksync_node_test_utils.workspace = true diff --git a/core/node/api_server/src/testonly.rs b/core/node/api_server/src/testonly.rs index 78adac9af47..5ee9cfb8ef1 100644 --- a/core/node/api_server/src/testonly.rs +++ b/core/node/api_server/src/testonly.rs @@ -1,5 +1,8 @@ //! Test utils shared among multiple modules. +use std::iter; + +use zk_evm_1_5_0::zkevm_opcode_defs::decoding::{EncodingModeProduction, VmEncodingMode}; use zksync_contracts::{ get_loadnext_contract, load_contract, read_bytecode, test_contracts::LoadnextContractExecutionParams, @@ -43,6 +46,15 @@ pub(crate) fn read_infinite_loop_contract_bytecode() -> Vec { read_bytecode(INFINITE_LOOP_CONTRACT_PATH) } +/// Inflates the provided bytecode by appending the specified amount of NOP instructions at the end. +pub(crate) fn inflate_bytecode(bytecode: &mut Vec, nop_count: usize) { + bytecode.extend( + iter::repeat(EncodingModeProduction::nop_encoding().to_be_bytes()) + .take(nop_count) + .flatten(), + ); +} + fn default_fee() -> Fee { Fee { gas_limit: 200_000.into(), @@ -103,7 +115,11 @@ impl TestAccount for K256PrivateKey { 0.into(), L2ChainId::default(), self, - get_loadnext_contract().factory_deps, + if params.deploys > 0 { + get_loadnext_contract().factory_deps + } else { + vec![] + }, PaymasterParams::default(), ) .unwrap() diff --git a/core/node/api_server/src/tx_sender/tests.rs b/core/node/api_server/src/tx_sender/tests.rs index d7fabce663f..36c95fa5db0 100644 --- a/core/node/api_server/src/tx_sender/tests.rs +++ b/core/node/api_server/src/tx_sender/tests.rs @@ -24,7 +24,7 @@ use super::{gas_estimation::GasEstimator, *}; use crate::{ execution_sandbox::BlockStartInfo, testonly::{ - read_counter_contract_bytecode, read_expensive_contract_bytecode, + inflate_bytecode, read_counter_contract_bytecode, read_expensive_contract_bytecode, read_infinite_loop_contract_bytecode, read_precompiles_contract_bytecode, TestAccount, COUNTER_CONTRACT_ADDRESS, EXPENSIVE_CONTRACT_ADDRESS, INFINITE_LOOP_CONTRACT_ADDRESS, LOAD_TEST_ADDRESS, PRECOMPILES_CONTRACT_ADDRESS, @@ -298,7 +298,7 @@ const LOAD_TEST_CASES: TestCases = test_casing: events: 0, ..LoadnextContractExecutionParams::default() }, - // Deep recursion + // Moderately deep recursion (very deep recursion is tested separately) LoadnextContractExecutionParams { recursive_calls: 10, ..LoadnextContractExecutionParams::default() @@ -333,6 +333,66 @@ async fn initial_estimate_for_load_test_transaction(tx_params: LoadnextContractE test_initial_estimate(state_override, tx, DEFAULT_MULTIPLIER).await; } +#[test_casing(2, [false, true])] +#[tokio::test] +async fn initial_estimate_for_deep_recursion(with_reads: bool) { + let alice = K256PrivateKey::random(); + let load_test_state = HashMap::from([(H256::zero(), H256::from_low_u64_be(100))]); + let load_test_overrides = OverrideAccount { + code: Some(Bytecode::new(get_loadnext_contract().bytecode).unwrap()), + state: Some(OverrideState::State(load_test_state)), + ..OverrideAccount::default() + }; + let state_override = + StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); + + // Reads are chosen because they represent the worst case. Reads don't influence the amount of pubdata; + // i.e., they don't make it easier to execute a transaction because of additional gas reserved for pubdata. + // OTOH, reads still increase the amount of computational gas used on each nested call. + // + // Initial pivot multipliers below are the smallest ones with 0.1 precision. `DEFAULT_MULTIPLIER` works for smaller + // recursion depths because the transaction emits enough pubdata to cover gas deductions due to the 63/64 rule. + let depths_and_multipliers: &[_] = if with_reads { + &[(25, DEFAULT_MULTIPLIER), (50, 1.2), (75, 1.4), (100, 1.7)] + } else { + &[ + (50, DEFAULT_MULTIPLIER), + (75, 1.2), + (100, 1.4), + (125, 1.7), + (150, 2.1), + ] + }; + for &(recursion_depth, multiplier) in depths_and_multipliers { + println!("Testing recursion depth {recursion_depth}"); + let tx = alice.create_load_test_tx(LoadnextContractExecutionParams { + recursive_calls: recursion_depth, + reads: if with_reads { 10 } else { 0 }, + ..LoadnextContractExecutionParams::empty() + }); + test_initial_estimate(state_override.clone(), tx, multiplier).await; + } +} + +#[tokio::test] +async fn initial_estimate_for_deep_recursion_with_large_bytecode() { + let alice = K256PrivateKey::random(); + let mut contract_bytecode = get_loadnext_contract().bytecode; + inflate_bytecode(&mut contract_bytecode, 50_000); + let load_test_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + let state_override = + StateOverride::new(HashMap::from([(LOAD_TEST_ADDRESS, load_test_overrides)])); + let tx = alice.create_load_test_tx(LoadnextContractExecutionParams { + recursive_calls: 100, + ..LoadnextContractExecutionParams::empty() + }); + + test_initial_estimate(state_override, tx, 1.35).await; +} + /// Tests the lower bound and initial pivot extracted from the initial estimate (one with effectively infinite gas amount). /// Returns the VM result for a VM run with the initial pivot. async fn test_initial_estimate( @@ -467,6 +527,27 @@ async fn initial_estimate_for_code_oracle_tx() { ); } +#[tokio::test] +async fn initial_estimate_with_large_free_bytecode() { + let alice = K256PrivateKey::random(); + let mut contract_bytecode = read_precompiles_contract_bytecode(); + inflate_bytecode(&mut contract_bytecode, 50_000); + let contract_bytecode_hash = hash_bytecode(&contract_bytecode); + let contract_keccak_hash = H256(keccak256(&contract_bytecode)); + let contract_overrides = OverrideAccount { + code: Some(Bytecode::new(contract_bytecode).unwrap()), + ..OverrideAccount::default() + }; + + let state_override = StateOverride::new(HashMap::from([( + PRECOMPILES_CONTRACT_ADDRESS, + contract_overrides, + )])); + // Ask the test contract to decommit itself. This should refund the decommit costs, but it will be charged at first. + let tx = alice.create_code_oracle_tx(contract_bytecode_hash, contract_keccak_hash); + test_initial_estimate(state_override, tx, 1.05).await; +} + #[tokio::test] async fn revert_during_initial_estimate() { let alice = K256PrivateKey::random(); From 1179fc28f6d10dc67bbcf71cabc7f815c8fdcc74 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Tue, 1 Oct 2024 13:54:51 +0300 Subject: [PATCH 12/14] Resolve smaller nits --- core/node/api_server/src/execution_sandbox/storage.rs | 1 - core/node/api_server/src/tx_sender/gas_estimation.rs | 11 +++++------ 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/core/node/api_server/src/execution_sandbox/storage.rs b/core/node/api_server/src/execution_sandbox/storage.rs index a6c5d7afcf1..bf775d48490 100644 --- a/core/node/api_server/src/execution_sandbox/storage.rs +++ b/core/node/api_server/src/execution_sandbox/storage.rs @@ -56,7 +56,6 @@ impl StorageWithOverrides { let code_key = get_code_key(account); let code_hash = code.hash(); self.overridden_slots.insert(code_key, code_hash); - // FIXME: is this addition correct? let known_code_key = get_known_code_key(&code_hash); self.overridden_slots .insert(known_code_key, H256::from_low_u64_be(1)); diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 2af7621b021..5b83f40aab1 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -8,12 +8,11 @@ use zksync_multivm::{ adjust_pubdata_price_for_tx, derive_base_fee_and_gas_per_pubdata, derive_overhead, get_max_batch_gas_limit, }, - zk_evm_latest::ethereum_types::H256, }; use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{ api::state_override::StateOverride, fee::Fee, fee_model::BatchFeeInput, get_code_key, - ExecuteTransactionCommon, PackedEthSignature, ProtocolVersionId, Transaction, + ExecuteTransactionCommon, PackedEthSignature, ProtocolVersionId, Transaction, H256, }; use super::{result::ApiCallResult, SubmitTxError, TxSender}; @@ -59,12 +58,13 @@ impl TxSender { let (bounds, initial_pivot) = match kind { BinarySearchKind::Full => { let lower_bound = initial_estimate.gas_charged_for_pubdata; - let upper_bound = MAX_L2_TX_GAS_LIMIT; + let upper_bound = MAX_L2_TX_GAS_LIMIT + initial_estimate.gas_charged_for_pubdata; (lower_bound..=upper_bound, None) } BinarySearchKind::Optimized => { - let lower_bound = optimized_lower_bound.unwrap_or(0); - let upper_bound = MAX_L2_TX_GAS_LIMIT; + let lower_bound = + optimized_lower_bound.unwrap_or(initial_estimate.gas_charged_for_pubdata); + let upper_bound = MAX_L2_TX_GAS_LIMIT + initial_estimate.gas_charged_for_pubdata; let initial_pivot = optimistic_gas_limit.filter(|&gas| { // If `optimistic_gas_limit` is greater than the ordinary binary search pivot, there's no sense using it. gas < (lower_bound + upper_bound) / 2 @@ -387,7 +387,6 @@ impl<'a> GasEstimator<'a> { .context("estimate_gas step failed")?; // If the transaction has failed with such a large gas limit, we return an API error here right away, // since the inferred gas bounds would be unreliable in this case. - // FIXME: double-check that returning early is OK result.check_api_call_result()?; // It is assumed that there is no overflow here From db22a7683955ff6419d8115623819f3152477ec1 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Tue, 1 Oct 2024 14:24:56 +0300 Subject: [PATCH 13/14] Allow configuring binary search kind --- core/bin/external_node/src/config/mod.rs | 10 ++++++++++ core/lib/config/src/configs/api.rs | 5 +++++ core/lib/config/src/testonly.rs | 1 + core/lib/env_config/src/api.rs | 1 + core/lib/protobuf_config/src/api.rs | 2 ++ core/lib/protobuf_config/src/proto/config/api.proto | 2 ++ core/node/api_server/src/tx_sender/gas_estimation.rs | 11 ++++++++++- core/node/api_server/src/web3/namespaces/eth.rs | 3 ++- core/node/api_server/src/web3/namespaces/zks.rs | 3 ++- core/node/api_server/src/web3/state.rs | 4 +++- 10 files changed, 38 insertions(+), 4 deletions(-) diff --git a/core/bin/external_node/src/config/mod.rs b/core/bin/external_node/src/config/mod.rs index f8241deae26..bbb39759c6a 100644 --- a/core/bin/external_node/src/config/mod.rs +++ b/core/bin/external_node/src/config/mod.rs @@ -327,6 +327,10 @@ pub(crate) struct OptionalENConfig { /// The max possible number of gas that `eth_estimateGas` is allowed to overestimate. #[serde(default = "OptionalENConfig::default_estimate_gas_acceptable_overestimation")] pub estimate_gas_acceptable_overestimation: u32, + /// Enables optimizations for the binary search of the gas limit in `eth_estimateGas`. These optimizations are currently + /// considered experimental. + #[serde(default)] + pub estimate_gas_optimize_search: bool, /// The multiplier to use when suggesting gas price. Should be higher than one, /// otherwise if the L1 prices soar, the suggested gas price won't be sufficient to be included in block. #[serde(default = "OptionalENConfig::default_gas_price_scale_factor")] @@ -558,6 +562,11 @@ impl OptionalENConfig { web3_json_rpc.estimate_gas_acceptable_overestimation, default_estimate_gas_acceptable_overestimation ), + estimate_gas_optimize_search: general_config + .api_config + .as_ref() + .map(|a| a.web3_json_rpc.estimate_gas_optimize_search) + .unwrap_or_default(), gas_price_scale_factor: load_config_or_default!( general_config.api_config, web3_json_rpc.gas_price_scale_factor, @@ -1382,6 +1391,7 @@ impl From<&ExternalNodeConfig> for InternalApiConfig { estimate_gas_acceptable_overestimation: config .optional .estimate_gas_acceptable_overestimation, + estimate_gas_optimize_search: config.optional.estimate_gas_optimize_search, bridge_addresses: BridgeAddresses { l1_erc20_default_bridge: config.remote.l1_erc20_bridge_proxy_addr, l2_erc20_default_bridge: config.remote.l2_erc20_bridge_addr, diff --git a/core/lib/config/src/configs/api.rs b/core/lib/config/src/configs/api.rs index ca42cd5e5f8..86c9ebd074d 100644 --- a/core/lib/config/src/configs/api.rs +++ b/core/lib/config/src/configs/api.rs @@ -169,6 +169,10 @@ pub struct Web3JsonRpcConfig { pub estimate_gas_scale_factor: f64, /// The max possible number of gas that `eth_estimateGas` is allowed to overestimate. pub estimate_gas_acceptable_overestimation: u32, + /// Enables optimizations for the binary search of the gas limit in `eth_estimateGas`. These optimizations are currently + /// considered experimental. + #[serde(default)] + pub estimate_gas_optimize_search: bool, /// Max possible size of an ABI encoded tx (in bytes). pub max_tx_size: usize, /// Max number of cache misses during one VM execution. If the number of cache misses exceeds this value, the API server panics. @@ -237,6 +241,7 @@ impl Web3JsonRpcConfig { gas_price_scale_factor: 1.2, estimate_gas_scale_factor: 1.2, estimate_gas_acceptable_overestimation: 1000, + estimate_gas_optimize_search: false, max_tx_size: 1000000, vm_execution_cache_misses_limit: Default::default(), vm_concurrency_limit: Default::default(), diff --git a/core/lib/config/src/testonly.rs b/core/lib/config/src/testonly.rs index 5a5a5430442..f873a2ad224 100644 --- a/core/lib/config/src/testonly.rs +++ b/core/lib/config/src/testonly.rs @@ -77,6 +77,7 @@ impl Distribution for EncodeDist { gas_price_scale_factor: self.sample(rng), estimate_gas_scale_factor: self.sample(rng), estimate_gas_acceptable_overestimation: self.sample(rng), + estimate_gas_optimize_search: self.sample(rng), max_tx_size: self.sample(rng), vm_execution_cache_misses_limit: self.sample(rng), vm_concurrency_limit: self.sample(rng), diff --git a/core/lib/env_config/src/api.rs b/core/lib/env_config/src/api.rs index c6485d54d6b..53efea9a784 100644 --- a/core/lib/env_config/src/api.rs +++ b/core/lib/env_config/src/api.rs @@ -69,6 +69,7 @@ mod tests { estimate_gas_scale_factor: 1.0f64, gas_price_scale_factor: 1.2, estimate_gas_acceptable_overestimation: 1000, + estimate_gas_optimize_search: false, max_tx_size: 1000000, vm_execution_cache_misses_limit: None, vm_concurrency_limit: Some(512), diff --git a/core/lib/protobuf_config/src/api.rs b/core/lib/protobuf_config/src/api.rs index f4718c9f796..a0c3825228a 100644 --- a/core/lib/protobuf_config/src/api.rs +++ b/core/lib/protobuf_config/src/api.rs @@ -84,6 +84,7 @@ impl ProtoRepr for proto::Web3JsonRpc { &self.estimate_gas_acceptable_overestimation, ) .context("acceptable_overestimation")?, + estimate_gas_optimize_search: self.estimate_gas_optimize_search.unwrap_or(false), max_tx_size: required(&self.max_tx_size) .and_then(|x| Ok((*x).try_into()?)) .context("max_tx_size")?, @@ -167,6 +168,7 @@ impl ProtoRepr for proto::Web3JsonRpc { estimate_gas_acceptable_overestimation: Some( this.estimate_gas_acceptable_overestimation, ), + estimate_gas_optimize_search: Some(this.estimate_gas_optimize_search), max_tx_size: Some(this.max_tx_size.try_into().unwrap()), vm_execution_cache_misses_limit: this .vm_execution_cache_misses_limit diff --git a/core/lib/protobuf_config/src/proto/config/api.proto b/core/lib/protobuf_config/src/proto/config/api.proto index e08677adc44..68475e442fd 100644 --- a/core/lib/protobuf_config/src/proto/config/api.proto +++ b/core/lib/protobuf_config/src/proto/config/api.proto @@ -40,6 +40,8 @@ message Web3JsonRpc { repeated MaxResponseSizeOverride max_response_body_size_overrides = 31; repeated string api_namespaces = 32; // Optional, if empty all namespaces are available optional bool extended_api_tracing = 33; // optional, default false + optional bool estimate_gas_optimize_search = 34; // optional, default false + reserved 15; reserved "l1_to_l2_transactions_compatibility_mode"; reserved 11; reserved "request_timeout"; reserved 12; reserved "account_pks"; diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 5b83f40aab1..5d571408039 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -23,10 +23,19 @@ pub(crate) enum BinarySearchKind { /// Full binary search. Full, /// Binary search with an optimized initial pivot. - #[allow(dead_code)] // FIXME Optimized, } +impl BinarySearchKind { + pub(crate) fn new(optimize: bool) -> Self { + if optimize { + Self::Optimized + } else { + Self::Full + } + } +} + impl TxSender { #[tracing::instrument(level = "debug", skip_all, fields( initiator = ?tx.initiator_account(), diff --git a/core/node/api_server/src/web3/namespaces/eth.rs b/core/node/api_server/src/web3/namespaces/eth.rs index 266e3901552..1d60d839e4e 100644 --- a/core/node/api_server/src/web3/namespaces/eth.rs +++ b/core/node/api_server/src/web3/namespaces/eth.rs @@ -130,6 +130,7 @@ impl EthNamespace { let scale_factor = self.state.api_config.estimate_gas_scale_factor; let acceptable_overestimation = self.state.api_config.estimate_gas_acceptable_overestimation; + let search_kind = BinarySearchKind::new(self.state.api_config.estimate_gas_optimize_search); let fee = self .state @@ -139,7 +140,7 @@ impl EthNamespace { scale_factor, acceptable_overestimation as u64, state_override, - BinarySearchKind::Full, + search_kind, ) .await?; Ok(fee.gas_limit) diff --git a/core/node/api_server/src/web3/namespaces/zks.rs b/core/node/api_server/src/web3/namespaces/zks.rs index b27f553a262..61456095d67 100644 --- a/core/node/api_server/src/web3/namespaces/zks.rs +++ b/core/node/api_server/src/web3/namespaces/zks.rs @@ -105,6 +105,7 @@ impl ZksNamespace { let scale_factor = self.state.api_config.estimate_gas_scale_factor; let acceptable_overestimation = self.state.api_config.estimate_gas_acceptable_overestimation; + let search_kind = BinarySearchKind::new(self.state.api_config.estimate_gas_optimize_search); Ok(self .state @@ -114,7 +115,7 @@ impl ZksNamespace { scale_factor, acceptable_overestimation as u64, state_override, - BinarySearchKind::Full, + search_kind, ) .await?) } diff --git a/core/node/api_server/src/web3/state.rs b/core/node/api_server/src/web3/state.rs index 5c8b47dabeb..8cbb75103cd 100644 --- a/core/node/api_server/src/web3/state.rs +++ b/core/node/api_server/src/web3/state.rs @@ -90,7 +90,7 @@ impl BlockStartInfo { } /// Configuration values for the API. -/// This structure is detached from `ZkSyncConfig`, since different node types (main, external, etc) +/// This structure is detached from `ZkSyncConfig`, since different node types (main, external, etc.) /// may require different configuration layouts. /// The intention is to only keep the actually used information here. #[derive(Debug, Clone)] @@ -101,6 +101,7 @@ pub struct InternalApiConfig { pub max_tx_size: usize, pub estimate_gas_scale_factor: f64, pub estimate_gas_acceptable_overestimation: u32, + pub estimate_gas_optimize_search: bool, pub bridge_addresses: api::BridgeAddresses, pub bridgehub_proxy_addr: Option
, pub state_transition_proxy_addr: Option
, @@ -128,6 +129,7 @@ impl InternalApiConfig { estimate_gas_scale_factor: web3_config.estimate_gas_scale_factor, estimate_gas_acceptable_overestimation: web3_config .estimate_gas_acceptable_overestimation, + estimate_gas_optimize_search: web3_config.estimate_gas_optimize_search, bridge_addresses: api::BridgeAddresses { l1_erc20_default_bridge: contracts_config.l1_erc20_bridge_proxy_addr, l2_erc20_default_bridge: contracts_config.l2_erc20_bridge_addr, From 102025cc326405790eba73d0a672e2fac02b1bd5 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Tue, 1 Oct 2024 16:50:04 +0300 Subject: [PATCH 14/14] Log search params / outputs --- .../api_server/src/tx_sender/gas_estimation.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/core/node/api_server/src/tx_sender/gas_estimation.rs b/core/node/api_server/src/tx_sender/gas_estimation.rs index 5d571408039..f5e42875a3d 100644 --- a/core/node/api_server/src/tx_sender/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/gas_estimation.rs @@ -82,7 +82,7 @@ impl TxSender { } }; - let unscaled_gas_limit = + let (unscaled_gas_limit, iteration_count) = Self::binary_search(&estimator, bounds, initial_pivot, acceptable_overestimation) .await?; // Metrics are intentionally reported regardless of the binary search mode, so that the collected stats can be used to adjust @@ -103,6 +103,14 @@ impl TxSender { .estimate_gas_optimistic_gas_limit_relative_diff .observe(diff); } + tracing::debug!( + optimized_lower_bound, + optimistic_gas_limit, + unscaled_gas_limit, + binary_search = ?kind, + iteration_count, + "Finished estimating gas limit for transaction" + ); let suggested_gas_limit = (unscaled_gas_limit as f64 * estimated_fee_scale_factor) as u64; estimator @@ -115,7 +123,7 @@ impl TxSender { bounds: ops::RangeInclusive, initial_pivot: Option, acceptable_overestimation: u64, - ) -> Result { + ) -> Result<(u64, usize), SubmitTxError> { let mut number_of_iterations = 0; let mut lower_bound = *bounds.start(); let mut upper_bound = *bounds.end(); @@ -157,7 +165,7 @@ impl TxSender { SANDBOX_METRICS .estimate_gas_binary_search_iterations .observe(number_of_iterations); - Ok(upper_bound) + Ok((upper_bound, number_of_iterations)) } async fn ensure_sufficient_balance(