From 2bf83c3e2c9d60df38fa2757b7d925fe1422d4ab Mon Sep 17 00:00:00 2001 From: ethanoroshiba Date: Fri, 22 Nov 2024 10:03:53 -0600 Subject: [PATCH 1/3] remove old tests, add missing blackbox tests --- crates/astria-composer/src/executor/tests.rs | 645 ------------------ crates/astria-composer/tests/blackbox/api.rs | 4 +- .../tests/blackbox/executor.rs | 224 ++++++ .../tests/blackbox/geth_collector.rs | 14 +- .../tests/blackbox/grpc_collector.rs | 8 +- .../blackbox/helper/mock_abci_sequencer.rs | 6 +- .../blackbox/helper/mock_grpc_sequencer.rs | 3 +- .../tests/blackbox/helper/mod.rs | 104 ++- crates/astria-composer/tests/blackbox/main.rs | 1 + crates/astria-test-utils/src/mock/geth.rs | 1 + 10 files changed, 343 insertions(+), 667 deletions(-) delete mode 100644 crates/astria-composer/src/executor/tests.rs create mode 100644 crates/astria-composer/tests/blackbox/executor.rs diff --git a/crates/astria-composer/src/executor/tests.rs b/crates/astria-composer/src/executor/tests.rs deleted file mode 100644 index 6698ff0519..0000000000 --- a/crates/astria-composer/src/executor/tests.rs +++ /dev/null @@ -1,645 +0,0 @@ -use std::{ - io::Write, - net::{ - IpAddr, - SocketAddr, - }, - sync::LazyLock, - time::Duration, -}; - -use astria_core::{ - generated::protocol::accounts::v1::NonceResponse, - primitive::v1::{ - asset::{ - Denom, - IbcPrefixed, - }, - RollupId, - ROLLUP_ID_LEN, - }, - protocol::transaction::v1::action::Sequence, -}; -use astria_eyre::eyre; -use prost::{ - bytes::Bytes, - Message as _, -}; -use sequencer_client::Transaction; -use serde_json::json; -use telemetry::Metrics as _; -use tempfile::NamedTempFile; -use tendermint::{ - consensus::{ - params::{ - AbciParams, - ValidatorParams, - }, - Params, - }, - Genesis, - Time, -}; -use tendermint_rpc::{ - endpoint::broadcast::tx_sync, - request, - response, - Id, -}; -use tokio::{ - sync::watch, - time, -}; -use tokio_util::sync::CancellationToken; -use tracing::debug; -use wiremock::{ - matchers::{ - body_partial_json, - body_string_contains, - }, - Mock, - MockGuard, - MockServer, - Request, - ResponseTemplate, -}; - -use crate::{ - executor, - executor::EnsureChainIdError, - metrics::Metrics, - test_utils::sequence_action_of_max_size, - Config, -}; - -static TELEMETRY: LazyLock<()> = LazyLock::new(|| { - // This config can be meaningless - it's only used inside `try_init` to init the metrics, but we - // haven't configured telemetry to provide metrics here. - let config = Config { - log: String::new(), - api_listen_addr: SocketAddr::new(IpAddr::from([0, 0, 0, 0]), 0), - sequencer_url: String::new(), - sequencer_chain_id: String::new(), - rollups: String::new(), - private_key_file: String::new(), - sequencer_address_prefix: String::new(), - block_time_ms: 0, - max_bytes_per_bundle: 0, - bundle_queue_capacity: 0, - force_stdout: false, - no_otel: false, - no_metrics: false, - metrics_http_listener_addr: String::new(), - pretty_print: false, - grpc_addr: SocketAddr::new(IpAddr::from([0, 0, 0, 0]), 0), - fee_asset: Denom::IbcPrefixed(IbcPrefixed::new([0; 32])), - }; - if std::env::var_os("TEST_LOG").is_some() { - let filter_directives = std::env::var("RUST_LOG").unwrap_or_else(|_| "info".into()); - telemetry::configure() - .set_no_otel(true) - .set_stdout_writer(std::io::stdout) - .set_filter_directives(&filter_directives) - .try_init::(&config) - .unwrap(); - } else { - telemetry::configure() - .set_no_otel(true) - .set_stdout_writer(std::io::sink) - .try_init::(&config) - .unwrap(); - } -}); - -fn sequence_action() -> Sequence { - Sequence { - rollup_id: RollupId::new([0; ROLLUP_ID_LEN]), - data: Bytes::new(), - fee_asset: "nria".parse().unwrap(), - } -} - -/// Start a mock sequencer server and mount a mock for the `accounts/nonce` query. -async fn setup() -> (MockServer, Config, NamedTempFile) { - LazyLock::force(&TELEMETRY); - let server = MockServer::start().await; - - let keyfile = NamedTempFile::new().unwrap(); - (&keyfile) - .write_all("2bd806c97f0e00af1a1fc3328fa763a9269723c8db8fac4f93af71db186d6e90".as_bytes()) - .unwrap(); - - let cfg = Config { - log: String::new(), - api_listen_addr: "127.0.0.1:0".parse().unwrap(), - rollups: String::new(), - sequencer_url: server.uri(), - sequencer_chain_id: "test-chain-1".to_string(), - private_key_file: keyfile.path().to_string_lossy().to_string(), - sequencer_address_prefix: "astria".into(), - block_time_ms: 2000, - max_bytes_per_bundle: 1000, - bundle_queue_capacity: 10, - no_otel: false, - force_stdout: false, - no_metrics: false, - metrics_http_listener_addr: String::new(), - pretty_print: true, - grpc_addr: "127.0.0.1:0".parse().unwrap(), - fee_asset: "nria".parse().unwrap(), - }; - (server, cfg, keyfile) -} - -/// Assert that given error is of correct type and contains the expected chain IDs. -#[track_caller] -fn assert_chain_id_err( - err: &EnsureChainIdError, - configured_expected: &str, - configured_actual: &tendermint::chain::Id, -) { - match err { - EnsureChainIdError::WrongChainId { - expected, - actual, - } => { - assert_eq!(*expected, configured_expected); - assert_eq!(*actual, *configured_actual); - } - other @ EnsureChainIdError::GetChainId(_) => { - panic!("expected `EnsureChainIdError::WrongChainId`, but got '{other:?}'") - } - } -} - -/// Mount a mock for the `abci_query` endpoint. -async fn mount_default_nonce_query_mock(server: &MockServer) -> MockGuard { - let query_path = "accounts/nonce"; - let response = NonceResponse { - height: 0, - nonce: 0, - }; - let expected_body = json!({ - "method": "abci_query" - }); - let response = tendermint_rpc::endpoint::abci_query::Response { - response: tendermint_rpc::endpoint::abci_query::AbciQuery { - value: response.encode_to_vec(), - ..Default::default() - }, - }; - let wrapper = response::Wrapper::new_with_id(Id::Num(1), Some(response), None); - Mock::given(body_partial_json(&expected_body)) - .and(body_string_contains(query_path)) - .respond_with( - ResponseTemplate::new(200) - .set_body_json(&wrapper) - .append_header("Content-Type", "application/json"), - ) - .up_to_n_times(1) - .expect(1) - .mount_as_scoped(server) - .await -} - -fn tx_from_request(request: &Request) -> Transaction { - use astria_core::generated::protocol::transaction::v1::Transaction as RawTransaction; - - let wrapped_tx_sync_req: request::Wrapper = - serde_json::from_slice(&request.body) - .expect("can't deserialize to JSONRPC wrapped tx_sync::Request"); - let raw_signed_tx = RawTransaction::decode(&*wrapped_tx_sync_req.params().tx) - .expect("can't deserialize signed sequencer tx from broadcast jsonrpc request"); - let signed_tx = Transaction::try_from_raw(raw_signed_tx) - .expect("can't convert raw signed tx to checked signed tx"); - debug!(?signed_tx, "sequencer mock received signed transaction"); - - signed_tx -} - -/// Deserializes the bytes contained in a `tx_sync::Request` to a signed sequencer transaction -/// and verifies that the contained sequence action is in the given `expected_rollup_ids` and -/// `expected_nonces`. -async fn mount_broadcast_tx_sync_seq_actions_mock(server: &MockServer) -> MockGuard { - let matcher = move |request: &Request| { - let signed_tx = tx_from_request(request); - let actions = signed_tx.actions(); - - // verify all received actions are sequence actions - actions.iter().all(|action| action.as_sequence().is_some()) - }; - let jsonrpc_rsp = response::Wrapper::new_with_id( - Id::Num(1), - Some(tx_sync::Response { - code: 0.into(), - data: vec![].into(), - log: String::new(), - hash: tendermint::Hash::Sha256([0; 32]), - }), - None, - ); - - Mock::given(matcher) - .respond_with(ResponseTemplate::new(200).set_body_json(&jsonrpc_rsp)) - .up_to_n_times(1) - .expect(1) - .mount_as_scoped(server) - .await -} - -/// Mounts genesis file with specified sequencer chain ID -async fn mount_genesis(server: &MockServer, mock_sequencer_chain_id: &str) { - Mock::given(body_partial_json( - json!({"jsonrpc": "2.0", "method": "genesis", "params": null}), - )) - .respond_with(ResponseTemplate::new(200).set_body_json( - tendermint_rpc::response::Wrapper::new_with_id( - tendermint_rpc::Id::uuid_v4(), - Some( - tendermint_rpc::endpoint::genesis::Response:: { - genesis: Genesis { - genesis_time: Time::from_unix_timestamp(1, 1).unwrap(), - chain_id: mock_sequencer_chain_id.try_into().unwrap(), - initial_height: 1, - consensus_params: Params { - block: tendermint::block::Size { - max_bytes: 1024, - max_gas: 1024, - time_iota_ms: 1000, - }, - evidence: tendermint::evidence::Params { - max_age_num_blocks: 1000, - max_age_duration: tendermint::evidence::Duration( - Duration::from_secs(3600), - ), - max_bytes: 1_048_576, - }, - validator: ValidatorParams { - pub_key_types: vec![tendermint::public_key::Algorithm::Ed25519], - }, - version: None, - abci: AbciParams::default(), - }, - validators: vec![], - app_hash: tendermint::hash::AppHash::default(), - app_state: serde_json::Value::Null, - }, - }, - ), - None, - ), - )) - .expect(1..) - .mount(server) - .await; -} - -/// Helper to wait for the executor to connect to the mock sequencer -async fn wait_for_startup( - mut status: watch::Receiver, - nonce_guard: MockGuard, -) -> eyre::Result<()> { - // wait to receive executor status - status - .wait_for(executor::Status::is_connected) - .await - .unwrap(); - - tokio::time::timeout( - Duration::from_millis(100), - nonce_guard.wait_until_satisfied(), - ) - .await - .unwrap(); - - Ok(()) -} - -/// Test to check that the executor sends a signed transaction to the sequencer as soon as it -/// receives a `SequenceAction` that fills it beyond its `max_bundle_size`. -#[tokio::test] -async fn full_bundle() { - // set up the executor, channel for writing seq actions, and the sequencer mock - let (sequencer, cfg, _keyfile) = setup().await; - let shutdown_token = CancellationToken::new(); - let metrics = Box::leak(Box::new(Metrics::noop_metrics(&cfg).unwrap())); - mount_genesis(&sequencer, &cfg.sequencer_chain_id).await; - let (executor, executor_handle) = executor::Builder { - sequencer_url: cfg.sequencer_url.clone(), - sequencer_chain_id: cfg.sequencer_chain_id.clone(), - private_key_file: cfg.private_key_file.clone(), - sequencer_address_prefix: "astria".into(), - block_time_ms: cfg.block_time_ms, - max_bytes_per_bundle: cfg.max_bytes_per_bundle, - bundle_queue_capacity: cfg.bundle_queue_capacity, - shutdown_token: shutdown_token.clone(), - metrics, - } - .build() - .unwrap(); - - let nonce_guard = mount_default_nonce_query_mock(&sequencer).await; - let status = executor.subscribe(); - - let _executor_task = tokio::spawn(executor.run_until_stopped()); - // wait for sequencer to get the initial nonce request from sequencer - wait_for_startup(status, nonce_guard).await.unwrap(); - - let response_guard = mount_broadcast_tx_sync_seq_actions_mock(&sequencer).await; - - // send two sequence actions to the executor, the first of which is large enough to fill the - // bundle sending the second should cause the first to immediately be submitted in - // order to make space for the second - let seq0 = sequence_action_of_max_size(cfg.max_bytes_per_bundle); - - let seq1 = Sequence { - rollup_id: RollupId::new([1; ROLLUP_ID_LEN]), - ..sequence_action_of_max_size(cfg.max_bytes_per_bundle) - }; - - // push both sequence actions to the executor in order to force the full bundle to be sent - executor_handle - .send_timeout(seq0.clone(), Duration::from_millis(1000)) - .await - .unwrap(); - executor_handle - .send_timeout(seq1.clone(), Duration::from_millis(1000)) - .await - .unwrap(); - - // wait for the mock sequencer to receive the signed transaction - tokio::time::timeout( - Duration::from_millis(100), - response_guard.wait_until_satisfied(), - ) - .await - .unwrap(); - - // verify only one signed transaction was received by the mock sequencer - // i.e. only the full bundle was sent and not the second one due to the block timer - let expected_seq_actions = [seq0]; - let requests = response_guard.received_requests().await; - assert_eq!(requests.len(), 1); - - // verify the expected sequence actions were received - let signed_tx = tx_from_request(&requests[0]); - let actions = signed_tx.actions(); - - assert_eq!( - actions.len(), - expected_seq_actions.len(), - "received more than one action, one was supposed to fill the bundle" - ); - - for (action, expected_seq_action) in actions.iter().zip(expected_seq_actions.iter()) { - let seq_action = action.as_sequence().unwrap(); - assert_eq!( - seq_action.rollup_id, expected_seq_action.rollup_id, - "chain id does not match. actual {:?} expected {:?}", - seq_action.rollup_id, expected_seq_action.rollup_id - ); - assert_eq!( - seq_action.data, expected_seq_action.data, - "data does not match expected data for action with rollup_id {:?}", - seq_action.rollup_id, - ); - } -} - -/// Test to check that the executor sends a signed transaction to the sequencer after its -/// `block_timer` has ticked -#[tokio::test] -async fn bundle_triggered_by_block_timer() { - // set up the executor, channel for writing seq actions, and the sequencer mock - let (sequencer, cfg, _keyfile) = setup().await; - let shutdown_token = CancellationToken::new(); - let metrics = Box::leak(Box::new(Metrics::noop_metrics(&cfg).unwrap())); - mount_genesis(&sequencer, &cfg.sequencer_chain_id).await; - let (executor, executor_handle) = executor::Builder { - sequencer_url: cfg.sequencer_url.clone(), - sequencer_chain_id: cfg.sequencer_chain_id.clone(), - private_key_file: cfg.private_key_file.clone(), - sequencer_address_prefix: "astria".into(), - block_time_ms: cfg.block_time_ms, - max_bytes_per_bundle: cfg.max_bytes_per_bundle, - bundle_queue_capacity: cfg.bundle_queue_capacity, - shutdown_token: shutdown_token.clone(), - metrics, - } - .build() - .unwrap(); - - let nonce_guard = mount_default_nonce_query_mock(&sequencer).await; - let status = executor.subscribe(); - - let _executor_task = tokio::spawn(executor.run_until_stopped()); - - // wait for sequencer to get the initial nonce request from sequencer - wait_for_startup(status, nonce_guard).await.unwrap(); - - let response_guard = mount_broadcast_tx_sync_seq_actions_mock(&sequencer).await; - - // send two sequence actions to the executor, both small enough to fit in a single bundle - // without filling it - let seq0 = Sequence { - data: vec![0u8; cfg.max_bytes_per_bundle / 4].into(), - ..sequence_action() - }; - - // make sure at least one block has passed so that the executor will submit the bundle - // despite it not being full - time::pause(); - executor_handle - .send_timeout(seq0.clone(), Duration::from_millis(1000)) - .await - .unwrap(); - time::advance(Duration::from_millis(cfg.block_time_ms)).await; - time::resume(); - - // wait for the mock sequencer to receive the signed transaction - tokio::time::timeout( - Duration::from_millis(100), - response_guard.wait_until_satisfied(), - ) - .await - .unwrap(); - - // verify only one signed transaction was received by the mock sequencer - let expected_seq_actions = [seq0]; - let requests = response_guard.received_requests().await; - assert_eq!(requests.len(), 1); - - // verify the expected sequence actions were received - let signed_tx = tx_from_request(&requests[0]); - let actions = signed_tx.actions(); - - assert_eq!( - actions.len(), - expected_seq_actions.len(), - "received more than one action, one was supposed to fill the bundle" - ); - - for (action, expected_seq_action) in actions.iter().zip(expected_seq_actions.iter()) { - let seq_action = action.as_sequence().unwrap(); - assert_eq!( - seq_action.rollup_id, expected_seq_action.rollup_id, - "chain id does not match. actual {:?} expected {:?}", - seq_action.rollup_id, expected_seq_action.rollup_id - ); - assert_eq!( - seq_action.data, expected_seq_action.data, - "data does not match expected data for action with rollup_id {:?}", - seq_action.rollup_id, - ); - } -} - -/// Test to check that the executor sends a signed transaction with two sequence actions to the -/// sequencer. -#[tokio::test] -async fn two_seq_actions_single_bundle() { - // set up the executor, channel for writing seq actions, and the sequencer mock - let (sequencer, cfg, _keyfile) = setup().await; - let shutdown_token = CancellationToken::new(); - let metrics = Box::leak(Box::new(Metrics::noop_metrics(&cfg).unwrap())); - mount_genesis(&sequencer, &cfg.sequencer_chain_id).await; - let (executor, executor_handle) = executor::Builder { - sequencer_url: cfg.sequencer_url.clone(), - sequencer_chain_id: cfg.sequencer_chain_id.clone(), - private_key_file: cfg.private_key_file.clone(), - sequencer_address_prefix: "astria".into(), - block_time_ms: cfg.block_time_ms, - max_bytes_per_bundle: cfg.max_bytes_per_bundle, - bundle_queue_capacity: cfg.bundle_queue_capacity, - shutdown_token: shutdown_token.clone(), - metrics, - } - .build() - .unwrap(); - - let nonce_guard = mount_default_nonce_query_mock(&sequencer).await; - let status = executor.subscribe(); - let _executor_task = tokio::spawn(executor.run_until_stopped()); - - // wait for sequencer to get the initial nonce request from sequencer - wait_for_startup(status, nonce_guard).await.unwrap(); - - let response_guard = mount_broadcast_tx_sync_seq_actions_mock(&sequencer).await; - - // send two sequence actions to the executor, both small enough to fit in a single bundle - // without filling it - let seq0 = Sequence { - data: vec![0u8; cfg.max_bytes_per_bundle / 4].into(), - ..sequence_action() - }; - - let seq1 = Sequence { - rollup_id: RollupId::new([1; ROLLUP_ID_LEN]), - data: vec![1u8; cfg.max_bytes_per_bundle / 4].into(), - ..sequence_action() - }; - - // make sure at least one block has passed so that the executor will submit the bundle - // despite it not being full - time::pause(); - executor_handle - .send_timeout(seq0.clone(), Duration::from_millis(1000)) - .await - .unwrap(); - executor_handle - .send_timeout(seq1.clone(), Duration::from_millis(1000)) - .await - .unwrap(); - time::advance(Duration::from_millis(cfg.block_time_ms)).await; - time::resume(); - - // wait for the mock sequencer to receive the signed transaction - tokio::time::timeout( - Duration::from_millis(100), - response_guard.wait_until_satisfied(), - ) - .await - .unwrap(); - - // verify only one signed transaction was received by the mock sequencer - let expected_seq_actions = [seq0, seq1]; - let requests = response_guard.received_requests().await; - assert_eq!(requests.len(), 1); - - // verify the expected sequence actions were received - let signed_tx = tx_from_request(&requests[0]); - let actions = signed_tx.actions(); - - assert_eq!( - actions.len(), - expected_seq_actions.len(), - "received more than one action, one was supposed to fill the bundle" - ); - - for (action, expected_seq_action) in actions.iter().zip(expected_seq_actions.iter()) { - let seq_action = action.as_sequence().unwrap(); - assert_eq!( - seq_action.rollup_id, expected_seq_action.rollup_id, - "chain id does not match. actual {:?} expected {:?}", - seq_action.rollup_id, expected_seq_action.rollup_id - ); - assert_eq!( - seq_action.data, expected_seq_action.data, - "data does not match expected data for action with rollup_id {:?}", - seq_action.rollup_id, - ); - } -} - -/// Test to check that executor's chain ID check is properly checked against the sequencer's chain -/// ID -#[tokio::test] -async fn chain_id_mismatch_returns_error() { - use tendermint::chain::Id; - - // set up sequencer mock - let (sequencer, cfg, _keyfile) = setup().await; - let shutdown_token = CancellationToken::new(); - let metrics = Box::leak(Box::new(Metrics::noop_metrics(&cfg).unwrap())); - - // mount a status response with an incorrect chain_id - mount_genesis(&sequencer, "bad-chain-id").await; - - // build the executor with the correct chain_id - let (executor, _executor_handle) = executor::Builder { - sequencer_url: cfg.sequencer_url.clone(), - sequencer_chain_id: cfg.sequencer_chain_id.clone(), - private_key_file: cfg.private_key_file.clone(), - sequencer_address_prefix: cfg.sequencer_address_prefix.clone(), - block_time_ms: cfg.block_time_ms, - max_bytes_per_bundle: cfg.max_bytes_per_bundle, - bundle_queue_capacity: cfg.bundle_queue_capacity, - shutdown_token: shutdown_token.clone(), - metrics, - } - .build() - .unwrap(); - - // ensure that run_until_stopped returns WrongChainId error - let err = executor.run_until_stopped().await.expect_err( - "should exit with an error when reading a bad chain ID, but exited with success", - ); - let mut found = false; - for cause in err.chain() { - if let Some(err) = cause.downcast_ref::() { - assert_chain_id_err( - err, - &cfg.sequencer_chain_id, - &Id::try_from("bad-chain-id".to_string()).unwrap(), - ); - found = true; - break; - } - } - - // ensure that the error chain contains the expected error - assert!( - found, - "expected `EnsureChainIdError::WrongChainId` in error chain, but it was not found" - ); -} diff --git a/crates/astria-composer/tests/blackbox/api.rs b/crates/astria-composer/tests/blackbox/api.rs index c65e2ff959..9d1275bbe6 100644 --- a/crates/astria-composer/tests/blackbox/api.rs +++ b/crates/astria-composer/tests/blackbox/api.rs @@ -4,7 +4,7 @@ async fn readyz_with_one_rollup() { // spawn_composer hits `/readyz` as part of starting the test // environment. If this future return then `readyz` must have // returned `status: ok`. - let _test_composer = spawn_composer(&["test1"]).await; + let _test_composer = spawn_composer(&["test1"], None, true).await; } #[tokio::test] @@ -12,5 +12,5 @@ async fn readyz_with_two_rollups() { // spawn_composer hits `/readyz` as part of starting the test // environment. If this future return then `readyz` must have // returned `status: ok`. - let _test_composer = spawn_composer(&["test1", "test2"]).await; + let _test_composer = spawn_composer(&["test1", "test2"], None, true).await; } diff --git a/crates/astria-composer/tests/blackbox/executor.rs b/crates/astria-composer/tests/blackbox/executor.rs new file mode 100644 index 0000000000..4abf7dd733 --- /dev/null +++ b/crates/astria-composer/tests/blackbox/executor.rs @@ -0,0 +1,224 @@ +use std::time::Duration; + +use astria_core::{ + generated::composer::v1::{ + grpc_collector_service_client::GrpcCollectorServiceClient, + SubmitRollupTransactionRequest, + }, + primitive::v1::{ + RollupId, + ROLLUP_ID_LEN, + }, + protocol::transaction::v1::action::RollupDataSubmission, +}; +use tokio::time; + +use crate::helper::{ + mount_broadcast_tx_sync_rollup_data_submissions_mock, + signed_tx_from_request, + spawn_composer, +}; + +/// Test to check that the executor sends a signed transaction to the sequencer after its +/// `block_timer` has ticked +#[tokio::test] +async fn bundle_triggered_by_block_timer() { + let test_composer = spawn_composer(&["test1"], None, true).await; + let mut composer_client = GrpcCollectorServiceClient::connect(format!( + "http://{}", + test_composer.grpc_collector_addr + )) + .await + .unwrap(); + + let response_guard = + mount_broadcast_tx_sync_rollup_data_submissions_mock(&test_composer.sequencer).await; + + // send two sequence actions to the executor, both small enough to fit in a single bundle + // without filling it + let rollup_id = RollupId::new([0; ROLLUP_ID_LEN]); + let data = vec![0u8; 1000]; + + let seq0 = RollupDataSubmission { + data: data.clone().into(), + rollup_id, + fee_asset: "nria".parse().unwrap(), + }; + + // make sure at least one block has passed so that the executor will submit the bundle + // despite it not being full + time::pause(); + time::timeout(Duration::from_millis(1000), async { + composer_client + .submit_rollup_transaction(SubmitRollupTransactionRequest { + rollup_id: Some(rollup_id.into_raw()), + data: data.into(), + }) + .await + .expect("rollup transactions should have been submitted successfully to grpc collector") + }) + .await + .unwrap(); + time::advance(Duration::from_millis(2000)).await; + time::resume(); + + // wait for the mock sequencer to receive the signed transaction + tokio::time::timeout( + Duration::from_millis(100), + response_guard.wait_until_satisfied(), + ) + .await + .unwrap(); + + // verify only one signed transaction was received by the mock sequencer + let expected_rollup_data_submissions = [seq0]; + let requests = response_guard.received_requests().await; + assert_eq!(requests.len(), 1); + + // verify the expected sequence actions were received + let signed_tx = signed_tx_from_request(&requests[0]); + let actions = signed_tx.actions(); + + assert_eq!( + actions.len(), + expected_rollup_data_submissions.len(), + "received more than one action, one was supposed to fill the bundle" + ); + + for (action, expected_rollup_data_submission) in + actions.iter().zip(expected_rollup_data_submissions.iter()) + { + let rollup_data_submission = action.as_rollup_data_submission().unwrap(); + assert_eq!( + rollup_data_submission.rollup_id, expected_rollup_data_submission.rollup_id, + "chain id does not match. actual {:?} expected {:?}", + rollup_data_submission.rollup_id, expected_rollup_data_submission.rollup_id + ); + assert_eq!( + rollup_data_submission.data, expected_rollup_data_submission.data, + "data does not match expected data for action with rollup_id {:?}", + rollup_data_submission.rollup_id, + ); + } +} + +/// Test to check that the executor sends a signed transaction with two sequence actions to the +/// sequencer. +#[tokio::test] +async fn two_rollup_data_submissions_single_bundle() { + let test_composer = spawn_composer(&["test1"], None, true).await; + let mut composer_client = GrpcCollectorServiceClient::connect(format!( + "http://{}", + test_composer.grpc_collector_addr + )) + .await + .unwrap(); + + let response_guard = + mount_broadcast_tx_sync_rollup_data_submissions_mock(&test_composer.sequencer).await; + + // send two sequence actions to the executor, both small enough to fit in a single bundle + // without filling it + let seq0 = RollupDataSubmission { + rollup_id: RollupId::new([0; ROLLUP_ID_LEN]), + data: vec![0u8; 1000].into(), + fee_asset: "nria".parse().unwrap(), + }; + + let seq1 = RollupDataSubmission { + rollup_id: RollupId::new([1; ROLLUP_ID_LEN]), + data: vec![1u8; 1000].into(), + fee_asset: "nria".parse().unwrap(), + }; + + // make sure at least one block has passed so that the executor will submit the bundle + // despite it not being full + time::pause(); + time::timeout(Duration::from_millis(1000), async { + composer_client + .submit_rollup_transaction(SubmitRollupTransactionRequest { + rollup_id: Some(seq0.rollup_id.into_raw()), + data: seq0.data.clone(), + }) + .await + .expect("rollup transactions should have been submitted successfully to grpc collector") + }) + .await + .unwrap(); + time::timeout(Duration::from_millis(1000), async { + composer_client + .submit_rollup_transaction(SubmitRollupTransactionRequest { + rollup_id: Some(seq1.rollup_id.into_raw()), + data: seq1.data.clone(), + }) + .await + .expect("rollup transactions should have been submitted successfully to grpc collector") + }) + .await + .unwrap(); + time::advance(Duration::from_millis(2000)).await; + time::resume(); + + // wait for the mock sequencer to receive the signed transaction + tokio::time::timeout( + Duration::from_millis(100), + response_guard.wait_until_satisfied(), + ) + .await + .unwrap(); + + // verify only one signed transaction was received by the mock sequencer + let expected_rollup_data_submissions = [seq0, seq1]; + let requests = response_guard.received_requests().await; + assert_eq!(requests.len(), 1); + + // verify the expected sequence actions were received + let signed_tx = signed_tx_from_request(&requests[0]); + let actions = signed_tx.actions(); + + assert_eq!( + actions.len(), + expected_rollup_data_submissions.len(), + "received more than one action, one was supposed to fill the bundle" + ); + + for (action, expected_rollup_data_submission) in + actions.iter().zip(expected_rollup_data_submissions.iter()) + { + let rollup_data_submission = action.as_rollup_data_submission().unwrap(); + assert_eq!( + rollup_data_submission.rollup_id, expected_rollup_data_submission.rollup_id, + "chain id does not match. actual {:?} expected {:?}", + rollup_data_submission.rollup_id, expected_rollup_data_submission.rollup_id + ); + assert_eq!( + rollup_data_submission.data, expected_rollup_data_submission.data, + "data does not match expected data for action with rollup_id {:?}", + rollup_data_submission.rollup_id, + ); + } +} + +/// Test to check that executor's chain ID check is properly checked against the sequencer's chain +/// ID +#[tokio::test] +async fn chain_id_mismatch_returns_error() { + // TODO(https://github.com/astriaorg/astria/issues/1833): this test will currently succeed if + // the executor fails for any reason on startup, not just if the chain ID is incorrect. This is + // a symptom of the current implementation of executor, though, which should be propagating + // errors. As such, I think it is out of the scope for the following test-only changes and + // should be fixed in a followup. + + let bad_chain_id = "bad_id"; + let test_composer = spawn_composer(&["test1"], Some(bad_chain_id), false).await; + let err = test_composer.composer.await.unwrap().unwrap_err(); + for cause in err.chain() { + if cause + .to_string() + .contains("executor failed while waiting for it to become ready") + { + return; + } + } + panic!("did not find expected executor error message") +} diff --git a/crates/astria-composer/tests/blackbox/geth_collector.rs b/crates/astria-composer/tests/blackbox/geth_collector.rs index 8acf6c8178..337a087245 100644 --- a/crates/astria-composer/tests/blackbox/geth_collector.rs +++ b/crates/astria-composer/tests/blackbox/geth_collector.rs @@ -15,7 +15,7 @@ use crate::helper::{ async fn tx_from_one_rollup_is_received_by_sequencer() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 - let test_composer = spawn_composer(&["test1"]).await; + let test_composer = spawn_composer(&["test1"], None, true).await; let expected_rollup_ids = vec![RollupId::from_unhashed_bytes("test1")]; let mock_guard = @@ -37,7 +37,7 @@ async fn tx_from_one_rollup_is_received_by_sequencer() { async fn collector_restarts_after_exit() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 - let test_composer = spawn_composer(&["test1"]).await; + let test_composer = spawn_composer(&["test1"], None, true).await; // get rollup node let rollup_node = test_composer.rollup_nodes.get("test1").unwrap(); @@ -71,7 +71,7 @@ async fn collector_restarts_after_exit() { async fn invalid_nonce_causes_resubmission_under_different_nonce() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 - let test_composer = spawn_composer(&["test1"]).await; + let test_composer = spawn_composer(&["test1"], None, true).await; // Reject the first transaction for invalid nonce let invalid_nonce_guard = mount_broadcast_tx_sync_invalid_nonce_mock( @@ -88,7 +88,7 @@ async fn invalid_nonce_causes_resubmission_under_different_nonce() { // Mount a response of 1 to a nonce query test_composer .sequencer_mock - .mount_pending_nonce_response(1, "setup correct nonce") + .mount_pending_nonce_response(1, "setup correct nonce", 1) .await; // Push a tx to the rollup node so that it is picked up by the composer and submitted with the @@ -117,7 +117,7 @@ async fn invalid_nonce_causes_resubmission_under_different_nonce() { async fn nonce_taken_causes_resubmission_under_different_nonce() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 - let test_composer = spawn_composer(&["test1"]).await; + let test_composer = spawn_composer(&["test1"], None, true).await; // Reject the first transaction for taken nonce let invalid_nonce_guard = mount_broadcast_tx_sync_invalid_nonce_mock( @@ -134,7 +134,7 @@ async fn nonce_taken_causes_resubmission_under_different_nonce() { // Mount a response of 1 to a nonce query test_composer .sequencer_mock - .mount_pending_nonce_response(1, "setup correct nonce") + .mount_pending_nonce_response(1, "setup correct nonce", 1) .await; // Push a tx to the rollup node so that it is picked up by the composer and submitted with the @@ -163,7 +163,7 @@ async fn nonce_taken_causes_resubmission_under_different_nonce() { async fn single_rollup_tx_payload_integrity() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 - let test_composer = spawn_composer(&["test1"]).await; + let test_composer = spawn_composer(&["test1"], None, true).await; let tx: Transaction = serde_json::from_str(TEST_ETH_TX_JSON).unwrap(); let mock_guard = diff --git a/crates/astria-composer/tests/blackbox/grpc_collector.rs b/crates/astria-composer/tests/blackbox/grpc_collector.rs index f8286a25d8..7b87348f63 100644 --- a/crates/astria-composer/tests/blackbox/grpc_collector.rs +++ b/crates/astria-composer/tests/blackbox/grpc_collector.rs @@ -20,7 +20,7 @@ use crate::helper::{ #[tokio::test] async fn tx_from_one_rollup_is_received_by_sequencer() { - let test_composer = spawn_composer(&[]).await; + let test_composer = spawn_composer(&[], None, true).await; let rollup_id = RollupId::from_unhashed_bytes("test1"); let expected_chain_ids = vec![rollup_id]; let mock_guard = @@ -56,7 +56,7 @@ async fn invalid_nonce_causes_resubmission_under_different_nonce() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 let rollup_id = RollupId::from_unhashed_bytes("test1"); - let test_composer = spawn_composer(&[]).await; + let test_composer = spawn_composer(&[], None, true).await; // Reject the first transaction for invalid nonce let invalid_nonce_guard = @@ -65,7 +65,7 @@ async fn invalid_nonce_causes_resubmission_under_different_nonce() { // Mount a response of 1 to a nonce query test_composer .sequencer_mock - .mount_pending_nonce_response(1, "setup correct nonce") + .mount_pending_nonce_response(1, "setup correct nonce", 1) .await; let expected_chain_ids = vec![rollup_id]; @@ -112,7 +112,7 @@ async fn single_rollup_tx_payload_integrity() { // Spawn a composer with a mock sequencer and a mock rollup node // Initial nonce is 0 let rollup_id = RollupId::from_unhashed_bytes("test1"); - let test_composer = spawn_composer(&[]).await; + let test_composer = spawn_composer(&[], None, true).await; let tx: Transaction = serde_json::from_str(TEST_ETH_TX_JSON).unwrap(); let mock_guard = diff --git a/crates/astria-composer/tests/blackbox/helper/mock_abci_sequencer.rs b/crates/astria-composer/tests/blackbox/helper/mock_abci_sequencer.rs index 28f5ca53a2..ead144030b 100644 --- a/crates/astria-composer/tests/blackbox/helper/mock_abci_sequencer.rs +++ b/crates/astria-composer/tests/blackbox/helper/mock_abci_sequencer.rs @@ -19,9 +19,11 @@ use wiremock::{ ResponseTemplate, }; -pub async fn start() -> MockServer { +use super::TEST_CHAIN_ID; + +pub async fn start(chain_id: Option<&str>) -> MockServer { let server = MockServer::start().await; - mount_genesis(&server, "test-chain-1").await; + mount_genesis(&server, chain_id.unwrap_or(TEST_CHAIN_ID)).await; server } diff --git a/crates/astria-composer/tests/blackbox/helper/mock_grpc_sequencer.rs b/crates/astria-composer/tests/blackbox/helper/mock_grpc_sequencer.rs index 23be34c31a..ef4c31fcbe 100644 --- a/crates/astria-composer/tests/blackbox/helper/mock_grpc_sequencer.rs +++ b/crates/astria-composer/tests/blackbox/helper/mock_grpc_sequencer.rs @@ -76,6 +76,7 @@ impl MockGrpcSequencer { &self, nonce_to_mount: u32, debug_name: impl Into, + expected_requests: u64, ) { let resp = GetPendingNonceResponse { inner: nonce_to_mount, @@ -86,7 +87,7 @@ impl MockGrpcSequencer { ) .respond_with(constant_response(resp)) .up_to_n_times(1) - .expect(1) + .expect(expected_requests) .with_name(debug_name) .mount(&self.mock_server) .await; diff --git a/crates/astria-composer/tests/blackbox/helper/mod.rs b/crates/astria-composer/tests/blackbox/helper/mod.rs index 93f7f6a1a7..c0fb62404c 100644 --- a/crates/astria-composer/tests/blackbox/helper/mod.rs +++ b/crates/astria-composer/tests/blackbox/helper/mod.rs @@ -15,6 +15,7 @@ use astria_composer::{ Metrics, }; use astria_core::{ + generated::protocol::accounts::v1::NonceResponse, primitive::v1::{ asset::{ Denom, @@ -31,6 +32,8 @@ use astria_core::{ use astria_eyre::eyre; use ethers::prelude::Transaction as EthersTransaction; use mock_grpc_sequencer::MockGrpcSequencer; +use prost::Message as _; +use serde_json::json; use telemetry::metrics; use tempfile::NamedTempFile; use tendermint_rpc::{ @@ -43,6 +46,10 @@ use test_utils::mock::Geth; use tokio::task::JoinHandle; use tracing::debug; use wiremock::{ + matchers::{ + body_partial_json, + body_string_contains, + }, Mock, MockGuard, MockServer, @@ -53,6 +60,8 @@ use wiremock::{ pub mod mock_abci_sequencer; pub mod mock_grpc_sequencer; +pub const TEST_CHAIN_ID: &str = "test-chain-1"; + static TELEMETRY: LazyLock<()> = LazyLock::new(|| { // This config can be meaningless - it's only used inside `try_init` to init the metrics, but we // haven't configured telemetry to provide metrics here. @@ -110,7 +119,11 @@ pub struct TestComposer { /// # Panics /// There is no explicit error handling in favour of panicking loudly /// and early. -pub async fn spawn_composer(rollup_ids: &[&str]) -> TestComposer { +pub async fn spawn_composer( + rollup_ids: &[&str], + sequencer_chain_id: Option<&str>, + loop_until_ready: bool, +) -> TestComposer { LazyLock::force(&TELEMETRY); let mut rollup_nodes = HashMap::new(); @@ -121,7 +134,7 @@ pub async fn spawn_composer(rollup_ids: &[&str]) -> TestComposer { rollup_nodes.insert((*id).to_string(), geth); rollups.push_str(&format!("{id}::{execution_url},")); } - let sequencer = mock_abci_sequencer::start().await; + let sequencer = mock_abci_sequencer::start(sequencer_chain_id).await; let grpc_server = MockGrpcSequencer::spawn().await; let sequencer_url = sequencer.uri(); let keyfile = NamedTempFile::new().unwrap(); @@ -131,7 +144,7 @@ pub async fn spawn_composer(rollup_ids: &[&str]) -> TestComposer { let config = Config { log: String::new(), api_listen_addr: "127.0.0.1:0".parse().unwrap(), - sequencer_chain_id: "test-chain-1".to_string(), + sequencer_chain_id: TEST_CHAIN_ID.to_string(), rollups, sequencer_abci_endpoint: sequencer_url.to_string(), sequencer_grpc_endpoint: format!("http://{}", grpc_server.local_addr), @@ -155,9 +168,15 @@ pub async fn spawn_composer(rollup_ids: &[&str]) -> TestComposer { .unwrap(); let metrics = Box::leak(Box::new(metrics)); + let expected_get_nonce_requests = loop_until_ready.into(); + // prepare get nonce response grpc_server - .mount_pending_nonce_response(0, "startup::wait_for_mempool()") + .mount_pending_nonce_response( + 0, + "startup::wait_for_mempool()", + expected_get_nonce_requests, + ) .await; let (composer_addr, grpc_collector_addr, composer_handle) = { @@ -168,7 +187,10 @@ pub async fn spawn_composer(rollup_ids: &[&str]) -> TestComposer { (composer_addr, grpc_collector_addr, task) }; - loop_until_composer_is_ready(composer_addr).await; + if loop_until_ready { + loop_until_composer_is_ready(composer_addr).await; + } + TestComposer { cfg: config, composer: composer_handle, @@ -206,7 +228,13 @@ pub async fn loop_until_composer_is_ready(addr: SocketAddr) { } } -fn signed_tx_from_request(request: &Request) -> Transaction { +/// Creates a signed transaction from a wiremock request. +/// +/// # Panics +/// +/// Panics if the request body can't be deserialiezed to a JSONRPC wrapped `tx_sync::Request`, or if +/// the deserialization from the JSONRPC request to the raw transaction fails. +pub fn signed_tx_from_request(request: &Request) -> Transaction { use astria_core::generated::protocol::transaction::v1::Transaction as RawTransaction; use prost::Message as _; @@ -370,6 +398,70 @@ pub async fn mount_broadcast_tx_sync_nonce_taken_mock( .await } +/// Deserializes the bytes contained in a `tx_sync::Request` to a signed sequencer transaction +/// and verifies that the contained rollup data submission action is in the given +/// `expected_rollup_ids` and `expected_nonces`. +pub async fn mount_broadcast_tx_sync_rollup_data_submissions_mock( + server: &MockServer, +) -> MockGuard { + let matcher = move |request: &Request| { + let signed_tx = signed_tx_from_request(request); + let actions = signed_tx.actions(); + + // verify all received actions are sequence actions + actions + .iter() + .all(|action| action.as_rollup_data_submission().is_some()) + }; + let jsonrpc_rsp = response::Wrapper::new_with_id( + Id::Num(1), + Some(tx_sync::Response { + code: 0.into(), + data: vec![].into(), + log: String::new(), + hash: tendermint::Hash::Sha256([0; 32]), + }), + None, + ); + + Mock::given(matcher) + .respond_with(ResponseTemplate::new(200).set_body_json(&jsonrpc_rsp)) + .up_to_n_times(1) + .expect(1) + .mount_as_scoped(server) + .await +} + +/// Mount a mock for the `abci_query` endpoint. +pub async fn mount_default_nonce_query_mock(server: &MockServer) -> MockGuard { + let query_path = "accounts/nonce"; + let response = NonceResponse { + height: 0, + nonce: 0, + }; + let expected_body = json!({ + "method": "abci_query" + }); + let response = tendermint_rpc::endpoint::abci_query::Response { + response: tendermint_rpc::endpoint::abci_query::AbciQuery { + value: response.encode_to_vec(), + ..Default::default() + }, + }; + let wrapper = response::Wrapper::new_with_id(Id::Num(1), Some(response), None); + Mock::given(body_partial_json(&expected_body)) + .and(body_string_contains(query_path)) + .respond_with( + ResponseTemplate::new(200) + .set_body_json(&wrapper) + .append_header("Content-Type", "application/json"), + ) + .up_to_n_times(1) + .expect(1) + .mount_as_scoped(server) + .await +} + // A Uniswap V2 DAI-ETH swap transaction from mainnet // Etherscan link: https://etherscan.io/tx/0x99850dd1cf325c8ede9ba62b9d8a11aa199794450b581ce3a7bb8c1e5bb7562f pub const TEST_ETH_TX_JSON: &str = r#"{"blockHash":"0xe365f2163edb844b617ebe3d2af183b31d6c7ffa794f21d0b2d111d63e979a02","blockNumber":"0x1157959","from":"0xdc975a9bb00f4c030e4eb3268f68e4b8d0fa0362","gas":"0xcdf49","gasPrice":"0x374128344","maxFeePerGas":"0x374128344","maxPriorityFeePerGas":"0x0","hash":"0x99850dd1cf325c8ede9ba62b9d8a11aa199794450b581ce3a7bb8c1e5bb7562f","input":"0x022c0d9f0000000000000000000000000000000000000000000000c88a1ad5e15105525500000000000000000000000000000000000000000000000000000000000000000000000000000000000000001a2d11cb90d1de13bb81ee7b772a08ac234a8058000000000000000000000000000000000000000000000000000000000000008000000000000000000000000000000000000000000000000000000000000001208974000000000000000000000000000000000000000000000000000000004de4000000000000000000000000000000000000000000000000017038152c223cb100000000000000000000000000000000000000000000000000000000000000600000000000000000000000000000000000000000000000000000000000000005200000000000000000000000000000000000000000000000000000000000000000000000000000000000000087870bca3f3fd6335c3f4ce8392d69350b4fa4e2000000000000000000000000ab12275f2d91f87b301a4f01c9af4e83b3f45baa0000000000000000000000006b175474e89094c44da98b954eedeac495271d0f000000000000000000000000c02aaa39b223fe8d0a0e5c4f27ead9083c756cc2","nonce":"0x28","to":"0xa478c2975ab1ea89e8196811f51a7b7ade33eb11","transactionIndex":"0x2","value":"0x0","type":"0x2","accessList":[{"address":"0x5f4ec3df9cbd43714fe2740f5e3616155c5b8419","storageKeys":["0x0000000000000000000000000000000000000000000000000000000000000005","0x0000000000000000000000000000000000000000000000000000000000000002"]},{"address":"0x7effd7b47bfd17e52fb7559d3f924201b9dbff3d","storageKeys":[]},{"address":"0x018008bfb33d285247a21d44e50697654f754e63","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc"]},{"address":"0x1a2d11cb90d1de13bb81ee7b772a08ac234a8058","storageKeys":[]},{"address":"0xe62b71cf983019bff55bc83b48601ce8419650cc","storageKeys":["0x9a09f352b299559621084d9b8d2625e8d5a97f382735872dd3bb1bdbdccc3fee","0x000000000000000000000000000000000000000000000000000000000000002b","0xfee3a99380070b792e111dd9a6a15e929983e2d0b7e170a5520e51b99be0c359"]},{"address":"0x87870bca3f3fd6335c3f4ce8392d69350b4fa4e2","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x070a95ec3546cae47592e0bcea195bf8f96287077fbb7a23785cc2887152941c","0x070a95ec3546cae47592e0bcea195bf8f96287077fbb7a23785cc28871529420","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec6","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4b","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ebf","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec0","0x4c0bd942d17410ca1f6d3278a62feef7078602605466e37de958808f1454efbd","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e48","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec3","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4f","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4a","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e50","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4d","0x4cb2b152c1b54ce671907a93c300fd5aa72383a9d4ec19a81e3333632ae92e00","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec4","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec7","0x4bea7244bd9088ac961c659a818b4f060de9712d20dc006c24f0985f19cf62d1","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e49","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec2","0x070a95ec3546cae47592e0bcea195bf8f96287077fbb7a23785cc2887152941d","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4c","0x5e14560e314427eb9d0c466a6058089f672317c8e26719a770a709c3f2481e4e","0x4480713a5820391a4815a640728dab70c3847e45854ef9e8117382da26ce9105","0x070a95ec3546cae47592e0bcea195bf8f96287077fbb7a23785cc2887152941f","0x000000000000000000000000000000000000000000000000000000000000003b","0x108718ddd11d4cf696a068770009c44aef387eb858097a37824291f99278d5e3","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec1","0xf81d8d79f42adb4c73cc3aa0c78e25d3343882d0313c0b80ece3d3a103ef1ec5"]},{"address":"0x2f39d218133afab8f2b819b1066c7e434ad94e9e","storageKeys":["0x740f710666bd7a12af42df98311e541e47f7fd33d382d11602457a6d540cbd63","0x0d2c1bcee56447b4f46248272f34207a580a5c40f666a31f4e2fbb470ea53ab8"]},{"address":"0xe7b67f44ea304dd7f6d215b13686637ff64cd2b2","storageKeys":[]},{"address":"0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2","storageKeys":["0x7f6377583d24615ddfe989626525aeed0d158f924ee8c91664ab0dffd7863d00","0x3afb575d989d656a39ee0690da12b019915f3bd8709cc522e681b8dd04237970","0xa535fbd0ab3e0ad4ee444570368f3d474545b71fcc49228fe96a6406676fc126","0xb064600732a82908427d092d333e607598a6238a59aeb45e1288cb0bac7161cf"]},{"address":"0x4d5f47fa6a74757f35c14fd3a6ef8e3c9bc514e8","storageKeys":["0x000000000000000000000000000000000000000000000000000000000000003c","0x14a553e31736f19e3e380cf55bfb2f82dfd6d880cd07235affb68d8d3e0cac4d","0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x5e8cc6ee686108b7fd15638e2dbb32555b30d0bd1a191628bb70b5459b86cedc","0x000000000000000000000000000000000000000000000000000000000000003d","0x0000000000000000000000000000000000000000000000000000000000000036","0x0000000000000000000000000000000000000000000000000000000000000039"]},{"address":"0x6b175474e89094c44da98b954eedeac495271d0f","storageKeys":["0xd86cc1e239204d48eb0055f151744c4bb3d2337612287be803ae8247e95a67d2","0xe7ab5c3b3c86286a122f1937d4c70a3170dba7ef4f7603d830e8bcf7c9af583b","0x87c358b8e65d7446f52ffce25e44c9673d2bf461b3d3e4748afcf1238e9224a3","0xad740bfd58072c0bd719418966c52da18e837afec1b47e07bba370568cc87fbb"]},{"address":"0xe175de51f29d822b86e46a9a61246ec90631210d","storageKeys":[]},{"address":"0xcf8d0c70c850859266f5c338b38f9d663181c314","storageKeys":["0x0000000000000000000000000000000000000000000000000000000000000037","0x000000000000000000000000000000000000000000000000000000000000003d","0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x000000000000000000000000000000000000000000000000000000000000003a","0x4bea7244bd9088ac961c659a818b4f060de9712d20dc006c24f0985f19cf62d1"]},{"address":"0x413adac9e2ef8683adf5ddaece8f19613d60d1bb","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x000000000000000000000000000000000000000000000000000000000000003f","0x000000000000000000000000000000000000000000000000000000000000003a","0x4bea7244bd9088ac961c659a818b4f060de9712d20dc006c24f0985f19cf62d1"]},{"address":"0xaed0c38402a5d19df6e4c03f4e2dced6e29c1ee9","storageKeys":["0x0000000000000000000000000000000000000000000000000000000000000005","0x0000000000000000000000000000000000000000000000000000000000000002"]},{"address":"0xea51d7853eefb32b6ee06b1c12e6dcca88be0ffe","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x000000000000000000000000000000000000000000000000000000000000003a"]},{"address":"0x54586be62e3c3580375ae3723c145253060ca0c2","storageKeys":["0x7145bb02480b505fc02ccfdba07d3ba3a9d821606f0688263abedd0ac6e5bec5","0x2a11cb67ca5c7e99dba99b50e02c11472d0f19c22ed5af42a1599a7f57e1c7a4","0x5306b8fbe80b30a74098357ee8e26fad8dc069da9011cca5f0870a0a5982e541"]},{"address":"0x478238a1c8b862498c74d0647329aef9ea6819ed","storageKeys":["0x9ef04667c5a1bd8192837ceac2ad5f2c41549d4db3406185e8c6aa95ea557bc5","0x000000000000000000000000000000000000000000000000000000000000002b","0x0020b304a2489d03d215fadd3bb6d3de2dda5a6a1235e76d693c30263e3cd054"]},{"address":"0xa700b4eb416be35b2911fd5dee80678ff64ff6c9","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x5e8cc6ee686108b7fd15638e2dbb32555b30d0bd1a191628bb70b5459b86cedc"]},{"address":"0x8164cc65827dcfe994ab23944cbc90e0aa80bfcb","storageKeys":["0x76f8b43dabb591eb6681562420f7f6aa393e6903d4e02e6f59e2957d94ceab20","0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x176062dac4e737f036c34baf4b07185f9c9fd3c1337ca36eb7c1f7a74aedb8ea"]},{"address":"0x9a158802cd924747ef336ca3f9de3bdb60cf43d3","storageKeys":[]},{"address":"0xac725cb59d16c81061bdea61041a8a5e73da9ec6","storageKeys":[]},{"address":"0x15c5620dffac7c7366eed66c20ad222ddbb1ed57","storageKeys":[]},{"address":"0x547a514d5e3769680ce22b2361c10ea13619e8a9","storageKeys":["0x0000000000000000000000000000000000000000000000000000000000000005","0x0000000000000000000000000000000000000000000000000000000000000002"]},{"address":"0x8116b273cd75d79c382afacc706659ded5e0a59d","storageKeys":["0x0fb35ae12d348b84dc0910bcce7d3b0a3f6d23a3e1d0b53bbe5f135078b97b13","0x000000000000000000000000000000000000000000000000000000000000002b","0x1d90d8e683e6736ac0564a19732a642e4be100e7ee8c225feba909bbdaf1522b"]},{"address":"0x9f8ccdafcc39f3c7d6ebf637c9151673cbc36b88","storageKeys":[]},{"address":"0xa478c2975ab1ea89e8196811f51a7b7ade33eb11","storageKeys":["0x0000000000000000000000000000000000000000000000000000000000000007","0x0000000000000000000000000000000000000000000000000000000000000009","0x000000000000000000000000000000000000000000000000000000000000000a","0x000000000000000000000000000000000000000000000000000000000000000c","0x0000000000000000000000000000000000000000000000000000000000000008","0x0000000000000000000000000000000000000000000000000000000000000006"]},{"address":"0xf1cd4193bbc1ad4a23e833170f49d60f3d35a621","storageKeys":[]},{"address":"0x102633152313c81cd80419b6ecf66d14ad68949a","storageKeys":["0x360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbc","0x000000000000000000000000000000000000000000000000000000000000003f","0x000000000000000000000000000000000000000000000000000000000000003a"]},{"address":"0xb02381b1d27aa9845e5012083ca288c1818884f0","storageKeys":[]}],"chainId":"0x1","v":"0x0","r":"0xcb4eccf09e298388220c5560a6539322bde17581cee6908d56a92a19575e28e2","s":"0x2b4e34adad48aee14b6600c6366ad683c00c63c9da88fc2a232308421cf69a21"}"#; diff --git a/crates/astria-composer/tests/blackbox/main.rs b/crates/astria-composer/tests/blackbox/main.rs index d0ed46b3bc..80ecf0e38b 100644 --- a/crates/astria-composer/tests/blackbox/main.rs +++ b/crates/astria-composer/tests/blackbox/main.rs @@ -1,4 +1,5 @@ pub mod api; +mod executor; pub mod geth_collector; mod grpc_collector; pub mod helper; diff --git a/crates/astria-test-utils/src/mock/geth.rs b/crates/astria-test-utils/src/mock/geth.rs index 4551f6e2c7..dbc91830a3 100644 --- a/crates/astria-test-utils/src/mock/geth.rs +++ b/crates/astria-test-utils/src/mock/geth.rs @@ -193,6 +193,7 @@ impl GethServer for GethImpl { /// A mocked geth server for subscribing to new transactions. /// /// Allows for explicitly pushing transactions to subscribed clients. +#[derive(Debug)] pub struct Geth { /// The local address to which the mocked jsonrpc server is bound. local_addr: SocketAddr, From f1cf32a791201a98d0e234c2e313367f414d3a14 Mon Sep 17 00:00:00 2001 From: ethanoroshiba Date: Fri, 22 Nov 2024 10:20:58 -0600 Subject: [PATCH 2/3] remove unused test_utils change --- crates/astria-test-utils/src/mock/geth.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/astria-test-utils/src/mock/geth.rs b/crates/astria-test-utils/src/mock/geth.rs index dbc91830a3..4551f6e2c7 100644 --- a/crates/astria-test-utils/src/mock/geth.rs +++ b/crates/astria-test-utils/src/mock/geth.rs @@ -193,7 +193,6 @@ impl GethServer for GethImpl { /// A mocked geth server for subscribing to new transactions. /// /// Allows for explicitly pushing transactions to subscribed clients. -#[derive(Debug)] pub struct Geth { /// The local address to which the mocked jsonrpc server is bound. local_addr: SocketAddr, From e27d2fcc6567a950c1308b3ddfff8342bd004608 Mon Sep 17 00:00:00 2001 From: ethanoroshiba Date: Thu, 5 Dec 2024 14:34:48 -0600 Subject: [PATCH 3/3] fix changes from merge --- crates/astria-composer/tests/blackbox/executor.rs | 2 +- crates/astria-composer/tests/blackbox/helper/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/astria-composer/tests/blackbox/executor.rs b/crates/astria-composer/tests/blackbox/executor.rs index 4abf7dd733..28f15191cd 100644 --- a/crates/astria-composer/tests/blackbox/executor.rs +++ b/crates/astria-composer/tests/blackbox/executor.rs @@ -1,7 +1,7 @@ use std::time::Duration; use astria_core::{ - generated::composer::v1::{ + generated::astria::composer::v1::{ grpc_collector_service_client::GrpcCollectorServiceClient, SubmitRollupTransactionRequest, }, diff --git a/crates/astria-composer/tests/blackbox/helper/mod.rs b/crates/astria-composer/tests/blackbox/helper/mod.rs index cdc97c6609..0efa11ec35 100644 --- a/crates/astria-composer/tests/blackbox/helper/mod.rs +++ b/crates/astria-composer/tests/blackbox/helper/mod.rs @@ -15,7 +15,7 @@ use astria_composer::{ Metrics, }; use astria_core::{ - generated::protocol::accounts::v1::NonceResponse, + generated::astria::protocol::accounts::v1::NonceResponse, primitive::v1::{ asset::{ Denom,