From 726203bab540e3d6ada10b6bc12bd3c09220d895 Mon Sep 17 00:00:00 2001 From: Grzegorz Prusak Date: Wed, 13 Nov 2024 12:40:03 +0100 Subject: [PATCH 1/7] feat(consensus): fallback json rpc syncing for consensus (#3211) if for any reason p2p syncing is behind, json RPC syncing will activate. This is a protective measure for while we deploy changes to the consensus algorithm. Fixes BFT-516 --- Cargo.lock | 2 + core/node/consensus/Cargo.toml | 1 + core/node/consensus/src/en.rs | 94 ++++++++++++------------ core/node/consensus/src/lib.rs | 1 + core/node/consensus/src/metrics.rs | 13 ++++ core/node/consensus/src/storage/store.rs | 8 +- core/node/consensus/src/testonly.rs | 39 +--------- core/node/consensus/src/tests/mod.rs | 65 +--------------- core/node/node_sync/Cargo.toml | 1 + core/node/node_sync/src/sync_state.rs | 21 +++--- 10 files changed, 85 insertions(+), 160 deletions(-) create mode 100644 core/node/consensus/src/metrics.rs diff --git a/Cargo.lock b/Cargo.lock index eb93300b1729..75591bca7293 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11499,6 +11499,7 @@ dependencies = [ "thiserror", "tokio", "tracing", + "vise", "zksync_concurrency", "zksync_config", "zksync_consensus_bft", @@ -11699,6 +11700,7 @@ dependencies = [ "vise", "zksync_concurrency", "zksync_config", + "zksync_consensus_roles", "zksync_contracts", "zksync_dal", "zksync_eth_client", diff --git a/core/node/consensus/Cargo.toml b/core/node/consensus/Cargo.toml index 120d355da9a8..1d6b80f475e7 100644 --- a/core/node/consensus/Cargo.toml +++ b/core/node/consensus/Cargo.toml @@ -42,6 +42,7 @@ thiserror.workspace = true tracing.workspace = true tokio.workspace = true semver.workspace = true +vise.workspace = true [dev-dependencies] zksync_node_genesis.workspace = true diff --git a/core/node/consensus/src/en.rs b/core/node/consensus/src/en.rs index 6f4d80233be4..e417b68cf2cb 100644 --- a/core/node/consensus/src/en.rs +++ b/core/node/consensus/src/en.rs @@ -17,13 +17,14 @@ use zksync_web3_decl::{ use super::{config, storage::Store, ConsensusConfig, ConsensusSecrets}; use crate::{ + metrics::METRICS, registry, storage::{self, ConnectionPool}, }; -/// If less than TEMPORARY_FETCHER_THRESHOLD certificates are missing, -/// the temporary fetcher will stop fetching blocks. -pub(crate) const TEMPORARY_FETCHER_THRESHOLD: u64 = 10; +/// Whenever more than FALLBACK_FETCHER_THRESHOLD certificates are missing, +/// the fallback fetcher is active. +pub(crate) const FALLBACK_FETCHER_THRESHOLD: u64 = 10; /// External node. pub(super) struct EN { @@ -115,11 +116,9 @@ impl EN { let store = store.clone(); async { let store = store; - self.temporary_block_fetcher(ctx, &store).await?; - tracing::info!( - "temporary block fetcher finished, switching to p2p fetching only" - ); - Ok(()) + self.fallback_block_fetcher(ctx, &store) + .await + .wrap("fallback_block_fetcher()") } }); @@ -191,7 +190,7 @@ impl EN { .new_payload_queue(ctx, actions, self.sync_state.clone()) .await .wrap("new_fetcher_cursor()")?; - self.fetch_blocks(ctx, &mut payload_queue, None).await + self.fetch_blocks(ctx, &mut payload_queue).await }) .await; match res { @@ -362,9 +361,14 @@ impl EN { } /// Fetches (with retries) the given block from the main node. - async fn fetch_block(&self, ctx: &ctx::Ctx, n: L2BlockNumber) -> ctx::Result { + async fn fetch_block( + &self, + ctx: &ctx::Ctx, + n: validator::BlockNumber, + ) -> ctx::Result { const RETRY_INTERVAL: time::Duration = time::Duration::seconds(5); - + let n = L2BlockNumber(n.0.try_into().context("overflow")?); + METRICS.fetch_block.inc(); loop { match ctx.wait(self.client.sync_l2_block(n, true)).await? { Ok(Some(block)) => return Ok(block.try_into()?), @@ -376,9 +380,8 @@ impl EN { } } - /// Fetches blocks from the main node directly, until the certificates - /// are backfilled. This allows for smooth transition from json RPC to p2p block syncing. - pub(crate) async fn temporary_block_fetcher( + /// Fetches blocks from the main node directly whenever the EN is lagging behind too much. + pub(crate) async fn fallback_block_fetcher( &self, ctx: &ctx::Ctx, store: &Store, @@ -386,66 +389,63 @@ impl EN { const MAX_CONCURRENT_REQUESTS: usize = 30; scope::run!(ctx, |ctx, s| async { let (send, mut recv) = ctx::channel::bounded(MAX_CONCURRENT_REQUESTS); - s.spawn(async { - let Some(mut next) = store.next_block(ctx).await? else { - return Ok(()); - }; - while store.persisted().borrow().next().0 + TEMPORARY_FETCHER_THRESHOLD < next.0 { - let n = L2BlockNumber(next.0.try_into().context("overflow")?); - self.sync_state.wait_for_main_node_block(ctx, n).await?; - send.send(ctx, s.spawn(self.fetch_block(ctx, n))).await?; + // TODO: metrics. + s.spawn::<()>(async { + let send = send; + let is_lagging = + |main| main >= store.persisted().borrow().next() + FALLBACK_FETCHER_THRESHOLD; + let mut next = store.next_block(ctx).await.wrap("next_block()")?; + loop { + // Wait until p2p syncing is lagging. + self.sync_state + .wait_for_main_node_block(ctx, is_lagging) + .await?; + // Determine the next block to fetch and wait for it to be available. + next = next.max(store.next_block(ctx).await.wrap("next_block()")?); + self.sync_state + .wait_for_main_node_block(ctx, |main| main >= next) + .await?; + // Fetch the block asynchronously. + send.send(ctx, s.spawn(self.fetch_block(ctx, next))).await?; next = next.next(); } - drop(send); - Ok(()) }); - while let Ok(block) = recv.recv_or_disconnected(ctx).await? { + loop { + let block = recv.recv(ctx).await?; store .queue_next_fetched_block(ctx, block.join(ctx).await?) .await .wrap("queue_next_fetched_block()")?; } - Ok(()) }) .await } - /// Fetches blocks from the main node in range `[cursor.next()..end)`. + /// Fetches blocks starting with `queue.next()`. async fn fetch_blocks( &self, ctx: &ctx::Ctx, queue: &mut storage::PayloadQueue, - end: Option, ) -> ctx::Result<()> { const MAX_CONCURRENT_REQUESTS: usize = 30; - let first = queue.next(); - let mut next = first; + let mut next = queue.next(); scope::run!(ctx, |ctx, s| async { let (send, mut recv) = ctx::channel::bounded(MAX_CONCURRENT_REQUESTS); - s.spawn(async { + s.spawn::<()>(async { let send = send; - while end.map_or(true, |end| next < end) { - let n = L2BlockNumber(next.0.try_into().context("overflow")?); - self.sync_state.wait_for_main_node_block(ctx, n).await?; - send.send(ctx, s.spawn(self.fetch_block(ctx, n))).await?; + loop { + self.sync_state + .wait_for_main_node_block(ctx, |main| main >= next) + .await?; + send.send(ctx, s.spawn(self.fetch_block(ctx, next))).await?; next = next.next(); } - Ok(()) }); - while end.map_or(true, |end| queue.next() < end) { + loop { let block = recv.recv(ctx).await?.join(ctx).await?; queue.send(block).await.context("queue.send()")?; } - Ok(()) }) - .await?; - // If fetched anything, wait for the last block to be stored persistently. - if first < queue.next() { - self.pool - .wait_for_payload(ctx, queue.next().prev().unwrap()) - .await - .wrap("wait_for_payload()")?; - } - Ok(()) + .await } } diff --git a/core/node/consensus/src/lib.rs b/core/node/consensus/src/lib.rs index 8bf078120aa9..d89aa5f5e829 100644 --- a/core/node/consensus/src/lib.rs +++ b/core/node/consensus/src/lib.rs @@ -9,6 +9,7 @@ mod abi; mod config; mod en; pub mod era; +mod metrics; mod mn; mod registry; mod storage; diff --git a/core/node/consensus/src/metrics.rs b/core/node/consensus/src/metrics.rs new file mode 100644 index 000000000000..f53bb9320917 --- /dev/null +++ b/core/node/consensus/src/metrics.rs @@ -0,0 +1,13 @@ +//! Consensus related metrics. + +#[derive(Debug, vise::Metrics)] +#[metrics(prefix = "zksync_node_consensus")] +pub(crate) struct Metrics { + /// Number of blocks that has been fetched via JSON-RPC. + /// It is used only as a fallback when the p2p syncing is disabled or falling behind. + /// so it shouldn't be increasing under normal circumstances if p2p syncing is enabled. + pub fetch_block: vise::Counter, +} + +#[vise::register] +pub(super) static METRICS: vise::Global = vise::Global::new(); diff --git a/core/node/consensus/src/storage/store.rs b/core/node/consensus/src/storage/store.rs index 154509e97b14..c42e78658dc2 100644 --- a/core/node/consensus/src/storage/store.rs +++ b/core/node/consensus/src/storage/store.rs @@ -114,14 +114,12 @@ impl Store { } /// Number of the next block to queue. - pub(crate) async fn next_block( - &self, - ctx: &ctx::Ctx, - ) -> ctx::OrCanceled> { + pub(crate) async fn next_block(&self, ctx: &ctx::Ctx) -> ctx::Result { Ok(sync::lock(ctx, &self.block_payloads) .await? .as_ref() - .map(|p| p.next())) + .context("payload_queue not set")? + .next()) } /// Queues the next block. diff --git a/core/node/consensus/src/testonly.rs b/core/node/consensus/src/testonly.rs index ef4226c915f0..faa895c86c71 100644 --- a/core/node/consensus/src/testonly.rs +++ b/core/node/consensus/src/testonly.rs @@ -45,10 +45,7 @@ use zksync_types::{ }; use zksync_web3_decl::client::{Client, DynClient, L2}; -use crate::{ - en, - storage::{ConnectionPool, Store}, -}; +use crate::{en, storage::ConnectionPool}; /// Fake StateKeeper for tests. #[derive(Debug)] @@ -413,40 +410,6 @@ impl StateKeeper { .await } - pub async fn run_temporary_fetcher( - self, - ctx: &ctx::Ctx, - client: Box>, - ) -> ctx::Result<()> { - scope::run!(ctx, |ctx, s| async { - let payload_queue = self - .pool - .connection(ctx) - .await - .wrap("connection()")? - .new_payload_queue(ctx, self.actions_sender, self.sync_state.clone()) - .await - .wrap("new_payload_queue()")?; - let (store, runner) = Store::new( - ctx, - self.pool.clone(), - Some(payload_queue), - Some(client.clone()), - ) - .await - .wrap("Store::new()")?; - s.spawn_bg(async { Ok(runner.run(ctx).await?) }); - en::EN { - pool: self.pool.clone(), - client, - sync_state: self.sync_state.clone(), - } - .temporary_block_fetcher(ctx, &store) - .await - }) - .await - } - /// Runs consensus node for the external node. pub async fn run_consensus( self, diff --git a/core/node/consensus/src/tests/mod.rs b/core/node/consensus/src/tests/mod.rs index 663ccab49904..efb8d14314c8 100644 --- a/core/node/consensus/src/tests/mod.rs +++ b/core/node/consensus/src/tests/mod.rs @@ -16,7 +16,7 @@ use zksync_types::ProtocolVersionId; use zksync_web3_decl::namespaces::EnNamespaceClient as _; use crate::{ - en::TEMPORARY_FETCHER_THRESHOLD, + en::FALLBACK_FETCHER_THRESHOLD, mn::run_main_node, storage::{ConnectionPool, Store}, testonly, @@ -665,7 +665,7 @@ async fn test_p2p_fetcher_backfill_certs(from_snapshot: bool, version: ProtocolV // Test temporary fetcher fetching blocks if a lot of certs are missing. #[test_casing(4, Product((FROM_SNAPSHOT,VERSIONS)))] #[tokio::test] -async fn test_temporary_fetcher(from_snapshot: bool, version: ProtocolVersionId) { +async fn test_fallback_fetcher(from_snapshot: bool, version: ProtocolVersionId) { zksync_concurrency::testonly::abort_on_panic(); let ctx = &ctx::test_root(&ctx::AffineClock::new(10.)); let rng = &mut ctx.rng(); @@ -705,7 +705,7 @@ async fn test_temporary_fetcher(from_snapshot: bool, version: ProtocolVersionId) s.spawn_bg(runner.run(ctx)); s.spawn_bg(node.run_fetcher(ctx, client.clone())); validator - .push_random_blocks(rng, account, TEMPORARY_FETCHER_THRESHOLD as usize + 1) + .push_random_blocks(rng, account, FALLBACK_FETCHER_THRESHOLD as usize + 1) .await; node_pool .wait_for_payload(ctx, validator.last_block()) @@ -715,58 +715,7 @@ async fn test_temporary_fetcher(from_snapshot: bool, version: ProtocolVersionId) .await .unwrap(); - tracing::info!( - "Run p2p fetcher. Blocks should be fetched by the temporary fetcher anyway." - ); - scope::run!(ctx, |ctx, s| async { - let (node, runner) = testonly::StateKeeper::new(ctx, node_pool.clone()).await?; - s.spawn_bg(runner.run(ctx)); - s.spawn_bg(node.run_consensus(ctx, client.clone(), node_cfg.clone())); - validator.push_random_blocks(rng, account, 5).await; - node_pool - .wait_for_payload(ctx, validator.last_block()) - .await?; - Ok(()) - }) - .await - .unwrap(); - Ok(()) - }) - .await - .unwrap(); -} - -// Test that temporary fetcher terminates once enough blocks have certs. -#[test_casing(4, Product((FROM_SNAPSHOT,VERSIONS)))] -#[tokio::test] -async fn test_temporary_fetcher_termination(from_snapshot: bool, version: ProtocolVersionId) { - zksync_concurrency::testonly::abort_on_panic(); - let ctx = &ctx::test_root(&ctx::AffineClock::new(10.)); - let rng = &mut ctx.rng(); - let setup = Setup::new(rng, 1); - let validator_cfg = testonly::new_configs(rng, &setup, 0)[0].clone(); - let node_cfg = validator_cfg.new_fullnode(rng); - let account = &mut Account::random(); - - scope::run!(ctx, |ctx, s| async { - tracing::info!("Spawn validator."); - let validator_pool = ConnectionPool::test(from_snapshot, version).await; - let (mut validator, runner) = - testonly::StateKeeper::new(ctx, validator_pool.clone()).await?; - s.spawn_bg(runner.run(ctx)); - s.spawn_bg(run_main_node( - ctx, - validator_cfg.config.clone(), - validator_cfg.secrets.clone(), - validator_pool.clone(), - )); - // API server needs at least 1 L1 batch to start. - validator.seal_batch().await; - let client = validator.connect(ctx).await?; - - let node_pool = ConnectionPool::test(from_snapshot, version).await; - - // Run the EN so the consensus is initialized on EN and wait for it to sync. + tracing::info!("Run p2p fetcher. Blocks should be fetched by the fallback fetcher anyway."); scope::run!(ctx, |ctx, s| async { let (node, runner) = testonly::StateKeeper::new(ctx, node_pool.clone()).await?; s.spawn_bg(runner.run(ctx)); @@ -779,12 +728,6 @@ async fn test_temporary_fetcher_termination(from_snapshot: bool, version: Protoc }) .await .unwrap(); - - // Run the temporary fetcher. It should terminate immediately, since EN is synced. - let (node, runner) = testonly::StateKeeper::new(ctx, node_pool.clone()).await?; - s.spawn_bg(runner.run(ctx)); - node.run_temporary_fetcher(ctx, client).await?; - Ok(()) }) .await diff --git a/core/node/node_sync/Cargo.toml b/core/node/node_sync/Cargo.toml index 9c5b0c000700..e42cbff85806 100644 --- a/core/node/node_sync/Cargo.toml +++ b/core/node/node_sync/Cargo.toml @@ -24,6 +24,7 @@ zksync_health_check.workspace = true zksync_utils.workspace = true zksync_eth_client.workspace = true zksync_concurrency.workspace = true +zksync_consensus_roles.workspace = true vise.workspace = true zksync_vm_executor.workspace = true diff --git a/core/node/node_sync/src/sync_state.rs b/core/node/node_sync/src/sync_state.rs index f8a2fe00ec09..1ffec757c9b1 100644 --- a/core/node/node_sync/src/sync_state.rs +++ b/core/node/node_sync/src/sync_state.rs @@ -4,6 +4,7 @@ use async_trait::async_trait; use serde::Serialize; use tokio::sync::watch; use zksync_concurrency::{ctx, sync}; +use zksync_consensus_roles::validator; use zksync_dal::{ConnectionPool, Core, CoreDal}; use zksync_health_check::{CheckHealth, Health, HealthStatus}; use zksync_shared_metrics::EN_METRICS; @@ -50,18 +51,20 @@ impl SyncState { .unwrap(); } + /// Waits until the main node block is greater or equal to the given block number. + /// Returns the current main node block number. pub async fn wait_for_main_node_block( &self, ctx: &ctx::Ctx, - want: L2BlockNumber, - ) -> ctx::OrCanceled<()> { - sync::wait_for( - ctx, - &mut self.0.subscribe(), - |inner| matches!(inner.main_node_block, Some(got) if got >= want), - ) - .await?; - Ok(()) + pred: impl Fn(validator::BlockNumber) -> bool, + ) -> ctx::OrCanceled { + sync::wait_for_some(ctx, &mut self.0.subscribe(), |inner| { + inner + .main_node_block + .map(|n| validator::BlockNumber(n.0.into())) + .filter(|n| pred(*n)) + }) + .await } pub fn set_main_node_block(&self, block: L2BlockNumber) { From 54e4b007b2d32d86b2701b01cd3bef3b3bc97087 Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Wed, 13 Nov 2024 17:55:03 +0200 Subject: [PATCH 2/7] fix(vm): Do not require experimental VM config (#3270) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Uses default experimental VM config when initializing the main node. ## Why ❔ Currently, the experimental VM config is required in certain cases, which may lead to panics during node initialization. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Code has been formatted via `zkstack dev fmt` and `zkstack dev lint`. --- core/bin/zksync_server/src/node_builder.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/bin/zksync_server/src/node_builder.rs b/core/bin/zksync_server/src/node_builder.rs index 32478ede5bf8..794c847a24d5 100644 --- a/core/bin/zksync_server/src/node_builder.rs +++ b/core/bin/zksync_server/src/node_builder.rs @@ -325,7 +325,11 @@ impl MainNodeBuilder { latest_values_cache_size: rpc_config.latest_values_cache_size() as u64, latest_values_max_block_lag: rpc_config.latest_values_max_block_lag(), }; - let vm_config = try_load_config!(self.configs.experimental_vm_config); + let vm_config = self + .configs + .experimental_vm_config + .clone() + .unwrap_or_default(); // On main node we always use master pool sink. self.node.add_layer(MasterPoolSinkLayer); @@ -597,7 +601,11 @@ impl MainNodeBuilder { } fn add_vm_playground_layer(mut self) -> anyhow::Result { - let vm_config = try_load_config!(self.configs.experimental_vm_config); + let vm_config = self + .configs + .experimental_vm_config + .clone() + .unwrap_or_default(); self.node.add_layer(VmPlaygroundLayer::new( vm_config.playground, self.genesis_config.l2_chain_id, From 8620a8e9b97a31e4db13f39912759c3db3111a11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bruno=20Fran=C3=A7a?= Date: Thu, 14 Nov 2024 02:21:06 +0000 Subject: [PATCH 3/7] fix: Fixes build issues related to foundy-zksync (#3272) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ - Docker build script checked for wrong forge version (foundry-zksync reports a forge version 0.0.2) - Dev setup guide stated the wrong command to install foundry-zksync (current one wouldn't work) --- docker/Makefile | 2 +- docs/guides/setup-dev.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/Makefile b/docker/Makefile index 4e0ca51f904e..19d5fee0907f 100644 --- a/docker/Makefile +++ b/docker/Makefile @@ -7,7 +7,7 @@ NODE_VERSION_MIN=20.17.0 YARN_VERSION_MIN=1.22.19 RUST_VERSION=nightly-2024-08-01 SQLX_CLI_VERSION=0.8.1 -FORGE_MIN_VERSION=0.2.0 +FORGE_MIN_VERSION=0.0.2 # Versions and packages checks check-nodejs: diff --git a/docs/guides/setup-dev.md b/docs/guides/setup-dev.md index a74f976c2b73..43350ac3314d 100644 --- a/docs/guides/setup-dev.md +++ b/docs/guides/setup-dev.md @@ -47,7 +47,7 @@ cargo install sqlx-cli --version 0.8.1 # Foundry ZKsync curl -L https://raw.githubusercontent.com/matter-labs/foundry-zksync/main/install-foundry-zksync | bash -foundryup-zksync --branch master +foundryup-zksync # Non CUDA (GPU) setup, can be skipped if the machine has a CUDA installed for provers # Don't do that if you intend to run provers on your machine. Check the prover docs for a setup instead. From 1edcabe0c6a02d5b6700c29c0d9f6220ec6fb03c Mon Sep 17 00:00:00 2001 From: Alex Ostrovski Date: Thu, 14 Nov 2024 12:01:47 +0200 Subject: [PATCH 4/7] refactor(utils): Move bytecode hashing to `basic_types` (#3258) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Moves bytecode hashing logic to `zksync_basic_types`. ## Why ❔ It belongs there by domain and further simplifies the dependency graph. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zkstack dev fmt` and `zkstack dev lint`. --- Cargo.lock | 21 +- .../system-constants-generator/src/utils.rs | 25 +- core/lib/basic_types/Cargo.toml | 2 + .../{utils => basic_types}/src/bytecode.rs | 168 +++++--- core/lib/basic_types/src/lib.rs | 1 + core/lib/contract_verifier/src/lib.rs | 4 +- core/lib/contract_verifier/src/tests/mod.rs | 6 +- core/lib/contract_verifier/src/tests/real.rs | 2 +- core/lib/contracts/Cargo.toml | 2 +- core/lib/contracts/src/lib.rs | 14 +- core/lib/contracts/src/serde_bytecode.rs | 4 +- core/lib/contracts/src/test_contracts.rs | 5 +- core/lib/dal/Cargo.toml | 1 - core/lib/dal/src/contract_verification_dal.rs | 4 +- core/lib/multivm/Cargo.toml | 1 - .../lib/multivm/src/pubdata_builders/utils.rs | 8 +- core/lib/multivm/src/utils/bytecode.rs | 16 +- core/lib/multivm/src/versions/shadow/mod.rs | 7 +- .../src/versions/testonly/block_tip.rs | 17 +- .../src/versions/testonly/code_oracle.rs | 11 +- .../src/versions/testonly/evm_emulator.rs | 26 +- .../versions/testonly/get_used_contracts.rs | 10 +- core/lib/multivm/src/versions/testonly/mod.rs | 16 +- .../multivm/src/versions/testonly/upgrade.rs | 10 +- .../versions/vm_1_3_2/oracles/decommitter.rs | 3 +- .../src/versions/vm_1_3_2/pubdata_utils.rs | 6 +- .../src/versions/vm_1_3_2/test_utils.rs | 4 +- .../src/versions/vm_1_3_2/transaction_data.rs | 7 +- core/lib/multivm/src/versions/vm_1_3_2/vm.rs | 5 +- .../versions/vm_1_3_2/vm_with_bootloader.rs | 12 +- .../vm_1_4_1/implementation/bytecode.rs | 14 +- .../vm_1_4_1/old_vm/oracles/decommitter.rs | 3 +- .../src/versions/vm_1_4_1/tracers/refunds.rs | 4 +- .../types/internals/transaction_data.rs | 4 +- .../vm_1_4_2/implementation/bytecode.rs | 14 +- .../vm_1_4_2/old_vm/oracles/decommitter.rs | 3 +- .../src/versions/vm_1_4_2/tracers/refunds.rs | 4 +- .../types/internals/transaction_data.rs | 4 +- .../implementation/bytecode.rs | 14 +- .../old_vm/oracles/decommitter.rs | 3 +- .../vm_boojum_integration/tracers/refunds.rs | 4 +- .../types/internals/transaction_data.rs | 4 +- .../multivm/src/versions/vm_fast/bytecode.rs | 7 +- .../src/versions/vm_fast/evm_deploy_tracer.rs | 6 +- .../src/versions/vm_fast/transaction_data.rs | 4 +- core/lib/multivm/src/versions/vm_fast/vm.rs | 4 +- .../vm_latest/implementation/bytecode.rs | 14 +- .../src/versions/vm_latest/tests/mod.rs | 8 +- .../vm_latest/tracers/evm_deploy_tracer.rs | 7 +- .../types/internals/transaction_data.rs | 4 +- .../src/versions/vm_m5/oracles/decommitter.rs | 3 +- .../src/versions/vm_m5/pubdata_utils.rs | 6 +- .../multivm/src/versions/vm_m5/test_utils.rs | 4 +- .../src/versions/vm_m5/transaction_data.rs | 7 +- .../src/versions/vm_m5/vm_with_bootloader.rs | 8 +- .../src/versions/vm_m6/oracles/decommitter.rs | 3 +- .../src/versions/vm_m6/pubdata_utils.rs | 6 +- .../multivm/src/versions/vm_m6/test_utils.rs | 4 +- .../src/versions/vm_m6/transaction_data.rs | 7 +- core/lib/multivm/src/versions/vm_m6/vm.rs | 5 +- .../src/versions/vm_m6/vm_with_bootloader.rs | 12 +- .../implementation/bytecode.rs | 14 +- .../old_vm/oracles/decommitter.rs | 3 +- .../vm_refunds_enhancement/tracers/refunds.rs | 4 +- .../types/internals/transaction_data.rs | 4 +- .../implementation/bytecode.rs | 14 +- .../old_vm/oracles/decommitter.rs | 3 +- .../vm_virtual_blocks/tracers/refunds.rs | 4 +- .../types/internals/transaction_data.rs | 4 +- core/lib/snapshots_applier/Cargo.toml | 1 - core/lib/snapshots_applier/src/lib.rs | 10 +- core/lib/types/Cargo.toml | 1 - core/lib/types/src/abi.rs | 5 +- core/lib/types/src/api/state_override.rs | 10 +- core/lib/types/src/l1/mod.rs | 18 +- core/lib/types/src/lib.rs | 8 +- core/lib/types/src/transaction_request.rs | 41 +- core/lib/types/src/tx/execute.rs | 4 +- core/lib/utils/Cargo.toml | 4 - core/lib/utils/src/lib.rs | 1 - core/lib/vm_executor/Cargo.toml | 1 - core/lib/vm_executor/src/oneshot/tests.rs | 3 +- .../lib/vm_interface/src/storage/in_memory.rs | 36 +- core/node/api_server/Cargo.toml | 1 - .../src/execution_sandbox/vm_metrics.rs | 9 +- .../src/tx_sender/tests/gas_estimation.rs | 8 +- .../api_server/src/web3/namespaces/eth.rs | 4 +- core/node/api_server/src/web3/tests/mod.rs | 12 +- .../contract_verification_server/Cargo.toml | 1 - .../src/api_impl.rs | 2 +- .../contract_verification_server/src/tests.rs | 10 +- core/node/genesis/Cargo.toml | 1 - core/node/genesis/src/lib.rs | 9 +- core/node/genesis/src/utils.rs | 6 +- core/node/node_sync/Cargo.toml | 1 - core/node/node_sync/src/client.rs | 3 +- core/node/state_keeper/Cargo.toml | 2 - core/node/state_keeper/src/io/tests/mod.rs | 12 +- .../src/updates/l2_block_updates.rs | 9 +- core/node/vm_runner/Cargo.toml | 1 - core/node/vm_runner/src/tests/mod.rs | 4 +- .../src/sdk/operations/deploy_contract.rs | 8 +- core/tests/test_account/Cargo.toml | 1 - core/tests/test_account/src/lib.rs | 11 +- core/tests/vm-benchmark/Cargo.toml | 1 - core/tests/vm-benchmark/src/transaction.rs | 11 +- core/tests/vm-benchmark/src/vm.rs | 3 +- prover/Cargo.lock | 11 +- zkstack_cli/Cargo.lock | 365 +++--------------- 109 files changed, 545 insertions(+), 766 deletions(-) rename core/lib/{utils => basic_types}/src/bytecode.rs (50%) diff --git a/Cargo.lock b/Cargo.lock index 75591bca7293..d1528200f797 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9634,7 +9634,6 @@ dependencies = [ "zksync_contracts", "zksync_multivm", "zksync_types", - "zksync_utils", "zksync_vlog", "zksync_vm2", ] @@ -10451,6 +10450,7 @@ dependencies = [ "anyhow", "bincode", "chrono", + "const-decoder", "ethabi", "hex", "num_enum 0.7.3", @@ -10458,6 +10458,7 @@ dependencies = [ "serde", "serde_json", "serde_with", + "sha2 0.10.8", "strum", "thiserror", "tiny-keccak 2.0.2", @@ -10792,7 +10793,6 @@ dependencies = [ "zksync_dal", "zksync_node_test_utils", "zksync_types", - "zksync_utils", ] [[package]] @@ -10846,11 +10846,11 @@ version = "0.1.0" dependencies = [ "bincode", "envy", - "ethabi", "hex", "once_cell", "serde", "serde_json", + "zksync_basic_types", "zksync_utils", ] @@ -10998,7 +10998,6 @@ dependencies = [ "zksync_system_constants", "zksync_test_account", "zksync_types", - "zksync_utils", "zksync_vm_interface", ] @@ -11427,7 +11426,6 @@ dependencies = [ "zksync_system_constants", "zksync_test_account", "zksync_types", - "zksync_utils", "zksync_vm2", "zksync_vm_interface", ] @@ -11480,7 +11478,6 @@ dependencies = [ "zksync_state_keeper", "zksync_system_constants", "zksync_types", - "zksync_utils", "zksync_vm_executor", "zksync_web3_decl", ] @@ -11655,7 +11652,6 @@ dependencies = [ "zksync_multivm", "zksync_system_constants", "zksync_types", - "zksync_utils", ] [[package]] @@ -11711,7 +11707,6 @@ dependencies = [ "zksync_state_keeper", "zksync_system_constants", "zksync_types", - "zksync_utils", "zksync_vm_executor", "zksync_web3_decl", ] @@ -11958,7 +11953,6 @@ dependencies = [ "zksync_health_check", "zksync_object_store", "zksync_types", - "zksync_utils", "zksync_web3_decl", ] @@ -12039,7 +12033,6 @@ dependencies = [ "zksync_system_constants", "zksync_test_account", "zksync_types", - "zksync_utils", "zksync_vm_executor", ] @@ -12118,7 +12111,6 @@ dependencies = [ "zksync_eth_signer", "zksync_system_constants", "zksync_types", - "zksync_utils", ] [[package]] @@ -12153,7 +12145,6 @@ dependencies = [ "zksync_protobuf", "zksync_protobuf_build", "zksync_system_constants", - "zksync_utils", ] [[package]] @@ -12162,16 +12153,12 @@ version = "0.1.0" dependencies = [ "anyhow", "assert_matches", - "const-decoder", "futures 0.3.31", "once_cell", "reqwest 0.12.9", "serde_json", - "thiserror", "tokio", "tracing", - "zk_evm 0.133.0", - "zksync_basic_types", "zksync_vlog", ] @@ -12236,7 +12223,6 @@ dependencies = [ "zksync_dal", "zksync_multivm", "zksync_types", - "zksync_utils", ] [[package]] @@ -12287,7 +12273,6 @@ dependencies = [ "zksync_storage", "zksync_test_account", "zksync_types", - "zksync_utils", "zksync_vm_executor", "zksync_vm_interface", ] diff --git a/core/bin/system-constants-generator/src/utils.rs b/core/bin/system-constants-generator/src/utils.rs index 2c08de7970b8..e3558de3e6a1 100644 --- a/core/bin/system-constants-generator/src/utils.rs +++ b/core/bin/system-constants-generator/src/utils.rs @@ -21,14 +21,13 @@ use zksync_multivm::{ zk_evm_latest::aux_structures::Timestamp, }; use zksync_types::{ - block::L2BlockHasher, ethabi::Token, fee::Fee, fee_model::BatchFeeInput, l1::L1Tx, l2::L2Tx, - u256_to_h256, utils::storage_key_for_eth_balance, AccountTreeId, Address, Execute, - K256PrivateKey, L1BatchNumber, L1TxCommonData, L2BlockNumber, L2ChainId, Nonce, - ProtocolVersionId, StorageKey, Transaction, BOOTLOADER_ADDRESS, SYSTEM_CONTEXT_ADDRESS, - SYSTEM_CONTEXT_GAS_PRICE_POSITION, SYSTEM_CONTEXT_TX_ORIGIN_POSITION, U256, - ZKPORTER_IS_AVAILABLE, + block::L2BlockHasher, bytecode::BytecodeHash, ethabi::Token, fee::Fee, + fee_model::BatchFeeInput, l1::L1Tx, l2::L2Tx, u256_to_h256, utils::storage_key_for_eth_balance, + AccountTreeId, Address, Execute, K256PrivateKey, L1BatchNumber, L1TxCommonData, L2BlockNumber, + L2ChainId, Nonce, ProtocolVersionId, StorageKey, Transaction, BOOTLOADER_ADDRESS, + SYSTEM_CONTEXT_ADDRESS, SYSTEM_CONTEXT_GAS_PRICE_POSITION, SYSTEM_CONTEXT_TX_ORIGIN_POSITION, + U256, ZKPORTER_IS_AVAILABLE, }; -use zksync_utils::bytecode::hash_bytecode; use crate::intrinsic_costs::VmSpentResourcesResult; @@ -63,7 +62,7 @@ impl VmTracer for SpecialBootloaderTracer pub static GAS_TEST_SYSTEM_CONTRACTS: Lazy = Lazy::new(|| { let bytecode = read_bootloader_code("gas_test"); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); let bootloader = SystemContractCode { code: bytecode, @@ -71,7 +70,7 @@ pub static GAS_TEST_SYSTEM_CONTRACTS: Lazy = Lazy::new(|| { }; let bytecode = read_sys_contract_bytecode("", "DefaultAccount", ContractLanguage::Sol); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); BaseSystemContracts { default_aa: SystemContractCode { @@ -208,12 +207,12 @@ fn default_l1_batch() -> L1BatchEnv { /// returns the amount of gas needed to perform and internal transfer, assuming no gas price /// per pubdata, i.e. under assumption that the refund will not touch any new slots. pub(super) fn execute_internal_transfer_test() -> u32 { - let raw_storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let raw_storage = InMemoryStorage::with_system_contracts(); let mut storage_view = StorageView::new(raw_storage); let bootloader_balance_key = storage_key_for_eth_balance(&BOOTLOADER_ADDRESS); storage_view.set_value(bootloader_balance_key, u256_to_h256(U256([0, 0, 1, 0]))); let bytecode = read_bootloader_test_code("transfer_test"); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); let bootloader = SystemContractCode { code: bytecode, hash, @@ -222,7 +221,7 @@ pub(super) fn execute_internal_transfer_test() -> u32 { let l1_batch = default_l1_batch(); let bytecode = read_sys_contract_bytecode("", "DefaultAccount", ContractLanguage::Sol); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); let default_aa = SystemContractCode { code: bytecode, hash, @@ -293,7 +292,7 @@ pub(super) fn execute_user_txs_in_test_gas_vm( .iter() .fold(U256::zero(), |sum, elem| sum + elem.gas_limit()); - let raw_storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let raw_storage = InMemoryStorage::with_system_contracts(); let mut storage_view = StorageView::new(raw_storage); for tx in txs.iter() { diff --git a/core/lib/basic_types/Cargo.toml b/core/lib/basic_types/Cargo.toml index 616b959b0783..6cac4f60f615 100644 --- a/core/lib/basic_types/Cargo.toml +++ b/core/lib/basic_types/Cargo.toml @@ -11,8 +11,10 @@ keywords.workspace = true categories.workspace = true [dependencies] +const-decoder.workspace = true ethabi.workspace = true hex.workspace = true +sha2.workspace = true tiny-keccak.workspace = true thiserror.workspace = true serde = { workspace = true, features = ["derive"] } diff --git a/core/lib/utils/src/bytecode.rs b/core/lib/basic_types/src/bytecode.rs similarity index 50% rename from core/lib/utils/src/bytecode.rs rename to core/lib/basic_types/src/bytecode.rs index fcba022f9277..585ba0ef8c88 100644 --- a/core/lib/utils/src/bytecode.rs +++ b/core/lib/basic_types/src/bytecode.rs @@ -1,22 +1,42 @@ -// FIXME (PLA-1064): move to basic_types +//! Bytecode-related types and utils. +//! +//! # Bytecode kinds +//! +//! ZKsync supports 2 kinds of bytecodes: EraVM and EVM ones. +//! +//! - **EraVM** bytecodes consist of 64-bit (8-byte) instructions for the corresponding VM. +//! - **EVM** bytecodes consist of ordinary EVM opcodes, preceded with a 32-byte big-endian code length (in bytes). +//! +//! Both bytecode kinds are right-padded to consist of an integer, odd number of 32-byte words. All methods +//! in this module operate on padded bytecodes unless explicitly specified otherwise. use anyhow::Context as _; -use zk_evm::k256::sha2::{Digest, Sha256}; -use zksync_basic_types::{H256, U256}; +use sha2::{Digest, Sha256}; + +use crate::{H256, U256}; const MAX_BYTECODE_LENGTH_IN_WORDS: usize = (1 << 16) - 1; const MAX_BYTECODE_LENGTH_BYTES: usize = MAX_BYTECODE_LENGTH_IN_WORDS * 32; -#[derive(Debug, thiserror::Error, PartialEq)] +/// Errors returned from [`validate_bytecode()`]. +#[derive(Debug, thiserror::Error)] +#[non_exhaustive] pub enum InvalidBytecodeError { + /// Bytecode is too long. #[error("Bytecode too long: {0} bytes, while max {1} allowed")] BytecodeTooLong(usize, usize), - #[error("Bytecode has even number of 32-byte words")] - BytecodeLengthInWordsIsEven, + /// Bytecode length isn't divisible by 32 (i.e., bytecode cannot be represented as a sequence of 32-byte EraVM words). #[error("Bytecode length is not divisible by 32")] BytecodeLengthIsNotDivisibleBy32, + /// Bytecode has an even number of 32-byte words. + #[error("Bytecode has even number of 32-byte words")] + BytecodeLengthInWordsIsEven, } +/// Validates that the given bytecode passes basic checks (e.g., not too long). +/// +/// The performed checks are universal both for EraVM and (padded) EVM bytecodes. If you need to additionally check EVM bytecode integrity, +/// use [`trim_padded_evm_bytecode()`]. pub fn validate_bytecode(code: &[u8]) -> Result<(), InvalidBytecodeError> { let bytecode_len = code.len(); @@ -40,37 +60,79 @@ pub fn validate_bytecode(code: &[u8]) -> Result<(), InvalidBytecodeError> { Ok(()) } -fn bytes_to_chunks(bytes: &[u8]) -> Vec<[u8; 32]> { - assert_eq!( - bytes.len() % 32, - 0, - "Bytes must be divisible by 32 to split into chunks" - ); - bytes - .chunks(32) - .map(|el| { - let mut chunk = [0u8; 32]; - chunk.copy_from_slice(el); - chunk - }) - .collect() -} +/// 32-byte bytecode hash. Besides a cryptographically secure hash of the bytecode contents, contains a [`BytecodeMarker`] +/// and the bytecode length. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub struct BytecodeHash(H256); -/// Hashes the provided EraVM bytecode. -pub fn hash_bytecode(code: &[u8]) -> H256 { - let chunked_code = bytes_to_chunks(code); - let hash = zk_evm::zkevm_opcode_defs::utils::bytecode_to_code_hash(&chunked_code) - .expect("Invalid bytecode"); +impl BytecodeHash { + /// Hashes the provided EraVM bytecode. + pub fn for_bytecode(bytecode: &[u8]) -> Self { + Self::for_generic_bytecode(BytecodeMarker::EraVm, bytecode) + } - H256(hash) -} + /// Hashes the provided padded EVM bytecode. + pub fn for_evm_bytecode(bytecode: &[u8]) -> Self { + Self::for_generic_bytecode(BytecodeMarker::Evm, bytecode) + } -pub fn bytecode_len_in_words(bytecodehash: &H256) -> u16 { - u16::from_be_bytes([bytecodehash[2], bytecodehash[3]]) + fn for_generic_bytecode(kind: BytecodeMarker, bytecode: &[u8]) -> Self { + validate_bytecode(bytecode).expect("invalid bytecode"); + + let mut hasher = Sha256::new(); + let len = match kind { + BytecodeMarker::EraVm => (bytecode.len() / 32) as u16, + BytecodeMarker::Evm => bytecode.len() as u16, + }; + hasher.update(bytecode); + let result = hasher.finalize(); + + let mut output = [0u8; 32]; + output[..].copy_from_slice(result.as_slice()); + output[0] = kind as u8; + output[1] = 0; + output[2..4].copy_from_slice(&len.to_be_bytes()); + + Self(H256(output)) + } + + /// Returns a marker / kind of this bytecode. + pub fn marker(&self) -> BytecodeMarker { + match self.0.as_bytes()[0] { + val if val == BytecodeMarker::EraVm as u8 => BytecodeMarker::EraVm, + val if val == BytecodeMarker::Evm as u8 => BytecodeMarker::Evm, + _ => unreachable!(), + } + } + + /// Returns the length of the hashed bytecode in bytes. + pub fn len_in_bytes(&self) -> usize { + let bytes = self.0.as_bytes(); + let raw_len = u16::from_be_bytes([bytes[2], bytes[3]]); + match self.marker() { + BytecodeMarker::EraVm => raw_len as usize * 32, + BytecodeMarker::Evm => raw_len as usize, + } + } + + /// Returns the underlying hash value. + pub fn value(self) -> H256 { + self.0 + } + + /// Returns the underlying hash value interpreted as a big-endian unsigned integer. + pub fn value_u256(self) -> U256 { + crate::h256_to_u256(self.0) + } } -pub fn bytecode_len_in_bytes(bytecodehash: H256) -> usize { - bytecode_len_in_words(&bytecodehash) as usize * 32 +impl TryFrom for BytecodeHash { + type Error = anyhow::Error; + + fn try_from(raw_hash: H256) -> Result { + BytecodeMarker::new(raw_hash).context("unknown bytecode hash marker")?; + Ok(Self(raw_hash)) + } } /// Bytecode marker encoded in the first byte of the bytecode hash. @@ -94,26 +156,10 @@ impl BytecodeMarker { } } -/// Hashes the provided EVM bytecode. The bytecode must be padded to an odd number of 32-byte words; -/// bytecodes stored in the known codes storage satisfy this requirement automatically. -pub fn hash_evm_bytecode(bytecode: &[u8]) -> H256 { - validate_bytecode(bytecode).expect("invalid EVM bytecode"); - - let mut hasher = Sha256::new(); - let len = bytecode.len() as u16; - hasher.update(bytecode); - let result = hasher.finalize(); - - let mut output = [0u8; 32]; - output[..].copy_from_slice(result.as_slice()); - output[0] = BytecodeMarker::Evm as u8; - output[1] = 0; - output[2..4].copy_from_slice(&len.to_be_bytes()); +/// Removes padding from an EVM bytecode, returning the original EVM bytecode. +pub fn trim_padded_evm_bytecode(raw: &[u8]) -> anyhow::Result<&[u8]> { + validate_bytecode(raw).context("bytecode fails basic validity checks")?; - H256(output) -} - -pub fn prepare_evm_bytecode(raw: &[u8]) -> anyhow::Result<&[u8]> { // EVM bytecodes are prefixed with a big-endian `U256` bytecode length. let bytecode_len_bytes = raw.get(..32).context("length < 32")?; let bytecode_len = U256::from_big_endian(bytecode_len_bytes); @@ -135,6 +181,7 @@ pub fn prepare_evm_bytecode(raw: &[u8]) -> anyhow::Result<&[u8]> { Ok(bytecode) } +#[doc(hidden)] // only useful for tests pub mod testonly { use const_decoder::Decoder; @@ -172,21 +219,18 @@ mod tests { #[test] fn bytecode_markers_are_valid() { - let bytecode_hash = hash_bytecode(&[0; 32]); - assert_eq!( - BytecodeMarker::new(bytecode_hash), - Some(BytecodeMarker::EraVm) - ); - let bytecode_hash = hash_evm_bytecode(&[0; 32]); - assert_eq!( - BytecodeMarker::new(bytecode_hash), - Some(BytecodeMarker::Evm) - ); + let bytecode_hash = BytecodeHash::for_bytecode(&[0; 32]); + assert_eq!(bytecode_hash.marker(), BytecodeMarker::EraVm); + assert_eq!(bytecode_hash.len_in_bytes(), 32); + + let bytecode_hash = BytecodeHash::for_evm_bytecode(&[0; 32]); + assert_eq!(bytecode_hash.marker(), BytecodeMarker::Evm); + assert_eq!(bytecode_hash.len_in_bytes(), 32); } #[test] fn preparing_evm_bytecode() { - let prepared = prepare_evm_bytecode(RAW_EVM_BYTECODE).unwrap(); + let prepared = trim_padded_evm_bytecode(RAW_EVM_BYTECODE).unwrap(); assert_eq!(prepared, PROCESSED_EVM_BYTECODE); } } diff --git a/core/lib/basic_types/src/lib.rs b/core/lib/basic_types/src/lib.rs index 5776416265d2..d79bc57cc5e1 100644 --- a/core/lib/basic_types/src/lib.rs +++ b/core/lib/basic_types/src/lib.rs @@ -29,6 +29,7 @@ pub use self::conversions::{ #[macro_use] mod macros; pub mod basic_fri_types; +pub mod bytecode; pub mod commitment; mod conversions; pub mod network; diff --git a/core/lib/contract_verifier/src/lib.rs b/core/lib/contract_verifier/src/lib.rs index e8bf05c72e81..7dc5d47d4562 100644 --- a/core/lib/contract_verifier/src/lib.rs +++ b/core/lib/contract_verifier/src/lib.rs @@ -13,13 +13,13 @@ use tokio::time; use zksync_dal::{contract_verification_dal::DeployedContractData, ConnectionPool, Core, CoreDal}; use zksync_queued_job_processor::{async_trait, JobProcessor}; use zksync_types::{ + bytecode::{trim_padded_evm_bytecode, BytecodeMarker}, contract_verification_api::{ self as api, CompilationArtifacts, VerificationIncomingRequest, VerificationInfo, VerificationRequest, }, Address, CONTRACT_DEPLOYER_ADDRESS, }; -use zksync_utils::bytecode::{prepare_evm_bytecode, BytecodeMarker}; use crate::{ compilers::{Solc, VyperInput, ZkSolc}, @@ -230,7 +230,7 @@ impl ContractVerifier { let deployed_bytecode = match bytecode_marker { BytecodeMarker::EraVm => deployed_contract.bytecode.as_slice(), - BytecodeMarker::Evm => prepare_evm_bytecode(&deployed_contract.bytecode) + BytecodeMarker::Evm => trim_padded_evm_bytecode(&deployed_contract.bytecode) .context("invalid stored EVM bytecode")?, }; diff --git a/core/lib/contract_verifier/src/tests/mod.rs b/core/lib/contract_verifier/src/tests/mod.rs index 2aad39a12e0e..395d467542dc 100644 --- a/core/lib/contract_verifier/src/tests/mod.rs +++ b/core/lib/contract_verifier/src/tests/mod.rs @@ -8,6 +8,7 @@ use zksync_dal::Connection; use zksync_node_test_utils::{create_l1_batch, create_l2_block}; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, contract_verification_api::{CompilerVersions, SourceCodeData, VerificationIncomingRequest}, get_code_key, get_known_code_key, l2::L2Tx, @@ -15,7 +16,6 @@ use zksync_types::{ Execute, L1BatchNumber, L2BlockNumber, ProtocolVersion, StorageLog, CONTRACT_DEPLOYER_ADDRESS, H256, U256, }; -use zksync_utils::bytecode::{hash_bytecode, hash_evm_bytecode}; use zksync_vm_interface::{tracer::ValidationTraces, TransactionExecutionMetrics, VmEvent}; use super::*; @@ -139,7 +139,7 @@ async fn mock_deployment( bytecode: Vec, constructor_args: &[Token], ) { - let bytecode_hash = hash_bytecode(&bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value(); let deployment = Execute::for_deploy(H256::zero(), bytecode.clone(), constructor_args); mock_deployment_inner(storage, address, bytecode_hash, bytecode, deployment).await; } @@ -160,7 +160,7 @@ async fn mock_evm_deployment( factory_deps: vec![], }; let bytecode = pad_evm_bytecode(deployed_bytecode); - let bytecode_hash = hash_evm_bytecode(&bytecode); + let bytecode_hash = BytecodeHash::for_evm_bytecode(&bytecode).value(); mock_deployment_inner(storage, address, bytecode_hash, bytecode, deployment).await; } diff --git a/core/lib/contract_verifier/src/tests/real.rs b/core/lib/contract_verifier/src/tests/real.rs index 4282e6de4ef8..4dbcf8860272 100644 --- a/core/lib/contract_verifier/src/tests/real.rs +++ b/core/lib/contract_verifier/src/tests/real.rs @@ -5,7 +5,7 @@ use std::{env, sync::Arc, time::Duration}; use assert_matches::assert_matches; -use zksync_utils::bytecode::validate_bytecode; +use zksync_types::bytecode::validate_bytecode; use super::*; diff --git a/core/lib/contracts/Cargo.toml b/core/lib/contracts/Cargo.toml index efe37b301e28..0a24012f1ba6 100644 --- a/core/lib/contracts/Cargo.toml +++ b/core/lib/contracts/Cargo.toml @@ -11,9 +11,9 @@ keywords.workspace = true categories.workspace = true [dependencies] +zksync_basic_types.workspace = true zksync_utils.workspace = true -ethabi.workspace = true serde_json.workspace = true serde.workspace = true once_cell.workspace = true diff --git a/core/lib/contracts/src/lib.rs b/core/lib/contracts/src/lib.rs index ad9f7739ba0d..63fdcd0c8545 100644 --- a/core/lib/contracts/src/lib.rs +++ b/core/lib/contracts/src/lib.rs @@ -10,10 +10,14 @@ use std::{ path::{Path, PathBuf}, }; -use ethabi::{ethereum_types::H256, Contract, Event, Function}; use once_cell::sync::Lazy; use serde::{Deserialize, Serialize}; -use zksync_utils::{bytecode::hash_bytecode, env::Workspace}; +use zksync_basic_types::{ + bytecode::BytecodeHash, + ethabi::{Contract, Event, Function}, + H256, +}; +use zksync_utils::env::Workspace; mod serde_bytecode; pub mod test_contracts; @@ -408,14 +412,14 @@ impl PartialEq for BaseSystemContracts { impl BaseSystemContracts { fn load_with_bootloader(bootloader_bytecode: Vec) -> Self { - let hash = hash_bytecode(&bootloader_bytecode); + let hash = BytecodeHash::for_bytecode(&bootloader_bytecode).value(); let bootloader = SystemContractCode { code: bootloader_bytecode, hash, }; let bytecode = read_sys_contract_bytecode("", "DefaultAccount", ContractLanguage::Sol); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); let default_aa = SystemContractCode { code: bytecode, hash, @@ -437,7 +441,7 @@ impl BaseSystemContracts { /// Loads the latest EVM emulator for these base system contracts. Logically, it only makes sense to do for the latest protocol version. pub fn with_latest_evm_emulator(mut self) -> Self { let bytecode = read_sys_contract_bytecode("", "EvmEmulator", ContractLanguage::Yul); - let hash = hash_bytecode(&bytecode); + let hash = BytecodeHash::for_bytecode(&bytecode).value(); self.evm_emulator = Some(SystemContractCode { code: bytecode, hash, diff --git a/core/lib/contracts/src/serde_bytecode.rs b/core/lib/contracts/src/serde_bytecode.rs index 43de12e8ddd1..8f250fe4672a 100644 --- a/core/lib/contracts/src/serde_bytecode.rs +++ b/core/lib/contracts/src/serde_bytecode.rs @@ -1,7 +1,7 @@ use std::fmt; -use ethabi::ethereum_types::U256; use serde::{de, de::SeqAccess, ser, ser::SerializeSeq, Deserializer, Serializer}; +use zksync_basic_types::U256; pub(super) fn serialize(bytes: &[u8], serializer: S) -> Result { if bytes.len() % 32 != 0 { @@ -43,8 +43,8 @@ pub(super) fn deserialize<'de, D: Deserializer<'de>>(deserializer: D) -> Result< #[cfg(test)] mod tests { - use ethabi::ethereum_types::{H256, U256}; use serde::{Deserialize, Serialize}; + use zksync_basic_types::{H256, U256}; use crate::SystemContractCode; diff --git a/core/lib/contracts/src/test_contracts.rs b/core/lib/contracts/src/test_contracts.rs index a670d930f049..c694743fc9e1 100644 --- a/core/lib/contracts/src/test_contracts.rs +++ b/core/lib/contracts/src/test_contracts.rs @@ -1,5 +1,8 @@ -use ethabi::{ethereum_types::U256, Bytes, Token}; use serde::Deserialize; +use zksync_basic_types::{ + ethabi::{Bytes, Token}, + U256, +}; use crate::get_loadnext_contract; diff --git a/core/lib/dal/Cargo.toml b/core/lib/dal/Cargo.toml index db03b8de9825..807a41ae0876 100644 --- a/core/lib/dal/Cargo.toml +++ b/core/lib/dal/Cargo.toml @@ -15,7 +15,6 @@ links = "zksync_dal_proto" [dependencies] vise.workspace = true zksync_vm_interface.workspace = true -zksync_utils.workspace = true zksync_system_constants.workspace = true zksync_contracts.workspace = true zksync_types.workspace = true diff --git a/core/lib/dal/src/contract_verification_dal.rs b/core/lib/dal/src/contract_verification_dal.rs index 0708063dff60..57bea5392cf8 100644 --- a/core/lib/dal/src/contract_verification_dal.rs +++ b/core/lib/dal/src/contract_verification_dal.rs @@ -567,11 +567,11 @@ mod tests { use std::collections::HashMap; use zksync_types::{ + bytecode::BytecodeHash, contract_verification_api::{CompilerVersions, SourceCodeData}, tx::IncludedTxLocation, Execute, L1BatchNumber, L2BlockNumber, ProtocolVersion, }; - use zksync_utils::bytecode::hash_bytecode; use zksync_vm_interface::{tracer::ValidationTraces, TransactionExecutionMetrics}; use super::*; @@ -598,7 +598,7 @@ mod tests { let deployed_address = Address::repeat_byte(12); let mut tx = mock_l2_transaction(); let bytecode = vec![1; 32]; - let bytecode_hash = hash_bytecode(&bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value(); tx.execute = Execute::for_deploy(H256::zero(), bytecode.clone(), &[]); conn.transactions_dal() .insert_transaction_l2( diff --git a/core/lib/multivm/Cargo.toml b/core/lib/multivm/Cargo.toml index 27130bc2720d..f2046b77d9cc 100644 --- a/core/lib/multivm/Cargo.toml +++ b/core/lib/multivm/Cargo.toml @@ -26,7 +26,6 @@ circuit_sequencer_api_1_5_0.workspace = true zksync_types.workspace = true zksync_contracts.workspace = true -zksync_utils.workspace = true zksync_system_constants.workspace = true zksync_vm_interface.workspace = true zksync_mini_merkle_tree.workspace = true diff --git a/core/lib/multivm/src/pubdata_builders/utils.rs b/core/lib/multivm/src/pubdata_builders/utils.rs index 57361a674fb7..83c9b9317640 100644 --- a/core/lib/multivm/src/pubdata_builders/utils.rs +++ b/core/lib/multivm/src/pubdata_builders/utils.rs @@ -1,6 +1,5 @@ use zksync_mini_merkle_tree::MiniMerkleTree; -use zksync_types::web3::keccak256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, web3::keccak256}; use crate::interface::pubdata::L1MessengerL2ToL1Log; @@ -49,8 +48,9 @@ pub(crate) fn build_chained_bytecode_hash(published_bytecodes: &[Vec]) -> Ve let mut chained_bytecode_hash = vec![0u8; 32]; for bytecode in published_bytecodes { - let hash = hash_bytecode(bytecode).to_fixed_bytes(); - + let hash = BytecodeHash::for_bytecode(bytecode) + .value() + .to_fixed_bytes(); chained_bytecode_hash = keccak256(&[chained_bytecode_hash, hash.to_vec()].concat()).to_vec(); } diff --git a/core/lib/multivm/src/utils/bytecode.rs b/core/lib/multivm/src/utils/bytecode.rs index c1937e992990..f5dee805864e 100644 --- a/core/lib/multivm/src/utils/bytecode.rs +++ b/core/lib/multivm/src/utils/bytecode.rs @@ -1,10 +1,10 @@ use std::collections::HashMap; use zksync_types::{ + bytecode::{validate_bytecode, BytecodeHash, InvalidBytecodeError}, ethabi::{self, Token}, Address, H256, U256, }; -use zksync_utils::bytecode::{hash_bytecode, validate_bytecode, InvalidBytecodeError}; use crate::interface::CompressedBytecodeInfo; @@ -46,6 +46,15 @@ pub(crate) fn be_bytes_to_safe_address(bytes: &[u8]) -> Option
{ } } +pub(crate) fn bytecode_len_in_words(bytecode_hash: &H256) -> u16 { + let bytes = bytecode_hash.as_bytes(); + u16::from_be_bytes([bytes[2], bytes[3]]) +} + +pub(crate) fn bytecode_len_in_bytes(bytecode_hash: &H256) -> u32 { + u32::from(bytecode_len_in_words(bytecode_hash)) * 32 +} + #[derive(Debug, thiserror::Error)] pub(crate) enum FailedToCompressBytecodeError { #[error("Number of unique 8-bytes bytecode chunks exceed the limit of 2^16 - 1")] @@ -128,7 +137,10 @@ pub(crate) fn compress( } pub(crate) fn encode_call(bytecode: &CompressedBytecodeInfo) -> Vec { - let mut bytecode_hash = hash_bytecode(&bytecode.original).as_bytes().to_vec(); + let mut bytecode_hash = BytecodeHash::for_bytecode(&bytecode.original) + .value() + .as_bytes() + .to_vec(); let empty_cell = [0_u8; 32]; bytecode_hash.extend_from_slice(&empty_cell); diff --git a/core/lib/multivm/src/versions/shadow/mod.rs b/core/lib/multivm/src/versions/shadow/mod.rs index 42a0fbb1b8ba..01eeb3669f43 100644 --- a/core/lib/multivm/src/versions/shadow/mod.rs +++ b/core/lib/multivm/src/versions/shadow/mod.rs @@ -12,7 +12,6 @@ use zksync_types::{ block::L2BlockHasher, fee::Fee, AccountTreeId, Address, Execute, L1BatchNumber, L2BlockNumber, ProtocolVersionId, StorageKey, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ interface::{ @@ -207,7 +206,7 @@ where { let system_env = default_system_env(); let l1_batch_env = default_l1_batch(L1BatchNumber(1)); - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); let mut harness = Harness::new(&l1_batch_env); harness.setup_storage(&mut storage); @@ -231,7 +230,7 @@ fn sanity_check_harness_on_new_vm() { fn sanity_check_shadow_vm() { let system_env = default_system_env(); let l1_batch_env = default_l1_batch(L1BatchNumber(1)); - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); let mut harness = Harness::new(&l1_batch_env); harness.setup_storage(&mut storage); @@ -258,7 +257,7 @@ fn shadow_vm_basics() { pretty_assertions::assert_eq!(replayed_dump, dump); // Check that the VM executes identically when reading from the original storage and one restored from the dump. - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); harness.setup_storage(&mut storage); let storage = StorageView::new(storage).to_rc_ptr(); diff --git a/core/lib/multivm/src/versions/testonly/block_tip.rs b/core/lib/multivm/src/versions/testonly/block_tip.rs index 61678e01a443..f4655610afcb 100644 --- a/core/lib/multivm/src/versions/testonly/block_tip.rs +++ b/core/lib/multivm/src/versions/testonly/block_tip.rs @@ -5,10 +5,10 @@ use zksync_system_constants::{ CONTRACT_FORCE_DEPLOYER_ADDRESS, KNOWN_CODES_STORAGE_ADDRESS, L1_MESSENGER_ADDRESS, }; use zksync_types::{ - commitment::SerializeCommitment, fee_model::BatchFeeInput, get_code_key, - l2_to_l1_log::L2ToL1Log, u256_to_h256, writes::StateDiffRecord, Address, Execute, H256, U256, + bytecode::BytecodeHash, commitment::SerializeCommitment, fee_model::BatchFeeInput, + get_code_key, l2_to_l1_log::L2ToL1Log, u256_to_h256, writes::StateDiffRecord, Address, Execute, + H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::{ default_pubdata_builder, get_complex_upgrade_abi, get_empty_storage, read_complex_upgrade, @@ -72,7 +72,9 @@ fn populate_mimic_calls(data: L1MessengerTestData) -> Vec> { data: l1_messenger .function("requestBytecodeL1Publication") .unwrap() - .encode_input(&[Token::FixedBytes(hash_bytecode(bytecode).0.to_vec())]) + .encode_input(&[Token::FixedBytes( + BytecodeHash::for_bytecode(bytecode).value().0.to_vec(), + )]) .unwrap(), }); @@ -118,9 +120,12 @@ fn execute_test(test_data: L1MessengerTestData) -> TestStatistics // For this test we'll just put the bytecode onto the force deployer address storage.set_value( get_code_key(&CONTRACT_FORCE_DEPLOYER_ADDRESS), - hash_bytecode(&complex_upgrade_code), + BytecodeHash::for_bytecode(&complex_upgrade_code).value(), + ); + storage.store_factory_dep( + BytecodeHash::for_bytecode(&complex_upgrade_code).value(), + complex_upgrade_code, ); - storage.store_factory_dep(hash_bytecode(&complex_upgrade_code), complex_upgrade_code); // We are measuring computational cost, so prices for pubdata don't matter, while they artificially dilute // the gas limit diff --git a/core/lib/multivm/src/versions/testonly/code_oracle.rs b/core/lib/multivm/src/versions/testonly/code_oracle.rs index 6a1fcfb28df3..ace3b116b038 100644 --- a/core/lib/multivm/src/versions/testonly/code_oracle.rs +++ b/core/lib/multivm/src/versions/testonly/code_oracle.rs @@ -1,9 +1,8 @@ use ethabi::Token; use zksync_types::{ - get_known_code_key, h256_to_u256, u256_to_h256, web3::keccak256, Address, Execute, - StorageLogWithPreviousValue, U256, + bytecode::BytecodeHash, get_known_code_key, h256_to_u256, u256_to_h256, web3::keccak256, + Address, Execute, StorageLogWithPreviousValue, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::{ get_empty_storage, load_precompiles_contract, read_precompiles_contract, read_test_contract, @@ -25,7 +24,7 @@ pub(crate) fn test_code_oracle() { // Filling the zkevm bytecode let normal_zkevm_bytecode = read_test_contract(); - let normal_zkevm_bytecode_hash = hash_bytecode(&normal_zkevm_bytecode); + let normal_zkevm_bytecode_hash = BytecodeHash::for_bytecode(&normal_zkevm_bytecode).value(); let normal_zkevm_bytecode_keccak_hash = keccak256(&normal_zkevm_bytecode); let mut storage = get_empty_storage(); storage.set_value( @@ -115,7 +114,7 @@ pub(crate) fn test_code_oracle_big_bytecode() { let precompile_contract_bytecode = read_precompiles_contract(); let big_zkevm_bytecode = generate_large_bytecode(); - let big_zkevm_bytecode_hash = hash_bytecode(&big_zkevm_bytecode); + let big_zkevm_bytecode_hash = BytecodeHash::for_bytecode(&big_zkevm_bytecode).value(); let big_zkevm_bytecode_keccak_hash = keccak256(&big_zkevm_bytecode); let mut storage = get_empty_storage(); @@ -173,7 +172,7 @@ pub(crate) fn test_refunds_in_code_oracle() { let precompile_contract_bytecode = read_precompiles_contract(); let normal_zkevm_bytecode = read_test_contract(); - let normal_zkevm_bytecode_hash = hash_bytecode(&normal_zkevm_bytecode); + let normal_zkevm_bytecode_hash = BytecodeHash::for_bytecode(&normal_zkevm_bytecode).value(); let normal_zkevm_bytecode_keccak_hash = keccak256(&normal_zkevm_bytecode); let mut storage = get_empty_storage(); storage.set_value( diff --git a/core/lib/multivm/src/versions/testonly/evm_emulator.rs b/core/lib/multivm/src/versions/testonly/evm_emulator.rs index 1e7e136e4b99..8442df86f93c 100644 --- a/core/lib/multivm/src/versions/testonly/evm_emulator.rs +++ b/core/lib/multivm/src/versions/testonly/evm_emulator.rs @@ -9,11 +9,11 @@ use zksync_system_constants::{ }; use zksync_test_account::TxType; use zksync_types::{ + bytecode::BytecodeHash, get_code_key, get_known_code_key, h256_to_u256, utils::{key_for_eth_balance, storage_key_for_eth_balance}, AccountTreeId, Address, Execute, StorageKey, H256, U256, }; -use zksync_utils::bytecode::{hash_bytecode, hash_evm_bytecode}; use super::{default_system_env, TestedVm, VmTester, VmTesterBuilder}; use crate::interface::{ @@ -30,9 +30,9 @@ const INCREMENTING_CONTRACT_PATH: &str = "etc/contracts-test-data/artifacts-zk/c fn override_system_contracts(storage: &mut InMemoryStorage) { let mock_deployer = read_bytecode(MOCK_DEPLOYER_PATH); - let mock_deployer_hash = hash_bytecode(&mock_deployer); + let mock_deployer_hash = BytecodeHash::for_bytecode(&mock_deployer).value(); let mock_known_code_storage = read_bytecode(MOCK_KNOWN_CODE_STORAGE_PATH); - let mock_known_code_storage_hash = hash_bytecode(&mock_known_code_storage); + let mock_known_code_storage_hash = BytecodeHash::for_bytecode(&mock_known_code_storage).value(); storage.set_value(get_code_key(&CONTRACT_DEPLOYER_ADDRESS), mock_deployer_hash); storage.set_value( @@ -62,7 +62,7 @@ impl EvmTestBuilder { fn new(deploy_emulator: bool, evm_contract_address: Address) -> Self { Self { deploy_emulator, - storage: InMemoryStorage::with_system_contracts(hash_bytecode), + storage: InMemoryStorage::with_system_contracts(), evm_contract_addresses: vec![evm_contract_address], } } @@ -83,7 +83,7 @@ impl EvmTestBuilder { let mut system_env = default_system_env(); if self.deploy_emulator { let evm_bytecode: Vec<_> = (0..32).collect(); - let evm_bytecode_hash = hash_evm_bytecode(&evm_bytecode); + let evm_bytecode_hash = BytecodeHash::for_evm_bytecode(&evm_bytecode).value(); storage.set_value( get_known_code_key(&evm_bytecode_hash), H256::from_low_u64_be(1), @@ -93,11 +93,11 @@ impl EvmTestBuilder { } system_env.base_system_smart_contracts.evm_emulator = Some(SystemContractCode { - hash: hash_bytecode(&mock_emulator), + hash: BytecodeHash::for_bytecode(&mock_emulator).value(), code: mock_emulator, }); } else { - let emulator_hash = hash_bytecode(&mock_emulator); + let emulator_hash = BytecodeHash::for_bytecode(&mock_emulator).value(); storage.set_value(get_known_code_key(&emulator_hash), H256::from_low_u64_be(1)); storage.store_factory_dep(emulator_hash, mock_emulator); @@ -121,7 +121,7 @@ impl EvmTestBuilder { } pub(crate) fn test_tracing_evm_contract_deployment() { - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); override_system_contracts(&mut storage); let mut system_env = default_system_env(); @@ -138,7 +138,7 @@ pub(crate) fn test_tracing_evm_contract_deployment() { let args = [Token::Bytes((0..32).collect())]; let evm_bytecode = ethabi::encode(&args); - let expected_bytecode_hash = hash_evm_bytecode(&evm_bytecode); + let expected_bytecode_hash = BytecodeHash::for_evm_bytecode(&evm_bytecode).value(); let execute = Execute::for_deploy(expected_bytecode_hash, vec![0; 32], &args); let deploy_tx = account.get_l2_tx_for_execute(execute, None); let (_, vm_result) = vm @@ -155,7 +155,7 @@ pub(crate) fn test_tracing_evm_contract_deployment() { // "Deploy" a bytecode in another transaction and check that the first tx doesn't interfere with the returned `dynamic_factory_deps`. let args = [Token::Bytes((0..32).rev().collect())]; let evm_bytecode = ethabi::encode(&args); - let expected_bytecode_hash = hash_evm_bytecode(&evm_bytecode); + let expected_bytecode_hash = BytecodeHash::for_evm_bytecode(&evm_bytecode).value(); let execute = Execute::for_deploy(expected_bytecode_hash, vec![0; 32], &args); let deploy_tx = account.get_l2_tx_for_execute(execute, None); let (_, vm_result) = vm @@ -334,7 +334,7 @@ pub(crate) fn test_mock_emulator_with_deployment(revert: bool) { let mock_emulator_abi = load_contract(MOCK_EMULATOR_PATH); let new_evm_bytecode = vec![0xfe; 96]; - let new_evm_bytecode_hash = hash_evm_bytecode(&new_evm_bytecode); + let new_evm_bytecode_hash = BytecodeHash::for_evm_bytecode(&new_evm_bytecode).value(); let test_fn = mock_emulator_abi.function("testDeploymentAndCall").unwrap(); let test_tx = account.get_l2_tx_for_execute( @@ -412,7 +412,7 @@ pub(crate) fn test_mock_emulator_with_recursive_deployment() { let bytecodes: HashMap<_, _> = (0_u8..10) .map(|byte| { let bytecode = vec![byte; 32]; - (hash_evm_bytecode(&bytecode), bytecode) + (BytecodeHash::for_evm_bytecode(&bytecode).value(), bytecode) }) .collect(); let test_fn = mock_emulator_abi @@ -458,7 +458,7 @@ fn test_mock_emulator_with_partial_reverts_and_rng(rng: &mut impl let all_bytecodes: HashMap<_, _> = (0_u8..10) .map(|_| { let bytecode = vec![rng.gen(); 32]; - (hash_evm_bytecode(&bytecode), bytecode) + (BytecodeHash::for_evm_bytecode(&bytecode).value(), bytecode) }) .collect(); let should_revert: Vec<_> = (0..10).map(|_| rng.gen::()).collect(); diff --git a/core/lib/multivm/src/versions/testonly/get_used_contracts.rs b/core/lib/multivm/src/versions/testonly/get_used_contracts.rs index fa6470b44c47..d5268d9c63ff 100644 --- a/core/lib/multivm/src/versions/testonly/get_used_contracts.rs +++ b/core/lib/multivm/src/versions/testonly/get_used_contracts.rs @@ -5,8 +5,9 @@ use ethabi::Token; use zk_evm_1_3_1::zkevm_opcode_defs::decoding::{EncodingModeProduction, VmEncodingMode}; use zksync_system_constants::CONTRACT_DEPLOYER_ADDRESS; use zksync_test_account::{Account, TxType}; -use zksync_types::{h256_to_u256, AccountTreeId, Address, Execute, StorageKey, H256, U256}; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{ + bytecode::BytecodeHash, h256_to_u256, AccountTreeId, Address, Execute, StorageKey, H256, U256, +}; use super::{ read_proxy_counter_contract, read_test_contract, @@ -76,8 +77,7 @@ pub(crate) fn test_get_used_contracts() { assert!(res2.result.is_failed()); for factory_dep in tx2.execute.factory_deps { - let hash = hash_bytecode(&factory_dep); - let hash_to_u256 = h256_to_u256(hash); + let hash_to_u256 = BytecodeHash::for_bytecode(&factory_dep).value_u256(); assert!(vm.vm.known_bytecode_hashes().contains(&hash_to_u256)); assert!(!vm.vm.decommitted_hashes().contains(&hash_to_u256)); } @@ -105,7 +105,7 @@ fn execute_proxy_counter( gas: u32, ) -> (VmTester, ProxyCounterData, VmExecutionResultAndLogs) { let counter_bytecode = inflated_counter_bytecode(); - let counter_bytecode_hash = h256_to_u256(hash_bytecode(&counter_bytecode)); + let counter_bytecode_hash = BytecodeHash::for_bytecode(&counter_bytecode).value_u256(); let counter_address = Address::repeat_byte(0x23); let mut vm = VmTesterBuilder::new() diff --git a/core/lib/multivm/src/versions/testonly/mod.rs b/core/lib/multivm/src/versions/testonly/mod.rs index 3377a49064f8..68044fe8228c 100644 --- a/core/lib/multivm/src/versions/testonly/mod.rs +++ b/core/lib/multivm/src/versions/testonly/mod.rs @@ -18,11 +18,10 @@ use zksync_contracts::{ SystemContractCode, }; use zksync_types::{ - block::L2BlockHasher, fee_model::BatchFeeInput, get_code_key, get_is_account_key, h256_to_u256, - u256_to_h256, utils::storage_key_for_eth_balance, Address, L1BatchNumber, L2BlockNumber, - L2ChainId, ProtocolVersionId, U256, + block::L2BlockHasher, bytecode::BytecodeHash, fee_model::BatchFeeInput, get_code_key, + get_is_account_key, h256_to_u256, u256_to_h256, utils::storage_key_for_eth_balance, Address, + L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, U256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_vm_interface::{ pubdata::PubdataBuilder, L1BatchEnv, L2BlockEnv, SystemEnv, TxExecutionMode, }; @@ -62,7 +61,7 @@ static BASE_SYSTEM_CONTRACTS: Lazy = Lazy::new(BaseSystemContracts::load_from_disk); fn get_empty_storage() -> InMemoryStorage { - InMemoryStorage::with_system_contracts(hash_bytecode) + InMemoryStorage::with_system_contracts() } pub(crate) fn read_test_contract() -> Vec { @@ -131,7 +130,7 @@ pub(crate) fn read_simple_transfer_contract() -> Vec { pub(crate) fn get_bootloader(test: &str) -> SystemContractCode { let bootloader_code = read_bootloader_code(test); - let bootloader_hash = hash_bytecode(&bootloader_code); + let bootloader_hash = BytecodeHash::for_bytecode(&bootloader_code).value(); SystemContractCode { code: bootloader_code, hash: bootloader_hash, @@ -223,12 +222,13 @@ impl ContractToDeploy { pub fn insert(&self, storage: &mut InMemoryStorage) { let deployer_code_key = get_code_key(&self.address); - storage.set_value(deployer_code_key, hash_bytecode(&self.bytecode)); + let bytecode_hash = BytecodeHash::for_bytecode(&self.bytecode).value(); + storage.set_value(deployer_code_key, bytecode_hash); if self.is_account { let is_account_key = get_is_account_key(&self.address); storage.set_value(is_account_key, u256_to_h256(1_u32.into())); } - storage.store_factory_dep(hash_bytecode(&self.bytecode), self.bytecode.clone()); + storage.store_factory_dep(bytecode_hash, self.bytecode.clone()); if self.is_funded { make_address_rich(storage, self.address); diff --git a/core/lib/multivm/src/versions/testonly/upgrade.rs b/core/lib/multivm/src/versions/testonly/upgrade.rs index 3fb6257f7070..b33862afe4b7 100644 --- a/core/lib/multivm/src/versions/testonly/upgrade.rs +++ b/core/lib/multivm/src/versions/testonly/upgrade.rs @@ -1,6 +1,7 @@ use zksync_contracts::{deployer_contract, load_sys_contract, read_bytecode}; use zksync_test_account::TxType; use zksync_types::{ + bytecode::BytecodeHash, ethabi::{Contract, Token}, get_code_key, get_known_code_key, h256_to_u256, protocol_upgrade::ProtocolUpgradeTxCommonData, @@ -8,7 +9,6 @@ use zksync_types::{ COMPLEX_UPGRADER_ADDRESS, CONTRACT_DEPLOYER_ADDRESS, CONTRACT_FORCE_DEPLOYER_ADDRESS, H256, REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::{ get_complex_upgrade_abi, get_empty_storage, read_complex_upgrade, read_test_contract, @@ -23,7 +23,7 @@ use crate::interface::{ /// - If present, this transaction must be the first one in block pub(crate) fn test_protocol_upgrade_is_first() { let mut storage = get_empty_storage(); - let bytecode_hash = hash_bytecode(&read_test_contract()); + let bytecode_hash = BytecodeHash::for_bytecode(&read_test_contract()).value(); storage.set_value(get_known_code_key(&bytecode_hash), u256_to_h256(1.into())); let mut vm = VmTesterBuilder::new() @@ -111,7 +111,7 @@ pub(crate) fn test_protocol_upgrade_is_first() { /// In this test we try to test how force deployments could be done via protocol upgrade transactions. pub(crate) fn test_force_deploy_upgrade() { let mut storage = get_empty_storage(); - let bytecode_hash = hash_bytecode(&read_test_contract()); + let bytecode_hash = BytecodeHash::for_bytecode(&read_test_contract()).value(); let known_code_key = get_known_code_key(&bytecode_hash); // It is generally expected that all the keys will be set as known prior to the protocol upgrade. storage.set_value(known_code_key, u256_to_h256(1.into())); @@ -156,8 +156,8 @@ pub(crate) fn test_force_deploy_upgrade() { /// Here we show how the work with the complex upgrader could be done. pub(crate) fn test_complex_upgrader() { let mut storage = get_empty_storage(); - let bytecode_hash = hash_bytecode(&read_complex_upgrade()); - let msg_sender_test_hash = hash_bytecode(&read_msg_sender_test()); + let bytecode_hash = BytecodeHash::for_bytecode(&read_complex_upgrade()).value(); + let msg_sender_test_hash = BytecodeHash::for_bytecode(&read_msg_sender_test()).value(); // Let's assume that the bytecode for the implementation of the complex upgrade // is already deployed in some address in user space let upgrade_impl = Address::repeat_byte(1); diff --git a/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs index 07fcdb0f522f..779fc126e72c 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_3_3::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{StoragePtr, WriteStorage}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_1_3_2::history_recorder::{HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory}, }; diff --git a/core/lib/multivm/src/versions/vm_1_3_2/pubdata_utils.rs b/core/lib/multivm/src/versions/vm_1_3_2/pubdata_utils.rs index d88ee70991bc..3c10bd8c48be 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/pubdata_utils.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/pubdata_utils.rs @@ -3,10 +3,10 @@ use std::collections::HashMap; use circuit_sequencer_api_1_3_3::sort_storage_access::sort_storage_access_queries; use zk_evm_1_3_3::aux_structures::Timestamp; use zksync_types::{StorageKey, PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{storage::WriteStorage, VmEvent}, + utils::bytecode::bytecode_len_in_bytes, vm_1_3_2::{history_recorder::HistoryMode, oracles::storage::storage_key_of_log, VmInstance}, }; @@ -30,9 +30,7 @@ impl VmInstance { let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| { - bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD - }) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_1_3_2/test_utils.rs b/core/lib/multivm/src/versions/vm_1_3_2/test_utils.rs index 42106bbdfa22..ac6ce7fcdfcf 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/test_utils.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/test_utils.rs @@ -13,12 +13,12 @@ use zk_evm_1_3_3::{aux_structures::Timestamp, vm_state::VmLocalState}; use zksync_contracts::deployer_contract; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{Address, Token}, h256_to_address, u256_to_h256, web3::keccak256, Execute, Nonce, StorageKey, StorageValue, CONTRACT_DEPLOYER_ADDRESS, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::interface::storage::WriteStorage; /// The tests here help us with the testing the VM @@ -145,7 +145,7 @@ pub fn get_create_execute(code: &[u8], calldata: &[u8]) -> Execute { let params = [ Token::FixedBytes(vec![0u8; 32]), - Token::FixedBytes(hash_bytecode(code).0.to_vec()), + Token::FixedBytes(BytecodeHash::for_bytecode(code).value().0.to_vec()), Token::Bytes(calldata.to_vec()), ]; let calldata = contract_function diff --git a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs index e63b6ec5a87d..c2dfe97ed076 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/transaction_data.rs @@ -1,6 +1,8 @@ use zk_evm_1_3_3::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; use zksync_types::{ - address_to_h256, ceil_div_u256, + address_to_h256, + bytecode::BytecodeHash, + ceil_div_u256, ethabi::{encode, Address, Token}, fee::encoding_len, h256_to_u256, @@ -8,7 +10,6 @@ use zksync_types::{ l2::TransactionType, ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::vm_with_bootloader::MAX_TXS_IN_BLOCK; use crate::{ @@ -194,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm.rs index 45b8a09c5a34..05902b736fbd 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm.rs @@ -1,7 +1,6 @@ use std::{collections::HashSet, rc::Rc}; -use zksync_types::{h256_to_u256, Transaction}; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, h256_to_u256, Transaction}; use zksync_vm_interface::{pubdata::PubdataBuilder, InspectExecutionMode}; use crate::{ @@ -119,7 +118,7 @@ impl VmInterface for Vm { let mut deps_hashes = HashSet::with_capacity(deps.len()); let mut bytecode_hashes = vec![]; let filtered_deps = deps.iter().filter_map(|bytecode| { - let bytecode_hash = hash_bytecode(bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(bytecode).value(); let is_known = !deps_hashes.insert(bytecode_hash) || self.vm.is_bytecode_known(&bytecode_hash); diff --git a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs index eee1baa59d60..ca9ba097d472 100644 --- a/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_1_3_2/vm_with_bootloader.rs @@ -15,10 +15,10 @@ use zk_evm_1_3_3::{ use zksync_contracts::BaseSystemContracts; use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{ - address_to_u256, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, l1::is_l1_tx_type, - Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + address_to_u256, bytecode::BytecodeHash, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, + l1::is_l1_tx_type, Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, + MAX_NEW_FACTORY_DEPS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ interface::{storage::WriteStorage, CompressedBytecodeInfo, L1BatchEnv}, @@ -518,7 +518,7 @@ pub fn push_raw_transaction_to_bootloader_memory PrimitiveValue { } } -// FIXME: &[u8] pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } diff --git a/core/lib/multivm/src/versions/vm_1_4_1/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_1_4_1/implementation/bytecode.rs index c8adcf116d74..0278e239522b 100644 --- a/core/lib/multivm/src/versions/vm_1_4_1/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_1_4_1/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytecode::bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_1_4_1/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_1_4_1/old_vm/oracles/decommitter.rs index e4df1e4c7fcd..0fe3efa30b68 100644 --- a/core/lib/multivm/src/versions/vm_1_4_1/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_1_4_1/old_vm/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_4_1::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{ReadStorage, StoragePtr}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_1_4_1::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }, diff --git a/core/lib/multivm/src/versions/vm_1_4_1/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_1_4_1/tracers/refunds.rs index ab883d9bcbd2..dc945e183a8f 100644 --- a/core/lib/multivm/src/versions/vm_1_4_1/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_1_4_1/tracers/refunds.rs @@ -11,7 +11,6 @@ use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS} use zksync_types::{ ceil_div_u256, l2_to_l1_log::L2ToL1Log, u256_to_h256, L1BatchNumber, H256, U256, }; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{ @@ -20,6 +19,7 @@ use crate::{ L1BatchEnv, Refunds, VmEvent, }, tracers::dynamic::vm_1_4_1::DynTracer, + utils::bytecode::bytecode_len_in_bytes, vm_1_4_1::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, @@ -350,7 +350,7 @@ pub(crate) fn pubdata_published( let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_1_4_1/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_1_4_1/types/internals/transaction_data.rs index 872ab3d05dd5..af9a93f647a2 100644 --- a/core/lib/multivm/src/versions/vm_1_4_1/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_1_4_1/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_1_4_2/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_1_4_2/implementation/bytecode.rs index 2a6ef1d1ab4d..f81deff48c25 100644 --- a/core/lib/multivm/src/versions/vm_1_4_2/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_1_4_2/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_1_4_2/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_1_4_2/old_vm/oracles/decommitter.rs index 3d0e9bda8030..9122a10c9266 100644 --- a/core/lib/multivm/src/versions/vm_1_4_2/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_1_4_2/old_vm/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_4_1::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{ReadStorage, StoragePtr}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_1_4_2::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }, diff --git a/core/lib/multivm/src/versions/vm_1_4_2/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_1_4_2/tracers/refunds.rs index 8cfcd8c327e4..324cad02b4eb 100644 --- a/core/lib/multivm/src/versions/vm_1_4_2/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_1_4_2/tracers/refunds.rs @@ -11,7 +11,6 @@ use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS} use zksync_types::{ ceil_div_u256, l2_to_l1_log::L2ToL1Log, u256_to_h256, L1BatchNumber, H256, U256, }; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{ @@ -20,6 +19,7 @@ use crate::{ L1BatchEnv, Refunds, VmEvent, }, tracers::dynamic::vm_1_4_1::DynTracer, + utils::bytecode::bytecode_len_in_bytes, vm_1_4_2::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, @@ -350,7 +350,7 @@ pub(crate) fn pubdata_published( let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_1_4_2/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_1_4_2/types/internals/transaction_data.rs index 693690e3b42e..e0f113f8a7ff 100644 --- a/core/lib/multivm/src/versions/vm_1_4_2/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_1_4_2/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs index 38d5b40af7e4..42507a589e50 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs index b61560afe39d..804bd7179781 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/old_vm/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_4_0::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{ReadStorage, StoragePtr}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_boojum_integration::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }, diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs index 0d944724afd2..682cbda5252a 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/tracers/refunds.rs @@ -8,7 +8,6 @@ use zk_evm_1_4_0::{ }; use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; use zksync_types::{ceil_div_u256, l2_to_l1_log::L2ToL1Log, u256_to_h256, L1BatchNumber, U256}; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{ @@ -17,6 +16,7 @@ use crate::{ L1BatchEnv, Refunds, VmEvent, }, tracers::dynamic::vm_1_4_0::DynTracer, + utils::bytecode::bytecode_len_in_bytes, vm_boojum_integration::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, @@ -339,7 +339,7 @@ pub(crate) fn pubdata_published( let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs index 774d2061beb2..9011fa486da2 100644 --- a/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_boojum_integration/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_fast/bytecode.rs b/core/lib/multivm/src/versions/vm_fast/bytecode.rs index abbd4461c25e..4dc52951c16c 100644 --- a/core/lib/multivm/src/versions/vm_fast/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_fast/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::{h256_to_u256, H256}; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, h256_to_u256, H256}; use super::Vm; use crate::{ @@ -15,7 +14,7 @@ impl Vm { .get_last_tx_compressed_bytecodes() .iter() .any(|info| { - let hash_bytecode = hash_bytecode(&info.original); + let hash_bytecode = BytecodeHash::for_bytecode(&info.original).value(); let is_bytecode_known = self.world.storage.is_bytecode_known(&hash_bytecode); let is_bytecode_known_cache = self @@ -36,7 +35,7 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !is_bytecode_known(hash_bytecode(dep))) + .filter(|(_idx, dep)| !is_bytecode_known(BytecodeHash::for_bytecode(dep).value())) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_fast/evm_deploy_tracer.rs b/core/lib/multivm/src/versions/vm_fast/evm_deploy_tracer.rs index fb619fa3e5ff..c443c99ccf9a 100644 --- a/core/lib/multivm/src/versions/vm_fast/evm_deploy_tracer.rs +++ b/core/lib/multivm/src/versions/vm_fast/evm_deploy_tracer.rs @@ -3,8 +3,7 @@ use std::{cell::RefCell, collections::HashMap, rc::Rc}; use zksync_system_constants::{CONTRACT_DEPLOYER_ADDRESS, KNOWN_CODES_STORAGE_ADDRESS}; -use zksync_types::{h256_to_u256, U256}; -use zksync_utils::bytecode::hash_evm_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use zksync_vm2::interface::{ CallframeInterface, CallingMode, GlobalStateInterface, Opcode, OpcodeType, ShouldStop, Tracer, }; @@ -66,7 +65,8 @@ impl EvmDeployTracer { Ok(decoded) => { // `unwrap`s should be safe since the function signature is checked above. let published_bytecode = decoded.into_iter().next().unwrap().into_bytes().unwrap(); - let bytecode_hash = h256_to_u256(hash_evm_bytecode(&published_bytecode)); + let bytecode_hash = + BytecodeHash::for_evm_bytecode(&published_bytecode).value_u256(); self.bytecodes.insert(bytecode_hash, published_bytecode); } Err(err) => tracing::error!("Unable to decode `publishEVMBytecode` call: {err}"), diff --git a/core/lib/multivm/src/versions/vm_fast/transaction_data.rs b/core/lib/multivm/src/versions/vm_fast/transaction_data.rs index afc0ef51e7d4..02697beee341 100644 --- a/core/lib/multivm/src/versions/vm_fast/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_fast/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_fast/vm.rs b/core/lib/multivm/src/versions/vm_fast/vm.rs index a91f0831ebbf..c935b1c0e7f5 100644 --- a/core/lib/multivm/src/versions/vm_fast/vm.rs +++ b/core/lib/multivm/src/versions/vm_fast/vm.rs @@ -5,6 +5,7 @@ use zk_evm_1_5_0::{ }; use zksync_contracts::SystemContractCode; use zksync_types::{ + bytecode::BytecodeHash, h256_to_u256, l1::is_l1_tx_type, l2_to_l1_log::UserL2ToL1Log, @@ -18,7 +19,6 @@ use zksync_types::{ Transaction, BOOTLOADER_ADDRESS, H160, H256, KNOWN_CODES_STORAGE_ADDRESS, L1_MESSENGER_ADDRESS, L2_BASE_TOKEN_ADDRESS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_vm2::{ interface::{CallframeInterface, HeapId, StateInterface, Tracer}, ExecutionEnd, FatPointer, Program, Settings, StorageSlot, VirtualMachine, @@ -463,7 +463,7 @@ impl Vm { pub(crate) fn insert_bytecodes<'a>(&mut self, bytecodes: impl IntoIterator) { for code in bytecodes { - let hash = h256_to_u256(hash_bytecode(code)); + let hash = BytecodeHash::for_bytecode(code).value_u256(); self.world.bytecode_cache.insert(hash, code.into()); } } diff --git a/core/lib/multivm/src/versions/vm_latest/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_latest/implementation/bytecode.rs index d0390444e1cb..655f55bc8fc2 100644 --- a/core/lib/multivm/src/versions/vm_latest/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_latest/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs index fc226f03ecea..b059c9716d89 100644 --- a/core/lib/multivm/src/versions/vm_latest/tests/mod.rs +++ b/core/lib/multivm/src/versions/vm_latest/tests/mod.rs @@ -9,9 +9,9 @@ use zk_evm_1_5_0::{ zkevm_opcode_defs::{ContractCodeSha256Format, VersionedHashLen32}, }; use zksync_types::{ - h256_to_u256, writes::StateDiffRecord, StorageKey, StorageValue, Transaction, H256, U256, + bytecode::BytecodeHash, writes::StateDiffRecord, StorageKey, StorageValue, Transaction, H256, + U256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_vm_interface::pubdata::PubdataBuilder; use super::{HistoryEnabled, Vm}; @@ -113,9 +113,9 @@ impl TestedVm for TestedLatestVm { let bytecodes = bytecodes .iter() .map(|&bytecode| { - let hash = hash_bytecode(bytecode); + let hash = BytecodeHash::for_bytecode(bytecode).value_u256(); let words = bytes_to_be_words(bytecode); - (h256_to_u256(hash), words) + (hash, words) }) .collect(); self.state diff --git a/core/lib/multivm/src/versions/vm_latest/tracers/evm_deploy_tracer.rs b/core/lib/multivm/src/versions/vm_latest/tracers/evm_deploy_tracer.rs index 98ae14ff7f89..2e6ab8089eb0 100644 --- a/core/lib/multivm/src/versions/vm_latest/tracers/evm_deploy_tracer.rs +++ b/core/lib/multivm/src/versions/vm_latest/tracers/evm_deploy_tracer.rs @@ -7,8 +7,9 @@ use zk_evm_1_5_0::{ FarCallOpcode, FatPointer, Opcode, CALL_IMPLICIT_CALLDATA_FAT_PTR_REGISTER, }, }; -use zksync_types::{h256_to_u256, CONTRACT_DEPLOYER_ADDRESS, KNOWN_CODES_STORAGE_ADDRESS}; -use zksync_utils::bytecode::hash_evm_bytecode; +use zksync_types::{ + bytecode::BytecodeHash, CONTRACT_DEPLOYER_ADDRESS, KNOWN_CODES_STORAGE_ADDRESS, +}; use super::{traits::VmTracer, utils::read_pointer}; use crate::{ @@ -94,7 +95,7 @@ impl VmTracer for EvmDeployTracer { ) -> TracerExecutionStatus { let timestamp = Timestamp(state.local_state.timestamp); for published_bytecode in mem::take(&mut self.pending_bytecodes) { - let hash = h256_to_u256(hash_evm_bytecode(&published_bytecode)); + let hash = BytecodeHash::for_evm_bytecode(&published_bytecode).value_u256(); let as_words = bytes_to_be_words(&published_bytecode); state .decommittment_processor diff --git a/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs index 544934665adf..33f923414eb3 100644 --- a/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_latest/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -208,7 +208,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_m5/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_m5/oracles/decommitter.rs index ca6fde506f87..7e2264201e11 100644 --- a/core/lib/multivm/src/versions/vm_m5/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_m5/oracles/decommitter.rs @@ -7,11 +7,10 @@ use zk_evm_1_3_1::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_m5::{ history_recorder::HistoryRecorder, storage::{Storage, StoragePtr}, diff --git a/core/lib/multivm/src/versions/vm_m5/pubdata_utils.rs b/core/lib/multivm/src/versions/vm_m5/pubdata_utils.rs index 1fd8c2460930..8eca2ef5cd86 100644 --- a/core/lib/multivm/src/versions/vm_m5/pubdata_utils.rs +++ b/core/lib/multivm/src/versions/vm_m5/pubdata_utils.rs @@ -4,11 +4,11 @@ use circuit_sequencer_api_1_3_3::sort_storage_access::sort_storage_access_querie use itertools::Itertools; use zk_evm_1_3_1::aux_structures::{LogQuery, Timestamp}; use zksync_types::{StorageKey, PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ glue::GlueInto, interface::VmEvent, + utils::bytecode::bytecode_len_in_bytes, vm_m5::{ oracles::storage::storage_key_of_log, storage::Storage, utils::collect_storage_log_queries_after_timestamp, vm_instance::VmInstance, @@ -35,9 +35,7 @@ impl VmInstance { let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| { - bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD - }) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_m5/test_utils.rs b/core/lib/multivm/src/versions/vm_m5/test_utils.rs index ff6ed0392c85..e0e377e85971 100644 --- a/core/lib/multivm/src/versions/vm_m5/test_utils.rs +++ b/core/lib/multivm/src/versions/vm_m5/test_utils.rs @@ -15,12 +15,12 @@ use zk_evm_1_3_1::{ use zksync_contracts::deployer_contract; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{Address, Token}, h256_to_address, u256_to_h256, web3::keccak256, Execute, Nonce, StorageKey, StorageValue, CONTRACT_DEPLOYER_ADDRESS, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::utils::StorageLogQuery; use crate::vm_m5::{ @@ -143,7 +143,7 @@ pub fn get_create_execute(code: &[u8], calldata: &[u8]) -> Execute { let params = [ Token::FixedBytes(vec![0u8; 32]), - Token::FixedBytes(hash_bytecode(code).0.to_vec()), + Token::FixedBytes(BytecodeHash::for_bytecode(code).value().0.to_vec()), Token::Bytes(calldata.to_vec()), ]; let calldata = contract_function diff --git a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs index 2307c5e24127..236c4c3d4122 100644 --- a/core/lib/multivm/src/versions/vm_m5/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m5/transaction_data.rs @@ -1,13 +1,14 @@ use zk_evm_1_3_1::zkevm_opcode_defs::system_params::{MAX_PUBDATA_PER_BLOCK, MAX_TX_ERGS_LIMIT}; use zksync_types::{ - address_to_h256, ceil_div_u256, + address_to_h256, + bytecode::BytecodeHash, + ceil_div_u256, ethabi::{encode, Address, Token}, fee::encoding_len, h256_to_u256, l2::TransactionType, ExecuteTransactionCommon, Transaction, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::vm_with_bootloader::MAX_GAS_PER_PUBDATA_BYTE; use crate::{ @@ -168,7 +169,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs index 706c0fbc717c..0a7df48df80f 100644 --- a/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m5/vm_with_bootloader.rs @@ -14,10 +14,9 @@ use zk_evm_1_3_1::{ use zksync_contracts::BaseSystemContracts; use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{ - address_to_u256, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, Address, Transaction, - BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + address_to_u256, bytecode::BytecodeHash, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, + Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ interface::L1BatchEnv, @@ -585,8 +584,7 @@ fn formal_calldata_abi() -> PrimitiveValue { } pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } diff --git a/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs index a43ec4ec4fd8..5bd33d6d49c1 100644 --- a/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_m6/oracles/decommitter.rs @@ -7,11 +7,10 @@ use zk_evm_1_3_1::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_m6::{ history_recorder::{HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory}, storage::{Storage, StoragePtr}, diff --git a/core/lib/multivm/src/versions/vm_m6/pubdata_utils.rs b/core/lib/multivm/src/versions/vm_m6/pubdata_utils.rs index 196883e1c936..97bf290a2162 100644 --- a/core/lib/multivm/src/versions/vm_m6/pubdata_utils.rs +++ b/core/lib/multivm/src/versions/vm_m6/pubdata_utils.rs @@ -4,11 +4,11 @@ use circuit_sequencer_api_1_3_3::sort_storage_access::sort_storage_access_querie use itertools::Itertools; use zk_evm_1_3_1::aux_structures::{LogQuery, Timestamp}; use zksync_types::{StorageKey, PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ glue::GlueInto, interface::VmEvent, + utils::bytecode::bytecode_len_in_bytes, vm_m6::{ history_recorder::HistoryMode, oracles::storage::storage_key_of_log, storage::Storage, utils::collect_storage_log_queries_after_timestamp, VmInstance, @@ -35,9 +35,7 @@ impl VmInstance { let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| { - bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD - }) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_m6/test_utils.rs b/core/lib/multivm/src/versions/vm_m6/test_utils.rs index 438a67129ac6..0debd8dea568 100644 --- a/core/lib/multivm/src/versions/vm_m6/test_utils.rs +++ b/core/lib/multivm/src/versions/vm_m6/test_utils.rs @@ -13,12 +13,12 @@ use zk_evm_1_3_1::{aux_structures::Timestamp, vm_state::VmLocalState}; use zksync_contracts::deployer_contract; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{Address, Token}, h256_to_address, u256_to_h256, web3::keccak256, Execute, Nonce, StorageKey, StorageValue, CONTRACT_DEPLOYER_ADDRESS, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::utils::StorageLogQuery; use crate::vm_m6::{ @@ -143,7 +143,7 @@ pub fn get_create_execute(code: &[u8], calldata: &[u8]) -> Execute { let params = [ Token::FixedBytes(vec![0u8; 32]), - Token::FixedBytes(hash_bytecode(code).0.to_vec()), + Token::FixedBytes(BytecodeHash::for_bytecode(code).value().0.to_vec()), Token::Bytes(calldata.to_vec()), ]; let calldata = contract_function diff --git a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs index cfd2ebf00e44..d0835b233009 100644 --- a/core/lib/multivm/src/versions/vm_m6/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_m6/transaction_data.rs @@ -1,6 +1,8 @@ use zk_evm_1_3_1::zkevm_opcode_defs::system_params::MAX_TX_ERGS_LIMIT; use zksync_types::{ - address_to_h256, ceil_div_u256, + address_to_h256, + bytecode::BytecodeHash, + ceil_div_u256, ethabi::{encode, Address, Token}, fee::encoding_len, h256_to_u256, @@ -8,7 +10,6 @@ use zksync_types::{ l2::TransactionType, ExecuteTransactionCommon, Transaction, MAX_L2_TX_GAS_LIMIT, U256, }; -use zksync_utils::bytecode::hash_bytecode; use super::vm_with_bootloader::{MAX_GAS_PER_PUBDATA_BYTE, MAX_TXS_IN_BLOCK}; use crate::{ @@ -195,7 +196,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_m6/vm.rs b/core/lib/multivm/src/versions/vm_m6/vm.rs index 2ed2666b2208..ff089ba902dd 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm.rs @@ -1,7 +1,6 @@ use std::{collections::HashSet, rc::Rc}; -use zksync_types::{h256_to_u256, vm::VmVersion, Transaction}; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, h256_to_u256, vm::VmVersion, Transaction}; use zksync_vm_interface::{pubdata::PubdataBuilder, InspectExecutionMode}; use crate::{ @@ -143,7 +142,7 @@ impl VmInterface for Vm { let mut deps_hashes = HashSet::with_capacity(deps.len()); let mut bytecode_hashes = vec![]; let filtered_deps = deps.iter().filter_map(|bytecode| { - let bytecode_hash = hash_bytecode(bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(bytecode).value(); let is_known = !deps_hashes.insert(bytecode_hash) || self.vm.is_bytecode_exists(&bytecode_hash); diff --git a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs index 24cddd5eb5ea..ff83abc45fcf 100644 --- a/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs +++ b/core/lib/multivm/src/versions/vm_m6/vm_with_bootloader.rs @@ -14,10 +14,9 @@ use zk_evm_1_3_1::{ use zksync_contracts::BaseSystemContracts; use zksync_system_constants::MAX_L2_TX_GAS_LIMIT; use zksync_types::{ - address_to_u256, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, Address, Transaction, - BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, + address_to_u256, bytecode::BytecodeHash, fee_model::L1PeggedBatchFeeModelInput, h256_to_u256, + Address, Transaction, BOOTLOADER_ADDRESS, L1_GAS_PER_PUBDATA_BYTE, MAX_NEW_FACTORY_DEPS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ interface::{CompressedBytecodeInfo, L1BatchEnv}, @@ -620,7 +619,7 @@ fn push_raw_transaction_to_bootloader_memory_v1( tx.factory_deps .iter() .filter_map(|bytecode| { - if vm.is_bytecode_exists(&hash_bytecode(bytecode)) { + if vm.is_bytecode_exists(&BytecodeHash::for_bytecode(bytecode).value()) { return None; } bytecode::compress(bytecode.clone()).ok() @@ -692,7 +691,7 @@ fn push_raw_transaction_to_bootloader_memory_v2( tx.factory_deps .iter() .filter_map(|bytecode| { - if vm.is_bytecode_exists(&hash_bytecode(bytecode)) { + if vm.is_bytecode_exists(&BytecodeHash::for_bytecode(bytecode).value()) { return None; } bytecode::compress(bytecode.clone()).ok() @@ -905,8 +904,7 @@ fn formal_calldata_abi() -> PrimitiveValue { } pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/bytecode.rs index 766cac391e32..38cfaa124b16 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs index fc9d0794b958..b0ce7edbc95d 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/old_vm/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_3_3::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{ReadStorage, StoragePtr}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_refunds_enhancement::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }, diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs index 98fee074a940..777f0d51460f 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/tracers/refunds.rs @@ -6,7 +6,6 @@ use zk_evm_1_3_3::{ }; use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS}; use zksync_types::{ceil_div_u256, l2_to_l1_log::L2ToL1Log, u256_to_h256, L1BatchNumber, U256}; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{ @@ -15,6 +14,7 @@ use crate::{ L1BatchEnv, Refunds, VmEvent, }, tracers::dynamic::vm_1_3_3::DynTracer, + utils::bytecode::bytecode_len_in_bytes, vm_refunds_enhancement::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, @@ -335,7 +335,7 @@ pub(crate) fn pubdata_published( let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs index 5bc13bfac2d0..64802d74c878 100644 --- a/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_refunds_enhancement/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/bytecode.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/bytecode.rs index 2b26d4fc9d6d..828b1c961708 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/bytecode.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/implementation/bytecode.rs @@ -1,6 +1,5 @@ use itertools::Itertools; -use zksync_types::U256; -use zksync_utils::bytecode::hash_bytecode; +use zksync_types::{bytecode::BytecodeHash, U256}; use crate::{ interface::{ @@ -25,15 +24,14 @@ impl Vm { .storage .get_ptr() .borrow_mut() - .is_bytecode_known(&hash_bytecode(&info.original)) + .is_bytecode_known(&BytecodeHash::for_bytecode(&info.original).value()) }) } } /// Converts bytecode to tokens and hashes it. pub(crate) fn bytecode_to_factory_dep(bytecode: Vec) -> (U256, Vec) { - let bytecode_hash = hash_bytecode(&bytecode); - let bytecode_hash = U256::from_big_endian(bytecode_hash.as_bytes()); + let bytecode_hash = BytecodeHash::for_bytecode(&bytecode).value_u256(); let bytecode_words = bytes_to_be_words(&bytecode); (bytecode_hash, bytecode_words) } @@ -47,7 +45,11 @@ pub(crate) fn compress_bytecodes( .enumerate() .sorted_by_key(|(_idx, dep)| *dep) .dedup_by(|x, y| x.1 == y.1) - .filter(|(_idx, dep)| !storage.borrow_mut().is_bytecode_known(&hash_bytecode(dep))) + .filter(|(_idx, dep)| { + !storage + .borrow_mut() + .is_bytecode_known(&BytecodeHash::for_bytecode(dep).value()) + }) .sorted_by_key(|(idx, _dep)| *idx) .filter_map(|(_idx, dep)| bytecode::compress(dep.clone()).ok()) .collect() diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs index fad51513dbca..a432e782f658 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/old_vm/oracles/decommitter.rs @@ -7,12 +7,11 @@ use zk_evm_1_3_3::{ }, }; use zksync_types::{u256_to_h256, U256}; -use zksync_utils::bytecode::bytecode_len_in_words; use super::OracleWithHistory; use crate::{ interface::storage::{ReadStorage, StoragePtr}, - utils::bytecode::bytes_to_be_words, + utils::bytecode::{bytecode_len_in_words, bytes_to_be_words}, vm_virtual_blocks::old_vm::history_recorder::{ HistoryEnabled, HistoryMode, HistoryRecorder, WithHistory, }, diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs index b35dfecfa400..59aa837cd8fb 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/tracers/refunds.rs @@ -11,7 +11,6 @@ use zksync_system_constants::{PUBLISH_BYTECODE_OVERHEAD, SYSTEM_CONTEXT_ADDRESS} use zksync_types::{ ceil_div_u256, l2_to_l1_log::L2ToL1Log, u256_to_h256, L1BatchNumber, StorageKey, U256, }; -use zksync_utils::bytecode::bytecode_len_in_bytes; use crate::{ interface::{ @@ -19,6 +18,7 @@ use crate::{ L1BatchEnv, Refunds, VmEvent, VmExecutionResultAndLogs, }, tracers::dynamic::vm_1_3_3::DynTracer, + utils::bytecode::bytecode_len_in_bytes, vm_virtual_blocks::{ bootloader_state::BootloaderState, constants::{BOOTLOADER_HEAP_PAGE, OPERATOR_REFUNDS_OFFSET, TX_GAS_LIMIT_OFFSET}, @@ -329,7 +329,7 @@ pub(crate) fn pubdata_published( let published_bytecode_bytes: u32 = VmEvent::extract_published_bytecodes(&events) .iter() - .map(|bytecodehash| bytecode_len_in_bytes(*bytecodehash) as u32 + PUBLISH_BYTECODE_OVERHEAD) + .map(|bytecode_hash| bytecode_len_in_bytes(bytecode_hash) + PUBLISH_BYTECODE_OVERHEAD) .sum(); storage_writes_pubdata_published diff --git a/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs b/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs index a2540d12a670..d13304c93285 100644 --- a/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs +++ b/core/lib/multivm/src/versions/vm_virtual_blocks/types/internals/transaction_data.rs @@ -2,6 +2,7 @@ use std::convert::TryInto; use zksync_types::{ address_to_h256, + bytecode::BytecodeHash, ethabi::{encode, Address, Token}, fee::{encoding_len, Fee}, h256_to_u256, @@ -11,7 +12,6 @@ use zksync_types::{ web3::Bytes, Execute, ExecuteTransactionCommon, L2ChainId, L2TxCommonData, Nonce, Transaction, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{ utils::bytecode::bytes_to_be_words, @@ -195,7 +195,7 @@ impl TransactionData { let factory_deps_hashes = self .factory_deps .iter() - .map(|dep| h256_to_u256(hash_bytecode(dep))) + .map(|dep| BytecodeHash::for_bytecode(dep).value_u256()) .collect(); self.abi_encode_with_custom_factory_deps(factory_deps_hashes) } diff --git a/core/lib/snapshots_applier/Cargo.toml b/core/lib/snapshots_applier/Cargo.toml index 4ab0c86843ef..d107aac6d4c6 100644 --- a/core/lib/snapshots_applier/Cargo.toml +++ b/core/lib/snapshots_applier/Cargo.toml @@ -17,7 +17,6 @@ zksync_health_check.workspace = true zksync_types.workspace = true zksync_object_store.workspace = true zksync_web3_decl.workspace = true -zksync_utils.workspace = true vise.workspace = true diff --git a/core/lib/snapshots_applier/src/lib.rs b/core/lib/snapshots_applier/src/lib.rs index b4d24a0b1851..2c68b56ca5c6 100644 --- a/core/lib/snapshots_applier/src/lib.rs +++ b/core/lib/snapshots_applier/src/lib.rs @@ -13,6 +13,7 @@ use zksync_health_check::{Health, HealthStatus, HealthUpdater, ReactiveHealthChe use zksync_object_store::{ObjectStore, ObjectStoreError}; use zksync_types::{ api, + bytecode::BytecodeHash, snapshots::{ SnapshotFactoryDependencies, SnapshotHeader, SnapshotRecoveryStatus, SnapshotStorageLog, SnapshotStorageLogsChunk, SnapshotStorageLogsStorageKey, SnapshotVersion, @@ -20,7 +21,6 @@ use zksync_types::{ tokens::TokenInfo, L1BatchNumber, L2BlockNumber, StorageKey, H256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_web3_decl::{ client::{DynClient, L2}, error::{ClientRpcContext, EnrichedClientError, EnrichedClientResult}, @@ -800,9 +800,15 @@ impl<'a> SnapshotsApplier<'a> { // in underlying query, see `https://www.postgresql.org/docs/current/limits.html` // there were around 100 thousand contracts on mainnet, where this issue first manifested for chunk in factory_deps.factory_deps.chunks(1000) { + // TODO: bytecode hashing is ambiguous with EVM bytecodes let chunk_deps_hashmap: HashMap> = chunk .iter() - .map(|dep| (hash_bytecode(&dep.bytecode.0), dep.bytecode.0.clone())) + .map(|dep| { + ( + BytecodeHash::for_bytecode(&dep.bytecode.0).value(), + dep.bytecode.0.clone(), + ) + }) .collect(); storage .factory_deps_dal() diff --git a/core/lib/types/Cargo.toml b/core/lib/types/Cargo.toml index ffa9d219f084..325fe22209a7 100644 --- a/core/lib/types/Cargo.toml +++ b/core/lib/types/Cargo.toml @@ -13,7 +13,6 @@ categories.workspace = true [dependencies] # **IMPORTANT.** Please do not add dependency on `zksync_config` etc. This crate has a heavy dependency graph as is. zksync_system_constants.workspace = true -zksync_utils.workspace = true zksync_basic_types.workspace = true zksync_contracts.workspace = true zksync_mini_merkle_tree.workspace = true diff --git a/core/lib/types/src/abi.rs b/core/lib/types/src/abi.rs index 1ce709617ccf..92d4cb4c8612 100644 --- a/core/lib/types/src/abi.rs +++ b/core/lib/types/src/abi.rs @@ -1,10 +1,9 @@ use anyhow::Context as _; -use zksync_utils::bytecode::hash_bytecode; use crate::{ + bytecode::BytecodeHash, ethabi, ethabi::{ParamType, Token}, - h256_to_u256, transaction_request::TransactionRequest, web3, Address, H256, U256, }; @@ -357,7 +356,7 @@ impl Transaction { // verify data integrity let factory_deps_hashes: Vec<_> = factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(); anyhow::ensure!(tx.factory_deps == factory_deps_hashes); tx.hash() diff --git a/core/lib/types/src/api/state_override.rs b/core/lib/types/src/api/state_override.rs index f2986610840a..69025d1a1f78 100644 --- a/core/lib/types/src/api/state_override.rs +++ b/core/lib/types/src/api/state_override.rs @@ -1,10 +1,12 @@ use std::collections::HashMap; use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; -use zksync_basic_types::{web3::Bytes, H256, U256}; -use zksync_utils::bytecode::{hash_bytecode, validate_bytecode, InvalidBytecodeError}; +use zksync_basic_types::{bytecode::BytecodeHash, web3::Bytes, H256, U256}; -use crate::Address; +use crate::{ + bytecode::{validate_bytecode, InvalidBytecodeError}, + Address, +}; /// Collection of overridden accounts. #[derive(Debug, Clone, Default, Serialize, Deserialize)] @@ -44,7 +46,7 @@ impl Bytecode { /// Returns the canonical hash of this bytecode. pub fn hash(&self) -> H256 { - hash_bytecode(&self.0 .0) + BytecodeHash::for_bytecode(&self.0 .0).value() } /// Converts this bytecode into bytes. diff --git a/core/lib/types/src/l1/mod.rs b/core/lib/types/src/l1/mod.rs index 0cc0f3b0e489..33225dd6b0c9 100644 --- a/core/lib/types/src/l1/mod.rs +++ b/core/lib/types/src/l1/mod.rs @@ -1,21 +1,21 @@ //! Definition of ZKsync network priority operations: operations initiated from the L1. -use std::convert::TryFrom; - use serde::{Deserialize, Serialize}; -use zksync_basic_types::{web3::Log, Address, L1BlockNumber, PriorityOpId, H256, U256}; -use zksync_utils::bytecode::hash_bytecode; use super::Transaction; use crate::{ - abi, address_to_u256, ethabi, h256_to_u256, + abi, address_to_u256, + bytecode::BytecodeHash, + ethabi, helpers::unix_timestamp_ms, l1::error::L1TxParseError, l2::TransactionType, priority_op_onchain_data::{PriorityOpOnchainData, PriorityOpOnchainMetadata}, tx::Execute, - u256_to_address, ExecuteTransactionCommon, PRIORITY_OPERATION_L2_TX_TYPE, - PROTOCOL_UPGRADE_TX_TYPE, + u256_to_address, + web3::Log, + Address, ExecuteTransactionCommon, L1BlockNumber, PriorityOpId, H256, + PRIORITY_OPERATION_L2_TX_TYPE, PROTOCOL_UPGRADE_TX_TYPE, U256, }; pub mod error; @@ -293,7 +293,7 @@ impl From for abi::NewPriorityRequest { signature: vec![], factory_deps: factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(), paymaster_input: vec![], reserved_dynamic: vec![], @@ -318,7 +318,7 @@ impl TryFrom for L1Tx { let factory_deps_hashes: Vec<_> = req .factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(); anyhow::ensure!(req.transaction.factory_deps == factory_deps_hashes); for item in &req.transaction.reserved[2..] { diff --git a/core/lib/types/src/lib.rs b/core/lib/types/src/lib.rs index 48ed7445ef5e..8ec98ec0571e 100644 --- a/core/lib/types/src/lib.rs +++ b/core/lib/types/src/lib.rs @@ -15,9 +15,9 @@ pub use protocol_upgrade::{ProtocolUpgrade, ProtocolVersion}; use serde::{Deserialize, Serialize}; pub use storage::*; pub use tx::Execute; +use zksync_basic_types::bytecode::BytecodeHash; pub use zksync_basic_types::{protocol_version::ProtocolVersionId, vm, *}; pub use zksync_crypto_primitives::*; -use zksync_utils::bytecode::hash_bytecode; use crate::{ l2::{L2Tx, TransactionType}, @@ -284,7 +284,7 @@ impl TryFrom for abi::Transaction { signature: vec![], factory_deps: factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(), paymaster_input: vec![], reserved_dynamic: vec![], @@ -315,7 +315,7 @@ impl TryFrom for abi::Transaction { signature: vec![], factory_deps: factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(), paymaster_input: vec![], reserved_dynamic: vec![], @@ -344,7 +344,7 @@ impl Transaction { } => { let factory_deps_hashes: Vec<_> = factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(); anyhow::ensure!(tx.factory_deps == factory_deps_hashes); for item in &tx.reserved[2..] { diff --git a/core/lib/types/src/transaction_request.rs b/core/lib/types/src/transaction_request.rs index 931615bad0fe..db66c6955bda 100644 --- a/core/lib/types/src/transaction_request.rs +++ b/core/lib/types/src/transaction_request.rs @@ -3,19 +3,18 @@ use std::convert::{TryFrom, TryInto}; use rlp::{DecoderError, Rlp, RlpStream}; use serde::{Deserialize, Serialize}; use thiserror::Error; -use zksync_basic_types::H256; use zksync_system_constants::{DEFAULT_L2_TX_GAS_PER_PUBDATA_BYTE, MAX_ENCODED_TX_SIZE}; -use zksync_utils::bytecode::{hash_bytecode, validate_bytecode, InvalidBytecodeError}; use super::{EIP_1559_TX_TYPE, EIP_2930_TX_TYPE, EIP_712_TX_TYPE}; use crate::{ + bytecode::{validate_bytecode, BytecodeHash, InvalidBytecodeError}, fee::Fee, l1::L1Tx, l2::{L2Tx, TransactionType}, u256_to_h256, web3::{keccak256, keccak256_concat, AccessList, Bytes}, Address, EIP712TypedStructure, Eip712Domain, L1TxCommonData, L2ChainId, Nonce, - PackedEthSignature, StructBuilder, LEGACY_TX_TYPE, U256, U64, + PackedEthSignature, StructBuilder, H256, LEGACY_TX_TYPE, U256, U64, }; /// Call contract request (eth_call / eth_estimateGas) @@ -174,7 +173,7 @@ impl CallRequestBuilder { } } -#[derive(Debug, Error, PartialEq)] +#[derive(Debug, Error)] pub enum SerializationTransactionError { #[error("transaction type is not supported")] UnknownTransactionFormat, @@ -353,7 +352,7 @@ impl EIP712TypedStructure for TransactionRequest { let factory_dep_hashes: Vec<_> = self .get_factory_deps() .into_iter() - .map(|dep| hash_bytecode(&dep)) + .map(|dep| BytecodeHash::for_bytecode(&dep).value()) .collect(); builder.add_member("factoryDeps", &factory_dep_hashes.as_slice()); @@ -1158,9 +1157,9 @@ mod tests { let decoded_tx = TransactionRequest::from_bytes(encoded_tx.as_slice(), L2ChainId::from(272)); - assert_eq!( - decoded_tx, - Err(SerializationTransactionError::WrongChainId(Some(270))) + assert_matches!( + decoded_tx.unwrap_err(), + SerializationTransactionError::WrongChainId(Some(270)) ); } @@ -1236,9 +1235,9 @@ mod tests { data.insert(0, EIP_1559_TX_TYPE); let decoded_tx = TransactionRequest::from_bytes(data.as_slice(), L2ChainId::from(270)); - assert_eq!( - decoded_tx, - Err(SerializationTransactionError::WrongChainId(Some(272))) + assert_matches!( + decoded_tx.unwrap_err(), + SerializationTransactionError::WrongChainId(Some(272)) ); } @@ -1276,9 +1275,9 @@ mod tests { data.insert(0, EIP_1559_TX_TYPE); let res = TransactionRequest::from_bytes(data.as_slice(), L2ChainId::from(270)); - assert_eq!( - res, - Err(SerializationTransactionError::AccessListsNotSupported) + assert_matches!( + res.unwrap_err(), + SerializationTransactionError::AccessListsNotSupported ); } @@ -1313,9 +1312,9 @@ mod tests { data.insert(0, EIP_2930_TX_TYPE); let res = TransactionRequest::from_bytes(data.as_slice(), L2ChainId::from(270)); - assert_eq!( - res, - Err(SerializationTransactionError::AccessListsNotSupported) + assert_matches!( + res.unwrap_err(), + SerializationTransactionError::AccessListsNotSupported ); } @@ -1341,7 +1340,7 @@ mod tests { }; let execute_tx2: Result = L2Tx::from_request(tx2, usize::MAX, true); - assert_eq!( + assert_matches!( execute_tx2.unwrap_err(), SerializationTransactionError::TooBigNonce ); @@ -1358,7 +1357,7 @@ mod tests { }; let execute_tx1: Result = L2Tx::from_request(tx1, usize::MAX, true); - assert_eq!( + assert_matches!( execute_tx1.unwrap_err(), SerializationTransactionError::MaxFeePerGasNotU64 ); @@ -1372,7 +1371,7 @@ mod tests { }; let execute_tx2: Result = L2Tx::from_request(tx2, usize::MAX, true); - assert_eq!( + assert_matches!( execute_tx2.unwrap_err(), SerializationTransactionError::MaxPriorityFeePerGasNotU64 ); @@ -1390,7 +1389,7 @@ mod tests { let execute_tx3: Result = L2Tx::from_request(tx3, usize::MAX, true); - assert_eq!( + assert_matches!( execute_tx3.unwrap_err(), SerializationTransactionError::MaxFeePerPubdataByteNotU64 ); diff --git a/core/lib/types/src/tx/execute.rs b/core/lib/types/src/tx/execute.rs index 0c3f63467cc4..f5cffef606eb 100644 --- a/core/lib/types/src/tx/execute.rs +++ b/core/lib/types/src/tx/execute.rs @@ -1,7 +1,7 @@ use once_cell::sync::Lazy; use serde::{Deserialize, Serialize}; +use zksync_basic_types::bytecode::BytecodeHash; use zksync_system_constants::CONTRACT_DEPLOYER_ADDRESS; -use zksync_utils::bytecode::hash_bytecode; use crate::{ ethabi, serde_wrappers::ZeroPrefixHexSerde, Address, EIP712TypedStructure, StructBuilder, H256, @@ -127,7 +127,7 @@ impl Execute { contract_bytecode: Vec, constructor_input: &[ethabi::Token], ) -> Self { - let bytecode_hash = hash_bytecode(&contract_bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(&contract_bytecode).value(); Self { contract_address: Some(CONTRACT_DEPLOYER_ADDRESS), calldata: Self::encode_deploy_params_create( diff --git a/core/lib/utils/Cargo.toml b/core/lib/utils/Cargo.toml index 9ba286a7e493..216f3b12d426 100644 --- a/core/lib/utils/Cargo.toml +++ b/core/lib/utils/Cargo.toml @@ -11,15 +11,11 @@ keywords.workspace = true categories.workspace = true [dependencies] -zksync_basic_types.workspace = true -zk_evm.workspace = true zksync_vlog.workspace = true -const-decoder.workspace = true tokio = { workspace = true, features = ["time"] } tracing.workspace = true anyhow.workspace = true -thiserror.workspace = true futures.workspace = true reqwest = { workspace = true, features = ["blocking"] } serde_json.workspace = true diff --git a/core/lib/utils/src/lib.rs b/core/lib/utils/src/lib.rs index e2ab70695113..85618a2e61ef 100644 --- a/core/lib/utils/src/lib.rs +++ b/core/lib/utils/src/lib.rs @@ -1,6 +1,5 @@ //! Various helpers used in the ZKsync stack. -pub mod bytecode; pub mod env; pub mod http_with_retries; pub mod panic_extractor; diff --git a/core/lib/vm_executor/Cargo.toml b/core/lib/vm_executor/Cargo.toml index 06a531252c54..0402b7828e58 100644 --- a/core/lib/vm_executor/Cargo.toml +++ b/core/lib/vm_executor/Cargo.toml @@ -15,7 +15,6 @@ zksync_contracts.workspace = true zksync_dal.workspace = true zksync_types.workspace = true zksync_multivm.workspace = true -zksync_utils.workspace = true async-trait.workspace = true once_cell.workspace = true diff --git a/core/lib/vm_executor/src/oneshot/tests.rs b/core/lib/vm_executor/src/oneshot/tests.rs index 65d2ff3727c0..9649f5b49905 100644 --- a/core/lib/vm_executor/src/oneshot/tests.rs +++ b/core/lib/vm_executor/src/oneshot/tests.rs @@ -4,7 +4,6 @@ use assert_matches::assert_matches; use test_casing::{test_casing, Product}; use zksync_multivm::interface::storage::InMemoryStorage; use zksync_types::{ProtocolVersionId, H256}; -use zksync_utils::bytecode::hash_bytecode; use super::*; use crate::testonly::{ @@ -75,7 +74,7 @@ fn setting_up_nonce_and_balance_in_storage() { #[tokio::test] async fn inspecting_transfer(exec_mode: TxExecutionMode, fast_vm_mode: FastVmMode) { let tx = create_l2_transaction(1_000_000_000.into(), Nonce(0)); - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); storage.set_value( storage_key_for_eth_balance(&tx.initiator_account()), u256_to_h256(u64::MAX.into()), diff --git a/core/lib/vm_interface/src/storage/in_memory.rs b/core/lib/vm_interface/src/storage/in_memory.rs index d83f675cd54e..f756e7a6d76f 100644 --- a/core/lib/vm_interface/src/storage/in_memory.rs +++ b/core/lib/vm_interface/src/storage/in_memory.rs @@ -1,9 +1,9 @@ use std::collections::{hash_map::Entry, BTreeMap, HashMap}; use zksync_types::{ - block::DeployedContract, get_code_key, get_known_code_key, get_system_context_init_logs, - system_contracts::get_system_smart_contracts, L2ChainId, StorageKey, StorageLog, StorageValue, - H256, + block::DeployedContract, bytecode::BytecodeHash, get_code_key, get_known_code_key, + get_system_context_init_logs, system_contracts::get_system_smart_contracts, L2ChainId, + StorageKey, StorageLog, StorageValue, H256, }; use super::ReadStorage; @@ -21,29 +21,20 @@ pub struct InMemoryStorage { impl InMemoryStorage { /// Constructs a storage that contains system smart contracts. - pub fn with_system_contracts(bytecode_hasher: impl Fn(&[u8]) -> H256) -> Self { - Self::with_system_contracts_and_chain_id( - L2ChainId::from(IN_MEMORY_STORAGE_DEFAULT_NETWORK_ID), - bytecode_hasher, - ) + pub fn with_system_contracts() -> Self { + Self::with_system_contracts_and_chain_id(L2ChainId::from( + IN_MEMORY_STORAGE_DEFAULT_NETWORK_ID, + )) } /// Constructs a storage that contains system smart contracts (with a given chain id). - pub fn with_system_contracts_and_chain_id( - chain_id: L2ChainId, - bytecode_hasher: impl Fn(&[u8]) -> H256, - ) -> Self { - Self::with_custom_system_contracts_and_chain_id( - chain_id, - bytecode_hasher, - get_system_smart_contracts(false), - ) + pub fn with_system_contracts_and_chain_id(chain_id: L2ChainId) -> Self { + Self::with_custom_system_contracts_and_chain_id(chain_id, get_system_smart_contracts(false)) } /// Constructs a storage that contains custom system contracts (provided in a vector). pub fn with_custom_system_contracts_and_chain_id( chain_id: L2ChainId, - bytecode_hasher: impl Fn(&[u8]) -> H256, contracts: Vec, ) -> Self { let system_context_init_log = get_system_context_init_logs(chain_id); @@ -51,7 +42,7 @@ impl InMemoryStorage { let state_without_indices: BTreeMap<_, _> = contracts .iter() .flat_map(|contract| { - let bytecode_hash = bytecode_hasher(&contract.bytecode); + let bytecode_hash = BytecodeHash::for_bytecode(&contract.bytecode).value(); let deployer_code_key = get_code_key(contract.account_id.address()); let is_known_code_key = get_known_code_key(&bytecode_hash); @@ -72,7 +63,12 @@ impl InMemoryStorage { let factory_deps = contracts .into_iter() - .map(|contract| (bytecode_hasher(&contract.bytecode), contract.bytecode)) + .map(|contract| { + ( + BytecodeHash::for_bytecode(&contract.bytecode).value(), + contract.bytecode, + ) + }) .collect(); let last_enum_index_set = state.len() as u64; diff --git a/core/node/api_server/Cargo.toml b/core/node/api_server/Cargo.toml index d0723a9d23e7..70d343430a58 100644 --- a/core/node/api_server/Cargo.toml +++ b/core/node/api_server/Cargo.toml @@ -25,7 +25,6 @@ zksync_state.workspace = true zksync_system_constants.workspace = true zksync_metadata_calculator.workspace = true zksync_web3_decl = { workspace = true, features = ["server"] } -zksync_utils.workspace = true zksync_protobuf.workspace = true zksync_mini_merkle_tree.workspace = true zksync_multivm.workspace = true 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 613475b6ef92..282d9bdf1b77 100644 --- a/core/node/api_server/src/execution_sandbox/vm_metrics.rs +++ b/core/node/api_server/src/execution_sandbox/vm_metrics.rs @@ -7,8 +7,7 @@ use zksync_multivm::{ interface::{TransactionExecutionMetrics, VmEvent, VmExecutionResultAndLogs}, utils::StorageWritesDeduplicator, }; -use zksync_types::H256; -use zksync_utils::bytecode::bytecode_len_in_bytes; +use zksync_types::{bytecode::BytecodeHash, H256}; use crate::utils::ReportFilter; @@ -149,7 +148,11 @@ pub(super) fn collect_tx_execution_metrics( .sum(); let published_bytecode_bytes = VmEvent::extract_published_bytecodes(&result.logs.events) .iter() - .map(|bytecode_hash| bytecode_len_in_bytes(*bytecode_hash)) + .map(|&bytecode_hash| { + BytecodeHash::try_from(bytecode_hash) + .expect("published unparseable bytecode hash") + .len_in_bytes() + }) .sum(); TransactionExecutionMetrics { diff --git a/core/node/api_server/src/tx_sender/tests/gas_estimation.rs b/core/node/api_server/src/tx_sender/tests/gas_estimation.rs index 5e0c67477ffe..954792f915cc 100644 --- a/core/node/api_server/src/tx_sender/tests/gas_estimation.rs +++ b/core/node/api_server/src/tx_sender/tests/gas_estimation.rs @@ -7,10 +7,10 @@ use test_casing::{test_casing, Product}; use zksync_system_constants::CODE_ORACLE_ADDRESS; use zksync_types::{ api::state_override::{OverrideAccount, OverrideState}, + bytecode::BytecodeHash, web3::keccak256, K256PrivateKey, }; -use zksync_utils::bytecode::hash_bytecode; use super::*; use crate::{ @@ -216,7 +216,7 @@ async fn initial_estimate_for_code_oracle_tx() { // 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_bytecode_hash = BytecodeHash::for_bytecode(&huge_contract_bytecode).value(); let huge_contract_keccak_hash = H256(keccak256(&huge_contract_bytecode)); let state_override = StateBuilder::default() @@ -240,7 +240,7 @@ async fn initial_estimate_for_code_oracle_tx() { (*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_bytecode_hash = BytecodeHash::for_bytecode(code_oracle_bytecode).value(); let code_oracle_keccak_hash = H256(keccak256(code_oracle_bytecode)); let warm_bytecode_hashes = [ @@ -444,7 +444,7 @@ async fn estimating_gas_for_code_oracle_tx() { // 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_bytecode_hash = BytecodeHash::for_bytecode(&huge_contract_bytecode).value(); let huge_contract_keccak_hash = H256(keccak256(&huge_contract_bytecode)); let state_override = StateBuilder::default() diff --git a/core/node/api_server/src/web3/namespaces/eth.rs b/core/node/api_server/src/web3/namespaces/eth.rs index 588316ce70e4..2765de2c2892 100644 --- a/core/node/api_server/src/web3/namespaces/eth.rs +++ b/core/node/api_server/src/web3/namespaces/eth.rs @@ -6,6 +6,7 @@ use zksync_types::{ state_override::StateOverride, BlockId, BlockNumber, FeeHistory, GetLogsFilter, Transaction, TransactionId, TransactionReceipt, TransactionVariant, }, + bytecode::{trim_padded_evm_bytecode, BytecodeMarker}, l2::{L2Tx, TransactionType}, transaction_request::CallRequest, u256_to_h256, @@ -13,7 +14,6 @@ use zksync_types::{ web3::{self, Bytes, SyncInfo, SyncState}, AccountTreeId, L2BlockNumber, StorageKey, H256, L2_BASE_TOKEN_ADDRESS, U256, }; -use zksync_utils::bytecode::{prepare_evm_bytecode, BytecodeMarker}; use zksync_web3_decl::{ error::Web3Error, types::{Address, Block, Filter, FilterChanges, Log, U64}, @@ -404,7 +404,7 @@ impl EthNamespace { // Check if the bytecode is an EVM bytecode, and if so, pre-process it correspondingly. let marker = BytecodeMarker::new(contract_code.bytecode_hash); let prepared_bytecode = if marker == Some(BytecodeMarker::Evm) { - prepare_evm_bytecode(&contract_code.bytecode) + trim_padded_evm_bytecode(&contract_code.bytecode) .with_context(|| { format!( "malformed EVM bytecode at address {address:?}, hash = {:?}", diff --git a/core/node/api_server/src/web3/tests/mod.rs b/core/node/api_server/src/web3/tests/mod.rs index 9080c5ba413c..feac8eb8d17f 100644 --- a/core/node/api_server/src/web3/tests/mod.rs +++ b/core/node/api_server/src/web3/tests/mod.rs @@ -33,6 +33,10 @@ use zksync_system_constants::{ use zksync_types::{ api, block::{pack_block_info, L2BlockHasher, L2BlockHeader}, + bytecode::{ + testonly::{PROCESSED_EVM_BYTECODE, RAW_EVM_BYTECODE}, + BytecodeHash, + }, fee_model::{BatchFeeInput, FeeParams}, get_nonce_key, l2::L2Tx, @@ -45,10 +49,6 @@ use zksync_types::{ AccountTreeId, Address, L1BatchNumber, Nonce, ProtocolVersionId, StorageKey, StorageLog, H256, U256, U64, }; -use zksync_utils::bytecode::{ - hash_bytecode, hash_evm_bytecode, - testonly::{PROCESSED_EVM_BYTECODE, RAW_EVM_BYTECODE}, -}; use zksync_vm_executor::oneshot::MockOneshotExecutor; use zksync_web3_decl::{ client::{Client, DynClient, L2}, @@ -676,7 +676,7 @@ impl HttpTest for StorageAccessWithSnapshotRecovery { fn storage_initialization(&self) -> StorageInitialization { let address = Address::repeat_byte(1); let code_key = get_code_key(&address); - let code_hash = hash_bytecode(&[0; 32]); + let code_hash = BytecodeHash::for_bytecode(&[0; 32]).value(); let balance_key = storage_key_for_eth_balance(&address); let logs = vec![ StorageLog::new_write_log(code_key, code_hash), @@ -1171,7 +1171,7 @@ impl GetBytecodeTest { at_block: L2BlockNumber, address: Address, ) -> anyhow::Result<()> { - let evm_bytecode_hash = hash_evm_bytecode(RAW_EVM_BYTECODE); + let evm_bytecode_hash = BytecodeHash::for_evm_bytecode(RAW_EVM_BYTECODE).value(); let code_log = StorageLog::new_write_log(get_code_key(&address), evm_bytecode_hash); connection .storage_logs_dal() diff --git a/core/node/contract_verification_server/Cargo.toml b/core/node/contract_verification_server/Cargo.toml index 038347debc64..e6a81fe6026a 100644 --- a/core/node/contract_verification_server/Cargo.toml +++ b/core/node/contract_verification_server/Cargo.toml @@ -13,7 +13,6 @@ categories.workspace = true [dependencies] zksync_dal.workspace = true zksync_types.workspace = true -zksync_utils.workspace = true vise.workspace = true anyhow.workspace = true diff --git a/core/node/contract_verification_server/src/api_impl.rs b/core/node/contract_verification_server/src/api_impl.rs index 94be65673bad..b0336fd284b6 100644 --- a/core/node/contract_verification_server/src/api_impl.rs +++ b/core/node/contract_verification_server/src/api_impl.rs @@ -9,12 +9,12 @@ use axum::{ }; use zksync_dal::{CoreDal, DalError}; use zksync_types::{ + bytecode::BytecodeMarker, contract_verification_api::{ CompilerVersions, VerificationIncomingRequest, VerificationInfo, VerificationRequestStatus, }, Address, }; -use zksync_utils::bytecode::BytecodeMarker; use super::{api_decl::RestApi, metrics::METRICS}; diff --git a/core/node/contract_verification_server/src/tests.rs b/core/node/contract_verification_server/src/tests.rs index b7b0d3e8efb4..c5c1d88b3d0c 100644 --- a/core/node/contract_verification_server/src/tests.rs +++ b/core/node/contract_verification_server/src/tests.rs @@ -12,10 +12,10 @@ use tower::ServiceExt; use zksync_dal::{Connection, Core, CoreDal}; use zksync_node_test_utils::create_l2_block; use zksync_types::{ - contract_verification_api::CompilerVersions, get_code_key, Address, L2BlockNumber, - ProtocolVersion, StorageLog, + bytecode::{BytecodeHash, BytecodeMarker}, + contract_verification_api::CompilerVersions, + get_code_key, Address, L2BlockNumber, ProtocolVersion, StorageLog, }; -use zksync_utils::bytecode::{hash_bytecode, hash_evm_bytecode, BytecodeMarker}; use super::*; use crate::api_impl::ApiError; @@ -53,8 +53,8 @@ async fn mock_deploy_contract( kind: BytecodeMarker, ) { let bytecode_hash = match kind { - BytecodeMarker::EraVm => hash_bytecode(&[0; 32]), - BytecodeMarker::Evm => hash_evm_bytecode(&[0; 96]), + BytecodeMarker::EraVm => BytecodeHash::for_bytecode(&[0; 32]).value(), + BytecodeMarker::Evm => BytecodeHash::for_evm_bytecode(&[0; 96]).value(), }; let deploy_log = StorageLog::new_write_log(get_code_key(&address), bytecode_hash); storage diff --git a/core/node/genesis/Cargo.toml b/core/node/genesis/Cargo.toml index 71c4c45e9e38..d625d7186bdf 100644 --- a/core/node/genesis/Cargo.toml +++ b/core/node/genesis/Cargo.toml @@ -20,7 +20,6 @@ zksync_contracts.workspace = true zksync_eth_client.workspace = true zksync_merkle_tree.workspace = true zksync_system_constants.workspace = true -zksync_utils.workspace = true tokio = { workspace = true, features = ["time"] } anyhow.workspace = true diff --git a/core/node/genesis/src/lib.rs b/core/node/genesis/src/lib.rs index 03f51d5c5fc3..0a0e77d97f95 100644 --- a/core/node/genesis/src/lib.rs +++ b/core/node/genesis/src/lib.rs @@ -17,6 +17,7 @@ use zksync_multivm::utils::get_max_gas_per_pubdata_byte; use zksync_system_constants::PRIORITY_EXPIRATION; use zksync_types::{ block::{BlockGasCount, DeployedContract, L1BatchHeader, L2BlockHasher, L2BlockHeader}, + bytecode::BytecodeHash, commitment::{CommitmentInput, L1BatchCommitment}, fee_model::BatchFeeInput, protocol_upgrade::decode_set_chain_id_event, @@ -27,7 +28,6 @@ use zksync_types::{ AccountTreeId, Address, Bloom, L1BatchNumber, L1ChainId, L2BlockNumber, L2ChainId, ProtocolVersion, ProtocolVersionId, StorageKey, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::utils::{ add_eth_token, get_deduped_log_queries, get_storage_logs, @@ -447,7 +447,12 @@ pub async fn create_genesis_l1_batch( let factory_deps = system_contracts .iter() - .map(|c| (hash_bytecode(&c.bytecode), c.bytecode.clone())) + .map(|c| { + ( + BytecodeHash::for_bytecode(&c.bytecode).value(), + c.bytecode.clone(), + ) + }) .collect(); insert_base_system_contracts_to_factory_deps(&mut transaction, base_system_contracts).await?; diff --git a/core/node/genesis/src/utils.rs b/core/node/genesis/src/utils.rs index d8076229fa54..a51f49a166a2 100644 --- a/core/node/genesis/src/utils.rs +++ b/core/node/genesis/src/utils.rs @@ -10,6 +10,7 @@ use zksync_multivm::{ use zksync_system_constants::{DEFAULT_ERA_CHAIN_ID, ETHEREUM_ADDRESS}; use zksync_types::{ block::{DeployedContract, L1BatchTreeData}, + bytecode::BytecodeHash, commitment::L1BatchCommitment, get_code_key, get_known_code_key, get_system_context_init_logs, h256_to_u256, tokens::{TokenInfo, TokenMetadata}, @@ -17,7 +18,6 @@ use zksync_types::{ zk_evm_types::{LogQuery, Timestamp}, AccountTreeId, L1BatchNumber, L2BlockNumber, L2ChainId, StorageKey, StorageLog, H256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::GenesisError; @@ -51,7 +51,7 @@ pub(super) fn get_storage_logs(system_contracts: &[DeployedContract]) -> Vec = system_contracts .iter() .map(|contract| { - let hash = hash_bytecode(&contract.bytecode); + let hash = BytecodeHash::for_bytecode(&contract.bytecode).value(); let known_code_key = get_known_code_key(&hash); let marked_known_value = H256::from_low_u64_be(1u64); @@ -63,7 +63,7 @@ pub(super) fn get_storage_logs(system_contracts: &[DeployedContract]) -> Vec = system_contracts .iter() .map(|contract| { - let hash = hash_bytecode(&contract.bytecode); + let hash = BytecodeHash::for_bytecode(&contract.bytecode).value(); let code_key = get_code_key(contract.account_id.address()); StorageLog::new_write_log(code_key, hash) }) diff --git a/core/node/node_sync/Cargo.toml b/core/node/node_sync/Cargo.toml index e42cbff85806..28dbf6b3150e 100644 --- a/core/node/node_sync/Cargo.toml +++ b/core/node/node_sync/Cargo.toml @@ -21,7 +21,6 @@ zksync_state_keeper.workspace = true zksync_shared_metrics.workspace = true zksync_web3_decl.workspace = true zksync_health_check.workspace = true -zksync_utils.workspace = true zksync_eth_client.workspace = true zksync_concurrency.workspace = true zksync_consensus_roles.workspace = true diff --git a/core/node/node_sync/src/client.rs b/core/node/node_sync/src/client.rs index d56d8ebc2631..ce6e08e29227 100644 --- a/core/node/node_sync/src/client.rs +++ b/core/node/node_sync/src/client.rs @@ -8,6 +8,7 @@ use zksync_health_check::{CheckHealth, Health, HealthStatus}; use zksync_system_constants::ACCOUNT_CODE_STORAGE_ADDRESS; use zksync_types::{ api::{self, en}, + bytecode::BytecodeHash, get_code_key, h256_to_u256, Address, L2BlockNumber, ProtocolVersionId, H256, U64, }; use zksync_web3_decl::{ @@ -57,7 +58,7 @@ impl MainNodeClient for Box> { .with_arg("hash", &hash) .await?; if let Some(bytecode) = &bytecode { - let actual_bytecode_hash = zksync_utils::bytecode::hash_bytecode(bytecode); + let actual_bytecode_hash = BytecodeHash::for_bytecode(bytecode).value(); if actual_bytecode_hash != hash { return Err(EnrichedClientError::custom( "Got invalid base system contract bytecode from main node", diff --git a/core/node/state_keeper/Cargo.toml b/core/node/state_keeper/Cargo.toml index 75d7c9f1e943..ff17ec2268ab 100644 --- a/core/node/state_keeper/Cargo.toml +++ b/core/node/state_keeper/Cargo.toml @@ -22,7 +22,6 @@ zksync_mempool.workspace = true zksync_shared_metrics.workspace = true zksync_config.workspace = true zksync_node_fee_model.workspace = true -zksync_utils.workspace = true zksync_contracts.workspace = true zksync_protobuf.workspace = true zksync_test_account.workspace = true @@ -32,7 +31,6 @@ zksync_vm_executor.workspace = true zksync_system_constants.workspace = true zksync_base_token_adjuster.workspace = true - anyhow.workspace = true async-trait.workspace = true tokio = { workspace = true, features = ["time"] } diff --git a/core/node/state_keeper/src/io/tests/mod.rs b/core/node/state_keeper/src/io/tests/mod.rs index 4ea3460e6e30..5a44bf71ad39 100644 --- a/core/node/state_keeper/src/io/tests/mod.rs +++ b/core/node/state_keeper/src/io/tests/mod.rs @@ -14,13 +14,13 @@ use zksync_node_test_utils::prepare_recovery_snapshot; use zksync_system_constants::KNOWN_CODES_STORAGE_ADDRESS; use zksync_types::{ block::{BlockGasCount, L2BlockHasher}, + bytecode::BytecodeHash, commitment::{L1BatchCommitmentMode, PubdataParams}, fee_model::{BatchFeeInput, PubdataIndependentBatchFeeModelInput}, l2::L2Tx, AccountTreeId, Address, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersion, ProtocolVersionId, StorageKey, TransactionTimeRangeConstraint, H256, U256, }; -use zksync_utils::bytecode::{hash_bytecode, hash_evm_bytecode}; use self::tester::Tester; use crate::{ @@ -438,13 +438,19 @@ async fn processing_dynamic_factory_deps_when_sealing_l2_block() { let static_factory_deps: Vec<_> = (0_u8..10) .map(|byte| { let era_bytecode = vec![byte; 32]; - (hash_bytecode(&era_bytecode), era_bytecode) + ( + BytecodeHash::for_bytecode(&era_bytecode).value(), + era_bytecode, + ) }) .collect(); let dynamic_factory_deps: Vec<_> = (0_u8..10) .map(|byte| { let evm_bytecode = vec![byte; 96]; - (hash_evm_bytecode(&evm_bytecode), evm_bytecode) + ( + BytecodeHash::for_evm_bytecode(&evm_bytecode).value(), + evm_bytecode, + ) }) .collect(); let mut all_factory_deps = static_factory_deps.clone(); diff --git a/core/node/state_keeper/src/updates/l2_block_updates.rs b/core/node/state_keeper/src/updates/l2_block_updates.rs index 6faa098d40a2..d258f8eeac0b 100644 --- a/core/node/state_keeper/src/updates/l2_block_updates.rs +++ b/core/node/state_keeper/src/updates/l2_block_updates.rs @@ -9,10 +9,10 @@ use zksync_multivm::{ }; use zksync_types::{ block::{BlockGasCount, L2BlockHasher}, + bytecode::BytecodeHash, l2_to_l1_log::{SystemL2ToL1Log, UserL2ToL1Log}, L2BlockNumber, ProtocolVersionId, StorageLogWithPreviousValue, Transaction, H256, }; -use zksync_utils::bytecode::hash_bytecode; use crate::metrics::KEEPER_METRICS; @@ -119,7 +119,12 @@ impl L2BlockUpdates { let factory_deps = &tx.execute.factory_deps; let mut tx_factory_deps: HashMap<_, _> = factory_deps .iter() - .map(|bytecode| (hash_bytecode(bytecode), bytecode.clone())) + .map(|bytecode| { + ( + BytecodeHash::for_bytecode(bytecode).value(), + bytecode.clone(), + ) + }) .collect(); // Ensure that *dynamic* factory deps (ones that may be created when executing EVM contracts) // are added into the lookup map as well. diff --git a/core/node/vm_runner/Cargo.toml b/core/node/vm_runner/Cargo.toml index 9c235ad6b291..ef479df17e30 100644 --- a/core/node/vm_runner/Cargo.toml +++ b/core/node/vm_runner/Cargo.toml @@ -17,7 +17,6 @@ zksync_dal.workspace = true zksync_contracts.workspace = true zksync_state.workspace = true zksync_storage.workspace = true -zksync_utils.workspace = true zksync_prover_interface.workspace = true zksync_object_store.workspace = true zksync_vm_executor.workspace = true diff --git a/core/node/vm_runner/src/tests/mod.rs b/core/node/vm_runner/src/tests/mod.rs index d56c70e5808d..97b679695d2a 100644 --- a/core/node/vm_runner/src/tests/mod.rs +++ b/core/node/vm_runner/src/tests/mod.rs @@ -12,6 +12,7 @@ use zksync_node_test_utils::{ use zksync_test_account::Account; use zksync_types::{ block::{L1BatchHeader, L2BlockHasher}, + bytecode::BytecodeHash, fee::Fee, get_intrinsic_constants, h256_to_u256, l2::L2Tx, @@ -20,7 +21,6 @@ use zksync_types::{ AccountTreeId, Address, Execute, L1BatchNumber, L2BlockNumber, ProtocolVersionId, StorageKey, StorageLog, StorageLogKind, StorageValue, H160, H256, L2_BASE_TOKEN_ADDRESS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_vm_interface::{ tracer::ValidationTraces, L1BatchEnv, L2BlockEnv, SystemEnv, TransactionExecutionMetrics, }; @@ -327,7 +327,7 @@ async fn store_l1_batches( header.used_contract_hashes = genesis_params .system_contracts() .iter() - .map(|contract| hash_bytecode(&contract.bytecode)) + .map(|contract| BytecodeHash::for_bytecode(&contract.bytecode).value()) .chain([genesis_params.base_system_contracts().hashes().default_aa]) .chain(genesis_params.base_system_contracts().hashes().evm_emulator) .map(h256_to_u256) diff --git a/core/tests/loadnext/src/sdk/operations/deploy_contract.rs b/core/tests/loadnext/src/sdk/operations/deploy_contract.rs index 67e877ae8efb..cac49559c468 100644 --- a/core/tests/loadnext/src/sdk/operations/deploy_contract.rs +++ b/core/tests/loadnext/src/sdk/operations/deploy_contract.rs @@ -1,8 +1,8 @@ use zksync_eth_signer::EthereumSigner; use zksync_types::{ - l2::L2Tx, transaction_request::PaymasterParams, Execute, Nonce, CONTRACT_DEPLOYER_ADDRESS, U256, + bytecode::BytecodeHash, l2::L2Tx, transaction_request::PaymasterParams, Execute, Nonce, + CONTRACT_DEPLOYER_ADDRESS, U256, }; -use zksync_utils::bytecode::hash_bytecode; use zksync_web3_decl::namespaces::EthNamespaceClient; use crate::sdk::{ @@ -60,7 +60,7 @@ where None => Nonce(self.wallet.get_nonce().await?), }; - let main_contract_hash = hash_bytecode(&bytecode); + let main_contract_hash = BytecodeHash::for_bytecode(&bytecode).value(); let execute_calldata = Execute::encode_deploy_params_create(Default::default(), main_contract_hash, calldata); @@ -141,7 +141,7 @@ where .unwrap_or_default(); let calldata = self.calldata.clone().unwrap_or_default(); - let main_contract_hash = hash_bytecode(&bytecode); + let main_contract_hash = BytecodeHash::for_bytecode(&bytecode).value(); let mut factory_deps = self.factory_deps.clone().unwrap_or_default(); factory_deps.push(bytecode); let l2_tx = L2Tx::new( diff --git a/core/tests/test_account/Cargo.toml b/core/tests/test_account/Cargo.toml index 0dda4f8ac777..9d32d4951f54 100644 --- a/core/tests/test_account/Cargo.toml +++ b/core/tests/test_account/Cargo.toml @@ -13,7 +13,6 @@ categories.workspace = true [dependencies] zksync_types.workspace = true zksync_system_constants.workspace = true -zksync_utils.workspace = true zksync_eth_signer.workspace = true zksync_contracts.workspace = true diff --git a/core/tests/test_account/src/lib.rs b/core/tests/test_account/src/lib.rs index b8c79923a4e8..6c4e57087326 100644 --- a/core/tests/test_account/src/lib.rs +++ b/core/tests/test_account/src/lib.rs @@ -8,11 +8,10 @@ use zksync_system_constants::{ REQUIRED_L1_TO_L2_GAS_PER_PUBDATA_BYTE, }; use zksync_types::{ - abi, address_to_u256, fee::Fee, h256_to_u256, l2::L2Tx, utils::deployed_address_create, - Address, Execute, K256PrivateKey, L2ChainId, Nonce, Transaction, H256, - PRIORITY_OPERATION_L2_TX_TYPE, U256, + abi, address_to_u256, bytecode::BytecodeHash, fee::Fee, l2::L2Tx, + utils::deployed_address_create, Address, Execute, K256PrivateKey, L2ChainId, Nonce, + Transaction, H256, PRIORITY_OPERATION_L2_TX_TYPE, U256, }; -use zksync_utils::bytecode::hash_bytecode; pub const L1_TEST_GAS_PER_PUBDATA_BYTE: u32 = 800; const BASE_FEE: u64 = 2_000_000_000; @@ -124,7 +123,7 @@ impl Account { let contract_function = deployer.function("create").unwrap(); let calldata = calldata.map(ethabi::encode); - let code_hash = hash_bytecode(code); + let code_hash = BytecodeHash::for_bytecode(code).value(); let params = [ Token::FixedBytes(vec![0u8; 32]), Token::FixedBytes(code_hash.0.to_vec()), @@ -185,7 +184,7 @@ impl Account { signature: vec![], factory_deps: factory_deps .iter() - .map(|b| h256_to_u256(hash_bytecode(b))) + .map(|b| BytecodeHash::for_bytecode(b).value_u256()) .collect(), paymaster_input: vec![], reserved_dynamic: vec![], diff --git a/core/tests/vm-benchmark/Cargo.toml b/core/tests/vm-benchmark/Cargo.toml index 892bcf1c1051..bbcc30f961fd 100644 --- a/core/tests/vm-benchmark/Cargo.toml +++ b/core/tests/vm-benchmark/Cargo.toml @@ -9,7 +9,6 @@ publish = false zksync_contracts.workspace = true zksync_multivm.workspace = true zksync_types.workspace = true -zksync_utils.workspace = true zksync_vlog.workspace = true zksync_vm2.workspace = true diff --git a/core/tests/vm-benchmark/src/transaction.rs b/core/tests/vm-benchmark/src/transaction.rs index c625018fb9bf..e5be25545377 100644 --- a/core/tests/vm-benchmark/src/transaction.rs +++ b/core/tests/vm-benchmark/src/transaction.rs @@ -3,6 +3,7 @@ pub use zksync_contracts::test_contracts::LoadnextContractExecutionParams as Loa use zksync_contracts::{deployer_contract, TestContract}; use zksync_multivm::utils::get_max_gas_per_pubdata_byte; use zksync_types::{ + bytecode::BytecodeHash, ethabi::{encode, Token}, fee::Fee, l2::L2Tx, @@ -10,7 +11,6 @@ use zksync_types::{ Address, K256PrivateKey, L2ChainId, Nonce, ProtocolVersionId, Transaction, CONTRACT_DEPLOYER_ADDRESS, H256, U256, }; -use zksync_utils::bytecode::hash_bytecode; const LOAD_TEST_MAX_READS: usize = 3000; @@ -37,7 +37,7 @@ pub fn get_deploy_tx_with_gas_limit(code: &[u8], gas_limit: u32, nonce: u32) -> salt[28..32].copy_from_slice(&nonce.to_be_bytes()); let params = [ Token::FixedBytes(salt), - Token::FixedBytes(hash_bytecode(code).0.to_vec()), + Token::FixedBytes(BytecodeHash::for_bytecode(code).value().0.to_vec()), Token::Bytes([].to_vec()), ]; let calldata = CREATE_FUNCTION_SIGNATURE @@ -96,7 +96,12 @@ pub fn get_load_test_deploy_tx() -> Transaction { let calldata = [Token::Uint(LOAD_TEST_MAX_READS.into())]; let params = [ Token::FixedBytes(vec![0_u8; 32]), - Token::FixedBytes(hash_bytecode(&LOAD_TEST_CONTRACT.bytecode).0.to_vec()), + Token::FixedBytes( + BytecodeHash::for_bytecode(&LOAD_TEST_CONTRACT.bytecode) + .value() + .0 + .to_vec(), + ), Token::Bytes(encode(&calldata)), ]; let create_calldata = CREATE_FUNCTION_SIGNATURE diff --git a/core/tests/vm-benchmark/src/vm.rs b/core/tests/vm-benchmark/src/vm.rs index e082b9c24da2..a855dfafb04e 100644 --- a/core/tests/vm-benchmark/src/vm.rs +++ b/core/tests/vm-benchmark/src/vm.rs @@ -18,14 +18,13 @@ use zksync_types::{ utils::storage_key_for_eth_balance, L1BatchNumber, L2BlockNumber, L2ChainId, ProtocolVersionId, Transaction, }; -use zksync_utils::bytecode::hash_bytecode; use crate::{instruction_counter::InstructionCounter, transaction::PRIVATE_KEY}; static SYSTEM_CONTRACTS: Lazy = Lazy::new(BaseSystemContracts::load_from_disk); static STORAGE: Lazy = Lazy::new(|| { - let mut storage = InMemoryStorage::with_system_contracts(hash_bytecode); + let mut storage = InMemoryStorage::with_system_contracts(); // Give `PRIVATE_KEY` some money let balance = U256::from(10u32).pow(U256::from(32)); //10^32 wei let key = storage_key_for_eth_balance(&PRIVATE_KEY.address()); diff --git a/prover/Cargo.lock b/prover/Cargo.lock index 3250c99deda6..a60f77d44dd7 100644 --- a/prover/Cargo.lock +++ b/prover/Cargo.lock @@ -7816,6 +7816,7 @@ version = "0.1.0" dependencies = [ "anyhow", "chrono", + "const-decoder 0.4.0", "ethabi", "hex", "num_enum 0.7.2", @@ -7823,6 +7824,7 @@ dependencies = [ "serde", "serde_json", "serde_with", + "sha2 0.10.8", "strum", "thiserror", "tiny-keccak 2.0.2", @@ -7994,11 +7996,11 @@ name = "zksync_contracts" version = "0.1.0" dependencies = [ "envy", - "ethabi", "hex", "once_cell", "serde", "serde_json", + "zksync_basic_types", "zksync_utils", ] @@ -8076,7 +8078,6 @@ dependencies = [ "zksync_protobuf_build", "zksync_system_constants", "zksync_types", - "zksync_utils", "zksync_vm_interface", ] @@ -8230,7 +8231,6 @@ dependencies = [ "zksync_mini_merkle_tree", "zksync_system_constants", "zksync_types", - "zksync_utils", "zksync_vm2", "zksync_vm_interface", ] @@ -8622,7 +8622,6 @@ dependencies = [ "zksync_protobuf", "zksync_protobuf_build", "zksync_system_constants", - "zksync_utils", ] [[package]] @@ -8630,16 +8629,12 @@ name = "zksync_utils" version = "0.1.0" dependencies = [ "anyhow", - "const-decoder 0.4.0", "futures 0.3.30", "once_cell", "reqwest 0.12.5", "serde_json", - "thiserror", "tokio", "tracing", - "zk_evm 0.133.0", - "zksync_basic_types", "zksync_vlog", ] diff --git a/zkstack_cli/Cargo.lock b/zkstack_cli/Cargo.lock index a582fff958f5..2206a1052f59 100644 --- a/zkstack_cli/Cargo.lock +++ b/zkstack_cli/Cargo.lock @@ -297,12 +297,6 @@ dependencies = [ "windows-targets 0.52.6", ] -[[package]] -name = "base16ct" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "349a06037c7bf932dd7e7d1f653678b2038b9ad46a74102f1fc7bd7872678cce" - [[package]] name = "base16ct" version = "0.2.0" @@ -659,7 +653,7 @@ dependencies = [ "coins-core", "digest", "hmac", - "k256 0.13.4", + "k256", "serde", "sha2", "thiserror", @@ -938,18 +932,6 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" -[[package]] -name = "crypto-bigint" -version = "0.4.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef2b4b23cddf68b89b8f8069890e8c270d54e2d5fe1b143820234805e4cb17ef" -dependencies = [ - "generic-array", - "rand_core", - "subtle", - "zeroize", -] - [[package]] name = "crypto-bigint" version = "0.5.5" @@ -1069,16 +1051,6 @@ dependencies = [ "uuid 1.10.0", ] -[[package]] -name = "der" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" -dependencies = [ - "const-oid", - "zeroize", -] - [[package]] name = "der" version = "0.7.9" @@ -1221,30 +1193,18 @@ version = "1.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" -[[package]] -name = "ecdsa" -version = "0.14.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "413301934810f597c1d19ca71c8710e99a3f1ba28a0d2ebc01551a2daeea3c5c" -dependencies = [ - "der 0.6.1", - "elliptic-curve 0.12.3", - "rfc6979 0.3.1", - "signature 1.6.4", -] - [[package]] name = "ecdsa" version = "0.16.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ee27f32b5c5292967d2d4a9d7f1e0b0aed2c15daded5a60300e4abb9d8020bca" dependencies = [ - "der 0.7.9", + "der", "digest", - "elliptic-curve 0.13.8", - "rfc6979 0.4.0", - "signature 2.2.0", - "spki 0.7.3", + "elliptic-curve", + "rfc6979", + "signature", + "spki", ] [[package]] @@ -1253,8 +1213,8 @@ version = "2.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "115531babc129696a58c64a4fef0a8bf9e9698629fb97e9e40767d235cfbcd53" dependencies = [ - "pkcs8 0.10.2", - "signature 2.2.0", + "pkcs8", + "signature", ] [[package]] @@ -1281,41 +1241,21 @@ dependencies = [ "serde", ] -[[package]] -name = "elliptic-curve" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7bb888ab5300a19b8e5bceef25ac745ad065f3c9f7efc6de1b91958110891d3" -dependencies = [ - "base16ct 0.1.1", - "crypto-bigint 0.4.9", - "der 0.6.1", - "digest", - "ff 0.12.1", - "generic-array", - "group 0.12.1", - "pkcs8 0.9.0", - "rand_core", - "sec1 0.3.0", - "subtle", - "zeroize", -] - [[package]] name = "elliptic-curve" version = "0.13.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b5e6043086bf7973472e0c7dff2142ea0b680d30e18d9cc40f267efbf222bd47" dependencies = [ - "base16ct 0.2.0", - "crypto-bigint 0.5.5", + "base16ct", + "crypto-bigint", "digest", - "ff 0.13.0", + "ff", "generic-array", - "group 0.13.0", - "pkcs8 0.10.2", + "group", + "pkcs8", "rand_core", - "sec1 0.7.3", + "sec1", "subtle", "zeroize", ] @@ -1362,7 +1302,7 @@ dependencies = [ "base64 0.21.7", "bytes", "hex", - "k256 0.13.4", + "k256", "log", "rand", "rlp", @@ -1587,11 +1527,11 @@ dependencies = [ "cargo_metadata", "chrono", "const-hex", - "elliptic-curve 0.13.8", + "elliptic-curve", "ethabi", "generic-array", - "k256 0.13.4", - "num_enum 0.7.3", + "k256", + "num_enum", "once_cell", "open-fastrlp", "rand", @@ -1696,7 +1636,7 @@ dependencies = [ "coins-bip32", "coins-bip39", "const-hex", - "elliptic-curve 0.13.8", + "elliptic-curve", "eth-keystore", "ethers-core", "rand", @@ -1775,16 +1715,6 @@ version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8c02a5121d4ea3eb16a80748c74f5549a5665e4c21333c6098f283870fbdea6" -[[package]] -name = "ff" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d013fc25338cc558c5c2cfbad646908fb23591e2404481826742b651c9af7160" -dependencies = [ - "rand_core", - "subtle", -] - [[package]] name = "ff" version = "0.13.0" @@ -2082,24 +2012,13 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "group" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dfbfb3a6cfbd390d5c9564ab283a0349b9b9fcd46a706c1eb10e0db70bfbac7" -dependencies = [ - "ff 0.12.1", - "rand_core", - "subtle", -] - [[package]] name = "group" version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f0f9ef7462f7c099f518d754361858f86d8a07af53ba9af0fe635bbccb151a63" dependencies = [ - "ff 0.13.0", + "ff", "rand_core", "subtle", ] @@ -2679,18 +2598,6 @@ dependencies = [ "simple_asn1", ] -[[package]] -name = "k256" -version = "0.11.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72c1e0b51e7ec0a97369623508396067a486bd0cbed95a2659a4b863d28cfc8b" -dependencies = [ - "cfg-if", - "ecdsa 0.14.8", - "elliptic-curve 0.12.3", - "sha2", -] - [[package]] name = "k256" version = "0.13.4" @@ -2698,11 +2605,11 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f6e3919bbaa2945715f0bb6d3934a173d1e9a59ac23767fbaaef277265a7411b" dependencies = [ "cfg-if", - "ecdsa 0.16.9", - "elliptic-curve 0.13.8", + "ecdsa", + "elliptic-curve", "once_cell", "sha2", - "signature 2.2.0", + "signature", ] [[package]] @@ -3130,34 +3037,13 @@ dependencies = [ "libc", ] -[[package]] -name = "num_enum" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a015b430d3c108a207fd776d2e2196aaf8b1cf8cf93253e3a097ff3085076a1" -dependencies = [ - "num_enum_derive 0.6.1", -] - [[package]] name = "num_enum" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4e613fc340b2220f734a8595782c551f1250e969d87d3be1ae0579e8d4065179" dependencies = [ - "num_enum_derive 0.7.3", -] - -[[package]] -name = "num_enum_derive" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96667db765a921f7b295ffee8b60472b686a51d4f21c2ee4ffdb94c7013b65a6" -dependencies = [ - "proc-macro-crate 1.3.1", - "proc-macro2", - "quote", - "syn 2.0.79", + "num_enum_derive", ] [[package]] @@ -3166,7 +3052,7 @@ version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af1844ef2428cc3e1cb900be36181049ef3d3193c63e43026cfe202983b27a56" dependencies = [ - "proc-macro-crate 3.2.0", + "proc-macro-crate", "proc-macro2", "quote", "syn 2.0.79", @@ -3412,7 +3298,7 @@ version = "3.6.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d830939c76d294956402033aee57a6da7b438f2294eb94864c37b0569053a42c" dependencies = [ - "proc-macro-crate 3.2.0", + "proc-macro-crate", "proc-macro2", "quote", "syn 1.0.109", @@ -3627,19 +3513,9 @@ version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c8ffb9f10fa047879315e6625af03c164b16962a5368d724ed16323b68ace47f" dependencies = [ - "der 0.7.9", - "pkcs8 0.10.2", - "spki 0.7.3", -] - -[[package]] -name = "pkcs8" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9eca2c590a5f85da82668fa685c09ce2888b9430e83299debf1f34b65fd4a4ba" -dependencies = [ - "der 0.6.1", - "spki 0.6.0", + "der", + "pkcs8", + "spki", ] [[package]] @@ -3648,8 +3524,8 @@ version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" dependencies = [ - "der 0.7.9", - "spki 0.7.3", + "der", + "spki", ] [[package]] @@ -3719,23 +3595,13 @@ dependencies = [ "uint", ] -[[package]] -name = "proc-macro-crate" -version = "1.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f4c021e1093a56626774e81216a4ce732a735e5bad4868a03f3ed65ca0c3919" -dependencies = [ - "once_cell", - "toml_edit 0.19.15", -] - [[package]] name = "proc-macro-crate" version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" dependencies = [ - "toml_edit 0.22.22", + "toml_edit", ] [[package]] @@ -4140,17 +4006,6 @@ dependencies = [ "windows-registry", ] -[[package]] -name = "rfc6979" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7743f17af12fa0b03b803ba12cd6a8d9483a587e89c69445e3909655c0b9fabb" -dependencies = [ - "crypto-bigint 0.4.9", - "hmac", - "zeroize", -] - [[package]] name = "rfc6979" version = "0.4.0" @@ -4234,10 +4089,10 @@ dependencies = [ "num-integer", "num-traits", "pkcs1", - "pkcs8 0.10.2", + "pkcs8", "rand_core", - "signature 2.2.0", - "spki 0.7.3", + "signature", + "spki", "subtle", "zeroize", ] @@ -4400,7 +4255,7 @@ version = "2.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d35494501194174bda522a32605929eefc9ecf7e0a326c26db1fdd85881eb62" dependencies = [ - "proc-macro-crate 3.2.0", + "proc-macro-crate", "proc-macro2", "quote", "syn 1.0.109", @@ -4443,30 +4298,16 @@ dependencies = [ "untrusted 0.9.0", ] -[[package]] -name = "sec1" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3be24c1842290c45df0a7bf069e0c268a747ad05a192f2fd7dcfdbc1cba40928" -dependencies = [ - "base16ct 0.1.1", - "der 0.6.1", - "generic-array", - "pkcs8 0.9.0", - "subtle", - "zeroize", -] - [[package]] name = "sec1" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3e97a565f76233a6003f9f5c54be1d9c5bdfa3eccfb189469f11ec4901c47dc" dependencies = [ - "base16ct 0.2.0", - "der 0.7.9", + "base16ct", + "der", "generic-array", - "pkcs8 0.10.2", + "pkcs8", "subtle", "zeroize", ] @@ -4771,17 +4612,6 @@ dependencies = [ "digest", ] -[[package]] -name = "sha2_ce" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eca2daa77078f4ddff27e75c4bf59e4c2697525f56dbb3c842d34a5d1f2b04a2" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest", -] - [[package]] name = "sha3" version = "0.10.8" @@ -4792,16 +4622,6 @@ dependencies = [ "keccak", ] -[[package]] -name = "sha3_ce" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34c9a08202c50378d8a07a5f458193a5f542d2828ac6640263dbc0c2533ea25e" -dependencies = [ - "digest", - "keccak", -] - [[package]] name = "sharded-slab" version = "0.1.7" @@ -4826,16 +4646,6 @@ dependencies = [ "libc", ] -[[package]] -name = "signature" -version = "1.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74233d3b3b2f6d4b006dc19dee745e73e2a6bfb6f93607cd3b02bd5b00797d7c" -dependencies = [ - "digest", - "rand_core", -] - [[package]] name = "signature" version = "2.2.0" @@ -4953,16 +4763,6 @@ dependencies = [ "lock_api", ] -[[package]] -name = "spki" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67cf02bbac7a337dc36e4f5a693db6c21e7863f45070f7064577eb4367a3212b" -dependencies = [ - "base64ct", - "der 0.6.1", -] - [[package]] name = "spki" version = "0.7.3" @@ -4970,7 +4770,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d91ed6c858b01f942cd56b37a94b3e0a1798290327d1236e4d9cf4eaca44d29d" dependencies = [ "base64ct", - "der 0.7.9", + "der", ] [[package]] @@ -5700,7 +5500,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "toml_edit 0.22.22", + "toml_edit", ] [[package]] @@ -5712,17 +5512,6 @@ dependencies = [ "serde", ] -[[package]] -name = "toml_edit" -version = "0.19.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" -dependencies = [ - "indexmap 2.6.0", - "toml_datetime", - "winnow 0.5.40", -] - [[package]] name = "toml_edit" version = "0.22.22" @@ -5733,7 +5522,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.20", + "winnow", ] [[package]] @@ -6530,15 +6319,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" -[[package]] -name = "winnow" -version = "0.5.40" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f593a95398737aeed53e489c785df13f3618e41dbcd6718c6addbf1395aa6876" -dependencies = [ - "memchr", -] - [[package]] name = "winnow" version = "0.6.20" @@ -6674,50 +6454,6 @@ dependencies = [ "zstd", ] -[[package]] -name = "zk_evm" -version = "0.133.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9af08e9284686a1b0c89ec4931eb915ac0729367f1247abd06164874fe738106" -dependencies = [ - "anyhow", - "lazy_static", - "num", - "serde", - "serde_json", - "static_assertions", - "zk_evm_abstractions", - "zkevm_opcode_defs", -] - -[[package]] -name = "zk_evm_abstractions" -version = "0.140.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be696258861eba4e6625a5665084b2266720bb67f4ba69819469700ac5c6a401" -dependencies = [ - "anyhow", - "num_enum 0.6.1", - "serde", - "static_assertions", - "zkevm_opcode_defs", -] - -[[package]] -name = "zkevm_opcode_defs" -version = "0.132.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0769f7b27d8fb06e715da3290c575cac5d04d10a557faef180e847afce50ac4" -dependencies = [ - "bitflags 2.6.0", - "blake2", - "ethereum-types", - "k256 0.11.6", - "lazy_static", - "sha2_ce", - "sha3_ce", -] - [[package]] name = "zkstack" version = "0.1.0" @@ -6767,13 +6503,15 @@ version = "0.1.0" dependencies = [ "anyhow", "chrono", + "const-decoder", "ethabi", "hex", - "num_enum 0.7.3", + "num_enum", "secrecy", "serde", "serde_json", "serde_with", + "sha2", "strum", "thiserror", "tiny-keccak", @@ -6822,9 +6560,9 @@ dependencies = [ "anyhow", "blst", "ed25519-dalek", - "elliptic-curve 0.13.8", + "elliptic-curve", "hex", - "k256 0.13.4", + "k256", "num-bigint", "num-traits", "rand", @@ -6873,11 +6611,11 @@ name = "zksync_contracts" version = "0.1.0" dependencies = [ "envy", - "ethabi", "hex", "once_cell", "serde", "serde_json", + "zksync_basic_types", "zksync_utils", ] @@ -6983,7 +6721,7 @@ dependencies = [ "hex", "itertools 0.10.5", "num", - "num_enum 0.7.3", + "num_enum", "once_cell", "prost 0.12.6", "rlp", @@ -7000,7 +6738,6 @@ dependencies = [ "zksync_protobuf", "zksync_protobuf_build", "zksync_system_constants", - "zksync_utils", ] [[package]] @@ -7008,16 +6745,12 @@ name = "zksync_utils" version = "0.1.0" dependencies = [ "anyhow", - "const-decoder", "futures", "once_cell", "reqwest 0.12.8", "serde_json", - "thiserror", "tokio", "tracing", - "zk_evm", - "zksync_basic_types", "zksync_vlog", ] From 98803effbca2d285cf600dbaeb78818f56b25951 Mon Sep 17 00:00:00 2001 From: D025 Date: Thu, 14 Nov 2024 17:20:09 +0200 Subject: [PATCH 5/7] ci: fix reusable workflow for build core (#3268) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Fix new version of reusable workflow ## Why ❔ It was broke publishing releases ## Checklist - [ ] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [ ] Tests for the changes have been added / updated. - [ ] Documentation comments have been added / updated. - [ ] Code has been formatted via `zkstack dev fmt` and `zkstack dev lint`. --- .github/workflows/ci.yml | 2 +- .github/workflows/new-build-core-template.yml | 31 +++++++------------ .../workflows/new-build-prover-template.yml | 10 +++++- .../new-build-witness-generator-template.yml | 7 +++++ .github/workflows/release-test-stage.yml | 12 ++++--- 5 files changed, 36 insertions(+), 26 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 2f29fe98f0e6..a4370457fecb 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -20,7 +20,6 @@ jobs: outputs: core: ${{ steps.changed-files.outputs.core_any_changed }} prover: ${{ steps.changed-files.outputs.prover_any_changed }} - zkstack_cli: ${{ steps.changed-files.outputs.zkstack_cli_any_changed }} docs: ${{ steps.changed-files.outputs.docs_any_changed }} all: ${{ steps.changed-files.outputs.all_any_changed }} steps: @@ -178,6 +177,7 @@ jobs: image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }}-avx512 action: "build" WITNESS_GENERATOR_RUST_FLAGS: "-Ctarget_feature=+avx512bw,+avx512cd,+avx512dq,+avx512f,+avx512vl" + ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/new-build-core-template.yml b/.github/workflows/new-build-core-template.yml index 350d689c4572..ab3854733e62 100644 --- a/.github/workflows/new-build-core-template.yml +++ b/.github/workflows/new-build-core-template.yml @@ -219,35 +219,28 @@ jobs: docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} gcloud auth configure-docker us-docker.pkg.dev -q - - name: Build and push + - name: Build docker image uses: docker/build-push-action@5cd11c3a4ced054e52742c5fd54dca954e0edd85 # v6.7.0 with: context: . - push: ${{ inputs.action == 'push' }} + load: true + platforms: ${{ matrix.platforms }} file: docker/${{ matrix.components }}/Dockerfile build-args: | SCCACHE_GCS_BUCKET=matterlabs-infra-sccache-storage SCCACHE_GCS_SERVICE_ACCOUNT=gha-ci-runners@matterlabs-infra.iam.gserviceaccount.com SCCACHE_GCS_RW_MODE=READ_WRITE RUSTC_WRAPPER=sccache - tags: | - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:latest - matterlabs/${{ matrix.components }}:latest - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:latest2.0 - matterlabs/${{ matrix.components }}:latest2.0 - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA }} - matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} + tags: | + us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + - name: Push docker image + if: ${{ inputs.action == 'push' }} + run: | + docker push us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + docker push matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + create_manifest: name: Create release manifest runs-on: matterlabs-ci-runner @@ -269,13 +262,11 @@ jobs: - uses: actions/checkout@a5ac7e51b41094c92402da3b24376905380afc29 # v4 - name: login to Docker registries - shell: bash run: | docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} gcloud auth configure-docker us-docker.pkg.dev -q - name: Create Docker manifest - shell: bash run: | docker_repositories=("matterlabs/${{ matrix.component.name }}" "us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.component.name }}") platforms=${{ matrix.component.platform }} diff --git a/.github/workflows/new-build-prover-template.yml b/.github/workflows/new-build-prover-template.yml index 046711d679e8..939c11db1819 100644 --- a/.github/workflows/new-build-prover-template.yml +++ b/.github/workflows/new-build-prover-template.yml @@ -152,18 +152,26 @@ jobs: uses: docker/build-push-action@5cd11c3a4ced054e52742c5fd54dca954e0edd85 # v6.7.0 with: context: . - push: ${{ inputs.action == 'push' }} + load: true build-args: | CUDA_ARCH=${{ inputs.CUDA_ARCH }} SCCACHE_GCS_BUCKET=matterlabs-infra-sccache-storage SCCACHE_GCS_SERVICE_ACCOUNT=gha-ci-runners@matterlabs-infra.iam.gserviceaccount.com SCCACHE_GCS_RW_MODE=READ_WRITE RUSTC_WRAPPER=sccache + PROTOCOL_VERSION=${{ env.PROTOCOL_VERSION }} + ERA_BELLMAN_CUDA_RELEASE=${{ inputs.ERA_BELLMAN_CUDA_RELEASE }} file: docker/${{ matrix.components }}/Dockerfile tags: | us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.PROTOCOL_VERSION }}-${{ env.IMAGE_TAG_SHA_TS }} matterlabs/${{ matrix.components }}:2.0-${{ env.PROTOCOL_VERSION }}-${{ env.IMAGE_TAG_SHA_TS }} + - name: Push docker image + if: ${{ inputs.action == 'push' }} + run: | + docker push us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.PROTOCOL_VERSION }}-${{ env.IMAGE_TAG_SHA_TS }} + docker push matterlabs/${{ matrix.components }}:2.0-${{ env.PROTOCOL_VERSION }}-${{ env.IMAGE_TAG_SHA_TS }} + copy-images: name: Copy images between docker registries needs: [build-images, get-protocol-version] diff --git a/.github/workflows/new-build-witness-generator-template.yml b/.github/workflows/new-build-witness-generator-template.yml index 2f1fc0b2dd86..2782e6fef6fa 100644 --- a/.github/workflows/new-build-witness-generator-template.yml +++ b/.github/workflows/new-build-witness-generator-template.yml @@ -9,6 +9,10 @@ on: description: "DOCKERHUB_TOKEN" required: true inputs: + ERA_BELLMAN_CUDA_RELEASE: + description: "ERA_BELLMAN_CUDA_RELEASE" + type: string + required: true image_tag_suffix: description: "Optional suffix to override tag name generation" type: string @@ -127,6 +131,9 @@ jobs: SCCACHE_GCS_SERVICE_ACCOUNT=gha-ci-runners@matterlabs-infra.iam.gserviceaccount.com SCCACHE_GCS_RW_MODE=READ_WRITE RUSTC_WRAPPER=sccache + PROTOCOL_VERSION=${{ env.PROTOCOL_VERSION }} + ERA_BELLMAN_CUDA_RELEASE=${{ inputs.ERA_BELLMAN_CUDA_RELEASE }} + RUST_FLAGS=${{ inputs.WITNESS_GENERATOR_RUST_FLAGS }} file: docker/${{ matrix.components }}/Dockerfile tags: | us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.PROTOCOL_VERSION }}-${{ env.IMAGE_TAG_SHA_TS }} diff --git a/.github/workflows/release-test-stage.yml b/.github/workflows/release-test-stage.yml index 18708420dab0..eb75ab179b8e 100644 --- a/.github/workflows/release-test-stage.yml +++ b/.github/workflows/release-test-stage.yml @@ -61,10 +61,11 @@ jobs: build-push-core-images: name: Build and push images needs: [setup, changed_files] - uses: ./.github/workflows/build-core-template.yml + uses: ./.github/workflows/new-build-core-template.yml if: needs.changed_files.outputs.core == 'true' || needs.changed_files.outputs.all == 'true' with: image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} + action: "push" secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} @@ -84,10 +85,11 @@ jobs: build-push-contract-verifier: name: Build and push images needs: [setup, changed_files] - uses: ./.github/workflows/build-contract-verifier-template.yml + uses: ./.github/workflows/new-build-contract-verifier-template.yml if: needs.changed_files.outputs.core == 'true' || needs.changed_files.outputs.all == 'true' with: image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} + action: "push" secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} @@ -95,12 +97,13 @@ jobs: build-push-prover-images: name: Build and push images needs: [setup, changed_files] - uses: ./.github/workflows/build-prover-template.yml + uses: ./.github/workflows/new-build-prover-template.yml if: needs.changed_files.outputs.prover == 'true' || needs.changed_files.outputs.all == 'true' with: image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }} ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} CUDA_ARCH: "60;70;75;80;89" + action: "push" secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} @@ -108,13 +111,14 @@ jobs: build-push-witness-generator-image-avx512: name: Build and push prover images with avx512 instructions needs: [setup, changed_files] - uses: ./.github/workflows/build-witness-generator-template.yml + uses: ./.github/workflows/new-build-witness-generator-template.yml if: needs.changed_files.outputs.prover == 'true' || needs.changed_files.outputs.all == 'true' with: image_tag_suffix: ${{ needs.setup.outputs.image_tag_suffix }}-avx512 ERA_BELLMAN_CUDA_RELEASE: ${{ vars.ERA_BELLMAN_CUDA_RELEASE }} CUDA_ARCH: "60;70;75;80;89" WITNESS_GENERATOR_RUST_FLAGS: "-Ctarget_feature=+avx512bw,+avx512cd,+avx512dq,+avx512f,+avx512vl " + action: "push" secrets: DOCKERHUB_USER: ${{ secrets.DOCKERHUB_USER }} DOCKERHUB_TOKEN: ${{ secrets.DOCKERHUB_TOKEN }} From 684465196ba71f59c81c9c78bc1f6db044574f8c Mon Sep 17 00:00:00 2001 From: D025 Date: Thu, 14 Nov 2024 19:03:04 +0200 Subject: [PATCH 6/7] ci: fix contract verifier workflow (#3283) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Fix for contract verifier workflow ## Why ❔ It's broken ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zkstack dev fmt` and `zkstack dev lint`. --- .../new-build-contract-verifier-template.yml | 26 +++++++------------ 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/.github/workflows/new-build-contract-verifier-template.yml b/.github/workflows/new-build-contract-verifier-template.yml index 7e48968a65c1..7d75f81fb73c 100644 --- a/.github/workflows/new-build-contract-verifier-template.yml +++ b/.github/workflows/new-build-contract-verifier-template.yml @@ -212,7 +212,8 @@ jobs: uses: docker/build-push-action@5cd11c3a4ced054e52742c5fd54dca954e0edd85 # v6.7.0 with: context: . - push: ${{ inputs.action == 'push' }} + load: true + platforms: ${{ matrix.platforms }} file: docker/${{ matrix.components }}/Dockerfile build-args: | SCCACHE_GCS_BUCKET=matterlabs-infra-sccache-storage @@ -220,23 +221,16 @@ jobs: SCCACHE_GCS_RW_MODE=READ_WRITE RUSTC_WRAPPER=sccache tags: | - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:latest - matterlabs/${{ matrix.components }}:latest - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:latest2.0 - matterlabs/${{ matrix.components }}:latest2.0 - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA }} - matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }} - matterlabs/${{ matrix.components }}:2.0-${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} - us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }} + us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + - name: Push docker image + if: ${{ inputs.action == 'push' }} + run: | + docker push us-docker.pkg.dev/matterlabs-infra/matterlabs-docker/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + docker push matterlabs/${{ matrix.components }}:${{ env.IMAGE_TAG_SHA_TS }}-${{ env.PLATFORM }} + + create_manifest: name: Create release manifest runs-on: matterlabs-ci-runner From bc00c4a44a212def3cc25567f3b271530d76b6a4 Mon Sep 17 00:00:00 2001 From: Yury Akudovich Date: Thu, 14 Nov 2024 18:55:21 +0100 Subject: [PATCH 7/7] feat: Add min_replicas for SimpleScaler, apply_min_to_namespace config (#3282) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What ❔ Breaking change: config option `min_provers` is a number now! Add `apply_min_to_namespace` to specify which namespace is primary now, `min_*` is applied only to primary namespace. Add `min_replicas` for SimpleScaler. ## Why ❔ To allow Witness Generators and Proof Compressors to always run, if needed. This will remove pod start delay of about 2 minutes when new job appears in the queue. ## Checklist - [x] PR title corresponds to the body of PR (we generate changelog entries from PRs). - [x] Tests for the changes have been added / updated. - [x] Documentation comments have been added / updated. - [x] Code has been formatted via `zkstack dev fmt` and `zkstack dev lint`. ref ZKD-1855 --- prover/crates/bin/prover_autoscaler/README.md | 9 +++-- .../bin/prover_autoscaler/src/config.rs | 10 ++++-- .../prover_autoscaler/src/global/scaler.rs | 36 +++++++++++++++---- 3 files changed, 43 insertions(+), 12 deletions(-) diff --git a/prover/crates/bin/prover_autoscaler/README.md b/prover/crates/bin/prover_autoscaler/README.md index 6a0ff84f88f2..3d1a9afe5a30 100644 --- a/prover/crates/bin/prover_autoscaler/README.md +++ b/prover/crates/bin/prover_autoscaler/README.md @@ -152,7 +152,8 @@ agent_config: - `protocol_versions` is a map namespaces to protocol version it processes. Should correspond binary versions running there! - `cluster_priorities` is a map cluster name to priority, the lower will be used first. -- `min_provers` is a map namespace to minimum number of provers to run even if the queue is empty. +- `apply_min_to_namespace` specifies current primary namespace to run min number of provers in it. +- `min_provers` is a minimum number of provers to run even if the queue is empty. Default: 0. - `max_provers` is a map of cluster name to map GPU type to maximum number of provers. - `prover_speed` is a map GPU to speed divider. Default: 500. - `long_pending_duration` is time after a pending pod considered long pending and will be relocated to different @@ -160,6 +161,7 @@ agent_config: - `scaler_targets` subsection is a list of Simple targets: - `queue_report_field` is name of corresponding queue report section. See example for possible options. - `deployment` is name of a Deployment to scale. + - `min_replicas` is a minimum number of replicas to run even if the queue is empty. Default: 0. - `max_replicas` is a map of cluster name to maximum number of replicas. - `speed` is a divider for corresponding queue. @@ -182,8 +184,8 @@ scaler_config: cluster1: 0 cluster2: 100 cluster3: 200 - min_provers: - prover-new: 0 + apply_min_to_namespace: prover-new + min_provers: 1 max_provers: cluster1: L4: 1 @@ -201,6 +203,7 @@ scaler_config: scaler_targets: - queue_report_field: basic_witness_jobs deployment: witness-generator-basic-fri + min_replicas: 1 max_replicas: cluster1: 10 cluster2: 20 diff --git a/prover/crates/bin/prover_autoscaler/src/config.rs b/prover/crates/bin/prover_autoscaler/src/config.rs index 777ffe89fc91..ff3bccf79c83 100644 --- a/prover/crates/bin/prover_autoscaler/src/config.rs +++ b/prover/crates/bin/prover_autoscaler/src/config.rs @@ -59,8 +59,11 @@ pub struct ProverAutoscalerScalerConfig { pub prover_speed: HashMap, /// Maximum number of provers which can be run per cluster/GPU. pub max_provers: HashMap>, - /// Minimum number of provers per namespace. - pub min_provers: HashMap, + /// Minimum number of provers globally. + #[serde(default)] + pub min_provers: u32, + /// Name of primary namespace, all min numbers are applied to it. + pub apply_min_to_namespace: Option, /// Duration after which pending pod considered long pending. #[serde( with = "humantime_serde", @@ -132,6 +135,9 @@ pub enum QueueReportFields { pub struct ScalerTarget { pub queue_report_field: QueueReportFields, pub deployment: String, + /// Min replicas globally. + #[serde(default)] + pub min_replicas: usize, /// Max replicas per cluster. pub max_replicas: HashMap, /// The queue will be divided by the speed and rounded up to get number of replicas. diff --git a/prover/crates/bin/prover_autoscaler/src/global/scaler.rs b/prover/crates/bin/prover_autoscaler/src/global/scaler.rs index 829b95dd7514..074da383b740 100644 --- a/prover/crates/bin/prover_autoscaler/src/global/scaler.rs +++ b/prover/crates/bin/prover_autoscaler/src/global/scaler.rs @@ -73,7 +73,8 @@ pub struct Scaler { pub struct GpuScaler { /// Which cluster to use first. cluster_priorities: HashMap, - min_provers: HashMap, + apply_min_to_namespace: Option, + min_provers: u32, max_provers: HashMap>, prover_speed: HashMap, long_pending_duration: chrono::Duration, @@ -84,6 +85,8 @@ pub struct SimpleScaler { deployment: String, /// Which cluster to use first. cluster_priorities: HashMap, + apply_min_to_namespace: Option, + min_replicas: usize, max_replicas: HashMap, speed: usize, long_pending_duration: chrono::Duration, @@ -126,6 +129,7 @@ impl Scaler { simple_scalers.push(SimpleScaler::new( c, config.cluster_priorities.clone(), + config.apply_min_to_namespace.clone(), chrono::Duration::seconds(config.long_pending_duration.as_secs() as i64), )) } @@ -144,6 +148,7 @@ impl GpuScaler { pub fn new(config: ProverAutoscalerScalerConfig) -> Self { Self { cluster_priorities: config.cluster_priorities, + apply_min_to_namespace: config.apply_min_to_namespace, min_provers: config.min_provers, max_provers: config.max_provers, prover_speed: config.prover_speed, @@ -287,10 +292,12 @@ impl GpuScaler { // Increase queue size, if it's too small, to make sure that required min_provers are // running. - let queue: u64 = self.min_provers.get(namespace).map_or(queue, |min| { + let queue: u64 = if self.apply_min_to_namespace.as_deref() == Some(namespace.as_str()) { self.normalize_queue(Gpu::L4, queue) - .max(self.provers_to_speed(Gpu::L4, *min)) - }); + .max(self.provers_to_speed(Gpu::L4, self.min_provers)) + } else { + queue + }; let mut total: i64 = 0; let mut provers: HashMap = HashMap::new(); @@ -424,12 +431,15 @@ impl SimpleScaler { pub fn new( config: &ScalerTarget, cluster_priorities: HashMap, + apply_min_to_namespace: Option, long_pending_duration: chrono::Duration, ) -> Self { Self { queue_report_field: config.queue_report_field, deployment: config.deployment.clone(), cluster_priorities, + apply_min_to_namespace, + min_replicas: config.min_replicas, max_replicas: config.max_replicas.clone(), speed: config.speed, long_pending_duration, @@ -521,6 +531,15 @@ impl SimpleScaler { &sorted_clusters ); + // Increase queue size, if it's too small, to make sure that required min_provers are + // running. + let queue: u64 = if self.apply_min_to_namespace.as_deref() == Some(namespace.as_str()) { + self.normalize_queue(queue) + .max(self.pods_to_speed(self.min_replicas)) + } else { + queue + }; + let mut total: i64 = 0; let mut pods: HashMap = HashMap::new(); for cluster in &sorted_clusters { @@ -719,7 +738,8 @@ mod tests { fn test_run() { let scaler = GpuScaler::new(ProverAutoscalerScalerConfig { cluster_priorities: [("foo".into(), 0), ("bar".into(), 10)].into(), - min_provers: [("prover-other".into(), 2)].into(), + apply_min_to_namespace: Some("prover-other".into()), + min_provers: 2, max_provers: [ ("foo".into(), [(Gpu::L4, 100)].into()), ("bar".into(), [(Gpu::L4, 100)].into()), @@ -857,7 +877,8 @@ mod tests { fn test_run_min_provers() { let scaler = GpuScaler::new(ProverAutoscalerScalerConfig { cluster_priorities: [("foo".into(), 0), ("bar".into(), 10)].into(), - min_provers: [("prover".into(), 2)].into(), + apply_min_to_namespace: Some("prover".into()), + min_provers: 2, max_provers: [ ("foo".into(), [(Gpu::L4, 100)].into()), ("bar".into(), [(Gpu::L4, 100)].into()), @@ -1052,7 +1073,8 @@ mod tests { fn test_run_need_move() { let scaler = GpuScaler::new(ProverAutoscalerScalerConfig { cluster_priorities: [("foo".into(), 0), ("bar".into(), 10)].into(), - min_provers: [("prover".into(), 2)].into(), + apply_min_to_namespace: Some("prover".into()), + min_provers: 2, max_provers: [ ("foo".into(), [(Gpu::L4, 100)].into()), ("bar".into(), [(Gpu::L4, 100)].into()),