From c1f2f5e33de90244f12fa22e10f8a40a4a2d9cdb Mon Sep 17 00:00:00 2001 From: pompon0 Date: Thu, 26 Oct 2023 13:18:04 +0200 Subject: [PATCH] refactor: made unit tests use separate real dbs (#213) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # What ❔ Made unit tests use separate real dbs created in a temporary postgres container. ## Why ❔ It makes the tests more realistic and greatly simplifies the dal crate. --- Cargo.lock | 14 +- Cargo.toml | 1 - core/lib/dal/Cargo.toml | 3 +- core/lib/dal/src/blocks_dal.rs | 11 +- core/lib/dal/src/blocks_web3_dal.rs | 36 +++-- core/lib/dal/src/connection/holder.rs | 8 +- core/lib/dal/src/connection/mod.rs | 145 +++++++++++------- core/lib/dal/src/connection/test_pool.rs | 124 --------------- core/lib/dal/src/events_dal.rs | 11 +- core/lib/dal/src/events_web3_dal.rs | 8 +- core/lib/dal/src/healthcheck.rs | 9 +- core/lib/dal/src/instrument.rs | 11 +- core/lib/dal/src/lib.rs | 10 +- core/lib/dal/src/storage_dal.rs | 6 +- core/lib/dal/src/storage_logs_dal.rs | 16 +- core/lib/dal/src/tests/mod.rs | 57 ++++--- core/lib/dal/src/transactions_web3_dal.rs | 15 +- core/lib/db_test_macro/Cargo.toml | 14 -- core/lib/db_test_macro/src/lib.rs | 131 ---------------- core/lib/state/Cargo.toml | 2 - core/lib/state/src/postgres/tests.rs | 36 +++-- core/lib/state/src/rocksdb/mod.rs | 16 +- core/lib/zksync_core/Cargo.toml | 1 - .../zksync_core/src/api_server/tree/tests.rs | 7 +- .../zksync_core/src/api_server/web3/tests.rs | 6 +- core/lib/zksync_core/src/eth_sender/tests.rs | 119 +++++++++----- core/lib/zksync_core/src/eth_watch/tests.rs | 53 ++++--- core/lib/zksync_core/src/genesis.rs | 11 +- core/lib/zksync_core/src/lib.rs | 10 -- .../src/metadata_calculator/helpers.rs | 21 +-- .../src/metadata_calculator/tests.rs | 87 ++++++----- .../state_keeper/batch_executor/tests/mod.rs | 61 +++++--- .../src/state_keeper/io/tests/mod.rs | 74 +++++---- core/lib/zksync_core/src/sync_layer/tests.rs | 26 ++-- docker-compose-runner.yml | 2 + docker-compose-unit-tests.yml | 11 ++ etc/env/base/private.toml | 2 +- etc/env/docker.toml | 2 +- etc/env/ext-node-docker.toml | 2 +- etc/env/ext-node.toml | 2 +- infrastructure/zk/src/database.ts | 16 +- infrastructure/zk/src/test/test.ts | 2 +- 42 files changed, 526 insertions(+), 673 deletions(-) delete mode 100644 core/lib/dal/src/connection/test_pool.rs delete mode 100644 core/lib/db_test_macro/Cargo.toml delete mode 100644 core/lib/db_test_macro/src/lib.rs create mode 100644 docker-compose-unit-tests.yml diff --git a/Cargo.lock b/Cargo.lock index 189f5d50f430..0768a93c7216 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1826,15 +1826,6 @@ dependencies = [ "parking_lot_core 0.9.8", ] -[[package]] -name = "db_test_macro" -version = "0.1.0" -dependencies = [ - "proc-macro2 1.0.66", - "quote 1.0.33", - "syn 1.0.109", -] - [[package]] name = "debugid" version = "0.8.0" @@ -7993,7 +7984,6 @@ dependencies = [ "bitflags 1.3.2", "chrono", "ctrlc", - "db_test_macro", "futures 0.3.28", "governor", "hex", @@ -8068,11 +8058,11 @@ dependencies = [ "assert_matches", "bigdecimal", "bincode", - "db_test_macro", "hex", "itertools", "num 0.3.1", "once_cell", + "rand 0.8.5", "serde", "serde_json", "sqlx", @@ -8080,6 +8070,7 @@ dependencies = [ "thiserror", "tokio", "tracing", + "url", "vise", "zksync_contracts", "zksync_health_check", @@ -8283,7 +8274,6 @@ name = "zksync_state" version = "0.1.0" dependencies = [ "anyhow", - "db_test_macro", "itertools", "mini-moka", "rand 0.8.5", diff --git a/Cargo.toml b/Cargo.toml index ad23eeac9c9f..a02652c3f617 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -21,7 +21,6 @@ members = [ "core/lib/circuit_breaker", "core/lib/commitment_utils", "core/lib/dal", - "core/lib/db_test_macro", "core/lib/eth_client", "core/lib/eth_signer", "core/lib/mempool", diff --git a/core/lib/dal/Cargo.toml b/core/lib/dal/Cargo.toml index 759267278589..616e8a32a9ef 100644 --- a/core/lib/dal/Cargo.toml +++ b/core/lib/dal/Cargo.toml @@ -20,6 +20,8 @@ zksync_health_check = { path = "../health_check" } itertools = "0.10.1" thiserror = "1.0" anyhow = "1.0" +url = "2" +rand = "0.8" tokio = { version = "1", features = ["full"] } sqlx = { version = "0.5.13", default-features = false, features = [ "runtime-tokio-native-tls", @@ -44,4 +46,3 @@ tracing = "0.1" [dev-dependencies] assert_matches = "1.5.0" -db_test_macro = { path = "../db_test_macro" } diff --git a/core/lib/dal/src/blocks_dal.rs b/core/lib/dal/src/blocks_dal.rs index 66092d0220a7..1ffdba4aaa95 100644 --- a/core/lib/dal/src/blocks_dal.rs +++ b/core/lib/dal/src/blocks_dal.rs @@ -1482,15 +1482,15 @@ impl BlocksDal<'_, '_> { #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{l2_to_l1_log::L2ToL1Log, Address, ProtocolVersion, ProtocolVersionId}; use super::*; use crate::ConnectionPool; - #[db_test(dal_crate)] - async fn loading_l1_batch_header(pool: ConnectionPool) { + #[tokio::test] + async fn loading_l1_batch_header() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_l1_batches(L1BatchNumber(0)) @@ -1549,8 +1549,9 @@ mod tests { .is_none()); } - #[db_test(dal_crate)] - async fn getting_predicted_gas(pool: ConnectionPool) { + #[tokio::test] + async fn getting_predicted_gas() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_l1_batches(L1BatchNumber(0)) diff --git a/core/lib/dal/src/blocks_web3_dal.rs b/core/lib/dal/src/blocks_web3_dal.rs index 86f84de09732..58f6730570ed 100644 --- a/core/lib/dal/src/blocks_web3_dal.rs +++ b/core/lib/dal/src/blocks_web3_dal.rs @@ -582,7 +582,6 @@ impl BlocksWeb3Dal<'_, '_> { #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{ block::{miniblock_hash, MiniblockHeader}, @@ -592,9 +591,10 @@ mod tests { use super::*; use crate::{tests::create_miniblock_header, ConnectionPool}; - #[db_test(dal_crate)] - async fn getting_web3_block_and_tx_count(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn getting_web3_block_and_tx_count() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_miniblocks(MiniblockNumber(0)) .await @@ -659,9 +659,10 @@ mod tests { } } - #[db_test(dal_crate)] - async fn resolving_earliest_block_id(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn resolving_earliest_block_id() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_miniblocks(MiniblockNumber(0)) .await @@ -674,9 +675,10 @@ mod tests { assert_eq!(miniblock_number.unwrap(), Some(MiniblockNumber(0))); } - #[db_test(dal_crate)] - async fn resolving_latest_block_id(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn resolving_latest_block_id() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_miniblocks(MiniblockNumber(0)) .await @@ -729,9 +731,10 @@ mod tests { assert_eq!(miniblock_number.unwrap(), Some(MiniblockNumber(1))); } - #[db_test(dal_crate)] - async fn resolving_block_by_hash(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn resolving_block_by_hash() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.blocks_dal() .delete_miniblocks(MiniblockNumber(0)) .await @@ -759,9 +762,10 @@ mod tests { assert_eq!(miniblock_number.unwrap(), None); } - #[db_test(dal_crate)] - async fn getting_miniblocks_for_virtual_block(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn getting_miniblocks_for_virtual_block() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.protocol_versions_dal() .save_protocol_version_with_tx(ProtocolVersion::default()) diff --git a/core/lib/dal/src/connection/holder.rs b/core/lib/dal/src/connection/holder.rs index 1ba455011f47..a579be8dd4a9 100644 --- a/core/lib/dal/src/connection/holder.rs +++ b/core/lib/dal/src/connection/holder.rs @@ -1,11 +1,7 @@ // Built-in deps -use std::fmt; -// External imports use sqlx::pool::PoolConnection; use sqlx::{postgres::Postgres, PgConnection, Transaction}; -// Workspace imports -// Local imports -use crate::connection::test_pool::TestPoolLock; +use std::fmt; /// Connection holder unifies the type of underlying connection, which /// can be either pooled or direct. @@ -13,7 +9,6 @@ pub enum ConnectionHolder<'a> { Pooled(PoolConnection), Direct(PgConnection), Transaction(Transaction<'a, Postgres>), - TestTransaction(TestPoolLock), } impl<'a> fmt::Debug for ConnectionHolder<'a> { @@ -22,7 +17,6 @@ impl<'a> fmt::Debug for ConnectionHolder<'a> { Self::Pooled(_) => write!(f, "Pooled connection"), Self::Direct(_) => write!(f, "Direct connection"), Self::Transaction(_) => write!(f, "Database Transaction"), - Self::TestTransaction(_) => write!(f, "Test Database Transaction"), } } } diff --git a/core/lib/dal/src/connection/mod.rs b/core/lib/dal/src/connection/mod.rs index c79d3f6ae974..4fe1844968c7 100644 --- a/core/lib/dal/src/connection/mod.rs +++ b/core/lib/dal/src/connection/mod.rs @@ -4,14 +4,12 @@ use sqlx::{ }; use anyhow::Context as _; +use std::fmt; use std::time::Duration; use zksync_utils::parse_env; pub mod holder; -pub mod test_pool; - -pub use self::test_pool::TestPool; use crate::{ get_master_database_url, get_prover_database_url, get_replica_database_url, @@ -23,6 +21,7 @@ pub enum DbVariant { Master, Replica, Prover, + TestTmp, } /// Builder for [`ConnectionPool`]s. @@ -56,19 +55,25 @@ impl ConnectionPoolBuilder { DbVariant::Master => get_master_database_url()?, DbVariant::Replica => get_replica_database_url()?, DbVariant::Prover => get_prover_database_url()?, + DbVariant::TestTmp => create_test_db() + .await + .context("create_test_db()")? + .to_string(), }; - Ok(self.build_inner(&database_url).await) + self.build_inner(&database_url) + .await + .context("build_inner()") } - pub async fn build_inner(&self, database_url: &str) -> ConnectionPool { + pub async fn build_inner(&self, database_url: &str) -> anyhow::Result { let max_connections = self .max_size .unwrap_or_else(|| parse_env("DATABASE_POOL_SIZE")); let options = PgPoolOptions::new().max_connections(max_connections); - let mut connect_options: PgConnectOptions = database_url.parse().unwrap_or_else(|err| { - panic!("Failed parsing {:?} database URL: {}", self.db, err); - }); + let mut connect_options: PgConnectOptions = database_url + .parse() + .with_context(|| format!("Failed parsing {:?} database URL", self.db))?; if let Some(timeout) = self.statement_timeout { let timeout_string = format!("{}s", timeout.as_secs()); connect_options = connect_options.options([("statement_timeout", timeout_string)]); @@ -76,26 +81,75 @@ impl ConnectionPoolBuilder { let pool = options .connect_with(connect_options) .await - .unwrap_or_else(|err| { - panic!("Failed connecting to {:?} database: {}", self.db, err); - }); + .with_context(|| format!("Failed connecting to {:?} database", self.db))?; tracing::info!( "Created pool for {db:?} database with {max_connections} max connections \ and {statement_timeout:?} statement timeout", db = self.db, statement_timeout = self.statement_timeout ); - ConnectionPool::Real(pool) + Ok(ConnectionPool(pool)) } } -#[derive(Debug, Clone)] -pub enum ConnectionPool { - Real(PgPool), - Test(TestPool), +/// Constructucts a new temporary database (with a randomized name) +/// by cloning the database template pointed by TEST_DATABASE_URL env var. +/// The template is expected to have all migrations from dal/migrations applied. +/// For efficiency, the postgres container of TEST_DATABASE_URL should be +/// configured with option "fsync=off" - it disables waiting for disk synchronization +/// whenever you write to the DBs, therefore making it as fast as an inmem postgres instance. +/// The database is not cleaned up automatically, but rather the whole postgres +/// container is recreated whenever you call "zk test rust". +pub(super) async fn create_test_db() -> anyhow::Result { + use rand::Rng as _; + use sqlx::{Connection as _, Executor as _}; + const PREFIX: &str = "test-"; + let db_url = crate::get_test_database_url().unwrap(); + let mut db_url = url::Url::parse(&db_url) + .with_context(|| format!("{} is not a valid database address", db_url))?; + let db_name = db_url + .path() + .strip_prefix('/') + .with_context(|| format!("{} is not a valid database address", db_url.as_ref()))? + .to_string(); + let db_copy_name = format!("{PREFIX}{}", rand::thread_rng().gen::()); + db_url.set_path(""); + let mut attempts = 10; + let mut conn = loop { + match sqlx::PgConnection::connect(db_url.as_ref()).await { + Ok(conn) => break conn, + Err(err) => { + attempts -= 1; + if attempts == 0 { + return Err(err).context("sqlx::PgConnection::connect()"); + } + } + } + tokio::time::sleep(std::time::Duration::from_millis(100)).await; + }; + conn.execute( + format!("CREATE DATABASE \"{db_copy_name}\" WITH TEMPLATE \"{db_name}\"").as_str(), + ) + .await + .context("failed to create a temporary database")?; + db_url.set_path(&db_copy_name); + Ok(db_url) +} + +#[derive(Clone)] +pub struct ConnectionPool(pub(crate) PgPool); + +impl fmt::Debug for ConnectionPool { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("ConnectionPool").finish() + } } impl ConnectionPool { + pub async fn test_pool() -> ConnectionPool { + Self::builder(DbVariant::TestTmp).build().await.unwrap() + } + /// Initializes a builder for connection pools. pub fn builder(db: DbVariant) -> ConnectionPoolBuilder { ConnectionPoolBuilder { @@ -143,32 +197,28 @@ impl ConnectionPool { &self, requester: Option<&'static str>, ) -> anyhow::Result> { - Ok(match self { - ConnectionPool::Real(real_pool) => { - let acquire_latency = CONNECTION_METRICS.acquire.start(); - let conn = Self::acquire_connection_retried(real_pool) - .await - .context("acquire_connection_retried()")?; - let elapsed = acquire_latency.observe(); - if let Some(requester) = requester { - CONNECTION_METRICS.acquire_tagged[&requester].observe(elapsed); - } - StorageProcessor::from_pool(conn) - } - ConnectionPool::Test(test) => test.access_storage().await, - }) + let acquire_latency = CONNECTION_METRICS.acquire.start(); + let conn = self + .acquire_connection_retried() + .await + .context("acquire_connection_retried()")?; + let elapsed = acquire_latency.observe(); + if let Some(requester) = requester { + CONNECTION_METRICS.acquire_tagged[&requester].observe(elapsed); + } + Ok(StorageProcessor::from_pool(conn)) } - async fn acquire_connection_retried(pool: &PgPool) -> anyhow::Result> { + async fn acquire_connection_retried(&self) -> anyhow::Result> { const DB_CONNECTION_RETRIES: u32 = 3; const BACKOFF_INTERVAL: Duration = Duration::from_secs(1); let mut retry_count = 0; while retry_count < DB_CONNECTION_RETRIES { - CONNECTION_METRICS.pool_size.observe(pool.size() as usize); - CONNECTION_METRICS.pool_idle.observe(pool.num_idle()); + CONNECTION_METRICS.pool_size.observe(self.0.size() as usize); + CONNECTION_METRICS.pool_idle.observe(self.0.num_idle()); - let connection = pool.acquire().await; + let connection = self.0.acquire().await; let connection_err = match connection { Ok(connection) => return Ok(connection), Err(err) => { @@ -185,7 +235,7 @@ impl ConnectionPool { } // Attempting to get the pooled connection for the last time - match pool.acquire().await { + match self.0.acquire().await { Ok(conn) => Ok(conn), Err(err) => { Self::report_connection_error(&err); @@ -197,15 +247,6 @@ impl ConnectionPool { fn report_connection_error(err: &sqlx::Error) { CONNECTION_METRICS.pool_acquire_error[&err.into()].inc(); } - - pub async fn access_test_storage(&self) -> StorageProcessor<'static> { - match self { - ConnectionPool::Test(test) => test.access_storage().await, - ConnectionPool::Real(_) => { - panic!("Attempt to access test storage with the real pool"); - } - } - } } #[cfg(test)] @@ -213,23 +254,19 @@ mod tests { use assert_matches::assert_matches; use super::*; - use crate::get_test_database_url; #[tokio::test] async fn setting_statement_timeout() { - // We cannot use an ordinary test pool here because it isn't created using `ConnectionPoolBuilder`. - // Since we don't need to mutate the DB for the test, using a real DB connection is OK. - let database_url = get_test_database_url().unwrap(); - let pool = ConnectionPool::builder(DbVariant::Master) + let pool = ConnectionPool::builder(DbVariant::TestTmp) .set_statement_timeout(Some(Duration::from_secs(1))) - .build_inner(&database_url) - .await; + .build() + .await + .unwrap(); - // NB. We must not mutate the database below! Doing so may break other tests. - let mut conn = pool.access_storage().await.unwrap(); + let mut storage = pool.access_storage().await.unwrap(); let err = sqlx::query("SELECT pg_sleep(2)") .map(drop) - .fetch_optional(conn.conn()) + .fetch_optional(storage.conn()) .await .unwrap_err(); assert_matches!( diff --git a/core/lib/dal/src/connection/test_pool.rs b/core/lib/dal/src/connection/test_pool.rs deleted file mode 100644 index 722f389584ac..000000000000 --- a/core/lib/dal/src/connection/test_pool.rs +++ /dev/null @@ -1,124 +0,0 @@ -// Built-in deps -use std::{fmt, mem, pin::Pin, sync::Arc, time::Duration}; -// External imports -use sqlx::{Acquire, Connection, PgConnection, Postgres, Transaction}; -use tokio::{ - sync::{Mutex, OwnedMutexGuard}, - time::timeout, -}; -// Local imports -use crate::StorageProcessor; - -/// Self-referential struct powering [`TestPool`]. -// Ideally, we'd want to use a readily available crate like `ouroboros` to define this struct, -// but `ouroboros` in particular doesn't satisfy our needs: -// -// - It doesn't provide mutable access to the tail field (`subtransaction`), only allowing -// to mutably access it in a closure. -// - There is an error borrowing from `transaction` since it implements `Drop`. -struct TestPoolInner { - // Mutably references `_transaction`. - subtransaction: Transaction<'static, Postgres>, - // Mutably references `_connection`. Must not be used anywhere since it's mutably borrowed! - _transaction: Pin>>, - // Must not be used anywhere since it's mutably borrowed! - _connection: Pin>, -} - -impl fmt::Debug for TestPoolInner { - fn fmt(&self, formatter: &mut fmt::Formatter<'_>) -> fmt::Result { - formatter - .debug_struct("Inner") - .field("subtransaction", &self.subtransaction) - .finish() - } -} - -impl TestPoolInner { - async fn new() -> Self { - let database_url = crate::get_test_database_url().unwrap(); - let connection = PgConnection::connect(&database_url).await.unwrap(); - let mut connection = Box::pin(connection); - - let transaction = Connection::begin(&mut *connection).await.unwrap(); - let transaction: Transaction<'static, Postgres> = unsafe { - // SAFETY: We extend `transaction` lifetime to `'static`. This is valid - // because the transaction borrows from `connection`, which outlives `transaction` - // (since it's a field in the same struct, and fields in a struct are dropped - // in the declaration order), is not moved after the borrow - // (due to being wrapped in a `Pin>`), and is not accessed afterwards. - mem::transmute(transaction) - }; - let mut transaction = Box::pin(transaction); - - let subtransaction = transaction.begin().await.unwrap(); - let subtransaction: Transaction<'static, Postgres> = unsafe { - // SAFETY: We extend `subtransaction` lifetime to `'static`. This is valid - // for the same reasons that apply for `transaction`. - mem::transmute(subtransaction) - }; - - Self { - subtransaction, - _transaction: transaction, - _connection: connection, - } - } -} - -#[derive(Debug)] -pub struct TestPoolLock { - lock: OwnedMutexGuard, -} - -impl TestPoolLock { - pub fn as_connection(&mut self) -> &mut PgConnection { - &mut self.lock.subtransaction - } -} - -/// Implementation of the test/fake connection pool to be used in tests. -/// This implementation works over an established transaction in order to reject -/// any changes made to the database, even if the tested component initiates and commits -/// its own transactions. -/// -/// # How it works -/// -/// Test pool uses an established transaction to be created. This transaction, in its turn, -/// is used to establish a *subtransaction*. Reference to this subtransaction will be used -/// as a connection to create `StorageProcessor` objects in test. -/// -/// Having a subtransaction is necessary: even if some component will (mistakenly) will not -/// initiate a transaction and will call `commit` on `StorageProcessor`, changes won't be -/// persisted, since top-level transaction will be dropped. -/// -/// At the same time, using *reference* to the subtransaction in created `StorageProcessor` -/// objects is also necessary: upon `drop`, transaction gets discarded. It means that if we -/// use transaction and somewhere in test `StorageProcessor` is created, used without -/// transaction and then dropped (which is a normal use case for e.g. test setup) -- such -/// changes would be discarded and test will not execute correctly. -#[derive(Debug, Clone)] -pub struct TestPool { - inner: Arc>, -} - -impl TestPool { - /// Constructs a new object using an already established transaction to the database. - /// This method is unsafe, since internally it extends lifetime of the provided `Transaction`. - pub async fn new() -> Self { - Self { - inner: Arc::new(Mutex::new(TestPoolInner::new().await)), - } - } - - pub async fn access_storage(&self) -> StorageProcessor<'static> { - const LOCK_TIMEOUT: Duration = Duration::from_secs(1); - - let lock = self.inner.clone().lock_owned(); - let lock = timeout(LOCK_TIMEOUT, lock).await.expect( - "Timed out waiting to acquire a lock in test `ConnectionPool`. \ - Check the backtrace and make sure that no `StorageProcessor`s are alive", - ); - StorageProcessor::from_test_transaction(TestPoolLock { lock }) - } -} diff --git a/core/lib/dal/src/events_dal.rs b/core/lib/dal/src/events_dal.rs index 53a672001ac9..2702099e5810 100644 --- a/core/lib/dal/src/events_dal.rs +++ b/core/lib/dal/src/events_dal.rs @@ -196,7 +196,6 @@ impl EventsDal<'_, '_> { mod tests { use super::*; use crate::{tests::create_miniblock_header, ConnectionPool}; - use db_test_macro::db_test; use zksync_types::{Address, L1BatchNumber, ProtocolVersion}; fn create_vm_event(index: u8, topic_count: u8) -> VmEvent { @@ -209,8 +208,9 @@ mod tests { } } - #[db_test(dal_crate)] - async fn storing_events(pool: ConnectionPool) { + #[tokio::test] + async fn storing_events() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.events_dal().rollback_events(MiniblockNumber(0)).await; conn.blocks_dal() @@ -284,8 +284,9 @@ mod tests { } } - #[db_test(dal_crate)] - async fn storing_l2_to_l1_logs(pool: ConnectionPool) { + #[tokio::test] + async fn storing_l2_to_l1_logs() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.events_dal() .rollback_l2_to_l1_logs(MiniblockNumber(0)) diff --git a/core/lib/dal/src/events_web3_dal.rs b/core/lib/dal/src/events_web3_dal.rs index 1b38362d6a90..82a65c18444b 100644 --- a/core/lib/dal/src/events_web3_dal.rs +++ b/core/lib/dal/src/events_web3_dal.rs @@ -172,16 +172,16 @@ impl EventsWeb3Dal<'_, '_> { #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_types::api::BlockNumber; use zksync_types::{Address, H256}; use super::*; use crate::connection::ConnectionPool; - #[db_test(dal_crate)] - async fn test_build_get_logs_where_clause(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; + #[tokio::test] + async fn test_build_get_logs_where_clause() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let events_web3_dal = EventsWeb3Dal { storage }; let filter = GetLogsFilter { from_block: MiniblockNumber(100), diff --git a/core/lib/dal/src/healthcheck.rs b/core/lib/dal/src/healthcheck.rs index 3143a631ac2f..902a235ce540 100644 --- a/core/lib/dal/src/healthcheck.rs +++ b/core/lib/dal/src/healthcheck.rs @@ -42,12 +42,7 @@ impl CheckHealth for ConnectionPoolHealthCheck { // This check is rather feeble, plan to make reliable here: // https://linear.app/matterlabs/issue/PLA-255/revamp-db-connection-health-check self.connection_pool.access_storage().await.unwrap(); - - let mut health = Health::from(HealthStatus::Ready); - if let ConnectionPool::Real(pool) = &self.connection_pool { - let details = ConnectionPoolHealthDetails::new(pool).await; - health = health.with_details(details); - } - health + let details = ConnectionPoolHealthDetails::new(&self.connection_pool.0).await; + Health::from(HealthStatus::Ready).with_details(details) } } diff --git a/core/lib/dal/src/instrument.rs b/core/lib/dal/src/instrument.rs index 0c03457c2524..cd761fb35004 100644 --- a/core/lib/dal/src/instrument.rs +++ b/core/lib/dal/src/instrument.rs @@ -241,14 +241,14 @@ where #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_types::{MiniblockNumber, H256}; use super::*; use crate::ConnectionPool; - #[db_test(dal_crate)] - async fn instrumenting_erroneous_query(pool: ConnectionPool) { + #[tokio::test] + async fn instrumenting_erroneous_query() { + let pool = ConnectionPool::test_pool().await; // Add `vlog::init()` here to debug this test let mut conn = pool.access_storage().await.unwrap(); @@ -262,8 +262,9 @@ mod tests { .unwrap_err(); } - #[db_test(dal_crate)] - async fn instrumenting_slow_query(pool: ConnectionPool) { + #[tokio::test] + async fn instrumenting_slow_query() { + let pool = ConnectionPool::test_pool().await; // Add `vlog::init()` here to debug this test let mut conn = pool.access_storage().await.unwrap(); diff --git a/core/lib/dal/src/lib.rs b/core/lib/dal/src/lib.rs index 627b46e453ee..6a3815379916 100644 --- a/core/lib/dal/src/lib.rs +++ b/core/lib/dal/src/lib.rs @@ -15,8 +15,8 @@ use crate::accounts_dal::AccountsDal; use crate::basic_witness_input_producer_dal::BasicWitnessInputProducerDal; use crate::blocks_dal::BlocksDal; use crate::blocks_web3_dal::BlocksWeb3Dal; +use crate::connection::holder::ConnectionHolder; pub use crate::connection::ConnectionPool; -use crate::connection::{holder::ConnectionHolder, test_pool::TestPoolLock}; use crate::contract_verification_dal::ContractVerificationDal; use crate::eth_sender_dal::EthSenderDal; use crate::events_dal::EventsDal; @@ -158,13 +158,6 @@ impl<'a> StorageProcessor<'a> { } } - pub fn from_test_transaction(conn: TestPoolLock) -> StorageProcessor<'static> { - StorageProcessor { - conn: ConnectionHolder::TestTransaction(conn), - in_transaction: true, - } - } - pub async fn commit(self) -> sqlx::Result<()> { if let ConnectionHolder::Transaction(transaction) = self.conn { transaction.commit().await @@ -188,7 +181,6 @@ impl<'a> StorageProcessor<'a> { ConnectionHolder::Pooled(conn) => conn, ConnectionHolder::Direct(conn) => conn, ConnectionHolder::Transaction(conn) => conn, - ConnectionHolder::TestTransaction(conn) => conn.as_connection(), } } diff --git a/core/lib/dal/src/storage_dal.rs b/core/lib/dal/src/storage_dal.rs index 94a1becc3c25..fdaaea386171 100644 --- a/core/lib/dal/src/storage_dal.rs +++ b/core/lib/dal/src/storage_dal.rs @@ -212,11 +212,11 @@ impl StorageDal<'_, '_> { mod tests { use super::*; use crate::ConnectionPool; - use db_test_macro::db_test; use zksync_types::{AccountTreeId, Address}; - #[db_test(dal_crate)] - async fn applying_storage_logs(pool: ConnectionPool) { + #[tokio::test] + async fn applying_storage_logs() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); let account = AccountTreeId::new(Address::repeat_byte(1)); diff --git a/core/lib/dal/src/storage_logs_dal.rs b/core/lib/dal/src/storage_logs_dal.rs index 286cd6c2e8a8..c368e5adc8da 100644 --- a/core/lib/dal/src/storage_logs_dal.rs +++ b/core/lib/dal/src/storage_logs_dal.rs @@ -531,7 +531,6 @@ impl StorageLogsDal<'_, '_> { mod tests { use super::*; use crate::{tests::create_miniblock_header, ConnectionPool}; - use db_test_macro::db_test; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{ block::{BlockGasCount, L1BatchHeader}, @@ -567,8 +566,9 @@ mod tests { .unwrap(); } - #[db_test(dal_crate)] - async fn inserting_storage_logs(pool: ConnectionPool) { + #[tokio::test] + async fn inserting_storage_logs() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal() @@ -659,8 +659,9 @@ mod tests { assert!(value.is_none()); } - #[db_test(dal_crate)] - async fn getting_storage_logs_for_revert(pool: ConnectionPool) { + #[tokio::test] + async fn getting_storage_logs_for_revert() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal() @@ -714,8 +715,9 @@ mod tests { } } - #[db_test(dal_crate)] - async fn reverting_keys_without_initial_write(pool: ConnectionPool) { + #[tokio::test] + async fn reverting_keys_without_initial_write() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal() diff --git a/core/lib/dal/src/tests/mod.rs b/core/lib/dal/src/tests/mod.rs index 4a9e1ed99799..c383ea7f9441 100644 --- a/core/lib/dal/src/tests/mod.rs +++ b/core/lib/dal/src/tests/mod.rs @@ -1,7 +1,6 @@ use std::fs; use std::time::Duration; -use db_test_macro::db_test; use zksync_contracts::BaseSystemContractsHashes; use zksync_types::{ block::{miniblock_hash, L1BatchHeader, MiniblockHeader}, @@ -118,9 +117,10 @@ pub(crate) fn mock_execution_result(transaction: L2Tx) -> TransactionExecutionRe } } -#[db_test(dal_crate)] -async fn workflow_with_submit_tx_equal_hashes(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn workflow_with_submit_tx_equal_hashes() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let mut transactions_dal = TransactionsDal { storage }; let tx = mock_l2_transaction(); @@ -137,9 +137,10 @@ async fn workflow_with_submit_tx_equal_hashes(connection_pool: ConnectionPool) { assert_eq!(result, L2TxSubmissionResult::Replaced); } -#[db_test(dal_crate)] -async fn workflow_with_submit_tx_diff_hashes(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn workflow_with_submit_tx_diff_hashes() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let mut transactions_dal = TransactionsDal { storage }; let tx = mock_l2_transaction(); @@ -163,9 +164,10 @@ async fn workflow_with_submit_tx_diff_hashes(connection_pool: ConnectionPool) { assert_eq!(result, L2TxSubmissionResult::Replaced); } -#[db_test(dal_crate)] -async fn remove_stuck_txs(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn remove_stuck_txs() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let mut protocol_versions_dal = ProtocolVersionsDal { storage }; protocol_versions_dal .save_protocol_version_with_tx(Default::default()) @@ -269,9 +271,10 @@ fn create_circuits() -> Vec<(&'static str, String)> { ] } -#[db_test(dal_crate)] -async fn test_duplicate_insert_prover_jobs(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn test_duplicate_insert_prover_jobs() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); storage .protocol_versions_dal() .save_protocol_version_with_tx(Default::default()) @@ -330,9 +333,10 @@ async fn test_duplicate_insert_prover_jobs(connection_pool: ConnectionPool) { assert_eq!(circuits.len(), jobs.len()); } -#[db_test(dal_crate)] -async fn test_requeue_prover_jobs(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn test_requeue_prover_jobs() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let protocol_version = ProtocolVersion::default(); storage .protocol_versions_dal() @@ -393,9 +397,10 @@ async fn test_requeue_prover_jobs(connection_pool: ConnectionPool) { } } -#[db_test(dal_crate)] -async fn test_move_leaf_aggregation_jobs_from_waiting_to_queued(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn test_move_leaf_aggregation_jobs_from_waiting_to_queued() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let protocol_version = ProtocolVersion::default(); storage .protocol_versions_dal() @@ -473,9 +478,10 @@ async fn test_move_leaf_aggregation_jobs_from_waiting_to_queued(connection_pool: assert_eq!(l1_batch_number, job.unwrap().block_number); } -#[db_test(dal_crate)] -async fn test_move_node_aggregation_jobs_from_waiting_to_queued(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn test_move_node_aggregation_jobs_from_waiting_to_queued() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let protocol_version = ProtocolVersion::default(); storage .protocol_versions_dal() @@ -560,9 +566,10 @@ async fn test_move_node_aggregation_jobs_from_waiting_to_queued(connection_pool: assert_eq!(l1_batch_number, job.unwrap().block_number); } -#[db_test(dal_crate)] -async fn test_move_scheduler_jobs_from_waiting_to_queued(connection_pool: ConnectionPool) { - let storage = &mut connection_pool.access_test_storage().await; +#[tokio::test] +async fn test_move_scheduler_jobs_from_waiting_to_queued() { + let connection_pool = ConnectionPool::test_pool().await; + let storage = &mut connection_pool.access_storage().await.unwrap(); let protocol_version = ProtocolVersion::default(); storage .protocol_versions_dal() diff --git a/core/lib/dal/src/transactions_web3_dal.rs b/core/lib/dal/src/transactions_web3_dal.rs index 8ad983a2218e..5e2342d05b75 100644 --- a/core/lib/dal/src/transactions_web3_dal.rs +++ b/core/lib/dal/src/transactions_web3_dal.rs @@ -353,7 +353,6 @@ impl TransactionsWeb3Dal<'_, '_> { #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_types::{ block::miniblock_hash, fee::TransactionExecutionMetrics, l2::L2Tx, ProtocolVersion, }; @@ -389,9 +388,10 @@ mod tests { .await; } - #[db_test(dal_crate)] - async fn getting_transaction(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn getting_transaction() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.protocol_versions_dal() .save_protocol_version_with_tx(ProtocolVersion::default()) .await; @@ -454,9 +454,10 @@ mod tests { } } - #[db_test(dal_crate)] - async fn getting_miniblock_transactions(connection_pool: ConnectionPool) { - let mut conn = connection_pool.access_test_storage().await; + #[tokio::test] + async fn getting_miniblock_transactions() { + let connection_pool = ConnectionPool::test_pool().await; + let mut conn = connection_pool.access_storage().await.unwrap(); conn.protocol_versions_dal() .save_protocol_version_with_tx(ProtocolVersion::default()) .await; diff --git a/core/lib/db_test_macro/Cargo.toml b/core/lib/db_test_macro/Cargo.toml deleted file mode 100644 index 40d0b4563de4..000000000000 --- a/core/lib/db_test_macro/Cargo.toml +++ /dev/null @@ -1,14 +0,0 @@ -[package] -name = "db_test_macro" -version = "0.1.0" -authors = ["The Matter Labs Team "] -edition = "2018" -license = "MIT OR Apache-2.0" - -[lib] -proc-macro = true - -[dependencies] -proc-macro2 = "1.0.7" -quote = "1" -syn = { version = "1.0.3", features = ["full"] } diff --git a/core/lib/db_test_macro/src/lib.rs b/core/lib/db_test_macro/src/lib.rs deleted file mode 100644 index baeae318af2c..000000000000 --- a/core/lib/db_test_macro/src/lib.rs +++ /dev/null @@ -1,131 +0,0 @@ -use proc_macro::TokenStream; -use quote::quote; -use syn::{ - parse::{Parse, ParseStream}, - punctuated::Punctuated, - FnArg, Ident, Token, -}; - -/// Argument that can be supplied to the `db_test` macro to be used in the `zksync_dal` crate. -const DAL_CRATE_MARKER_ARG: &str = "dal_crate"; -/// Name of the type that represents the connection pool in DAL. -const TYPE_NAME: &str = "ConnectionPool"; - -#[derive(Debug)] -struct Args { - vars: Vec, -} - -impl Parse for Args { - fn parse(input: ParseStream) -> syn::Result { - let vars = Punctuated::::parse_terminated(input)?; - Ok(Args { - vars: vars.into_iter().collect(), - }) - } -} - -fn parse_knobs(mut input: syn::ItemFn, inside_dal_crate: bool) -> Result { - let sig = &mut input.sig; - let body = &input.block; - let attrs = &input.attrs; - let vis = input.vis; - - if sig.asyncness.is_none() { - let msg = "the async keyword is missing from the function declaration"; - return Err(syn::Error::new_spanned(sig.fn_token, msg)); - } - - sig.asyncness = None; - - let rt = quote! { tokio::runtime::Builder::new_current_thread() }; - let header = quote! { - #[::core::prelude::v1::test] - }; - let dal_crate_id = if inside_dal_crate { - quote! { crate } - } else { - quote! { zksync_dal } - }; - - let Some(pool_arg) = sig.inputs.pop() else { - let msg = format!( - "DB test function must take one or two arguments of type `{}`", - TYPE_NAME - ); - return Err(syn::Error::new_spanned(&sig.inputs, msg)); - }; - - let FnArg::Typed(pool_arg) = pool_arg.value() else { - let msg = "Pool argument must be typed"; - return Err(syn::Error::new_spanned(&pool_arg, msg)); - }; - let main_pool_arg_name = &pool_arg.pat; - let main_pool_arg_type = &pool_arg.ty; - - let prover_pool_arg = sig.inputs.pop(); - let pools_tokens = if let Some(pool_arg) = prover_pool_arg { - let FnArg::Typed(prover_pool_arg) = pool_arg.value() else { - let msg = "Pool argument must be typed"; - return Err(syn::Error::new_spanned(&pool_arg, msg)); - }; - let prover_pool_arg_name = &prover_pool_arg.pat; - let prover_pool_arg_type = &prover_pool_arg.ty; - quote! { - let __test_main_pool = #dal_crate_id::connection::TestPool::new().await; - let __test_prover_pool = #dal_crate_id::connection::TestPool::new().await; - let #main_pool_arg_name: #main_pool_arg_type = #dal_crate_id::ConnectionPool::Test(__test_main_pool); - let #prover_pool_arg_name: #prover_pool_arg_type = #dal_crate_id::ConnectionPool::Test(__test_prover_pool); - } - } else { - quote! { - let __test_main_pool = #dal_crate_id::connection::TestPool::new().await; - let #main_pool_arg_name: #main_pool_arg_type = #dal_crate_id::ConnectionPool::Test(__test_main_pool); - } - }; - let result = quote! { - #header - #(#attrs)* - #vis #sig { - #rt.enable_all() - .build() - .unwrap() - .block_on(async { - #pools_tokens - { - #body - } - }) - } - }; - - Ok(result.into()) -} - -#[proc_macro_attribute] -pub fn db_test(raw_args: TokenStream, item: TokenStream) -> TokenStream { - let input = syn::parse_macro_input!(item as syn::ItemFn); - let args = syn::parse_macro_input!(raw_args as Args); - - // There may be only one argument, and it should match the exact expected value. - if args.vars.len() > 1 || (args.vars.len() == 1 && args.vars[0] != DAL_CRATE_MARKER_ARG) { - let msg = format!("only '{DAL_CRATE_MARKER_ARG}' argument is supported"); - return syn::Error::new_spanned(&args.vars[0], msg) - .to_compile_error() - .into(); - } - let inside_dal_crate = args - .vars - .first() - .map(|arg| arg == DAL_CRATE_MARKER_ARG) - .unwrap_or(false); - - for attr in &input.attrs { - if attr.path.is_ident("test") { - let msg = "second test attribute is supplied"; - return syn::Error::new_spanned(attr, msg).to_compile_error().into(); - } - } - - parse_knobs(input, inside_dal_crate).unwrap_or_else(|e| e.to_compile_error().into()) -} diff --git a/core/lib/state/Cargo.toml b/core/lib/state/Cargo.toml index 6d4c3ab1c1a8..b613266a650f 100644 --- a/core/lib/state/Cargo.toml +++ b/core/lib/state/Cargo.toml @@ -23,7 +23,5 @@ tracing = "0.1" itertools = "0.10.3" [dev-dependencies] -db_test_macro = { path = "../db_test_macro" } - rand = "0.8.5" tempfile = "3.0.2" diff --git a/core/lib/state/src/postgres/tests.rs b/core/lib/state/src/postgres/tests.rs index 740659045fd4..213360bb73de 100644 --- a/core/lib/state/src/postgres/tests.rs +++ b/core/lib/state/src/postgres/tests.rs @@ -8,7 +8,6 @@ use rand::{ use rand::rngs::StdRng; use std::{collections::HashMap, mem}; -use db_test_macro::db_test; use zksync_dal::ConnectionPool; use zksync_types::StorageLog; @@ -126,8 +125,9 @@ fn test_postgres_storage_basics( } } -#[db_test] -async fn postgres_storage_basics(pool: ConnectionPool) { +#[tokio::test] +async fn postgres_storage_basics() { + let pool = ConnectionPool::test_pool().await; tokio::task::spawn_blocking(move || { test_postgres_storage_basics(&pool, Handle::current(), false); }) @@ -135,8 +135,9 @@ async fn postgres_storage_basics(pool: ConnectionPool) { .unwrap(); } -#[db_test] -async fn postgres_storage_with_initial_writes_cache(pool: ConnectionPool) { +#[tokio::test] +async fn postgres_storage_with_initial_writes_cache() { + let pool = ConnectionPool::test_pool().await; tokio::task::spawn_blocking(move || { test_postgres_storage_basics(&pool, Handle::current(), true); }) @@ -190,8 +191,9 @@ fn test_postgres_storage_after_sealing_miniblock( } } -#[db_test] -async fn postgres_storage_after_sealing_miniblock(pool: ConnectionPool) { +#[tokio::test] +async fn postgres_storage_after_sealing_miniblock() { + let pool = ConnectionPool::test_pool().await; tokio::task::spawn_blocking(move || { println!("Considering new L1 batch"); test_postgres_storage_after_sealing_miniblock(&pool, Handle::current(), true); @@ -242,8 +244,9 @@ fn test_factory_deps_cache(pool: &ConnectionPool, rt_handle: Handle) { assert_eq!(caches.factory_deps.get(&zero_addr), Some(vec![1, 2, 3])); } -#[db_test] -async fn using_factory_deps_cache(pool: ConnectionPool) { +#[tokio::test] +async fn using_factory_deps_cache() { + let pool = ConnectionPool::test_pool().await; let handle = Handle::current(); tokio::task::spawn_blocking(move || test_factory_deps_cache(&pool, handle)) .await @@ -347,8 +350,9 @@ fn test_initial_writes_cache(pool: &ConnectionPool, rt_handle: Handle) { assert!(storage.is_write_initial(&non_existing_key)); } -#[db_test] -async fn using_initial_writes_cache(pool: ConnectionPool) { +#[tokio::test] +async fn using_initial_writes_cache() { + let pool = ConnectionPool::test_pool().await; let handle = Handle::current(); tokio::task::spawn_blocking(move || test_initial_writes_cache(&pool, handle)) .await @@ -489,8 +493,9 @@ fn test_values_cache(pool: &ConnectionPool, rt_handle: Handle) { assert_final_cache(); } -#[db_test] -async fn using_values_cache(pool: ConnectionPool) { +#[tokio::test] +async fn using_values_cache() { + let pool = ConnectionPool::test_pool().await; let handle = Handle::current(); tokio::task::spawn_blocking(move || test_values_cache(&pool, handle)) .await @@ -579,9 +584,10 @@ fn mini_fuzz_values_cache_inner(rng: &mut impl Rng, pool: &ConnectionPool, mut r } } -#[db_test] -async fn mini_fuzz_values_cache(pool: ConnectionPool) { +#[tokio::test] +async fn mini_fuzz_values_cache() { const RNG_SEED: u64 = 123; + let pool = ConnectionPool::test_pool().await; let handle = Handle::current(); let mut rng = StdRng::seed_from_u64(RNG_SEED); diff --git a/core/lib/state/src/rocksdb/mod.rs b/core/lib/state/src/rocksdb/mod.rs index b871e9e9f035..c6a1e1243da8 100644 --- a/core/lib/state/src/rocksdb/mod.rs +++ b/core/lib/state/src/rocksdb/mod.rs @@ -493,7 +493,6 @@ impl ReadStorage for RocksdbStorage { #[cfg(test)] mod tests { - use db_test_macro::db_test; use tempfile::TempDir; use super::*; @@ -539,8 +538,9 @@ mod tests { } } - #[db_test] - async fn rocksdb_storage_syncing_with_postgres(pool: ConnectionPool) { + #[tokio::test] + async fn rocksdb_storage_syncing_with_postgres() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); prepare_postgres(&mut conn).await; let storage_logs = gen_storage_logs(20..40); @@ -570,8 +570,9 @@ mod tests { .await; } - #[db_test] - async fn rocksdb_storage_revert(pool: ConnectionPool) { + #[tokio::test] + async fn rocksdb_storage_revert() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); prepare_postgres(&mut conn).await; let storage_logs = gen_storage_logs(20..40); @@ -641,8 +642,9 @@ mod tests { } } - #[db_test] - async fn rocksdb_enum_index_migration(pool: ConnectionPool) { + #[tokio::test] + async fn rocksdb_enum_index_migration() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); prepare_postgres(&mut conn).await; let storage_logs = gen_storage_logs(20..40); diff --git a/core/lib/zksync_core/Cargo.toml b/core/lib/zksync_core/Cargo.toml index 8ee2e2aad92f..4d84757b6fa1 100644 --- a/core/lib/zksync_core/Cargo.toml +++ b/core/lib/zksync_core/Cargo.toml @@ -84,7 +84,6 @@ actix-web = "4.0.0-beta.8" tracing = "0.1.26" [dev-dependencies] -db_test_macro = { path = "../db_test_macro" } assert_matches = "1.5" zksync_test_account = { path = "../test_account" } diff --git a/core/lib/zksync_core/src/api_server/tree/tests.rs b/core/lib/zksync_core/src/api_server/tree/tests.rs index ea057b8bcc42..2f90b9fabdf0 100644 --- a/core/lib/zksync_core/src/api_server/tree/tests.rs +++ b/core/lib/zksync_core/src/api_server/tree/tests.rs @@ -4,7 +4,6 @@ use tempfile::TempDir; use std::net::Ipv4Addr; -use db_test_macro::db_test; use zksync_dal::ConnectionPool; use super::*; @@ -12,8 +11,10 @@ use crate::metadata_calculator::tests::{ gen_storage_logs, reset_db_state, run_calculator, setup_calculator, }; -#[db_test] -async fn merkle_tree_api(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn merkle_tree_api() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (calculator, _) = setup_calculator(temp_dir.path(), &pool).await; let api_addr = (Ipv4Addr::LOCALHOST, 0).into(); diff --git a/core/lib/zksync_core/src/api_server/web3/tests.rs b/core/lib/zksync_core/src/api_server/web3/tests.rs index 6d6ac9389a61..5fde6cc3aebf 100644 --- a/core/lib/zksync_core/src/api_server/web3/tests.rs +++ b/core/lib/zksync_core/src/api_server/web3/tests.rs @@ -2,7 +2,6 @@ use tokio::sync::watch; use std::{sync::Arc, time::Instant}; -use db_test_macro::db_test; use zksync_config::configs::{ api::Web3JsonRpcConfig, chain::{NetworkConfig, StateKeeperConfig}, @@ -101,8 +100,9 @@ pub(crate) async fn spawn_http_server( .expect("Failed spawning JSON-RPC server") } -#[db_test] -async fn http_server_can_start(pool: ConnectionPool) { +#[tokio::test] +async fn http_server_can_start() { + let pool = ConnectionPool::test_pool().await; let network_config = NetworkConfig::from_env().unwrap(); let mut storage = pool.access_storage().await.unwrap(); if storage.blocks_dal().is_genesis_needed().await.unwrap() { diff --git a/core/lib/zksync_core/src/eth_sender/tests.rs b/core/lib/zksync_core/src/eth_sender/tests.rs index d08c22f185bd..25b54830eaf5 100644 --- a/core/lib/zksync_core/src/eth_sender/tests.rs +++ b/core/lib/zksync_core/src/eth_sender/tests.rs @@ -1,7 +1,6 @@ use assert_matches::assert_matches; use std::sync::{atomic::Ordering, Arc}; -use db_test_macro::db_test; use zksync_config::{ configs::eth_sender::{ProofSendingMode, SenderConfig}, ContractsConfig, ETHSenderConfig, GasAdjusterConfig, @@ -119,8 +118,8 @@ impl EthSenderTester { } } - async fn storage(&self) -> StorageProcessor<'static> { - self.conn.access_test_storage().await + async fn storage(&self) -> StorageProcessor<'_> { + self.conn.access_storage().await.unwrap() } async fn get_block_numbers(&self) -> L1BlockNumbers { @@ -131,8 +130,9 @@ impl EthSenderTester { } // Tests that we send multiple transactions and confirm them all in one iteration. -#[db_test] -async fn confirm_many(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn confirm_many() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![10; 100], false).await; let mut hashes = vec![]; @@ -140,12 +140,15 @@ async fn confirm_many(connection_pool: ConnectionPool) -> anyhow::Result<()> { for _ in 0..5 { let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &DUMMY_OPERATION) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &DUMMY_OPERATION, + ) .await?; let hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &tx, 0, L1BlockNumber(tester.gateway.block_number("").await?.as_u32()), @@ -176,7 +179,7 @@ async fn confirm_many(connection_pool: ConnectionPool) -> anyhow::Result<()> { let to_resend = tester .manager .monitor_inflight_transactions( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), tester.get_block_numbers().await, ) .await?; @@ -200,8 +203,9 @@ async fn confirm_many(connection_pool: ConnectionPool) -> anyhow::Result<()> { } // Tests that we resend first unmined transaction every block with an increased gas price. -#[db_test] -async fn resend_each_block(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn resend_each_block() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![7, 6, 5, 5, 5, 2, 1], false).await; // after this, median should be 6 @@ -211,12 +215,20 @@ async fn resend_each_block(connection_pool: ConnectionPool) -> anyhow::Result<() let block = L1BlockNumber(tester.gateway.block_number("").await?.as_u32()); let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &DUMMY_OPERATION) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &DUMMY_OPERATION, + ) .await?; let hash = tester .manager - .send_eth_tx(&mut tester.storage().await, &tx, 0, block) + .send_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &tx, + 0, + block, + ) .await?; // check that we sent something and stored it in the db @@ -244,14 +256,17 @@ async fn resend_each_block(connection_pool: ConnectionPool) -> anyhow::Result<() let (to_resend, _) = tester .manager - .monitor_inflight_transactions(&mut tester.storage().await, block_numbers) + .monitor_inflight_transactions( + &mut tester.conn.access_storage().await.unwrap(), + block_numbers, + ) .await? .unwrap(); let resent_hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &to_resend, 1, block_numbers.latest, @@ -280,19 +295,23 @@ async fn resend_each_block(connection_pool: ConnectionPool) -> anyhow::Result<() // Tests that if transaction was mined, but not enough blocks has been mined since, // we won't mark it as confirmed but also won't resend it. -#[db_test] -async fn dont_resend_already_mined(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn dont_resend_already_mined() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![100; 100], false).await; let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &DUMMY_OPERATION) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &DUMMY_OPERATION, + ) .await .unwrap(); let hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &tx, 0, L1BlockNumber(tester.gateway.block_number("").await.unwrap().as_u32()), @@ -321,7 +340,7 @@ async fn dont_resend_already_mined(connection_pool: ConnectionPool) -> anyhow::R let to_resend = tester .manager .monitor_inflight_transactions( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), tester.get_block_numbers().await, ) .await?; @@ -344,22 +363,26 @@ async fn dont_resend_already_mined(connection_pool: ConnectionPool) -> anyhow::R Ok(()) } -#[db_test] -async fn three_scenarios(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn three_scenarios() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool.clone(), vec![100; 100], false).await; let mut hashes = vec![]; for _ in 0..3 { let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &DUMMY_OPERATION) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &DUMMY_OPERATION, + ) .await .unwrap(); let hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &tx, 0, L1BlockNumber(tester.gateway.block_number("").await.unwrap().as_u32()), @@ -386,7 +409,7 @@ async fn three_scenarios(connection_pool: ConnectionPool) -> anyhow::Result<()> let (to_resend, _) = tester .manager .monitor_inflight_transactions( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), tester.get_block_numbers().await, ) .await? @@ -411,20 +434,24 @@ async fn three_scenarios(connection_pool: ConnectionPool) -> anyhow::Result<()> } #[should_panic(expected = "We can't operate after tx fail")] -#[db_test] -async fn failed_eth_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn failed_eth_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool.clone(), vec![100; 100], false).await; let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &DUMMY_OPERATION) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &DUMMY_OPERATION, + ) .await .unwrap(); let hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &tx, 0, L1BlockNumber(tester.gateway.block_number("").await.unwrap().as_u32()), @@ -440,7 +467,7 @@ async fn failed_eth_tx(connection_pool: ConnectionPool) { tester .manager .monitor_inflight_transactions( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), tester.get_block_numbers().await, ) .await @@ -478,8 +505,9 @@ fn l1_batch_with_metadata(header: L1BatchHeader) -> L1BatchWithMetadata { } } -#[db_test] -async fn correct_order_for_confirmations(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn correct_order_for_confirmations() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![100; 100], true).await; insert_genesis_protocol_version(&tester).await; let genesis_l1_batch = insert_l1_batch(&tester, L1BatchNumber(0)).await; @@ -538,8 +566,9 @@ async fn correct_order_for_confirmations(connection_pool: ConnectionPool) -> any Ok(()) } -#[db_test] -async fn skipped_l1_batch_at_the_start(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn skipped_l1_batch_at_the_start() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![100; 100], true).await; insert_genesis_protocol_version(&tester).await; let genesis_l1_batch = insert_l1_batch(&tester, L1BatchNumber(0)).await; @@ -630,8 +659,9 @@ async fn skipped_l1_batch_at_the_start(connection_pool: ConnectionPool) -> anyho Ok(()) } -#[db_test] -async fn skipped_l1_batch_in_the_middle(connection_pool: ConnectionPool) -> anyhow::Result<()> { +#[tokio::test] +async fn skipped_l1_batch_in_the_middle() -> anyhow::Result<()> { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![100; 100], true).await; insert_genesis_protocol_version(&tester).await; let genesis_l1_batch = insert_l1_batch(&tester, L1BatchNumber(0)).await; @@ -716,8 +746,9 @@ async fn skipped_l1_batch_in_the_middle(connection_pool: ConnectionPool) -> anyh Ok(()) } -#[db_test] -async fn test_parse_multicall_data(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_parse_multicall_data() { + let connection_pool = ConnectionPool::test_pool().await; let tester = EthSenderTester::new(connection_pool, vec![100; 100], false).await; let original_correct_form_data = Token::Array(vec![ @@ -795,8 +826,9 @@ async fn test_parse_multicall_data(connection_pool: ConnectionPool) { } } -#[db_test] -async fn get_multicall_data(connection_pool: ConnectionPool) { +#[tokio::test] +async fn get_multicall_data() { + let connection_pool = ConnectionPool::test_pool().await; let mut tester = EthSenderTester::new(connection_pool, vec![100; 100], false).await; let multicall_data = tester.aggregator.get_multicall_data(&tester.gateway).await; assert!(multicall_data.is_ok()); @@ -889,14 +921,17 @@ async fn send_operation( ) -> H256 { let tx = tester .aggregator - .save_eth_tx(&mut tester.storage().await, &aggregated_operation) + .save_eth_tx( + &mut tester.conn.access_storage().await.unwrap(), + &aggregated_operation, + ) .await .unwrap(); let hash = tester .manager .send_eth_tx( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), &tx, 0, L1BlockNumber(tester.gateway.block_number("").await.unwrap().as_u32()), @@ -919,7 +954,7 @@ async fn confirm_tx(tester: &mut EthSenderTester, hash: H256) { tester .manager .monitor_inflight_transactions( - &mut tester.storage().await, + &mut tester.conn.access_storage().await.unwrap(), tester.get_block_numbers().await, ) .await diff --git a/core/lib/zksync_core/src/eth_watch/tests.rs b/core/lib/zksync_core/src/eth_watch/tests.rs index 01fb83b98c01..cf68b191af66 100644 --- a/core/lib/zksync_core/src/eth_watch/tests.rs +++ b/core/lib/zksync_core/src/eth_watch/tests.rs @@ -4,7 +4,6 @@ use std::sync::Arc; use tokio::sync::RwLock; -use db_test_macro::db_test; use zksync_contracts::zksync_contract; use zksync_dal::{ConnectionPool, StorageProcessor}; use zksync_types::protocol_version::{ProtocolUpgradeTx, ProtocolUpgradeTxCommonData}; @@ -202,8 +201,9 @@ fn build_upgrade_tx(id: ProtocolVersionId, eth_block: u64) -> ProtocolUpgradeTx } } -#[db_test] -async fn test_normal_operation_l1_txs(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_normal_operation_l1_txs() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -216,7 +216,7 @@ async fn test_normal_operation_l1_txs(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_transactions(&[build_l1_tx(0, 10), build_l1_tx(1, 14), build_l1_tx(2, 18)]) .await; @@ -249,8 +249,9 @@ async fn test_normal_operation_l1_txs(connection_pool: ConnectionPool) { assert_eq!(db_tx.common_data.serial_id.0, 2); } -#[db_test] -async fn test_normal_operation_upgrades(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_normal_operation_upgrades() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -263,7 +264,7 @@ async fn test_normal_operation_upgrades(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_diamond_upgrades(&[ ( @@ -309,8 +310,9 @@ async fn test_normal_operation_upgrades(connection_pool: ConnectionPool) { assert_eq!(tx.common_data.upgrade_id, ProtocolVersionId::next()); } -#[db_test] -async fn test_gap_in_upgrades(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_gap_in_upgrades() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -323,7 +325,7 @@ async fn test_gap_in_upgrades(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_diamond_upgrades(&[( ProtocolUpgrade { @@ -347,8 +349,9 @@ async fn test_gap_in_upgrades(connection_pool: ConnectionPool) { assert_eq!(db_ids[1], next_version); } -#[db_test] -async fn test_normal_operation_governance_upgrades(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_normal_operation_governance_upgrades() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -361,7 +364,7 @@ async fn test_normal_operation_governance_upgrades(connection_pool: ConnectionPo ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_governance_upgrades(&[ ( @@ -407,9 +410,10 @@ async fn test_normal_operation_governance_upgrades(connection_pool: ConnectionPo assert_eq!(tx.common_data.upgrade_id, ProtocolVersionId::next()); } -#[db_test] +#[tokio::test] #[should_panic] -async fn test_gap_in_single_batch(connection_pool: ConnectionPool) { +async fn test_gap_in_single_batch() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -422,7 +426,7 @@ async fn test_gap_in_single_batch(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_transactions(&[ build_l1_tx(0, 10), @@ -436,9 +440,10 @@ async fn test_gap_in_single_batch(connection_pool: ConnectionPool) { watcher.loop_iteration(&mut storage).await.unwrap(); } -#[db_test] +#[tokio::test] #[should_panic] -async fn test_gap_between_batches(connection_pool: ConnectionPool) { +async fn test_gap_between_batches() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -451,7 +456,7 @@ async fn test_gap_between_batches(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_transactions(&[ // this goes to the first batch @@ -471,8 +476,9 @@ async fn test_gap_between_batches(connection_pool: ConnectionPool) { watcher.loop_iteration(&mut storage).await.unwrap(); } -#[db_test] -async fn test_overlapping_batches(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_overlapping_batches() { + let connection_pool = ConnectionPool::test_pool().await; setup_db(&connection_pool).await; let mut client = FakeEthClient::new(); @@ -485,7 +491,7 @@ async fn test_overlapping_batches(connection_pool: ConnectionPool) { ) .await; - let mut storage = connection_pool.access_test_storage().await; + let mut storage = connection_pool.access_storage().await.unwrap(); client .add_transactions(&[ // this goes to the first batch @@ -761,8 +767,9 @@ fn upgrade_into_diamond_cut(upgrade: ProtocolUpgrade) -> Token { async fn setup_db(connection_pool: &ConnectionPool) { connection_pool - .access_test_storage() + .access_storage() .await + .unwrap() .protocol_versions_dal() .save_protocol_version_with_tx(ProtocolVersion { id: (ProtocolVersionId::latest() as u16 - 1).try_into().unwrap(), diff --git a/core/lib/zksync_core/src/genesis.rs b/core/lib/zksync_core/src/genesis.rs index ec16bd4d17f9..b9f2a7af7805 100644 --- a/core/lib/zksync_core/src/genesis.rs +++ b/core/lib/zksync_core/src/genesis.rs @@ -394,14 +394,14 @@ pub(crate) async fn save_genesis_l1_batch_metadata( #[cfg(test)] mod tests { - use db_test_macro::db_test; use zksync_dal::ConnectionPool; use zksync_types::system_contracts::get_system_smart_contracts; use super::*; - #[db_test] - async fn running_genesis(pool: ConnectionPool) { + #[tokio::test] + async fn running_genesis() { + let pool = ConnectionPool::test_pool().await; let mut conn = pool.access_storage().await.unwrap(); conn.blocks_dal().delete_genesis().await.unwrap(); @@ -432,8 +432,9 @@ mod tests { .unwrap(); } - #[db_test] - async fn running_genesis_with_big_chain_id(pool: ConnectionPool) { + #[tokio::test] + async fn running_genesis_with_big_chain_id() { + let pool = ConnectionPool::test_pool().await; let mut conn: StorageProcessor<'_> = pool.access_storage().await.unwrap(); conn.blocks_dal().delete_genesis().await.unwrap(); diff --git a/core/lib/zksync_core/src/lib.rs b/core/lib/zksync_core/src/lib.rs index 23e631dec5ad..b52a07845632 100644 --- a/core/lib/zksync_core/src/lib.rs +++ b/core/lib/zksync_core/src/lib.rs @@ -1308,13 +1308,3 @@ async fn circuit_breakers_for_components( } Ok(circuit_breakers) } - -#[tokio::test] -async fn test_house_keeper_components_get_added() { - let (core_task_handles, _, _, _) = initialize_components(vec![Component::Housekeeper], false) - .await - .unwrap(); - // circuit-breaker, prometheus-exporter components are run, irrespective of other components. - let always_running_component_count = 2; - assert_eq!(15, core_task_handles.len() - always_running_component_count); -} diff --git a/core/lib/zksync_core/src/metadata_calculator/helpers.rs b/core/lib/zksync_core/src/metadata_calculator/helpers.rs index 34e187d85977..ffd87b92d164 100644 --- a/core/lib/zksync_core/src/metadata_calculator/helpers.rs +++ b/core/lib/zksync_core/src/metadata_calculator/helpers.rs @@ -346,7 +346,6 @@ impl L1BatchWithLogs { mod tests { use tempfile::TempDir; - use db_test_macro::db_test; use zksync_dal::ConnectionPool; use zksync_types::{proofs::PrepareBasicCircuitsJob, L2ChainId, StorageKey, StorageLogKind}; @@ -418,8 +417,9 @@ mod tests { } } - #[db_test] - async fn loaded_logs_equivalence_basics(pool: ConnectionPool) { + #[tokio::test] + async fn loaded_logs_equivalence_basics() { + let pool = ConnectionPool::test_pool().await; ensure_genesis_state( &mut pool.access_storage().await.unwrap(), L2ChainId::from(270), @@ -442,8 +442,9 @@ mod tests { } } - #[db_test] - async fn loaded_logs_equivalence_with_zero_no_op_logs(pool: ConnectionPool) { + #[tokio::test] + async fn loaded_logs_equivalence_with_zero_no_op_logs() { + let pool = ConnectionPool::test_pool().await; let mut storage = pool.access_storage().await.unwrap(); ensure_genesis_state(&mut storage, L2ChainId::from(270), &GenesisParams::mock()) .await @@ -531,8 +532,9 @@ mod tests { } } - #[db_test] - async fn loaded_logs_equivalence_with_non_zero_no_op_logs(pool: ConnectionPool) { + #[tokio::test] + async fn loaded_logs_equivalence_with_non_zero_no_op_logs() { + let pool = ConnectionPool::test_pool().await; let mut storage = pool.access_storage().await.unwrap(); ensure_genesis_state(&mut storage, L2ChainId::from(270), &GenesisParams::mock()) .await @@ -577,8 +579,9 @@ mod tests { } } - #[db_test] - async fn loaded_logs_equivalence_with_protective_reads(pool: ConnectionPool) { + #[tokio::test] + async fn loaded_logs_equivalence_with_protective_reads() { + let pool = ConnectionPool::test_pool().await; let mut storage = pool.access_storage().await.unwrap(); ensure_genesis_state(&mut storage, L2ChainId::from(270), &GenesisParams::mock()) .await diff --git a/core/lib/zksync_core/src/metadata_calculator/tests.rs b/core/lib/zksync_core/src/metadata_calculator/tests.rs index 80fc9b112b7d..490022b5d5dc 100644 --- a/core/lib/zksync_core/src/metadata_calculator/tests.rs +++ b/core/lib/zksync_core/src/metadata_calculator/tests.rs @@ -1,5 +1,4 @@ use assert_matches::assert_matches; -use db_test_macro::db_test; use itertools::Itertools; use tempfile::TempDir; use tokio::sync::{mpsc, watch}; @@ -38,8 +37,10 @@ where } } -#[db_test] -async fn genesis_creation(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn genesis_creation() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (calculator, _) = setup_calculator(temp_dir.path(), &pool).await; @@ -53,8 +54,11 @@ async fn genesis_creation(pool: ConnectionPool, prover_pool: ConnectionPool) { // TODO (SMA-1726): Restore tests for tree backup mode -#[db_test] -async fn basic_workflow(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn basic_workflow() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; + let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (calculator, object_store) = setup_calculator(temp_dir.path(), &pool).await; @@ -96,8 +100,10 @@ async fn expected_tree_hash(pool: &ConnectionPool) -> H256 { ZkSyncTree::process_genesis_batch(&all_logs).root_hash } -#[db_test] -async fn status_receiver_has_correct_states(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn status_receiver_has_correct_states() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (mut calculator, _) = setup_calculator(temp_dir.path(), &pool).await; @@ -143,8 +149,11 @@ async fn status_receiver_has_correct_states(pool: ConnectionPool, prover_pool: C ); } -#[db_test] -async fn multi_l1_batch_workflow(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn multi_l1_batch_workflow() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; + // Collect all storage logs in a single L1 batch let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (calculator, _) = setup_calculator(temp_dir.path(), &pool).await; @@ -177,11 +186,11 @@ async fn multi_l1_batch_workflow(pool: ConnectionPool, prover_pool: ConnectionPo } } -#[db_test] -async fn running_metadata_calculator_with_additional_blocks( - pool: ConnectionPool, - prover_pool: ConnectionPool, -) { +#[tokio::test] +async fn running_metadata_calculator_with_additional_blocks() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; + let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let calculator = setup_lightweight_calculator(temp_dir.path(), &pool).await; reset_db_state(&pool, 5).await; @@ -229,8 +238,10 @@ async fn running_metadata_calculator_with_additional_blocks( assert_eq!(root_hash_for_full_tree, updated_root_hash); } -#[db_test] -async fn shutting_down_calculator(pool: ConnectionPool, prover_pool: ConnectionPool) { +#[tokio::test] +async fn shutting_down_calculator() { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let (db_config, mut operation_config) = create_config(temp_dir.path()); operation_config.delay_interval = 30_000; // ms; chosen to be larger than `RUN_TIMEOUT` @@ -256,11 +267,11 @@ async fn shutting_down_calculator(pool: ConnectionPool, prover_pool: ConnectionP } async fn test_postgres_backup_recovery( - pool: ConnectionPool, - prover_pool: ConnectionPool, sleep_between_batches: bool, insert_batch_without_metadata: bool, ) { + let pool = ConnectionPool::test_pool().await; + let prover_pool = ConnectionPool::test_pool().await; let temp_dir = TempDir::new().expect("failed get temporary directory for RocksDB"); let calculator = setup_lightweight_calculator(temp_dir.path(), &pool).await; reset_db_state(&pool, 5).await; @@ -309,12 +320,13 @@ async fn test_postgres_backup_recovery( // Re-insert L1 batches to the storage after recovery. let mut storage = pool.access_storage().await.unwrap(); for batch_header in &removed_batches { - storage - .blocks_dal() + let mut txn = storage.start_transaction().await.unwrap(); + txn.blocks_dal() .insert_l1_batch(batch_header, &[], BlockGasCount::default(), &[], &[]) .await .unwrap(); - insert_initial_writes_for_batch(&mut storage, batch_header.number).await; + insert_initial_writes_for_batch(&mut txn, batch_header.number).await; + txn.commit().await.unwrap(); if sleep_between_batches { tokio::time::sleep(Duration::from_millis(100)).await; } @@ -339,25 +351,19 @@ async fn test_postgres_backup_recovery( .unwrap(); } -#[db_test] -async fn postgres_backup_recovery(pool: ConnectionPool, prover_pool: ConnectionPool) { - test_postgres_backup_recovery(pool, prover_pool, false, false).await; +#[tokio::test] +async fn postgres_backup_recovery() { + test_postgres_backup_recovery(false, false).await; } -#[db_test] -async fn postgres_backup_recovery_with_delay_between_batches( - pool: ConnectionPool, - prover_pool: ConnectionPool, -) { - test_postgres_backup_recovery(pool, prover_pool, true, false).await; +#[tokio::test] +async fn postgres_backup_recovery_with_delay_between_batches() { + test_postgres_backup_recovery(true, false).await; } -#[db_test] -async fn postgres_backup_recovery_with_excluded_metadata( - pool: ConnectionPool, - prover_pool: ConnectionPool, -) { - test_postgres_backup_recovery(pool, prover_pool, false, true).await; +#[tokio::test] +async fn postgres_backup_recovery_with_excluded_metadata() { + test_postgres_backup_recovery(false, true).await; } pub(crate) async fn setup_calculator( @@ -468,6 +474,7 @@ pub(super) async fn extend_db_state( storage: &mut StorageProcessor<'_>, new_logs: impl IntoIterator>, ) { + let mut storage = storage.start_transaction().await.unwrap(); let next_l1_batch = storage .blocks_dal() .get_sealed_l1_batch_number() @@ -528,8 +535,9 @@ pub(super) async fn extend_db_state( .mark_miniblocks_as_executed_in_l1_batch(batch_number) .await .unwrap(); - insert_initial_writes_for_batch(storage, batch_number).await; + insert_initial_writes_for_batch(&mut storage, batch_number).await; } + storage.commit().await.unwrap(); } async fn insert_initial_writes_for_batch( @@ -629,8 +637,9 @@ async fn remove_l1_batches( batch_headers } -#[db_test] -async fn deduplication_works_as_expected(pool: ConnectionPool) { +#[tokio::test] +async fn deduplication_works_as_expected() { + let pool = ConnectionPool::test_pool().await; let mut storage = pool.access_storage().await.unwrap(); ensure_genesis_state(&mut storage, L2ChainId::from(270), &GenesisParams::mock()) .await diff --git a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs index bae8b81fc134..05a8220bb832 100644 --- a/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/batch_executor/tests/mod.rs @@ -1,5 +1,4 @@ use assert_matches::assert_matches; -use db_test_macro::db_test; use zksync_dal::ConnectionPool; use zksync_types::PriorityOpId; @@ -33,8 +32,9 @@ fn assert_reverted(execution_result: &TxExecutionResult) { } /// Checks that we can successfully execute a single L2 tx in batch executor. -#[db_test] -async fn execute_l2_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn execute_l2_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -49,8 +49,9 @@ async fn execute_l2_tx(connection_pool: ConnectionPool) { } /// Checks that we can successfully execute a single L1 tx in batch executor. -#[db_test] -async fn execute_l1_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn execute_l1_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -65,8 +66,9 @@ async fn execute_l1_tx(connection_pool: ConnectionPool) { } /// Checks that we can successfully execute a single L2 tx and a single L1 tx in batch executor. -#[db_test] -async fn execute_l2_and_l1_txs(connection_pool: ConnectionPool) { +#[tokio::test] +async fn execute_l2_and_l1_txs() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -84,8 +86,9 @@ async fn execute_l2_and_l1_txs(connection_pool: ConnectionPool) { } /// Checks that we can successfully rollback the transaction and execute it once again. -#[db_test] -async fn rollback(connection_pool: ConnectionPool) { +#[tokio::test] +async fn rollback() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -127,8 +130,9 @@ async fn rollback(connection_pool: ConnectionPool) { } /// Checks that incorrect transactions are marked as rejected. -#[db_test] -async fn reject_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn reject_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -142,8 +146,9 @@ async fn reject_tx(connection_pool: ConnectionPool) { } /// Checks that tx with too big gas limit is correctly rejected. -#[db_test] -async fn too_big_gas_limit(connection_pool: ConnectionPool) { +#[tokio::test] +async fn too_big_gas_limit() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -188,8 +193,9 @@ async fn too_big_gas_limit(connection_pool: ConnectionPool) { } /// Checks that we can't execute the same transaction twice. -#[db_test] -async fn tx_cant_be_reexecuted(connection_pool: ConnectionPool) { +#[tokio::test] +async fn tx_cant_be_reexecuted() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -207,8 +213,9 @@ async fn tx_cant_be_reexecuted(connection_pool: ConnectionPool) { } /// Checks that we can deploy and call the loadnext contract. -#[db_test] -async fn deploy_and_call_loadtest(connection_pool: ConnectionPool) { +#[tokio::test] +async fn deploy_and_call_loadtest() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -232,8 +239,9 @@ async fn deploy_and_call_loadtest(connection_pool: ConnectionPool) { } /// Checks that a tx that is reverted by the VM still can be included into a batch. -#[db_test] -async fn execute_reverted_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn execute_reverted_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::new(connection_pool); @@ -258,8 +266,9 @@ async fn execute_reverted_tx(connection_pool: ConnectionPool) { /// Runs the batch executor through a semi-realistic basic scenario: /// a batch with different operations, both successful and not. -#[db_test] -async fn execute_realistic_scenario(connection_pool: ConnectionPool) { +#[tokio::test] +async fn execute_realistic_scenario() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let mut bob = Account::random(); @@ -307,8 +316,9 @@ async fn execute_realistic_scenario(connection_pool: ConnectionPool) { } /// Checks that we handle the bootloader out of gas error on execution phase. -#[db_test] -async fn bootloader_out_of_gas_for_any_tx(connection_pool: ConnectionPool) { +#[tokio::test] +async fn bootloader_out_of_gas_for_any_tx() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let tester = Tester::with_config( @@ -331,9 +341,10 @@ async fn bootloader_out_of_gas_for_any_tx(connection_pool: ConnectionPool) { } /// Checks that we can handle the bootloader out of gas error on tip phase. -#[db_test] +#[tokio::test] #[ignore] // This test fails. -async fn bootloader_tip_out_of_gas(connection_pool: ConnectionPool) { +async fn bootloader_tip_out_of_gas() { + let connection_pool = ConnectionPool::test_pool().await; let mut alice = Account::random(); let mut tester = Tester::new(connection_pool); diff --git a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs index db4083fe1407..e70964c4957f 100644 --- a/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs +++ b/core/lib/zksync_core/src/state_keeper/io/tests/mod.rs @@ -2,8 +2,6 @@ use futures::FutureExt; use std::time::Duration; -use db_test_macro::db_test; - use multivm::vm_latest::utils::fee::derive_base_fee_and_gas_per_pubdata; use zksync_contracts::BaseSystemContractsHashes; use zksync_dal::ConnectionPool; @@ -31,8 +29,9 @@ mod tester; use self::tester::Tester; /// Ensure that MempoolIO.filter is correctly initialized right after mempool initialization. -#[db_test] -async fn test_filter_initialization(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_filter_initialization() { + let connection_pool = ConnectionPool::test_pool().await; let tester = Tester::new(); // Genesis is needed for proper mempool initialization. @@ -44,8 +43,9 @@ async fn test_filter_initialization(connection_pool: ConnectionPool) { } /// Ensure that MempoolIO.filter is modified correctly if there is a pending batch upon mempool initialization. -#[db_test] -async fn test_filter_with_pending_batch(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_filter_with_pending_batch() { + let connection_pool = ConnectionPool::test_pool().await; let tester = Tester::new(); tester.genesis(&connection_pool).await; @@ -87,8 +87,9 @@ async fn test_filter_with_pending_batch(connection_pool: ConnectionPool) { } /// Ensure that MempoolIO.filter is modified correctly if there is no pending batch. -#[db_test] -async fn test_filter_with_no_pending_batch(connection_pool: ConnectionPool) { +#[tokio::test] +async fn test_filter_with_no_pending_batch() { + let connection_pool = ConnectionPool::test_pool().await; let tester = Tester::new(); tester.genesis(&connection_pool).await; @@ -157,34 +158,37 @@ async fn test_timestamps_are_distinct( assert!(batch_params.1.timestamp > prev_miniblock_timestamp); } -#[db_test] -async fn l1_batch_timestamp_basics(connection_pool: ConnectionPool) { +#[tokio::test] +async fn l1_batch_timestamp_basics() { + let connection_pool = ConnectionPool::test_pool().await; let current_timestamp = seconds_since_epoch(); test_timestamps_are_distinct(connection_pool, current_timestamp, false).await; } -#[db_test] -async fn l1_batch_timestamp_with_clock_skew(connection_pool: ConnectionPool) { +#[tokio::test] +async fn l1_batch_timestamp_with_clock_skew() { + let connection_pool = ConnectionPool::test_pool().await; let current_timestamp = seconds_since_epoch(); test_timestamps_are_distinct(connection_pool, current_timestamp + 2, false).await; } -#[db_test] -async fn l1_batch_timestamp_respects_prev_miniblock(connection_pool: ConnectionPool) { +#[tokio::test] +async fn l1_batch_timestamp_respects_prev_miniblock() { + let connection_pool = ConnectionPool::test_pool().await; let current_timestamp = seconds_since_epoch(); test_timestamps_are_distinct(connection_pool, current_timestamp, true).await; } -#[db_test] -async fn l1_batch_timestamp_respects_prev_miniblock_with_clock_skew( - connection_pool: ConnectionPool, -) { +#[tokio::test] +async fn l1_batch_timestamp_respects_prev_miniblock_with_clock_skew() { + let connection_pool = ConnectionPool::test_pool().await; let current_timestamp = seconds_since_epoch(); test_timestamps_are_distinct(connection_pool, current_timestamp + 2, true).await; } -#[db_test] -async fn processing_storage_logs_when_sealing_miniblock(connection_pool: ConnectionPool) { +#[tokio::test] +async fn processing_storage_logs_when_sealing_miniblock() { + let connection_pool = ConnectionPool::test_pool().await; let mut miniblock = MiniblockUpdates::new(0, 1, H256::zero(), 1, Some(ProtocolVersionId::latest())); @@ -277,8 +281,9 @@ async fn processing_storage_logs_when_sealing_miniblock(connection_pool: Connect } } -#[db_test] -async fn processing_events_when_sealing_miniblock(pool: ConnectionPool) { +#[tokio::test] +async fn processing_events_when_sealing_miniblock() { + let pool = ConnectionPool::test_pool().await; let l1_batch_number = L1BatchNumber(2); let mut miniblock = MiniblockUpdates::new(0, 1, H256::zero(), 1, Some(ProtocolVersionId::latest())); @@ -406,18 +411,21 @@ async fn test_miniblock_and_l1_batch_processing( assert!(l1_batch_header.is_finished); } -#[db_test] -async fn miniblock_and_l1_batch_processing(pool: ConnectionPool) { +#[tokio::test] +async fn miniblock_and_l1_batch_processing() { + let pool = ConnectionPool::test_pool().await; test_miniblock_and_l1_batch_processing(pool, 1).await; } -#[db_test] -async fn miniblock_and_l1_batch_processing_with_sync_sealer(pool: ConnectionPool) { +#[tokio::test] +async fn miniblock_and_l1_batch_processing_with_sync_sealer() { + let pool = ConnectionPool::test_pool().await; test_miniblock_and_l1_batch_processing(pool, 0).await; } -#[db_test] -async fn miniblock_sealer_handle_blocking(pool: ConnectionPool) { +#[tokio::test] +async fn miniblock_sealer_handle_blocking() { + let pool = ConnectionPool::test_pool().await; let (mut sealer, mut sealer_handle) = MiniblockSealer::new(pool, 1); // The first command should be successfully submitted immediately. @@ -469,8 +477,9 @@ async fn miniblock_sealer_handle_blocking(pool: ConnectionPool) { sealer_handle.wait_for_all_commands().await; } -#[db_test] -async fn miniblock_sealer_handle_parallel_processing(pool: ConnectionPool) { +#[tokio::test] +async fn miniblock_sealer_handle_parallel_processing() { + let pool = ConnectionPool::test_pool().await; let (mut sealer, mut sealer_handle) = MiniblockSealer::new(pool, 5); // 5 miniblock sealing commands can be submitted without blocking. @@ -494,8 +503,9 @@ async fn miniblock_sealer_handle_parallel_processing(pool: ConnectionPool) { } /// Ensure that subsequent miniblocks that belong to the same L1 batch have different timestamps -#[db_test] -async fn different_timestamp_for_miniblocks_in_same_batch(connection_pool: ConnectionPool) { +#[tokio::test] +async fn different_timestamp_for_miniblocks_in_same_batch() { + let connection_pool = ConnectionPool::test_pool().await; let tester = Tester::new(); // Genesis is needed for proper mempool initialization. diff --git a/core/lib/zksync_core/src/sync_layer/tests.rs b/core/lib/zksync_core/src/sync_layer/tests.rs index 02b68d7a8ed9..8fe61fe16e6d 100644 --- a/core/lib/zksync_core/src/sync_layer/tests.rs +++ b/core/lib/zksync_core/src/sync_layer/tests.rs @@ -9,7 +9,6 @@ use std::{ time::{Duration, Instant}, }; -use db_test_macro::db_test; use zksync_config::configs::chain::NetworkConfig; use zksync_contracts::{BaseSystemContractsHashes, SystemContractCode}; use zksync_dal::{ConnectionPool, StorageProcessor}; @@ -239,8 +238,9 @@ fn extract_tx_hashes<'a>(actions: impl IntoIterator) -> V .collect() } -#[db_test] -async fn external_io_basics(pool: ConnectionPool) { +#[tokio::test] +async fn external_io_basics() { + let pool = ConnectionPool::test_pool().await; let open_l1_batch = open_l1_batch(1, 1, 1); let tx = create_l2_transaction(10, 100); let tx_hash = tx.hash(); @@ -324,8 +324,9 @@ async fn run_state_keeper_with_multiple_miniblocks(pool: ConnectionPool) -> Vec< tx_hashes } -#[db_test] -async fn external_io_with_multiple_miniblocks(pool: ConnectionPool) { +#[tokio::test] +async fn external_io_with_multiple_miniblocks() { + let pool = ConnectionPool::test_pool().await; let tx_hashes = run_state_keeper_with_multiple_miniblocks(pool.clone()).await; assert_eq!(tx_hashes.len(), 8); @@ -417,8 +418,9 @@ async fn mock_l1_batch_hash_computation(pool: ConnectionPool, number: u32) { } } -#[db_test] -async fn external_io_with_multiple_l1_batches(pool: ConnectionPool) { +#[tokio::test] +async fn external_io_with_multiple_l1_batches() { + let pool = ConnectionPool::test_pool().await; let l1_batch = open_l1_batch(1, 1, 1); let first_tx = create_l2_transaction(10, 100); let first_tx_hash = first_tx.hash(); @@ -489,8 +491,9 @@ async fn external_io_with_multiple_l1_batches(pool: ConnectionPool) { assert_eq!(fictive_miniblock.l2_tx_count, 0); } -#[db_test] -async fn fetcher_basics(pool: ConnectionPool) { +#[tokio::test] +async fn fetcher_basics() { + let pool = ConnectionPool::test_pool().await; let mut storage = pool.access_storage().await.unwrap(); ensure_genesis(&mut storage).await; let fetcher_cursor = MainNodeFetcherCursor::new(&mut storage).await.unwrap(); @@ -567,8 +570,9 @@ async fn fetcher_basics(pool: ConnectionPool) { fetcher_task.await.unwrap().unwrap(); } -#[db_test] -async fn fetcher_with_real_server(pool: ConnectionPool) { +#[tokio::test] +async fn fetcher_with_real_server() { + let pool = ConnectionPool::test_pool().await; // Fill in transactions grouped in multiple miniblocks in the storage. let tx_hashes = run_state_keeper_with_multiple_miniblocks(pool.clone()).await; let mut tx_hashes = VecDeque::from(tx_hashes); diff --git a/docker-compose-runner.yml b/docker-compose-runner.yml index bcf01f1ffac0..7bd2fa5a43bd 100644 --- a/docker-compose-runner.yml +++ b/docker-compose-runner.yml @@ -28,6 +28,8 @@ services: - GITHUB_WORKSPACE=$GITHUB_WORKSPACE env_file: - ./.env + extra_hosts: + - "host:host-gateway" postgres: image: "postgres:14" ports: diff --git a/docker-compose-unit-tests.yml b/docker-compose-unit-tests.yml new file mode 100644 index 000000000000..8a1699019956 --- /dev/null +++ b/docker-compose-unit-tests.yml @@ -0,0 +1,11 @@ +version: '3.2' +name: unit_tests +services: + # An instance of postgres configured to execute Rust unit-tests, tuned for performance. + postgres: + image: "postgres:14" + command: "postgres -c fsync=false" + ports: + - "5433:5432" + environment: + - POSTGRES_HOST_AUTH_METHOD=trust diff --git a/etc/env/base/private.toml b/etc/env/base/private.toml index 2d77e3e805e1..60d5bfb2d9c8 100644 --- a/etc/env/base/private.toml +++ b/etc/env/base/private.toml @@ -2,7 +2,7 @@ # Values provided here are valid for the development infrastructure only. database_url="postgres://postgres@localhost/zksync_local" -test_database_url="postgres://postgres@localhost/zksync_local_test" +test_database_url="postgres://postgres@localhost:5433/zksync_local_test" [eth_sender.sender] # Set in env file for development, production, staging and testnet. diff --git a/etc/env/docker.toml b/etc/env/docker.toml index 0e5eadf1014e..c35560d416f8 100644 --- a/etc/env/docker.toml +++ b/etc/env/docker.toml @@ -1,5 +1,5 @@ database_url = "postgres://postgres@postgres/zksync_local" -test_database_url = "postgres://postgres@postgres/zksync_local_test" +test_database_url = "postgres://postgres@host:5433/zksync_local_test" # for loadtest l1_rpc_address = "http://geth:8545" diff --git a/etc/env/ext-node-docker.toml b/etc/env/ext-node-docker.toml index c51006336562..4db60a3c19d6 100644 --- a/etc/env/ext-node-docker.toml +++ b/etc/env/ext-node-docker.toml @@ -1,5 +1,5 @@ database_url = "postgres://postgres@postgres/zksync_local_ext_node" -test_database_url = "postgres://postgres@postgres/zksync_local_test_ext_node" +test_database_url = "postgres://postgres@host:5433/zksync_local_test_ext_node" database_pool_size = 50 zksync_action="dont_ask" diff --git a/etc/env/ext-node.toml b/etc/env/ext-node.toml index 59e8c791fb1b..697580d19386 100644 --- a/etc/env/ext-node.toml +++ b/etc/env/ext-node.toml @@ -2,7 +2,7 @@ # All the variables must be provided explicitly. # This is on purpose: if EN will accidentally depend on the main node env, it may cause problems. -test_database_url = "postgres://postgres@localhost/zksync_local_test_ext_node" +test_database_url = "postgres://postgres@localhost:5433/zksync_local_test_ext_node" database_url = "postgres://postgres@localhost/zksync_local_ext_node" database_pool_size = 50 zksync_action="dont_ask" diff --git a/infrastructure/zk/src/database.ts b/infrastructure/zk/src/database.ts index 0c8e0f2212a6..721ff734730a 100644 --- a/infrastructure/zk/src/database.ts +++ b/infrastructure/zk/src/database.ts @@ -9,11 +9,18 @@ export async function reset() { } export async function resetTest() { - const databaseUrl = process.env.DATABASE_URL as string; - process.env.DATABASE_URL = databaseUrl.replace('zksync_local', 'zksync_local_test'); + process.env.DATABASE_URL = process.env.TEST_DATABASE_URL; await utils.confirmAction(); - await drop(); + console.log('recreating postgres container for unit tests'); + await utils.spawn('docker compose -f docker-compose-unit-tests.yml down'); + await utils.spawn('docker compose -f docker-compose-unit-tests.yml up -d'); + await wait(100); + console.log('setting up a database template'); await setup(); + console.log('disallowing connections to the template'); + await utils.spawn( + `psql "${process.env.DATABASE_URL}" -c "update pg_database set datallowconn = false where datname = current_database()"` + ); } export async function drop() { @@ -59,7 +66,8 @@ export async function wait(tries: number = 4) { for (let i = 0; i < tries; i++) { const result = await utils.allowFail(utils.exec(`pg_isready -d "${process.env.DATABASE_URL}"`)); if (result !== null) return; // null means failure - await utils.sleep(5); + console.log(`waiting for postgres ${process.env.DATABASE_URL}`); + await utils.sleep(1); } await utils.exec(`pg_isready -d "${process.env.DATABASE_URL}"`); } diff --git a/infrastructure/zk/src/test/test.ts b/infrastructure/zk/src/test/test.ts index 1a017882495b..7bcdd813514b 100644 --- a/infrastructure/zk/src/test/test.ts +++ b/infrastructure/zk/src/test/test.ts @@ -33,7 +33,7 @@ export async function rust(options: string[]) { test_runner = 'cargo test'; } - let cmd = `${test_runner} --release ${options.join(' ')}`; + let cmd = `SQLX_OFFLINE=true ${test_runner} --release ${options.join(' ')}`; console.log(`running unit tests with '${cmd}'`); await utils.spawn(cmd);