From 8bf7f2d9f347c21ebef8dbe6440a348970b222c0 Mon Sep 17 00:00:00 2001 From: frisitano Date: Sun, 5 Oct 2025 00:24:04 +0800 Subject: [PATCH 01/12] atomic rollup node --- Cargo.lock | 14 + crates/chain-orchestrator/Cargo.toml | 11 + crates/chain-orchestrator/src/action.rs | 57 - crates/chain-orchestrator/src/config.rs | 39 + crates/chain-orchestrator/src/consensus.rs | 163 + .../chain-orchestrator/src/consolidation.rs | 68 + crates/chain-orchestrator/src/error.rs | 7 + .../chain-orchestrator/src/handle/command.rs | 27 + .../chain-orchestrator/src/handle/metrics.rs | 10 + crates/chain-orchestrator/src/handle/mod.rs | 81 + crates/chain-orchestrator/src/lib.rs | 2997 ++++++++--------- crates/chain-orchestrator/src/retry.rs | 6 + crates/chain-orchestrator/src/status.rs | 11 + crates/chain-orchestrator/src/sync.rs | 66 + crates/database/db/src/db.rs | 87 +- crates/database/db/src/error.rs | 3 + crates/database/db/src/lib.rs | 2 +- crates/database/db/src/operations.rs | 214 +- crates/database/migration/src/lib.rs | 2 + ...m20250929_161536_add_additional_indexes.rs | 34 + crates/derivation-pipeline/src/lib.rs | 15 +- crates/derivation-pipeline/src/new.rs | 228 ++ crates/engine/src/driver.rs | 18 +- crates/engine/src/error.rs | 40 + crates/engine/src/fcs.rs | 53 +- crates/engine/src/lib.rs | 6 +- crates/engine/src/new.rs | 120 + crates/engine/src/payload.rs | 2 +- crates/manager/src/lib.rs | 8 +- crates/manager/src/manager/mod.rs | 29 +- crates/network/src/event.rs | 4 +- crates/network/src/handle.rs | 18 +- crates/network/src/lib.rs | 24 +- crates/network/src/manager.rs | 120 +- crates/node/src/add_ons/handle.rs | 4 +- crates/node/src/add_ons/rollup.rs | 4 +- crates/node/src/add_ons/rpc.rs | 10 +- crates/node/src/args.rs | 134 +- crates/node/tests/e2e.rs | 20 +- crates/primitives/src/batch.rs | 31 + crates/primitives/src/lib.rs | 2 +- crates/providers/src/l1/message.rs | 7 +- crates/sequencer/Cargo.toml | 4 + crates/sequencer/src/config.rs | 81 + crates/sequencer/src/error.rs | 10 + crates/sequencer/src/event.rs | 11 + crates/sequencer/src/lib.rs | 482 ++- crates/sequencer/tests/e2e.rs | 1439 ++++---- 48 files changed, 3913 insertions(+), 2910 deletions(-) delete mode 100644 crates/chain-orchestrator/src/action.rs create mode 100644 crates/chain-orchestrator/src/config.rs create mode 100644 crates/chain-orchestrator/src/consensus.rs create mode 100644 crates/chain-orchestrator/src/consolidation.rs create mode 100644 crates/chain-orchestrator/src/handle/command.rs create mode 100644 crates/chain-orchestrator/src/handle/metrics.rs create mode 100644 crates/chain-orchestrator/src/handle/mod.rs create mode 100644 crates/chain-orchestrator/src/status.rs create mode 100644 crates/chain-orchestrator/src/sync.rs create mode 100644 crates/database/migration/src/m20250929_161536_add_additional_indexes.rs create mode 100644 crates/derivation-pipeline/src/new.rs create mode 100644 crates/engine/src/new.rs create mode 100644 crates/sequencer/src/config.rs create mode 100644 crates/sequencer/src/event.rs diff --git a/Cargo.lock b/Cargo.lock index 6cea3ec5..301a0557 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10813,6 +10813,7 @@ dependencies = [ "alloy-primitives", "alloy-provider", "alloy-rpc-client", + "alloy-rpc-types-engine 1.0.30", "alloy-transport", "arbitrary", "futures", @@ -10823,19 +10824,29 @@ dependencies = [ "reqwest", "reth-chainspec", "reth-eth-wire-types", + "reth-network-api", "reth-network-p2p", "reth-network-peers", "reth-primitives-traits", "reth-scroll-chainspec", + "reth-scroll-engine-primitives", "reth-scroll-forks", + "reth-scroll-node", "reth-scroll-primitives", + "reth-tasks", + "reth-tokio-util", "reth-tracing", "rollup-node-primitives", + "rollup-node-providers", + "rollup-node-signer", "rollup-node-watcher", "scroll-alloy-consensus", "scroll-alloy-hardforks", "scroll-alloy-network", + "scroll-alloy-provider", "scroll-db", + "scroll-derivation-pipeline", + "scroll-engine", "scroll-network", "serde_json", "strum 0.27.2", @@ -10952,16 +10963,19 @@ dependencies = [ "reth-e2e-test-utils", "reth-node-core", "reth-scroll-chainspec", + "reth-scroll-engine-primitives", "reth-scroll-node", "reth-scroll-primitives", "reth-tracing", "rollup-node", + "rollup-node-chain-orchestrator", "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", "rollup-node-sequencer", "rollup-node-signer", "scroll-alloy-consensus", + "scroll-alloy-hardforks", "scroll-alloy-provider", "scroll-alloy-rpc-types-engine", "scroll-db", diff --git a/crates/chain-orchestrator/Cargo.toml b/crates/chain-orchestrator/Cargo.toml index 3462acaf..fc35008e 100644 --- a/crates/chain-orchestrator/Cargo.toml +++ b/crates/chain-orchestrator/Cargo.toml @@ -16,25 +16,36 @@ alloy-eips = { workspace = true } alloy-json-rpc.workspace = true alloy-primitives.workspace = true alloy-provider.workspace = true +alloy-rpc-types-engine.workspace = true alloy-transport.workspace = true # rollup-node +scroll-alloy-provider.workspace = true scroll-db.workspace = true +scroll-engine.workspace = true +rollup-node-providers.workspace = true rollup-node-primitives.workspace = true +rollup-node-signer.workspace = true rollup-node-watcher.workspace = true # scroll +reth-scroll-engine-primitives.workspace = true +reth-scroll-node.workspace = true reth-scroll-primitives.workspace = true scroll-alloy-consensus.workspace = true scroll-alloy-hardforks.workspace = true scroll-alloy-network.workspace = true +scroll-derivation-pipeline.workspace = true scroll-network.workspace = true # reth reth-chainspec.workspace = true +reth-network-api.workspace = true reth-network-p2p.workspace = true reth-network-peers.workspace = true reth-primitives-traits.workspace = true +reth-tasks.workspace = true +reth-tokio-util.workspace = true # misc futures.workspace = true diff --git a/crates/chain-orchestrator/src/action.rs b/crates/chain-orchestrator/src/action.rs deleted file mode 100644 index 1baadcf7..00000000 --- a/crates/chain-orchestrator/src/action.rs +++ /dev/null @@ -1,57 +0,0 @@ -use super::{ChainOrchestratorError, ChainOrchestratorEvent}; -use std::{ - fmt, - future::Future, - pin::Pin, - task::{Context, Poll}, -}; - -/// A future that resolves to a `Result`. -pub(super) type PendingChainOrchestratorFuture = Pin< - Box, ChainOrchestratorError>> + Send>, ->; - -/// A type that represents a future that is being executed by the chain orchestrator. -pub(super) enum ChainOrchestratorFuture { - HandleReorg(PendingChainOrchestratorFuture), - HandleFinalized(PendingChainOrchestratorFuture), - HandleBatchCommit(PendingChainOrchestratorFuture), - HandleBatchFinalization(PendingChainOrchestratorFuture), - HandleL1Message(PendingChainOrchestratorFuture), - HandleDerivedBlock(PendingChainOrchestratorFuture), - HandleL2Block(PendingChainOrchestratorFuture), -} - -impl ChainOrchestratorFuture { - /// Polls the future to completion. - pub(super) fn poll( - &mut self, - cx: &mut Context<'_>, - ) -> Poll, ChainOrchestratorError>> { - match self { - Self::HandleReorg(fut) | - Self::HandleFinalized(fut) | - Self::HandleBatchCommit(fut) | - Self::HandleBatchFinalization(fut) | - Self::HandleL1Message(fut) | - Self::HandleDerivedBlock(fut) | - Self::HandleL2Block(fut) => fut.as_mut().poll(cx), - } - } -} - -// We implement the Debug trait for ChainOrchestratorFuture to provide a human-readable -// representation of the enum variants. -impl fmt::Debug for ChainOrchestratorFuture { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Self::HandleReorg(_) => write!(f, "HandleReorg"), - Self::HandleFinalized(_) => write!(f, "HandleFinalized"), - Self::HandleBatchCommit(_) => write!(f, "HandleBatchCommit"), - Self::HandleBatchFinalization(_) => write!(f, "HandleBatchFinalization"), - Self::HandleL1Message(_) => write!(f, "HandleL1Message"), - Self::HandleDerivedBlock(_) => write!(f, "HandleDerivedBlock"), - Self::HandleL2Block(_) => write!(f, "HandleL2Block"), - } - } -} diff --git a/crates/chain-orchestrator/src/config.rs b/crates/chain-orchestrator/src/config.rs new file mode 100644 index 00000000..9c4eb108 --- /dev/null +++ b/crates/chain-orchestrator/src/config.rs @@ -0,0 +1,39 @@ +use std::sync::Arc; + +/// Configuration for the chain orchestrator. +#[derive(Debug)] +pub struct ChainOrchestratorConfig { + /// The chain specification. + chain_spec: Arc, + /// The threshold for optimistic sync. If the received block is more than this many blocks + /// ahead of the current chain, we optimistically sync the chain. + optimistic_sync_threshold: u64, + /// The L1 message queue index at which the V2 L1 message queue was enabled. + l1_v2_message_queue_start_index: u64, +} + +impl ChainOrchestratorConfig { + /// Creates a new chain configuration. + pub fn new( + chain_spec: Arc, + optimistic_sync_threshold: u64, + l1_v2_message_queue_start_index: u64, + ) -> Self { + Self { chain_spec, optimistic_sync_threshold, l1_v2_message_queue_start_index } + } + + /// Returns a reference to the chain specification. + pub fn chain_spec(&self) -> &Arc { + &self.chain_spec + } + + /// Returns the optimistic sync threshold. + pub fn optimistic_sync_threshold(&self) -> u64 { + self.optimistic_sync_threshold + } + + /// Returns the L1 message queue index at which the V2 L1 message queue was enabled. + pub fn l1_v2_message_queue_start_index(&self) -> u64 { + self.l1_v2_message_queue_start_index + } +} diff --git a/crates/chain-orchestrator/src/consensus.rs b/crates/chain-orchestrator/src/consensus.rs new file mode 100644 index 00000000..0c969c8f --- /dev/null +++ b/crates/chain-orchestrator/src/consensus.rs @@ -0,0 +1,163 @@ +use alloy_primitives::{Address, Signature}; +use metrics::Counter; +use metrics_derive::Metrics; +use reth_primitives_traits::GotExpected; +use reth_scroll_primitives::ScrollBlock; +use rollup_node_primitives::{sig_encode_hash, ConsensusUpdate}; +use scroll_network::ConsensusError; +use std::fmt::Debug; + +/// A trait for consensus implementations. +pub trait Consensus: Send + Sync + Debug { + /// Updates the current config for the consensus. + fn update_config(&mut self, update: &ConsensusUpdate); + /// Validates a new block with the given signature. + fn validate_new_block( + &self, + block: &ScrollBlock, + signature: &Signature, + ) -> Result<(), ConsensusError>; + /// Returns a boolean indicating whether the sequencer should sequence a block. + fn should_sequence_block(&self, sequencer: &Address) -> bool; +} + +/// A no-op consensus instance. +#[non_exhaustive] +#[derive(Debug, Default)] +pub struct NoopConsensus; + +impl Consensus for NoopConsensus { + fn update_config(&mut self, _: &ConsensusUpdate) {} + + fn validate_new_block( + &self, + _block: &ScrollBlock, + _signature: &Signature, + ) -> Result<(), ConsensusError> { + Ok(()) + } + + fn should_sequence_block(&self, _sequencer: &Address) -> bool { + true + } +} + +/// The metrics for the [`SystemContractConsensus`]. +#[derive(Metrics, Clone)] +#[metrics(scope = "consensus")] +pub(crate) struct SystemContractConsensusMetrics { + /// System contract validate new block failed counter. + pub validate_new_block_failed: Counter, +} + +/// The system contract consensus. +#[derive(Debug)] +pub struct SystemContractConsensus { + authorized_signer: Address, + + /// The metrics for the [`SystemContractConsensus`]. + metrics: SystemContractConsensusMetrics, +} + +impl SystemContractConsensus { + /// Creates a new [`SystemContractConsensus`] consensus instance with the given authorized + /// signers. + pub fn new(authorized_signer: Address) -> Self { + Self { authorized_signer, metrics: SystemContractConsensusMetrics::default() } + } +} + +impl Consensus for SystemContractConsensus { + fn update_config(&mut self, update: &ConsensusUpdate) { + match update { + ConsensusUpdate::AuthorizedSigner(signer) => self.authorized_signer = *signer, + }; + } + + fn validate_new_block( + &self, + block: &ScrollBlock, + signature: &Signature, + ) -> Result<(), ConsensusError> { + let hash = sig_encode_hash(&block.header); + let signer = reth_primitives_traits::crypto::secp256k1::recover_signer(signature, hash)?; + + if self.authorized_signer != signer { + self.metrics.validate_new_block_failed.increment(1); + return Err(ConsensusError::IncorrectSigner(GotExpected { + got: signer, + expected: self.authorized_signer, + })) + } + Ok(()) + } + + fn should_sequence_block(&self, sequencer: &Address) -> bool { + sequencer == &self.authorized_signer + } +} + +#[cfg(test)] +mod tests { + use super::*; + use alloy_consensus::{Signed, TxEip1559}; + use alloy_primitives::{address, b256, bloom, bytes, TxKind, B64, U256}; + use reth_primitives_traits::Header; + use reth_scroll_primitives::ScrollBlockBody; + use std::{str::FromStr, sync::OnceLock}; + + #[test] + fn test_should_validate_block() { + let consensus = + SystemContractConsensus::new(address!("d83c4892bb5aa241b63d8c4c134920111e142a20")); + let signature = Signature::from_raw(&bytes!("6d2b8ef87f0956ea4dd10fb0725fa7196ad80c6d567a161f6b4367f95b5de6ec279142b540d3b248f08ed337bb962fa3fd83d21de622f7d6c8207272558fd15a00")).unwrap(); + + let tx_hash = OnceLock::new(); + tx_hash.get_or_init(|| { + b256!("f257edab88796a76f6d19a9fadad44b2b16c28e7aa70322cc4c6abc857128998") + }); + + let block = ScrollBlock { + header: Header { + parent_hash: b256!("3ccf36621e1f75cd1bfd2ac39ff6a00d8a5bec02e52aa7064a4860a0d02d6013"), + ommers_hash: b256!("1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347"), + beneficiary: address!("0000000000000000000000000000000000000000"), + state_root: b256!("bc6c2ccfdb3e0e78b53134f583e6d42760adcaebb23e7a6bab59503c4b98daeb"), + transactions_root: b256!("a11e1b74f0aada603d9b4e645a57d60259dc2545c0372b88e16e6b59cecac8b6"), + receipts_root: b256!("72de16699164034d2ed9930a021820e32e103ea7162b6f6a9a535d0a98f3fac0"), + logs_bloom: bloom!("0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000008000400000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000008000000000010000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"), + difficulty: U256::ONE, + number: 14916920, + gas_limit: 10000000, + gas_used: 245760, + timestamp: 1745337938, + extra_data: bytes!("0x"), + mix_hash: b256!("0000000000000000000000000000000000000000000000000000000000000000"), + nonce: B64::from(0x0000000000000000i64), + base_fee_per_gas: Some(40306624), + withdrawals_root: None, + blob_gas_used: None, + excess_blob_gas: None, + parent_beacon_block_root: None, + requests_hash: None, + }, + body: ScrollBlockBody { + transactions: vec![ + Signed::new_unhashed(TxEip1559 { + chain_id: 534352, + nonce: 145014, + gas_limit: 600000, + max_fee_per_gas: 52355852, + max_priority_fee_per_gas: 0, + to: TxKind::Call(address!("802b65b5d9016621e66003aed0b16615093f328b")), + value: U256::ZERO, + access_list: Default::default(), + input: bytes!("a00597a00000000000000000000000000000000000000000000000000000000001826cbe0000000000000000000000000000000000005eb6831c1aa0faf2055c7d53270e00000000000000000000000006efdbff2a14a7c8e15944d1f4a48f9f95f663a40000000000000000000000000000000000000000000000000000000000000001000000000000000000000000813df550a32d4a9d42010d057386429ad2328ed9000000000000000000000000000000000000000000000000000000006807befd"), + }, Signature::new(U256::from_str("12217337930795921874768983252881296563512928283585900928219483692173266513447").unwrap(), U256::from_str("37490897792770890087946325233571758133021734266092518377537449521790010698782").unwrap(), true)).into()], + ommers: vec![], + withdrawals: None, + }, + }; + consensus.validate_new_block(&block, &signature).unwrap() + } +} diff --git a/crates/chain-orchestrator/src/consolidation.rs b/crates/chain-orchestrator/src/consolidation.rs new file mode 100644 index 00000000..0da77177 --- /dev/null +++ b/crates/chain-orchestrator/src/consolidation.rs @@ -0,0 +1,68 @@ +use super::ChainOrchestratorError; +use alloy_provider::Provider; +use futures::{stream::FuturesOrdered, TryStreamExt}; +use rollup_node_primitives::{BatchInfo, BlockInfo}; +use scroll_alloy_network::Scroll; +use scroll_derivation_pipeline::{BatchDerivationResult, DerivedAttributes}; +use scroll_engine::block_matches_attributes; + +/// Reconciles a batch of derived attributes with the L2 chain to produce a reconciliation result. +pub(crate) async fn reconcile_batch>( + l2_provider: L2P, + batch: BatchDerivationResult, +) -> Result { + let mut futures = FuturesOrdered::new(); + for attributes in batch.attributes { + let fut = async { + // Fetch the block corresponding to the derived attributes from the L2 provider. + let current_block = l2_provider + .get_block(attributes.block_number.into()) + .await? + .map(|b| b.into_consensus().map_transactions(|tx| tx.inner.into_inner())); + + let current_block = if let Some(block) = current_block { + block + } else { + // The block does not exist, a reorg is needed. + return Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Reorg(attributes)); + }; + + // Check if the block matches the derived attributes. + if block_matches_attributes( + &attributes.attributes, + ¤t_block, + current_block.parent_hash, + ) { + // The block matches the derived attributes, no action is needed. + Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::UpdateSafeHead( + BlockInfo { number: current_block.number, hash: current_block.hash_slow() }, + )) + } else { + // The block does not match the derived attributes, a reorg is needed. + Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Reorg(attributes)) + } + }; + futures.push_back(fut); + } + + let actions: Vec = futures.try_collect().await?; + Ok(BatchReconciliationResult { batch_info: batch.batch_info, actions }) +} + +/// The result of reconciling a batch with the L2 chain. +#[derive(Debug)] +pub(crate) struct BatchReconciliationResult { + /// The batch info for the consolidated batch. + pub batch_info: BatchInfo, + /// The actions that must be performed on the L2 chain to consolidate the batch. + pub actions: Vec, +} + +/// An action that must be performed on the L2 chain to consolidate a block. +#[derive(Debug, Clone)] +pub(crate) enum BlockConsolidationAction { + /// Update the safe head to the given block. + UpdateSafeHead(BlockInfo), + /// Reorganize the chain with the given derived attributes. + Reorg(DerivedAttributes), +} diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index e4338bbc..11421060 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -2,6 +2,7 @@ use alloy_json_rpc::RpcError; use alloy_primitives::B256; use alloy_transport::TransportErrorKind; use scroll_db::{DatabaseError, L1MessageStart}; +use scroll_engine::EngineError; /// A type that represents an error that occurred in the chain orchestrator. #[derive(Debug, thiserror::Error)] @@ -9,6 +10,9 @@ pub enum ChainOrchestratorError { /// An error occurred while interacting with the database. #[error("database error occurred: {0}")] DatabaseError(#[from] DatabaseError), + /// An error occurred in the engine. + #[error("engine error occurred: {0}")] + EngineError(#[from] EngineError), /// An error occurred while trying to fetch the L2 block from the database. #[error("L2 block not found - block number: {0}")] L2BlockNotFoundInDatabase(u64), @@ -51,4 +55,7 @@ pub enum ChainOrchestratorError { /// An error occurred while making a JSON-RPC request to the Execution Node (EN). #[error("An error occurred while making a JSON-RPC request to the EN: {0}")] RpcError(#[from] RpcError), + /// Received an invalid block from peer. + #[error("Received an invalid block from peer")] + InvalidBlock, } diff --git a/crates/chain-orchestrator/src/handle/command.rs b/crates/chain-orchestrator/src/handle/command.rs new file mode 100644 index 00000000..5f7ca5b4 --- /dev/null +++ b/crates/chain-orchestrator/src/handle/command.rs @@ -0,0 +1,27 @@ +use crate::{ChainOrchestratorEvent, ChainOrchestratorStatus}; + +use reth_network_api::FullNetwork; +use reth_scroll_node::ScrollNetworkPrimitives; +use reth_tokio_util::EventStream; +use rollup_node_primitives::BlockInfo; +use scroll_network::ScrollNetworkHandle; +use tokio::sync::oneshot; + +/// The commands that can be sent to the rollup manager. +#[derive(Debug)] +pub enum ChainOrchestratorCommand> { + /// Command to build a new block. + BuildBlock, + /// Returns an event stream for rollup manager events. + EventListener(oneshot::Sender>), + /// Report the current status of the manager via the oneshot channel. + Status(oneshot::Sender), + /// Returns the network handle. + NetworkHandle(oneshot::Sender>), + /// Update the head of the fcs in the engine driver. + UpdateFcsHead(BlockInfo), + /// Enable automatic sequencing. + EnableAutomaticSequencing(oneshot::Sender), + /// Disable automatic sequencing. + DisableAutomaticSequencing(oneshot::Sender), +} diff --git a/crates/chain-orchestrator/src/handle/metrics.rs b/crates/chain-orchestrator/src/handle/metrics.rs new file mode 100644 index 00000000..9126de40 --- /dev/null +++ b/crates/chain-orchestrator/src/handle/metrics.rs @@ -0,0 +1,10 @@ +use metrics::Counter; +use metrics_derive::Metrics; + +/// The metrics for the [`super::ChainOrchestratorHandle`]. +#[derive(Metrics, Clone)] +#[metrics(scope = "NodeManager")] +pub(crate) struct ChainOrchestratorHandleMetrics { + /// Failed to send command to rollup manager from handle counter. + pub handle_send_command_failed: Counter, +} diff --git a/crates/chain-orchestrator/src/handle/mod.rs b/crates/chain-orchestrator/src/handle/mod.rs new file mode 100644 index 00000000..983ffcf9 --- /dev/null +++ b/crates/chain-orchestrator/src/handle/mod.rs @@ -0,0 +1,81 @@ +use super::ChainOrchestratorEvent; +// use crate::manager::metrics::HandleMetrics; +use reth_network_api::FullNetwork; +use reth_scroll_node::ScrollNetworkPrimitives; +use reth_tokio_util::EventStream; +use rollup_node_primitives::BlockInfo; +use scroll_network::ScrollNetworkHandle; +use tokio::sync::{mpsc, oneshot}; +use tracing::error; + +mod command; +pub use command::ChainOrchestratorCommand; + +mod metrics; +use metrics::ChainOrchestratorHandleMetrics; + +/// The handle used to send commands to the rollup manager. +#[derive(Debug, Clone)] +pub struct ChainOrchestratorHandle> { + /// The channel used to send commands to the rollup manager. + to_manager_tx: mpsc::UnboundedSender>, + handle_metrics: ChainOrchestratorHandleMetrics, +} + +impl> ChainOrchestratorHandle { + /// Create a new rollup manager handle. + pub fn new(to_manager_tx: mpsc::UnboundedSender>) -> Self { + Self { to_manager_tx, handle_metrics: ChainOrchestratorHandleMetrics::default() } + } + + /// Sends a command to the rollup manager. + pub fn send_command(&self, command: ChainOrchestratorCommand) { + if let Err(err) = self.to_manager_tx.send(command) { + self.handle_metrics.handle_send_command_failed.increment(1); + error!(target: "rollup::manager::handle", "Failed to send command to rollup manager: {}", err); + } + } + + /// Sends a command to the rollup manager to build a block. + pub async fn build_block(&self) { + self.send_command(ChainOrchestratorCommand::BuildBlock); + } + + /// Sends a command to the rollup manager to get the network handle. + pub async fn get_network_handle( + &self, + ) -> Result, oneshot::error::RecvError> { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::NetworkHandle(tx)); + rx.await + } + + /// Sends a command to the rollup manager to fetch an event listener for the rollup node + /// manager. + pub async fn get_event_listener( + &self, + ) -> Result, oneshot::error::RecvError> { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::EventListener(tx)); + rx.await + } + + /// Sends a command to the rollup manager to update the head of the FCS in the engine driver. + pub async fn update_fcs_head(&self, head: BlockInfo) { + self.send_command(ChainOrchestratorCommand::UpdateFcsHead(head)); + } + + /// Sends a command to the rollup manager to enable automatic sequencing. + pub async fn enable_automatic_sequencing(&self) -> Result { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::EnableAutomaticSequencing(tx)); + rx.await + } + + /// Sends a command to the rollup manager to disable automatic sequencing. + pub async fn disable_automatic_sequencing(&self) -> Result { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::DisableAutomaticSequencing(tx)); + rx.await + } +} diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index 99d59d3c..dadd8483 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -2,41 +2,56 @@ //! L2 p2p network. use alloy_consensus::Header; -use alloy_eips::{BlockHashOrNumber, Encodable2718}; -use alloy_primitives::{b256, keccak256, B256}; +use alloy_eips::Encodable2718; +use alloy_primitives::{b256, bytes::Bytes, keccak256, B256}; use alloy_provider::Provider; -use futures::{pin_mut, task::AtomicWaker, Stream, StreamExt, TryStreamExt}; +use alloy_rpc_types_engine::ExecutionData; +use futures::StreamExt; use reth_chainspec::EthChainSpec; -use reth_network_p2p::{BlockClient, BodiesClient}; +use reth_network_api::{BlockDownloaderProvider, FullNetwork, HeadersClient}; +use reth_network_p2p::{headers::client::HeadersRequest, BodiesClient}; +use reth_scroll_engine_primitives::try_into_block; +use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; +use reth_tasks::shutdown::GracefulShutdown; +use reth_tokio_util::{EventSender, EventStream}; use rollup_node_primitives::{ - BatchCommitData, BatchInfo, BlockInfo, BoundedVec, ChainImport, L1MessageEnvelope, - L2BlockInfoWithL1Messages, + BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, + L1MessageEnvelope, L2BlockInfoWithL1Messages, }; -use rollup_node_watcher::L1Notification; +use rollup_node_providers::L1Provider; +use rollup_node_signer::SignatureAsBytes; +use rollup_node_watcher::{test_utils::chain, L1Notification}; use scroll_alloy_consensus::TxL1Message; use scroll_alloy_hardforks::ScrollHardforks; use scroll_alloy_network::Scroll; +use scroll_alloy_provider::ScrollEngineApi; use scroll_db::{ Database, DatabaseError, DatabaseReadOperations, DatabaseTransactionProvider, DatabaseWriteOperations, L1MessageStart, UnwindResult, }; -use scroll_network::NewBlockWithPeer; +use scroll_derivation_pipeline::{BatchDerivationResult, DerivationPipelineNew}; +use scroll_engine::Engine; +use scroll_network::{ + BlockImportOutcome, NewBlockWithPeer, ScrollNetwork, ScrollNetworkManagerEvent, +}; use std::{ collections::{HashMap, VecDeque}, - pin::Pin, - sync::{ - atomic::{AtomicU64, Ordering}, - Arc, - }, - task::{Context, Poll}, + sync::Arc, time::Instant, + vec, }; use strum::IntoEnumIterator; -use tokio::sync::Mutex; +use tokio::sync::mpsc::{self, Receiver, UnboundedReceiver}; + +mod config; +pub use config::ChainOrchestratorConfig; -mod action; -use action::{ChainOrchestratorFuture, PendingChainOrchestratorFuture}; +mod consensus; +pub use consensus::{Consensus, NoopConsensus, SystemContractConsensus}; + +mod consolidation; +use consolidation::reconcile_batch; mod event; pub use event::ChainOrchestratorEvent; @@ -44,596 +59,303 @@ pub use event::ChainOrchestratorEvent; mod error; pub use error::ChainOrchestratorError; +mod handle; +pub use handle::{ChainOrchestratorCommand, ChainOrchestratorHandle}; + mod metrics; pub use metrics::{ChainOrchestratorItem, ChainOrchestratorMetrics}; mod retry; pub use retry::Retry; +mod sync; +pub use sync::SyncState; + +mod status; +pub use status::ChainOrchestratorStatus; + +use crate::consolidation::BlockConsolidationAction; + /// The mask used to mask the L1 message queue hash. const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); -type Chain = BoundedVec
; +const HEADER_FETCH_COUNT: u64 = 4000; + +const CONSOLIDATION_BATCH_SIZE: u64 = 500; + +const EVENT_CHANNEL_SIZE: usize = 5000; /// The [`ChainOrchestrator`] is responsible for orchestrating the progression of the L2 chain /// based on data consolidated from L1 and the data received over the p2p network. #[derive(Debug)] -pub struct ChainOrchestrator { +pub struct ChainOrchestrator< + N: FullNetwork, + ChainSpec, + L1P, + L2P, + EC, +> { + /// The configuration for the chain orchestrator. + config: ChainOrchestratorConfig, + /// The receiver for commands sent to the chain orchestrator. + handle_rx: UnboundedReceiver>, /// The `BlockClient` that is used to fetch blocks from peers over p2p. - network_client: Arc, + network_client: Arc<::Client>, /// The L2 client that is used to interact with the L2 chain. - l2_client: Arc

, - /// An in-memory representation of the optimistic chain we are following. - chain: Arc>, + l2_client: Arc, /// A reference to the database used to persist the indexed data. database: Arc, - /// A queue of pending futures. - pending_futures: VecDeque, - /// The block number of the L1 finalized block. - l1_finalized_block_number: Arc, - /// The chain specification for the chain orchestrator. - chain_spec: Arc, /// The metrics for the chain orchestrator. metrics: HashMap, - /// A boolean to represent if the [`ChainOrchestrator`] is in optimistic mode. - optimistic_mode: Arc>, - /// The threshold for optimistic sync. If the received block is more than this many blocks - /// ahead of the current chain, we optimistically sync the chain. - optimistic_sync_threshold: u64, - /// The size of the in-memory chain buffer. - chain_buffer_size: usize, - /// A boolean to represent if the L1 has been synced. - l1_synced: bool, - /// The L1 message queue index at which the V2 L1 message queue was enabled. - l1_v2_message_queue_start_index: u64, - /// The waker to notify when the engine driver should be polled. - waker: AtomicWaker, + /// The current sync state of the [`ChainOrchestrator`]. + sync_state: SyncState, + /// A receiver for [`L1Notification`]s from the [`rollup_node_watcher::L1Watcher`]. + l1_notification_rx: Receiver>, + /// The network manager that manages the scroll p2p network. + network: ScrollNetwork, + /// The consensus algorithm used by the rollup node. + consensus: Box, + /// The engine used to communicate with the execution layer. + engine: Engine, + /// The derivation pipeline used to derive L2 blocks from batches. + derivation_pipeline: DerivationPipelineNew, + /// Optional event sender for broadcasting events to listeners. + event_sender: Option>, } impl< + N: FullNetwork + Send + Sync + 'static, ChainSpec: ScrollHardforks + EthChainSpec + Send + Sync + 'static, - BC: BlockClient + Send + Sync + 'static, - P: Provider + 'static, - > ChainOrchestrator + L1P: L1Provider + Unpin + Clone + Send + Sync + 'static, + L2P: Provider + 'static, + EC: ScrollEngineApi + Sync + Send + 'static, + > ChainOrchestrator { /// Creates a new chain orchestrator. pub async fn new( database: Arc, - chain_spec: Arc, - block_client: BC, - l2_client: P, - optimistic_sync_threshold: u64, - chain_buffer_size: usize, - l1_v2_message_queue_start_index: u64, - ) -> Result { - let chain = init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; - Ok(Self { - network_client: Arc::new(block_client), - l2_client: Arc::new(l2_client), - chain: Arc::new(Mutex::new(chain)), - database, - pending_futures: Default::default(), - l1_finalized_block_number: Arc::new(AtomicU64::new(0)), - chain_spec, - metrics: ChainOrchestratorItem::iter() - .map(|i| { - let label = i.as_str(); - (i, ChainOrchestratorMetrics::new_with_labels(&[("item", label)])) - }) - .collect(), - optimistic_mode: Arc::new(Mutex::new(false)), - optimistic_sync_threshold, - chain_buffer_size, - l1_synced: false, - l1_v2_message_queue_start_index, - waker: AtomicWaker::new(), - }) - } - - /// Returns the number of pending futures. - pub fn pending_futures_len(&self) -> usize { - self.pending_futures.len() - } - - /// Wraps a pending chain orchestrator future, metering the completion of it. - pub fn handle_metered( - &mut self, - item: ChainOrchestratorItem, - chain_orchestrator_fut: PendingChainOrchestratorFuture, - ) -> PendingChainOrchestratorFuture { - let metric = self.metrics.get(&item).expect("metric exists").clone(); - let fut_wrapper = Box::pin(async move { - let now = Instant::now(); - let res = chain_orchestrator_fut.await; - metric.task_duration.record(now.elapsed().as_secs_f64()); - res - }); - fut_wrapper - } - - /// Sets the L1 synced status to the provided value. - pub fn set_l1_synced_status(&mut self, l1_synced: bool) { - self.l1_synced = l1_synced; - } - - /// Handles a new block received from a peer. - pub fn handle_block_from_peer(&mut self, block_with_peer: NewBlockWithPeer) { - let ctx = HandleBlockContext { - chain: self.chain.clone(), - l2_client: self.l2_client.clone(), - optimistic_mode: self.optimistic_mode.clone(), - optimistic_sync_threshold: self.optimistic_sync_threshold, - network_client: self.network_client.clone(), - database: self.database.clone(), - chain_buffer_size: self.chain_buffer_size, - }; - - let fut = self.handle_metered( - ChainOrchestratorItem::NewBlock, - Box::pin(async move { - Self::do_handle_block_from_peer(ctx, block_with_peer).await.map(Into::into) - }), - ); - self.pending_futures.push_back(ChainOrchestratorFuture::HandleL2Block(fut)); - self.waker.wake(); - } - - /// Handles a sequenced block. - pub fn handle_sequenced_block(&mut self, block_with_peer: NewBlockWithPeer) { - tracing::trace!( - target: "scroll::chain_orchestrator", - "Handling sequenced block {:?}", - Into::::into(&block_with_peer.block) - ); - let ctx = HandleBlockContext { - chain: self.chain.clone(), - l2_client: self.l2_client.clone(), - optimistic_mode: self.optimistic_mode.clone(), - optimistic_sync_threshold: self.optimistic_sync_threshold, - network_client: self.network_client.clone(), - database: self.database.clone(), - chain_buffer_size: self.chain_buffer_size, - }; - - let fut = self.handle_metered( - ChainOrchestratorItem::NewBlock, - Box::pin(async move { - Self::do_handle_sequenced_block(ctx, block_with_peer).await.map(Into::into) - }), - ); - self.pending_futures.push_back(ChainOrchestratorFuture::HandleL2Block(fut)); - self.waker.wake(); - } - - /// Handles a sequenced block by inserting it into the database and returning an event. - async fn do_handle_sequenced_block( - ctx: HandleBlockContext, - block_with_peer: NewBlockWithPeer, - ) -> Result { - let database = ctx.database.clone(); - let block_info: L2BlockInfoWithL1Messages = (&block_with_peer.block).into(); - Self::do_handle_block_from_peer(ctx, block_with_peer).await?; - Retry::default() - .retry("update_l1_messages_with_l2_block", || async { - let tx = database.tx_mut().await?; - tx.update_l1_messages_with_l2_block(block_info.clone()).await?; - tx.commit().await?; - Ok::<_, ChainOrchestratorError>(()) - }) - .await?; - Ok(ChainOrchestratorEvent::L2ChainCommitted(block_info, None, true)) + config: ChainOrchestratorConfig, + block_client: ::Client, + l2_provider: L2P, + l1_notification_rx: Receiver>, + network: ScrollNetwork, + consensus: Box, + engine: Engine, + derivation_pipeline: DerivationPipelineNew, + ) -> Result<(Self, ChainOrchestratorHandle), ChainOrchestratorError> { + let (handle_tx, handle_rx) = mpsc::unbounded_channel(); + let handle = ChainOrchestratorHandle::new(handle_tx); + Ok(( + Self { + network_client: Arc::new(block_client), + l2_client: Arc::new(l2_provider), + database, + config, + metrics: ChainOrchestratorItem::iter() + .map(|i| { + let label = i.as_str(); + (i, ChainOrchestratorMetrics::new_with_labels(&[("item", label)])) + }) + .collect(), + sync_state: SyncState::default(), + l1_notification_rx, + network, + consensus, + engine, + derivation_pipeline, + handle_rx, + event_sender: None, + }, + handle, + )) } - /// Handles a new block received from the network. - async fn do_handle_block_from_peer( - ctx: HandleBlockContext, - block_with_peer: NewBlockWithPeer, - ) -> Result { - let HandleBlockContext { - chain, - l2_client, - optimistic_mode, - optimistic_sync_threshold, - network_client, - database, - chain_buffer_size, - } = ctx; - let NewBlockWithPeer { block: received_block, peer_id, signature } = block_with_peer; - let mut current_chain_headers = chain.lock().await.clone().into_inner(); - let max_block_number = current_chain_headers.back().expect("chain can not be empty").number; - let min_block_number = - current_chain_headers.front().expect("chain can not be empty").number; - - // If the received block has a block number that is greater than the tip - // of the chain by the optimistic sync threshold, we optimistically sync the chain and - // update the in-memory buffer to represent the optimistic chain. - if (received_block.header.number.saturating_sub(max_block_number)) >= - optimistic_sync_threshold - { - // fetch the latest `chain_buffer_size` headers from the network for the - // optimistic chain. - let mut optimistic_headers = Chain::new(chain_buffer_size); - optimistic_headers.push_front(received_block.header.clone()); - while optimistic_headers.len() < chain_buffer_size && - optimistic_headers.first().expect("chain can not be empty").number != 0 - { - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(optimistic_headers.first().expect("chain can not be empty").number - 1), "fetching block"); - let parent_hash = - optimistic_headers.first().expect("chain can not be empty").parent_hash; - let header = Retry::default() - .retry("network_client_get_header", || async { - let header = - network_client.get_header(BlockHashOrNumber::Hash(parent_hash)).await?; - Ok::<_, ChainOrchestratorError>(header) - }) - .await?; - let header = header - .into_data() - .ok_or(ChainOrchestratorError::MissingBlockHeader { hash: parent_hash })?; - optimistic_headers.push_front(header); + /// Main run loop for the chain orchestrator. + pub async fn run(&mut self) { + loop { + tokio::select! { + Some(command) = self.handle_rx.recv() => { + tracing::info!(target: "scroll::chain_orchestrator", ?command, "Received command"); + self.handle_command(command).await; + } + Some(batch) = self.derivation_pipeline.next() => { + // Process the derived batch. + tracing::info!(target: "scroll::chain_orchestrator", ?batch, "Derived new L2 block from batch"); + if let Err(err) = self.handle_derived_batch(batch).await { + tracing::error!(target: "scroll::chain_orchestrator", ?err, "Error handling derived batch"); + } + } + Some(event) = self.network.events().next() => { + if let Err(err) = self.handle_network_event(event).await { + tracing::error!(target: "scroll::chain_orchestrator", ?err, "Error handling network event"); + } + } + Some(notification) = self.l1_notification_rx.recv(), if self.sync_state.l2().is_synced() && self.derivation_pipeline.len().await == 0 => { + let res = self.handle_l1_notification(notification).await; + self.handle_outcome(res); + } } - - *chain.lock().await = optimistic_headers; - *optimistic_mode.lock().await = true; - return Ok(ChainOrchestratorEvent::OptimisticSync(received_block)); } + } - // Check if we have already have this block in memory. - if received_block.number <= max_block_number && - received_block.number >= min_block_number && - current_chain_headers.iter().any(|h| h == &received_block.header) - { - tracing::debug!(target: "scroll::chain_orchestrator", block_hash = ?received_block.header.hash_slow(), "block already in chain"); - return Ok(ChainOrchestratorEvent::BlockAlreadyKnown( - received_block.header.hash_slow(), - peer_id, - )); + pub fn handle_outcome( + &self, + outcome: Result, ChainOrchestratorError>, + ) { + match (outcome, self.event_sender.as_ref()) { + (Ok(Some(event)), Some(sender)) => sender.notify(event), + (Err(err), _) => { + tracing::error!(target: "scroll::chain_orchestrator", ?err, "Encountered error in the chain orchestrator"); + } + _ => {} } + } - // If we are in optimistic mode and the received block has a number that is less than the - // oldest block we have in the in-memory chain we return an event signalling we have - // insufficient data to process the received block. This is an edge case. - if *optimistic_mode.lock().await && (received_block.header.number <= min_block_number) { - return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( - received_block.header.hash_slow(), - )); - }; - - // We fetch headers for the received chain until we can reconcile it with the chain we - // currently have in-memory. - let mut received_chain_headers = VecDeque::from(vec![received_block.header.clone()]); - - // We should never have a re-org that is deeper than the current safe head. - let (latest_safe_block, _) = Retry::default() - .retry("get_latest_safe_l2_info", || async { - let tx = database.tx().await?; - let (latest_safe_block, batch_info) = - tx.get_latest_safe_l2_info().await?.expect("safe block must exist"); - Ok::<_, ChainOrchestratorError>((latest_safe_block, batch_info)) - }) - .await?; - - // We search for the re-org index in the in-memory chain. - const BATCH_FETCH_SIZE: usize = 50; - let reorg_index = loop { - // If we are in optimistic mode and the received chain can not be reconciled with the - // in-memory chain we break. We will reconcile after optimistic sync has completed. - if *optimistic_mode.lock().await && - received_chain_headers.front().expect("chain can not be empty").number < - current_chain_headers.front().expect("chain can not be empty").number - { - return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( - received_block.hash_slow(), - )); + async fn handle_command(&mut self, command: ChainOrchestratorCommand) { + match command { + ChainOrchestratorCommand::BuildBlock => { + todo!(); } - - // If the current header block number is less than the latest safe block number then - // we should error. - if received_chain_headers.front().expect("chain can not be empty").number <= - latest_safe_block.number - { - return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + ChainOrchestratorCommand::EventListener(tx) => { + let _ = tx.send(self.event_listener()); } - - // If the received header tail has a block number that is less than the current header - // tail then we should fetch more headers for the current chain to aid - // reconciliation. - if received_chain_headers.front().expect("chain can not be empty").number < - current_chain_headers.front().expect("chain can not be empty").number - { - for _ in 0..BATCH_FETCH_SIZE { - if current_chain_headers - .front() - .expect("chain can not be empty") - .number - .saturating_sub(1) <= - latest_safe_block.number - { - tracing::info!(target: "scroll::chain_orchestrator", hash = %latest_safe_block.hash, number = %latest_safe_block.number, "reached safe block number for current chain - terminating fetching."); - break; - } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(current_chain_headers.front().expect("chain can not be empty").number - 1), "fetching block for current chain"); - if let Some(block) = l2_client - .get_block_by_hash( - current_chain_headers - .front() - .expect("chain can not be empty") - .parent_hash, - ) - .await? - { - let header = block.into_consensus_header(); - current_chain_headers.push_front(header.clone()); - } else { - return Err(ChainOrchestratorError::MissingBlockHeader { - hash: current_chain_headers - .front() - .expect("chain can not be empty") - .parent_hash, - }); - } - } + ChainOrchestratorCommand::Status(tx) => { + let status = ChainOrchestratorStatus { + sync_state: self.sync_state.clone(), + forkchoice_state: self.engine.fcs().clone(), + }; + let _ = tx.send(status); } - - // We search the in-memory chain to see if we can reconcile the block import. - if let Some(pos) = current_chain_headers.iter().rposition(|h| { - h.hash_slow() == - received_chain_headers.front().expect("chain can not be empty").parent_hash - }) { - // If the received fork is older than the current chain, we return an event - // indicating that we have received an old fork. - if (pos < current_chain_headers.len() - 1) && - current_chain_headers.get(pos + 1).expect("chain can not be empty").timestamp > - received_chain_headers - .front() - .expect("chain can not be empty") - .timestamp - { - return Ok(ChainOrchestratorEvent::OldForkReceived { - headers: received_chain_headers.into(), - peer_id, - signature, - }); - } - break pos; + ChainOrchestratorCommand::NetworkHandle(tx) => { + let _ = tx.send(self.network.handle().clone()); } - - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_chain_headers.front().expect("chain can not be empty").number - 1), "fetching block"); - if let Some(header) = Retry::default() - .retry("network_client_get_header", || async { - let header = network_client - .get_header(BlockHashOrNumber::Hash( - received_chain_headers - .front() - .expect("chain can not be empty") - .parent_hash, - )) - .await? - .into_data(); - Ok::<_, ChainOrchestratorError>(header) - }) - .await? - { - received_chain_headers.push_front(header.clone()); - } else { - return Err(ChainOrchestratorError::MissingBlockHeader { - hash: received_chain_headers - .front() - .expect("chain can not be empty") - .parent_hash, - }); + ChainOrchestratorCommand::UpdateFcsHead(_head) => { + todo!() } - }; - - // Fetch the blocks associated with the new chain headers. - let new_blocks = if received_chain_headers.len() == 1 { - vec![received_block] - } else { - fetch_blocks_from_network(received_chain_headers.clone().into(), network_client.clone()) - .await - }; - - // If we are not in optimistic mode, we validate the L1 messages in the new blocks. - if !*optimistic_mode.lock().await { - validate_l1_messages(&new_blocks, &database).await?; - } - - match reorg_index { - // If this is a simple chain extension, we can just extend the in-memory chain and emit - // a ChainExtended event. - position if position == current_chain_headers.len() - 1 => { - // Update the chain with the new blocks. - current_chain_headers.extend(new_blocks.iter().map(|b| b.header.clone())); - let mut new_chain = Chain::new(chain_buffer_size); - new_chain.extend(current_chain_headers); - *chain.lock().await = new_chain; - - Ok(ChainOrchestratorEvent::ChainExtended(ChainImport::new( - new_blocks, peer_id, signature, - ))) + ChainOrchestratorCommand::EnableAutomaticSequencing(_tx) => { + todo!() } - // If we are re-organizing the in-memory chain, we need to split the chain at the reorg - // point and extend it with the new blocks. - position => { - // reorg the in-memory chain to the new chain and issue a reorg event. - let mut new_chain = Chain::new(chain_buffer_size); - new_chain.extend(current_chain_headers.iter().take(position).cloned()); - new_chain.extend(received_chain_headers); - *chain.lock().await = new_chain; - - Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( - new_blocks, peer_id, signature, - ))) + ChainOrchestratorCommand::DisableAutomaticSequencing(_tx) => { + todo!() } } } - /// Persist L1 consolidate blocks in the database. - pub fn persist_l1_consolidated_blocks( - &mut self, - block_infos: Vec, - batch_info: BatchInfo, - ) { - let database = self.database.clone(); - let fut = self.handle_metered( - ChainOrchestratorItem::InsertConsolidatedL2Blocks, - Box::pin(async move { - let head = block_infos.last().expect("block info must not be empty").clone(); - Retry::default() - .retry("insert_block", || async { - let tx = database.tx_mut().await?; - for block in block_infos.clone() { - tx.insert_block(block, batch_info).await?; - } - tx.commit().await?; - Ok::<_, ChainOrchestratorError>(()) - }) - .await?; - Result::<_, ChainOrchestratorError>::Ok(Some( - ChainOrchestratorEvent::L2ConsolidatedBlockCommitted(head), - )) - }), - ); - - self.pending_futures.push_back(ChainOrchestratorFuture::HandleDerivedBlock(fut)); - self.waker.wake(); + /// Returns a new event listener for the rollup node manager. + pub fn event_listener(&mut self) -> EventStream { + if let Some(event_sender) = &self.event_sender { + return event_sender.new_listener(); + }; + + let event_sender = EventSender::new(EVENT_CHANNEL_SIZE); + let event_listener = event_sender.new_listener(); + self.event_sender = Some(event_sender); + + event_listener } - /// Consolidates L2 blocks from the network which have been validated - pub fn consolidate_validated_l2_blocks(&mut self, block_info: Vec) { - let database = self.database.clone(); - let l1_synced = self.l1_synced; - let optimistic_mode = self.optimistic_mode.clone(); - let chain = self.chain.clone(); - let l2_client = self.l2_client.clone(); - let chain_buffer_size = self.chain_buffer_size; - let fut = self.handle_metered( - ChainOrchestratorItem::InsertL2Block, - Box::pin(async move { - // If we are in optimistic mode and the L1 is synced, we consolidate the chain - // and disable optimistic mode to enter consolidated mode - // (consolidated_mode = !optimistic_mode). - let consolidated = if !*optimistic_mode.lock().await { - true - } else if l1_synced && *optimistic_mode.lock().await { - consolidate_chain( - database.clone(), - block_info.clone(), - chain, - l2_client, - chain_buffer_size, + /// Handles a derived batch by inserting the derived blocks into the database. + async fn handle_derived_batch( + &mut self, + batch: BatchDerivationResult, + ) -> Result<(), ChainOrchestratorError> { + tracing::info!(target: "scroll::chain_orchestrator", batch_info = ?batch.batch_info, num_blocks = batch.attributes.len(), "Handling derived batch"); + + let batch_reconciliation_result = reconcile_batch(&self.l2_client, batch).await?; + let mut batch_consolidation_result = + BatchConsolidationOutcome::new(batch_reconciliation_result.batch_info); + for action in batch_reconciliation_result.actions { + match action { + BlockConsolidationAction::UpdateSafeHead(block_info) => { + tracing::info!(target: "scroll::chain_orchestrator", ?block_info, "Updating safe head to consolidated block"); + self.engine.update_fcs(None, Some(block_info), Some(block_info)).await?; + batch_consolidation_result + .push_block(BlockConsolidationOutcome::Consolidated(block_info)); + } + BlockConsolidationAction::Reorg(attributes) => { + tracing::info!(target: "scroll::chain_orchestrator", block_number = ?attributes.block_number, "Reorging chain to derived block"); + // We reorg the head to the safe block and then build the payload for the + // attributes. + let head = *self.engine.fcs().safe_block_info(); + let fcu = self.engine.build_payload(Some(head), attributes.attributes).await?; + let payload = self + .engine + .get_payload(fcu.payload_id.expect("payload_id can not be None")) + .await?; + let block: ScrollBlock = try_into_block( + ExecutionData { payload: payload.into(), sidecar: Default::default() }, + self.config.chain_spec().clone(), ) - .await?; - *optimistic_mode.lock().await = false; - true - } else { - false - }; + .expect("block must be valid"); - // Insert the blocks into the database. - let head = block_info.last().expect("block info must not be empty").clone(); - Retry::default() - .retry("update_l1_messages_from_l2_blocks", || async { - let tx = database.tx_mut().await?; - tx.update_l1_messages_from_l2_blocks(block_info.clone()).await?; - tx.commit().await?; - Ok::<_, ChainOrchestratorError>(()) - }) - .await?; + // Update the forkchoice state to the new head. + self.engine.update_fcs(Some((&block).into()), None, None).await?; - Result::<_, ChainOrchestratorError>::Ok(Some( - ChainOrchestratorEvent::L2ChainCommitted(head, None, consolidated), - )) - }), - ); + let block_info: L2BlockInfoWithL1Messages = (&block).into(); + batch_consolidation_result + .push_block(BlockConsolidationOutcome::Reorged(block_info)); + } + } + } + + // Insert the batch consolidation outcome into the database. + let tx = self.database.tx_mut().await?; + tx.insert_batch_consolidation_outcome(batch_consolidation_result).await?; + tx.commit().await?; - self.pending_futures.push_back(ChainOrchestratorFuture::HandleDerivedBlock(fut)); - self.waker.wake(); + Ok(()) } - /// Handles an event from the L1. - pub fn handle_l1_notification(&mut self, event: L1Notification) { - let fut = match event { - L1Notification::Reorg(block_number) => { - ChainOrchestratorFuture::HandleReorg(self.handle_metered( - ChainOrchestratorItem::L1Reorg, - Box::pin(Self::handle_l1_reorg( - self.database.clone(), - self.chain_spec.clone(), - block_number, - self.l2_client.clone(), - self.chain.clone(), - )), - )) - } - L1Notification::NewBlock(_) | L1Notification::Consensus(_) => return, - L1Notification::Finalized(block_number) => { - ChainOrchestratorFuture::HandleFinalized(self.handle_metered( - ChainOrchestratorItem::L1Finalization, - Box::pin(Self::handle_finalized( - self.database.clone(), - block_number, - self.l1_finalized_block_number.clone(), - )), - )) - } - L1Notification::BatchCommit(batch) => { - ChainOrchestratorFuture::HandleBatchCommit(self.handle_metered( - ChainOrchestratorItem::BatchCommit, - Box::pin(Self::handle_batch_commit(self.database.clone(), batch)), - )) + /// Handles an L1 notification. + async fn handle_l1_notification( + &mut self, + notification: Arc, + ) -> Result, ChainOrchestratorError> { + match &*notification { + L1Notification::Reorg(block_number) => self.handle_l1_reorg(*block_number).await, + L1Notification::Consensus(update) => { + self.consensus.update_config(update); + Ok(None) } + L1Notification::NewBlock(_) => Ok(None), + L1Notification::Finalized(block_number) => self.handle_finalized(*block_number).await, + L1Notification::BatchCommit(batch) => self.handle_batch_commit(batch.clone()).await, L1Notification::L1Message { message, block_number, block_timestamp: _ } => { - ChainOrchestratorFuture::HandleL1Message(self.handle_metered( - ChainOrchestratorItem::L1Message, - Box::pin(Self::handle_l1_message( - self.l1_v2_message_queue_start_index, - self.database.clone(), - message, - block_number, - )), - )) - } - L1Notification::BatchFinalization { hash: _hash, index, block_number } => { - ChainOrchestratorFuture::HandleBatchFinalization(self.handle_metered( - ChainOrchestratorItem::BatchFinalization, - Box::pin(Self::handle_batch_finalization( - self.database.clone(), - index, - block_number, - self.l1_finalized_block_number.clone(), - )), - )) + self.handle_l1_message(message.clone(), *block_number).await } L1Notification::Synced => { - self.set_l1_synced_status(true); - return + tracing::info!(target: "scroll::chain_orchestrator", "L1 is now synced"); + self.sync_state.l1_mut().set_synced(); + if self.sync_state.is_synced() { + self.consolidate_chain().await?; + } + Ok(None) } - }; - - self.pending_futures.push_back(fut); + L1Notification::BatchFinalization { hash: _hash, index, block_number } => { + self.handle_batch_finalization(*index, *block_number).await + } + } } /// Handles a reorganization event by deleting all indexed data which is greater than the /// provided block number. async fn handle_l1_reorg( - database: Arc, - chain_spec: Arc, - l1_block_number: u64, - l2_client: Arc

, - current_chain: Arc>, + &mut self, + block_number: u64, ) -> Result, ChainOrchestratorError> { + let metric = self.metrics.get(&ChainOrchestratorItem::L1Reorg).expect("metric exists"); + let now = Instant::now(); let UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info } = Retry::default() .retry("unwind", || async { - let txn = database.tx_mut().await?; + let txn = self.database.tx_mut().await?; let UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info, - } = txn.unwind(chain_spec.genesis_hash(), l1_block_number).await?; + } = txn.unwind(self.config.chain_spec().genesis_hash(), block_number).await?; txn.commit().await?; Ok::<_, ChainOrchestratorError>(UnwindResult { l1_block_number, @@ -643,21 +365,27 @@ impl< }) }) .await?; + let l2_head_block_info = if let Some(block_number) = l2_head_block_number { // Fetch the block hash of the new L2 head block. - let block_hash = l2_client + let block_hash = self + .l2_client .get_block_by_number(block_number.into()) .await? .expect("L2 head block must exist") .header .hash_slow(); - // Remove all blocks in the in-memory chain that are greater than the new L2 head block. - let mut current_chain_headers = current_chain.lock().await; - current_chain_headers.inner_mut().retain(|h| h.number <= block_number); + Some(BlockInfo { number: block_number, hash: block_hash }) } else { None }; + + // TODO: Add retry logic + self.engine.update_fcs(l2_head_block_info, l2_safe_block_info, None).await?; + + metric.task_duration.record(now.elapsed().as_secs_f64()); + Ok(Some(ChainOrchestratorEvent::L1Reorg { l1_block_number, queue_index, @@ -669,16 +397,19 @@ impl< /// Handles a finalized event by updating the chain orchestrator L1 finalized block, returning /// the new finalized L2 chain block and the list of finalized batches. async fn handle_finalized( - database: Arc, + &mut self, block_number: u64, - l1_block_number: Arc, ) -> Result, ChainOrchestratorError> { + let metric = + self.metrics.get(&ChainOrchestratorItem::L1Finalization).expect("metric exists"); + let now = Instant::now(); + let finalized_batches = Retry::default() .retry("handle_finalized", || async { - let tx = database.tx_mut().await?; + let tx = self.database.tx_mut().await?; // Set the latest finalized L1 block in the database. - tx.set_latest_finalized_l1_block_number(block_number).await?; + tx.set_finalized_l1_block_number(block_number).await?; // Get all unprocessed batches that have been finalized by this L1 block // finalization. @@ -691,55 +422,26 @@ impl< }) .await?; - // Update the chain orchestrator L1 block number. - l1_block_number.store(block_number, Ordering::Relaxed); - - Ok(Some(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_batches))) - } - - /// Handles an L1 message by inserting it into the database. - async fn handle_l1_message( - l1_v2_message_queue_start_index: u64, - database: Arc, - l1_message: TxL1Message, - l1_block_number: u64, - ) -> Result, ChainOrchestratorError> { - let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); - let queue_hash = - compute_l1_message_queue_hash(&database, &l1_message, l1_v2_message_queue_start_index) - .await?; - let l1_message = L1MessageEnvelope::new(l1_message, l1_block_number, None, queue_hash); + for batch in &finalized_batches { + self.derivation_pipeline.push(Arc::new(*batch)).await; + } - // Perform a consistency check to ensure the previous L1 message exists in the database. - Retry::default() - .retry("handle_l1_message", || async { - let tx = database.tx_mut().await?; - if l1_message.transaction.queue_index > 0 && - tx.get_l1_message_by_index(l1_message.transaction.queue_index - 1) - .await? - .is_none() - { - return Err(ChainOrchestratorError::L1MessageQueueGap( - l1_message.transaction.queue_index, - )) - } + metric.task_duration.record(now.elapsed().as_secs_f64()); - tx.insert_l1_message(l1_message.clone()).await?; - tx.commit().await?; - Ok::<_, ChainOrchestratorError>(()) - }) - .await?; - Ok(Some(event)) + Ok(Some(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_batches))) } /// Handles a batch input by inserting it into the database. async fn handle_batch_commit( - database: Arc, + &self, batch: BatchCommitData, ) -> Result, ChainOrchestratorError> { + let metric = self.metrics.get(&ChainOrchestratorItem::BatchCommit).expect("metric exists"); + let now = Instant::now(); + let event = Retry::default() .retry("handle_batch_commit", || async { - let tx = database.tx_mut().await?; + let tx = self.database.tx_mut().await?; let batch_clone = batch.clone(); let prev_batch_index = batch_clone.clone().index - 1; @@ -774,31 +476,36 @@ impl< }) .await?; + metric.task_duration.record(now.elapsed().as_secs_f64()); + Ok(event) } /// Handles a batch finalization event by updating the batch input in the database. async fn handle_batch_finalization( - database: Arc, + &mut self, batch_index: u64, block_number: u64, - finalized_block_number: Arc, ) -> Result, ChainOrchestratorError> { - Retry::default() + let event = Retry::default() .retry("handle_batch_finalization", || async { - let tx = database.tx_mut().await?; + let tx = self.database.tx_mut().await?; // finalize all batches up to `batch_index`. tx.finalize_batches_up_to_index(batch_index, block_number).await?; // Get all unprocessed batches that have been finalized by this L1 block // finalization. - let finalized_block_number = finalized_block_number.load(Ordering::Relaxed); + let finalized_block_number = tx + .get_finalized_l1_block_number() + .await? + .expect("finalized block number must exist"); if finalized_block_number >= block_number { let finalized_batches = tx .fetch_and_update_unprocessed_finalized_batches(finalized_block_number) .await?; tx.commit().await?; + return Ok(Some(ChainOrchestratorEvent::BatchFinalized( block_number, finalized_batches, @@ -808,1032 +515,1218 @@ impl< tx.commit().await?; Ok::<_, ChainOrchestratorError>(None) }) - .await - } -} - -/// Computes the queue hash by taking the previous queue hash and performing a 2-to-1 hash with the -/// current transaction hash using keccak. It then applies a mask to the last 32 bits as these bits -/// are used to store the timestamp at which the message was enqueued in the contract. For the first -/// message in the queue, the previous queue hash is zero. If the L1 message queue index is before -/// migration to `L1MessageQueueV2`, the queue hash will be None. -/// -/// The solidity contract (`L1MessageQueueV2.sol`) implementation is defined here: -async fn compute_l1_message_queue_hash( - database: &Arc, - l1_message: &TxL1Message, - l1_v2_message_queue_start_index: u64, -) -> Result>, ChainOrchestratorError> { - let queue_hash = if l1_message.queue_index == l1_v2_message_queue_start_index { - let mut input = B256::default().to_vec(); - input.append(&mut l1_message.tx_hash().to_vec()); - Some(keccak256(input) & L1_MESSAGE_QUEUE_HASH_MASK) - } else if l1_message.queue_index > l1_v2_message_queue_start_index { - let index = l1_message.queue_index - 1; - let mut input = Retry::default() - .retry("get_l1_message_by_index", || async { - let tx = database.tx().await?; - let input = tx.get_l1_message_by_index(index).await?; - Ok::<_, ChainOrchestratorError>(input) - }) - .await? - .map(|m| m.queue_hash) - .ok_or(DatabaseError::L1MessageNotFound(L1MessageStart::Index(index)))? - .unwrap_or_default() - .to_vec(); - input.append(&mut l1_message.tx_hash().to_vec()); - Some(keccak256(input) & L1_MESSAGE_QUEUE_HASH_MASK) - } else { - None - }; - Ok(queue_hash) -} - -async fn init_chain_from_db + 'static>( - database: &Arc, - l2_client: &P, - chain_buffer_size: usize, -) -> Result, ChainOrchestratorError> { - let blocks = { - let mut blocks = Vec::with_capacity(chain_buffer_size); - let tx = Retry::default() - .retry("get_l2_blocks_new_tx", || async { - let tx = database.tx().await?; - Ok::<_, ChainOrchestratorError>(tx) - }) - .await?; - let blocks_stream = Retry::default() - .retry("get_l2_blocks", || async { - let stream = tx.get_l2_blocks().await?; - Ok::<_, ChainOrchestratorError>(stream) - }) - .await? - .take(chain_buffer_size); - pin_mut!(blocks_stream); - while let Some(block_info) = blocks_stream.as_mut().try_next().await? { - let header = l2_client - .get_block_by_hash(block_info.hash) - .await? - .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_info.number))? - .header - .into_consensus(); - blocks.push(header); - } - blocks.reverse(); - blocks - }; - let mut chain: Chain = Chain::new(chain_buffer_size); - chain.extend(blocks); - Ok(chain) -} + .await; -impl< - ChainSpec: ScrollHardforks + 'static, - BC: BlockClient + Send + Sync + 'static, - P: Provider + Send + Sync + 'static, - > Stream for ChainOrchestrator -{ - type Item = Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - // Register the waker such that we can wake when required. - self.waker.register(cx.waker()); - - // Remove and poll the next future in the queue - while let Some(mut action) = self.pending_futures.pop_front() { - match action.poll(cx) { - Poll::Ready(result) => match result { - Ok(None) => {} - Ok(Some(event)) => return Poll::Ready(Some(Ok(event))), - Err(e) => return Poll::Ready(Some(Err(e))), - }, - Poll::Pending => { - self.pending_futures.push_front(action); - return Poll::Pending - } - }; + if let Ok(Some(ChainOrchestratorEvent::BatchFinalized(_, batches))) = &event { + for batch in batches { + self.derivation_pipeline.push(Arc::new(*batch)).await; + } } - Poll::Pending + event } -} -struct HandleBlockContext { - pub chain: Arc>, - pub l2_client: Arc

, - pub optimistic_mode: Arc>, - pub optimistic_sync_threshold: u64, - pub network_client: Arc, - pub database: Arc, - pub chain_buffer_size: usize, -} - -/// Consolidates the chain by reconciling the in-memory chain with the L2 client and database. -/// This is used to ensure that the in-memory chain is consistent with the L2 chain. -async fn consolidate_chain + 'static>( - database: Arc, - validated_chain: Vec, - current_chain: Arc>, - l2_client: P, - chain_buffer_size: usize, -) -> Result<(), ChainOrchestratorError> { - // Take the current in memory chain. - let chain = current_chain.lock().await.clone(); - - // Find highest common ancestor between the in-memory chain and the validated chain import. - let hca_index = chain.iter().rposition(|h| { - let h_hash = h.hash_slow(); - validated_chain.iter().any(|b| b.block_info.hash == h_hash) - }); - - // If we do not have a common ancestor this means that the chain has reorged recently and the - // validated chain import is no longer valid. This case should be very rare. If this occurs we - // return an error and wait for the next validated block import to reconcile the chain. This is - // more a safety check to ensure that we do not accidentally consolidate a chain that is not - // part of the in-memory chain. - if hca_index.is_none() { - // If we do not have a common ancestor, we return an error. - *current_chain.lock().await = chain; - return Err(ChainOrchestratorError::ChainInconsistency); - } + /// Handles an L1 message by inserting it into the database. + async fn handle_l1_message( + &self, + l1_message: TxL1Message, + l1_block_number: u64, + ) -> Result, ChainOrchestratorError> { + let metric = self.metrics.get(&ChainOrchestratorItem::L1Message).expect("metric exists"); + let now = Instant::now(); - // From this point on we are no longer interested in the validated chain import as we have - // already concluded it is part of the in-memory chain. The remainder of this function is - // concerned with reconciling the in-memory chain with the safe head determined from L1 - // consolidation. - - // Fetch the safe head from the database. We use this as a trust anchor to reconcile the chain - // back to. - let safe_head = Retry::default() - .retry("get_latest_safe_l2_info", || async { - let tx = database.tx().await?; - let safe_head = tx.get_latest_safe_l2_info().await?.expect("safe head must exist").0; - Ok::<_, ChainOrchestratorError>(safe_head) - }) + let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); + let queue_hash = compute_l1_message_queue_hash( + &self.database, + &l1_message, + self.config.l1_v2_message_queue_start_index(), + ) .await?; + let l1_message = L1MessageEnvelope::new(l1_message, l1_block_number, None, queue_hash); - // If the in-memory chain contains the safe head, we check if the safe hash from the - // database (L1 consolidation) matches the in-memory value. If it does not match, we return an - // error as the in-memory chain is a fork that does not respect L1 consolidated data. This edge - // case should not happen unless the sequencer is trying to reorg a safe block. - let in_mem_safe_hash = - chain.iter().find(|b| b.number == safe_head.number).map(|b| b.hash_slow()); - if let Some(in_mem_safe_hash) = in_mem_safe_hash { - if in_mem_safe_hash != safe_head.hash { - // If we did not consolidate back to the safe head, we return an error. - *current_chain.lock().await = - init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; - - return Err(ChainOrchestratorError::ChainInconsistency); - } - }; - - let mut blocks_to_consolidate = VecDeque::new(); - for header in chain.iter() { - let block = l2_client.get_block_by_hash(header.hash_slow()).full().await.unwrap().unwrap(); - let block = block.into_consensus().map_transactions(|tx| tx.inner.into_inner()); - blocks_to_consolidate.push_back(block); - } - - // If we do not have the safe header in the in-memory chain we should recursively fetch blocks - // from the EN until we reach the safe block and assert that the safe head matches. - if in_mem_safe_hash.is_none() { - while blocks_to_consolidate.front().expect("chain can not be empty").header.number > - safe_head.number - { - let parent_hash = - blocks_to_consolidate.front().expect("chain can not be empty").header.parent_hash; - let block = l2_client.get_block_by_hash(parent_hash).full().await.unwrap().unwrap(); - let block = block.into_consensus().map_transactions(|tx| tx.inner.into_inner()); - blocks_to_consolidate.push_front(block); - } - - // If the safe head of the fetched chain does not match the safe head stored in database we - // should return an error. - if blocks_to_consolidate.front().unwrap().header.hash_slow() != safe_head.hash { - *current_chain.lock().await = - init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; - return Err(ChainOrchestratorError::ChainInconsistency); - } - } - - // TODO: modify `validate_l1_messages` to accept any type that can provide an iterator over - // `&ScrollBlock` instead of requiring a `Vec`. - let blocks_to_consolidate: Vec = blocks_to_consolidate.into_iter().collect(); - validate_l1_messages(&blocks_to_consolidate, &database).await?; - - // Set the chain which has now been consolidated. - *current_chain.lock().await = chain; - - Ok(()) -} - -async fn fetch_blocks_from_network + Send + Sync + 'static>( - headers: Vec

, - client: Arc, -) -> Vec { - let mut blocks = Vec::new(); - // TODO: migrate to `get_block_bodies_with_range_hint`. - let bodies = client - .get_block_bodies(headers.iter().map(|h| h.hash_slow()).collect()) - .await - .expect("Failed to fetch block bodies") - .into_data(); - - for (header, body) in headers.into_iter().zip(bodies) { - blocks.push(ScrollBlock::new(header, body)); - } + // Perform a consistency check to ensure the previous L1 message exists in the database. + Retry::default() + .retry("handle_l1_message", || async { + let tx = self.database.tx_mut().await?; + if l1_message.transaction.queue_index > 0 && + tx.get_l1_message_by_index(l1_message.transaction.queue_index - 1) + .await? + .is_none() + { + return Err(ChainOrchestratorError::L1MessageQueueGap( + l1_message.transaction.queue_index, + )) + } - blocks -} + tx.insert_l1_message(l1_message.clone()).await?; + tx.commit().await?; + Ok::<_, ChainOrchestratorError>(()) + }) + .await?; -/// Validates the L1 messages in the provided blocks against the expected L1 messages synced from -/// L1. -async fn validate_l1_messages( - blocks: &[ScrollBlock], - database: &Arc, -) -> Result<(), ChainOrchestratorError> { - let l1_message_hashes = blocks - .iter() - .flat_map(|block| { - // Get the L1 messages from the block body. - block - .body - .transactions() - .filter(|&tx| tx.is_l1_message()) - // The hash for L1 messages is the trie hash of the transaction. - .map(|tx| tx.trie_hash()) - .collect::>() - }) - .collect::>(); - // TODO: instead of using `l1_message_hashes.first().map(|tx| L1MessageStart::Hash(*tx))` to - // determine the start of the L1 message stream, we should use a more robust method to determine - // the start of the L1 message stream. - let tx = Retry::default() - .retry("get_l1_messages_new_tx", || async { - let tx = database.tx().await?; - Ok::<_, ChainOrchestratorError>(tx) - }) - .await?; - let l1_message_stream = Retry::default() - .retry("get_l1_messages", || async { - let messages = tx - .get_l1_messages(l1_message_hashes.first().map(|tx| L1MessageStart::Hash(*tx))) - .await?; - Ok::<_, ChainOrchestratorError>(messages) - }) - .await?; - pin_mut!(l1_message_stream); - - for message_hash in l1_message_hashes { - // Get the expected L1 message from the database. - let expected_hash = l1_message_stream - .as_mut() - .next() - .await - .map(|m| m.map(|msg| msg.transaction.tx_hash())) - .transpose()? - .ok_or(ChainOrchestratorError::L1MessageNotFound(L1MessageStart::Hash(message_hash)))?; - - // If the received and expected L1 messages do not match return an error. - if message_hash != expected_hash { - return Err(ChainOrchestratorError::L1MessageMismatch { - expected: expected_hash, - actual: message_hash, - }); - } - } - Ok(()) -} + metric.task_duration.record(now.elapsed().as_secs_f64()); -#[cfg(test)] -mod test { - use std::vec; - - use super::*; - use alloy_consensus::Header; - use alloy_eips::{BlockHashOrNumber, BlockNumHash}; - use alloy_primitives::{address, bytes, B256, U256}; - use alloy_provider::{ProviderBuilder, RootProvider}; - use alloy_transport::mock::Asserter; - use arbitrary::{Arbitrary, Unstructured}; - use futures::StreamExt; - use parking_lot::Mutex; - use rand::Rng; - use reth_eth_wire_types::HeadersDirection; - use reth_network_p2p::{ - download::DownloadClient, - error::PeerRequestResult, - headers::client::{HeadersClient, HeadersRequest}, - priority::Priority, - BodiesClient, - }; - use reth_network_peers::{PeerId, WithPeerId}; - use reth_primitives_traits::Block; - use reth_scroll_chainspec::{ScrollChainSpec, SCROLL_MAINNET}; - use rollup_node_primitives::BatchCommitData; - use scroll_alloy_network::Scroll; - use scroll_db::test_utils::setup_test_db; - use std::{collections::HashMap, ops::RangeInclusive, sync::Arc}; - - type ScrollBody = ::Body; - - const TEST_OPTIMISTIC_SYNC_THRESHOLD: u64 = 100; - const TEST_CHAIN_BUFFER_SIZE: usize = 2000; - const TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY: u64 = 953885; - - /// A headers+bodies client that stores the headers and bodies in memory, with an artificial - /// soft bodies response limit that is set to 20 by default. - /// - /// This full block client can be [Clone]d and shared between multiple tasks. - #[derive(Clone, Debug)] - struct TestScrollFullBlockClient { - headers: Arc>>, - bodies: Arc::Body>>>, - // soft response limit, max number of bodies to respond with - soft_limit: usize, + Ok(Some(event)) } - impl Default for TestScrollFullBlockClient { - fn default() -> Self { - let mainnet_genesis: reth_scroll_primitives::ScrollBlock = - serde_json::from_str(include_str!("../testdata/genesis_block.json")).unwrap(); - let (header, body) = mainnet_genesis.split(); - let hash = header.hash_slow(); - let headers = HashMap::from([(hash, header)]); - let bodies = HashMap::from([(hash, body)]); - Self { - headers: Arc::new(Mutex::new(headers)), - bodies: Arc::new(Mutex::new(bodies)), - soft_limit: 20, + // /// Wraps a pending chain orchestrator future, metering the completion of it. + // pub fn handle_metered( + // &mut self, + // item: ChainOrchestratorItem, + // chain_orchestrator_fut: PendingChainOrchestratorFuture, + // ) -> PendingChainOrchestratorFuture { + // let metric = self.metrics.get(&item).expect("metric exists").clone(); + // let fut_wrapper = Box::pin(async move { + // let now = Instant::now(); + // let res = chain_orchestrator_fut.await; + // metric.task_duration.record(now.elapsed().as_secs_f64()); + // res + // }); + // fut_wrapper + // } + + async fn handle_network_event( + &mut self, + event: ScrollNetworkManagerEvent, + ) -> Result<(), ChainOrchestratorError> { + match event { + ScrollNetworkManagerEvent::NewBlock(block_with_peer) => { + self.handle_block_from_peer(block_with_peer).await? } } + Ok(()) } - impl DownloadClient for TestScrollFullBlockClient { - /// Reports a bad message from a specific peer. - fn report_bad_message(&self, _peer_id: PeerId) {} + /// Handles a new block received from a peer. + async fn handle_block_from_peer( + &mut self, + block_with_peer: NewBlockWithPeer, + ) -> Result<(), ChainOrchestratorError> { + tracing::debug!(target: "scroll::chain_orchestrator", block_hash = ?block_with_peer.block.header.hash_slow(), block_number = ?block_with_peer.block.number, peer_id = ?block_with_peer.peer_id, "Received new block from peer"); + + if let Err(err) = + self.consensus.validate_new_block(&block_with_peer.block, &block_with_peer.signature) + { + tracing::error!(target: "scroll::node::manager", ?err, "consensus checks failed on block {:?} from peer {:?}", block_with_peer.block.hash_slow(), block_with_peer.peer_id); + self.network.handle().block_import_outcome(BlockImportOutcome { + peer: block_with_peer.peer_id, + result: Err(err.into()), + }); - /// Retrieves the number of connected peers. - /// - /// Returns the number of connected peers in the test scenario (1). - fn num_connected_peers(&self) -> usize { - 1 + return Ok(()); } - } - /// Implements the `HeadersClient` trait for the `TestFullBlockClient` struct. - impl HeadersClient for TestScrollFullBlockClient { - type Header = Header; - /// Specifies the associated output type. - type Output = futures::future::Ready>>; - - /// Retrieves headers with a given priority level. - /// - /// # Arguments - /// - /// * `request` - A `HeadersRequest` indicating the headers to retrieve. - /// * `_priority` - A `Priority` level for the request. - /// - /// # Returns - /// - /// A `Ready` future containing a `PeerRequestResult` with a vector of retrieved headers. - fn get_headers_with_priority( - &self, - request: HeadersRequest, - _priority: Priority, - ) -> Self::Output { - let headers = self.headers.lock(); - - // Initializes the block hash or number. - let mut block: BlockHashOrNumber = match request.start { - BlockHashOrNumber::Hash(hash) => headers.get(&hash).cloned(), - BlockHashOrNumber::Number(num) => { - headers.values().find(|h| h.number == num).cloned() - } - } - .map(|h| h.number.into()) - .unwrap(); - - // Retrieves headers based on the provided limit and request direction. - let resp = (0..request.limit) - .filter_map(|_| { - headers.iter().find_map(|(hash, header)| { - // Checks if the header matches the specified block or number. - BlockNumHash::new(header.number, *hash).matches_block_or_num(&block).then( - || { - match request.direction { - HeadersDirection::Falling => block = header.parent_hash.into(), - HeadersDirection::Rising => block = (header.number + 1).into(), - } - header.clone() - }, - ) - }) + let received_block_number = block_with_peer.block.number; + let received_block_hash = block_with_peer.block.header.hash_slow(); + let current_head_number = self.engine.fcs().head_block_info().number; + let current_head_hash = self.engine.fcs().head_block_info().hash; + + // If the received block number has a block number greater than the current head by more + // than the optimistic sync threshold, we optimistically sync the chain. + if received_block_number > current_head_number + self.config.optimistic_sync_threshold() { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); + self.engine + .optimistic_sync(BlockInfo { + number: received_block_number, + hash: block_with_peer.block.header.hash_slow(), }) - .collect::>(); - - // Returns a future containing the retrieved headers with a random peer ID. - futures::future::ready(Ok(WithPeerId::new(PeerId::random(), resp))) - } - } + .await?; + self.sync_state.l2_mut().set_syncing(); - /// Implements the `BodiesClient` trait for the `TestFullBlockClient` struct. - impl BodiesClient for TestScrollFullBlockClient { - type Body = ScrollBody; - /// Defines the output type of the function. - type Output = futures::future::Ready>>; - - /// Retrieves block bodies corresponding to provided hashes with a given priority. - /// - /// # Arguments - /// - /// * `hashes` - A vector of block hashes to retrieve bodies for. - /// * `_priority` - Priority level for block body retrieval (unused in this implementation). - /// - /// # Returns - /// - /// A future containing the result of the block body retrieval operation. - fn get_block_bodies_with_priority_and_range_hint( - &self, - hashes: Vec, - _priority: Priority, - _range_hint: Option>, - ) -> Self::Output { - // Acquire a lock on the bodies. - let bodies = self.bodies.lock(); - - // Create a future that immediately returns the result of the block body retrieval - // operation. - futures::future::ready(Ok(WithPeerId::new( - PeerId::random(), - hashes - .iter() - .filter_map(|hash| bodies.get(hash).cloned()) - .take(self.soft_limit) - .collect(), - ))) + // Purge all L1 message to L2 block mappings as they may be invalid after an + // optimistic sync. + let tx = self.database.tx_mut().await?; + tx.purge_l1_message_to_l2_block_mappings(None).await?; + tx.commit().await?; } - } - - impl BlockClient for TestScrollFullBlockClient { - type Block = ScrollBlock; - } - - async fn setup_test_chain_orchestrator() -> ( - ChainOrchestrator>, - Arc, - ) { - // Get a provider to the node. - // TODO: update to use a real node URL. - let assertor = Asserter::new(); - let mainnet_genesis: ::BlockResponse = - serde_json::from_str(include_str!("../testdata/genesis_block_rpc.json")) - .expect("Failed to parse mainnet genesis block"); - assertor.push_success(&mainnet_genesis); - let provider = ProviderBuilder::<_, _, Scroll>::default().connect_mocked_client(assertor); - - let db = Arc::new(setup_test_db().await); - ( - ChainOrchestrator::new( - db.clone(), - SCROLL_MAINNET.clone(), - TestScrollFullBlockClient::default(), - provider, - TEST_OPTIMISTIC_SYNC_THRESHOLD, - TEST_CHAIN_BUFFER_SIZE, - TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY, - ) - .await - .unwrap(), - db, - ) - } - #[tokio::test] - async fn test_handle_commit_batch() { - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; - - // Generate unstructured bytes. - let mut bytes = [0u8; 1024]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); - - // Insert a batch commit in the database to satisfy the chain orchestrator consistency - // checks - let batch_0 = BatchCommitData { index: 0, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_batch(batch_0).await.unwrap(); - tx.commit().await.unwrap(); - - let batch_1 = BatchCommitData { index: 1, ..Arbitrary::arbitrary(&mut u).unwrap() }; - chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); - - let event = chain_orchestrator.next().await.unwrap().unwrap(); + // If the block number is greater than the current head we attempt to extend the chain. + let mut new_headers = if received_block_number > self.engine.fcs().head_block_info().number + { + // Fetch the headers for the received block until we can reconcile it with the current + // chain head. + let block_number_diff = received_block_number - current_head_number; + let new_headers = if received_block_number > current_head_number + 1 { + self.network_client + .get_headers(HeadersRequest::falling( + received_block_hash.into(), + block_number_diff, + )) + .await? + .into_data() + } else { + vec![block_with_peer.block.header.clone()] + }; - // Verify the event structure - match event { - ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, batch_1.index); - assert_eq!(batch_info.hash, batch_1.hash); - assert_eq!(safe_head, None); // No safe head since no batch revert + // If the first header in the new headers has a parent hash that matches the current + // head hash, we can import the chain. + if new_headers.first().expect("at least one header exists").parent_hash == + current_head_hash + { + self.import_chain(new_headers, block_with_peer).await?; + return Ok(()); } - _ => panic!("Expected BatchCommitIndexed event"), - } - - let tx = db.tx().await.unwrap(); - let batch_commit_result = tx.get_batch_by_index(batch_1.index).await.unwrap().unwrap(); - assert_eq!(batch_1, batch_commit_result); - } - #[tokio::test] - async fn test_handle_batch_commit_with_revert() { - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; - - // Generate unstructured bytes. - let mut bytes = [0u8; 1024]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); - - // Insert batch 0 into the database to satisfy the consistency conditions in the chain - // orchestrator - let batch_0 = BatchCommitData { - index: 99, - calldata: Arc::new(vec![].into()), - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_batch(batch_0).await.unwrap(); - tx.commit().await.unwrap(); - - // Create sequential batches - let batch_1 = BatchCommitData { - index: 100, - calldata: Arc::new(vec![].into()), - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - let batch_2 = BatchCommitData { - index: 101, - calldata: Arc::new(vec![].into()), - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - let batch_3 = BatchCommitData { - index: 102, - calldata: Arc::new(vec![].into()), - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - - // Index first batch - chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - match event { - ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 100); - assert_eq!(safe_head, None); + VecDeque::from(new_headers) + } else { + // If the block is less than or equal to the current head check if we already have it in + // the chain. + let current_chain_block = + self.l2_client.get_block_by_number(received_block_number.into()).await?.ok_or( + ChainOrchestratorError::L2BlockNotFoundInL2Client(received_block_number), + )?; + if current_chain_block.header.hash_slow() == received_block_hash { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that is already in the chain"); + return Ok(()) } - _ => panic!("Expected BatchCommitIndexed event"), - } - // Index second batch - chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_2.clone())); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - match event { - ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 101); - assert_eq!(safe_head, None); + // Assert that we are not reorging below the safe head. + let current_safe_info = self.engine.fcs().safe_block_info(); + if received_block_number <= current_safe_info.number { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); + return Ok(()) } - _ => panic!("Expected BatchCommitIndexed event"), - } - // Index third batch - chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_3.clone())); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - match event { - ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 102); - assert_eq!(safe_head, None); + // Check if the parent hash of the received block is in the chain. + let parent_block = + self.l2_client.get_block_by_hash(block_with_peer.block.header.parent_hash).await?; + if parent_block.is_some() { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends an earlier part of the chain"); + return self + .import_chain(vec![block_with_peer.block.header.clone()], block_with_peer) + .await } - _ => panic!("Expected BatchCommitIndexed event"), - } - - // Add some L2 blocks for the batches - let batch_1_info = BatchInfo::new(batch_1.index, batch_1.hash); - let batch_2_info = BatchInfo::new(batch_2.index, batch_2.hash); - let block_1 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 500, hash: Arbitrary::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; - let block_2 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 501, hash: Arbitrary::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], + VecDeque::from([block_with_peer.block.header.clone()]) }; - let block_3 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 502, hash: Arbitrary::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; - - chain_orchestrator.persist_l1_consolidated_blocks(vec![block_1.clone()], batch_1_info); - chain_orchestrator.next().await.unwrap().unwrap(); - - chain_orchestrator.persist_l1_consolidated_blocks(vec![block_2.clone()], batch_2_info); - chain_orchestrator.next().await.unwrap().unwrap(); - - chain_orchestrator.persist_l1_consolidated_blocks(vec![block_3.clone()], batch_2_info); - chain_orchestrator.next().await.unwrap().unwrap(); - // Now simulate a batch revert by submitting a new batch with index 101 - // This should delete batch 102 and any blocks associated with it - let new_batch_2 = BatchCommitData { - index: 101, - calldata: Arc::new(vec![1, 2, 3].into()), // Different data - ..Arbitrary::arbitrary(&mut u).unwrap() - }; + // Check to assert that we have received a newer chain. + let current_head = self + .l2_client + .get_block_by_number(current_head_number.into()) + .await? + .expect("current head block must exist"); + if new_headers.back().expect("at least one header exists").timestamp < + current_head.header.timestamp + { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_head_hash = ?current_head.header.hash_slow(), current_head_number = current_head_number, "Received block from peer that is older than the current head - ignoring"); + return Ok(()) + } - chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(new_batch_2.clone())); - let event = chain_orchestrator.next().await.unwrap().unwrap(); + // If we reach this point, we have a block that is not in the current chain and does not + // extend the current head. This implies a reorg. We attempt to reconcile the fork. + let current_safe_head = self.engine.fcs().safe_block_info(); + while current_safe_head.number + 1 < + new_headers.front().expect("at least one header exists").number + { + let parent_hash = new_headers.front().expect("at least one header exists").parent_hash; + let parent_number = new_headers.front().expect("at least one header exists").number - 1; + let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_head.number); + let headers = self + .network_client + .get_headers(HeadersRequest::falling(parent_hash.into(), fetch_count)) + .await? + .into_data(); + + let mut index = None; + for (i, header) in headers.iter().enumerate() { + let current_block = self + .l2_client + .get_block_by_number(header.number.into()) + .await? + .expect("block must exist"); + if header.hash_slow() == current_block.header.hash_slow() { + index = Some(i); + break; + } + } - // Verify the event indicates a batch revert - match event { - ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 101); - assert_eq!(batch_info.hash, new_batch_2.hash); - // Safe head should be the highest block from batch index <= 100 - assert_eq!(safe_head, Some(block_1.block_info)); + if let Some(index) = index { + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, common_ancestor = ?headers[index].hash_slow(), common_ancestor_number = headers[index].number, "Found common ancestor for fork - reorging to new chain"); + for header in headers.into_iter().skip(index).rev() { + new_headers.push_front(header); + } + self.import_chain(new_headers.into(), block_with_peer).await?; + return Ok(()); + } else { + for header in headers.into_iter().rev() { + new_headers.push_front(header); + } } - _ => panic!("Expected BatchCommitIndexed event"), } - // Verify batch 102 was deleted - let tx = db.tx().await.unwrap(); - let batch_102 = tx.get_batch_by_index(102).await.unwrap(); - assert!(batch_102.is_none()); - - // Verify batch 101 was replaced with new data - let updated_batch_101 = tx.get_batch_by_index(101).await.unwrap().unwrap(); - assert_eq!(updated_batch_101, new_batch_2); - - // Verify batch 100 still exists - let batch_100 = tx.get_batch_by_index(100).await.unwrap(); - assert!(batch_100.is_some()); + Ok(()) } - #[tokio::test] - async fn test_handle_l1_message() { - reth_tracing::init_test_tracing(); - - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + /// Imports a chain of headers into the L2 chain. + async fn import_chain( + &mut self, + chain: Vec
, + block_with_peer: NewBlockWithPeer, + ) -> Result<(), ChainOrchestratorError> { + let chain_head_hash = chain.last().expect("at least one header exists").hash_slow(); + let chain_head_number = chain.last().expect("at least one header exists").number; + tracing::info!(target: "scroll::chain_orchestrator", num_blocks = chain.len(), ?chain_head_hash, ?chain_head_number, "Received new block from peer that extends the current head"); + + // Fetch the block bodies for the new headers and construct the full blocks. + let new_bodies = self + .network_client + .get_block_bodies(chain.iter().map(|h| h.hash_slow()).collect()) + .await? + .into_data(); + let new_blocks: Vec = chain + .into_iter() + .zip(new_bodies.into_iter()) + .map(|(header, body)| ScrollBlock { header, body }) + .collect(); + + // If we are in consolidated mode, validate the L1 messages in the new blocks. + if self.sync_state.is_synced() { + self.validate_l1_messages(&new_blocks).await?; + } - // Generate unstructured bytes. - let mut bytes = [0u8; 1024]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); + // Validate the new blocks by sending them to the engine. + for block in new_blocks.iter() { + let status = self.engine.new_payload(&block).await?; - // Insert an initial message in the database to satisfy the consistency checks in the chain - // orchestrator. - let message_0 = L1MessageEnvelope { - transaction: TxL1Message { - queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 2, - ..Arbitrary::arbitrary(&mut u).unwrap() - }, - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_l1_message(message_0).await.unwrap(); - tx.commit().await.unwrap(); + if status.is_invalid() { + tracing::warn!(target: "scroll::chain_orchestrator", block_number = block.number, block_hash = ?block.hash_slow(), ?status, "Received invalid block from peer"); + self.network.handle().block_import_outcome(BlockImportOutcome::invalid_block( + block_with_peer.peer_id, + )); + return Err(ChainOrchestratorError::InvalidBlock); + } + } - let message_1 = TxL1Message { - queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 1, - ..Arbitrary::arbitrary(&mut u).unwrap() - }; - let block_number = u64::arbitrary(&mut u).unwrap(); - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: message_1.clone(), - block_number, - block_timestamp: 0, - }); + // Update the FCS to the new head. + let result = self + .engine + .update_fcs( + Some(BlockInfo { number: chain_head_number, hash: chain_head_hash }), + None, + None, + ) + .await?; - let _ = chain_orchestrator.next().await; + // If the FCS update resulted in an invalid state, we return an error. + if result.is_invalid() { + tracing::warn!(target: "scroll::chain_orchestrator", ?chain_head_hash, ?chain_head_number, ?result, "Failed to update FCS after importing new chain from peer"); + return Err(ChainOrchestratorError::InvalidBlock) + } - let tx = db.tx().await.unwrap(); - let l1_message_result = - tx.get_l1_message_by_index(message_1.queue_index).await.unwrap().unwrap(); - let l1_message = L1MessageEnvelope::new(message_1, block_number, None, None); + // Persist the mapping of L1 messages to L2 blocks such that we can react to L1 reorgs. + let blocks = new_blocks.iter().map(|block| block.into()).collect::>(); + let tx = self.database.tx_mut().await?; + tx.update_l1_messages_from_l2_blocks(blocks).await?; + tx.commit().await?; + + // If we were previously in L2 syncing mode and the FCS update resulted in a valid state, we + // transition the L2 sync state to synced and consolidate the chain. + if result.is_valid() && self.sync_state.l2().is_syncing() { + tracing::info!(target: "scroll::chain_orchestrator", "L2 is now synced"); + self.sync_state.l2_mut().set_synced(); + + // If both L1 and L2 are now synced, we transition to consolidated mode by consolidating + // the chain. + if self.sync_state.is_synced() { + self.consolidate_chain().await?; + } + } - assert_eq!(l1_message, l1_message_result); + // Persist the signature for the block and notify the network manager of a successful + // import. + let tx = self.database.tx_mut().await?; + tx.insert_signature(chain_head_hash, block_with_peer.signature).await?; + tx.commit().await?; + self.network.handle().block_import_outcome(BlockImportOutcome::valid_block( + block_with_peer.peer_id, + block_with_peer.block, + Bytes::copy_from_slice(&block_with_peer.signature.sig_as_bytes()), + )); + + return Ok(()) } - #[tokio::test] - async fn test_l1_message_hash_queue() { - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; - - // Insert the previous l1 message in the database to satisfy the chain orchestrator - // consistency checks. - let message = L1MessageEnvelope { - transaction: TxL1Message { - queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 1, - ..Default::default() - }, - l1_block_number: 1475587, - l2_block_number: None, - queue_hash: None, - }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_l1_message(message).await.unwrap(); - tx.commit().await.unwrap(); - - // insert the previous L1 message in database. - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: TxL1Message { - queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY, - ..Default::default() - }, - block_number: 1475588, - block_timestamp: 1745305199, - }); - let _ = chain_orchestrator.next().await.unwrap().unwrap(); - - // - let message = TxL1Message { - queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY + 1, - gas_limit: 168000, - to: address!("Ba50f5340FB9F3Bd074bD638c9BE13eCB36E603d"), - value: U256::ZERO, - sender: address!("61d8d3E7F7c656493d1d76aAA1a836CEdfCBc27b"), - input: bytes!("8ef1332e000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d6996000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d699600000000000000000000000000000000000000000000000000038d7ea4c6800000000000000000000000000000000000000000000000000000000000001034de00000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000"), - }; - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: message.clone(), - block_number: 14755883, - block_timestamp: 1745305200, - }); - - let _ = chain_orchestrator.next().await.unwrap().unwrap(); - - let tx = db.tx().await.unwrap(); - let l1_message_result = - tx.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); - - assert_eq!( - b256!("b2331b9010aac89f012d648fccc1f0a9aa5ef7b7b2afe21be297dd1a00000000"), - l1_message_result.queue_hash.unwrap() - ); - } + /// Consolidates the chain by validating all unsafe blocks from the current safe head to the + /// current head. + /// + /// This involves validating the L1 messages in the blocks against the expected L1 messages + /// synced from L1. + async fn consolidate_chain(&mut self) -> Result<(), ChainOrchestratorError> { + tracing::trace!(target: "scroll::chain_orchestrator", fcs = ?self.engine.fcs(), "Consolidating chain from safe to head"); + + let mut start_block_number = self.engine.fcs().safe_block_info().number + 1; + let end_block_number = self.engine.fcs().head_block_info().number; + + while start_block_number <= end_block_number { + let batch_end = + (start_block_number + CONSOLIDATION_BATCH_SIZE - 1).min(end_block_number); + tracing::trace!(target: "scroll::chain_orchestrator", start_block = ?start_block_number, end_block = ?batch_end, "Consolidating batch of blocks"); + + // TODO: Make fetching parallel but ensure concurrency limits are respected. + let mut blocks_to_validate = vec![]; + for block_number in start_block_number..=batch_end { + let block = self + .l2_client + .get_block_by_number(block_number.into()) + .await? + .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_number))? + .into_consensus() + .map_transactions(|tx| tx.inner.into_inner()); + blocks_to_validate.push(block); + } - #[tokio::test] - async fn test_handle_reorg() { - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; - - // Generate unstructured bytes. - let mut bytes = [0u8; 1024]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); - - // Insert batch 0 into the database to satisfy the consistency checks in the chain - // orchestrator - let batch_0 = - BatchCommitData { index: 0, block_number: 0, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_batch(batch_0).await.unwrap(); - - // Insert l1 message into the database to satisfy the consistency checks in the chain - // orchestrator - let l1_message = L1MessageEnvelope { - queue_hash: None, - l1_block_number: 0, - l2_block_number: None, - transaction: TxL1Message { queue_index: 0, ..Arbitrary::arbitrary(&mut u).unwrap() }, - }; - tx.insert_l1_message(l1_message).await.unwrap(); - tx.commit().await.unwrap(); - - // Generate a 3 random batch inputs and set their block numbers - let mut batch_commit_block_1 = BatchCommitData::arbitrary(&mut u).unwrap(); - batch_commit_block_1.block_number = 1; - batch_commit_block_1.index = 1; - let batch_commit_block_1 = batch_commit_block_1; - - let mut batch_commit_block_2 = BatchCommitData::arbitrary(&mut u).unwrap(); - batch_commit_block_2.block_number = 2; - batch_commit_block_2.index = 2; - let batch_commit_block_2 = batch_commit_block_2; - - let mut batch_commit_block_3 = BatchCommitData::arbitrary(&mut u).unwrap(); - batch_commit_block_3.block_number = 3; - batch_commit_block_3.index = 3; - let batch_commit_block_3 = batch_commit_block_3; - - // Index batch inputs - chain_orchestrator - .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); - chain_orchestrator - .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_2.clone())); - chain_orchestrator - .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_3.clone())); - - // Generate 3 random L1 messages and set their block numbers - let l1_message_block_1 = L1MessageEnvelope { - queue_hash: None, - l1_block_number: 1, - l2_block_number: None, - transaction: TxL1Message { queue_index: 1, ..Arbitrary::arbitrary(&mut u).unwrap() }, - }; - let l1_message_block_2 = L1MessageEnvelope { - queue_hash: None, - l1_block_number: 2, - l2_block_number: None, - transaction: TxL1Message { queue_index: 2, ..Arbitrary::arbitrary(&mut u).unwrap() }, - }; - let l1_message_block_3 = L1MessageEnvelope { - queue_hash: None, - l1_block_number: 3, - l2_block_number: None, - transaction: TxL1Message { queue_index: 3, ..Arbitrary::arbitrary(&mut u).unwrap() }, - }; + self.validate_l1_messages(&blocks_to_validate).await?; - // Index L1 messages - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: l1_message_block_1.clone().transaction, - block_number: l1_message_block_1.clone().l1_block_number, - block_timestamp: 0, - }); - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: l1_message_block_2.clone().transaction, - block_number: l1_message_block_2.clone().l1_block_number, - block_timestamp: 0, - }); - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: l1_message_block_3.clone().transaction, - block_number: l1_message_block_3.clone().l1_block_number, - block_timestamp: 0, - }); - - // Reorg at block 2 - chain_orchestrator.handle_l1_notification(L1Notification::Reorg(2)); - - for _ in 0..7 { - chain_orchestrator.next().await.unwrap().unwrap(); + start_block_number = batch_end + 1; } - let tx = db.tx().await.unwrap(); - - // Check that the batch input at block 30 is deleted - let batch_commits = - tx.get_batches().await.unwrap().map(|res| res.unwrap()).collect::>().await; - - assert_eq!(3, batch_commits.len()); - assert!(batch_commits.contains(&batch_commit_block_1)); - assert!(batch_commits.contains(&batch_commit_block_2)); - - // check that the L1 message at block 30 is deleted - let l1_messages = tx - .get_l1_messages(None) - .await - .unwrap() - .map(|res| res.unwrap()) - .collect::>() - .await; - assert_eq!(3, l1_messages.len()); - assert!(l1_messages.contains(&l1_message_block_1)); - assert!(l1_messages.contains(&l1_message_block_2)); + Ok(()) } - // We ignore this test for now as it requires a more complex setup which leverages an L2 node - // and is already covered in the integration test `can_handle_reorgs_while_sequencing` - #[ignore] - #[tokio::test] - async fn test_handle_reorg_executed_l1_messages() { - // Instantiate chain orchestrator and db - let (mut chain_orchestrator, _database) = setup_test_chain_orchestrator().await; - - // Generate unstructured bytes. - let mut bytes = [0u8; 8192]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); - - // Generate a 3 random batch inputs and set their block numbers - let batch_commit_block_1 = - BatchCommitData { block_number: 5, index: 5, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let batch_commit_block_10 = BatchCommitData { - block_number: 10, - index: 10, - ..Arbitrary::arbitrary(&mut u).unwrap() - }; + /// Validates the L1 messages in the provided blocks against the expected L1 messages synced + /// from L1. + async fn validate_l1_messages( + &self, + blocks: &[ScrollBlock], + ) -> Result<(), ChainOrchestratorError> { + for block in blocks { + let tx = self.database.tx().await?; + let database_txs = + tx.get_l1_messages(Some(L1MessageStart::block_number(block.number))).await?; + let mut chain_txs = + block.body.transactions().filter(|tx| tx.is_l1_message()).map(|tx| tx.trie_hash()); + + if let Some(mut txs) = database_txs { + for tx in chain_txs { + let expected_message = txs.next().await.ok_or( + ChainOrchestratorError::L1MessageNotFound(L1MessageStart::Hash(tx)), + )??; + if expected_message.transaction.tx_hash() != tx { + return Err(ChainOrchestratorError::L1MessageMismatch { + expected: expected_message.transaction.tx_hash(), + actual: tx, + }) + } + } + continue; + } - // Index batch inputs - chain_orchestrator - .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); - chain_orchestrator - .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_10.clone())); - for _ in 0..2 { - let _event = chain_orchestrator.next().await.unwrap().unwrap(); + if chain_txs.next().is_some() { + return Err(ChainOrchestratorError::L1MessageNotFound(L1MessageStart::BlockNumber( + block.number, + ))) + } } - let batch_1 = BatchInfo::new(batch_commit_block_1.index, batch_commit_block_1.hash); - let batch_10 = BatchInfo::new(batch_commit_block_10.index, batch_commit_block_10.hash); - - const UNITS_FOR_TESTING: u64 = 20; - const L1_MESSAGES_NOT_EXECUTED_COUNT: u64 = 7; - let mut l1_messages = Vec::with_capacity(UNITS_FOR_TESTING as usize); - for l1_message_queue_index in 0..UNITS_FOR_TESTING { - let l1_message = L1MessageEnvelope { - queue_hash: None, - l1_block_number: l1_message_queue_index, - l2_block_number: (UNITS_FOR_TESTING - l1_message_queue_index > - L1_MESSAGES_NOT_EXECUTED_COUNT) - .then_some(l1_message_queue_index), - transaction: TxL1Message { - queue_index: l1_message_queue_index, - ..Arbitrary::arbitrary(&mut u).unwrap() - }, - }; - chain_orchestrator.handle_l1_notification(L1Notification::L1Message { - message: l1_message.transaction.clone(), - block_number: l1_message.l1_block_number, - block_timestamp: 0, - }); - chain_orchestrator.next().await.unwrap().unwrap(); - l1_messages.push(l1_message); - } + Ok(()) + } - let mut blocks = Vec::with_capacity(UNITS_FOR_TESTING as usize); - for block_number in 0..UNITS_FOR_TESTING { - let l2_block = L2BlockInfoWithL1Messages { - block_info: BlockInfo { - number: block_number, - hash: Arbitrary::arbitrary(&mut u).unwrap(), - }, - l1_messages: (UNITS_FOR_TESTING - block_number > L1_MESSAGES_NOT_EXECUTED_COUNT) - .then_some(vec![l1_messages[block_number as usize].transaction.tx_hash()]) - .unwrap_or_default(), - }; - let batch_info = if block_number < 5 { - Some(batch_1) - } else if block_number < 10 { - Some(batch_10) - } else { - None - }; - if let Some(batch_info) = batch_info { - chain_orchestrator - .persist_l1_consolidated_blocks(vec![l2_block.clone()], batch_info); - } else { - chain_orchestrator.consolidate_validated_l2_blocks(vec![l2_block.clone()]); + /// Drives the [`RollupNodeManager`] future until a [`GracefulShutdown`] signal is received. + pub async fn run_until_graceful_shutdown(mut self, mut shutdown: GracefulShutdown) { + loop { + tokio::select! { + Some(command) = self.handle_rx.recv() => { + self.handle_command(command).await; + } + Some(batch) = self.derivation_pipeline.next() => { + if let Err(err) = self.handle_derived_batch(batch).await { + tracing::error!(?err, "Error handling derived batch"); + } + } + Some(event) = self.network.events().next() => { + if let Err(err) = self.handle_network_event(event).await { + tracing::error!(?err, "Error handling network event"); + } + } + Some(notification) = self.l1_notification_rx.recv(), + if self.sync_state.l2().is_synced() && self.derivation_pipeline.len().await == 0 => + { + if let Err(err) = self.handle_l1_notification(notification).await { + tracing::error!(?err, "Error handling L1 notification"); + } + } + _guard = &mut shutdown => { + break; + } } - - chain_orchestrator.next().await.unwrap().unwrap(); - blocks.push(l2_block); } - - // First we assert that we dont reorg the L2 or message queue hash for a higher block - // than any of the L1 messages. - chain_orchestrator.handle_l1_notification(L1Notification::Reorg(17)); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - assert_eq!( - event, - ChainOrchestratorEvent::L1Reorg { - l1_block_number: 17, - queue_index: None, - l2_head_block_info: None, - l2_safe_block_info: None - } - ); - - // Reorg at block 7 which is one of the messages that has not been executed yet. No reorg - // but we should ensure the L1 messages have been deleted. - chain_orchestrator.handle_l1_notification(L1Notification::Reorg(7)); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - - assert_eq!( - event, - ChainOrchestratorEvent::L1Reorg { - l1_block_number: 7, - queue_index: Some(8), - l2_head_block_info: Some(blocks[7].block_info), - l2_safe_block_info: Some(blocks[4].block_info) - } - ); - - // Now reorg at block 5 which contains L1 messages that have been executed . - chain_orchestrator.handle_l1_notification(L1Notification::Reorg(3)); - let event = chain_orchestrator.next().await.unwrap().unwrap(); - - assert_eq!( - event, - ChainOrchestratorEvent::L1Reorg { - l1_block_number: 3, - queue_index: Some(4), - l2_head_block_info: Some(blocks[3].block_info), - l2_safe_block_info: Some(BlockInfo::new( - 0, - chain_orchestrator.chain_spec.genesis_hash() - )), - } - ); } } + +/// Computes the queue hash by taking the previous queue hash and performing a 2-to-1 hash with the +/// current transaction hash using keccak. It then applies a mask to the last 32 bits as these bits +/// are used to store the timestamp at which the message was enqueued in the contract. For the first +/// message in the queue, the previous queue hash is zero. If the L1 message queue index is before +/// migration to `L1MessageQueueV2`, the queue hash will be None. +/// +/// The solidity contract (`L1MessageQueueV2.sol`) implementation is defined here: +async fn compute_l1_message_queue_hash( + database: &Arc, + l1_message: &TxL1Message, + l1_v2_message_queue_start_index: u64, +) -> Result>, ChainOrchestratorError> { + let queue_hash = if l1_message.queue_index == l1_v2_message_queue_start_index { + let mut input = B256::default().to_vec(); + input.append(&mut l1_message.tx_hash().to_vec()); + Some(keccak256(input) & L1_MESSAGE_QUEUE_HASH_MASK) + } else if l1_message.queue_index > l1_v2_message_queue_start_index { + let index = l1_message.queue_index - 1; + let mut input = Retry::default() + .retry("get_l1_message_by_index", || async { + let tx = database.tx().await?; + let input = tx.get_l1_message_by_index(index).await?; + Ok::<_, ChainOrchestratorError>(input) + }) + .await? + .map(|m| m.queue_hash) + .ok_or(DatabaseError::L1MessageNotFound(L1MessageStart::Index(index)))? + .unwrap_or_default() + .to_vec(); + input.append(&mut l1_message.tx_hash().to_vec()); + Some(keccak256(input) & L1_MESSAGE_QUEUE_HASH_MASK) + } else { + None + }; + Ok(queue_hash) +} + +// #[cfg(test)] +// mod test { +// use std::vec; + +// use super::*; +// use alloy_consensus::Header; +// use alloy_eips::{BlockHashOrNumber, BlockNumHash}; +// use alloy_primitives::{address, bytes, B256, U256}; +// use alloy_provider::{ProviderBuilder, RootProvider}; +// use alloy_transport::mock::Asserter; +// use arbitrary::{Arbitrary, Unstructured}; +// use futures::StreamExt; +// use parking_lot::Mutex; +// use rand::Rng; +// use reth_eth_wire_types::HeadersDirection; +// use reth_network_api::BlockClient; +// use reth_network_p2p::{ +// download::DownloadClient, +// error::PeerRequestResult, +// headers::client::{HeadersClient, HeadersRequest}, +// priority::Priority, +// BodiesClient, +// }; +// use reth_network_peers::{PeerId, WithPeerId}; +// use reth_primitives_traits::Block; +// use reth_scroll_chainspec::{ScrollChainSpec, SCROLL_MAINNET}; +// use rollup_node_primitives::BatchCommitData; +// use scroll_alloy_network::Scroll; +// use scroll_db::test_utils::setup_test_db; +// use std::{collections::HashMap, ops::RangeInclusive, sync::Arc}; + +// type ScrollBody = ::Body; + +// const TEST_OPTIMISTIC_SYNC_THRESHOLD: u64 = 100; +// const TEST_CHAIN_BUFFER_SIZE: usize = 2000; +// const TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY: u64 = 953885; + +// /// A headers+bodies client that stores the headers and bodies in memory, with an artificial +// /// soft bodies response limit that is set to 20 by default. +// /// +// /// This full block client can be [Clone]d and shared between multiple tasks. +// #[derive(Clone, Debug)] +// struct TestScrollFullBlockClient { +// headers: Arc>>, +// bodies: Arc::Body>>>, +// // soft response limit, max number of bodies to respond with +// soft_limit: usize, +// } + +// impl Default for TestScrollFullBlockClient { +// fn default() -> Self { +// let mainnet_genesis: reth_scroll_primitives::ScrollBlock = +// serde_json::from_str(include_str!("../testdata/genesis_block.json")).unwrap(); +// let (header, body) = mainnet_genesis.split(); +// let hash = header.hash_slow(); +// let headers = HashMap::from([(hash, header)]); +// let bodies = HashMap::from([(hash, body)]); +// Self { +// headers: Arc::new(Mutex::new(headers)), +// bodies: Arc::new(Mutex::new(bodies)), +// soft_limit: 20, +// } +// } +// } + +// impl DownloadClient for TestScrollFullBlockClient { +// /// Reports a bad message from a specific peer. +// fn report_bad_message(&self, _peer_id: PeerId) {} + +// /// Retrieves the number of connected peers. +// /// +// /// Returns the number of connected peers in the test scenario (1). +// fn num_connected_peers(&self) -> usize { +// 1 +// } +// } + +// /// Implements the `HeadersClient` trait for the `TestFullBlockClient` struct. +// impl HeadersClient for TestScrollFullBlockClient { +// type Header = Header; +// /// Specifies the associated output type. +// type Output = futures::future::Ready>>; + +// /// Retrieves headers with a given priority level. +// /// +// /// # Arguments +// /// +// /// * `request` - A `HeadersRequest` indicating the headers to retrieve. +// /// * `_priority` - A `Priority` level for the request. +// /// +// /// # Returns +// /// +// /// A `Ready` future containing a `PeerRequestResult` with a vector of retrieved headers. +// fn get_headers_with_priority( +// &self, +// request: HeadersRequest, +// _priority: Priority, +// ) -> Self::Output { +// let headers = self.headers.lock(); + +// // Initializes the block hash or number. +// let mut block: BlockHashOrNumber = match request.start { +// BlockHashOrNumber::Hash(hash) => headers.get(&hash).cloned(), +// BlockHashOrNumber::Number(num) => { +// headers.values().find(|h| h.number == num).cloned() +// } +// } +// .map(|h| h.number.into()) +// .unwrap(); + +// // Retrieves headers based on the provided limit and request direction. +// let resp = (0..request.limit) +// .filter_map(|_| { +// headers.iter().find_map(|(hash, header)| { +// // Checks if the header matches the specified block or number. +// BlockNumHash::new(header.number, +// *hash).matches_block_or_num(&block).then( || { +// match request.direction { +// HeadersDirection::Falling => block = +// header.parent_hash.into(), HeadersDirection::Rising => block +// = (header.number + 1).into(), } +// header.clone() +// }, +// ) +// }) +// }) +// .collect::>(); + +// // Returns a future containing the retrieved headers with a random peer ID. +// futures::future::ready(Ok(WithPeerId::new(PeerId::random(), resp))) +// } +// } + +// /// Implements the `BodiesClient` trait for the `TestFullBlockClient` struct. +// impl BodiesClient for TestScrollFullBlockClient { +// type Body = ScrollBody; +// /// Defines the output type of the function. +// type Output = futures::future::Ready>>; + +// /// Retrieves block bodies corresponding to provided hashes with a given priority. +// /// +// /// # Arguments +// /// +// /// * `hashes` - A vector of block hashes to retrieve bodies for. +// /// * `_priority` - Priority level for block body retrieval (unused in this +// implementation). /// +// /// # Returns +// /// +// /// A future containing the result of the block body retrieval operation. +// fn get_block_bodies_with_priority_and_range_hint( +// &self, +// hashes: Vec, +// _priority: Priority, +// _range_hint: Option>, +// ) -> Self::Output { +// // Acquire a lock on the bodies. +// let bodies = self.bodies.lock(); + +// // Create a future that immediately returns the result of the block body retrieval +// // operation. +// futures::future::ready(Ok(WithPeerId::new( +// PeerId::random(), +// hashes +// .iter() +// .filter_map(|hash| bodies.get(hash).cloned()) +// .take(self.soft_limit) +// .collect(), +// ))) +// } +// } + +// impl BlockClient for TestScrollFullBlockClient { +// type Block = ScrollBlock; +// } + +// async fn setup_test_chain_orchestrator() -> ( +// ChainOrchestrator< +// ScrollChainSpec, +// TestScrollFullBlockClient, +// RootProvider, +// RootProvider, +// >, +// Arc, +// ) { +// // Get a provider to the node. +// // TODO: update to use a real node URL. +// let assertor = Asserter::new(); +// let mainnet_genesis: ::BlockResponse = +// serde_json::from_str(include_str!("../testdata/genesis_block_rpc.json")) +// .expect("Failed to parse mainnet genesis block"); +// assertor.push_success(&mainnet_genesis); +// let provider = ProviderBuilder::<_, _, +// Scroll>::default().connect_mocked_client(assertor); + +// let db = Arc::new(setup_test_db().await); +// ( +// ChainOrchestrator::new( +// db.clone(), +// SCROLL_MAINNET.clone(), +// TestScrollFullBlockClient::default(), +// provider, +// TEST_OPTIMISTIC_SYNC_THRESHOLD, +// TEST_CHAIN_BUFFER_SIZE, +// TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY, +// ) +// .await +// .unwrap(), +// db, +// ) +// } + +// #[tokio::test] +// async fn test_handle_commit_batch() { +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + +// // Generate unstructured bytes. +// let mut bytes = [0u8; 1024]; +// rand::rng().fill(bytes.as_mut_slice()); +// let mut u = Unstructured::new(&bytes); + +// // Insert a batch commit in the database to satisfy the chain orchestrator consistency +// // checks +// let batch_0 = BatchCommitData { index: 0, ..Arbitrary::arbitrary(&mut u).unwrap() }; +// let tx = db.tx_mut().await.unwrap(); +// tx.insert_batch(batch_0).await.unwrap(); +// tx.commit().await.unwrap(); + +// let batch_1 = BatchCommitData { index: 1, ..Arbitrary::arbitrary(&mut u).unwrap() }; +// chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); + +// let event = chain_orchestrator.next().await.unwrap().unwrap(); + +// // Verify the event structure +// match event { +// ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { +// assert_eq!(batch_info.index, batch_1.index); +// assert_eq!(batch_info.hash, batch_1.hash); +// assert_eq!(safe_head, None); // No safe head since no batch revert +// } +// _ => panic!("Expected BatchCommitIndexed event"), +// } + +// let tx = db.tx().await.unwrap(); +// let batch_commit_result = tx.get_batch_by_index(batch_1.index).await.unwrap().unwrap(); +// assert_eq!(batch_1, batch_commit_result); +// } + +// #[tokio::test] +// async fn test_handle_batch_commit_with_revert() { +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + +// // Generate unstructured bytes. +// let mut bytes = [0u8; 1024]; +// rand::rng().fill(bytes.as_mut_slice()); +// let mut u = Unstructured::new(&bytes); + +// // Insert batch 0 into the database to satisfy the consistency conditions in the chain +// // orchestrator +// let batch_0 = BatchCommitData { +// index: 99, +// calldata: Arc::new(vec![].into()), +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; +// let tx = db.tx_mut().await.unwrap(); +// tx.insert_batch(batch_0).await.unwrap(); +// tx.commit().await.unwrap(); + +// // Create sequential batches +// let batch_1 = BatchCommitData { +// index: 100, +// calldata: Arc::new(vec![].into()), +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; +// let batch_2 = BatchCommitData { +// index: 101, +// calldata: Arc::new(vec![].into()), +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; +// let batch_3 = BatchCommitData { +// index: 102, +// calldata: Arc::new(vec![].into()), +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; + +// // Index first batch +// chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); +// match event { +// ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { +// assert_eq!(batch_info.index, 100); +// assert_eq!(safe_head, None); +// } +// _ => panic!("Expected BatchCommitIndexed event"), +// } + +// // Index second batch +// chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_2.clone())); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); +// match event { +// ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { +// assert_eq!(batch_info.index, 101); +// assert_eq!(safe_head, None); +// } +// _ => panic!("Expected BatchCommitIndexed event"), +// } + +// // Index third batch +// chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(batch_3.clone())); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); +// match event { +// ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { +// assert_eq!(batch_info.index, 102); +// assert_eq!(safe_head, None); +// } +// _ => panic!("Expected BatchCommitIndexed event"), +// } + +// // Add some L2 blocks for the batches +// let batch_1_info = BatchInfo::new(batch_1.index, batch_1.hash); +// let batch_2_info = BatchInfo::new(batch_2.index, batch_2.hash); + +// let block_1 = L2BlockInfoWithL1Messages { +// block_info: BlockInfo { number: 500, hash: Arbitrary::arbitrary(&mut u).unwrap() }, +// l1_messages: vec![], +// }; +// let block_2 = L2BlockInfoWithL1Messages { +// block_info: BlockInfo { number: 501, hash: Arbitrary::arbitrary(&mut u).unwrap() }, +// l1_messages: vec![], +// }; +// let block_3 = L2BlockInfoWithL1Messages { +// block_info: BlockInfo { number: 502, hash: Arbitrary::arbitrary(&mut u).unwrap() }, +// l1_messages: vec![], +// }; + +// chain_orchestrator.persist_l1_consolidated_blocks(vec![block_1.clone()], batch_1_info); +// chain_orchestrator.next().await.unwrap().unwrap(); + +// chain_orchestrator.persist_l1_consolidated_blocks(vec![block_2.clone()], batch_2_info); +// chain_orchestrator.next().await.unwrap().unwrap(); + +// chain_orchestrator.persist_l1_consolidated_blocks(vec![block_3.clone()], batch_2_info); +// chain_orchestrator.next().await.unwrap().unwrap(); + +// // Now simulate a batch revert by submitting a new batch with index 101 +// // This should delete batch 102 and any blocks associated with it +// let new_batch_2 = BatchCommitData { +// index: 101, +// calldata: Arc::new(vec![1, 2, 3].into()), // Different data +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; + +// chain_orchestrator.handle_l1_notification(L1Notification::BatchCommit(new_batch_2. +// clone())); let event = chain_orchestrator.next().await.unwrap().unwrap(); + +// // Verify the event indicates a batch revert +// match event { +// ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { +// assert_eq!(batch_info.index, 101); +// assert_eq!(batch_info.hash, new_batch_2.hash); +// // Safe head should be the highest block from batch index <= 100 +// assert_eq!(safe_head, Some(block_1.block_info)); +// } +// _ => panic!("Expected BatchCommitIndexed event"), +// } + +// // Verify batch 102 was deleted +// let tx = db.tx().await.unwrap(); +// let batch_102 = tx.get_batch_by_index(102).await.unwrap(); +// assert!(batch_102.is_none()); + +// // Verify batch 101 was replaced with new data +// let updated_batch_101 = tx.get_batch_by_index(101).await.unwrap().unwrap(); +// assert_eq!(updated_batch_101, new_batch_2); + +// // Verify batch 100 still exists +// let batch_100 = tx.get_batch_by_index(100).await.unwrap(); +// assert!(batch_100.is_some()); +// } + +// #[tokio::test] +// async fn test_handle_l1_message() { +// reth_tracing::init_test_tracing(); + +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + +// // Generate unstructured bytes. +// let mut bytes = [0u8; 1024]; +// rand::rng().fill(bytes.as_mut_slice()); +// let mut u = Unstructured::new(&bytes); + +// // Insert an initial message in the database to satisfy the consistency checks in the +// chain // orchestrator. +// let message_0 = L1MessageEnvelope { +// transaction: TxL1Message { +// queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 2, +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }, +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; +// let tx = db.tx_mut().await.unwrap(); +// tx.insert_l1_message(message_0).await.unwrap(); +// tx.commit().await.unwrap(); + +// let message_1 = TxL1Message { +// queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 1, +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; +// let block_number = u64::arbitrary(&mut u).unwrap(); +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: message_1.clone(), +// block_number, +// block_timestamp: 0, +// }); + +// let _ = chain_orchestrator.next().await; + +// let tx = db.tx().await.unwrap(); +// let l1_message_result = +// tx.get_l1_message_by_index(message_1.queue_index).await.unwrap().unwrap(); +// let l1_message = L1MessageEnvelope::new(message_1, block_number, None, None); + +// assert_eq!(l1_message, l1_message_result); +// } + +// #[tokio::test] +// async fn test_l1_message_hash_queue() { +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + +// // Insert the previous l1 message in the database to satisfy the chain orchestrator +// // consistency checks. +// let message = L1MessageEnvelope { +// transaction: TxL1Message { +// queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY - 1, +// ..Default::default() +// }, +// l1_block_number: 1475587, +// l2_block_number: None, +// queue_hash: None, +// }; +// let tx = db.tx_mut().await.unwrap(); +// tx.insert_l1_message(message).await.unwrap(); +// tx.commit().await.unwrap(); + +// // insert the previous L1 message in database. +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: TxL1Message { +// queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY, +// ..Default::default() +// }, +// block_number: 1475588, +// block_timestamp: 1745305199, +// }); +// let _ = chain_orchestrator.next().await.unwrap().unwrap(); + +// // +// let message = TxL1Message { +// queue_index: TEST_L1_MESSAGE_QUEUE_INDEX_BOUNDARY + 1, +// gas_limit: 168000, +// to: address!("Ba50f5340FB9F3Bd074bD638c9BE13eCB36E603d"), +// value: U256::ZERO, +// sender: address!("61d8d3E7F7c656493d1d76aAA1a836CEdfCBc27b"), +// input: +// bytes!("8ef1332e000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d6996000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d699600000000000000000000000000000000000000000000000000038d7ea4c6800000000000000000000000000000000000000000000000000000000000001034de00000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000" +// ), }; +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: message.clone(), +// block_number: 14755883, +// block_timestamp: 1745305200, +// }); + +// let _ = chain_orchestrator.next().await.unwrap().unwrap(); + +// let tx = db.tx().await.unwrap(); +// let l1_message_result = +// tx.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); + +// assert_eq!( +// b256!("b2331b9010aac89f012d648fccc1f0a9aa5ef7b7b2afe21be297dd1a00000000"), +// l1_message_result.queue_hash.unwrap() +// ); +// } + +// #[tokio::test] +// async fn test_handle_reorg() { +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; + +// // Generate unstructured bytes. +// let mut bytes = [0u8; 1024]; +// rand::rng().fill(bytes.as_mut_slice()); +// let mut u = Unstructured::new(&bytes); + +// // Insert batch 0 into the database to satisfy the consistency checks in the chain +// // orchestrator +// let batch_0 = +// BatchCommitData { index: 0, block_number: 0, ..Arbitrary::arbitrary(&mut u).unwrap() +// }; let tx = db.tx_mut().await.unwrap(); +// tx.insert_batch(batch_0).await.unwrap(); + +// // Insert l1 message into the database to satisfy the consistency checks in the chain +// // orchestrator +// let l1_message = L1MessageEnvelope { +// queue_hash: None, +// l1_block_number: 0, +// l2_block_number: None, +// transaction: TxL1Message { queue_index: 0, ..Arbitrary::arbitrary(&mut u).unwrap() }, +// }; +// tx.insert_l1_message(l1_message).await.unwrap(); +// tx.commit().await.unwrap(); + +// // Generate a 3 random batch inputs and set their block numbers +// let mut batch_commit_block_1 = BatchCommitData::arbitrary(&mut u).unwrap(); +// batch_commit_block_1.block_number = 1; +// batch_commit_block_1.index = 1; +// let batch_commit_block_1 = batch_commit_block_1; + +// let mut batch_commit_block_2 = BatchCommitData::arbitrary(&mut u).unwrap(); +// batch_commit_block_2.block_number = 2; +// batch_commit_block_2.index = 2; +// let batch_commit_block_2 = batch_commit_block_2; + +// let mut batch_commit_block_3 = BatchCommitData::arbitrary(&mut u).unwrap(); +// batch_commit_block_3.block_number = 3; +// batch_commit_block_3.index = 3; +// let batch_commit_block_3 = batch_commit_block_3; + +// // Index batch inputs +// chain_orchestrator +// .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); +// chain_orchestrator +// .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_2.clone())); +// chain_orchestrator +// .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_3.clone())); + +// // Generate 3 random L1 messages and set their block numbers +// let l1_message_block_1 = L1MessageEnvelope { +// queue_hash: None, +// l1_block_number: 1, +// l2_block_number: None, +// transaction: TxL1Message { queue_index: 1, ..Arbitrary::arbitrary(&mut u).unwrap() }, +// }; +// let l1_message_block_2 = L1MessageEnvelope { +// queue_hash: None, +// l1_block_number: 2, +// l2_block_number: None, +// transaction: TxL1Message { queue_index: 2, ..Arbitrary::arbitrary(&mut u).unwrap() }, +// }; +// let l1_message_block_3 = L1MessageEnvelope { +// queue_hash: None, +// l1_block_number: 3, +// l2_block_number: None, +// transaction: TxL1Message { queue_index: 3, ..Arbitrary::arbitrary(&mut u).unwrap() }, +// }; + +// // Index L1 messages +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: l1_message_block_1.clone().transaction, +// block_number: l1_message_block_1.clone().l1_block_number, +// block_timestamp: 0, +// }); +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: l1_message_block_2.clone().transaction, +// block_number: l1_message_block_2.clone().l1_block_number, +// block_timestamp: 0, +// }); +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: l1_message_block_3.clone().transaction, +// block_number: l1_message_block_3.clone().l1_block_number, +// block_timestamp: 0, +// }); + +// // Reorg at block 2 +// chain_orchestrator.handle_l1_notification(L1Notification::Reorg(2)); + +// for _ in 0..7 { +// chain_orchestrator.next().await.unwrap().unwrap(); +// } + +// let tx = db.tx().await.unwrap(); + +// // Check that the batch input at block 30 is deleted +// let batch_commits = +// tx.get_batches().await.unwrap().map(|res| res.unwrap()).collect::>().await; + +// assert_eq!(3, batch_commits.len()); +// assert!(batch_commits.contains(&batch_commit_block_1)); +// assert!(batch_commits.contains(&batch_commit_block_2)); + +// // check that the L1 message at block 30 is deleted +// let l1_messages = tx +// .get_l1_messages(None) +// .await +// .unwrap() +// .map(|res| res.unwrap()) +// .collect::>() +// .await; +// assert_eq!(3, l1_messages.len()); +// assert!(l1_messages.contains(&l1_message_block_1)); +// assert!(l1_messages.contains(&l1_message_block_2)); +// } + +// // We ignore this test for now as it requires a more complex setup which leverages an L2 node +// // and is already covered in the integration test `can_handle_reorgs_while_sequencing` +// #[ignore] +// #[tokio::test] +// async fn test_handle_reorg_executed_l1_messages() { +// // Instantiate chain orchestrator and db +// let (mut chain_orchestrator, _database) = setup_test_chain_orchestrator().await; + +// // Generate unstructured bytes. +// let mut bytes = [0u8; 8192]; +// rand::rng().fill(bytes.as_mut_slice()); +// let mut u = Unstructured::new(&bytes); + +// // Generate a 3 random batch inputs and set their block numbers +// let batch_commit_block_1 = +// BatchCommitData { block_number: 5, index: 5, ..Arbitrary::arbitrary(&mut u).unwrap() +// }; let batch_commit_block_10 = BatchCommitData { +// block_number: 10, +// index: 10, +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }; + +// // Index batch inputs +// chain_orchestrator +// .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); +// chain_orchestrator +// .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_10.clone())); +// for _ in 0..2 { +// let _event = chain_orchestrator.next().await.unwrap().unwrap(); +// } + +// let batch_1 = BatchInfo::new(batch_commit_block_1.index, batch_commit_block_1.hash); +// let batch_10 = BatchInfo::new(batch_commit_block_10.index, batch_commit_block_10.hash); + +// const UNITS_FOR_TESTING: u64 = 20; +// const L1_MESSAGES_NOT_EXECUTED_COUNT: u64 = 7; +// let mut l1_messages = Vec::with_capacity(UNITS_FOR_TESTING as usize); +// for l1_message_queue_index in 0..UNITS_FOR_TESTING { +// let l1_message = L1MessageEnvelope { +// queue_hash: None, +// l1_block_number: l1_message_queue_index, +// l2_block_number: (UNITS_FOR_TESTING - l1_message_queue_index > +// L1_MESSAGES_NOT_EXECUTED_COUNT) +// .then_some(l1_message_queue_index), +// transaction: TxL1Message { +// queue_index: l1_message_queue_index, +// ..Arbitrary::arbitrary(&mut u).unwrap() +// }, +// }; +// chain_orchestrator.handle_l1_notification(L1Notification::L1Message { +// message: l1_message.transaction.clone(), +// block_number: l1_message.l1_block_number, +// block_timestamp: 0, +// }); +// chain_orchestrator.next().await.unwrap().unwrap(); +// l1_messages.push(l1_message); +// } + +// let mut blocks = Vec::with_capacity(UNITS_FOR_TESTING as usize); +// for block_number in 0..UNITS_FOR_TESTING { +// let l2_block = L2BlockInfoWithL1Messages { +// block_info: BlockInfo { +// number: block_number, +// hash: Arbitrary::arbitrary(&mut u).unwrap(), +// }, +// l1_messages: (UNITS_FOR_TESTING - block_number > L1_MESSAGES_NOT_EXECUTED_COUNT) +// .then_some(vec![l1_messages[block_number as usize].transaction.tx_hash()]) +// .unwrap_or_default(), +// }; +// let batch_info = if block_number < 5 { +// Some(batch_1) +// } else if block_number < 10 { +// Some(batch_10) +// } else { +// None +// }; +// if let Some(batch_info) = batch_info { +// chain_orchestrator +// .persist_l1_consolidated_blocks(vec![l2_block.clone()], batch_info); +// } else { +// chain_orchestrator.consolidate_validated_l2_blocks(vec![l2_block.clone()]); +// } + +// chain_orchestrator.next().await.unwrap().unwrap(); +// blocks.push(l2_block); +// } + +// // First we assert that we dont reorg the L2 or message queue hash for a higher block +// // than any of the L1 messages. +// chain_orchestrator.handle_l1_notification(L1Notification::Reorg(17)); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); +// assert_eq!( +// event, +// ChainOrchestratorEvent::L1Reorg { +// l1_block_number: 17, +// queue_index: None, +// l2_head_block_info: None, +// l2_safe_block_info: None +// } +// ); + +// // Reorg at block 7 which is one of the messages that has not been executed yet. No reorg +// // but we should ensure the L1 messages have been deleted. +// chain_orchestrator.handle_l1_notification(L1Notification::Reorg(7)); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); + +// assert_eq!( +// event, +// ChainOrchestratorEvent::L1Reorg { +// l1_block_number: 7, +// queue_index: Some(8), +// l2_head_block_info: Some(blocks[7].block_info), +// l2_safe_block_info: Some(blocks[4].block_info) +// } +// ); + +// // Now reorg at block 5 which contains L1 messages that have been executed . +// chain_orchestrator.handle_l1_notification(L1Notification::Reorg(3)); +// let event = chain_orchestrator.next().await.unwrap().unwrap(); + +// assert_eq!( +// event, +// ChainOrchestratorEvent::L1Reorg { +// l1_block_number: 3, +// queue_index: Some(4), +// l2_head_block_info: Some(blocks[3].block_info), +// l2_safe_block_info: Some(BlockInfo::new( +// 0, +// chain_orchestrator.chain_spec.genesis_hash() +// )), +// } +// ); +// } +// } diff --git a/crates/chain-orchestrator/src/retry.rs b/crates/chain-orchestrator/src/retry.rs index 35a7fa9d..b48b9518 100644 --- a/crates/chain-orchestrator/src/retry.rs +++ b/crates/chain-orchestrator/src/retry.rs @@ -98,6 +98,12 @@ impl CanRetry for crate::error::ChainOrchestratorError { } } +impl CanRetry for scroll_engine::EngineError { + fn can_retry(&self) -> bool { + matches!(self, Self::TransportError(_)) + } +} + #[cfg(test)] mod tests { use super::{CanRetry, Retry}; diff --git a/crates/chain-orchestrator/src/status.rs b/crates/chain-orchestrator/src/status.rs new file mode 100644 index 00000000..3f7ced2b --- /dev/null +++ b/crates/chain-orchestrator/src/status.rs @@ -0,0 +1,11 @@ +use super::SyncState; +use scroll_engine::ForkchoiceState; + +/// The current status of the chain orchestrator. +#[derive(Debug)] +pub struct ChainOrchestratorStatus { + /// The current sync state of the orchestrator. + pub sync_state: SyncState, + /// The current FCS for the manager. + pub forkchoice_state: ForkchoiceState, +} diff --git a/crates/chain-orchestrator/src/sync.rs b/crates/chain-orchestrator/src/sync.rs new file mode 100644 index 00000000..22c8baaf --- /dev/null +++ b/crates/chain-orchestrator/src/sync.rs @@ -0,0 +1,66 @@ +/// The sync state of the chain orchestrator. +#[derive(Debug, Default, Clone)] +pub struct SyncState { + /// The sync mode for L1. + l1: SyncMode, + /// The sync mode for L2. + l2: SyncMode, +} + +impl SyncState { + /// Returns a reference to the sync mode of L1. + pub fn l1(&self) -> &SyncMode { + &self.l1 + } + + /// Returns a reference to the sync mode of L2. + pub fn l2(&self) -> &SyncMode { + &self.l2 + } + + /// Returns a mutable reference to the sync mode of L1. + pub fn l1_mut(&mut self) -> &mut SyncMode { + &mut self.l1 + } + + /// Returns a mutable reference to the sync mode of L2. + pub fn l2_mut(&mut self) -> &mut SyncMode { + &mut self.l2 + } + + /// Returns true if both L1 and L2 are synced. + pub fn is_synced(&self) -> bool { + self.l1.is_synced() && self.l2.is_synced() + } +} + +/// The sync mode of the chain orchestrator. +#[derive(Debug, Default, Clone)] +pub enum SyncMode { + /// Syncing mode. + #[default] + Syncing, + /// Synced mode. + Synced, +} + +impl SyncMode { + /// Returns true if the sync mode is [`Self::Syncing`]. + pub fn is_syncing(&self) -> bool { + matches!(self, Self::Syncing) + } + + pub fn is_synced(&self) -> bool { + matches!(self, Self::Synced) + } + + /// Sets the sync mode to [`Self::Synced]. + pub fn set_synced(&mut self) { + *self = Self::Synced; + } + + /// Sets the sync mode to [`Self::Syncing`]. + pub fn set_syncing(&mut self) { + *self = Self::Syncing; + } +} diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index 174aac90..da7b858a 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -308,13 +308,8 @@ mod test { tx.insert_batch(data).await.unwrap(); for _ in 0..10 { - let block_info = L2BlockInfoWithL1Messages { - block_info: BlockInfo { - number: block_number, - hash: B256::arbitrary(&mut u).unwrap(), - }, - l1_messages: vec![], - }; + let block_info = + BlockInfo { number: block_number, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_block(block_info, batch_info).await.unwrap(); block_number += 1; } @@ -353,13 +348,8 @@ mod test { tx.insert_batch(second_batch).await.unwrap(); for _ in 0..10 { - let block_info = L2BlockInfoWithL1Messages { - block_info: BlockInfo { - number: block_number, - hash: B256::arbitrary(&mut u).unwrap(), - }, - l1_messages: vec![], - }; + let block_info = + BlockInfo { number: block_number, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_block(block_info, first_batch_info).await.unwrap(); block_number += 1; } @@ -526,6 +516,7 @@ mod test { .get_l1_messages(None) .await .unwrap() + .unwrap() .map(|res| res.unwrap()) .collect::>() .await; @@ -599,7 +590,7 @@ mod test { let mut block_infos = Vec::new(); for i in 200..205 { let block_info = BlockInfo { number: i, hash: B256::arbitrary(&mut u).unwrap() }; - let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; + let l2_block = block_info; block_infos.push(block_info); tx.insert_block(l2_block, batch_info).await.unwrap(); } @@ -641,19 +632,9 @@ mod test { let safe_block_1 = BlockInfo { number: 200, hash: B256::arbitrary(&mut u).unwrap() }; let safe_block_2 = BlockInfo { number: 201, hash: B256::arbitrary(&mut u).unwrap() }; - tx.insert_block( - L2BlockInfoWithL1Messages { block_info: safe_block_1, l1_messages: vec![] }, - batch_info, - ) - .await - .unwrap(); + tx.insert_block(safe_block_1, batch_info).await.unwrap(); - tx.insert_block( - L2BlockInfoWithL1Messages { block_info: safe_block_2, l1_messages: vec![] }, - batch_info, - ) - .await - .unwrap(); + tx.insert_block(safe_block_2, batch_info).await.unwrap(); // Should return the highest safe block (block 201) let latest_safe = tx.get_latest_safe_l2_info().await.unwrap(); @@ -676,12 +657,7 @@ mod test { for i in 400..410 { let block_info = BlockInfo { number: i, hash: B256::arbitrary(&mut u).unwrap() }; - tx.insert_block( - L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }, - batch_info, - ) - .await - .unwrap(); + tx.insert_block(block_info, batch_info).await.unwrap(); } // Delete blocks with number > 405 @@ -728,9 +704,8 @@ mod test { let batch_info: BatchInfo = batch_data.into(); let block_info = BlockInfo { number: 500 + i, hash: B256::arbitrary(&mut u).unwrap() }; - let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - tx.insert_block(l2_block, batch_info).await.unwrap(); + tx.insert_block(block_info, batch_info).await.unwrap(); } // Delete L2 blocks with batch index > 105 @@ -787,7 +762,7 @@ mod test { L2BlockInfoWithL1Messages { block_info, l1_messages: l1_message_hashes.clone() }; // Insert block - tx.insert_block(l2_block.clone(), batch_info).await.unwrap(); + tx.insert_block(l2_block.block_info, batch_info).await.unwrap(); tx.update_l1_messages_with_l2_block(l2_block).await.unwrap(); // Verify block was inserted @@ -823,9 +798,7 @@ mod test { // Insert initial block let block_info = BlockInfo { number: 600, hash: B256::arbitrary(&mut u).unwrap() }; - let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - - tx.insert_block(l2_block, batch_info_1).await.unwrap(); + tx.insert_block(block_info, batch_info_1).await.unwrap(); // Verify initial insertion let retrieved_block = tx.get_l2_block_info_by_number(600).await.unwrap(); @@ -845,10 +818,8 @@ mod test { assert_eq!(initial_batch_info, batch_info_1); // Update the same block with different batch info (upsert) - let updated_l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - let tx = db.tx_mut().await.unwrap(); - tx.insert_block(updated_l2_block, batch_info_2).await.unwrap(); + tx.insert_block(block_info, batch_info_2).await.unwrap(); tx.commit().await.unwrap(); // Verify the block still exists and was updated @@ -883,14 +854,8 @@ mod test { // Insert batch 1 and associate it with two blocks in the database let batch_data_1 = BatchCommitData { index: 1, block_number: 10, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let block_1 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 1, hash: B256::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; - let block_2 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 2, hash: B256::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; + let block_1 = BlockInfo { number: 1, hash: B256::arbitrary(&mut u).unwrap() }; + let block_2 = BlockInfo { number: 2, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_1.clone()).await.unwrap(); tx.insert_block(block_1.clone(), batch_data_1.clone().into()).await.unwrap(); tx.insert_block(block_2.clone(), batch_data_1.clone().into()).await.unwrap(); @@ -898,10 +863,7 @@ mod test { // Insert batch 2 and associate it with one block in the database let batch_data_2 = BatchCommitData { index: 2, block_number: 20, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let block_3 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 3, hash: B256::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; + let block_3 = BlockInfo { number: 3, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_2.clone()).await.unwrap(); tx.insert_block(block_3.clone(), batch_data_2.clone().into()).await.unwrap(); @@ -909,14 +871,11 @@ mod test { // block let batch_data_3 = BatchCommitData { index: 3, block_number: 20, ..Arbitrary::arbitrary(&mut u).unwrap() }; - let block_4 = L2BlockInfoWithL1Messages { - block_info: BlockInfo { number: 4, hash: B256::arbitrary(&mut u).unwrap() }, - l1_messages: vec![], - }; + let block_4 = BlockInfo { number: 4, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_3.clone()).await.unwrap(); tx.insert_block(block_4.clone(), batch_data_3.clone().into()).await.unwrap(); - tx.set_latest_finalized_l1_block_number(21).await.unwrap(); + tx.set_finalized_l1_block_number(21).await.unwrap(); tx.commit().await.unwrap(); // Verify the batches and blocks were inserted correctly @@ -933,10 +892,10 @@ mod test { assert_eq!(retrieved_batch_1, batch_data_1); assert_eq!(retrieved_batch_2, batch_data_2); assert_eq!(retrieved_batch_3, batch_data_3); - assert_eq!(retried_block_1, block_1.block_info); - assert_eq!(retried_block_2, block_2.block_info); - assert_eq!(retried_block_3, block_3.block_info); - assert_eq!(retried_block_4, block_4.block_info); + assert_eq!(retried_block_1, block_1); + assert_eq!(retried_block_2, block_2); + assert_eq!(retried_block_3, block_3); + assert_eq!(retried_block_4, block_4); // Call prepare_on_startup which should not error let tx = db.tx_mut().await.unwrap(); @@ -944,7 +903,7 @@ mod test { tx.commit().await.unwrap(); // verify the result - assert_eq!(result, (Some(block_2.block_info), Some(11))); + assert_eq!(result, (Some(block_2), Some(11))); // Verify that batches 2 and 3 are deleted let tx = db.tx().await.unwrap(); diff --git a/crates/database/db/src/error.rs b/crates/database/db/src/error.rs index 0320a03f..1a579f93 100644 --- a/crates/database/db/src/error.rs +++ b/crates/database/db/src/error.rs @@ -16,4 +16,7 @@ pub enum DatabaseError { /// The L1 message was not found in database. #[error("L1 message at index [{0}] not found in database")] L1MessageNotFound(L1MessageStart), + /// The finalized L1 block was not found in database. + #[error("Finalized L1 block not found in database")] + FinalizedL1BlockNotFound, } diff --git a/crates/database/db/src/lib.rs b/crates/database/db/src/lib.rs index 80325faa..6c944828 100644 --- a/crates/database/db/src/lib.rs +++ b/crates/database/db/src/lib.rs @@ -16,7 +16,7 @@ pub use models::*; mod operations; pub use operations::{ - DatabaseReadOperations, DatabaseWriteOperations, L1MessageStart, UnwindResult, + DatabaseReadOperations, DatabaseWriteOperations, L1MessageStart, NotIncludedStart, UnwindResult, }; mod transaction; diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 05a79d82..b2a17de1 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -4,7 +4,8 @@ use crate::{ReadConnectionProvider, WriteConnectionProvider}; use alloy_primitives::{Signature, B256}; use futures::{Stream, StreamExt}; use rollup_node_primitives::{ - BatchCommitData, BatchInfo, BlockInfo, L1MessageEnvelope, L2BlockInfoWithL1Messages, Metadata, + BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, + L1MessageEnvelope, L2BlockInfoWithL1Messages, Metadata, }; use scroll_alloy_rpc_types_engine::BlockDataHint; use sea_orm::{ @@ -64,12 +65,9 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati Ok(()) } - /// Set the latest finalized L1 block number. - async fn set_latest_finalized_l1_block_number( - &self, - block_number: u64, - ) -> Result<(), DatabaseError> { - tracing::trace!(target: "scroll::db", block_number, "Updating the latest finalized L1 block number in the database."); + /// Set the finalized L1 block number. + async fn set_finalized_l1_block_number(&self, block_number: u64) -> Result<(), DatabaseError> { + tracing::trace!(target: "scroll::db", block_number, "Updating the finalized L1 block number in the database."); let metadata: models::metadata::ActiveModel = Metadata { l1_finalized_block: block_number }.into(); Ok(models::metadata::Entity::insert(metadata) @@ -257,7 +255,7 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati /// Insert multiple blocks into the database. async fn insert_blocks( &self, - blocks: Vec, + blocks: Vec, batch_info: BatchInfo, ) -> Result<(), DatabaseError> { for block in blocks { @@ -269,7 +267,7 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati /// Insert a new block in the database. async fn insert_block( &self, - block_info: L2BlockInfoWithL1Messages, + block_info: BlockInfo, batch_info: BatchInfo, ) -> Result<(), DatabaseError> { // We only insert safe blocks into the database, we do not persist unsafe blocks. @@ -277,11 +275,11 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati target: "scroll::db", batch_hash = ?batch_info.hash, batch_index = batch_info.index, - block_number = block_info.block_info.number, - block_hash = ?block_info.block_info.hash, + block_number = block_info.number, + block_hash = ?block_info.hash, "Inserting block into database." ); - let l2_block: models::l2_block::ActiveModel = (block_info.block_info, batch_info).into(); + let l2_block: models::l2_block::ActiveModel = (block_info, batch_info).into(); models::l2_block::Entity::insert(l2_block) .on_conflict( OnConflict::column(models::l2_block::Column::BlockNumber) @@ -300,10 +298,7 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati /// Insert the genesis block into the database. async fn insert_genesis_block(&self, genesis_hash: B256) -> Result<(), DatabaseError> { - let genesis_block = L2BlockInfoWithL1Messages { - block_info: BlockInfo::new(0, genesis_hash), - l1_messages: vec![], - }; + let genesis_block = BlockInfo::new(0, genesis_hash); let genesis_batch = BatchInfo::new(0, B256::ZERO); self.insert_block(genesis_block, genesis_batch).await } @@ -313,6 +308,13 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati &self, blocks: Vec, ) -> Result<(), DatabaseError> { + tracing::trace!(target: "scroll::db", num_blocks = blocks.len(), "Updating executed L1 messages from blocks with L2 block number in the database."); + + // First, purge all existing mappings for unsafe blocks. + self.purge_l1_message_to_l2_block_mappings(blocks.first().map(|b| b.block_info.number)) + .await?; + + // Then, update the executed L1 messages for each block. for block in blocks { self.update_l1_messages_with_l2_block(block).await?; } @@ -345,6 +347,51 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati Ok(()) } + /// Purge all L1 message to L2 block mappings from the database for blocks greater or equal to + /// the provided block number. If the no block number is provided, purge mappings for all + /// unsafe blocks. + async fn purge_l1_message_to_l2_block_mappings( + &self, + block_number: Option, + ) -> Result<(), DatabaseError> { + tracing::trace!(target: "scroll::db", ?block_number, "Purging L1 message to L2 block mappings from database."); + + let filter = if let Some(block_number) = block_number { + models::l1_message::Column::L2BlockNumber.gte(block_number as i64) + } else { + let safe_block_number = self.get_latest_safe_l2_info().await?; + models::l1_message::Column::L2BlockNumber + .gt(safe_block_number.map(|(block_info, _)| block_info.number as i64).unwrap_or(0)) + }; + + models::l1_message::Entity::update_many() + .col_expr(models::l1_message::Column::L2BlockNumber, Expr::value(None::)) + .filter(filter) + .exec(self.get_connection()) + .await?; + + Ok(()) + } + + /// Insert the outcome of a batch consolidation into the database. + async fn insert_batch_consolidation_outcome( + &self, + outcome: BatchConsolidationOutcome, + ) -> Result<(), DatabaseError> { + for block in outcome.blocks { + match block { + BlockConsolidationOutcome::Consolidated(block_info) => { + self.insert_block(block_info, outcome.batch_info).await?; + } + BlockConsolidationOutcome::Reorged(block_info) => { + self.insert_block(block_info.block_info, outcome.batch_info).await?; + self.update_l1_messages_with_l2_block(block_info).await?; + } + } + } + Ok(()) + } + /// Unwinds the chain orchestrator by deleting all indexed data greater than the provided L1 /// block number. async fn unwind( @@ -500,10 +547,12 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { async fn get_l1_messages<'a>( &'a self, start: Option, - ) -> Result> + 'a, DatabaseError> - { + ) -> Result< + Option> + 'a>, + DatabaseError, + > { let queue_index = match start { - Some(L1MessageStart::Index(i)) => i, + Some(L1MessageStart::Index(i)) => Ok::<_, DatabaseError>(Some(i)), Some(L1MessageStart::Hash(ref h)) => { // Lookup message by hash let record = models::l1_message::Entity::find() @@ -512,16 +561,88 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .await? .ok_or_else(|| DatabaseError::L1MessageNotFound(L1MessageStart::Hash(*h)))?; - record.queue_index as u64 + Ok(Some(record.queue_index as u64)) } - None => 0, - }; + Some(L1MessageStart::BlockNumber(block_number)) => { + let exact_match = models::l1_message::Entity::find() + .filter(models::l1_message::Column::L1BlockNumber.eq(block_number as i64)) + .order_by_desc(models::l1_message::Column::QueueIndex) + .into_tuple::() + .one(self.get_connection()) + .await?; + + if let Some(queue_index) = exact_match { + Ok(Some(queue_index as u64)) + } else { + // If no exact match is found, find the last message before the block number + Ok(models::l1_message::Entity::find() + .filter(models::l1_message::Column::L1BlockNumber.lt(block_number as i64)) + .order_by_desc(models::l1_message::Column::L1BlockNumber) + .order_by_desc(models::l1_message::Column::QueueIndex) + .into_tuple::() + .one(self.get_connection()) + .await? + .map(|x| x as u64)) + } + } + Some(L1MessageStart::NotIncluded(NotIncludedStart::Finalized)) => { + let finalized_block_number = self + .get_finalized_l1_block_number() + .await? + .ok_or(DatabaseError::FinalizedL1BlockNotFound)?; + let condition = Condition::all() + .add( + models::l1_message::Column::L1BlockNumber + .lte(finalized_block_number as i64), + ) + .add(models::l1_message::Column::L2BlockNumber.is_null()); + Ok(models::l1_message::Entity::find() + .filter(condition) + .order_by_desc(models::l1_message::Column::QueueIndex) + .into_tuple::() + .one(self.get_connection()) + .await? + .map(|x| x as u64)) + } + Some(L1MessageStart::NotIncluded(NotIncludedStart::BlockDepth(depth))) => { + // TODO: USE LATEST BLOCK NUMBER NOT FINALIZED + let finalized_block_number = self + .get_finalized_l1_block_number() + .await? + .ok_or(DatabaseError::FinalizedL1BlockNotFound)?; + let target_block_number = finalized_block_number.checked_sub(depth); + if let Some(target_block_number) = target_block_number { + let condition = Condition::all() + .add( + models::l1_message::Column::L1BlockNumber + .lte(target_block_number as i64), + ) + .add(models::l1_message::Column::L2BlockNumber.is_null()); + Ok(models::l1_message::Entity::find() + .filter(condition) + .order_by_desc(models::l1_message::Column::QueueIndex) + .into_tuple::() + .one(self.get_connection()) + .await? + .map(|x| x as u64)) + } else { + Ok(None) + } + } + None => Ok(Some(0)), + }?; - Ok(models::l1_message::Entity::find() - .filter(models::l1_message::Column::QueueIndex.gte(queue_index)) - .stream(self.get_connection()) - .await? - .map(|res| Ok(res.map(Into::into)?))) + let queue_index = + if let Some(queue_index) = queue_index { queue_index } else { return Ok(None) }; + + Ok(Some( + models::l1_message::Entity::find() + .filter(models::l1_message::Column::QueueIndex.gte(queue_index)) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(|res| Ok(res.map(Into::into)?)), + )) } /// Get the extra data for the provided block number. @@ -667,6 +788,32 @@ pub enum L1MessageStart { Index(u64), /// Start from the provided queue hash. Hash(B256), + /// Start from the first message for the provided block number. + BlockNumber(u64), + /// Start from messages that have not been included in a block yet. + NotIncluded(NotIncludedStart), +} + +/// This type defines where to start when fetching L1 messages that have not been included in a +/// block yet. +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum NotIncludedStart { + /// Start from finalized messages that have not been included in a block yet. + Finalized, + /// Start from unfinalized messages that are included in L1 blocks at a specific depth. + BlockDepth(u64), +} + +impl L1MessageStart { + /// Creates a new [`L1MessageStart`] for the provided queue index. + pub fn index(index: u64) -> Self { + Self::Index(index) + } + + /// Creates a new [`L1MessageStart`] for the provided block number. + pub fn block_number(number: u64) -> Self { + Self::BlockNumber(number) + } } impl fmt::Display for L1MessageStart { @@ -674,16 +821,17 @@ impl fmt::Display for L1MessageStart { match self { Self::Index(index) => write!(f, "Index({index})"), Self::Hash(hash) => write!(f, "Hash({hash:#x})"), + Self::BlockNumber(number) => write!(f, "BlockNumber({number})"), + Self::NotIncluded(start) => match start { + NotIncludedStart::Finalized => write!(f, "NotIncluded(Finalized)"), + NotIncludedStart::BlockDepth(depth) => { + write!(f, "NotIncluded(BlockDepth({depth}))") + } + }, } } } -impl From for L1MessageStart { - fn from(value: u64) -> Self { - Self::Index(value) - } -} - impl From for L1MessageStart { fn from(value: B256) -> Self { Self::Hash(value) diff --git a/crates/database/migration/src/lib.rs b/crates/database/migration/src/lib.rs index 5cadddb5..d74c6e04 100644 --- a/crates/database/migration/src/lib.rs +++ b/crates/database/migration/src/lib.rs @@ -11,6 +11,7 @@ mod m20250829_042803_add_table_indexes; mod m20250901_102341_add_commit_batch_processed_column; mod m20250904_175949_block_signature; mod m20250923_135359_add_index_block_hash; +mod m20250929_161536_add_additional_indexes; mod migration_info; pub use migration_info::{ MigrationInfo, ScrollDevMigrationInfo, ScrollMainnetMigrationInfo, ScrollSepoliaMigrationInfo, @@ -33,6 +34,7 @@ impl MigratorTrait for Migrator { Box::new(m20250901_102341_add_commit_batch_processed_column::Migration), Box::new(m20250904_175949_block_signature::Migration), Box::new(m20250923_135359_add_index_block_hash::Migration), + Box::new(m20250929_161536_add_additional_indexes::Migration), ] } } diff --git a/crates/database/migration/src/m20250929_161536_add_additional_indexes.rs b/crates/database/migration/src/m20250929_161536_add_additional_indexes.rs new file mode 100644 index 00000000..cd7aab2f --- /dev/null +++ b/crates/database/migration/src/m20250929_161536_add_additional_indexes.rs @@ -0,0 +1,34 @@ +use super::m20250304_125946_add_l1_msg_table::L1Message; +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Create index for the l1 message to l2 block mapping. + manager + .create_index( + Index::create() + .name("idx_l1_message_l2_block") + .col(L1Message::L2BlockNumber) + .table(L1Message::Table) + .to_owned(), + ) + .await?; + + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // Drop index for the l1 message to l2 block mapping. + manager + .drop_index( + Index::drop().name("idx_l1_message_l2_block").table(L1Message::Table).to_owned(), + ) + .await?; + + Ok(()) + } +} diff --git a/crates/derivation-pipeline/src/lib.rs b/crates/derivation-pipeline/src/lib.rs index 7fee71bc..83d26e69 100644 --- a/crates/derivation-pipeline/src/lib.rs +++ b/crates/derivation-pipeline/src/lib.rs @@ -11,6 +11,9 @@ pub use error::DerivationPipelineError; mod metrics; pub use metrics::DerivationPipelineMetrics; +mod new; +pub use new::{BatchDerivationResult, DerivationPipelineNew, DerivedAttributes}; + use crate::data_source::CodecDataSource; use std::{boxed::Box, collections::VecDeque, fmt::Formatter, sync::Arc, time::Instant, vec::Vec}; @@ -30,7 +33,7 @@ use rollup_node_primitives::{ use rollup_node_providers::{BlockDataProvider, L1Provider}; use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; use scroll_codec::{decoding::payload::PayloadData, Codec}; -use scroll_db::{Database, DatabaseReadOperations, DatabaseTransactionProvider}; +use scroll_db::{Database, DatabaseReadOperations, DatabaseTransactionProvider, L1MessageStart}; use tokio::time::Interval; /// A future that resolves to a stream of [`ScrollPayloadAttributesWithBatchInfo`]. @@ -352,7 +355,10 @@ async fn iter_l1_messages_from_payload( let total_l1_messages = data.blocks.iter().map(|b| b.context.num_l1_messages as u64).sum(); let messages = if let Some(index) = data.queue_index_start() { - provider.get_n_messages(index.into(), total_l1_messages).await.map_err(Into::into)? + provider + .get_n_messages(L1MessageStart::index(index), total_l1_messages) + .await + .map_err(Into::into)? } else if let Some(hash) = data.prev_l1_message_queue_hash() { // If the message queue hash is zero then we should use the V2 L1 message queue start // index. We must apply this branch logic because we do not have a L1 @@ -360,7 +366,10 @@ async fn iter_l1_messages_from_payload( // hash for the first L1 message of the V2 contract). if hash == &B256::ZERO { provider - .get_n_messages(l1_v2_message_queue_start_index.into(), total_l1_messages) + .get_n_messages( + L1MessageStart::index(l1_v2_message_queue_start_index), + total_l1_messages, + ) .await .map_err(Into::into)? } else { diff --git a/crates/derivation-pipeline/src/new.rs b/crates/derivation-pipeline/src/new.rs new file mode 100644 index 00000000..be621c06 --- /dev/null +++ b/crates/derivation-pipeline/src/new.rs @@ -0,0 +1,228 @@ +use super::*; +use futures::{stream::FuturesOrdered, StreamExt}; +use tokio::sync::Mutex; + +/// A structure holding the current unresolved futures for the derivation pipeline. +#[derive(Debug)] +pub struct DerivationPipelineNew

{ + /// The active batch derivation futures. + futures: Arc>>, + /// A reference to the database. + database: Arc, + /// A L1 provider. + l1_provider: P, + /// The L1 message queue index at which the V2 L1 message queue was enabled. + l1_v2_message_queue_start_index: u64, + /// The metrics of the pipeline. + metrics: DerivationPipelineMetrics, +} + +impl

DerivationPipelineNew

{ + /// Returns a new instance of the [`DerivationPipelineNew`]. + pub fn new( + l1_provider: P, + database: Arc, + l1_v2_message_queue_start_index: u64, + ) -> Self { + Self { + futures: Arc::new(Mutex::new(FuturesOrdered::new())), + database, + l1_provider, + l1_v2_message_queue_start_index, + metrics: DerivationPipelineMetrics::default(), + } + } +} + +impl

DerivationPipelineNew

+where + P: L1Provider + Clone + Send + Sync + 'static, +{ + /// Pushes a new batch info to the derivation pipeline. + pub async fn push(&mut self, batch_info: Arc) { + let fut = self.derivation_future(batch_info); + self.futures.lock().await.push_back(fut); + } + + /// Returns the number of unresolved futures in the derivation pipeline. + pub async fn len(&self) -> usize { + self.futures.lock().await.len() + } + + fn derivation_future(&self, batch_info: Arc) -> DerivationPipelineFutureNew { + let database = self.database.clone(); + let metrics = self.metrics.clone(); + let provider = self.l1_provider.clone(); + let l1_v2_message_queue_start_index = self.l1_v2_message_queue_start_index; + + Box::pin(async move { + let derive_start = Instant::now(); + + // get the batch commit data. + let tx = database.tx().await.map_err(|e| (batch_info.clone(), e.into()))?; + let batch = tx + .get_batch_by_index(batch_info.index) + .await + .map_err(|err| (batch_info.clone(), err.into()))? + .ok_or(( + batch_info.clone(), + DerivationPipelineError::UnknownBatch(batch_info.index), + ))?; + + // derive the attributes and attach the corresponding batch info. + let result = derive_new(batch, provider, tx, l1_v2_message_queue_start_index) + .await + .map_err(|err| (batch_info.clone(), err))?; + + // update metrics. + metrics.derived_blocks.increment(result.attributes.len() as u64); + let execution_duration = derive_start.elapsed().as_secs_f64(); + metrics.blocks_per_second.set(result.attributes.len() as f64 / execution_duration); + Ok(result) + }) + } +} + +impl

Stream for DerivationPipelineNew

+where + P: L1Provider + Unpin + Clone + Send + Sync + 'static, +{ + type Item = BatchDerivationResult; + + fn poll_next( + self: Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + let this = self.get_mut(); + + // Poll the next future in the ordered set of futures. + match this.futures.try_lock() { + Ok(mut guard) => { + match guard.poll_next_unpin(cx) { + // If the derivation failed then push it to the front of the queue to be + // retried. + Poll::Ready(Some(Err((batch_info, err)))) => { + tracing::error!(target: "scroll::derivation_pipeline", ?batch_info, ?err, "Failed to derive payload attributes"); + guard.push_front(this.derivation_future(batch_info.clone())); + return Poll::Pending + } + // If the derivation succeeded then return the attributes. + Poll::Ready(Some(Ok(result))) => return Poll::Ready(Some(result)), + // If there are no more futures then return pending. + _ => return Poll::Pending, + } + } + Err(_) => { + // Could not acquire the lock, return pending. + cx.waker().wake_by_ref(); + return Poll::Pending + } + } + } +} + +/// The result of deriving a batch. +#[derive(Debug)] +pub struct BatchDerivationResult { + /// The derived payload attributes. + pub attributes: Vec, + /// The batch info associated with the derived attributes. + pub batch_info: BatchInfo, +} + +/// The derived attributes along with the block number they correspond to. +#[derive(Debug, Clone)] +pub struct DerivedAttributes { + /// The block number the attributes correspond to. + pub block_number: u64, + /// The derived payload attributes. + pub attributes: ScrollPayloadAttributes, +} + +/// A future that resolves to a stream of [`ScrollPayloadAttributesWithBatchInfo`]. +type DerivationPipelineFutureNew = Pin< + Box< + dyn Future< + Output = Result, DerivationPipelineError)>, + > + Send, + >, +>; + +/// Returns a vector of [`ScrollPayloadAttributes`] from the [`BatchCommitData`] and a +/// [`L1Provider`]. +pub async fn derive_new( + batch: BatchCommitData, + l1_provider: L1P, + l2_provider: L2P, + l1_v2_message_queue_start_index: u64, +) -> Result { + // fetch the blob then decode the input batch. + let blob = if let Some(hash) = batch.blob_versioned_hash { + l1_provider.blob(batch.block_timestamp, hash).await? + } else { + None + }; + let data = CodecDataSource { calldata: batch.calldata.as_ref(), blob: blob.as_deref() }; + let decoded = Codec::decode(&data)?; + + // set the cursor for the l1 provider. + let payload_data = &decoded.data; + let mut l1_messages_iter = + iter_l1_messages_from_payload(&l1_provider, payload_data, l1_v2_message_queue_start_index) + .await?; + + let skipped_l1_messages = decoded.data.skipped_l1_message_bitmap.clone().unwrap_or_default(); + let mut skipped_l1_messages = skipped_l1_messages.into_iter(); + let blocks = decoded.data.into_l2_blocks(); + let mut attributes = Vec::with_capacity(blocks.len()); + for mut block in blocks { + // query the appropriate amount of l1 messages. + let mut txs = Vec::with_capacity(block.context.num_transactions as usize); + for _ in 0..block.context.num_l1_messages { + // check if the next l1 message should be skipped. + if matches!(skipped_l1_messages.next(), Some(bit) if bit) { + let _ = l1_messages_iter.next(); + continue; + } + + let l1_message = l1_messages_iter + .next() + .ok_or(DerivationPipelineError::MissingL1Message(block.clone()))?; + let mut bytes = Vec::with_capacity(l1_message.transaction.eip2718_encoded_length()); + l1_message.transaction.eip2718_encode(&mut bytes); + txs.push(bytes.into()); + } + + // add the block transactions. + txs.append(&mut block.transactions); + + // get the block data for the l2 block. + let number = block.context.number; + // TODO(performance): can this be improved by adding block_data_range. + let block_data = l2_provider.block_data(number).await.map_err(Into::into)?; + + // construct the payload attributes. + let attribute = DerivedAttributes { + block_number: number, + attributes: ScrollPayloadAttributes { + payload_attributes: PayloadAttributes { + timestamp: block.context.timestamp, + suggested_fee_recipient: Address::ZERO, + prev_randao: B256::ZERO, + withdrawals: None, + parent_beacon_block_root: None, + }, + transactions: Some(txs), + no_tx_pool: true, + block_data_hint: block_data.unwrap_or_else(BlockDataHint::none), + gas_limit: Some(block.context.gas_limit), + }, + }; + attributes.push(attribute); + } + + Ok(BatchDerivationResult { + attributes, + batch_info: BatchInfo { index: batch.index, hash: batch.hash }, + }) +} diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs index c2cff1f5..e85f861f 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -101,17 +101,17 @@ where /// Sets the finalized block info. pub fn set_finalized_block_info(&mut self, block_info: BlockInfo) { - self.fcs.update_finalized_block_info(block_info); + let _ = self.fcs.update_finalized_block_info(block_info); } /// Sets the safe block info. pub fn set_safe_block_info(&mut self, block_info: BlockInfo) { - self.fcs.update_safe_block_info(block_info); + let _ = self.fcs.update_safe_block_info(block_info); } /// Sets the head block info. pub fn set_head_block_info(&mut self, block_info: BlockInfo) { - self.fcs.update_head_block_info(block_info); + let _ = self.fcs.update_head_block_info(block_info); } /// Sets the payload building duration. @@ -255,7 +255,7 @@ where // Update the unsafe block info if let Some(block_info) = block_info { tracing::trace!(target: "scroll::engine", ?block_info, "updating unsafe block info"); - self.fcs.update_head_block_info(block_info); + let _ = self.fcs.update_head_block_info(block_info); }; // Update the sync status @@ -286,13 +286,13 @@ where // safe and finalized block info. Update this once we implement issue #273. // Update the safe and finalized block info and return the block info. tracing::trace!(target: "scroll::engine", ?block_info, "updating safe and finalized block info from block derived from L1"); - self.fcs.update_safe_block_info(block_info.block_info); - self.fcs.update_finalized_block_info(block_info.block_info); + let _ = self.fcs.update_safe_block_info(block_info.block_info); + let _ = self.fcs.update_finalized_block_info(block_info.block_info); // If we reorged, update the head block info if consolidation_outcome.is_reorg() { tracing::warn!(target: "scroll::engine", ?block_info, "reorging head to l1 derived block"); - self.fcs.update_head_block_info(block_info.block_info); + let _ = self.fcs.update_head_block_info(block_info.block_info); } // record the metric. @@ -325,7 +325,7 @@ where // Update the unsafe block info and return the block let block_info = BlockInfo::new(block.number, block.hash_slow()); tracing::trace!(target: "scroll::engine", ?block_info, "updating unsafe block info from new payload"); - self.fcs.update_head_block_info(block_info); + let _ = self.fcs.update_head_block_info(block_info); // record the metrics. self.metrics.build_new_payload_duration.record(duration.as_secs_f64()); @@ -463,7 +463,7 @@ where // If we have an optimistic sync target, issue the optimistic sync. if let Some(block_info) = this.optimistic_sync_target.take() { - this.fcs.update_head_block_info(block_info); + let _ = this.fcs.update_head_block_info(block_info); let fcs = this.fcs.get_alloy_optimistic_fcs(); this.engine_future = Some(MeteredFuture::new(EngineFuture::optimistic_sync(this.client.clone(), fcs))); diff --git a/crates/engine/src/error.rs b/crates/engine/src/error.rs index 40d6c79e..f7e47a95 100644 --- a/crates/engine/src/error.rs +++ b/crates/engine/src/error.rs @@ -27,3 +27,43 @@ pub enum EngineDriverError { #[error("Forkchoice update response missing payload id for payload building job")] PayloadBuildingMissingPayloadId(ScrollPayloadAttributes), } + +/// The error type for the fork choice state. +#[derive(Debug, thiserror::Error)] +pub enum FcsError { + /// No update was provided for head, safe or finalized. + #[error("No update provided for head, safe or finalized")] + NoUpdateProvided, + /// Finalized block number not increasing. + #[error("Finalized block number not increasing")] + FinalizedBlockNumberNotIncreasing, + /// Head block number cannot be below safe block number. + #[error("Safe block number can not be below the head block number")] + HeadBelowSafe, + /// Safe block number cannot be below finalized block number. + #[error("Safe block number can not be below the finalized block number")] + SafeBelowFinalized, +} + +/// The error type for the Engine. +#[derive(Debug, thiserror::Error)] +pub enum EngineError { + /// An error occurred in the fork choice state. + #[error("Fork choice state error: {0}")] + FcsError(#[from] FcsError), + /// An error occurred in the transport layer. + #[error("Transport error: {0}")] + TransportError(#[from] scroll_alloy_provider::ScrollEngineApiError), +} + +impl EngineError { + /// Creates a new [`EngineError`] for a [`FcsError::NoUpdateProvided`]. + pub fn fcs_no_update_provided() -> Self { + Self::FcsError(FcsError::NoUpdateProvided) + } + + /// Creates a new [`EngineError`] for a [`FcsError::FinalizedBlockNumberNotIncreasing`]. + pub fn fcs_finalized_block_number_not_increasing() -> Self { + Self::FcsError(FcsError::FinalizedBlockNumberNotIncreasing) + } +} diff --git a/crates/engine/src/fcs.rs b/crates/engine/src/fcs.rs index 784d57d6..d5ea53c2 100644 --- a/crates/engine/src/fcs.rs +++ b/crates/engine/src/fcs.rs @@ -1,3 +1,4 @@ +use crate::FcsError; use alloy_chains::NamedChain; use alloy_eips::{BlockId, BlockNumberOrTag}; use alloy_primitives::{Sealable, B256}; @@ -68,19 +69,59 @@ impl ForkchoiceState { Some(Self::head_from_genesis(genesis_hash_from_chain_spec(chain_spec)?)) } + /// Update the forkchoice state with the given `head`, `safe` and `finalized` block info. + pub fn update( + &mut self, + head: Option, + safe: Option, + finalized: Option, + ) -> Result<(), FcsError> { + tracing::debug!(target: "scroll::engine::fcs", ?head, ?safe, ?finalized, current = ?self, "Updating fork choice state"); + // Check that at least one of head, safe or finalized is Some. + if head.is_none() && safe.is_none() && finalized.is_none() { + return Err(FcsError::NoUpdateProvided); + } + + // Build the candidate new state. + let new_finalized = finalized.unwrap_or(self.finalized.clone()); + let new_safe = safe.unwrap_or(self.safe.clone()); + let new_head = head.unwrap_or(self.head.clone()); + + // Check that the finalized block number is increasing. + if new_finalized.number <= self.finalized.number && new_finalized != self.finalized { + return Err(FcsError::FinalizedBlockNumberNotIncreasing); + } + + // Assert invariants: head >= safe >= finalized. + if new_head.number < new_safe.number { + return Err(FcsError::HeadBelowSafe); + } + + if new_safe.number < new_finalized.number { + return Err(FcsError::SafeBelowFinalized); + } + + // Commit the state. + self.head = new_head; + self.safe = new_safe; + self.finalized = new_finalized; + + Ok(()) + } + /// Updates the `head` block info. - pub fn update_head_block_info(&mut self, head: BlockInfo) { - self.head = head; + pub fn update_head_block_info(&mut self, head: BlockInfo) -> Result<(), FcsError> { + self.update(Some(head), None, None) } /// Updates the `safe` block info. - pub fn update_safe_block_info(&mut self, safe: BlockInfo) { - self.safe = safe; + pub fn update_safe_block_info(&mut self, safe: BlockInfo) -> Result<(), FcsError> { + self.update(None, Some(safe), None) } /// Updates the `finalized` block info. - pub fn update_finalized_block_info(&mut self, finalized: BlockInfo) { - self.finalized = finalized; + pub fn update_finalized_block_info(&mut self, finalized: BlockInfo) -> Result<(), FcsError> { + self.update(None, None, Some(finalized)) } /// Returns the block info for the `head` block. diff --git a/crates/engine/src/lib.rs b/crates/engine/src/lib.rs index a649bf68..d9a31277 100644 --- a/crates/engine/src/lib.rs +++ b/crates/engine/src/lib.rs @@ -6,7 +6,7 @@ pub(crate) mod api; pub use driver::EngineDriver; mod driver; -pub use error::EngineDriverError; +pub use error::{EngineDriverError, EngineError, FcsError}; mod error; pub use event::EngineDriverEvent; @@ -22,6 +22,10 @@ pub use metrics::EngineDriverMetrics; mod metrics; mod payload; +pub use payload::block_matches_attributes; + +mod new; +pub use new::Engine; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; diff --git a/crates/engine/src/new.rs b/crates/engine/src/new.rs new file mode 100644 index 00000000..3ab19af0 --- /dev/null +++ b/crates/engine/src/new.rs @@ -0,0 +1,120 @@ +use super::{EngineError, ForkchoiceState}; +use alloy_rpc_types_engine::{ExecutionPayloadV1, ForkchoiceUpdated, PayloadStatus}; +use reth_scroll_primitives::ScrollBlock; +use rollup_node_primitives::BlockInfo; +use scroll_alloy_provider::ScrollEngineApi; +use std::sync::Arc; + +/// The engine that communicates with the execution layer. +#[derive(Debug, Clone)] +pub struct Engine { + /// The engine API client. + client: Arc, + /// The fork choice state of the engine. + fcs: ForkchoiceState, +} + +impl Engine +where + EC: ScrollEngineApi + Sync + 'static, +{ + /// Create a new [`Engine`]. + pub fn new(client: Arc, fcs: ForkchoiceState) -> Self { + Self { client, fcs } + } + + /// Get a reference to the current fork choice state. + pub fn fcs(&self) -> &ForkchoiceState { + &self.fcs + } + + /// Update the fork choice state and issue an update to the engine. + pub async fn update_fcs( + &mut self, + head: Option, + safe: Option, + finalized: Option, + ) -> Result { + tracing::trace!(target: "scroll::engine", ?head, ?safe, ?finalized, current = ?self.fcs, "Updating fork choice state"); + if head.is_none() && safe.is_none() && finalized.is_none() { + return Err(EngineError::fcs_no_update_provided()); + } + + // clone the fcs before updating it + let mut fcs = self.fcs.clone(); + fcs.update(head, safe, finalized)?; + + // send the fcs update request to the engine + let result = self.client.fork_choice_updated_v1(fcs.get_alloy_fcs(), None).await?; + + // update the internal fcs state if the update was successful + // If the result is invalid, do not update the fcs + // If the result is valid or sync, update the fcs + if !result.is_invalid() { + self.fcs = fcs; + } + + Ok(result) + } + + /// Optimistically sync to the given block. + pub async fn optimistic_sync( + &mut self, + block: BlockInfo, + ) -> Result { + tracing::trace!(target: "scroll::engine", ?block, current = ?self.fcs, "Optimistically syncing to block"); + + // Update the fork choice state to the new block target + let mut fcs = self.fcs.clone(); + fcs.update(Some(block), None, None)?; + + // Send the optimistic sync request to the engine + let result = + self.client.fork_choice_updated_v1(fcs.get_alloy_optimistic_fcs(), None).await?; + + // update the internal fcs state if the update was successful + // If the result is invalid, do not update the fcs + // If the result is valid or sync, update the fcs + if !result.is_invalid() { + self.fcs = fcs; + } + + Ok(result) + } + + /// Submit a new payload to the engine. + pub async fn new_payload(&self, block: &ScrollBlock) -> Result { + tracing::trace!(target: "scroll::engine", block_number = block.number, block_hash = ?block.hash_slow(), "Submitting new payload to engine"); + let payload = ExecutionPayloadV1::from_block_slow(block); + let result = self.client.new_payload_v1(payload).await?; + Ok(result) + } + + /// Build a new payload with the given attributes. + pub async fn build_payload( + &self, + head: Option, + attributes: scroll_alloy_rpc_types_engine::ScrollPayloadAttributes, + ) -> Result { + tracing::trace!(target: "scroll::engine", ?attributes, "Building new payload with attributes"); + + let mut fcs = self.fcs.clone(); + if let Some(head) = head { + fcs.update(Some(head), None, None)?; + } + + let result = + self.client.fork_choice_updated_v1(fcs.get_alloy_fcs(), Some(attributes)).await?; + + Ok(result) + } + + /// Get a payload by its ID. + pub async fn get_payload( + &self, + payload_id: alloy_rpc_types_engine::PayloadId, + ) -> Result { + let payload = self.client.get_payload_v1(payload_id).await?; + Ok(payload) + } +} diff --git a/crates/engine/src/payload.rs b/crates/engine/src/payload.rs index bf8788d6..acc6b5a2 100644 --- a/crates/engine/src/payload.rs +++ b/crates/engine/src/payload.rs @@ -10,7 +10,7 @@ use tracing::debug; /// - timestamps are equal. /// - `prev_randaos` are equal. /// - `block_data_hint` matches the block data if present. -pub(crate) fn block_matches_attributes( +pub fn block_matches_attributes( attributes: &ScrollPayloadAttributes, block: &B, parent_hash: B256, diff --git a/crates/manager/src/lib.rs b/crates/manager/src/lib.rs index 8e2903aa..f02c2a2a 100644 --- a/crates/manager/src/lib.rs +++ b/crates/manager/src/lib.rs @@ -3,7 +3,7 @@ pub use consensus::{Consensus, NoopConsensus, SystemContractConsensus}; mod consensus; -mod manager; -pub use manager::{ - RollupManagerCommand, RollupManagerEvent, RollupManagerHandle, RollupNodeManager, -}; +// mod manager; +// pub use manager::{ +// RollupManagerCommand, RollupManagerEvent, RollupManagerHandle, RollupNodeManager, +// }; diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 6e033ea9..5d5cd964 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -25,7 +25,7 @@ use scroll_alloy_network::Scroll; use scroll_alloy_provider::ScrollEngineApi; use scroll_engine::{EngineDriver, EngineDriverEvent, ForkchoiceState}; use scroll_network::{ - BlockImportOutcome, NetworkManagerEvent, NewBlockWithPeer, ScrollNetworkManager, + BlockImportOutcome, NewBlockWithPeer, ScrollNetworkManager, ScrollNetworkManagerEvent, }; use std::{ fmt::{self, Debug, Formatter}, @@ -98,20 +98,8 @@ pub struct RollupNodeManager< > { /// The handle receiver used to receive commands. handle_rx: Receiver>, - /// The chain spec used by the rollup node. - chain_spec: Arc, - /// The network manager that manages the scroll p2p network. - network: ScrollNetworkManager, - /// The engine driver used to communicate with the engine. - engine: EngineDriver, - /// The derivation pipeline, used to derive payload attributes from batches. - derivation_pipeline: DerivationPipeline, - /// A receiver for [`L1Notification`]s from the [`rollup_node_watcher::L1Watcher`]. - l1_notification_rx: Option>>, /// The chain orchestrator. chain: ChainOrchestrator::Client, P>, - /// The consensus algorithm used by the rollup node. - consensus: Box, /// An event sender for sending events to subscribers of the rollup node manager. event_sender: Option>, /// The sequencer which is responsible for sequencing transactions and producing new blocks. @@ -148,7 +136,6 @@ impl< { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { f.debug_struct("RollupNodeManager") - .field("chain_spec", &self.chain_spec) .field("network", &self.network) .field("engine", &self.engine) .field("derivation_pipeline", &self.derivation_pipeline) @@ -183,7 +170,6 @@ where database: Arc, l1_notification_rx: Option>>, consensus: Box, - chain_spec: Arc, sequencer: Option>, signer: Option, block_time: Option, @@ -196,13 +182,7 @@ where DerivationPipeline::new(l1_provider, database.clone(), l1_v2_message_queue_start_index); let rnm = Self { handle_rx, - chain_spec, - network, - engine, - derivation_pipeline, - l1_notification_rx: l1_notification_rx.map(Into::into), chain: chain_orchestrator, - consensus, event_sender: None, sequencer, signer, @@ -290,9 +270,9 @@ where /// Handles a network manager event. /// /// Currently the network manager only emits a `NewBlock` event. - fn handle_network_manager_event(&mut self, event: NetworkManagerEvent) { + fn handle_network_manager_event(&mut self, event: ScrollNetworkManagerEvent) { match event { - NetworkManagerEvent::NewBlock(block) => self.handle_new_block(block), + ScrollNetworkManagerEvent::NewBlock(block) => self.handle_new_block(block), } } @@ -525,7 +505,8 @@ where sequencer.handle_new_l1_block(new_block) } } - _ => self.chain.handle_l1_notification(notification), + // _ => self.chain.handle_l1_notification(notification), + _ => (), } } diff --git a/crates/network/src/event.rs b/crates/network/src/event.rs index 20695b66..52c73489 100644 --- a/crates/network/src/event.rs +++ b/crates/network/src/event.rs @@ -11,7 +11,7 @@ pub struct NewBlockWithPeer { } /// An event that is emitted by the network manager to its subscribers. -#[derive(Debug)] -pub enum NetworkManagerEvent { +#[derive(Debug, Clone)] +pub enum ScrollNetworkManagerEvent { NewBlock(NewBlockWithPeer), } diff --git a/crates/network/src/handle.rs b/crates/network/src/handle.rs index 705c3fe7..8f988355 100644 --- a/crates/network/src/handle.rs +++ b/crates/network/src/handle.rs @@ -1,6 +1,9 @@ +use super::ScrollNetwork; +use crate::ScrollNetworkManagerEvent; use alloy_primitives::Signature; use reth_network_api::FullNetwork; use reth_scroll_primitives::ScrollBlock; +use reth_tokio_util::EventStream; use std::sync::Arc; use tokio::sync::{mpsc::UnboundedSender, oneshot}; @@ -12,8 +15,7 @@ pub struct ScrollNetworkHandle { } impl ScrollNetworkHandle { - /// Creates a new [`ScrollNetworkHandle`] instance from the given [`UnboundedSender`] and - /// [`FullNetwork`]. + /// Creates a new [`ScrollNetworkHandle`] instance. pub fn new( to_manager_tx: UnboundedSender, inner_network_handle: N, @@ -21,6 +23,11 @@ impl ScrollNetworkHandle { let inner = NetworkInner { to_manager_tx, inner_network_handle }; Self { inner: Arc::new(inner) } } + + pub async fn into_scroll_network(self) -> ScrollNetwork { + let events = self.event_listener().await; + ScrollNetwork { handle: self, events } + } } /// The inner state of the [`ScrollNetworkHandle`]. @@ -63,6 +70,12 @@ impl ScrollNetworkHandle { pub fn local_node_record(&self) -> reth_network_peers::NodeRecord { self.inner.inner_network_handle.local_node_record() } + + pub async fn event_listener(&self) -> EventStream { + let (tx, rx) = oneshot::channel(); + self.send_message(NetworkHandleMessage::EventListener(tx)); + rx.await.expect("network manager dropped") + } } /// A message type used for communication between the [`ScrollNetworkHandle`] and the @@ -72,4 +85,5 @@ pub enum NetworkHandleMessage { AnnounceBlock { block: ScrollBlock, signature: Signature }, BlockImportOutcome(super::BlockImportOutcome), Shutdown(oneshot::Sender<()>), + EventListener(oneshot::Sender>), } diff --git a/crates/network/src/lib.rs b/crates/network/src/lib.rs index 9f4bb173..9ece171e 100644 --- a/crates/network/src/lib.rs +++ b/crates/network/src/lib.rs @@ -1,5 +1,5 @@ mod event; -pub use event::{NetworkManagerEvent, NewBlockWithPeer}; +pub use event::{NewBlockWithPeer, ScrollNetworkManagerEvent}; mod handle; pub use handle::{NetworkHandleMessage, ScrollNetworkHandle}; @@ -15,4 +15,26 @@ pub use manager::ScrollNetworkManager; pub use reth_network::{EthNetworkPrimitives, NetworkConfigBuilder}; pub use reth_scroll_chainspec::SCROLL_MAINNET; +use reth_tokio_util::EventStream; pub use scroll_wire::ScrollWireConfig; + +/// The main network struct that encapsulates the network handle and event stream. +#[derive(Debug)] +pub struct ScrollNetwork { + /// The network handle to interact with the network manager. + handle: ScrollNetworkHandle, + /// Event stream for network manager events. + events: EventStream, +} + +impl ScrollNetwork { + /// Creates a new instance of `ScrollNetwork`. + pub fn handle(&self) -> &ScrollNetworkHandle { + &self.handle + } + + /// Returns a mutable reference to the event stream. + pub fn events(&mut self) -> &mut EventStream { + &mut self.events + } +} diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index 970bcb68..28a1d203 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -1,11 +1,11 @@ use crate::{BlockImportError, BlockValidationError}; use super::{ - BlockImportOutcome, BlockValidation, NetworkHandleMessage, NetworkManagerEvent, - NewBlockWithPeer, ScrollNetworkHandle, + BlockImportOutcome, BlockValidation, NetworkHandleMessage, NewBlockWithPeer, + ScrollNetworkHandle, ScrollNetworkManagerEvent, }; use alloy_primitives::{Address, FixedBytes, Signature, B256, U128}; -use futures::{FutureExt, Stream, StreamExt}; +use futures::{Future, FutureExt, StreamExt}; use reth_chainspec::EthChainSpec; use reth_eth_wire_types::NewBlock as EthWireNewBlock; use reth_network::{ @@ -16,7 +16,7 @@ use reth_network_api::{block::NewBlockWithPeer as RethNewBlockWithPeer, FullNetw use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; use reth_storage_api::BlockNumReader as BlockNumReaderT; -use reth_tokio_util::EventStream; +use reth_tokio_util::{EventSender, EventStream}; use rollup_node_primitives::sig_encode_hash; use scroll_alloy_hardforks::ScrollHardforks; use scroll_wire::{ @@ -35,6 +35,8 @@ use tracing::trace; /// The size of the ECDSA signature in bytes. const ECDSA_SIGNATURE_LEN: usize = 65; +const EVENT_CHANNEL_SIZE: usize = 5000; + /// [`ScrollNetworkManager`] manages the state of the scroll p2p network. /// /// This manager drives the state of the entire network forward and includes the following @@ -47,8 +49,8 @@ const ECDSA_SIGNATURE_LEN: usize = 65; pub struct ScrollNetworkManager { /// The chain spec used by the rollup node. chain_spec: Arc, - /// A handle used to interact with the network manager. - handle: ScrollNetworkHandle, + /// The inner network handle which is used to communicate with the inner network. + inner_network_handle: N, /// Receiver half of the channel set up between this type and the [`FullNetwork`], receives /// [`NetworkHandleMessage`]s. from_handle_rx: UnboundedReceiverStream, @@ -62,6 +64,8 @@ pub struct ScrollNetworkManager { td_constant: U128, /// The authorized signer for the network. authorized_signer: Option

, + /// The event sender for network events. + event_sender: EventSender, } impl @@ -76,7 +80,7 @@ impl eth_wire_listener: Option>>, td_constant: U128, authorized_signer: Option
, - ) -> Self { + ) -> (Self, ScrollNetworkHandle>) { // Create the scroll-wire protocol handler. let (scroll_wire_handler, events) = ScrollWireProtocolHandler::new(scroll_wire_config); @@ -91,7 +95,8 @@ impl // Create the channel for sending messages to the network manager. let (to_manager_tx, from_handle_rx) = mpsc::unbounded_channel(); - let handle = ScrollNetworkHandle::new(to_manager_tx, inner_network_handle); + let handle = ScrollNetworkHandle::new(to_manager_tx, inner_network_handle.clone()); + let event_sender = EventSender::new(EVENT_CHANNEL_SIZE); // Create the scroll-wire protocol manager. let scroll_wire = ScrollWireManager::new(events); @@ -101,16 +106,20 @@ impl // Spawn the inner network manager. tokio::spawn(inner_network_manager); - Self { - chain_spec, + ( + Self { + chain_spec, + inner_network_handle, + from_handle_rx: from_handle_rx.into(), + scroll_wire, + blocks_seen, + eth_wire_listener, + td_constant, + authorized_signer, + event_sender, + }, handle, - from_handle_rx: from_handle_rx.into(), - scroll_wire, - blocks_seen, - eth_wire_listener, - td_constant, - authorized_signer, - } + ) } } @@ -130,37 +139,32 @@ impl< eth_wire_listener: Option>>, td_constant: U128, authorized_signer: Option
, - ) -> Self { + ) -> (Self, ScrollNetworkHandle) { // Create the channel for sending messages to the network manager from the network handle. let (to_manager_tx, from_handle_rx) = mpsc::unbounded_channel(); // Create the scroll-wire protocol manager. let scroll_wire = ScrollWireManager::new(events); - let handle = ScrollNetworkHandle::new(to_manager_tx, inner_network_handle); + let handle = ScrollNetworkHandle::new(to_manager_tx, inner_network_handle.clone()); + let event_sender = EventSender::new(EVENT_CHANNEL_SIZE); let blocks_seen = LruCache::new(LRU_CACHE_SIZE); - Self { - chain_spec, + ( + Self { + chain_spec, + inner_network_handle, + from_handle_rx: from_handle_rx.into(), + scroll_wire, + blocks_seen, + eth_wire_listener, + td_constant, + authorized_signer, + event_sender, + }, handle, - from_handle_rx: from_handle_rx.into(), - scroll_wire, - blocks_seen, - eth_wire_listener, - td_constant, - authorized_signer, - } - } - - /// Returns a new [`ScrollNetworkHandle`] instance. - pub fn handle(&self) -> &ScrollNetworkHandle { - &self.handle - } - - /// Returns an inner network handle [`RethNetworkHandle`]. - pub fn inner_network_handle(&self) -> &N { - self.handle.inner() + ) } /// Announces a new block to the network. @@ -205,7 +209,7 @@ impl< eth_wire_block.header.extra_data = block.signature.clone().into(); EthWireNewBlock { block: eth_wire_block, td } }; - self.inner_network_handle().eth_wire_announce_block(eth_wire_new_block, hash); + self.inner_network_handle.eth_wire_announce_block(eth_wire_new_block, hash); } // Announce block to the filtered set of peers @@ -216,7 +220,10 @@ impl< } /// Handler for received events from the [`ScrollWireManager`]. - fn on_scroll_wire_event(&mut self, event: ScrollWireEvent) -> Option { + fn on_scroll_wire_event( + &mut self, + event: ScrollWireEvent, + ) -> Option { match event { ScrollWireEvent::NewBlock { peer_id, block, signature } => { let block_hash = block.hash_slow(); @@ -233,7 +240,7 @@ impl< // Update the state of the block cache i.e. we have seen this block. self.blocks_seen.insert((block.hash_slow(), signature)); - Some(NetworkManagerEvent::NewBlock(NewBlockWithPeer { + Some(ScrollNetworkManagerEvent::NewBlock(NewBlockWithPeer { peer_id, block, signature, @@ -261,6 +268,9 @@ impl< // self.perform_network_shutdown().await; // let _ = tx.send(()); } + NetworkHandleMessage::EventListener(tx) => { + let _ = tx.send(self.event_sender.new_listener()); + } } } @@ -275,12 +285,12 @@ impl< } Err(BlockImportError::Consensus(err)) => { trace!(target: "scroll::network::manager", peer_id = ?peer, ?err, "Block import failed - consensus error - penalizing peer"); - self.inner_network_handle() + self.inner_network_handle .reputation_change(peer, reth_network_api::ReputationChangeKind::BadBlock); } Err(BlockImportError::Validation(BlockValidationError::InvalidBlock)) => { trace!(target: "scroll::network::manager", peer_id = ?peer, "Block import failed - invalid block - penalizing peer"); - self.inner_network_handle() + self.inner_network_handle .reputation_change(peer, reth_network_api::ReputationChangeKind::BadBlock); } } @@ -290,7 +300,7 @@ impl< fn handle_eth_wire_block( &mut self, block: reth_network_api::block::NewBlockWithPeer, - ) -> Option { + ) -> Option { let reth_network_api::block::NewBlockWithPeer { peer_id, mut block } = block; // We purge the extra data field post euclid v2 to align with protocol specification. @@ -326,10 +336,14 @@ impl< // Update the state of the block cache i.e. we have seen this block. self.blocks_seen.insert((block_hash, signature)); - Some(NetworkManagerEvent::NewBlock(NewBlockWithPeer { peer_id, block, signature })) + Some(ScrollNetworkManagerEvent::NewBlock(NewBlockWithPeer { + peer_id, + block, + signature, + })) } else { tracing::warn!(target: "scroll::bridge::import", peer_id = %peer_id, "Failed to extract signature from block extra data, penalizing peer"); - self.inner_network_handle() + self.inner_network_handle .reputation_change(peer_id, reth_network_api::ReputationChangeKind::BadBlock); None } @@ -339,11 +353,11 @@ impl< impl< N: FullNetwork, CS: ScrollHardforks + EthChainSpec + Send + Sync + 'static, - > Stream for ScrollNetworkManager + > Future for ScrollNetworkManager { - type Item = NetworkManagerEvent; + type Output = (); - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); // We handle the messages from the network handle. @@ -355,7 +369,7 @@ impl< } // All network handles have been dropped so we can shutdown the network. Poll::Ready(None) => { - return Poll::Ready(None); + return Poll::Ready(()); } // No additional messages exist break. Poll::Pending => break, @@ -363,9 +377,9 @@ impl< } // Next we handle the scroll-wire events. - if let Poll::Ready(event) = this.scroll_wire.poll_unpin(cx) { + while let Poll::Ready(event) = this.scroll_wire.poll_unpin(cx) { if let Some(event) = this.on_scroll_wire_event(event) { - return Poll::Ready(Some(event)); + this.event_sender.notify(event); } } @@ -374,7 +388,7 @@ impl< this.eth_wire_listener.as_mut().map(|new_block_rx| new_block_rx.poll_next_unpin(cx)) { if let Some(event) = this.handle_eth_wire_block(block) { - return Poll::Ready(Some(event)); + this.event_sender.notify(event); } } diff --git a/crates/node/src/add_ons/handle.rs b/crates/node/src/add_ons/handle.rs index 757e7931..a293e6b5 100644 --- a/crates/node/src/add_ons/handle.rs +++ b/crates/node/src/add_ons/handle.rs @@ -3,7 +3,7 @@ use reth_node_api::FullNodeComponents; use reth_node_builder::rpc::{RpcHandle, RpcHandleProvider}; use reth_rpc_eth_api::EthApiTypes; use reth_scroll_node::ScrollNetworkPrimitives; -use rollup_node_manager::RollupManagerHandle; +use rollup_node_chain_orchestrator::ChainOrchestratorHandle; #[cfg(feature = "test-utils")] use {rollup_node_watcher::L1Notification, std::sync::Arc, tokio::sync::mpsc::Sender}; @@ -14,7 +14,7 @@ pub struct ScrollAddOnsHandle< EthApi: EthApiTypes, > { /// The handle used to send commands to the rollup manager. - pub rollup_manager_handle: RollupManagerHandle, + pub rollup_manager_handle: ChainOrchestratorHandle, /// The handle used to send commands to the RPC server. pub rpc_handle: RpcHandle, /// An optional channel used to send `L1Watcher` notifications to the `RollupNodeManager`. diff --git a/crates/node/src/add_ons/rollup.rs b/crates/node/src/add_ons/rollup.rs index f60553f9..851aef40 100644 --- a/crates/node/src/add_ons/rollup.rs +++ b/crates/node/src/add_ons/rollup.rs @@ -8,7 +8,7 @@ use reth_node_builder::{rpc::RpcHandle, AddOnsContext, FullNodeComponents}; use reth_rpc_eth_api::EthApiTypes; use reth_scroll_chainspec::{ChainConfig, ScrollChainConfig, ScrollChainSpec}; use reth_scroll_node::ScrollNetworkPrimitives; -use rollup_node_manager::RollupManagerHandle; +use rollup_node_chain_orchestrator::ChainOrchestratorHandle; use rollup_node_watcher::L1Notification; use scroll_alloy_hardforks::ScrollHardforks; use scroll_wire::ScrollWireEvent; @@ -55,7 +55,7 @@ impl RollupManagerAddOn { self, ctx: AddOnsContext<'_, N>, rpc: RpcHandle, - ) -> eyre::Result<(RollupManagerHandle, Option>>)> + ) -> eyre::Result<(ChainOrchestratorHandle, Option>>)> where <::Types as NodeTypes>::ChainSpec: ChainConfig + ScrollHardforks + IsDevChain, diff --git a/crates/node/src/add_ons/rpc.rs b/crates/node/src/add_ons/rpc.rs index b253c7fc..4260b660 100644 --- a/crates/node/src/add_ons/rpc.rs +++ b/crates/node/src/add_ons/rpc.rs @@ -6,7 +6,7 @@ use jsonrpsee::{ }; use reth_network_api::FullNetwork; use reth_scroll_node::ScrollNetworkPrimitives; -use rollup_node_manager::RollupManagerHandle; +use rollup_node_chain_orchestrator::ChainOrchestratorHandle; use tokio::sync::{oneshot, Mutex, OnceCell}; /// RPC extension for rollup node management operations. @@ -20,9 +20,9 @@ where N: FullNetwork, { /// Cached rollup manager handle, initialized lazily via `OnceCell` - handle: tokio::sync::OnceCell>, + handle: tokio::sync::OnceCell>, /// Oneshot channel receiver for obtaining the rollup manager handle during initialization - rx: Mutex>>>, + rx: Mutex>>>, } impl RollupNodeRpcExt @@ -30,7 +30,7 @@ where N: FullNetwork, { /// Creates a new RPC extension with a receiver for the rollup manager handle. - pub fn new(rx: oneshot::Receiver>) -> Self { + pub fn new(rx: oneshot::Receiver>) -> Self { Self { rx: Mutex::new(Some(rx)), handle: OnceCell::new() } } @@ -38,7 +38,7 @@ where /// /// This method lazily initializes the rollup manager handle by consuming the oneshot /// receiver. Subsequent calls will return the cached handle. - async fn rollup_manager_handle(&self) -> eyre::Result<&RollupManagerHandle> { + async fn rollup_manager_handle(&self) -> eyre::Result<&ChainOrchestratorHandle> { self.handle .get_or_try_init(|| async { let rx = { diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index ab386b92..b83f81f0 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -25,15 +25,20 @@ use reth_scroll_chainspec::{ ChainConfig, ScrollChainConfig, ScrollChainSpec, SCROLL_FEE_VAULT_ADDRESS, }; use reth_scroll_node::ScrollNetworkPrimitives; -use rollup_node_chain_orchestrator::ChainOrchestrator; -use rollup_node_manager::{ - Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, +use rollup_node_chain_orchestrator::{ + ChainOrchestrator, ChainOrchestratorConfig, ChainOrchestratorHandle, Consensus, NoopConsensus, + SystemContractConsensus, }; +// use rollup_node_manager::{ +// Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, +// }; use rollup_node_primitives::{BlockInfo, NodeConfig}; use rollup_node_providers::{ BlobProvidersBuilder, FullL1Provider, L1MessageProvider, L1Provider, SystemContractProvider, }; -use rollup_node_sequencer::{L1MessageInclusionMode, Sequencer}; +use rollup_node_sequencer::{ + L1MessageInclusionMode, PayloadBuildingConfig, Sequencer, SequencerConfig, +}; use rollup_node_watcher::{L1Notification, L1Watcher}; use scroll_alloy_hardforks::ScrollHardforks; use scroll_alloy_network::Scroll; @@ -42,7 +47,8 @@ use scroll_db::{ Database, DatabaseConnectionProvider, DatabaseReadOperations, DatabaseTransactionProvider, DatabaseWriteOperations, }; -use scroll_engine::{genesis_hash_from_chain_spec, EngineDriver, ForkchoiceState}; +use scroll_derivation_pipeline::DerivationPipelineNew; +use scroll_engine::{genesis_hash_from_chain_spec, Engine, EngineDriver, ForkchoiceState}; use scroll_migration::traits::ScrollMigrator; use scroll_network::ScrollNetworkManager; use scroll_wire::ScrollWireEvent; @@ -153,15 +159,14 @@ impl ScrollRollupNodeConfig { events: UnboundedReceiver, rpc_server_handles: RethRpcServerHandles, ) -> eyre::Result<( - RollupNodeManager< + ChainOrchestrator< N, - impl ScrollEngineApi, - impl Provider + Clone, - impl L1Provider + Clone, - impl L1MessageProvider, impl ScrollHardforks + EthChainSpec + IsDevChain + Clone + 'static, + impl L1Provider + Clone, + impl Provider + Clone, + impl ScrollEngineApi, >, - RollupManagerHandle, + ChainOrchestratorHandle, Option>>, )> where @@ -268,7 +273,7 @@ impl ScrollRollupNodeConfig { // TODO: remove this once we deprecate l2geth. let authorized_signer = self.network_args.effective_signer(chain_spec.chain().named()); - let scroll_network_manager = ScrollNetworkManager::from_parts( + let (scroll_network_manager, scroll_network_handle) = ScrollNetworkManager::from_parts( chain_spec.clone(), ctx.network.clone(), events, @@ -285,24 +290,26 @@ impl ScrollRollupNodeConfig { tx.prepare_on_startup(chain_spec.genesis_hash()).await?; tx.commit().await?; if let Some(block_info) = startup_safe_block { - fcs.update_safe_block_info(block_info); + fcs.update_safe_block_info(block_info).expect("failed to set safe block info"); } else { fcs.update_safe_block_info(BlockInfo { hash: genesis_hash_from_chain_spec(chain_spec.clone()).unwrap(), number: 0, - }); + }) + .expect("failed to set safe block info to genesis"); } tracing::info!(target: "scroll::node::args", fcs = ?fcs, payload_building_duration = ?self.sequencer_args.payload_building_duration, "Starting engine driver"); - let engine = EngineDriver::new( - Arc::new(engine_api), - chain_spec.clone(), - Some(l2_provider.clone()), - fcs, - self.engine_driver_args.sync_at_startup && !self.test && !chain_spec.is_dev_chain(), - Duration::from_millis(self.sequencer_args.payload_building_duration), - self.sequencer_args.allow_empty_blocks, - ); + let engine = Engine::new(Arc::new(engine_api), fcs); + // let engine = EngineDriver::new( + // Arc::new(engine_api), + // chain_spec.clone(), + // Some(l2_provider.clone()), + // fcs, + // self.engine_driver_args.sync_at_startup && !self.test && !chain_spec.is_dev_chain(), + // Duration::from_millis(self.sequencer_args.payload_building_duration), + // self.sequencer_args.allow_empty_blocks, + // ); // Create the consensus. let authorized_signer = if let Some(provider) = l1_provider.as_ref() { @@ -352,22 +359,28 @@ impl ScrollRollupNodeConfig { let latest_l1_message = db.tx().await?.get_latest_executed_l1_message().await?; let sequencer_l1_messages_queue_index = latest_l1_message.map(|msg| msg.transaction.queue_index + 1).unwrap_or_default(); - let (sequencer, block_time, auto_start) = if self.sequencer_args.sequencer_enabled { + let sequencer = if self.sequencer_args.sequencer_enabled { let args = &self.sequencer_args; - let sequencer = Sequencer::new( - Arc::new(l1_messages_provider), - args.fee_recipient, - ctx.block_gas_limit, - self.sequencer_args - .max_l1_messages - .unwrap_or(chain_config.l1_config.num_l1_messages_per_block), - 0, - self.sequencer_args.l1_message_inclusion_mode, - sequencer_l1_messages_queue_index, - ); - (Some(sequencer), (args.block_time != 0).then_some(args.block_time), args.auto_start) + let config = SequencerConfig { + chain_spec: chain_spec.clone(), + fee_recipient: args.fee_recipient, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: ctx.block_gas_limit, + max_l1_messages_per_block: self + .sequencer_args + .max_l1_messages + .unwrap_or(chain_config.l1_config.num_l1_messages_per_block), + l1_message_inclusion_mode: args.l1_message_inclusion_mode, + }, + auto_start: args.auto_start, + block_time: args.block_time, + allow_empty_blocks: args.allow_empty_blocks, + payload_building_duration: args.payload_building_duration, + }; + let sequencer = Sequencer::new(Arc::new(l1_messages_provider), config); + Some(sequencer) } else { - (None, None, false) + None }; // Instantiate the signer @@ -383,43 +396,40 @@ impl ScrollRollupNodeConfig { }; // Instantiate the chain orchestrator - let block_client = scroll_network_manager - .handle() + let block_client = scroll_network_handle .inner() .fetch_client() .await .expect("failed to fetch block client"); let l1_v2_message_queue_start_index = l1_v2_message_queue_start_index(chain_spec.chain().named()); - let chain_orchestrator = ChainOrchestrator::new( + let config: ChainOrchestratorConfig> = ChainOrchestratorConfig::new( + chain_spec, + self.chain_orchestrator_args.optimistic_sync_trigger, + l1_v2_message_queue_start_index, + ); + + // Instantiate the derivation pipeline + let derivation_pipeline = DerivationPipelineNew::new( + l1_provider.clone(), db.clone(), - chain_spec.clone(), + l1_v2_message_queue_start_index, + ); + + let (chain_orchestrator, handle) = ChainOrchestrator::new( + db.clone(), + config, block_client, l2_provider, - self.chain_orchestrator_args.optimistic_sync_trigger, - self.chain_orchestrator_args.chain_buffer_size, - l1_v2_message_queue_start_index, + l1_notification_rx.expect("L1 notification receiver should be set"), + scroll_network_handle.into_scroll_network().await, + consensus, + engine, + derivation_pipeline, ) .await?; - // Spawn the rollup node manager - let (rnm, handle) = RollupNodeManager::new( - scroll_network_manager, - engine, - l1_provider, - db, - l1_notification_rx, - consensus, - chain_spec, - sequencer, - signer, - block_time, - auto_start, - chain_orchestrator, - l1_v2_message_queue_start_index, - ) - .await; - Ok((rnm, handle, l1_notification_tx)) + Ok((chain_orchestrator, handle, l1_notification_tx)) } } diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index e0651894..15859629 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -29,8 +29,7 @@ use rollup_node::{ RollupNodeGasPriceOracleArgs, RollupNodeNetworkArgs as ScrollNetworkArgs, RpcArgs, ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, }; -use rollup_node_chain_orchestrator::ChainOrchestratorEvent; -use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent}; +use rollup_node_chain_orchestrator::{ChainOrchestratorEvent, ChainOrchestratorHandle}; use rollup_node_primitives::{sig_encode_hash, BatchCommitData, ConsensusUpdate}; use rollup_node_sequencer::L1MessageInclusionMode; use rollup_node_watcher::L1Notification; @@ -79,8 +78,8 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { let (mut nodes, _tasks, _wallet) = setup_engine(node_args, 1, chain_spec, false, false).await?; let node = nodes.pop().unwrap(); - let rnm_handle = node.inner.add_ons_handle.rollup_manager_handle.clone(); - let mut rnm_events = rnm_handle.get_event_listener().await?; + let chain_orchestrator = node.inner.add_ons_handle.rollup_manager_handle.clone(); + let mut events = chain_orchestrator.get_event_listener().await?; let l1_watcher_tx = node.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let l1_message = TxL1Message { @@ -100,12 +99,9 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { .await?; wait_n_events( - &mut rnm_events, + &mut events, |e| { - if let RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(index), - ) = e - { + if let ChainOrchestratorEvent::L1MessageCommitted(index) = e { assert_eq!(index, 0); true } else { @@ -116,10 +112,10 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { ) .await; - rnm_handle.build_block().await; + chain_orchestrator.build_block().await; wait_n_events( - &mut rnm_events, + &mut events, |e| { if let RollupManagerEvent::BlockSequenced(block) = e { assert_eq!(block.body.transactions.len(), 1); @@ -796,7 +792,7 @@ async fn can_bridge_blocks() { network_handle.announce_block(new_block_1, block_1_hash); // Assert block received from the bridge node on the scroll wire protocol is correct - if let Some(scroll_network::NetworkManagerEvent::NewBlock(NewBlockWithPeer { + if let Some(scroll_network::ScrollNetworkManagerEvent::NewBlock(NewBlockWithPeer { peer_id, block, signature, diff --git a/crates/primitives/src/batch.rs b/crates/primitives/src/batch.rs index 9b958ac6..a1c37c50 100644 --- a/crates/primitives/src/batch.rs +++ b/crates/primitives/src/batch.rs @@ -1,5 +1,6 @@ use std::sync::Arc; +use super::{BlockInfo, L2BlockInfoWithL1Messages}; use alloy_primitives::{Bytes, B256}; /// The batch information. @@ -46,6 +47,36 @@ impl From for BatchInfo { } } +/// The outcome of consolidating a batch with the L2 chain. +#[derive(Debug)] +pub struct BatchConsolidationOutcome { + /// The batch info for the consolidated batch. + pub batch_info: BatchInfo, + /// The consolidation outcomes for each block in the batch. + pub blocks: Vec, +} + +impl BatchConsolidationOutcome { + /// Creates a new empty batch consolidation outcome for the given batch info. + pub fn new(batch_info: BatchInfo) -> Self { + Self { batch_info, blocks: Vec::new() } + } + + /// Pushes a block consolidation outcome to the batch. + pub fn push_block(&mut self, block: BlockConsolidationOutcome) { + self.blocks.push(block); + } +} + +/// The outcome of consolidating a block with the L2 chain. +#[derive(Debug)] +pub enum BlockConsolidationOutcome { + /// The derived block was already part of the chain, no action needed. + Consolidated(BlockInfo), + /// The derived block resulted in a reorg of the L2 chain. + Reorged(L2BlockInfoWithL1Messages), +} + #[cfg(feature = "arbitrary")] mod arbitrary_impl { use super::*; diff --git a/crates/primitives/src/lib.rs b/crates/primitives/src/lib.rs index 137b368d..23c79fbb 100644 --- a/crates/primitives/src/lib.rs +++ b/crates/primitives/src/lib.rs @@ -14,7 +14,7 @@ pub use block::{ }; mod batch; -pub use batch::{BatchCommitData, BatchInfo}; +pub use batch::{BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome}; mod bounded_vec; pub use bounded_vec::BoundedVec; diff --git a/crates/providers/src/l1/message.rs b/crates/providers/src/l1/message.rs index 2136ac53..88b1a0a9 100644 --- a/crates/providers/src/l1/message.rs +++ b/crates/providers/src/l1/message.rs @@ -42,8 +42,11 @@ where n: u64, ) -> Result, Self::Error> { let tx = self.tx().await?; - let messages = - tx.get_l1_messages(Some(start)).await?.take(n as usize).try_collect().await?; + let messages = if let Some(stream) = tx.get_l1_messages(Some(start)).await? { + stream.take(n as usize).try_collect().await? + } else { + vec![] + }; Ok(messages) } } diff --git a/crates/sequencer/Cargo.toml b/crates/sequencer/Cargo.toml index 49c6cdc4..6f5c7b02 100644 --- a/crates/sequencer/Cargo.toml +++ b/crates/sequencer/Cargo.toml @@ -16,15 +16,18 @@ alloy-primitives.workspace = true alloy-rpc-types-engine.workspace = true # scroll reth +reth-scroll-engine-primitives.workspace = true reth-scroll-primitives.workspace = true # scroll-alloy scroll-alloy-consensus.workspace = true +scroll-alloy-hardforks.workspace = true scroll-alloy-provider.workspace = true scroll-alloy-rpc-types-engine.workspace = true # rollup-node scroll-db = { workspace = true } +scroll-engine.workspace = true rollup-node-providers.workspace = true rollup-node-primitives.workspace = true @@ -59,6 +62,7 @@ reth-scroll-node = { workspace = true, features = ["test-utils"] } # rollup-node rollup-node = { workspace = true, features = ["test-utils"] } +rollup-node-chain-orchestrator.workspace = true rollup-node-primitives.workspace = true rollup-node-providers = { workspace = true, features = ["test-utils"] } rollup-node-sequencer.workspace = true diff --git a/crates/sequencer/src/config.rs b/crates/sequencer/src/config.rs new file mode 100644 index 00000000..79c95352 --- /dev/null +++ b/crates/sequencer/src/config.rs @@ -0,0 +1,81 @@ +use alloy_primitives::Address; +use scroll_db::{L1MessageStart, NotIncludedStart}; +use std::{fmt, str::FromStr, sync::Arc}; + +/// Configuration for the sequencer. +#[derive(Debug)] +pub struct SequencerConfig { + /// The chain spec. + pub chain_spec: Arc, + /// The fee recipient. + pub fee_recipient: Address, + /// Whether the sequencer should start automatically. + pub auto_start: bool, + /// The payload building config. + pub payload_building_config: PayloadBuildingConfig, + /// The block time in milliseconds. + pub block_time: u64, + /// The duration in seconds to build payload attributes. + pub payload_building_duration: u64, + /// Whether to allow empty blocks. + pub allow_empty_blocks: bool, +} + +/// Configuration for building payloads. +#[derive(Debug, Clone)] +pub struct PayloadBuildingConfig { + /// The block gas limit. + pub block_gas_limit: u64, + /// The number of L1 messages to include in each block. + pub max_l1_messages_per_block: u64, + /// The L1 message inclusion mode configuration. + pub l1_message_inclusion_mode: L1MessageInclusionMode, +} + +/// Configuration for L1 message inclusion strategy. +#[derive(Debug, Default, Clone, Copy)] +pub enum L1MessageInclusionMode { + /// Include L1 messages based on block depth. + BlockDepth(u64), + /// Include only finalized L1 messages. + #[default] + Finalized, +} + +impl FromStr for L1MessageInclusionMode { + type Err = String; + + fn from_str(s: &str) -> Result { + if s.eq_ignore_ascii_case("finalized") { + Ok(Self::Finalized) + } else if let Some(rest) = s.strip_prefix("depth:") { + rest.parse::() + .map(Self::BlockDepth) + .map_err(|_| format!("Expected a valid number after 'depth:', got '{rest}'")) + } else { + Err("Expected 'finalized' or 'depth:{number}' (e.g. 'depth:10')".to_string()) + } + } +} + +impl fmt::Display for L1MessageInclusionMode { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Finalized => write!(f, "finalized"), + Self::BlockDepth(depth) => write!(f, "depth:{depth}"), + } + } +} + +impl From for L1MessageStart { + fn from(mode: L1MessageInclusionMode) -> Self { + match mode { + L1MessageInclusionMode::Finalized => { + L1MessageStart::NotIncluded(NotIncludedStart::Finalized) + } + L1MessageInclusionMode::BlockDepth(depth) => { + L1MessageStart::NotIncluded(NotIncludedStart::BlockDepth(depth)) + } + } + } +} diff --git a/crates/sequencer/src/error.rs b/crates/sequencer/src/error.rs index d59953b3..ca68a567 100644 --- a/crates/sequencer/src/error.rs +++ b/crates/sequencer/src/error.rs @@ -1,5 +1,6 @@ use rollup_node_providers::L1ProviderError; use scroll_db::DatabaseError; +use scroll_engine::EngineError; // TODO: make the error types more fine grained. @@ -9,6 +10,15 @@ pub enum SequencerError { /// The sequencer encountered an error when interacting with the database. #[error("Encountered an error interacting with the database: {0}")] DatabaseError(#[from] DatabaseError), + /// The engine encountered an error. + #[error("Encountered an error interacting with the execution engine: {0}")] + EngineError(#[from] EngineError), + /// Missing payload id after requesting a new payload. + #[error("Missing payload id after requesting a new payload")] + MissingPayloadId, + /// The sequencer encountered an error when converting a payload into a scroll block. + #[error("Encountered an error converting a payload into a scroll block")] + PayloadError, /// The sequencer encountered an error when interacting with the L1 message provider. #[error("Encountered an error interacting with the L1 message provider: {0}")] L1MessageProviderError(#[from] L1ProviderError), diff --git a/crates/sequencer/src/event.rs b/crates/sequencer/src/event.rs new file mode 100644 index 00000000..8524e3a2 --- /dev/null +++ b/crates/sequencer/src/event.rs @@ -0,0 +1,11 @@ +use alloy_rpc_types_engine::PayloadId; +use reth_scroll_primitives::ScrollBlock; + +/// Events emitted by the sequencer. +#[derive(Debug, Clone)] +pub enum SequencerEvent { + /// A new slot has started. + NewSlot, + /// The payload with the given ID is ready to be retrieved from the execution node. + PayloadReady(PayloadId), +} diff --git a/crates/sequencer/src/lib.rs b/crates/sequencer/src/lib.rs index 58ef4979..4f4d963e 100644 --- a/crates/sequencer/src/lib.rs +++ b/crates/sequencer/src/lib.rs @@ -5,380 +5,280 @@ use std::{ fmt, future::Future, pin::Pin, - str::FromStr, sync::Arc, task::{Context, Poll}, time::{Instant, SystemTime, UNIX_EPOCH}, }; use alloy_eips::eip2718::Encodable2718; -use alloy_primitives::Address; -use alloy_rpc_types_engine::PayloadAttributes; +use alloy_rpc_types_engine::{ExecutionData, ExecutionPayloadV1, PayloadAttributes, PayloadId}; use futures::{task::AtomicWaker, Stream}; +use reth_scroll_engine_primitives::try_into_block; use reth_scroll_primitives::ScrollBlock; -use rollup_node_primitives::{L1MessageEnvelope, DEFAULT_BLOCK_DIFFICULTY}; +use rollup_node_primitives::{BlockInfo, DEFAULT_BLOCK_DIFFICULTY}; use rollup_node_providers::{L1MessageProvider, L1ProviderError}; -use scroll_alloy_consensus::ScrollTransaction; +use scroll_alloy_hardforks::ScrollHardforks; +use scroll_alloy_provider::ScrollEngineApi; use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; +use scroll_engine::Engine; +use tokio::time::Interval; + +mod config; +pub use config::{L1MessageInclusionMode, PayloadBuildingConfig, SequencerConfig}; mod error; pub use error::SequencerError; +mod event; +pub use event::SequencerEvent; + mod metrics; pub use metrics::SequencerMetrics; /// A type alias for the payload building job future. -pub type PayloadBuildingJobFuture = - Pin> + Send>>; - -/// Configuration for L1 message inclusion strategy. -#[derive(Debug, Default, Clone, Copy)] -pub enum L1MessageInclusionMode { - /// Include L1 messages based on block depth. - BlockDepth(u64), - /// Include only finalized L1 messages. - #[default] - Finalized, -} - -impl FromStr for L1MessageInclusionMode { - type Err = String; - - fn from_str(s: &str) -> Result { - if s.eq_ignore_ascii_case("finalized") { - Ok(Self::Finalized) - } else if let Some(rest) = s.strip_prefix("depth:") { - rest.parse::() - .map(Self::BlockDepth) - .map_err(|_| format!("Expected a valid number after 'depth:', got '{rest}'")) - } else { - Err("Expected 'finalized' or 'depth:{number}' (e.g. 'depth:10')".to_string()) - } - } -} - -impl fmt::Display for L1MessageInclusionMode { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Self::Finalized => write!(f, "finalized"), - Self::BlockDepth(depth) => write!(f, "depth:{depth}"), - } - } -} +pub type PayloadBuildingJobFuture = Pin + Send>>; /// The sequencer is responsible for sequencing transactions and producing new blocks. -pub struct Sequencer

{ - /// A reference to the database. +pub struct Sequencer { + /// A reference to the provider. provider: Arc

, - /// The fee recipient. - fee_recipient: Address, - /// The block gas limit. - block_gas_limit: u64, - /// The number of L1 messages to include in each block. - max_l1_messages_per_block: u64, - /// The current l1 block number. - l1_block_number: u64, - /// The L1 finalized block number. - l1_finalized_block_number: u64, - /// The L1 message inclusion mode configuration. - l1_message_inclusion_mode: L1MessageInclusionMode, - /// The inflight payload attributes job - payload_attributes_job: Option, - /// The current L1 messages queue index. - l1_messages_queue_index: u64, + /// The configuration for the sequencer. + config: SequencerConfig, + /// The interval trigger for building a new block. + trigger: Option, + /// The inflight payload building job + payload_building_job: Option, /// The sequencer metrics. metrics: SequencerMetrics, /// A waker to notify when the Sequencer should be polled. waker: AtomicWaker, } -impl

Sequencer

+impl Sequencer where P: L1MessageProvider + Unpin + Send + Sync + 'static, + CS: ScrollHardforks, { /// Creates a new sequencer. - pub fn new( - provider: Arc

, - fee_recipient: Address, - block_gas_limit: u64, - max_l1_messages_per_block: u64, - l1_block_number: u64, - l1_message_inclusion_mode: L1MessageInclusionMode, - l1_messages_queue_index: u64, - ) -> Self { + pub fn new(provider: Arc

, config: SequencerConfig) -> Self { Self { provider, - fee_recipient, - block_gas_limit, - max_l1_messages_per_block, - l1_block_number, - l1_finalized_block_number: 0, - l1_message_inclusion_mode, - l1_messages_queue_index, - payload_attributes_job: None, + trigger: config.auto_start.then(|| delayed_interval(config.block_time)), + config, + payload_building_job: None, metrics: SequencerMetrics::default(), waker: AtomicWaker::new(), } } - /// Set the L1 finalized block number. - pub fn set_l1_finalized_block_number(&mut self, l1_finalized_block_number: u64) { - self.l1_finalized_block_number = l1_finalized_block_number; + /// Returns a reference to the payload building job. + pub fn payload_building_job(&self) -> Option<&PayloadBuildingJob> { + self.payload_building_job.as_ref() + } + + /// Cancels the current payload building job, if any. + pub fn cancel_payload_building_job(&mut self) { + self.payload_building_job = None; + } + + /// Enables the sequencer. + pub fn enable(&mut self) { + if self.trigger.is_none() { + self.trigger = Some(delayed_interval(self.config.block_time)); + } + } + + /// Disables the sequencer. + pub fn disable(&mut self) { + self.trigger = None; } /// Creates a new block using the pending transactions from the message queue and /// the transaction pool. - pub fn build_payload_attributes(&mut self) { + pub async fn start_payload_building( + &mut self, + engine: &mut Engine, + ) -> Result<(), SequencerError> { tracing::info!(target: "rollup_node::sequencer", "New payload attributes request received."); - if self.payload_attributes_job.is_some() { - tracing::error!(target: "rollup_node::sequencer", "A payload attributes building job is already in progress"); - return; - } - let timestamp = SystemTime::now() .duration_since(UNIX_EPOCH) .expect("Time can't go backwards") .as_secs(); let payload_attributes = PayloadAttributes { timestamp, - suggested_fee_recipient: self.fee_recipient, + suggested_fee_recipient: self.config.fee_recipient, parent_beacon_block_root: None, prev_randao: Default::default(), withdrawals: None, }; - let max_l1_messages = self.max_l1_messages_per_block; - let database = self.provider.clone(); - let block_gas_limit = self.block_gas_limit; - let l1_block_number = self.l1_block_number; - let l1_finalized_block_number = self.l1_finalized_block_number; - let l1_message_inclusion_mode = self.l1_message_inclusion_mode; - let l1_messages_queue_index = self.l1_messages_queue_index; - let metrics = self.metrics.clone(); - - self.payload_attributes_job = Some(Box::pin(async move { - let now = Instant::now(); - let res = build_payload_attributes( - database, - max_l1_messages, - payload_attributes, - block_gas_limit, - l1_block_number, - l1_finalized_block_number, - l1_message_inclusion_mode, - l1_messages_queue_index, + + let now = Instant::now(); + let mut l1_messages = vec![]; + let mut cumulative_gas_used = 0; + + // Collect L1 messages to include in payload. + let db_l1_messages = self + .provider + .get_n_messages( + self.config.payload_building_config.l1_message_inclusion_mode.into(), + self.config.payload_building_config.max_l1_messages_per_block, ) - .await; - metrics.payload_attributes_building_duration.record(now.elapsed().as_secs_f64()); - res - })); + .await + .map_err(Into::::into)?; + + let l1_origin = db_l1_messages.first().map(|msg| msg.l1_block_number); + for msg in db_l1_messages { + // TODO (greg): we only check the DA limit on the execution node side. We should also + // check it here. + let fits_in_block = msg.transaction.gas_limit + cumulative_gas_used <= + self.config.payload_building_config.block_gas_limit; + if !fits_in_block { + break; + } + + cumulative_gas_used += msg.transaction.gas_limit; + l1_messages.push(msg.transaction.encoded_2718().into()); + } + + let payload_attributes = ScrollPayloadAttributes { + payload_attributes, + transactions: (!l1_messages.is_empty()).then_some(l1_messages), + no_tx_pool: false, + block_data_hint: BlockDataHint { + difficulty: Some(DEFAULT_BLOCK_DIFFICULTY), + ..Default::default() + }, + // If setting the gas limit to None, the Reth payload builder will use the gas limit + // passed via the `builder.gaslimit` CLI arg. + gas_limit: None, + }; + + self.metrics.payload_attributes_building_duration.record(now.elapsed().as_secs_f64()); + + // Request the engine to build a new payload. + let fcu = engine.build_payload(None, payload_attributes).await?; + let payload_id = fcu.payload_id.ok_or(SequencerError::MissingPayloadId)?; + + // Create a job that will wait for the configured duration before marking the payload as + // ready. + let payload_building_duration = self.config.payload_building_duration; + self.payload_building_job = Some(PayloadBuildingJob { + l1_origin, + future: Box::pin(async move { + // wait the configured duration for the execution node to build the payload. + tokio::time::sleep(tokio::time::Duration::from_millis(payload_building_duration)) + .await; + payload_id + }), + }); self.waker.wake(); + + Ok(()) } - /// Handle a reorg event. - pub fn handle_reorg(&mut self, queue_index: Option, l1_block_number: u64) { - if let Some(index) = queue_index { - self.l1_messages_queue_index = index; + /// Handles a new payload by fetching it from the engine and updating the FCS head. + pub async fn finalize_payload_building( + &mut self, + payload_id: PayloadId, + engine: &mut Engine, + ) -> Result, SequencerError> { + let payload = engine.get_payload(payload_id).await?; + + if payload.transactions.is_empty() { + tracing::trace!(target: "rollup_node::sequencer", "Built empty payload with id {payload_id:?}."); + Ok(None) + } else { + tracing::info!(target: "rollup_node::sequencer", "Built payload with id {payload_id:?}, hash: {:#x}, number: {} containing {} transactions.", payload.block_hash, payload.block_number, payload.transactions.len()); + let block_info = BlockInfo { hash: payload.block_hash, number: payload.block_number }; + engine.update_fcs(Some(block_info), None, None).await?; + let block: ScrollBlock = try_into_block( + ExecutionData { payload: payload.into(), sidecar: Default::default() }, + self.config.chain_spec.clone(), + ) + .map_err(|_| SequencerError::PayloadError)?; + Ok(Some(block)) } - self.l1_block_number = l1_block_number; } +} + +/// A job that builds a new payload. +pub struct PayloadBuildingJob { + /// The L1 origin block number of the first included L1 message, if any. + l1_origin: Option, + /// The future that resolves to the payload ID once the job is complete. + future: PayloadBuildingJobFuture, +} - /// Handle a new L1 block. - pub fn handle_new_l1_block(&mut self, block_number: u64) { - self.l1_block_number = block_number; +impl std::fmt::Debug for PayloadBuildingJob { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("PayloadBuildingJob") + .field("l1_origin", &self.l1_origin) + .field("future", &"PayloadBuildingJobFuture") + .finish() } +} - /// Handle new payload by updating the L1 messages queue index. - pub fn handle_new_payload(&mut self, block: &ScrollBlock) { - let queue_index = block.body.transactions.iter().filter_map(|tx| tx.queue_index()).max(); - if let Some(queue_index) = queue_index { - // only update the queue index if it has advanced - if queue_index + 1 > self.l1_messages_queue_index { - tracing::trace!(target: "rollup_node::sequencer", "Advancing L1 messages queue index from {} to {}", self.l1_messages_queue_index, queue_index + 1); - self.l1_messages_queue_index = queue_index + 1; - } else { - tracing::warn!(target: "rollup_node::sequencer", "Skipping L1 messages queue index update, current index is {}, new payload has max index {}", self.l1_messages_queue_index, queue_index); - } - } +impl PayloadBuildingJob { + /// Returns the L1 origin block number of the first included L1 message, if any. + pub fn l1_origin(&self) -> Option { + self.l1_origin } } /// A stream that produces payload attributes. -impl Stream for Sequencer { - type Item = ScrollPayloadAttributes; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - self.waker.register(cx.waker()); - - if let Some(payload_building_job) = self.payload_attributes_job.as_mut() { - match payload_building_job.as_mut().poll(cx) { - Poll::Ready(Ok(block)) => { - self.payload_attributes_job = None; - Poll::Ready(Some(block)) +impl Stream for Sequencer { + type Item = SequencerEvent; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + this.waker.register(cx.waker()); + + // Poll the trigger to see if it's time to build a new block. + if let Some(trigger) = this.trigger.as_mut() { + match trigger.poll_tick(cx) { + Poll::Ready(_) => { + // If there's no inflight job, emit a new slot event. + if this.payload_building_job.is_none() { + return Poll::Ready(Some(SequencerEvent::NewSlot)); + } else { + tracing::trace!(target: "rollup_node::sequencer", "Payload building job already in progress, skipping slot."); + }; } - Poll::Ready(Err(err)) => { - tracing::error!(target: "rollup_node::sequencer", "Error building payload attributes: {err}"); - self.payload_attributes_job = None; - Poll::Ready(None) - } - Poll::Pending => Poll::Pending, + Poll::Pending => {} } - } else { - Poll::Pending } - } -} -/// Builds the payload attributes for the sequencer using the given L1 message provider. -/// It collects the L1 messages to include in the payload and returns a `ScrollPayloadAttributes` -/// instance. -#[allow(clippy::too_many_arguments)] -async fn build_payload_attributes( - provider: Arc

, - max_l1_messages: u64, - payload_attributes: PayloadAttributes, - block_gas_limit: u64, - current_l1_block_number: u64, - l1_finalized_block_number: u64, - l1_message_inclusion_mode: L1MessageInclusionMode, - l1_messages_queue_index: u64, -) -> Result { - let mut l1_messages = vec![]; - let mut cumulative_gas_used = 0; - let expected_index = l1_messages_queue_index; - - // Collect L1 messages to include in payload. - let db_l1_messages = provider - .get_n_messages(l1_messages_queue_index.into(), max_l1_messages) - .await - .map_err(Into::::into)?; - - for msg in db_l1_messages { - // TODO (greg): we only check the DA limit on the execution node side. We should also check - // it here. - let fits_in_block = msg.transaction.gas_limit + cumulative_gas_used <= block_gas_limit; - let l1_inclusion_requirement_met = meets_l1_inclusion_requirement( - &msg, - l1_message_inclusion_mode, - current_l1_block_number, - l1_finalized_block_number, - ); - if !fits_in_block || !l1_inclusion_requirement_met { - break; - } - - // Defensively ensure L1 messages are contiguous. - if msg.transaction.queue_index != expected_index { - return Err(SequencerError::NonContiguousL1Messages { - expected: expected_index, - got: msg.transaction.queue_index, - }); + // If there is an inflight payload building job, poll it. + if let Some(payload_building_job) = this.payload_building_job.as_mut() { + match payload_building_job.future.as_mut().poll(cx) { + Poll::Ready(payload_id) => { + this.payload_building_job = None; + return Poll::Ready(Some(SequencerEvent::PayloadReady(payload_id))); + } + Poll::Pending => {} + } } - cumulative_gas_used += msg.transaction.gas_limit; - l1_messages.push(msg.transaction.encoded_2718().into()); + Poll::Pending } - - Ok(ScrollPayloadAttributes { - payload_attributes, - transactions: (!l1_messages.is_empty()).then_some(l1_messages), - no_tx_pool: false, - block_data_hint: BlockDataHint { - difficulty: Some(DEFAULT_BLOCK_DIFFICULTY), - ..Default::default() - }, - // If setting the gas limit to None, the Reth payload builder will use the gas limit passed - // via the `builder.gaslimit` CLI arg. - gas_limit: None, - }) } -/// Returns true if the L1 message should be included in the payload based on the inclusion mode, -/// the current L1 block number and the L1 finalized block number. -const fn meets_l1_inclusion_requirement( - l1_msg: &L1MessageEnvelope, - inclusion_mode: L1MessageInclusionMode, - current_l1_block_number: u64, - l1_finalized_block_number: u64, -) -> bool { - match inclusion_mode { - L1MessageInclusionMode::BlockDepth(depth) => { - l1_msg.l1_block_number + depth <= current_l1_block_number - } - L1MessageInclusionMode::Finalized => l1_msg.l1_block_number <= l1_finalized_block_number, - } -} - -impl fmt::Debug for Sequencer { +impl fmt::Debug for Sequencer { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("Sequencer") .field("provider", &"SequencerMessageProvider") - .field("fee_recipient", &self.fee_recipient) + .field("config", &self.config) .field("payload_building_job", &"PayloadBuildingJob") - .field("l1_message_per_block", &self.max_l1_messages_per_block) - .field("l1_message_inclusion_mode", &self.l1_message_inclusion_mode) .finish() } } -#[cfg(test)] -mod tests { - use super::*; - use scroll_alloy_consensus::TxL1Message; - - #[test] - fn test_l1_message_predicate() { - // block depth not met. - assert!(!meets_l1_inclusion_requirement( - &L1MessageEnvelope { - transaction: Default::default(), - l1_block_number: 10, - l2_block_number: None, - queue_hash: None, - }, - L1MessageInclusionMode::BlockDepth(5), - 10, - 10, - )); - - // block depth met. - assert!(meets_l1_inclusion_requirement( - &L1MessageEnvelope { - transaction: TxL1Message { gas_limit: 1, ..Default::default() }, - l1_block_number: 5, - l2_block_number: None, - queue_hash: None, - }, - L1MessageInclusionMode::BlockDepth(5), - 10, - 10, - )); - - // not finalized. - assert!(!meets_l1_inclusion_requirement( - &L1MessageEnvelope { - transaction: TxL1Message { gas_limit: 1, ..Default::default() }, - l1_block_number: 15, - l2_block_number: None, - queue_hash: None, - }, - L1MessageInclusionMode::Finalized, - 10, - 10, - )); - - // finalized. - assert!(meets_l1_inclusion_requirement( - &L1MessageEnvelope { - transaction: TxL1Message { gas_limit: 1, ..Default::default() }, - l1_block_number: 10, - l2_block_number: None, - queue_hash: None, - }, - L1MessageInclusionMode::Finalized, - 10, - 10, - )); - } +/// Creates a delayed interval that will not skip ticks if the interval is missed but will delay +/// the next tick until the interval has passed. +fn delayed_interval(interval: u64) -> Interval { + let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(interval)); + interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + interval } diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 4550db25..fbf1edf9 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -15,16 +15,18 @@ use rollup_node::{ RollupNodeDatabaseArgs, RollupNodeGasPriceOracleArgs, RollupNodeNetworkArgs, RpcArgs, ScrollRollupNodeConfig, SequencerArgs, SignerArgs, }; -use rollup_node_manager::RollupManagerEvent; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_primitives::{sig_encode_hash, BlockInfo, L1MessageEnvelope}; use rollup_node_providers::ScrollRootProvider; -use rollup_node_sequencer::{L1MessageInclusionMode, Sequencer}; +use rollup_node_sequencer::{ + L1MessageInclusionMode, PayloadBuildingConfig, Sequencer, SequencerConfig, SequencerEvent, +}; use rollup_node_signer::SignerEvent; use scroll_alloy_consensus::TxL1Message; use scroll_alloy_provider::ScrollAuthApiEngineClient; use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; use scroll_db::{test_utils::setup_test_db, DatabaseTransactionProvider, DatabaseWriteOperations}; -use scroll_engine::{EngineDriver, EngineDriverEvent, ForkchoiceState}; +use scroll_engine::{Engine, EngineDriver, EngineDriverEvent, ForkchoiceState}; use std::{ io::Write, path::PathBuf, @@ -47,7 +49,7 @@ async fn skip_block_with_no_transactions() { let (mut nodes, _tasks, _wallet) = setup(1, false).await.unwrap(); let node = nodes.pop().unwrap(); - // create a forkchoice state + // create a fork choice state let genesis_hash = node.inner.chain_spec().genesis_hash(); let fcs = ForkchoiceState::new( BlockInfo { hash: genesis_hash, number: 0 }, @@ -58,39 +60,41 @@ async fn skip_block_with_no_transactions() { // create the engine driver connected to the node let auth_client = node.inner.engine_http_client(); let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - false, - ); + let mut engine = Engine::new(Arc::new(engine_client), fcs); // create a test database let database = Arc::new(setup_test_db().await); let provider = database.clone(); + // Set the finalized block number + let tx = provider.tx_mut().await.unwrap(); + tx.set_finalized_l1_block_number(0).await.unwrap(); + tx.commit().await.unwrap(); + // create a sequencer - let mut sequencer = Sequencer::new( - provider, - Default::default(), - SCROLL_GAS_LIMIT, - 4, - 1, - L1MessageInclusionMode::BlockDepth(0), - 0, - ); + let config = SequencerConfig { + chain_spec: node.inner.chain_spec().clone(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + }, + block_time: 1, + payload_building_duration: 0, + allow_empty_blocks: true, + }; + let mut sequencer = Sequencer::new(provider, config); // send a new payload attributes request. - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - // assert that no new payload event is emitted - let res = tokio::time::timeout(Duration::from_secs(1), engine_driver.next()).await; - assert!(res.is_err(), "expected no new payload, but a block was built: {:?}", res.ok()); + sequencer.start_payload_building(&mut engine).await.unwrap(); + if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_none(), "expected no new payload, but got: {:?}", result); + } else { + panic!("expected a payload ready event"); + }; } #[tokio::test] @@ -115,30 +119,32 @@ async fn can_build_blocks() { // create the engine driver connected to the node let auth_client = node.inner.engine_http_client(); let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); + let mut engine = Engine::new(Arc::new(engine_client), fcs); // create a test database let database = Arc::new(setup_test_db().await); let provider = database.clone(); + // Set the finalized block number + let tx = provider.tx_mut().await.unwrap(); + tx.set_finalized_l1_block_number(5).await.unwrap(); + tx.commit().await.unwrap(); + // create a sequencer - let mut sequencer = Sequencer::new( - provider, - Default::default(), - SCROLL_GAS_LIMIT, - 4, - 1, - L1MessageInclusionMode::BlockDepth(0), - 0, - ); + let config = SequencerConfig { + chain_spec: node.inner.chain_spec().clone(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + }, + block_time: 1, + payload_building_duration: 0, + allow_empty_blocks: true, + }; + let mut sequencer = Sequencer::new(provider, config); // add a transaction to the pool let mut wallet_lock = wallet.lock().await; @@ -153,14 +159,13 @@ async fn can_build_blocks() { let tx_hash = node.rpc.inject_tx(raw_tx).await.unwrap(); // send a new payload attributes request. - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); + panic!("expected a payload ready event"); }; // wait for the block to be built @@ -201,14 +206,13 @@ async fn can_build_blocks() { tokio::time::sleep(std::time::Duration::from_secs(2)).await; // send a new block request this block should include the L1 message - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); + panic!("expected a payload ready event"); }; // make some assertions on the transaction inclusion of the block @@ -245,30 +249,27 @@ async fn can_build_blocks_with_delayed_l1_messages() { // create the engine driver connected to the node let auth_client = node.inner.engine_http_client(); let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); + let mut engine = Engine::new(Arc::new(engine_client), fcs); // create a test database let database = Arc::new(setup_test_db().await); let provider = database.clone(); // create a sequencer - let mut sequencer = Sequencer::new( - provider, - Default::default(), - SCROLL_GAS_LIMIT, - 4, - 0, - L1MessageInclusionMode::BlockDepth(L1_MESSAGE_DELAY), - 0, - ); + let config = SequencerConfig { + chain_spec: node.inner.chain_spec().clone(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + }, + block_time: 0, + payload_building_duration: 0, + allow_empty_blocks: true, + }; + let mut sequencer = Sequencer::new(provider, config); // now lets add an L1 message to the database (this transaction should not be included until the // l1 block number is 3) @@ -305,14 +306,13 @@ async fn can_build_blocks_with_delayed_l1_messages() { let tx_hash = node.rpc.inject_tx(raw_tx).await.unwrap(); // send a new payload attributes request. - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block + sequencer.start_payload_building(&mut engine); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); + panic!("expected a payload ready event"); }; // wait for the block to be built @@ -332,14 +332,13 @@ async fn can_build_blocks_with_delayed_l1_messages() { sequencer.handle_new_l1_block(3); // send a new block request this block should include the L1 message - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block + sequencer.start_payload_building(&mut engine); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); + panic!("expected a payload ready event"); }; // make some assertions on the transaction inclusion of the block @@ -349,630 +348,630 @@ async fn can_build_blocks_with_delayed_l1_messages() { assert_eq!(block.header.parent_hash, block_1_hash); } -#[tokio::test] -async fn can_build_blocks_with_finalized_l1_messages() { - reth_tracing::init_test_tracing(); - - let chain_spec = SCROLL_DEV.clone(); - const BLOCK_BUILDING_DURATION: Duration = tokio::time::Duration::from_millis(0); - // setup a test node - let (mut nodes, _tasks, wallet) = - setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) - .await - .unwrap(); - let node = nodes.pop().unwrap(); - let wallet = Arc::new(Mutex::new(wallet)); - - // create a forkchoice state - let genesis_hash = node.inner.chain_spec().genesis_hash(); - let fcs = ForkchoiceState::new( - BlockInfo { hash: genesis_hash, number: 0 }, - Default::default(), - Default::default(), - ); - - // create the engine driver connected to the node - let auth_client = node.inner.engine_http_client(); - let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); - - // create a test database - let database = Arc::new(setup_test_db().await); - let provider = database.clone(); - - // create a sequencer with Finalized mode - let mut sequencer = Sequencer::new( - provider, - Default::default(), - SCROLL_GAS_LIMIT, - 4, - 5, // current L1 block number - L1MessageInclusionMode::Finalized, - 0, - ); - - // set L1 finalized block number to 2 - sequencer.set_l1_finalized_block_number(2); - - // add L1 messages to database - let wallet_lock = wallet.lock().await; - - // this message should be included (before finalized block) - let finalized_l1_message = L1MessageEnvelope { - l1_block_number: 2, // <= 2 (finalized block) - l2_block_number: None, - queue_hash: None, - transaction: TxL1Message { - queue_index: 0, - gas_limit: 21000, - to: Address::random(), - value: U256::from(1), - sender: wallet_lock.inner.address(), - input: vec![].into(), - }, - }; - - // this message should not be included (after finalized block) - let unfinalized_l1_message = L1MessageEnvelope { - l1_block_number: 3, // > 2 (finalized block) - l2_block_number: None, - queue_hash: None, - transaction: TxL1Message { - queue_index: 1, - gas_limit: 21000, - to: Address::random(), - value: U256::from(2), - sender: wallet_lock.inner.address(), - input: vec![].into(), - }, - }; - drop(wallet_lock); - - let finalized_message_hash = finalized_l1_message.transaction.tx_hash(); - let unfinalized_message_hash = unfinalized_l1_message.transaction.tx_hash(); - - let tx = database.tx_mut().await.unwrap(); - tx.insert_l1_message(finalized_l1_message).await.unwrap(); - tx.insert_l1_message(unfinalized_l1_message).await.unwrap(); - tx.commit().await.unwrap(); - - // build payload, should only include finalized message - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block - } else { - panic!("expected a new payload event"); - }; - - // verify only finalized L1 message is included - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &finalized_message_hash); - - // ensure unfinalized message is not included - assert!(!block.body.transactions.iter().any(|tx| tx.tx_hash() == &unfinalized_message_hash)); - - // Handle the build block with the sequencer in order to update L1 message queue index. - sequencer.handle_new_payload(&block); - - // update finalized block number to 3, now both messages should be available - sequencer.set_l1_finalized_block_number(3); - - // sleep 2 seconds (ethereum header timestamp has granularity of seconds and proceeding header - // must have a greater timestamp than the last) - tokio::time::sleep(std::time::Duration::from_secs(2)).await; - - // build new payload - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block - } else { - panic!("expected a new payload event"); - }; - - // now should include the previously unfinalized message - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &unfinalized_message_hash); -} - -#[tokio::test] -async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { - reth_tracing::init_test_tracing(); - - // Create temporary private key file - let mut temp_file = NamedTempFile::new()?; - let private_key_hex = "0x1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; - temp_file.write_all(private_key_hex.as_bytes())?; - temp_file.flush()?; - - // Create expected signer - let expected_key_bytes = - hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef")?; - let expected_signer = alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes)?; - let expected_address = expected_signer.address(); - - let chain_spec = (*SCROLL_DEV).clone(); - let rollup_manager_args = ScrollRollupNodeConfig { - test: false, // disable test mode to enable real signing - network_args: RollupNodeNetworkArgs::default(), - database_args: RollupNodeDatabaseArgs { - rn_db_path: Some(PathBuf::from("sqlite::memory:")), - }, - l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs::default(), - chain_orchestrator_args: ChainOrchestratorArgs::default(), - sequencer_args: SequencerArgs { - sequencer_enabled: true, - auto_start: true, - block_time: 0, - l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), - payload_building_duration: 1000, - allow_empty_blocks: true, - ..SequencerArgs::default() - }, - blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, - signer_args: SignerArgs { - key_file: Some(temp_file.path().to_path_buf()), - aws_kms_key_id: None, - private_key: None, - }, - gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), - consensus_args: ConsensusArgs::noop(), - database: None, - rpc_args: RpcArgs::default(), - }; - - let (nodes, _tasks, wallet) = - setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; - let wallet = Arc::new(Mutex::new(wallet)); - - let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); - let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; - - // Generate and inject transaction - let mut wallet_lock = wallet.lock().await; - let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( - wallet_lock.chain_id, - wallet_lock.inner.clone(), - wallet_lock.inner_nonce, - ) - .await; - wallet_lock.inner_nonce += 1; - drop(wallet_lock); - let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; - - // Build block - sequencer_rnm_handle.build_block().await; - - // Verify block was successfully sequenced - if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); - } else { - panic!("Failed to receive BlockSequenced event"); - } - - // Verify signing event and signature correctness - if let Some(RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { - block: signed_block, - signature, - })) = sequencer_events.next().await - { - let hash = sig_encode_hash(&signed_block); - let recovered_address = signature.recover_address_from_prehash(&hash)?; - assert_eq!(recovered_address, expected_address); - } else { - panic!("Failed to receive SignerEvent with signed block"); - } - - Ok(()) -} - -#[tokio::test] -async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result<()> { - reth_tracing::init_test_tracing(); - - // Create temporary private key file (without 0x prefix) - let mut temp_file = NamedTempFile::new().unwrap(); - let private_key_hex = "1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; - temp_file.write_all(private_key_hex.as_bytes()).unwrap(); - temp_file.flush().unwrap(); - - // Create expected signer - let expected_key_bytes = - hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef").unwrap(); - let expected_signer = - alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes).unwrap(); - let expected_address = expected_signer.address(); - - let chain_spec = (*SCROLL_DEV).clone(); - let rollup_manager_args = ScrollRollupNodeConfig { - test: false, // disable test mode to enable real signing - network_args: RollupNodeNetworkArgs::default(), - database_args: RollupNodeDatabaseArgs { - rn_db_path: Some(PathBuf::from("sqlite::memory:")), - }, - l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs::default(), - chain_orchestrator_args: ChainOrchestratorArgs::default(), - sequencer_args: SequencerArgs { - sequencer_enabled: true, - auto_start: true, - block_time: 0, - l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), - payload_building_duration: 1000, - allow_empty_blocks: true, - ..SequencerArgs::default() - }, - blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, - signer_args: SignerArgs { - key_file: Some(temp_file.path().to_path_buf()), - aws_kms_key_id: None, - private_key: None, - }, - gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), - consensus_args: ConsensusArgs::noop(), - database: None, - rpc_args: RpcArgs::default(), - }; - - let (nodes, _tasks, wallet) = - setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; - let wallet = Arc::new(Mutex::new(wallet)); - - let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); - let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; - - // Generate and inject transaction - let mut wallet_lock = wallet.lock().await; - let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( - wallet_lock.chain_id, - wallet_lock.inner.clone(), - wallet_lock.inner_nonce, - ) - .await; - wallet_lock.inner_nonce += 1; - drop(wallet_lock); - let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; - - // Build block - sequencer_rnm_handle.build_block().await; - - // Verify block was successfully sequenced - if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); - } else { - panic!("Failed to receive BlockSequenced event"); - } - - // Verify signing event and signature correctness - while let Some(event) = sequencer_events.next().await { - if let RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { - block: signed_block, - signature, - }) = event - { - let hash = sig_encode_hash(&signed_block); - let recovered_address = signature.recover_address_from_prehash(&hash)?; - assert_eq!(recovered_address, expected_address); - break; - } - } - - Ok(()) -} - -#[tokio::test] -async fn can_build_blocks_and_exit_at_gas_limit() { - reth_tracing::init_test_tracing(); - - let chain_spec = SCROLL_DEV.clone(); - const MIN_TRANSACTION_GAS_COST: u64 = 21_000; - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); - const TRANSACTIONS_COUNT: usize = 2000; - - // setup a test node. use a high value for the payload building duration to be sure we don't - // exit early. - let (mut nodes, _tasks, wallet) = setup_engine( - ScrollRollupNodeConfig { - sequencer_args: SequencerArgs { payload_building_duration: 1000, ..Default::default() }, - ..default_test_scroll_rollup_node_config() - }, - 1, - chain_spec, - false, - false, - ) - .await - .unwrap(); - let node = nodes.pop().unwrap(); - let wallet = Arc::new(Mutex::new(wallet)); - - // add transactions. - let mut wallet_lock = wallet.lock().await; - for _ in 0..TRANSACTIONS_COUNT { - let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( - wallet_lock.chain_id, - wallet_lock.inner.clone(), - wallet_lock.inner_nonce, - ) - .await; - wallet_lock.inner_nonce += 1; - node.rpc.inject_tx(raw_tx).await.unwrap(); - } - drop(wallet_lock); - - // create a forkchoice state - let genesis_hash = node.inner.chain_spec().genesis_hash(); - let fcs = ForkchoiceState::new( - BlockInfo { hash: genesis_hash, number: 0 }, - Default::default(), - Default::default(), - ); - - // create the engine driver connected to the node - let auth_client = node.inner.engine_http_client(); - let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); - - // issue a new payload to the execution layer. - let timestamp = - SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); - engine_driver.handle_build_new_payload(ScrollPayloadAttributes { - payload_attributes: PayloadAttributes { - timestamp, - prev_randao: Default::default(), - suggested_fee_recipient: Default::default(), - withdrawals: None, - parent_beacon_block_root: None, - }, - transactions: None, - no_tx_pool: false, - block_data_hint: BlockDataHint::none(), - gas_limit: None, - }); - - // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. - if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); - } else { - panic!("expected a new payload event"); - } -} - -#[tokio::test] -async fn can_build_blocks_and_exit_at_time_limit() { - reth_tracing::init_test_tracing(); - - let chain_spec = SCROLL_DEV.clone(); - const MIN_TRANSACTION_GAS_COST: u64 = 21_000; - const BLOCK_BUILDING_DURATION: Duration = Duration::from_secs(1); - const TRANSACTIONS_COUNT: usize = 2000; - - // setup a test node. use a low payload building duration in order to exit before we reach the - // gas limit. - let (mut nodes, _tasks, wallet) = setup_engine( - ScrollRollupNodeConfig { - sequencer_args: SequencerArgs { payload_building_duration: 10, ..Default::default() }, - ..default_test_scroll_rollup_node_config() - }, - 1, - chain_spec, - false, - false, - ) - .await - .unwrap(); - let node = nodes.pop().unwrap(); - let wallet = Arc::new(Mutex::new(wallet)); - - // add transactions. - let mut wallet_lock = wallet.lock().await; - for _ in 0..TRANSACTIONS_COUNT { - let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( - wallet_lock.chain_id, - wallet_lock.inner.clone(), - wallet_lock.inner_nonce, - ) - .await; - wallet_lock.inner_nonce += 1; - node.rpc.inject_tx(raw_tx).await.unwrap(); - } - drop(wallet_lock); - - // create a forkchoice state - let genesis_hash = node.inner.chain_spec().genesis_hash(); - let fcs = ForkchoiceState::new( - BlockInfo { hash: genesis_hash, number: 0 }, - Default::default(), - Default::default(), - ); - - // create the engine driver connected to the node - let auth_client = node.inner.engine_http_client(); - let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); - - // start timer. - let start = Instant::now(); - - // issue a new payload to the execution layer. - let timestamp = - SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); - engine_driver.handle_build_new_payload(ScrollPayloadAttributes { - payload_attributes: PayloadAttributes { - timestamp, - prev_randao: Default::default(), - suggested_fee_recipient: Default::default(), - withdrawals: None, - parent_beacon_block_root: None, - }, - transactions: None, - no_tx_pool: false, - block_data_hint: BlockDataHint::none(), - gas_limit: None, - }); - - if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - let payload_building_duration = start.elapsed(); - // verify that the block building duration is within 10% of the target (we allow for 10% - // mismatch due to slower performance of debug mode). - assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); - assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); - } else { - panic!("expected a new payload event"); - } -} - -#[tokio::test] -async fn should_limit_l1_message_cumulative_gas() { - reth_tracing::init_test_tracing(); - - let chain_spec = SCROLL_DEV.clone(); - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - - // setup a test node - let (mut nodes, _tasks, wallet) = - setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) - .await - .unwrap(); - let node = nodes.pop().unwrap(); - let wallet = Arc::new(Mutex::new(wallet)); - - // create a forkchoice state - let genesis_hash = node.inner.chain_spec().genesis_hash(); - let fcs = ForkchoiceState::new( - BlockInfo { hash: genesis_hash, number: 0 }, - Default::default(), - Default::default(), - ); - - // create the engine driver connected to the node - let auth_client = node.inner.engine_http_client(); - let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); - - // create a test database - let database = Arc::new(setup_test_db().await); - let provider = database.clone(); - - // create a sequencer with Finalized mode - let mut sequencer = Sequencer::new( - provider, - Default::default(), - SCROLL_GAS_LIMIT, - 4, - 5, // current L1 block number - L1MessageInclusionMode::Finalized, - 0, - ); - sequencer.set_l1_finalized_block_number(1); - - // add L1 messages to database - let wallet_lock = wallet.lock().await; - let l1_messages = [ - L1MessageEnvelope { - l1_block_number: 1, - l2_block_number: None, - queue_hash: None, - transaction: TxL1Message { - queue_index: 0, - gas_limit: SCROLL_GAS_LIMIT / 2, - to: Address::random(), - value: U256::from(1), - sender: wallet_lock.inner.address(), - input: vec![].into(), - }, - }, - L1MessageEnvelope { - l1_block_number: 1, - l2_block_number: None, - queue_hash: None, - transaction: TxL1Message { - queue_index: 1, - gas_limit: SCROLL_GAS_LIMIT / 2 + 1, - to: Address::random(), - value: U256::from(1), - sender: wallet_lock.inner.address(), - input: vec![].into(), - }, - }, - ]; - let tx = database.tx_mut().await.unwrap(); - for l1_message in l1_messages { - tx.insert_l1_message(l1_message).await.unwrap(); - } - tx.commit().await.unwrap(); - - // build payload, should only include first l1 message - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block - } else { - panic!("expected a new payload event"); - }; - - // verify only one L1 message is included - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.header.gas_used, 21_000); - - // sleep 1 seconds (ethereum header timestamp has granularity of seconds and proceeding header - // must have a greater timestamp than the last) - tokio::time::sleep(Duration::from_secs(1)).await; - - // build new payload - sequencer.build_payload_attributes(); - let payload_attributes = sequencer.next().await.unwrap(); - engine_driver.handle_build_new_payload(payload_attributes); - - let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - block - } else { - panic!("expected a new payload event"); - }; - - // now should include the next l1 message. - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.header.gas_used(), 21_000); -} +// #[tokio::test] +// async fn can_build_blocks_with_finalized_l1_messages() { +// reth_tracing::init_test_tracing(); + +// let chain_spec = SCROLL_DEV.clone(); +// const BLOCK_BUILDING_DURATION: Duration = tokio::time::Duration::from_millis(0); +// // setup a test node +// let (mut nodes, _tasks, wallet) = +// setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) +// .await +// .unwrap(); +// let node = nodes.pop().unwrap(); +// let wallet = Arc::new(Mutex::new(wallet)); + +// // create a forkchoice state +// let genesis_hash = node.inner.chain_spec().genesis_hash(); +// let fcs = ForkchoiceState::new( +// BlockInfo { hash: genesis_hash, number: 0 }, +// Default::default(), +// Default::default(), +// ); + +// // create the engine driver connected to the node +// let auth_client = node.inner.engine_http_client(); +// let engine_client = ScrollAuthApiEngineClient::new(auth_client); +// let mut engine_driver = EngineDriver::new( +// Arc::new(engine_client), +// (*SCROLL_DEV).clone(), +// None::, +// fcs, +// false, +// BLOCK_BUILDING_DURATION, +// true, +// ); + +// // create a test database +// let database = Arc::new(setup_test_db().await); +// let provider = database.clone(); + +// // create a sequencer with Finalized mode +// let mut sequencer = Sequencer::new( +// provider, +// Default::default(), +// SCROLL_GAS_LIMIT, +// 4, +// 5, // current L1 block number +// L1MessageInclusionMode::Finalized, +// 0, +// ); + +// // set L1 finalized block number to 2 +// sequencer.set_l1_finalized_block_number(2); + +// // add L1 messages to database +// let wallet_lock = wallet.lock().await; + +// // this message should be included (before finalized block) +// let finalized_l1_message = L1MessageEnvelope { +// l1_block_number: 2, // <= 2 (finalized block) +// l2_block_number: None, +// queue_hash: None, +// transaction: TxL1Message { +// queue_index: 0, +// gas_limit: 21000, +// to: Address::random(), +// value: U256::from(1), +// sender: wallet_lock.inner.address(), +// input: vec![].into(), +// }, +// }; + +// // this message should not be included (after finalized block) +// let unfinalized_l1_message = L1MessageEnvelope { +// l1_block_number: 3, // > 2 (finalized block) +// l2_block_number: None, +// queue_hash: None, +// transaction: TxL1Message { +// queue_index: 1, +// gas_limit: 21000, +// to: Address::random(), +// value: U256::from(2), +// sender: wallet_lock.inner.address(), +// input: vec![].into(), +// }, +// }; +// drop(wallet_lock); + +// let finalized_message_hash = finalized_l1_message.transaction.tx_hash(); +// let unfinalized_message_hash = unfinalized_l1_message.transaction.tx_hash(); + +// let tx = database.tx_mut().await.unwrap(); +// tx.insert_l1_message(finalized_l1_message).await.unwrap(); +// tx.insert_l1_message(unfinalized_l1_message).await.unwrap(); +// tx.commit().await.unwrap(); + +// // build payload, should only include finalized message +// sequencer.start_payload_building(); +// let payload_attributes = sequencer.next().await.unwrap(); +// engine_driver.handle_build_new_payload(payload_attributes); + +// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// block +// } else { +// panic!("expected a new payload event"); +// }; + +// // verify only finalized L1 message is included +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &finalized_message_hash); + +// // ensure unfinalized message is not included +// assert!(!block.body.transactions.iter().any(|tx| tx.tx_hash() == &unfinalized_message_hash)); + +// // Handle the build block with the sequencer in order to update L1 message queue index. +// sequencer.handle_new_payload(&block); + +// // update finalized block number to 3, now both messages should be available +// sequencer.set_l1_finalized_block_number(3); + +// // sleep 2 seconds (ethereum header timestamp has granularity of seconds and proceeding +// header // must have a greater timestamp than the last) +// tokio::time::sleep(std::time::Duration::from_secs(2)).await; + +// // build new payload +// sequencer.start_payload_building(); +// let payload_attributes = sequencer.next().await.unwrap(); +// engine_driver.handle_build_new_payload(payload_attributes); + +// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// block +// } else { +// panic!("expected a new payload event"); +// }; + +// // now should include the previously unfinalized message +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &unfinalized_message_hash); +// } + +// #[tokio::test] +// async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { +// reth_tracing::init_test_tracing(); + +// // Create temporary private key file +// let mut temp_file = NamedTempFile::new()?; +// let private_key_hex = "0x1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; +// temp_file.write_all(private_key_hex.as_bytes())?; +// temp_file.flush()?; + +// // Create expected signer +// let expected_key_bytes = +// hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef")?; +// let expected_signer = alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes)?; +// let expected_address = expected_signer.address(); + +// let chain_spec = (*SCROLL_DEV).clone(); +// let rollup_manager_args = ScrollRollupNodeConfig { +// test: false, // disable test mode to enable real signing +// network_args: RollupNodeNetworkArgs::default(), +// database_args: RollupNodeDatabaseArgs { +// rn_db_path: Some(PathBuf::from("sqlite::memory:")), +// }, +// l1_provider_args: L1ProviderArgs::default(), +// engine_driver_args: EngineDriverArgs::default(), +// chain_orchestrator_args: ChainOrchestratorArgs::default(), +// sequencer_args: SequencerArgs { +// sequencer_enabled: true, +// auto_start: true, +// block_time: 0, +// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), +// payload_building_duration: 1000, +// allow_empty_blocks: true, +// ..SequencerArgs::default() +// }, +// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, +// signer_args: SignerArgs { +// key_file: Some(temp_file.path().to_path_buf()), +// aws_kms_key_id: None, +// private_key: None, +// }, +// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), +// consensus_args: ConsensusArgs::noop(), +// database: None, +// rpc_args: RpcArgs::default(), +// }; + +// let (nodes, _tasks, wallet) = +// setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; +// let wallet = Arc::new(Mutex::new(wallet)); + +// let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); +// let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + +// // Generate and inject transaction +// let mut wallet_lock = wallet.lock().await; +// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( +// wallet_lock.chain_id, +// wallet_lock.inner.clone(), +// wallet_lock.inner_nonce, +// ) +// .await; +// wallet_lock.inner_nonce += 1; +// drop(wallet_lock); +// let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; + +// // Build block +// sequencer_rnm_handle.build_block().await; + +// // Verify block was successfully sequenced +// if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); +// } else { +// panic!("Failed to receive BlockSequenced event"); +// } + +// // Verify signing event and signature correctness +// if let Some(ChainOrchestratorEvent::SignerEvent(SignerEvent::SignedBlock { +// block: signed_block, +// signature, +// })) = sequencer_events.next().await +// { +// let hash = sig_encode_hash(&signed_block); +// let recovered_address = signature.recover_address_from_prehash(&hash)?; +// assert_eq!(recovered_address, expected_address); +// } else { +// panic!("Failed to receive SignerEvent with signed block"); +// } + +// Ok(()) +// } + +// #[tokio::test] +// async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result<()> { +// reth_tracing::init_test_tracing(); + +// // Create temporary private key file (without 0x prefix) +// let mut temp_file = NamedTempFile::new().unwrap(); +// let private_key_hex = "1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; +// temp_file.write_all(private_key_hex.as_bytes()).unwrap(); +// temp_file.flush().unwrap(); + +// // Create expected signer +// let expected_key_bytes = +// hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef").unwrap(); +// let expected_signer = +// alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes).unwrap(); +// let expected_address = expected_signer.address(); + +// let chain_spec = (*SCROLL_DEV).clone(); +// let rollup_manager_args = ScrollRollupNodeConfig { +// test: false, // disable test mode to enable real signing +// network_args: RollupNodeNetworkArgs::default(), +// database_args: RollupNodeDatabaseArgs { +// rn_db_path: Some(PathBuf::from("sqlite::memory:")), +// }, +// l1_provider_args: L1ProviderArgs::default(), +// engine_driver_args: EngineDriverArgs::default(), +// chain_orchestrator_args: ChainOrchestratorArgs::default(), +// sequencer_args: SequencerArgs { +// sequencer_enabled: true, +// auto_start: true, +// block_time: 0, +// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), +// payload_building_duration: 1000, +// allow_empty_blocks: true, +// ..SequencerArgs::default() +// }, +// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, +// signer_args: SignerArgs { +// key_file: Some(temp_file.path().to_path_buf()), +// aws_kms_key_id: None, +// private_key: None, +// }, +// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), +// consensus_args: ConsensusArgs::noop(), +// database: None, +// rpc_args: RpcArgs::default(), +// }; + +// let (nodes, _tasks, wallet) = +// setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; +// let wallet = Arc::new(Mutex::new(wallet)); + +// let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); +// let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + +// // Generate and inject transaction +// let mut wallet_lock = wallet.lock().await; +// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( +// wallet_lock.chain_id, +// wallet_lock.inner.clone(), +// wallet_lock.inner_nonce, +// ) +// .await; +// wallet_lock.inner_nonce += 1; +// drop(wallet_lock); +// let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; + +// // Build block +// sequencer_rnm_handle.build_block().await; + +// // Verify block was successfully sequenced +// if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); +// } else { +// panic!("Failed to receive BlockSequenced event"); +// } + +// // Verify signing event and signature correctness +// while let Some(event) = sequencer_events.next().await { +// if let RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { +// block: signed_block, +// signature, +// }) = event +// { +// let hash = sig_encode_hash(&signed_block); +// let recovered_address = signature.recover_address_from_prehash(&hash)?; +// assert_eq!(recovered_address, expected_address); +// break; +// } +// } + +// Ok(()) +// } + +// #[tokio::test] +// async fn can_build_blocks_and_exit_at_gas_limit() { +// reth_tracing::init_test_tracing(); + +// let chain_spec = SCROLL_DEV.clone(); +// const MIN_TRANSACTION_GAS_COST: u64 = 21_000; +// const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); +// const TRANSACTIONS_COUNT: usize = 2000; + +// // setup a test node. use a high value for the payload building duration to be sure we don't +// // exit early. +// let (mut nodes, _tasks, wallet) = setup_engine( +// ScrollRollupNodeConfig { +// sequencer_args: SequencerArgs { payload_building_duration: 1000, ..Default::default() +// }, ..default_test_scroll_rollup_node_config() +// }, +// 1, +// chain_spec, +// false, +// false, +// ) +// .await +// .unwrap(); +// let node = nodes.pop().unwrap(); +// let wallet = Arc::new(Mutex::new(wallet)); + +// // add transactions. +// let mut wallet_lock = wallet.lock().await; +// for _ in 0..TRANSACTIONS_COUNT { +// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( +// wallet_lock.chain_id, +// wallet_lock.inner.clone(), +// wallet_lock.inner_nonce, +// ) +// .await; +// wallet_lock.inner_nonce += 1; +// node.rpc.inject_tx(raw_tx).await.unwrap(); +// } +// drop(wallet_lock); + +// // create a forkchoice state +// let genesis_hash = node.inner.chain_spec().genesis_hash(); +// let fcs = ForkchoiceState::new( +// BlockInfo { hash: genesis_hash, number: 0 }, +// Default::default(), +// Default::default(), +// ); + +// // create the engine driver connected to the node +// let auth_client = node.inner.engine_http_client(); +// let engine_client = ScrollAuthApiEngineClient::new(auth_client); +// let mut engine_driver = EngineDriver::new( +// Arc::new(engine_client), +// (*SCROLL_DEV).clone(), +// None::, +// fcs, +// false, +// BLOCK_BUILDING_DURATION, +// true, +// ); + +// // issue a new payload to the execution layer. +// let timestamp = +// SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); +// engine_driver.handle_build_new_payload(ScrollPayloadAttributes { +// payload_attributes: PayloadAttributes { +// timestamp, +// prev_randao: Default::default(), +// suggested_fee_recipient: Default::default(), +// withdrawals: None, +// parent_beacon_block_root: None, +// }, +// transactions: None, +// no_tx_pool: false, +// block_data_hint: BlockDataHint::none(), +// gas_limit: None, +// }); + +// // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. +// if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); +// } else { +// panic!("expected a new payload event"); +// } +// } + +// #[tokio::test] +// async fn can_build_blocks_and_exit_at_time_limit() { +// reth_tracing::init_test_tracing(); + +// let chain_spec = SCROLL_DEV.clone(); +// const MIN_TRANSACTION_GAS_COST: u64 = 21_000; +// const BLOCK_BUILDING_DURATION: Duration = Duration::from_secs(1); +// const TRANSACTIONS_COUNT: usize = 2000; + +// // setup a test node. use a low payload building duration in order to exit before we reach +// the // gas limit. +// let (mut nodes, _tasks, wallet) = setup_engine( +// ScrollRollupNodeConfig { +// sequencer_args: SequencerArgs { payload_building_duration: 10, ..Default::default() +// }, ..default_test_scroll_rollup_node_config() +// }, +// 1, +// chain_spec, +// false, +// false, +// ) +// .await +// .unwrap(); +// let node = nodes.pop().unwrap(); +// let wallet = Arc::new(Mutex::new(wallet)); + +// // add transactions. +// let mut wallet_lock = wallet.lock().await; +// for _ in 0..TRANSACTIONS_COUNT { +// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( +// wallet_lock.chain_id, +// wallet_lock.inner.clone(), +// wallet_lock.inner_nonce, +// ) +// .await; +// wallet_lock.inner_nonce += 1; +// node.rpc.inject_tx(raw_tx).await.unwrap(); +// } +// drop(wallet_lock); + +// // create a forkchoice state +// let genesis_hash = node.inner.chain_spec().genesis_hash(); +// let fcs = ForkchoiceState::new( +// BlockInfo { hash: genesis_hash, number: 0 }, +// Default::default(), +// Default::default(), +// ); + +// // create the engine driver connected to the node +// let auth_client = node.inner.engine_http_client(); +// let engine_client = ScrollAuthApiEngineClient::new(auth_client); +// let mut engine_driver = EngineDriver::new( +// Arc::new(engine_client), +// (*SCROLL_DEV).clone(), +// None::, +// fcs, +// false, +// BLOCK_BUILDING_DURATION, +// true, +// ); + +// // start timer. +// let start = Instant::now(); + +// // issue a new payload to the execution layer. +// let timestamp = +// SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); +// engine_driver.handle_build_new_payload(ScrollPayloadAttributes { +// payload_attributes: PayloadAttributes { +// timestamp, +// prev_randao: Default::default(), +// suggested_fee_recipient: Default::default(), +// withdrawals: None, +// parent_beacon_block_root: None, +// }, +// transactions: None, +// no_tx_pool: false, +// block_data_hint: BlockDataHint::none(), +// gas_limit: None, +// }); + +// if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// let payload_building_duration = start.elapsed(); +// // verify that the block building duration is within 10% of the target (we allow for 10% +// // mismatch due to slower performance of debug mode). +// assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); +// assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); +// } else { +// panic!("expected a new payload event"); +// } +// } + +// #[tokio::test] +// async fn should_limit_l1_message_cumulative_gas() { +// reth_tracing::init_test_tracing(); + +// let chain_spec = SCROLL_DEV.clone(); +// const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); + +// // setup a test node +// let (mut nodes, _tasks, wallet) = +// setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) +// .await +// .unwrap(); +// let node = nodes.pop().unwrap(); +// let wallet = Arc::new(Mutex::new(wallet)); + +// // create a forkchoice state +// let genesis_hash = node.inner.chain_spec().genesis_hash(); +// let fcs = ForkchoiceState::new( +// BlockInfo { hash: genesis_hash, number: 0 }, +// Default::default(), +// Default::default(), +// ); + +// // create the engine driver connected to the node +// let auth_client = node.inner.engine_http_client(); +// let engine_client = ScrollAuthApiEngineClient::new(auth_client); +// let mut engine_driver = EngineDriver::new( +// Arc::new(engine_client), +// (*SCROLL_DEV).clone(), +// None::, +// fcs, +// false, +// BLOCK_BUILDING_DURATION, +// true, +// ); + +// // create a test database +// let database = Arc::new(setup_test_db().await); +// let provider = database.clone(); + +// // create a sequencer with Finalized mode +// let mut sequencer = Sequencer::new( +// provider, +// Default::default(), +// SCROLL_GAS_LIMIT, +// 4, +// 5, // current L1 block number +// L1MessageInclusionMode::Finalized, +// 0, +// ); +// sequencer.set_l1_finalized_block_number(1); + +// // add L1 messages to database +// let wallet_lock = wallet.lock().await; +// let l1_messages = [ +// L1MessageEnvelope { +// l1_block_number: 1, +// l2_block_number: None, +// queue_hash: None, +// transaction: TxL1Message { +// queue_index: 0, +// gas_limit: SCROLL_GAS_LIMIT / 2, +// to: Address::random(), +// value: U256::from(1), +// sender: wallet_lock.inner.address(), +// input: vec![].into(), +// }, +// }, +// L1MessageEnvelope { +// l1_block_number: 1, +// l2_block_number: None, +// queue_hash: None, +// transaction: TxL1Message { +// queue_index: 1, +// gas_limit: SCROLL_GAS_LIMIT / 2 + 1, +// to: Address::random(), +// value: U256::from(1), +// sender: wallet_lock.inner.address(), +// input: vec![].into(), +// }, +// }, +// ]; +// let tx = database.tx_mut().await.unwrap(); +// for l1_message in l1_messages { +// tx.insert_l1_message(l1_message).await.unwrap(); +// } +// tx.commit().await.unwrap(); + +// // build payload, should only include first l1 message +// sequencer.start_payload_building(); +// let payload_attributes = sequencer.next().await.unwrap(); +// engine_driver.handle_build_new_payload(payload_attributes); + +// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// block +// } else { +// panic!("expected a new payload event"); +// }; + +// // verify only one L1 message is included +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.header.gas_used, 21_000); + +// // sleep 1 seconds (ethereum header timestamp has granularity of seconds and proceeding +// header // must have a greater timestamp than the last) +// tokio::time::sleep(Duration::from_secs(1)).await; + +// // build new payload +// sequencer.start_payload_building(); +// let payload_attributes = sequencer.next().await.unwrap(); +// engine_driver.handle_build_new_payload(payload_attributes); + +// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { +// block +// } else { +// panic!("expected a new payload event"); +// }; + +// // now should include the next l1 message. +// assert_eq!(block.body.transactions.len(), 1); +// assert_eq!(block.header.gas_used(), 21_000); +// } From c3f113e9f88de180e1e858b5e24b51d4dbdcf53a Mon Sep 17 00:00:00 2001 From: frisitano Date: Tue, 7 Oct 2025 15:06:36 +0800 Subject: [PATCH 02/12] atomic rollup node --- Cargo.lock | 5 + Cargo.toml | 1 + crates/chain-orchestrator/Cargo.toml | 2 + .../chain-orchestrator/src/consolidation.rs | 38 +- crates/chain-orchestrator/src/error.rs | 20 + crates/chain-orchestrator/src/event.rs | 32 +- crates/chain-orchestrator/src/handle/mod.rs | 9 + crates/chain-orchestrator/src/lib.rs | 549 ++++--- crates/chain-orchestrator/src/sync.rs | 8 +- crates/database/db/src/db.rs | 6 +- crates/database/db/src/error.rs | 3 + crates/database/db/src/operations.rs | 199 ++- crates/database/migration/src/lib.rs | 2 + ...005_160938_add_initial_l1_block_numbers.rs | 39 + crates/derivation-pipeline/src/new.rs | 1 + crates/node/Cargo.toml | 5 +- crates/node/src/add_ons/rollup.rs | 6 +- crates/node/src/args.rs | 59 +- crates/node/src/test_utils.rs | 4 +- crates/node/tests/e2e.rs | 470 +++--- crates/node/tests/sync.rs | 737 +++++----- crates/primitives/src/batch.rs | 25 +- crates/primitives/src/block.rs | 6 + crates/sequencer/Cargo.toml | 1 + crates/sequencer/src/config.rs | 8 +- crates/sequencer/src/event.rs | 1 - crates/sequencer/src/lib.rs | 9 +- crates/sequencer/tests/e2e.rs | 1280 +++++++++-------- 28 files changed, 1968 insertions(+), 1557 deletions(-) create mode 100644 crates/database/migration/src/m20251005_160938_add_initial_l1_block_numbers.rs diff --git a/Cargo.lock b/Cargo.lock index d1aa8c19..cf4a39c7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10754,6 +10754,7 @@ dependencies = [ "reth-evm", "reth-network", "reth-network-api", + "reth-network-p2p", "reth-node-api", "reth-node-builder", "reth-node-core", @@ -10767,6 +10768,7 @@ dependencies = [ "reth-rpc-server-types", "reth-scroll-chainspec", "reth-scroll-cli", + "reth-scroll-consensus", "reth-scroll-engine-primitives", "reth-scroll-evm", "reth-scroll-node", @@ -10830,6 +10832,7 @@ dependencies = [ "reth-network-peers", "reth-primitives-traits", "reth-scroll-chainspec", + "reth-scroll-consensus", "reth-scroll-engine-primitives", "reth-scroll-forks", "reth-scroll-node", @@ -10839,6 +10842,7 @@ dependencies = [ "reth-tracing", "rollup-node-primitives", "rollup-node-providers", + "rollup-node-sequencer", "rollup-node-signer", "rollup-node-watcher", "scroll-alloy-consensus", @@ -10976,6 +10980,7 @@ dependencies = [ "rollup-node-providers", "rollup-node-sequencer", "rollup-node-signer", + "rollup-node-watcher", "scroll-alloy-consensus", "scroll-alloy-hardforks", "scroll-alloy-provider", diff --git a/Cargo.toml b/Cargo.toml index 32b0c3b1..054824ac 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -179,6 +179,7 @@ reth-cli-util = { git = "https://github.com/scroll-tech/reth.git", default-featu # reth-scroll reth-scroll-chainspec = { git = "https://github.com/scroll-tech/reth.git", default-features = false } +reth-scroll-consensus = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-scroll-cli = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-scroll-evm = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-scroll-rpc = { git = "https://github.com/scroll-tech/reth.git", default-features = false } diff --git a/crates/chain-orchestrator/Cargo.toml b/crates/chain-orchestrator/Cargo.toml index 6ac8bc01..0bf42e7c 100644 --- a/crates/chain-orchestrator/Cargo.toml +++ b/crates/chain-orchestrator/Cargo.toml @@ -25,11 +25,13 @@ scroll-db.workspace = true scroll-engine.workspace = true rollup-node-providers.workspace = true rollup-node-primitives.workspace = true +rollup-node-sequencer.workspace = true rollup-node-signer.workspace = true rollup-node-watcher.workspace = true # scroll reth-scroll-engine-primitives.workspace = true +reth-scroll-consensus.workspace = true reth-scroll-node.workspace = true reth-scroll-primitives.workspace = true scroll-alloy-consensus.workspace = true diff --git a/crates/chain-orchestrator/src/consolidation.rs b/crates/chain-orchestrator/src/consolidation.rs index 0da77177..6cd2f3b3 100644 --- a/crates/chain-orchestrator/src/consolidation.rs +++ b/crates/chain-orchestrator/src/consolidation.rs @@ -4,12 +4,13 @@ use futures::{stream::FuturesOrdered, TryStreamExt}; use rollup_node_primitives::{BatchInfo, BlockInfo}; use scroll_alloy_network::Scroll; use scroll_derivation_pipeline::{BatchDerivationResult, DerivedAttributes}; -use scroll_engine::block_matches_attributes; +use scroll_engine::{block_matches_attributes, ForkchoiceState}; /// Reconciles a batch of derived attributes with the L2 chain to produce a reconciliation result. pub(crate) async fn reconcile_batch>( l2_provider: L2P, batch: BatchDerivationResult, + fcs: &ForkchoiceState, ) -> Result { let mut futures = FuturesOrdered::new(); for attributes in batch.attributes { @@ -17,6 +18,7 @@ pub(crate) async fn reconcile_batch>( // Fetch the block corresponding to the derived attributes from the L2 provider. let current_block = l2_provider .get_block(attributes.block_number.into()) + .full() .await? .map(|b| b.into_consensus().map_transactions(|tx| tx.inner.into_inner())); @@ -33,10 +35,19 @@ pub(crate) async fn reconcile_batch>( ¤t_block, current_block.parent_hash, ) { - // The block matches the derived attributes, no action is needed. - Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::UpdateSafeHead( - BlockInfo { number: current_block.number, hash: current_block.hash_slow() }, - )) + // The block matches the derived attributes and the block is below or equal to the + // safe current safe head. + if attributes.block_number <= fcs.safe_block_info().number { + Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Skip(BlockInfo { + number: current_block.number, + hash: current_block.hash_slow(), + })) + } else { + // The block matches the derived attributes, no action is needed. + Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::UpdateSafeHead( + BlockInfo { number: current_block.number, hash: current_block.hash_slow() }, + )) + } } else { // The block does not match the derived attributes, a reorg is needed. Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Reorg(attributes)) @@ -63,6 +74,23 @@ pub(crate) struct BatchReconciliationResult { pub(crate) enum BlockConsolidationAction { /// Update the safe head to the given block. UpdateSafeHead(BlockInfo), + /// The derived attributes match the L2 chain and the safe head is already at or beyond the + /// block, so no action is needed. + Skip(BlockInfo), /// Reorganize the chain with the given derived attributes. Reorg(DerivedAttributes), } + +impl std::fmt::Display for BlockConsolidationAction { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + BlockConsolidationAction::UpdateSafeHead(info) => { + write!(f, "UpdateSafeHead to block {}", info.number) + } + BlockConsolidationAction::Skip(info) => write!(f, "Skip block {}", info.number), + BlockConsolidationAction::Reorg(attrs) => { + write!(f, "Reorg to block {}", attrs.block_number) + } + } + } +} diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index 93c65dea..d7e38e2a 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -1,6 +1,9 @@ use alloy_json_rpc::RpcError; use alloy_primitives::B256; use alloy_transport::TransportErrorKind; +use rollup_node_primitives::{BatchInfo, BlockInfo}; +use rollup_node_sequencer::SequencerError; +use rollup_node_signer::SignerError; use scroll_db::{DatabaseError, L1MessageKey}; use scroll_engine::EngineError; @@ -46,6 +49,14 @@ pub enum ChainOrchestratorError { /// The hash of the block header that was requested. hash: B256, }, + /// The peer did not provide the correct number of blocks. + #[error("The peer did not provide the correct number of blocks. Expected: {expected}, Actual: {actual}")] + BlockFetchMismatch { + /// The expected number of blocks. + expected: usize, + /// The actual number of blocks. + actual: usize, + }, /// A gap was detected in batch commit events: the previous batch before index {0} is missing. #[error("Batch commit gap detected at index {0}, previous batch commit not found")] BatchCommitGap(u64), @@ -58,4 +69,13 @@ pub enum ChainOrchestratorError { /// Received an invalid block from peer. #[error("Received an invalid block from peer")] InvalidBlock, + /// An error occurred at the sequencer level. + #[error("An error occurred at the sequencer level: {0}")] + SequencerError(#[from] SequencerError), + /// An error occurred at the signing level. + #[error("An error occurred at the signer level: {0}")] + SignerError(#[from] SignerError), + /// The derivation pipeline found an invalid block for the given batch. + #[error("The derivation pipeline found an invalid block: {0} for batch: {1}")] + InvalidBatch(BlockInfo, BatchInfo), } diff --git a/crates/chain-orchestrator/src/event.rs b/crates/chain-orchestrator/src/event.rs index 154f2da1..aba25235 100644 --- a/crates/chain-orchestrator/src/event.rs +++ b/crates/chain-orchestrator/src/event.rs @@ -2,11 +2,18 @@ use alloy_consensus::Header; use alloy_primitives::{Signature, B256}; use reth_network_peers::PeerId; use reth_scroll_primitives::ScrollBlock; -use rollup_node_primitives::{BatchInfo, BlockInfo, ChainImport, L2BlockInfoWithL1Messages}; +use rollup_node_primitives::{ + BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, ChainImport, + L2BlockInfoWithL1Messages, +}; +use scroll_db::L1MessageKey; +use scroll_network::NewBlockWithPeer; /// An event emitted by the `ChainOrchestrator`. #[derive(Debug, Clone, PartialEq, Eq)] pub enum ChainOrchestratorEvent { + /// A received block failed the consensus checks. + BlockFailedConsensusChecks(B256, PeerId), /// A new block has been received from the network but we have insufficient data to process it /// due to being in optimistic mode. InsufficientDataForReceivedBlock(B256), @@ -22,7 +29,7 @@ pub enum ChainOrchestratorEvent { signature: Signature, }, /// The chain should be optimistically synced to the provided block. - OptimisticSync(ScrollBlock), + OptimisticSync(BlockInfo), /// The chain has been extended, returning the new blocks. ChainExtended(ChainImport), /// The chain has reorged, returning the new chain and the peer that provided them. @@ -40,6 +47,8 @@ pub enum ChainOrchestratorEvent { }, /// A batch has been finalized returning a list of finalized batches. BatchFinalized(u64, Vec), + /// A new L1 block has been received returning the L1 block number. + NewL1Block(u64), /// An L1 block has been finalized returning the L1 block number and the list of finalized /// batches. L1BlockFinalized(u64, Vec), @@ -58,9 +67,28 @@ pub enum ChainOrchestratorEvent { /// The L2 safe block info. l2_safe_block_info: Option, }, + /// The chain orchestrator has synced to the L1 head. + L1Synced, /// An L2 block has been committed returning the [`L2BlockInfoWithL1Messages`] and an /// optional [`BatchInfo`] if the block is associated with a committed batch. L2ChainCommitted(L2BlockInfoWithL1Messages, Option, bool), /// An L2 consolidated block has been committed returning the [`L2BlockInfoWithL1Messages`]. L2ConsolidatedBlockCommitted(L2BlockInfoWithL1Messages), + /// A new block has been sequenced by the sequencer. + BlockSequenced(ScrollBlock), + /// A new block has been signed by the signer. + SignedBlock { + /// The signed block. + block: ScrollBlock, + /// The signature of the signed block. + signature: Signature, + }, + /// An L1 message was not found in the database. + L1MessageNotFoundInDatabase(L1MessageKey), + /// A new block has been received from the network. + NewBlockReceived(NewBlockWithPeer), + /// A batch consolidation has occurred, returning the outcome of the consolidation. + BatchConsolidated(BatchConsolidationOutcome), + /// An L1 derived block has been consolidated, returning the outcome of the consolidation. + BlockConsolidated(BlockConsolidationOutcome), } diff --git a/crates/chain-orchestrator/src/handle/mod.rs b/crates/chain-orchestrator/src/handle/mod.rs index 983ffcf9..3d0e74ac 100644 --- a/crates/chain-orchestrator/src/handle/mod.rs +++ b/crates/chain-orchestrator/src/handle/mod.rs @@ -1,3 +1,5 @@ +use crate::ChainOrchestratorStatus; + use super::ChainOrchestratorEvent; // use crate::manager::metrics::HandleMetrics; use reth_network_api::FullNetwork; @@ -78,4 +80,11 @@ impl> ChainOrchestratorHand self.send_command(ChainOrchestratorCommand::DisableAutomaticSequencing(tx)); rx.await } + + /// Sends a command to the rollup manager to get the current status. + pub async fn status(&self) -> Result { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::Status(tx)); + rx.await + } } diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index 50c73bd6..9b4da549 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -1,27 +1,27 @@ //! A library responsible for orchestrating the L2 chain based on data received from L1 and over the //! L2 p2p network. -use alloy_consensus::Header; use alloy_eips::Encodable2718; use alloy_primitives::{b256, bytes::Bytes, keccak256, B256}; use alloy_provider::Provider; use alloy_rpc_types_engine::ExecutionData; use futures::StreamExt; use reth_chainspec::EthChainSpec; -use reth_network_api::{BlockDownloaderProvider, FullNetwork, HeadersClient}; -use reth_network_p2p::{headers::client::HeadersRequest, BodiesClient}; +use reth_network_api::{BlockDownloaderProvider, FullNetwork}; +use reth_network_p2p::FullBlockClient; use reth_scroll_engine_primitives::try_into_block; use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; -use reth_tasks::shutdown::GracefulShutdown; +use reth_tasks::shutdown::Shutdown; use reth_tokio_util::{EventSender, EventStream}; use rollup_node_primitives::{ BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, - L1MessageEnvelope, L2BlockInfoWithL1Messages, + ChainImport, L1MessageEnvelope, L2BlockInfoWithL1Messages, }; -use rollup_node_providers::L1Provider; -use rollup_node_signer::SignatureAsBytes; -use rollup_node_watcher::{test_utils::chain, L1Notification}; +use rollup_node_providers::{L1MessageProvider, L1Provider}; +use rollup_node_sequencer::{Sequencer, SequencerEvent}; +use rollup_node_signer::{SignatureAsBytes, SignerEvent, SignerHandle}; +use rollup_node_watcher::L1Notification; use scroll_alloy_consensus::TxL1Message; use scroll_alloy_hardforks::ScrollHardforks; use scroll_alloy_network::Scroll; @@ -93,6 +93,7 @@ pub struct ChainOrchestrator< N: FullNetwork, ChainSpec, L1P, + L1MP, L2P, EC, > { @@ -101,7 +102,7 @@ pub struct ChainOrchestrator< /// The receiver for commands sent to the chain orchestrator. handle_rx: UnboundedReceiver>, /// The `BlockClient` that is used to fetch blocks from peers over p2p. - network_client: Arc<::Client>, + block_client: Arc::Client>>, /// The L2 client that is used to interact with the L2 chain. l2_client: Arc, /// A reference to the database used to persist the indexed data. @@ -118,6 +119,10 @@ pub struct ChainOrchestrator< consensus: Box, /// The engine used to communicate with the execution layer. engine: Engine, + /// The sequencer used to build blocks. + sequencer: Option>, + /// The signer used to sign messages. + signer: Option, /// The derivation pipeline used to derive L2 blocks from batches. derivation_pipeline: DerivationPipelineNew, /// Optional event sender for broadcasting events to listeners. @@ -128,27 +133,30 @@ impl< N: FullNetwork + Send + Sync + 'static, ChainSpec: ScrollHardforks + EthChainSpec + Send + Sync + 'static, L1P: L1Provider + Unpin + Clone + Send + Sync + 'static, + L1MP: L1MessageProvider + Unpin + Clone + Send + Sync + 'static, L2P: Provider + 'static, EC: ScrollEngineApi + Sync + Send + 'static, - > ChainOrchestrator + > ChainOrchestrator { /// Creates a new chain orchestrator. pub async fn new( database: Arc, config: ChainOrchestratorConfig, - block_client: ::Client, + block_client: Arc::Client>>, l2_provider: L2P, l1_notification_rx: Receiver>, network: ScrollNetwork, consensus: Box, engine: Engine, + sequencer: Option>, + signer: Option, derivation_pipeline: DerivationPipelineNew, ) -> Result<(Self, ChainOrchestratorHandle), ChainOrchestratorError> { let (handle_tx, handle_rx) = mpsc::unbounded_channel(); let handle = ChainOrchestratorHandle::new(handle_tx); Ok(( Self { - network_client: Arc::new(block_client), + block_client, l2_client: Arc::new(l2_provider), database, config, @@ -163,6 +171,8 @@ impl< network, consensus, engine, + sequencer, + signer, derivation_pipeline, handle_rx, event_sender: None, @@ -171,35 +181,60 @@ impl< )) } - /// Main run loop for the chain orchestrator. - pub async fn run(&mut self) { + /// Drives the [`ChainOrchestrator`] future until a [`Shutdown`] signal is received. + pub async fn run_until_shutdown(mut self, mut shutdown: Shutdown) { loop { tokio::select! { + biased; + + _guard = &mut shutdown => { + break; + } Some(command) = self.handle_rx.recv() => { - tracing::info!(target: "scroll::chain_orchestrator", ?command, "Received command"); - self.handle_command(command).await; + if let Err(err) = self.handle_command(command).await { + tracing::error!(target: "scroll::chain_orchestrator", ?err, "Error handling command"); + } } - Some(batch) = self.derivation_pipeline.next() => { - // Process the derived batch. - tracing::info!(target: "scroll::chain_orchestrator", ?batch, "Derived new L2 block from batch"); - if let Err(err) = self.handle_derived_batch(batch).await { - tracing::error!(target: "scroll::chain_orchestrator", ?err, "Error handling derived batch"); + Some(event) = async { + if let Some(event) = self.signer.as_mut() { + event.next().await + } else { + unreachable!() } + }, if self.signer.is_some() => { + let res = self.handle_signer_event(event).await; + self.handle_outcome(res); } - Some(event) = self.network.events().next() => { - if let Err(err) = self.handle_network_event(event).await { - tracing::error!(target: "scroll::chain_orchestrator", ?err, "Error handling network event"); + Some(event) = async { + if let Some(seq) = self.sequencer.as_mut() { + seq.next().await + } else { + unreachable!() } + }, if self.sequencer.is_some() => { + let res = self.handle_sequencer_event(event).await; + self.handle_outcome(res); + } + Some(batch) = self.derivation_pipeline.next() => { + let res = self.handle_derived_batch(batch).await; + self.handle_outcome(res); + } + Some(event) = self.network.events().next() => { + let res = self.handle_network_event(event).await; + self.handle_outcome(res); } Some(notification) = self.l1_notification_rx.recv(), if self.sync_state.l2().is_synced() && self.derivation_pipeline.len().await == 0 => { let res = self.handle_l1_notification(notification).await; self.handle_outcome(res); } + } } } - pub fn handle_outcome( + /// Handles the outcome of an operation, logging errors and notifying event listeners as + /// appropriate. + fn handle_outcome( &self, outcome: Result, ChainOrchestratorError>, ) { @@ -212,10 +247,82 @@ impl< } } - async fn handle_command(&mut self, command: ChainOrchestratorCommand) { + /// Handles an event from the signer. + async fn handle_signer_event( + &mut self, + event: rollup_node_signer::SignerEvent, + ) -> Result, ChainOrchestratorError> { + tracing::info!(target: "scroll::chain_orchestrator", ?event, "Handling signer event"); + match event { + SignerEvent::SignedBlock { block, signature } => { + let tx = self.database.tx_mut().await?; + tx.insert_signature(block.hash_slow(), signature).await?; + tx.commit().await?; + self.network.handle().announce_block(block.clone(), signature); + Ok(Some(ChainOrchestratorEvent::SignedBlock { block, signature })) + } + } + } + + /// Handles an event from the sequencer. + async fn handle_sequencer_event( + &mut self, + event: rollup_node_sequencer::SequencerEvent, + ) -> Result, ChainOrchestratorError> { + tracing::info!(target: "scroll::chain_orchestrator", ?event, "Handling sequencer event"); + match event { + SequencerEvent::NewSlot => { + if self.consensus.should_sequence_block( + self.signer + .as_ref() + .map(|s| &s.address) + .expect("signer must be set if sequencer is present"), + ) { + self.sequencer + .as_mut() + .expect("sequencer must be present") + .start_payload_building(&mut self.engine) + .await?; + } + } + SequencerEvent::PayloadReady(payload_id) => { + if let Some(block) = self + .sequencer + .as_mut() + .expect("sequencer must be present") + .finalize_payload_building(payload_id, &mut self.engine) + .await? + { + let tx = self.database.tx_mut().await?; + let block_info: L2BlockInfoWithL1Messages = (&block).into(); + tx.update_l1_messages_from_l2_blocks(vec![block_info.clone()]).await?; + tx.set_l2_head_block_number(block_info.block_info.number).await?; + tx.commit().await?; + self.signer + .as_mut() + .expect("signer must be present") + .sign_block(block.clone())?; + return Ok(Some(ChainOrchestratorEvent::BlockSequenced(block))) + } + } + } + + Ok(None) + } + + /// Handles a command sent to the chain orchestrator. + async fn handle_command( + &mut self, + command: ChainOrchestratorCommand, + ) -> Result<(), ChainOrchestratorError> { + tracing::debug!(target: "scroll::chain_orchestrator", ?command, "Handling command"); match command { ChainOrchestratorCommand::BuildBlock => { - todo!(); + if let Some(sequencer) = self.sequencer.as_mut() { + sequencer.start_payload_building(&mut self.engine).await?; + } else { + tracing::error!(target: "scroll::chain_orchestrator", "Received BuildBlock command but sequencer is not configured"); + } } ChainOrchestratorCommand::EventListener(tx) => { let _ = tx.send(self.event_listener()); @@ -233,13 +340,27 @@ impl< ChainOrchestratorCommand::UpdateFcsHead(_head) => { todo!() } - ChainOrchestratorCommand::EnableAutomaticSequencing(_tx) => { - todo!() + ChainOrchestratorCommand::EnableAutomaticSequencing(tx) => { + if let Some(sequencer) = self.sequencer.as_mut() { + sequencer.enable(); + let _ = tx.send(true); + } else { + tracing::error!(target: "scroll::chain_orchestrator", "Received EnableAutomaticSequencing command but sequencer is not configured"); + let _ = tx.send(false); + } } - ChainOrchestratorCommand::DisableAutomaticSequencing(_tx) => { - todo!() + ChainOrchestratorCommand::DisableAutomaticSequencing(tx) => { + if let Some(sequencer) = self.sequencer.as_mut() { + sequencer.disable(); + let _ = tx.send(true); + } else { + tracing::error!(target: "scroll::chain_orchestrator", "Received DisableAutomaticSequencing command but sequencer is not configured"); + let _ = tx.send(false); + } } } + + Ok(()) } /// Returns a new event listener for the rollup node manager. @@ -259,19 +380,23 @@ impl< async fn handle_derived_batch( &mut self, batch: BatchDerivationResult, - ) -> Result<(), ChainOrchestratorError> { + ) -> Result, ChainOrchestratorError> { tracing::info!(target: "scroll::chain_orchestrator", batch_info = ?batch.batch_info, num_blocks = batch.attributes.len(), "Handling derived batch"); - let batch_reconciliation_result = reconcile_batch(&self.l2_client, batch).await?; + let batch_reconciliation_result = + reconcile_batch(&self.l2_client, batch, self.engine.fcs()).await?; let mut batch_consolidation_result = BatchConsolidationOutcome::new(batch_reconciliation_result.batch_info); for action in batch_reconciliation_result.actions { - match action { + let outcome = match action { + BlockConsolidationAction::Skip(block_info) => { + tracing::info!(target: "scroll::chain_orchestrator", ?block_info, "Skipping consolidation of block as it matches the current chain and is below or equal to the safe head"); + BlockConsolidationOutcome::Skipped(block_info) + } BlockConsolidationAction::UpdateSafeHead(block_info) => { tracing::info!(target: "scroll::chain_orchestrator", ?block_info, "Updating safe head to consolidated block"); self.engine.update_fcs(None, Some(block_info), Some(block_info)).await?; - batch_consolidation_result - .push_block(BlockConsolidationOutcome::Consolidated(block_info)); + BlockConsolidationOutcome::Consolidated(block_info) } BlockConsolidationAction::Reorg(attributes) => { tracing::info!(target: "scroll::chain_orchestrator", block_number = ?attributes.block_number, "Reorging chain to derived block"); @@ -289,22 +414,39 @@ impl< ) .expect("block must be valid"); - // Update the forkchoice state to the new head. - self.engine.update_fcs(Some((&block).into()), None, None).await?; + let result = self.engine.new_payload(&block).await?; + if result.is_invalid() { + return Err(ChainOrchestratorError::InvalidBatch( + (&block).into(), + batch_consolidation_result.batch_info, + )); + } + // Update the forkchoice state to the new head. let block_info: L2BlockInfoWithL1Messages = (&block).into(); - batch_consolidation_result - .push_block(BlockConsolidationOutcome::Reorged(block_info)); + self.engine + .update_fcs( + Some(block_info.block_info), + Some(block_info.block_info), + Some(block_info.block_info), + ) + .await?; + + BlockConsolidationOutcome::Reorged(block_info) } - } + }; + self.event_sender + .as_ref() + .map(|s| s.notify(ChainOrchestratorEvent::BlockConsolidated(outcome.clone()))); + batch_consolidation_result.push_block(outcome); } // Insert the batch consolidation outcome into the database. let tx = self.database.tx_mut().await?; - tx.insert_batch_consolidation_outcome(batch_consolidation_result).await?; + tx.insert_batch_consolidation_outcome(batch_consolidation_result.clone()).await?; tx.commit().await?; - Ok(()) + Ok(Some(ChainOrchestratorEvent::BatchConsolidated(batch_consolidation_result))) } /// Handles an L1 notification. @@ -318,32 +460,53 @@ impl< self.consensus.update_config(update); Ok(None) } - L1Notification::NewBlock(_) => Ok(None), - L1Notification::Finalized(block_number) => self.handle_finalized(*block_number).await, + L1Notification::NewBlock(block_number) => self.handle_l1_new_block(*block_number).await, + L1Notification::Finalized(block_number) => { + self.handle_l1_finalized(*block_number).await + } L1Notification::BatchCommit(batch) => self.handle_batch_commit(batch.clone()).await, L1Notification::L1Message { message, block_number, block_timestamp: _ } => { self.handle_l1_message(message.clone(), *block_number).await } L1Notification::Synced => { tracing::info!(target: "scroll::chain_orchestrator", "L1 is now synced"); + println!("L1 is now synced"); self.sync_state.l1_mut().set_synced(); if self.sync_state.is_synced() { self.consolidate_chain().await?; } + self.event_sender.as_ref().map(|s| s.notify(ChainOrchestratorEvent::L1Synced)); Ok(None) } L1Notification::BatchFinalization { hash: _hash, index, block_number } => { - self.handle_batch_finalization(*index, *block_number).await + self.handle_l1_batch_finalization(*index, *block_number).await } } } + async fn handle_l1_new_block( + &mut self, + block_number: u64, + ) -> Result, ChainOrchestratorError> { + Retry::default() + .retry("handle_new_block", || async { + let tx = self.database.tx_mut().await?; + tx.set_latest_l1_block_number(block_number).await?; + tx.commit().await?; + Ok::<_, ChainOrchestratorError>(()) + }) + .await?; + + Ok(Some(ChainOrchestratorEvent::NewL1Block(block_number))) + } + /// Handles a reorganization event by deleting all indexed data which is greater than the /// provided block number. async fn handle_l1_reorg( &mut self, block_number: u64, ) -> Result, ChainOrchestratorError> { + println!("handling l1 reorg to block number {}", block_number); let metric = self.metrics.get(&ChainOrchestratorItem::L1Reorg).expect("metric exists"); let now = Instant::now(); let UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info } = @@ -371,6 +534,7 @@ impl< let block_hash = self .l2_client .get_block_by_number(block_number.into()) + .full() .await? .expect("L2 head block must exist") .header @@ -382,21 +546,40 @@ impl< }; // TODO: Add retry logic - self.engine.update_fcs(l2_head_block_info, l2_safe_block_info, None).await?; + if l2_head_block_info.is_some() || l2_safe_block_info.is_some() { + self.engine.update_fcs(l2_head_block_info, l2_safe_block_info, None).await?; + } + + if let Some(block_info) = l2_head_block_info { + Retry::default() + .retry("set_l2_head_block_info", || async { + let tx = self.database.tx_mut().await?; + tx.set_l2_head_block_number(block_info.number).await?; + tx.commit().await?; + Ok::<_, ChainOrchestratorError>(()) + }) + .await?; + } metric.task_duration.record(now.elapsed().as_secs_f64()); - Ok(Some(ChainOrchestratorEvent::L1Reorg { + println!("completed the reorg notification"); + + let event = ChainOrchestratorEvent::L1Reorg { l1_block_number, queue_index, l2_head_block_info, l2_safe_block_info, - })) + }; + + println!("event {:?}", event); + + Ok(Some(event)) } /// Handles a finalized event by updating the chain orchestrator L1 finalized block, returning /// the new finalized L2 chain block and the list of finalized batches. - async fn handle_finalized( + async fn handle_l1_finalized( &mut self, block_number: u64, ) -> Result, ChainOrchestratorError> { @@ -482,7 +665,7 @@ impl< } /// Handles a batch finalization event by updating the batch input in the database. - async fn handle_batch_finalization( + async fn handle_l1_batch_finalization( &mut self, batch_index: u64, block_number: u64, @@ -588,20 +771,22 @@ impl< async fn handle_network_event( &mut self, event: ScrollNetworkManagerEvent, - ) -> Result<(), ChainOrchestratorError> { + ) -> Result, ChainOrchestratorError> { match event { ScrollNetworkManagerEvent::NewBlock(block_with_peer) => { - self.handle_block_from_peer(block_with_peer).await? + self.event_sender.as_ref().map(|s| { + s.notify(ChainOrchestratorEvent::NewBlockReceived(block_with_peer.clone())) + }); + Ok(self.handle_block_from_peer(block_with_peer).await?) } } - Ok(()) } /// Handles a new block received from a peer. async fn handle_block_from_peer( &mut self, block_with_peer: NewBlockWithPeer, - ) -> Result<(), ChainOrchestratorError> { + ) -> Result, ChainOrchestratorError> { tracing::debug!(target: "scroll::chain_orchestrator", block_hash = ?block_with_peer.block.header.hash_slow(), block_number = ?block_with_peer.block.number, peer_id = ?block_with_peer.peer_id, "Received new block from peer"); if let Err(err) = @@ -613,9 +798,21 @@ impl< result: Err(err.into()), }); - return Ok(()); + return Ok(Some(ChainOrchestratorEvent::BlockFailedConsensusChecks( + block_with_peer.block.header.hash_slow(), + block_with_peer.peer_id, + ))); } + // We optimistically persist the signature upon passing consensus checks. + let tx = self.database.tx_mut().await?; + tx.insert_signature( + block_with_peer.block.header.hash_slow(), + block_with_peer.signature.clone(), + ) + .await?; + tx.commit().await?; + let received_block_number = block_with_peer.block.number; let received_block_hash = block_with_peer.block.header.hash_slow(); let current_head_number = self.engine.fcs().head_block_info().number; @@ -625,14 +822,17 @@ impl< // than the optimistic sync threshold, we optimistically sync the chain. if received_block_number > current_head_number + self.config.optimistic_sync_threshold() { tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); - self.engine - .optimistic_sync(BlockInfo { - number: received_block_number, - hash: block_with_peer.block.header.hash_slow(), - }) - .await?; + let block_info = BlockInfo { + number: received_block_number, + hash: block_with_peer.block.header.hash_slow(), + }; + self.engine.optimistic_sync(block_info).await?; self.sync_state.l2_mut().set_syncing(); + self.event_sender + .as_ref() + .map(|s| s.notify(ChainOrchestratorEvent::OptimisticSync(block_info))); + // Purge all L1 message to L2 block mappings as they may be invalid after an // optimistic sync. let tx = self.database.tx_mut().await?; @@ -647,15 +847,15 @@ impl< // chain head. let block_number_diff = received_block_number - current_head_number; let new_headers = if received_block_number > current_head_number + 1 { - self.network_client - .get_headers(HeadersRequest::falling( - received_block_hash.into(), - block_number_diff, - )) - .await? - .into_data() + self.block_client + .get_full_block_range(received_block_hash.into(), block_number_diff) + .await + .into_iter() + .rev() + .map(|b| b.into_block()) + .collect() } else { - vec![block_with_peer.block.header.clone()] + vec![block_with_peer.block.clone()] }; // If the first header in the new headers has a parent hash that matches the current @@ -663,56 +863,71 @@ impl< if new_headers.first().expect("at least one header exists").parent_hash == current_head_hash { - self.import_chain(new_headers, block_with_peer).await?; - return Ok(()); + let chain_import = self.import_chain(new_headers, block_with_peer).await?; + return Ok(Some(ChainOrchestratorEvent::ChainExtended(chain_import))); } VecDeque::from(new_headers) } else { + // Assert that we are not reorging below the safe head. + let current_safe_info = self.engine.fcs().safe_block_info(); + if received_block_number <= current_safe_info.number { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + } + // If the block is less than or equal to the current head check if we already have it in // the chain. - let current_chain_block = - self.l2_client.get_block_by_number(received_block_number.into()).await?.ok_or( - ChainOrchestratorError::L2BlockNotFoundInL2Client(received_block_number), - )?; + println!("checking if we already have the block in the chain"); + let current_chain_block = self + .l2_client + .get_block_by_number(received_block_number.into()) + .full() + .await? + .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(received_block_number))?; if current_chain_block.header.hash_slow() == received_block_hash { tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that is already in the chain"); - return Ok(()) + return Ok(Some(ChainOrchestratorEvent::BlockAlreadyKnown( + received_block_hash, + block_with_peer.peer_id, + ))); } - // Assert that we are not reorging below the safe head. - let current_safe_info = self.engine.fcs().safe_block_info(); - if received_block_number <= current_safe_info.number { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); - return Ok(()) + // Check to assert that we have received a newer chain. + let current_head = self + .l2_client + .get_block_by_number(current_head_number.into()) + .full() + .await? + .expect("current head block must exist"); + + // If the timestamp of the received block is less than or equal to the current head, + // we ignore it. + if block_with_peer.block.header.timestamp <= current_head.header.timestamp { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_head_hash = ?current_head.header.hash_slow(), current_head_number = current_head_number, "Received block from peer that is older than the current head - ignoring"); + return Ok(Some(ChainOrchestratorEvent::OldForkReceived { + headers: vec![block_with_peer.block.header], + peer_id: block_with_peer.peer_id, + signature: block_with_peer.signature, + })) } // Check if the parent hash of the received block is in the chain. - let parent_block = - self.l2_client.get_block_by_hash(block_with_peer.block.header.parent_hash).await?; + let parent_block = self + .l2_client + .get_block_by_hash(block_with_peer.block.header.parent_hash) + .full() + .await?; if parent_block.is_some() { tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends an earlier part of the chain"); - return self - .import_chain(vec![block_with_peer.block.header.clone()], block_with_peer) - .await + let chain_import = + self.import_chain(vec![block_with_peer.block.clone()], block_with_peer).await?; + return Ok(Some(ChainOrchestratorEvent::ChainReorged(chain_import))); } - VecDeque::from([block_with_peer.block.header.clone()]) + VecDeque::from([block_with_peer.block.clone()]) }; - // Check to assert that we have received a newer chain. - let current_head = self - .l2_client - .get_block_by_number(current_head_number.into()) - .await? - .expect("current head block must exist"); - if new_headers.back().expect("at least one header exists").timestamp < - current_head.header.timestamp - { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_head_hash = ?current_head.header.hash_slow(), current_head_number = current_head_number, "Received block from peer that is older than the current head - ignoring"); - return Ok(()) - } - // If we reach this point, we have a block that is not in the current chain and does not // extend the current head. This implies a reorg. We attempt to reconcile the fork. let current_safe_head = self.engine.fcs().safe_block_info(); @@ -722,17 +937,21 @@ impl< let parent_hash = new_headers.front().expect("at least one header exists").parent_hash; let parent_number = new_headers.front().expect("at least one header exists").number - 1; let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_head.number); - let headers = self - .network_client - .get_headers(HeadersRequest::falling(parent_hash.into(), fetch_count)) - .await? - .into_data(); + let headers: Vec = self + .block_client + .get_full_block_range(parent_hash.into(), fetch_count) + .await + .into_iter() + .rev() + .map(|b| b.into_block()) + .collect(); let mut index = None; for (i, header) in headers.iter().enumerate() { let current_block = self .l2_client .get_block_by_number(header.number.into()) + .full() .await? .expect("block must exist"); if header.hash_slow() == current_block.header.hash_slow() { @@ -746,8 +965,8 @@ impl< for header in headers.into_iter().skip(index).rev() { new_headers.push_front(header); } - self.import_chain(new_headers.into(), block_with_peer).await?; - return Ok(()); + let chain_import = self.import_chain(new_headers.into(), block_with_peer).await?; + return Ok(Some(ChainOrchestratorEvent::ChainReorged(chain_import))); } else { for header in headers.into_iter().rev() { new_headers.push_front(header); @@ -755,39 +974,28 @@ impl< } } - Ok(()) + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected) } /// Imports a chain of headers into the L2 chain. async fn import_chain( &mut self, - chain: Vec

, + chain: Vec, block_with_peer: NewBlockWithPeer, - ) -> Result<(), ChainOrchestratorError> { + ) -> Result { let chain_head_hash = chain.last().expect("at least one header exists").hash_slow(); let chain_head_number = chain.last().expect("at least one header exists").number; - tracing::info!(target: "scroll::chain_orchestrator", num_blocks = chain.len(), ?chain_head_hash, ?chain_head_number, "Received new block from peer that extends the current head"); - - // Fetch the block bodies for the new headers and construct the full blocks. - let new_bodies = self - .network_client - .get_block_bodies(chain.iter().map(|h| h.hash_slow()).collect()) - .await? - .into_data(); - let new_blocks: Vec = chain - .into_iter() - .zip(new_bodies.into_iter()) - .map(|(header, body)| ScrollBlock { header, body }) - .collect(); + tracing::info!(target: "scroll::chain_orchestrator", num_blocks = chain.len(), ?chain_head_hash, ?chain_head_number, "Received chain from peer"); // If we are in consolidated mode, validate the L1 messages in the new blocks. if self.sync_state.is_synced() { - self.validate_l1_messages(&new_blocks).await?; + self.validate_l1_messages(&chain).await?; } // Validate the new blocks by sending them to the engine. - for block in new_blocks.iter() { + for block in chain.iter() { let status = self.engine.new_payload(&block).await?; + tracing::debug!(target: "scroll::chain_orchestrator", block_number = block.number, block_hash = ?block.hash_slow(), ?status, "New payload status from engine"); if status.is_invalid() { tracing::warn!(target: "scroll::chain_orchestrator", block_number = block.number, block_hash = ?block.hash_slow(), ?status, "Received invalid block from peer"); @@ -815,7 +1023,7 @@ impl< } // Persist the mapping of L1 messages to L2 blocks such that we can react to L1 reorgs. - let blocks = new_blocks.iter().map(|block| block.into()).collect::>(); + let blocks = chain.iter().map(|block| block.into()).collect::>(); let tx = self.database.tx_mut().await?; tx.update_l1_messages_from_l2_blocks(blocks).await?; tx.commit().await?; @@ -844,7 +1052,11 @@ impl< Bytes::copy_from_slice(&block_with_peer.signature.sig_as_bytes()), )); - return Ok(()) + return Ok(ChainImport { + chain, + peer_id: block_with_peer.peer_id, + signature: block_with_peer.signature, + }) } /// Consolidates the chain by validating all unsafe blocks from the current safe head to the @@ -857,18 +1069,22 @@ impl< let mut start_block_number = self.engine.fcs().safe_block_info().number + 1; let end_block_number = self.engine.fcs().head_block_info().number; + println!("consolidating chain from block {} to {}", start_block_number, end_block_number); while start_block_number <= end_block_number { let batch_end = (start_block_number + CONSOLIDATION_BATCH_SIZE - 1).min(end_block_number); + println!("consolidating batch from block {} to {}", start_block_number, batch_end); tracing::trace!(target: "scroll::chain_orchestrator", start_block = ?start_block_number, end_block = ?batch_end, "Consolidating batch of blocks"); // TODO: Make fetching parallel but ensure concurrency limits are respected. let mut blocks_to_validate = vec![]; for block_number in start_block_number..=batch_end { + println!("fetching block {}", block_number); let block = self .l2_client .get_block_by_number(block_number.into()) + .full() .await? .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_number))? .into_consensus() @@ -890,68 +1106,49 @@ impl< &self, blocks: &[ScrollBlock], ) -> Result<(), ChainOrchestratorError> { + let first_block_number = if let Some(block) = blocks.first() { + block.number + } else { + return Ok(()); + }; + + let tx = self.database.tx().await?; + let mut database_txs = + tx.get_l1_messages(Some(L1MessageKey::block_number(first_block_number))).await?; for block in blocks { - let tx = self.database.tx().await?; - let database_txs = - tx.get_l1_messages(Some(L1MessageStart::block_number(block.number))).await?; - let mut chain_txs = + let block_txs = block.body.transactions().filter(|tx| tx.is_l1_message()).map(|tx| tx.trie_hash()); - if let Some(mut txs) = database_txs { - for tx in chain_txs { - let expected_message = txs.next().await.ok_or( - ChainOrchestratorError::L1MessageNotFound(L1MessageStart::Hash(tx)), - )??; + // If we have L1 messages in the block we validate them against the expected L1 + // messages from the database. + for tx in block_txs { + let expected_message = if let Some(txs) = database_txs.as_mut() { + txs.next().await.transpose()? + } else { + None + }; + if let Some(expected_message) = expected_message { if expected_message.transaction.tx_hash() != tx { return Err(ChainOrchestratorError::L1MessageMismatch { expected: expected_message.transaction.tx_hash(), actual: tx, }) } + } else { + self.event_sender.as_ref().map(|sender| { + sender.notify(ChainOrchestratorEvent::L1MessageNotFoundInDatabase( + L1MessageKey::TransactionHash(tx), + )); + }); + return Err(ChainOrchestratorError::L1MessageNotFound( + L1MessageKey::TransactionHash(tx), + )) } - continue; - } - - if chain_txs.next().is_some() { - return Err(ChainOrchestratorError::L1MessageNotFound(L1MessageStart::BlockNumber( - block.number, - ))) } } Ok(()) } - - /// Drives the [`RollupNodeManager`] future until a [`GracefulShutdown`] signal is received. - pub async fn run_until_graceful_shutdown(mut self, mut shutdown: GracefulShutdown) { - loop { - tokio::select! { - Some(command) = self.handle_rx.recv() => { - self.handle_command(command).await; - } - Some(batch) = self.derivation_pipeline.next() => { - if let Err(err) = self.handle_derived_batch(batch).await { - tracing::error!(?err, "Error handling derived batch"); - } - } - Some(event) = self.network.events().next() => { - if let Err(err) = self.handle_network_event(event).await { - tracing::error!(?err, "Error handling network event"); - } - } - Some(notification) = self.l1_notification_rx.recv(), - if self.sync_state.l2().is_synced() && self.derivation_pipeline.len().await == 0 => - { - if let Err(err) = self.handle_l1_notification(notification).await { - tracing::error!(?err, "Error handling L1 notification"); - } - } - _guard = &mut shutdown => { - break; - } - } - } - } } /// Computes the queue hash by taking the previous queue hash and performing a 2-to-1 hash with the @@ -980,7 +1177,7 @@ async fn compute_l1_message_queue_hash( }) .await? .map(|m| m.queue_hash) - .ok_or(DatabaseError::L1MessageNotFound(L1MessageStart::Index(index)))? + .ok_or(DatabaseError::L1MessageNotFound(L1MessageKey::QueueIndex(index)))? .unwrap_or_default() .to_vec(); input.append(&mut l1_message.tx_hash().to_vec()); diff --git a/crates/chain-orchestrator/src/sync.rs b/crates/chain-orchestrator/src/sync.rs index 22c8baaf..e790844a 100644 --- a/crates/chain-orchestrator/src/sync.rs +++ b/crates/chain-orchestrator/src/sync.rs @@ -1,5 +1,5 @@ /// The sync state of the chain orchestrator. -#[derive(Debug, Default, Clone)] +#[derive(Debug, Clone)] pub struct SyncState { /// The sync mode for L1. l1: SyncMode, @@ -7,6 +7,12 @@ pub struct SyncState { l2: SyncMode, } +impl Default for SyncState { + fn default() -> Self { + Self { l1: SyncMode::default(), l2: SyncMode::Synced } + } +} + impl SyncState { /// Returns a reference to the sync mode of L1. pub fn l1(&self) -> &SyncMode { diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index 65b9c85d..faa4a6f2 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -940,13 +940,13 @@ mod test { // Generate and insert a block info as the head. let block_info = BlockInfo::arbitrary(&mut u).unwrap(); - tx.set_l2_head_block_info(block_info).await.unwrap(); + tx.set_l2_head_block_number(block_info.number).await.unwrap(); tx.commit().await.unwrap(); // Retrieve and verify the head block info. let tx = db.tx().await.unwrap(); - let head_block_info = tx.get_l2_head_block_info().await.unwrap().unwrap(); + let head_block_info = tx.get_l2_head_block_number().await.unwrap().unwrap(); - assert_eq!(head_block_info, block_info); + assert_eq!(head_block_info, block_info.number); } } diff --git a/crates/database/db/src/error.rs b/crates/database/db/src/error.rs index 60400d68..d138438c 100644 --- a/crates/database/db/src/error.rs +++ b/crates/database/db/src/error.rs @@ -22,4 +22,7 @@ pub enum DatabaseError { /// The finalized L1 block was not found in database. #[error("Finalized L1 block not found in database")] FinalizedL1BlockNotFound, + /// The latest L1 block was not found in database. + #[error("Latest L1 block not found in database")] + LatestL1BlockNotFound, } diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 690e98c4..47840d73 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -65,6 +65,22 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati Ok(()) } + /// Set the latest L1 block number. + async fn set_latest_l1_block_number(&self, block_number: u64) -> Result<(), DatabaseError> { + tracing::trace!(target: "scroll::db", block_number, "Updating the latest L1 block number in the database."); + let metadata: models::metadata::ActiveModel = + Metadata { key: "l1_latest_block".to_string(), value: block_number.to_string() }.into(); + Ok(models::metadata::Entity::insert(metadata) + .on_conflict( + OnConflict::column(models::metadata::Column::Key) + .update_column(models::metadata::Column::Value) + .to_owned(), + ) + .exec(self.get_connection()) + .await + .map(|_| ())?) + } + /// Set the finalized L1 block number. async fn set_finalized_l1_block_number(&self, block_number: u64) -> Result<(), DatabaseError> { tracing::trace!(target: "scroll::db", block_number, "Updating the finalized L1 block number in the database."); @@ -82,14 +98,11 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati .map(|_| ())?) } - /// Set the L2 head block info. - async fn set_l2_head_block_info(&self, block_info: BlockInfo) -> Result<(), DatabaseError> { - tracing::trace!(target: "scroll::db", ?block_info, "Updating the L2 head block info in the database."); - let metadata: models::metadata::ActiveModel = Metadata { - key: "l2_head_block".to_string(), - value: serde_json::to_string(&block_info)?, - } - .into(); + /// Set the L2 head block number. + async fn set_l2_head_block_number(&self, number: u64) -> Result<(), DatabaseError> { + tracing::trace!(target: "scroll::db", ?number, "Updating the L2 head block number in the database."); + let metadata: models::metadata::ActiveModel = + Metadata { key: "l2_head_block".to_string(), value: number.to_string() }.into(); Ok(models::metadata::Entity::insert(metadata) .on_conflict( OnConflict::column(models::metadata::Column::Key) @@ -403,6 +416,11 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati BlockConsolidationOutcome::Consolidated(block_info) => { self.insert_block(block_info, outcome.batch_info).await?; } + BlockConsolidationOutcome::Skipped(block_info) => { + // No action needed, the block has already been previously consolidated however + // we will insert it again defensively + self.insert_block(block_info, outcome.batch_info).await?; + } BlockConsolidationOutcome::Reorged(block_info) => { self.insert_block(block_info.block_info, outcome.batch_info).await?; self.update_l1_messages_with_l2_block(block_info).await?; @@ -453,6 +471,14 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati None }; + // delete mapping for l1 messages that were included in unsafe blocks after the reorg point + if l2_head_block_number.is_some() { + self.purge_l1_message_to_l2_block_mappings( + l2_head_block_number.map(|x| x.saturating_add(1)), + ) + .await?; + } + // commit the transaction Ok(UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info }) } @@ -499,6 +525,18 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .map(|x| x.map(Into::into))?) } + /// Get the latest L1 block number from the database. + async fn get_latest_l1_block_number(&self) -> Result, DatabaseError> { + Ok(models::metadata::Entity::find() + .filter(models::metadata::Column::Key.eq("l1_latest_block")) + .select_only() + .column(models::metadata::Column::Value) + .into_tuple::() + .one(self.get_connection()) + .await + .map(|x| x.and_then(|x| x.parse::().ok()))?) + } + /// Get the finalized L1 block number from the database. async fn get_finalized_l1_block_number(&self) -> Result, DatabaseError> { Ok(models::metadata::Entity::find() @@ -512,7 +550,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { } /// Get the latest L2 head block info. - async fn get_l2_head_block_info(&self) -> Result, DatabaseError> { + async fn get_l2_head_block_number(&self) -> Result, DatabaseError> { Ok(models::metadata::Entity::find() .filter(models::metadata::Column::Key.eq("l2_head_block")) .select_only() @@ -583,8 +621,15 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { Option> + 'a>, DatabaseError, > { - let queue_index = match start { - Some(L1MessageKey::QueueIndex(i)) => Ok::<_, DatabaseError>(Some(i)), + match start { + Some(L1MessageKey::QueueIndex(queue_index)) => Ok(Some( + models::l1_message::Entity::find() + .filter(models::l1_message::Column::QueueIndex.gte(queue_index)) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )), Some(L1MessageKey::TransactionHash(ref h)) => { // Lookup message by hash let record = models::l1_message::Entity::find() @@ -594,8 +639,14 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .ok_or_else(|| { DatabaseError::L1MessageNotFound(L1MessageKey::TransactionHash(*h)) })?; - - Ok(Some(record.queue_index as u64)) + Ok(Some( + models::l1_message::Entity::find() + .filter(models::l1_message::Column::QueueIndex.gte(record.queue_index)) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } Some(L1MessageKey::QueueHash(ref h)) => { // Lookup message by queue hash @@ -609,31 +660,45 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .await? .ok_or_else(|| DatabaseError::L1MessageNotFound(L1MessageKey::QueueHash(*h)))?; - Ok(Some(record.queue_index as u64)) + Ok(Some( + models::l1_message::Entity::find() + .filter(models::l1_message::Column::QueueIndex.gte(record.queue_index)) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } - Some(L1MessageStart::BlockNumber(block_number)) => { - let exact_match = models::l1_message::Entity::find() - .filter(models::l1_message::Column::L1BlockNumber.eq(block_number as i64)) + Some(L1MessageKey::BlockNumber(block_number)) => { + if let Some(record) = models::l1_message::Entity::find() + .filter(models::l1_message::Column::L2BlockNumber.lt(block_number as i64)) + .order_by_desc(models::l1_message::Column::L2BlockNumber) .order_by_desc(models::l1_message::Column::QueueIndex) - .into_tuple::() .one(self.get_connection()) - .await?; - - if let Some(queue_index) = exact_match { - Ok(Some(queue_index as u64)) + .await? + { + Ok(Some( + models::l1_message::Entity::find() + .filter( + // We add 1 to the queue index to constrain across block boundaries + models::l1_message::Column::QueueIndex.gte(record.queue_index + 1), + ) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } else { - // If no exact match is found, find the last message before the block number - Ok(models::l1_message::Entity::find() - .filter(models::l1_message::Column::L1BlockNumber.lt(block_number as i64)) - .order_by_desc(models::l1_message::Column::L1BlockNumber) - .order_by_desc(models::l1_message::Column::QueueIndex) - .into_tuple::() - .one(self.get_connection()) - .await? - .map(|x| x as u64)) + Ok(Some( + models::l1_message::Entity::find() + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } } - Some(L1MessageStart::NotIncluded(NotIncludedStart::Finalized)) => { + Some(L1MessageKey::NotIncluded(NotIncludedStart::Finalized)) => { let finalized_block_number = self .get_finalized_l1_block_number() .await? @@ -644,21 +709,22 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .lte(finalized_block_number as i64), ) .add(models::l1_message::Column::L2BlockNumber.is_null()); - Ok(models::l1_message::Entity::find() - .filter(condition) - .order_by_desc(models::l1_message::Column::QueueIndex) - .into_tuple::() - .one(self.get_connection()) - .await? - .map(|x| x as u64)) + Ok(Some( + models::l1_message::Entity::find() + .filter(condition) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } - Some(L1MessageStart::NotIncluded(NotIncludedStart::BlockDepth(depth))) => { - // TODO: USE LATEST BLOCK NUMBER NOT FINALIZED - let finalized_block_number = self - .get_finalized_l1_block_number() + Some(L1MessageKey::NotIncluded(NotIncludedStart::BlockDepth(depth))) => { + let latest_block_number = self + .get_latest_l1_block_number() .await? - .ok_or(DatabaseError::FinalizedL1BlockNotFound)?; - let target_block_number = finalized_block_number.checked_sub(depth); + .ok_or(DatabaseError::LatestL1BlockNotFound)?; + + let target_block_number = latest_block_number.checked_sub(depth); if let Some(target_block_number) = target_block_number { let condition = Condition::all() .add( @@ -666,31 +732,26 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .lte(target_block_number as i64), ) .add(models::l1_message::Column::L2BlockNumber.is_null()); - Ok(models::l1_message::Entity::find() - .filter(condition) - .order_by_desc(models::l1_message::Column::QueueIndex) - .into_tuple::() - .one(self.get_connection()) - .await? - .map(|x| x as u64)) + Ok(Some( + models::l1_message::Entity::find() + .filter(condition) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } else { Ok(None) } } - None => Ok(Some(0)), - }?; - - let queue_index = - if let Some(queue_index) = queue_index { queue_index } else { return Ok(None) }; - - Ok(Some( - models::l1_message::Entity::find() - .filter(models::l1_message::Column::QueueIndex.gte(queue_index)) - .order_by_asc(models::l1_message::Column::QueueIndex) - .stream(self.get_connection()) - .await? - .map(|res| Ok(res.map(Into::into)?)), - )) + None => Ok(Some( + models::l1_message::Entity::find() + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )), + } } /// Get the extra data for the provided block number. @@ -877,6 +938,12 @@ pub enum NotIncludedStart { BlockDepth(u64), } +fn map_l1_message_result( + res: Result, +) -> Result { + Ok(res.map(Into::into)?) +} + impl fmt::Display for L1MessageKey { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { diff --git a/crates/database/migration/src/lib.rs b/crates/database/migration/src/lib.rs index 3a5f2465..bd05f56c 100644 --- a/crates/database/migration/src/lib.rs +++ b/crates/database/migration/src/lib.rs @@ -13,6 +13,7 @@ mod m20250904_175949_block_signature; mod m20250923_135359_add_index_block_hash; mod m20250929_161536_add_additional_indexes; mod m20251001_125444_add_index_processed; +mod m20251005_160938_add_initial_l1_block_numbers; mod migration_info; pub use migration_info::{ @@ -38,6 +39,7 @@ impl MigratorTrait for Migrator { Box::new(m20250923_135359_add_index_block_hash::Migration), Box::new(m20250929_161536_add_additional_indexes::Migration), Box::new(m20251001_125444_add_index_processed::Migration), + Box::new(m20251005_160938_add_initial_l1_block_numbers::Migration), ] } } diff --git a/crates/database/migration/src/m20251005_160938_add_initial_l1_block_numbers.rs b/crates/database/migration/src/m20251005_160938_add_initial_l1_block_numbers.rs new file mode 100644 index 00000000..b8c1b340 --- /dev/null +++ b/crates/database/migration/src/m20251005_160938_add_initial_l1_block_numbers.rs @@ -0,0 +1,39 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + let db = manager.get_connection(); + + // Insert both keys if they don't already exist + db.execute_unprepared( + r#" + INSERT INTO metadata (key, value) + VALUES + ('l1_finalized_block', '0'), + ('l1_latest_block', '0') + ON CONFLICT(key) DO NOTHING; + "#, + ) + .await?; + + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + let db = manager.get_connection(); + + db.execute_unprepared( + r#" + DELETE FROM metadata + WHERE key IN ('l1_finalized_block', 'l1_latest_block'); + "#, + ) + .await?; + + Ok(()) + } +} diff --git a/crates/derivation-pipeline/src/new.rs b/crates/derivation-pipeline/src/new.rs index be621c06..55e5c2c4 100644 --- a/crates/derivation-pipeline/src/new.rs +++ b/crates/derivation-pipeline/src/new.rs @@ -104,6 +104,7 @@ where Poll::Ready(Some(Err((batch_info, err)))) => { tracing::error!(target: "scroll::derivation_pipeline", ?batch_info, ?err, "Failed to derive payload attributes"); guard.push_front(this.derivation_future(batch_info.clone())); + cx.waker().wake_by_ref(); return Poll::Pending } // If the derivation succeeded then return the attributes. diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index 4fca7863..4dc2f8ee 100644 --- a/crates/node/Cargo.toml +++ b/crates/node/Cargo.toml @@ -37,6 +37,7 @@ reth-primitives-traits.workspace = true reth-scroll-engine-primitives.workspace = true reth-scroll-evm.workspace = true reth-scroll-cli.workspace = true +reth-scroll-consensus.workspace = true reth-scroll-primitives.workspace = true reth-scroll-chainspec.workspace = true reth-scroll-node.workspace = true @@ -52,6 +53,7 @@ reth-node-core.workspace = true reth-node-types.workspace = true reth-network.workspace = true reth-network-api.workspace = true +reth-network-p2p.workspace = true reth-revm.workspace = true reth-rpc-api.workspace = true reth-rpc-eth-api.workspace = true @@ -68,6 +70,7 @@ rollup-node-providers.workspace = true rollup-node-sequencer.workspace = true rollup-node-signer.workspace = true rollup-node-watcher.workspace = true +scroll-derivation-pipeline.workspace = true # AWS dependencies aws-config = "1.8.0" @@ -80,7 +83,6 @@ reth-engine-local = { workspace = true, optional = true } reth-provider = { workspace = true, optional = true } reth-rpc-server-types = { workspace = true, optional = true } scroll-alloy-rpc-types-engine = { workspace = true, optional = true } -scroll-derivation-pipeline = { workspace = true, optional = true } scroll-alloy-rpc-types.workspace = true scroll-db.workspace = true @@ -140,6 +142,5 @@ test-utils = [ "reth-rpc-server-types", "scroll-alloy-rpc-types-engine", "alloy-rpc-types-engine", - "scroll-derivation-pipeline", "reth-primitives-traits/test-utils", ] diff --git a/crates/node/src/add_ons/rollup.rs b/crates/node/src/add_ons/rollup.rs index 851aef40..93e88461 100644 --- a/crates/node/src/add_ons/rollup.rs +++ b/crates/node/src/add_ons/rollup.rs @@ -61,14 +61,14 @@ impl RollupManagerAddOn { ChainConfig + ScrollHardforks + IsDevChain, N::Network: NetworkProtocols + FullNetwork, { - let (rnm, handle, l1_notification_tx) = self + let (chain_orchestrator, handle, l1_notification_tx) = self .config .build((&ctx).into(), self.scroll_wire_event, rpc.rpc_server_handles) .await?; ctx.node .task_executor() - .spawn_critical_with_graceful_shutdown_signal("rollup_node_manager", |shutdown| { - rnm.run_until_graceful_shutdown(shutdown) + .spawn_critical_with_shutdown_signal("rollup_node_manager", |shutdown| { + chain_orchestrator.run_until_shutdown(shutdown) }); Ok((handle, l1_notification_tx)) } diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 5068d3b8..3f10e898 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -19,11 +19,13 @@ use clap::ArgAction; use reth_chainspec::EthChainSpec; use reth_network::NetworkProtocols; use reth_network_api::FullNetwork; +use reth_network_p2p::FullBlockClient; use reth_node_builder::{rpc::RethRpcServerHandles, NodeConfig as RethNodeConfig}; use reth_node_core::primitives::BlockHeader; use reth_scroll_chainspec::{ ChainConfig, ScrollChainConfig, ScrollChainSpec, SCROLL_FEE_VAULT_ADDRESS, }; +use reth_scroll_consensus::ScrollBeaconConsensus; use reth_scroll_node::ScrollNetworkPrimitives; use rollup_node_chain_orchestrator::{ ChainOrchestrator, ChainOrchestratorConfig, ChainOrchestratorHandle, Consensus, NoopConsensus, @@ -163,6 +165,7 @@ impl ScrollRollupNodeConfig { N, impl ScrollHardforks + EthChainSpec + IsDevChain + Clone + 'static, impl L1Provider + Clone, + impl L1MessageProvider + Clone, impl Provider + Clone, impl ScrollEngineApi, >, @@ -268,19 +271,32 @@ impl ScrollRollupNodeConfig { let (startup_safe_block, l1_start_block_number) = tx.prepare_on_startup(chain_spec.genesis_hash()).await?; tx.commit().await?; - if let Some(block_info) = startup_safe_block { - fcs.update_safe_block_info(block_info); - } else { - fcs.update_safe_block_info(BlockInfo { - hash: genesis_hash_from_chain_spec(chain_spec.clone()).unwrap(), - number: 0, - }); - } + // if let Some(block_info) = startup_safe_block { + // fcs.update(None, Some(block_info), Some(block_info))?; + // } else { + // fcs.update( + // None, + // Some(BlockInfo { + // hash: genesis_hash_from_chain_spec(chain_spec.clone()).unwrap(), + // number: 0, + // }), + // None, + // )?; + // } // Update the head block info if available and ahead of finalized. - if let Some(latest_block) = db.tx().await?.get_l2_head_block_info().await? { - if latest_block > *fcs.finalized_block_info() { - fcs.update_head_block_info(latest_block); + if let Some(latest_block_number) = db.tx().await?.get_l2_head_block_number().await? { + if latest_block_number > fcs.finalized_block_info().number { + let block = l2_provider + .get_block(latest_block_number.into()) + .full() + .await? + .expect("latest block from db should exist") + .into_consensus() + .map_transactions(|tx| tx.inner.into_inner()); + let block_info: BlockInfo = (&block).into(); + + fcs.update(Some(block_info), None, None)?; } } @@ -303,6 +319,7 @@ impl ScrollRollupNodeConfig { td_constant(chain_spec.chain().named()), authorized_signer, ); + tokio::spawn(scroll_network_manager); tracing::info!(target: "scroll::node::args", fcs = ?fcs, payload_building_duration = ?self.sequencer_args.payload_building_duration, "Starting engine driver"); let engine = Engine::new(Arc::new(engine_api), fcs); @@ -361,9 +378,6 @@ impl ScrollRollupNodeConfig { // Construct the Sequencer. let chain_config = chain_spec.chain_config(); - let latest_l1_message = db.tx().await?.get_latest_executed_l1_message().await?; - let sequencer_l1_messages_queue_index = - latest_l1_message.map(|msg| msg.transaction.queue_index + 1).unwrap_or_default(); let sequencer = if self.sequencer_args.sequencer_enabled { let args = &self.sequencer_args; let config = SequencerConfig { @@ -401,11 +415,14 @@ impl ScrollRollupNodeConfig { }; // Instantiate the chain orchestrator - let block_client = scroll_network_handle - .inner() - .fetch_client() - .await - .expect("failed to fetch block client"); + let block_client = FullBlockClient::new( + scroll_network_handle + .inner() + .fetch_client() + .await + .expect("failed to fetch block client"), + Arc::new(ScrollBeaconConsensus::new(chain_spec.clone())), + ); let l1_v2_message_queue_start_index = l1_v2_message_queue_start_index(chain_spec.chain().named()); let config: ChainOrchestratorConfig> = ChainOrchestratorConfig::new( @@ -424,12 +441,14 @@ impl ScrollRollupNodeConfig { let (chain_orchestrator, handle) = ChainOrchestrator::new( db.clone(), config, - block_client, + Arc::new(block_client), l2_provider, l1_notification_rx.expect("L1 notification receiver should be set"), scroll_network_handle.into_scroll_network().await, consensus, engine, + sequencer, + signer, derivation_pipeline, ) .await?; diff --git a/crates/node/src/test_utils.rs b/crates/node/src/test_utils.rs index e1902cf3..b9c7aed5 100644 --- a/crates/node/src/test_utils.rs +++ b/crates/node/src/test_utils.rs @@ -186,8 +186,8 @@ pub fn default_sequencer_test_scroll_rollup_node_config() -> ScrollRollupNodeCon }, sequencer_args: SequencerArgs { sequencer_enabled: true, - auto_start: true, - block_time: 0, + auto_start: false, + block_time: 100, payload_building_duration: 40, fee_recipient: Default::default(), l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 49148cd2..852d3512 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -18,6 +18,10 @@ use reth_scroll_chainspec::{ScrollChainSpec, SCROLL_DEV, SCROLL_MAINNET, SCROLL_ use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; use reth_storage_api::BlockReader; +use reth_tasks::{ + shutdown::{self, signal as shutdown_signal}, + TaskExecutor, +}; use reth_tokio_util::EventStream; use rollup_node::{ constants::SCROLL_GAS_LIMIT, @@ -31,7 +35,7 @@ use rollup_node::{ ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, }; use rollup_node_chain_orchestrator::{ChainOrchestratorEvent, ChainOrchestratorHandle}; -use rollup_node_primitives::{sig_encode_hash, BatchCommitData, ConsensusUpdate}; +use rollup_node_primitives::{sig_encode_hash, BatchCommitData, BlockInfo, ConsensusUpdate}; use rollup_node_sequencer::L1MessageInclusionMode; use rollup_node_watcher::L1Notification; use scroll_alloy_consensus::TxL1Message; @@ -70,7 +74,7 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { chain_orchestrator_args: ChainOrchestratorArgs::default(), sequencer_args: SequencerArgs { sequencer_enabled: true, - auto_start: true, + auto_start: false, block_time: 0, l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), allow_empty_blocks: true, @@ -125,7 +129,7 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { wait_n_events( &mut events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { assert_eq!(block.body.transactions.len(), 1); assert_eq!( block.body.transactions[0].as_l1_message().unwrap().inner(), @@ -165,7 +169,7 @@ async fn can_sequence_and_gossip_blocks() { chain_orchestrator_args: ChainOrchestratorArgs::default(), sequencer_args: SequencerArgs { sequencer_enabled: true, - auto_start: true, + auto_start: false, block_time: 0, l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), payload_building_duration: 1000, @@ -199,7 +203,7 @@ async fn can_sequence_and_gossip_blocks() { wait_n_events( &mut sequencer_events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { assert_eq!(block.body.transactions.len(), 1); true } else { @@ -214,7 +218,7 @@ async fn can_sequence_and_gossip_blocks() { wait_n_events( &mut follower_events, |e| { - if let RollupManagerEvent::NewBlockReceived(block_with_peer) = e { + if let ChainOrchestratorEvent::NewBlockReceived(block_with_peer) = e { assert_eq!(block_with_peer.block.body.transactions.len(), 1); true } else { @@ -228,21 +232,10 @@ async fn can_sequence_and_gossip_blocks() { // assert that a chain extension is triggered on the follower node wait_n_events( &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::ChainExtended(_) - ) - ) - }, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 1, ) .await; - - // assert that the block was successfully imported by the follower node - wait_n_events(&mut follower_events, |e| matches!(e, RollupManagerEvent::BlockImported(_)), 1) - .await; } #[tokio::test] @@ -266,7 +259,7 @@ async fn can_penalize_peer_for_invalid_block() { engine_driver_args: EngineDriverArgs::default(), sequencer_args: SequencerArgs { sequencer_enabled: true, - auto_start: true, + auto_start: false, block_time: 0, l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), payload_building_duration: 1000, @@ -376,16 +369,18 @@ async fn can_penalize_peer_for_invalid_signature() -> eyre::Result<()> { node0_rmn_handle.build_block().await; // Wait for the sequencer to build the block - let block0 = if let Some(RollupManagerEvent::BlockSequenced(block)) = node0_events.next().await - { - assert_eq!(block.body.transactions.len(), 0, "Block should have no transactions"); - block - } else { - panic!("Failed to receive block from sequencer"); - }; + let block0 = + if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = node0_events.next().await { + assert_eq!(block.body.transactions.len(), 0, "Block should have no transactions"); + block + } else { + panic!("Failed to receive block from sequencer"); + }; // Node1 should receive and accept the valid block - if let Some(RollupManagerEvent::NewBlockReceived(block_with_peer)) = node1_events.next().await { + if let Some(ChainOrchestratorEvent::NewBlockReceived(block_with_peer)) = + node1_events.next().await + { assert_eq!(block0.hash_slow(), block_with_peer.block.hash_slow()); // Verify the signature is from the authorized signer @@ -397,7 +392,7 @@ async fn can_penalize_peer_for_invalid_signature() -> eyre::Result<()> { } // Wait for successful import - wait_n_events(&mut node1_events, |e| matches!(e, RollupManagerEvent::BlockImported(_)), 1) + wait_n_events(&mut node1_events, |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 1) .await; // === Phase 2: Create and send valid block with unauthorized signer signature === @@ -422,7 +417,7 @@ async fn can_penalize_peer_for_invalid_signature() -> eyre::Result<()> { // Node1 should receive and process the invalid block wait_for_event_predicate_5s(&mut node1_events, |e| { - if let RollupManagerEvent::NewBlockReceived(block_with_peer) = e { + if let ChainOrchestratorEvent::NewBlockReceived(block_with_peer) = e { assert_eq!(block1.hash_slow(), block_with_peer.block.hash_slow()); // Verify the signature is from the unauthorized signer @@ -486,10 +481,7 @@ async fn can_forward_tx_to_sequencer() { // create 2 nodes let mut sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); - sequencer_node_config.sequencer_args.block_time = 0; - sequencer_node_config.network_args.enable_eth_scroll_wire_bridge = false; let mut follower_node_config = default_test_scroll_rollup_node_config(); - follower_node_config.network_args.enable_eth_scroll_wire_bridge = false; // Create the chain spec for scroll mainnet with Euclid v2 activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); @@ -523,15 +515,19 @@ async fn can_forward_tx_to_sequencer() { sequencer_rnm_handle.build_block().await; // wait for the sequencer to build a block with no transactions - if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { + if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { assert_eq!(block.body.transactions.len(), 0); } else { panic!("Failed to receive block from rollup node"); } // assert that the follower node has received the block from the peer - wait_n_events(&mut follower_events, |e| matches!(e, RollupManagerEvent::BlockImported(_)), 1) - .await; + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), + 1, + ) + .await; // inject a transaction into the pool of the follower node let tx = generate_tx(wallet).await; @@ -546,7 +542,7 @@ async fn can_forward_tx_to_sequencer() { wait_n_events( &mut sequencer_events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { assert_eq!(block.header.number, 2); assert_eq!(block.body.transactions.len(), 1); return true @@ -557,14 +553,11 @@ async fn can_forward_tx_to_sequencer() { ) .await; - // skip the chain committed event - let _ = follower_events.next().await; - // assert that the follower node has received the block from the peer wait_n_events( &mut follower_events, |e| { - if let RollupManagerEvent::NewBlockReceived(block_with_peer) = e { + if let ChainOrchestratorEvent::NewBlockReceived(block_with_peer) = e { assert_eq!(block_with_peer.block.body.transactions.len(), 1); true } else { @@ -578,29 +571,7 @@ async fn can_forward_tx_to_sequencer() { // assert that a chain extension is triggered on the follower node wait_n_events( &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::ChainExtended(_) - ) - ) - }, - 1, - ) - .await; - - // assert that the block was successfully imported by the follower node - wait_n_events( - &mut follower_events, - |e| { - if let RollupManagerEvent::BlockImported(block) = e { - assert_eq!(block.body.transactions.len(), 1); - true - } else { - false - } - }, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 1, ) .await; @@ -646,15 +617,19 @@ async fn can_sequence_and_gossip_transactions() { sequencer_rnm_handle.build_block().await; // wait for the sequencer to build a block with no transactions - if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { + if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { assert_eq!(block.body.transactions.len(), 0); } else { panic!("Failed to receive block from rollup node"); } // assert that the follower node has received the block from the peer - wait_n_events(&mut follower_events, |e| matches!(e, RollupManagerEvent::BlockImported(_)), 1) - .await; + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), + 1, + ) + .await; // inject a transaction into the pool of the follower node let tx = generate_tx(wallet).await; @@ -669,7 +644,7 @@ async fn can_sequence_and_gossip_transactions() { wait_n_events( &mut sequencer_events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { assert_eq!(block.header.number, 2); assert_eq!(block.body.transactions.len(), 1); return true @@ -680,11 +655,8 @@ async fn can_sequence_and_gossip_transactions() { ) .await; - // skip the chain committed event - let _ = follower_events.next().await; - // assert that the follower node has received the block from the peer - if let Some(RollupManagerEvent::NewBlockReceived(block_with_peer)) = + if let Some(ChainOrchestratorEvent::NewBlockReceived(block_with_peer)) = follower_events.next().await { assert_eq!(block_with_peer.block.body.transactions.len(), 1); @@ -692,14 +664,13 @@ async fn can_sequence_and_gossip_transactions() { panic!("Failed to receive block from rollup node"); } - // skip the chain extension event - let _ = follower_events.next().await; - // assert that the block was successfully imported by the follower node wait_n_events( &mut follower_events, |e| { - if let RollupManagerEvent::BlockImported(block) = e { + if let ChainOrchestratorEvent::ChainExtended(chain) = e { + assert_eq!(chain.chain.len(), 1); + let block = chain.chain.first().unwrap(); assert_eq!(block.body.transactions.len(), 1); true } else { @@ -743,7 +714,7 @@ async fn can_bridge_blocks() { .with_pow() .build_with_noop_provider(chain_spec.clone()); let scroll_wire_config = ScrollWireConfig::new(true); - let mut scroll_network = scroll_network::ScrollNetworkManager::new( + let (scroll_network, scroll_network_handle) = scroll_network::ScrollNetworkManager::new( chain_spec.clone(), network_config, scroll_wire_config, @@ -752,7 +723,8 @@ async fn can_bridge_blocks() { None, ) .await; - let scroll_network_handle = scroll_network.handle(); + tokio::spawn(scroll_network); + let mut scroll_network_events = scroll_network_handle.event_listener().await; // Connect the scroll-wire node to the scroll NetworkManager. bridge_node.network.add_peer(scroll_network_handle.local_node_record()).await; @@ -804,7 +776,7 @@ async fn can_bridge_blocks() { peer_id, block, signature, - })) = scroll_network.next().await + })) = scroll_network_events.next().await { assert_eq!(peer_id, bridge_peer_id); assert_eq!(block.hash_slow(), block_1_hash); @@ -821,7 +793,7 @@ async fn can_bridge_blocks() { /// Test that when the rollup node manager is shutdown, it consolidates the most recent batch /// on startup. #[tokio::test] -async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<()> { +async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<()> { reth_tracing::init_test_tracing(); let chain_spec = (*SCROLL_MAINNET).clone(); @@ -841,7 +813,7 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: config.hydrate(node.inner.config.clone()).await?; let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (rnm, handle, l1_notification_tx) = config + let (chain_orchestrator, handle, l1_notification_tx) = config .clone() .build( RollupNodeContext::new( @@ -856,15 +828,26 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: .await?; // Spawn a task that constantly polls the rnm to make progress. - let rnm_join_handle = tokio::spawn(async { - let _ = rnm.await; + let (signal, shutdown) = shutdown_signal(); + tokio::spawn(async { + let (_signal, inner) = shutdown_signal(); + let chain_orchestrator = chain_orchestrator.run_until_shutdown(inner); + tokio::select! { + biased; + + _ = shutdown => { + return; + }, + _ = chain_orchestrator => {}, + } }); // Request an event stream from the rollup node manager. let mut rnm_events = handle.get_event_listener().await?; // Extract the L1 notification sender - let l1_notification_tx = l1_notification_tx.unwrap(); + let l1_notification_tx: tokio::sync::mpsc::Sender> = + l1_notification_tx.unwrap(); // Load test batches let raw_calldata_0 = read_to_bytes("./tests/testdata/batch_0_calldata.bin")?; @@ -902,22 +885,13 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: l1_notification_tx.send(Arc::new(L1Notification::Finalized(batch_0_data.block_number))).await?; // Lets iterate over all blocks expected to be derived from the first batch commit. - let mut i = 1; - loop { - let block_info = loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = - rnm_events.next().await - { - assert!(consolidation_outcome.block_info().block_info.number == i); - break consolidation_outcome.block_info().block_info; - } - }; - - if block_info.number == 4 { - break - }; - i += 1; - } + let consolidation_outcome = loop { + let event = rnm_events.next().await; + if let Some(ChainOrchestratorEvent::BatchConsolidated(consolidation_outcome)) = event { + break consolidation_outcome; + } + }; + assert_eq!(consolidation_outcome.blocks.len(), 4, "Expected 4 blocks to be consolidated"); // Now we send the second batch commit and finalize it. l1_notification_tx.send(Arc::new(L1Notification::BatchCommit(batch_1_data.clone()))).await?; @@ -937,11 +911,11 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: let mut i = 5; let hash = loop { let hash = loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = + if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - assert!(consolidation_outcome.block_info().block_info.number == i); - break consolidation_outcome.block_info().block_info.hash; + assert_eq!(consolidation_outcome.block_info().number, i); + break consolidation_outcome.block_info().hash; } }; if i == 40 { @@ -958,17 +932,25 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: rpc.block_by_number(BlockNumberOrTag::Latest, false).await?.expect("head block must exist"); // Assert that the safe block hash is the same as the hash of the last consolidated block. - assert_eq!(safe_block_hash.header.hash, hash, "Safe block hash does not match expected hash"); - assert_eq!(head_block_hash.header.hash, hash, "Head block hash does not match expected hash"); + assert_eq!( + safe_block_hash.header.hash, hash, + "Safe block hash does not match expected + hash" + ); + assert_eq!( + head_block_hash.header.hash, hash, + "Head block hash does not match + expected hash" + ); // Simulate a shutdown of the rollup node manager by dropping it. - rnm_join_handle.abort(); + signal.fire(); drop(l1_notification_tx); drop(rnm_events); // Start the RNM again. let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (rnm, handle, l1_notification_tx) = config + let (chain_orchestrator, handle, l1_notification_tx) = config .clone() .build( RollupNodeContext::new( @@ -984,8 +966,18 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: let l1_notification_tx = l1_notification_tx.unwrap(); // Spawn a task that constantly polls the rnm to make progress. + let (_signal, shutdown) = shutdown_signal(); tokio::spawn(async { - let _ = rnm.await; + let (_signal, inner) = shutdown_signal(); + let chain_orchestrator = chain_orchestrator.run_until_shutdown(inner); + tokio::select! { + biased; + + _ = shutdown => { + return; + }, + _ = chain_orchestrator => {}, + } }); // Request an event stream from the rollup node manager. @@ -1005,7 +997,7 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: // Lets fetch the first consolidated block event - this should be the first block of the batch. let l2_block = loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = + if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { break consolidation_outcome.block_info().clone(); @@ -1015,19 +1007,16 @@ async fn graceful_shutdown_consolidates_most_recent_batch_on_startup() -> eyre:: // One issue #273 is completed, we will again have safe blocks != finalized blocks, and this // should be changed to 1. Assert that the consolidated block is the first block of the // batch. - assert_eq!( - l2_block.block_info.number, 5, - "Consolidated block number does not match expected number" - ); + assert_eq!(l2_block.number, 5, "Consolidated block number does not match expected number"); // Lets now iterate over all remaining blocks expected to be derived from the second batch // commit. for i in 6..=57 { loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = + if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - assert!(consolidation_outcome.block_info().block_info.number == i); + assert!(consolidation_outcome.block_info().number == i); break; } } @@ -1083,7 +1072,7 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( config.hydrate(node.inner.config.clone()).await?; let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (mut rnm, handle, _) = config + let (rnm, handle, _) = config .clone() .build( RollupNodeContext::new( @@ -1096,6 +1085,8 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( node.inner.add_ons_handle.rpc_handle.rpc_server_handles.clone(), ) .await?; + let (signal, shutdown) = shutdown_signal(); + let mut rnm = Box::pin(rnm.run_until_shutdown(shutdown)); // Poll the rnm until we get an event stream listener. let mut rnm_events_fut = pin!(handle.get_event_listener()); @@ -1115,7 +1106,7 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( handle.build_block().await; let block_number = loop { let _ = rnm.poll_unpin(&mut Context::from_waker(noop_waker_ref())); - if let Poll::Ready(Some(RollupManagerEvent::BlockSequenced(block))) = + if let Poll::Ready(Some(ChainOrchestratorEvent::BlockSequenced(block))) = rnm_events.poll_next_unpin(&mut Context::from_waker(noop_waker_ref())) { break block.header.number @@ -1133,28 +1124,15 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( .map(|b| BlockInfo { number: b.number, hash: b.hash_slow() }) .expect("block exists"); - // Build one block, and only wait for the block sequenced event. - handle.build_block().await; - loop { - let _ = rnm.poll_unpin(&mut Context::from_waker(noop_waker_ref())); - if let Poll::Ready(Some(RollupManagerEvent::BlockSequenced(_))) = - rnm_events.poll_next_unpin(&mut Context::from_waker(noop_waker_ref())) - { - break - } - tokio::time::sleep(Duration::from_millis(10)).await; - } - // At this point, we have the EN synced to a block > 10 and the RNM has sequenced one additional // block, validating it with the EN, but not updating the last sequenced block in the DB. - // Simulate a shutdown of the rollup node manager by dropping it. drop(rnm_events); drop(rnm); // Start the RNM again. let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (_rnm, handle, _) = config + let (rnm, handle, _) = config .clone() .build( RollupNodeContext::new( @@ -1169,14 +1147,13 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( .await?; // Launch the rnm in a task. - tokio::spawn(async move { - let _ = _rnm.await; + tokio::spawn(async { + let (_signal, inner) = shutdown_signal(); + rnm.run_until_shutdown(inner).await; }); // Check the fcs. - let (tx, rx) = oneshot::channel(); - handle.send_command(RollupManagerCommand::Status(tx)).await; - let status = rx.await?; + let status = handle.status().await?; // The fcs should be set to the database head. assert_eq!(status.forkchoice_state.head_block_info(), &db_head_block_info); @@ -1238,10 +1215,10 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { // Read the first 4 blocks. loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = + if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - if consolidation_outcome.block_info().block_info.number == 4 { + if consolidation_outcome.block_info().number == 4 { break } } @@ -1252,19 +1229,16 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { // Read the next 42 blocks. loop { - if let Some(RollupManagerEvent::L1DerivedBlockConsolidated(consolidation_outcome)) = + if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - if consolidation_outcome.block_info().block_info.number == 46 { + if consolidation_outcome.block_info().number == 46 { break } } } - let (tx, rx) = oneshot::channel(); - handle.send_command(RollupManagerCommand::Status(tx)).await; - - let status = rx.await?; + let status = handle.status().await?; // Assert the forkchoice state is above 4 assert!(status.forkchoice_state.head_block_info().number > 4); @@ -1276,10 +1250,7 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { // Wait for the third batch to be proceeded. tokio::time::sleep(Duration::from_millis(300)).await; - let (tx, rx) = oneshot::channel(); - handle.send_command(RollupManagerCommand::Status(tx)).await; - - let status = rx.await?; + let status = handle.status().await?; // Assert the forkchoice state was reset to 4. assert_eq!(status.forkchoice_state.head_block_info().number, 4); @@ -1311,10 +1282,12 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { let node1_l1_watcher_tx = node1.inner.add_ons_handle.l1_watcher_tx.as_ref().unwrap(); // Let the sequencer build 10 blocks before performing the reorg process. + let mut reorg_block = None; for i in 1..=10 { node0_rnm_handle.build_block().await; let b = wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; tracing::info!(target: "scroll::test", block_number = ?b.header.number, block_hash = ?b.header.hash_slow(), "Sequenced block"); + reorg_block = Some(b); } // Assert that the follower node has received all 10 blocks from the sequencer node. @@ -1337,26 +1310,20 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Send the L1 message to the sequencer node. node0_l1_watcher_tx.send(Arc::new(l1_message_notification.clone())).await?; node0_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(10))).await?; - wait_for_event_5s( - &mut node0_rnm_events, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::L1MessageCommitted(0)), - ) - .await?; + wait_for_event_5s(&mut node0_rnm_events, ChainOrchestratorEvent::L1MessageCommitted(0)).await?; + wait_for_event_5s(&mut node0_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; // Send L1 the L1 message to follower node. node1_l1_watcher_tx.send(Arc::new(l1_message_notification)).await?; node1_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(10))).await?; - wait_for_event_5s( - &mut node1_rnm_events, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::L1MessageCommitted(0)), - ) - .await?; + wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::L1MessageCommitted(0)).await?; + wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; // Build block that contains the L1 message. let mut block11_before_reorg = None; node0_rnm_handle.build_block().await; wait_for_event_predicate_5s(&mut node0_rnm_events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { if block.header.number == 11 && block.body.transactions.len() == 1 && block.body.transactions.iter().any(|tx| tx.is_l1_message()) @@ -1388,9 +1355,29 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Issue and wait for the reorg. node0_l1_watcher_tx.send(Arc::new(L1Notification::Reorg(9))).await?; - wait_for_event_5s(&mut node0_rnm_events, RollupManagerEvent::Reorg(9)).await?; + + let reorg_block = reorg_block.as_ref().map(|b| Into::::into(b)); + wait_for_event_5s( + &mut node0_rnm_events, + ChainOrchestratorEvent::L1Reorg { + l1_block_number: 9, + queue_index: Some(0), + l2_head_block_info: reorg_block, + l2_safe_block_info: None, + }, + ) + .await?; node1_l1_watcher_tx.send(Arc::new(L1Notification::Reorg(9))).await?; - wait_for_event_5s(&mut node1_rnm_events, RollupManagerEvent::Reorg(9)).await?; + wait_for_event_5s( + &mut node1_rnm_events, + ChainOrchestratorEvent::L1Reorg { + l1_block_number: 9, + queue_index: Some(0), + l2_head_block_info: reorg_block, + l2_safe_block_info: None, + }, + ) + .await?; // Since the L1 reorg reverted the L1 message included in block 11, the sequencer // should produce a new block at height 11. @@ -1400,10 +1387,6 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Assert that the follower node has received the new block from the sequencer node. wait_for_block_imported_5s(&mut node1_rnm_events, 11).await?; - // Assert ChainOrchestrator finished processing block. - wait_for_chain_committed_5s(&mut node0_rnm_events, 11, true).await?; - wait_for_chain_committed_5s(&mut node1_rnm_events, 11, true).await?; - // Assert both nodes are at block 11. assert_latest_block_on_rpc_by_number(&node0, 11).await; let node0_latest_block = latest_block(&node0).await?; @@ -1559,6 +1542,7 @@ async fn can_rpc_enable_disable_sequencing() -> eyre::Result<()> { // Launch sequencer node with automatic sequencing enabled. let mut config = default_sequencer_test_scroll_rollup_node_config(); config.sequencer_args.block_time = 40; // Enable automatic block production + config.sequencer_args.auto_start = true; let (mut nodes, _tasks, _) = setup_engine(config, 2, chain_spec.clone(), false, false).await?; let node0 = nodes.remove(0); @@ -1658,6 +1642,10 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { let mut node1_rnm_events = node1_rnm_handle.get_event_listener().await?; let node1_l1_watcher_tx = node1.inner.add_ons_handle.l1_watcher_tx.as_ref().unwrap(); + // Set L1 synced + node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + node1_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + // Let the sequencer build 10 blocks before performing the reorg process. for i in 1..=10 { node0_rnm_handle.build_block().await; @@ -1665,6 +1653,8 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { tracing::info!(target: "scroll::test", block_number = ?b.header.number, block_hash = ?b.header.hash_slow(), "Sequenced block") } + println!("Sequencer has built up to block 10"); + // Assert that the follower node has received all 10 blocks from the sequencer node. wait_for_block_imported_5s(&mut node1_rnm_events, 10).await?; @@ -1685,16 +1675,13 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { // Send the L1 message to the sequencer node but not to follower node. node0_l1_watcher_tx.send(Arc::new(l1_message_notification.clone())).await?; node0_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(10))).await?; - wait_for_event_5s( - &mut node0_rnm_events, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::L1MessageCommitted(0)), - ) - .await?; + wait_for_event_5s(&mut node0_rnm_events, ChainOrchestratorEvent::L1MessageCommitted(0)).await?; + wait_for_event_5s(&mut node0_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; // Build block that contains the L1 message. node0_rnm_handle.build_block().await; wait_for_event_predicate_5s(&mut node0_rnm_events, |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { if block.header.number == 11 && block.body.transactions.len() == 1 && block.body.transactions.iter().any(|tx| tx.is_l1_message()) @@ -1712,13 +1699,13 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; } + println!("Sequencer has built up to block 15 with the L1 message included in block 11"); + wait_for_event_5s( &mut node1_rnm_events, - RollupManagerEvent::L1MessageMissingInDatabase { - key: L1MessageKey::TransactionHash(b256!( - "0x0a2f8e75392ab51a26a2af835042c614eb141cd934fe1bdd4934c10f2fe17e98" - )), - }, + ChainOrchestratorEvent::L1MessageNotFoundInDatabase(L1MessageKey::TransactionHash(b256!( + "0x0a2f8e75392ab51a26a2af835042c614eb141cd934fe1bdd4934c10f2fe17e98" + ))), ) .await?; @@ -1729,11 +1716,10 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { // Finally send L1 the L1 message to follower node. node1_l1_watcher_tx.send(Arc::new(l1_message_notification)).await?; node1_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(10))).await?; - wait_for_event_5s( - &mut node1_rnm_events, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::L1MessageCommitted(0)), - ) - .await?; + wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::L1MessageCommitted(0)).await?; + wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; + + println!("Follower has received the L1 message"); // Produce another block and send to follower node. node0_rnm_handle.build_block().await; @@ -1764,6 +1750,7 @@ async fn can_gossip_over_eth_wire() -> eyre::Result<()> { let mut config = default_sequencer_test_scroll_rollup_node_config(); config.sequencer_args.block_time = 40; + config.sequencer_args.auto_start = true; // Setup the rollup node manager. let (mut nodes, _tasks, _) = @@ -1803,6 +1790,7 @@ async fn signer_rotation() -> eyre::Result<()> { sequencer_1_config.consensus_args.authorized_signer = Some(signer_1_address); sequencer_1_config.signer_args.private_key = Some(signer_1); sequencer_1_config.sequencer_args.block_time = 40; + sequencer_1_config.sequencer_args.auto_start = true; sequencer_1_config.network_args.enable_eth_scroll_wire_bridge = false; let mut sequencer_2_config = default_sequencer_test_scroll_rollup_node_config(); @@ -1811,6 +1799,7 @@ async fn signer_rotation() -> eyre::Result<()> { sequencer_2_config.consensus_args.authorized_signer = Some(signer_1_address); sequencer_2_config.signer_args.private_key = Some(signer_2); sequencer_2_config.sequencer_args.block_time = 40; + sequencer_2_config.sequencer_args.auto_start = true; sequencer_2_config.network_args.enable_eth_scroll_wire_bridge = false; // Setup two sequencer nodes. @@ -1842,7 +1831,7 @@ async fn signer_rotation() -> eyre::Result<()> { wait_n_events( &mut follower_events, |event| { - if let RollupManagerEvent::NewBlockReceived(block) = event { + if let ChainOrchestratorEvent::NewBlockReceived(block) = event { let signature = block.signature; let hash = sig_encode_hash(&block.block); // Verify that the block is signed by the first sequencer. @@ -1857,7 +1846,7 @@ async fn signer_rotation() -> eyre::Result<()> { .await; wait_n_events( &mut follower_events, - |event| matches!(event, RollupManagerEvent::BlockImported(_)), + |event| matches!(event, ChainOrchestratorEvent::ChainExtended(_)), 1, ) .await; @@ -1865,7 +1854,7 @@ async fn signer_rotation() -> eyre::Result<()> { wait_n_events( &mut sequencer_2_events, - |e| matches!(e, RollupManagerEvent::BlockImported(_)), + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 5, ) .await; @@ -1890,7 +1879,7 @@ async fn signer_rotation() -> eyre::Result<()> { wait_n_events( &mut follower_events, |event| { - if let RollupManagerEvent::NewBlockReceived(block) = event { + if let ChainOrchestratorEvent::NewBlockReceived(block) = event { let signature = block.signature; let hash = sig_encode_hash(&block.block); let recovered_address = signature.recover_address_from_prehash(&hash).unwrap(); @@ -1928,7 +1917,7 @@ async fn latest_block( } async fn wait_for_block_sequenced( - events: &mut EventStream, + events: &mut EventStream, block_number: u64, timeout: Duration, ) -> eyre::Result { @@ -1937,7 +1926,7 @@ async fn wait_for_block_sequenced( wait_for_event_predicate( events, |e| { - if let RollupManagerEvent::BlockSequenced(b) = e { + if let ChainOrchestratorEvent::BlockSequenced(b) = e { if b.header.number == block_number { block = Some(b); return true; @@ -1954,14 +1943,14 @@ async fn wait_for_block_sequenced( } async fn wait_for_block_sequenced_5s( - events: &mut EventStream, + events: &mut EventStream, block_number: u64, ) -> eyre::Result { wait_for_block_sequenced(events, block_number, Duration::from_secs(5)).await } -async fn wait_for_block_imported( - events: &mut EventStream, +async fn wait_for_chain_extended( + events: &mut EventStream, block_number: u64, timeout: Duration, ) -> eyre::Result { @@ -1970,9 +1959,10 @@ async fn wait_for_block_imported( wait_for_event_predicate( events, |e| { - if let RollupManagerEvent::BlockImported(b) = e { + if let ChainOrchestratorEvent::ChainExtended(b) = e { + let b = &b.chain[0]; if b.header.number == block_number { - block = Some(b); + block = Some(b.clone()); return true; } } @@ -1987,53 +1977,53 @@ async fn wait_for_block_imported( } async fn wait_for_block_imported_5s( - events: &mut EventStream, + events: &mut EventStream, block_number: u64, ) -> eyre::Result { - wait_for_block_imported(events, block_number, Duration::from_secs(5)).await -} - -async fn wait_for_chain_committed_5s( - events: &mut EventStream, - expected_block_number: u64, - expected_consolidated: bool, -) -> eyre::Result<()> { - wait_for_chain_committed( - events, - expected_block_number, - expected_consolidated, - Duration::from_secs(5), - ) - .await + wait_for_chain_extended(events, block_number, Duration::from_secs(5)).await } -async fn wait_for_chain_committed( - events: &mut EventStream, - expected_block_number: u64, - expected_consolidated: bool, - timeout: Duration, -) -> eyre::Result<()> { - wait_for_event_predicate( - events, - |e| { - if let RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L2ChainCommitted(block_info, _, consolidated), - ) = e - { - return block_info.block_info.number == expected_block_number && - expected_consolidated == consolidated; - } - - false - }, - timeout, - ) - .await -} +// async fn wait_for_chain_committed_5s( +// events: &mut EventStream, +// expected_block_number: u64, +// expected_consolidated: bool, +// ) -> eyre::Result<()> { +// wait_for_chain_committed( +// events, +// expected_block_number, +// expected_consolidated, +// Duration::from_secs(5), +// ) +// .await +// } + +// async fn wait_for_chain_committed( +// events: &mut EventStream, +// expected_block_number: u64, +// expected_consolidated: bool, +// timeout: Duration, +// ) -> eyre::Result<()> { +// wait_for_event_predicate( +// events, +// |e| { +// if let RollupManagerEvent::ChainOrchestratorEvent( +// ChainOrchestratorEvent::L2ChainCommitted(block_info, _, consolidated), +// ) = e +// { +// return block_info.block_info.number == expected_block_number && +// expected_consolidated == consolidated; +// } + +// false +// }, +// timeout, +// ) +// .await +// } async fn wait_for_event_predicate( - event_stream: &mut EventStream, - mut predicate: impl FnMut(RollupManagerEvent) -> bool, + event_stream: &mut EventStream, + mut predicate: impl FnMut(ChainOrchestratorEvent) -> bool, timeout: Duration, ) -> eyre::Result<()> { let sleep = tokio::time::sleep(timeout); @@ -2059,31 +2049,31 @@ async fn wait_for_event_predicate( } async fn wait_for_event_predicate_5s( - event_stream: &mut EventStream, - predicate: impl FnMut(RollupManagerEvent) -> bool, + event_stream: &mut EventStream, + predicate: impl FnMut(ChainOrchestratorEvent) -> bool, ) -> eyre::Result<()> { wait_for_event_predicate(event_stream, predicate, Duration::from_secs(5)).await } async fn wait_for_event( - event_stream: &mut EventStream, - event: RollupManagerEvent, + event_stream: &mut EventStream, + event: ChainOrchestratorEvent, timeout: Duration, ) -> eyre::Result<()> { wait_for_event_predicate(event_stream, |e| e == event, timeout).await } async fn wait_for_event_5s( - event_stream: &mut EventStream, - event: RollupManagerEvent, + event_stream: &mut EventStream, + event: ChainOrchestratorEvent, ) -> eyre::Result<()> { wait_for_event(event_stream, event, Duration::from_secs(5)).await } /// Waits for n events to be emitted. async fn wait_n_events( - events: &mut EventStream, - mut matches: impl FnMut(RollupManagerEvent) -> bool, + events: &mut EventStream, + mut matches: impl FnMut(ChainOrchestratorEvent) -> bool, mut n: u64, ) { // TODO: refactor using `wait_for_event_predicate` diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 0b391ae0..05420724 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -19,7 +19,6 @@ use rollup_node::{ ScrollRollupNodeConfig, SequencerArgs, }; use rollup_node_chain_orchestrator::ChainOrchestratorEvent; -use rollup_node_manager::RollupManagerEvent; use rollup_node_primitives::BlockInfo; use rollup_node_sequencer::L1MessageInclusionMode; use rollup_node_watcher::L1Notification; @@ -106,6 +105,7 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result< let node_config = default_test_scroll_rollup_node_config(); let mut sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); sequencer_node_config.sequencer_args.block_time = 40; + sequencer_node_config.sequencer_args.auto_start = true; // Create the chain spec for scroll mainnet with Feynman activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); @@ -125,7 +125,7 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result< let optimistic_sync_trigger = node_config.chain_orchestrator_args.optimistic_sync_trigger + 1; wait_n_events( &mut synced_events, - |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), + |e| matches!(e, ChainOrchestratorEvent::BlockSequenced(_)), optimistic_sync_trigger, ) .await; @@ -138,14 +138,7 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result< // Assert that the unsynced node triggers optimistic sync. wait_n_events( &mut unsynced_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::OptimisticSync( - _ - )) - ) - }, + |e| matches!(e, ChainOrchestratorEvent::OptimisticSync(_)), 1, ) .await; @@ -167,14 +160,7 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result< // Assert that the unsynced node triggers a chain extension on the optimistic chain. wait_n_events( &mut unsynced_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::ChainExtended( - _ - )) - ) - }, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 1, ) .await; @@ -202,7 +188,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { sequencer_args: SequencerArgs { sequencer_enabled: true, auto_start: true, - block_time: 0, + block_time: 20, l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), allow_empty_blocks: true, ..SequencerArgs::default() @@ -248,6 +234,8 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { l1_messages.push(l1_message); } + println!("Im here"); + // Add the L1 messages to the sequencer node. for (i, l1_message) in l1_messages.iter().enumerate() { sequencer_l1_watcher_tx @@ -263,11 +251,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { |e| { matches!( e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted( - _ - ) - ) + rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) ) }, 1, @@ -276,42 +260,41 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(i as u64))).await.unwrap(); wait_n_events( &mut sequencer_events, - |e| matches!(e, RollupManagerEvent::L1NotificationEvent(L1Notification::NewBlock(_))), + |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), 1, ) .await; sequencer_handle.build_block().await; wait_n_events( &mut sequencer_events, - |e: RollupManagerEvent| matches!(e, RollupManagerEvent::BlockSequenced(_)), + |e: ChainOrchestratorEvent| matches!(e, ChainOrchestratorEvent::BlockSequenced(_)), 1, ) .await; } + println!("Im here 2"); + // Connect the nodes together. sequencer.network.add_peer(follower.network.record()).await; follower.network.next_session_established().await; sequencer.network.next_session_established().await; + println!("Im here 3"); + // trigger a new block on the sequencer node. sequencer_handle.build_block().await; // Assert that the unsynced node triggers optimistic sync. wait_n_events( &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::OptimisticSync( - _ - )) - ) - }, + |e| matches!(e, ChainOrchestratorEvent::OptimisticSync(_)), 1, ) .await; + println!("Im here 4"); + // Let the unsynced node process the optimistic sync. tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; @@ -327,14 +310,10 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { .unwrap(); wait_n_events( &mut follower_events, - |e: RollupManagerEvent| { + |e: ChainOrchestratorEvent| { matches!( e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted( - _ - ) - ) + rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) ) }, 1, @@ -342,16 +321,15 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { .await; } + println!("Im here 4.1"); + // Send a notification to the unsynced node that the L1 watcher is synced. follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); // Wait for the unsynced node to sync to the L1 watcher. - wait_n_events( - &mut follower_events, - |e| matches!(e, RollupManagerEvent::L1NotificationEvent(L1Notification::Synced)), - 1, - ) - .await; + wait_n_events(&mut follower_events, |e| matches!(e, ChainOrchestratorEvent::L1Synced), 1).await; + + println!("Im here 5"); // Let the unsynced node process the L1 messages. tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; @@ -359,21 +337,18 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { // build a new block on the sequencer node to trigger consolidation on the unsynced node. sequencer_handle.build_block().await; + println!("Im here 6"); + // Assert that the unsynced node consolidates the chain. wait_n_events( &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L2ChainCommitted(_, _, true) - ) - ) - }, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), 1, ) .await; + println!("Im here 7"); + // Now push a L1 message to the sequencer node and build a new block. sequencer_l1_watcher_tx .send(Arc::new(L1Notification::L1Message { @@ -392,29 +367,18 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { .unwrap(); wait_n_events( &mut sequencer_events, - |e: RollupManagerEvent| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, + |e: ChainOrchestratorEvent| matches!(e, ChainOrchestratorEvent::L1MessageCommitted(_)), 1, ) .await; sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(201))).await.unwrap(); - wait_n_events( - &mut sequencer_events, - |e| matches!(e, RollupManagerEvent::L1NotificationEvent(L1Notification::NewBlock(_))), - 1, - ) - .await; + wait_n_events(&mut sequencer_events, |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), 1) + .await; sequencer_handle.build_block().await; wait_n_events( &mut follower_events, - |e| matches!(e, RollupManagerEvent::NewBlockReceived(_)), + |e| matches!(e, ChainOrchestratorEvent::NewBlockReceived(_)), 1, ) .await; @@ -423,7 +387,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { // message. wait_n_events( &mut follower_events, - |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { key: _ }), + |e| matches!(e, ChainOrchestratorEvent::L1MessageNotFoundInDatabase(_)), 1, ) .await; @@ -431,326 +395,325 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { Ok(()) } -#[allow(clippy::large_stack_frames)] -#[tokio::test] -async fn test_consolidation() -> eyre::Result<()> { - reth_tracing::init_test_tracing(); - let node_config = default_test_scroll_rollup_node_config(); - let sequencer_node_config = ScrollRollupNodeConfig { - test: true, - network_args: RollupNodeNetworkArgs { - enable_eth_scroll_wire_bridge: true, - enable_scroll_wire: true, - sequencer_url: None, - signer_address: None, - }, - database_args: RollupNodeDatabaseArgs { - rn_db_path: Some(PathBuf::from("sqlite::memory:")), - }, - l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs::default(), - chain_orchestrator_args: ChainOrchestratorArgs::default(), - sequencer_args: SequencerArgs { - sequencer_enabled: true, - auto_start: true, - block_time: 0, - l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), - allow_empty_blocks: true, - ..SequencerArgs::default() - }, - blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, - signer_args: Default::default(), - gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), - consensus_args: ConsensusArgs::noop(), - database: None, - rpc_args: RpcArgs::default(), - }; - - // Create the chain spec for scroll dev with Feynman activated and a test genesis. - let chain_spec = (*SCROLL_DEV).clone(); - - // Create a sequencer node and an unsynced node. - let (mut nodes, _tasks, _) = - setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); - let mut sequencer = nodes.pop().unwrap(); - let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); - let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); - let mut sequencer_events = sequencer_handle.get_event_listener().await?; - - let (mut nodes, _tasks, _) = - setup_engine(node_config.clone(), 1, chain_spec, false, false).await.unwrap(); - let mut follower = nodes.pop().unwrap(); - let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); - let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; - - // Connect the nodes together. - sequencer.network.add_peer(follower.network.record()).await; - follower.network.next_session_established().await; - sequencer.network.next_session_established().await; - - // Create a L1 message and send it to both nodes. - let l1_message = TxL1Message { - queue_index: 0, - gas_limit: 21000, - sender: Address::random(), - to: Address::random(), - value: U256::from(1), - input: Default::default(), - }; - sequencer_l1_watcher_tx - .send(Arc::new(L1Notification::L1Message { - message: l1_message.clone(), - block_number: 0, - block_timestamp: 0, - })) - .await - .unwrap(); - wait_n_events( - &mut sequencer_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, - 1, - ) - .await; - sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(2))).await.unwrap(); - - follower_l1_watcher_tx - .send(Arc::new(L1Notification::L1Message { - message: l1_message, - block_number: 0, - block_timestamp: 0, - })) - .await - .unwrap(); - wait_n_events( - &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, - 1, - ) - .await; - - // Send a notification to both nodes that the L1 watcher is synced. - sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); - follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); - - // Build a new block on the sequencer node. - sequencer_handle.build_block().await; - - // Assert that the unsynced node consolidates the chain. - wait_n_events( - &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L2ChainCommitted(_, _, true) - ) - ) - }, - 1, - ) - .await; - - // Now push a L1 message to the sequencer node and build a new block. - sequencer_l1_watcher_tx - .send(Arc::new(L1Notification::L1Message { - message: TxL1Message { - queue_index: 1, - gas_limit: 21000, - sender: Address::random(), - to: Address::random(), - value: U256::from(1), - input: Default::default(), - }, - block_number: 1, - block_timestamp: 10, - })) - .await - .unwrap(); - wait_n_events( - &mut sequencer_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, - 1, - ) - .await; - sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(5))).await.unwrap(); - sequencer_handle.build_block().await; - - // Assert that the follower node rejects the new block as it hasn't received the L1 message. - wait_n_events( - &mut follower_events, - |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { key: _ }), - 1, - ) - .await; - - Ok(()) -} - -#[allow(clippy::large_stack_frames)] -#[tokio::test] -async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { - reth_tracing::init_test_tracing(); - let node_config = default_test_scroll_rollup_node_config(); - let sequencer_node_config = ScrollRollupNodeConfig { - test: true, - network_args: RollupNodeNetworkArgs { - enable_eth_scroll_wire_bridge: false, - enable_scroll_wire: true, - ..Default::default() - }, - database_args: RollupNodeDatabaseArgs { - rn_db_path: Some(PathBuf::from("sqlite::memory:")), - }, - l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs::default(), - chain_orchestrator_args: ChainOrchestratorArgs::default(), - sequencer_args: SequencerArgs { - sequencer_enabled: true, - auto_start: true, - block_time: 0, - l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), - allow_empty_blocks: true, - ..SequencerArgs::default() - }, - blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, - signer_args: Default::default(), - gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), - consensus_args: ConsensusArgs::noop(), - database: None, - rpc_args: RpcArgs::default(), - }; - - // Create the chain spec for scroll dev with Feynman activated and a test genesis. - let chain_spec = (*SCROLL_DEV).clone(); - - // Create a sequencer node and an unsynced node. - let (mut nodes, _tasks, _) = - setup_engine(sequencer_node_config.clone(), 1, chain_spec.clone(), false, false) - .await - .unwrap(); - let mut sequencer = nodes.pop().unwrap(); - let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); - let mut sequencer_events = sequencer_handle.get_event_listener().await?; - - let (mut nodes, _tasks, _) = - setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); - let mut follower = nodes.pop().unwrap(); - let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; - - // Connect the nodes together. - sequencer.connect(&mut follower).await; - - // initially the sequencer should build 100 empty blocks and the follower should follow them - let mut reorg_block_info = BlockInfo::default(); - for i in 0..100 { - sequencer_handle.build_block().await; - wait_n_events( - &mut sequencer_events, - |e| { - if let RollupManagerEvent::BlockSequenced(block) = e { - if i == 95 { - reorg_block_info = (&block).into(); - } - true - } else { - false - } - }, - 1, - ) - .await; - wait_n_events( - &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L2ChainCommitted(_, _, _) - ) - ) - }, - 1, - ) - .await; - } - - // disconnect the two nodes - let mut sequencer_network_events = sequencer.inner.network.event_listener(); - let mut follower_network_events = follower.inner.network.event_listener(); - sequencer.inner.network.peers_handle().remove_peer(follower.network.record().id); - while let Some(ev) = sequencer_network_events.next().await { - if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { - break - } - } - while let Some(ev) = sequencer_network_events.next().await { - if let NetworkEvent::Peer(PeerEvent::PeerRemoved(_)) = ev { - break - } - } - while let Some(ev) = follower_network_events.next().await { - if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { - break - } - } - - sequencer_handle.update_fcs_head(reorg_block_info).await; - - // Have the sequencer build 2 new blocks, one containing the L1 message. - sequencer_handle.build_block().await; - wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), 1) - .await; - sequencer_handle.build_block().await; - wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), 1) - .await; - - // connect the two nodes again - follower.connect(&mut sequencer).await; - - // now build a final block - sequencer_handle.build_block().await; - - // Wait for the follower node to reorg to the new chain. - wait_n_events( - &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::ChainReorged(_)) - ) - }, - 1, - ) - .await; - - Ok(()) -} +// #[allow(clippy::large_stack_frames)] +// #[tokio::test] +// async fn test_consolidation() -> eyre::Result<()> { +// reth_tracing::init_test_tracing(); +// let node_config = default_test_scroll_rollup_node_config(); +// let sequencer_node_config = ScrollRollupNodeConfig { +// test: true, +// network_args: RollupNodeNetworkArgs { +// enable_eth_scroll_wire_bridge: true, +// enable_scroll_wire: true, +// sequencer_url: None, +// signer_address: None, +// }, +// database_args: RollupNodeDatabaseArgs { +// rn_db_path: Some(PathBuf::from("sqlite::memory:")), +// }, +// l1_provider_args: L1ProviderArgs::default(), +// engine_driver_args: EngineDriverArgs::default(), +// chain_orchestrator_args: ChainOrchestratorArgs::default(), +// sequencer_args: SequencerArgs { +// sequencer_enabled: true, +// auto_start: true, +// block_time: 0, +// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), +// allow_empty_blocks: true, +// ..SequencerArgs::default() +// }, +// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, +// signer_args: Default::default(), +// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), +// consensus_args: ConsensusArgs::noop(), +// database: None, +// rpc_args: RpcArgs::default(), +// }; + +// // Create the chain spec for scroll dev with Feynman activated and a test genesis. +// let chain_spec = (*SCROLL_DEV).clone(); + +// // Create a sequencer node and an unsynced node. +// let (mut nodes, _tasks, _) = +// setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); +// let mut sequencer = nodes.pop().unwrap(); +// let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); +// let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); +// let mut sequencer_events = sequencer_handle.get_event_listener().await?; + +// let (mut nodes, _tasks, _) = +// setup_engine(node_config.clone(), 1, chain_spec, false, false).await.unwrap(); +// let mut follower = nodes.pop().unwrap(); +// let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); +// let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; + +// // Connect the nodes together. +// sequencer.network.add_peer(follower.network.record()).await; +// follower.network.next_session_established().await; +// sequencer.network.next_session_established().await; + +// // Create a L1 message and send it to both nodes. +// let l1_message = TxL1Message { +// queue_index: 0, +// gas_limit: 21000, +// sender: Address::random(), +// to: Address::random(), +// value: U256::from(1), +// input: Default::default(), +// }; +// sequencer_l1_watcher_tx +// .send(Arc::new(L1Notification::L1Message { +// message: l1_message.clone(), +// block_number: 0, +// block_timestamp: 0, +// })) +// .await +// .unwrap(); +// wait_n_events( +// &mut sequencer_events, +// |e| { +// matches!( +// e, +// RollupManagerEvent::ChainOrchestratorEvent( +// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) +// ) +// ) +// }, +// 1, +// ) +// .await; +// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(2))).await.unwrap(); + +// follower_l1_watcher_tx +// .send(Arc::new(L1Notification::L1Message { +// message: l1_message, +// block_number: 0, +// block_timestamp: 0, +// })) +// .await +// .unwrap(); +// wait_n_events( +// &mut follower_events, +// |e| { +// matches!( +// e, +// RollupManagerEvent::ChainOrchestratorEvent( +// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) +// ) +// ) +// }, +// 1, +// ) +// .await; + +// // Send a notification to both nodes that the L1 watcher is synced. +// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); +// follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + +// // Build a new block on the sequencer node. +// sequencer_handle.build_block().await; + +// // Assert that the unsynced node consolidates the chain. +// wait_n_events( +// &mut follower_events, +// |e| { +// matches!( +// e, +// RollupManagerEvent::ChainOrchestratorEvent( +// ChainOrchestratorEvent::L2ChainCommitted(_, _, true) +// ) +// ) +// }, +// 1, +// ) +// .await; + +// // Now push a L1 message to the sequencer node and build a new block. +// sequencer_l1_watcher_tx +// .send(Arc::new(L1Notification::L1Message { +// message: TxL1Message { +// queue_index: 1, +// gas_limit: 21000, +// sender: Address::random(), +// to: Address::random(), +// value: U256::from(1), +// input: Default::default(), +// }, +// block_number: 1, +// block_timestamp: 10, +// })) +// .await +// .unwrap(); +// wait_n_events( +// &mut sequencer_events, +// |e| { +// matches!( +// e, +// RollupManagerEvent::ChainOrchestratorEvent( +// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) +// ) +// ) +// }, +// 1, +// ) +// .await; +// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(5))).await.unwrap(); +// sequencer_handle.build_block().await; + +// // Assert that the follower node rejects the new block as it hasn't received the L1 message. +// wait_n_events( +// &mut follower_events, +// |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { key: _ }), +// 1, +// ) +// .await; + +// Ok(()) +// } + +// #[allow(clippy::large_stack_frames)] +// #[tokio::test] +// async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { +// reth_tracing::init_test_tracing(); +// let node_config = default_test_scroll_rollup_node_config(); +// let sequencer_node_config = ScrollRollupNodeConfig { +// test: true, +// network_args: RollupNodeNetworkArgs { +// enable_eth_scroll_wire_bridge: false, +// enable_scroll_wire: true, +// ..Default::default() +// }, +// database_args: RollupNodeDatabaseArgs { +// rn_db_path: Some(PathBuf::from("sqlite::memory:")), +// }, +// l1_provider_args: L1ProviderArgs::default(), +// engine_driver_args: EngineDriverArgs::default(), +// chain_orchestrator_args: ChainOrchestratorArgs::default(), +// sequencer_args: SequencerArgs { +// sequencer_enabled: true, +// auto_start: true, +// block_time: 0, +// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), +// allow_empty_blocks: true, +// ..SequencerArgs::default() +// }, +// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, +// signer_args: Default::default(), +// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), +// consensus_args: ConsensusArgs::noop(), +// database: None, +// rpc_args: RpcArgs::default(), +// }; + +// // Create the chain spec for scroll dev with Feynman activated and a test genesis. +// let chain_spec = (*SCROLL_DEV).clone(); + +// // Create a sequencer node and an unsynced node. +// let (mut nodes, _tasks, _) = +// setup_engine(sequencer_node_config.clone(), 1, chain_spec.clone(), false, false) +// .await +// .unwrap(); +// let mut sequencer = nodes.pop().unwrap(); +// let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); +// let mut sequencer_events = sequencer_handle.get_event_listener().await?; + +// let (mut nodes, _tasks, _) = +// setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); +// let mut follower = nodes.pop().unwrap(); +// let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; + +// // Connect the nodes together. +// sequencer.connect(&mut follower).await; + +// // initially the sequencer should build 100 empty blocks and the follower should follow them +// let mut reorg_block_info = BlockInfo::default(); +// for i in 0..100 { +// sequencer_handle.build_block().await; +// wait_n_events( +// &mut sequencer_events, +// |e| { +// if let RollupManagerEvent::BlockSequenced(block) = e { +// if i == 95 { +// reorg_block_info = (&block).into(); +// } +// true +// } else { +// false +// } +// }, +// 1, +// ) +// .await; +// wait_n_events( +// &mut follower_events, +// |e| { +// matches!( +// e, +// RollupManagerEvent::ChainOrchestratorEvent( +// ChainOrchestratorEvent::L2ChainCommitted(_, _, _) +// ) +// ) +// }, +// 1, +// ) +// .await; +// } + +// // disconnect the two nodes +// let mut sequencer_network_events = sequencer.inner.network.event_listener(); +// let mut follower_network_events = follower.inner.network.event_listener(); +// sequencer.inner.network.peers_handle().remove_peer(follower.network.record().id); +// while let Some(ev) = sequencer_network_events.next().await { +// if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { +// break +// } +// } +// while let Some(ev) = sequencer_network_events.next().await { +// if let NetworkEvent::Peer(PeerEvent::PeerRemoved(_)) = ev { +// break +// } +// } +// while let Some(ev) = follower_network_events.next().await { +// if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { +// break +// } +// } + +// sequencer_handle.update_fcs_head(reorg_block_info).await; + +// // Have the sequencer build 2 new blocks, one containing the L1 message. +// sequencer_handle.build_block().await; +// wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), +// 1) .await; sequencer_handle.build_block().await; wait_n_events(&mut sequencer_events, |e| +// matches!(e, RollupManagerEvent::BlockSequenced(_)), +// 1) .await; + +// // connect the two nodes again +// follower.connect(&mut sequencer).await; + +// // now build a final block +// sequencer_handle.build_block().await; + +// // Wait for the follower node to reorg to the new chain. +// wait_n_events( +// &mut follower_events, +// |e| { +// matches!( +// e, +// +// RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::ChainReorged(_)) ) +// }, +// 1, +// ) +// .await; + +// Ok(()) +// } /// Waits for n events to be emitted. async fn wait_n_events( - events: &mut EventStream, - mut matches: impl FnMut(RollupManagerEvent) -> bool, + events: &mut EventStream, + mut matches: impl FnMut(ChainOrchestratorEvent) -> bool, mut n: u64, ) { while let Some(event) = events.next().await { diff --git a/crates/primitives/src/batch.rs b/crates/primitives/src/batch.rs index a1c37c50..6d5192b3 100644 --- a/crates/primitives/src/batch.rs +++ b/crates/primitives/src/batch.rs @@ -19,6 +19,12 @@ impl BatchInfo { } } +impl std::fmt::Display for BatchInfo { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "BatchInfo {{ index: {}, hash: 0x{} }}", self.index, self.hash) + } +} + /// The input data for a batch. /// /// This is used as input for the derivation pipeline. All data remains in its raw serialized form. @@ -48,7 +54,7 @@ impl From for BatchInfo { } /// The outcome of consolidating a batch with the L2 chain. -#[derive(Debug)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct BatchConsolidationOutcome { /// The batch info for the consolidated batch. pub batch_info: BatchInfo, @@ -69,14 +75,27 @@ impl BatchConsolidationOutcome { } /// The outcome of consolidating a block with the L2 chain. -#[derive(Debug)] +#[derive(Debug, Clone, PartialEq, Eq)] pub enum BlockConsolidationOutcome { - /// The derived block was already part of the chain, no action needed. + /// The derived block was already part of the chain, update the fork choice state. Consolidated(BlockInfo), + /// The fork choice state was already ahead of the derived block. + Skipped(BlockInfo), /// The derived block resulted in a reorg of the L2 chain. Reorged(L2BlockInfoWithL1Messages), } +impl BlockConsolidationOutcome { + /// Returns the block info for the consolidated block. + pub fn block_info(&self) -> &BlockInfo { + match self { + BlockConsolidationOutcome::Consolidated(info) => info, + BlockConsolidationOutcome::Skipped(info) => info, + BlockConsolidationOutcome::Reorged(info) => &info.block_info, + } + } +} + #[cfg(feature = "arbitrary")] mod arbitrary_impl { use super::*; diff --git a/crates/primitives/src/block.rs b/crates/primitives/src/block.rs index ab7d8c08..ef40c89a 100644 --- a/crates/primitives/src/block.rs +++ b/crates/primitives/src/block.rs @@ -74,6 +74,12 @@ impl From
for BlockInfo { } } +impl std::fmt::Display for BlockInfo { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "BlockInfo {{ number: {}, hash: 0x{} }}", self.number, self.hash) + } +} + #[cfg(feature = "arbitrary")] impl arbitrary::Arbitrary<'_> for BlockInfo { fn arbitrary(u: &mut arbitrary::Unstructured<'_>) -> arbitrary::Result { diff --git a/crates/sequencer/Cargo.toml b/crates/sequencer/Cargo.toml index 6f5c7b02..5903959c 100644 --- a/crates/sequencer/Cargo.toml +++ b/crates/sequencer/Cargo.toml @@ -66,6 +66,7 @@ rollup-node-chain-orchestrator.workspace = true rollup-node-primitives.workspace = true rollup-node-providers = { workspace = true, features = ["test-utils"] } rollup-node-sequencer.workspace = true +rollup-node-watcher.workspace = true scroll-db = { workspace = true, features = ["test-utils"] } scroll-engine.workspace = true diff --git a/crates/sequencer/src/config.rs b/crates/sequencer/src/config.rs index 79c95352..b977ab79 100644 --- a/crates/sequencer/src/config.rs +++ b/crates/sequencer/src/config.rs @@ -1,5 +1,5 @@ use alloy_primitives::Address; -use scroll_db::{L1MessageStart, NotIncludedStart}; +use scroll_db::{L1MessageKey, NotIncludedStart}; use std::{fmt, str::FromStr, sync::Arc}; /// Configuration for the sequencer. @@ -67,14 +67,14 @@ impl fmt::Display for L1MessageInclusionMode { } } -impl From for L1MessageStart { +impl From for L1MessageKey { fn from(mode: L1MessageInclusionMode) -> Self { match mode { L1MessageInclusionMode::Finalized => { - L1MessageStart::NotIncluded(NotIncludedStart::Finalized) + L1MessageKey::NotIncluded(NotIncludedStart::Finalized) } L1MessageInclusionMode::BlockDepth(depth) => { - L1MessageStart::NotIncluded(NotIncludedStart::BlockDepth(depth)) + L1MessageKey::NotIncluded(NotIncludedStart::BlockDepth(depth)) } } } diff --git a/crates/sequencer/src/event.rs b/crates/sequencer/src/event.rs index 8524e3a2..8eb47ea4 100644 --- a/crates/sequencer/src/event.rs +++ b/crates/sequencer/src/event.rs @@ -1,5 +1,4 @@ use alloy_rpc_types_engine::PayloadId; -use reth_scroll_primitives::ScrollBlock; /// Events emitted by the sequencer. #[derive(Debug, Clone)] diff --git a/crates/sequencer/src/lib.rs b/crates/sequencer/src/lib.rs index 94d451bd..aa28d2ac 100644 --- a/crates/sequencer/src/lib.rs +++ b/crates/sequencer/src/lib.rs @@ -11,7 +11,7 @@ use std::{ }; use alloy_eips::eip2718::Encodable2718; -use alloy_rpc_types_engine::{ExecutionData, ExecutionPayloadV1, PayloadAttributes, PayloadId}; +use alloy_rpc_types_engine::{ExecutionData, PayloadAttributes, PayloadId}; use futures::{task::AtomicWaker, Stream}; use reth_scroll_engine_primitives::try_into_block; use reth_scroll_primitives::ScrollBlock; @@ -34,10 +34,9 @@ pub use event::SequencerEvent; mod metrics; pub use metrics::SequencerMetrics; -use scroll_db::L1MessageKey; /// A type alias for the payload building job future. -pub type PayloadBuildingJobFuture = Pin + Send>>; +pub type PayloadBuildingJobFuture = Pin + Send + Sync>>; /// The sequencer is responsible for sequencing transactions and producing new blocks. pub struct Sequencer { @@ -187,8 +186,8 @@ where ) -> Result, SequencerError> { let payload = engine.get_payload(payload_id).await?; - if payload.transactions.is_empty() { - tracing::trace!(target: "rollup_node::sequencer", "Built empty payload with id {payload_id:?}."); + if payload.transactions.is_empty() && !self.config.allow_empty_blocks { + tracing::trace!(target: "rollup_node::sequencer", "Built empty payload with id {payload_id:?}, discarding payload."); Ok(None) } else { tracing::info!(target: "rollup_node::sequencer", "Built payload with id {payload_id:?}, hash: {:#x}, number: {} containing {} transactions.", payload.block_hash, payload.block_number, payload.transactions.len()); diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index fbf1edf9..3974d701 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -22,6 +22,7 @@ use rollup_node_sequencer::{ L1MessageInclusionMode, PayloadBuildingConfig, Sequencer, SequencerConfig, SequencerEvent, }; use rollup_node_signer::SignerEvent; +use rollup_node_watcher::L1Notification; use scroll_alloy_consensus::TxL1Message; use scroll_alloy_provider::ScrollAuthApiEngineClient; use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; @@ -66,9 +67,9 @@ async fn skip_block_with_no_transactions() { let database = Arc::new(setup_test_db().await); let provider = database.clone(); - // Set the finalized block number + // Set the latest block number let tx = provider.tx_mut().await.unwrap(); - tx.set_finalized_l1_block_number(0).await.unwrap(); + tx.set_latest_l1_block_number(0).await.unwrap(); tx.commit().await.unwrap(); // create a sequencer @@ -83,7 +84,7 @@ async fn skip_block_with_no_transactions() { }, block_time: 1, payload_building_duration: 0, - allow_empty_blocks: true, + allow_empty_blocks: false, }; let mut sequencer = Sequencer::new(provider, config); @@ -125,9 +126,9 @@ async fn can_build_blocks() { let database = Arc::new(setup_test_db().await); let provider = database.clone(); - // Set the finalized block number + // Set the latest block number let tx = provider.tx_mut().await.unwrap(); - tx.set_finalized_l1_block_number(5).await.unwrap(); + tx.set_latest_l1_block_number(5).await.unwrap(); tx.commit().await.unwrap(); // create a sequencer @@ -227,7 +228,6 @@ async fn can_build_blocks_with_delayed_l1_messages() { reth_tracing::init_test_tracing(); let chain_spec = SCROLL_DEV.clone(); - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); const L1_MESSAGE_DELAY: u64 = 2; // setup a test node @@ -235,6 +235,7 @@ async fn can_build_blocks_with_delayed_l1_messages() { setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) .await .unwrap(); + let node = nodes.pop().unwrap(); let wallet = Arc::new(Mutex::new(wallet)); @@ -255,6 +256,11 @@ async fn can_build_blocks_with_delayed_l1_messages() { let database = Arc::new(setup_test_db().await); let provider = database.clone(); + // Set the latest block number + let tx = provider.tx_mut().await.unwrap(); + tx.set_latest_l1_block_number(1).await.unwrap(); + tx.commit().await.unwrap(); + // create a sequencer let config = SequencerConfig { chain_spec: node.inner.chain_spec().clone(), @@ -263,7 +269,7 @@ async fn can_build_blocks_with_delayed_l1_messages() { payload_building_config: PayloadBuildingConfig { block_gas_limit: SCROLL_GAS_LIMIT, max_l1_messages_per_block: 4, - l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(L1_MESSAGE_DELAY), }, block_time: 0, payload_building_duration: 0, @@ -306,7 +312,7 @@ async fn can_build_blocks_with_delayed_l1_messages() { let tx_hash = node.rpc.inject_tx(raw_tx).await.unwrap(); // send a new payload attributes request. - sequencer.start_payload_building(&mut engine); + sequencer.start_payload_building(&mut engine).await.unwrap(); let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); assert!(result.is_some(), "expected a new payload, but got: {:?}", result); @@ -329,10 +335,12 @@ async fn can_build_blocks_with_delayed_l1_messages() { tokio::time::sleep(std::time::Duration::from_secs(2)).await; // set the l1 block number to 3 - sequencer.handle_new_l1_block(3); + let tx = database.tx_mut().await.unwrap(); + tx.set_latest_l1_block_number(3).await.unwrap(); + tx.commit().await.unwrap(); // send a new block request this block should include the L1 message - sequencer.start_payload_building(&mut engine); + sequencer.start_payload_building(&mut engine).await.unwrap(); let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); assert!(result.is_some(), "expected a new payload, but got: {:?}", result); @@ -348,630 +356,628 @@ async fn can_build_blocks_with_delayed_l1_messages() { assert_eq!(block.header.parent_hash, block_1_hash); } -// #[tokio::test] -// async fn can_build_blocks_with_finalized_l1_messages() { -// reth_tracing::init_test_tracing(); - -// let chain_spec = SCROLL_DEV.clone(); -// const BLOCK_BUILDING_DURATION: Duration = tokio::time::Duration::from_millis(0); -// // setup a test node -// let (mut nodes, _tasks, wallet) = -// setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) -// .await -// .unwrap(); -// let node = nodes.pop().unwrap(); -// let wallet = Arc::new(Mutex::new(wallet)); - -// // create a forkchoice state -// let genesis_hash = node.inner.chain_spec().genesis_hash(); -// let fcs = ForkchoiceState::new( -// BlockInfo { hash: genesis_hash, number: 0 }, -// Default::default(), -// Default::default(), -// ); - -// // create the engine driver connected to the node -// let auth_client = node.inner.engine_http_client(); -// let engine_client = ScrollAuthApiEngineClient::new(auth_client); -// let mut engine_driver = EngineDriver::new( -// Arc::new(engine_client), -// (*SCROLL_DEV).clone(), -// None::, -// fcs, -// false, -// BLOCK_BUILDING_DURATION, -// true, -// ); - -// // create a test database -// let database = Arc::new(setup_test_db().await); -// let provider = database.clone(); - -// // create a sequencer with Finalized mode -// let mut sequencer = Sequencer::new( -// provider, -// Default::default(), -// SCROLL_GAS_LIMIT, -// 4, -// 5, // current L1 block number -// L1MessageInclusionMode::Finalized, -// 0, -// ); - -// // set L1 finalized block number to 2 -// sequencer.set_l1_finalized_block_number(2); - -// // add L1 messages to database -// let wallet_lock = wallet.lock().await; - -// // this message should be included (before finalized block) -// let finalized_l1_message = L1MessageEnvelope { -// l1_block_number: 2, // <= 2 (finalized block) -// l2_block_number: None, -// queue_hash: None, -// transaction: TxL1Message { -// queue_index: 0, -// gas_limit: 21000, -// to: Address::random(), -// value: U256::from(1), -// sender: wallet_lock.inner.address(), -// input: vec![].into(), -// }, -// }; - -// // this message should not be included (after finalized block) -// let unfinalized_l1_message = L1MessageEnvelope { -// l1_block_number: 3, // > 2 (finalized block) -// l2_block_number: None, -// queue_hash: None, -// transaction: TxL1Message { -// queue_index: 1, -// gas_limit: 21000, -// to: Address::random(), -// value: U256::from(2), -// sender: wallet_lock.inner.address(), -// input: vec![].into(), -// }, -// }; -// drop(wallet_lock); - -// let finalized_message_hash = finalized_l1_message.transaction.tx_hash(); -// let unfinalized_message_hash = unfinalized_l1_message.transaction.tx_hash(); - -// let tx = database.tx_mut().await.unwrap(); -// tx.insert_l1_message(finalized_l1_message).await.unwrap(); -// tx.insert_l1_message(unfinalized_l1_message).await.unwrap(); -// tx.commit().await.unwrap(); - -// // build payload, should only include finalized message -// sequencer.start_payload_building(); -// let payload_attributes = sequencer.next().await.unwrap(); -// engine_driver.handle_build_new_payload(payload_attributes); - -// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// block -// } else { -// panic!("expected a new payload event"); -// }; - -// // verify only finalized L1 message is included -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &finalized_message_hash); - -// // ensure unfinalized message is not included -// assert!(!block.body.transactions.iter().any(|tx| tx.tx_hash() == &unfinalized_message_hash)); - -// // Handle the build block with the sequencer in order to update L1 message queue index. -// sequencer.handle_new_payload(&block); - -// // update finalized block number to 3, now both messages should be available -// sequencer.set_l1_finalized_block_number(3); - -// // sleep 2 seconds (ethereum header timestamp has granularity of seconds and proceeding -// header // must have a greater timestamp than the last) -// tokio::time::sleep(std::time::Duration::from_secs(2)).await; - -// // build new payload -// sequencer.start_payload_building(); -// let payload_attributes = sequencer.next().await.unwrap(); -// engine_driver.handle_build_new_payload(payload_attributes); - -// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// block -// } else { -// panic!("expected a new payload event"); -// }; - -// // now should include the previously unfinalized message -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &unfinalized_message_hash); -// } - -// #[tokio::test] -// async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { -// reth_tracing::init_test_tracing(); - -// // Create temporary private key file -// let mut temp_file = NamedTempFile::new()?; -// let private_key_hex = "0x1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; -// temp_file.write_all(private_key_hex.as_bytes())?; -// temp_file.flush()?; - -// // Create expected signer -// let expected_key_bytes = -// hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef")?; -// let expected_signer = alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes)?; -// let expected_address = expected_signer.address(); - -// let chain_spec = (*SCROLL_DEV).clone(); -// let rollup_manager_args = ScrollRollupNodeConfig { -// test: false, // disable test mode to enable real signing -// network_args: RollupNodeNetworkArgs::default(), -// database_args: RollupNodeDatabaseArgs { -// rn_db_path: Some(PathBuf::from("sqlite::memory:")), -// }, -// l1_provider_args: L1ProviderArgs::default(), -// engine_driver_args: EngineDriverArgs::default(), -// chain_orchestrator_args: ChainOrchestratorArgs::default(), -// sequencer_args: SequencerArgs { -// sequencer_enabled: true, -// auto_start: true, -// block_time: 0, -// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), -// payload_building_duration: 1000, -// allow_empty_blocks: true, -// ..SequencerArgs::default() -// }, -// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, -// signer_args: SignerArgs { -// key_file: Some(temp_file.path().to_path_buf()), -// aws_kms_key_id: None, -// private_key: None, -// }, -// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), -// consensus_args: ConsensusArgs::noop(), -// database: None, -// rpc_args: RpcArgs::default(), -// }; - -// let (nodes, _tasks, wallet) = -// setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; -// let wallet = Arc::new(Mutex::new(wallet)); - -// let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); -// let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; - -// // Generate and inject transaction -// let mut wallet_lock = wallet.lock().await; -// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( -// wallet_lock.chain_id, -// wallet_lock.inner.clone(), -// wallet_lock.inner_nonce, -// ) -// .await; -// wallet_lock.inner_nonce += 1; -// drop(wallet_lock); -// let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; - -// // Build block -// sequencer_rnm_handle.build_block().await; - -// // Verify block was successfully sequenced -// if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); -// } else { -// panic!("Failed to receive BlockSequenced event"); -// } - -// // Verify signing event and signature correctness -// if let Some(ChainOrchestratorEvent::SignerEvent(SignerEvent::SignedBlock { -// block: signed_block, -// signature, -// })) = sequencer_events.next().await -// { -// let hash = sig_encode_hash(&signed_block); -// let recovered_address = signature.recover_address_from_prehash(&hash)?; -// assert_eq!(recovered_address, expected_address); -// } else { -// panic!("Failed to receive SignerEvent with signed block"); -// } - -// Ok(()) -// } - -// #[tokio::test] -// async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result<()> { -// reth_tracing::init_test_tracing(); - -// // Create temporary private key file (without 0x prefix) -// let mut temp_file = NamedTempFile::new().unwrap(); -// let private_key_hex = "1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; -// temp_file.write_all(private_key_hex.as_bytes()).unwrap(); -// temp_file.flush().unwrap(); - -// // Create expected signer -// let expected_key_bytes = -// hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef").unwrap(); -// let expected_signer = -// alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes).unwrap(); -// let expected_address = expected_signer.address(); - -// let chain_spec = (*SCROLL_DEV).clone(); -// let rollup_manager_args = ScrollRollupNodeConfig { -// test: false, // disable test mode to enable real signing -// network_args: RollupNodeNetworkArgs::default(), -// database_args: RollupNodeDatabaseArgs { -// rn_db_path: Some(PathBuf::from("sqlite::memory:")), -// }, -// l1_provider_args: L1ProviderArgs::default(), -// engine_driver_args: EngineDriverArgs::default(), -// chain_orchestrator_args: ChainOrchestratorArgs::default(), -// sequencer_args: SequencerArgs { -// sequencer_enabled: true, -// auto_start: true, -// block_time: 0, -// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), -// payload_building_duration: 1000, -// allow_empty_blocks: true, -// ..SequencerArgs::default() -// }, -// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, -// signer_args: SignerArgs { -// key_file: Some(temp_file.path().to_path_buf()), -// aws_kms_key_id: None, -// private_key: None, -// }, -// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), -// consensus_args: ConsensusArgs::noop(), -// database: None, -// rpc_args: RpcArgs::default(), -// }; - -// let (nodes, _tasks, wallet) = -// setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; -// let wallet = Arc::new(Mutex::new(wallet)); - -// let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); -// let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; - -// // Generate and inject transaction -// let mut wallet_lock = wallet.lock().await; -// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( -// wallet_lock.chain_id, -// wallet_lock.inner.clone(), -// wallet_lock.inner_nonce, -// ) -// .await; -// wallet_lock.inner_nonce += 1; -// drop(wallet_lock); -// let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; - -// // Build block -// sequencer_rnm_handle.build_block().await; - -// // Verify block was successfully sequenced -// if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); -// } else { -// panic!("Failed to receive BlockSequenced event"); -// } - -// // Verify signing event and signature correctness -// while let Some(event) = sequencer_events.next().await { -// if let RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { -// block: signed_block, -// signature, -// }) = event -// { -// let hash = sig_encode_hash(&signed_block); -// let recovered_address = signature.recover_address_from_prehash(&hash)?; -// assert_eq!(recovered_address, expected_address); -// break; -// } -// } - -// Ok(()) -// } - -// #[tokio::test] -// async fn can_build_blocks_and_exit_at_gas_limit() { -// reth_tracing::init_test_tracing(); - -// let chain_spec = SCROLL_DEV.clone(); -// const MIN_TRANSACTION_GAS_COST: u64 = 21_000; -// const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); -// const TRANSACTIONS_COUNT: usize = 2000; - -// // setup a test node. use a high value for the payload building duration to be sure we don't -// // exit early. -// let (mut nodes, _tasks, wallet) = setup_engine( -// ScrollRollupNodeConfig { -// sequencer_args: SequencerArgs { payload_building_duration: 1000, ..Default::default() -// }, ..default_test_scroll_rollup_node_config() -// }, -// 1, -// chain_spec, -// false, -// false, -// ) -// .await -// .unwrap(); -// let node = nodes.pop().unwrap(); -// let wallet = Arc::new(Mutex::new(wallet)); - -// // add transactions. -// let mut wallet_lock = wallet.lock().await; -// for _ in 0..TRANSACTIONS_COUNT { -// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( -// wallet_lock.chain_id, -// wallet_lock.inner.clone(), -// wallet_lock.inner_nonce, -// ) -// .await; -// wallet_lock.inner_nonce += 1; -// node.rpc.inject_tx(raw_tx).await.unwrap(); -// } -// drop(wallet_lock); - -// // create a forkchoice state -// let genesis_hash = node.inner.chain_spec().genesis_hash(); -// let fcs = ForkchoiceState::new( -// BlockInfo { hash: genesis_hash, number: 0 }, -// Default::default(), -// Default::default(), -// ); - -// // create the engine driver connected to the node -// let auth_client = node.inner.engine_http_client(); -// let engine_client = ScrollAuthApiEngineClient::new(auth_client); -// let mut engine_driver = EngineDriver::new( -// Arc::new(engine_client), -// (*SCROLL_DEV).clone(), -// None::, -// fcs, -// false, -// BLOCK_BUILDING_DURATION, -// true, -// ); - -// // issue a new payload to the execution layer. -// let timestamp = -// SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); -// engine_driver.handle_build_new_payload(ScrollPayloadAttributes { -// payload_attributes: PayloadAttributes { -// timestamp, -// prev_randao: Default::default(), -// suggested_fee_recipient: Default::default(), -// withdrawals: None, -// parent_beacon_block_root: None, -// }, -// transactions: None, -// no_tx_pool: false, -// block_data_hint: BlockDataHint::none(), -// gas_limit: None, -// }); - -// // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. -// if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); -// } else { -// panic!("expected a new payload event"); -// } -// } - -// #[tokio::test] -// async fn can_build_blocks_and_exit_at_time_limit() { -// reth_tracing::init_test_tracing(); - -// let chain_spec = SCROLL_DEV.clone(); -// const MIN_TRANSACTION_GAS_COST: u64 = 21_000; -// const BLOCK_BUILDING_DURATION: Duration = Duration::from_secs(1); -// const TRANSACTIONS_COUNT: usize = 2000; - -// // setup a test node. use a low payload building duration in order to exit before we reach -// the // gas limit. -// let (mut nodes, _tasks, wallet) = setup_engine( -// ScrollRollupNodeConfig { -// sequencer_args: SequencerArgs { payload_building_duration: 10, ..Default::default() -// }, ..default_test_scroll_rollup_node_config() -// }, -// 1, -// chain_spec, -// false, -// false, -// ) -// .await -// .unwrap(); -// let node = nodes.pop().unwrap(); -// let wallet = Arc::new(Mutex::new(wallet)); - -// // add transactions. -// let mut wallet_lock = wallet.lock().await; -// for _ in 0..TRANSACTIONS_COUNT { -// let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( -// wallet_lock.chain_id, -// wallet_lock.inner.clone(), -// wallet_lock.inner_nonce, -// ) -// .await; -// wallet_lock.inner_nonce += 1; -// node.rpc.inject_tx(raw_tx).await.unwrap(); -// } -// drop(wallet_lock); - -// // create a forkchoice state -// let genesis_hash = node.inner.chain_spec().genesis_hash(); -// let fcs = ForkchoiceState::new( -// BlockInfo { hash: genesis_hash, number: 0 }, -// Default::default(), -// Default::default(), -// ); - -// // create the engine driver connected to the node -// let auth_client = node.inner.engine_http_client(); -// let engine_client = ScrollAuthApiEngineClient::new(auth_client); -// let mut engine_driver = EngineDriver::new( -// Arc::new(engine_client), -// (*SCROLL_DEV).clone(), -// None::, -// fcs, -// false, -// BLOCK_BUILDING_DURATION, -// true, -// ); - -// // start timer. -// let start = Instant::now(); - -// // issue a new payload to the execution layer. -// let timestamp = -// SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); -// engine_driver.handle_build_new_payload(ScrollPayloadAttributes { -// payload_attributes: PayloadAttributes { -// timestamp, -// prev_randao: Default::default(), -// suggested_fee_recipient: Default::default(), -// withdrawals: None, -// parent_beacon_block_root: None, -// }, -// transactions: None, -// no_tx_pool: false, -// block_data_hint: BlockDataHint::none(), -// gas_limit: None, -// }); - -// if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// let payload_building_duration = start.elapsed(); -// // verify that the block building duration is within 10% of the target (we allow for 10% -// // mismatch due to slower performance of debug mode). -// assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); -// assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); -// } else { -// panic!("expected a new payload event"); -// } -// } - -// #[tokio::test] -// async fn should_limit_l1_message_cumulative_gas() { -// reth_tracing::init_test_tracing(); - -// let chain_spec = SCROLL_DEV.clone(); -// const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - -// // setup a test node -// let (mut nodes, _tasks, wallet) = -// setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) -// .await -// .unwrap(); -// let node = nodes.pop().unwrap(); -// let wallet = Arc::new(Mutex::new(wallet)); - -// // create a forkchoice state -// let genesis_hash = node.inner.chain_spec().genesis_hash(); -// let fcs = ForkchoiceState::new( -// BlockInfo { hash: genesis_hash, number: 0 }, -// Default::default(), -// Default::default(), -// ); - -// // create the engine driver connected to the node -// let auth_client = node.inner.engine_http_client(); -// let engine_client = ScrollAuthApiEngineClient::new(auth_client); -// let mut engine_driver = EngineDriver::new( -// Arc::new(engine_client), -// (*SCROLL_DEV).clone(), -// None::, -// fcs, -// false, -// BLOCK_BUILDING_DURATION, -// true, -// ); - -// // create a test database -// let database = Arc::new(setup_test_db().await); -// let provider = database.clone(); - -// // create a sequencer with Finalized mode -// let mut sequencer = Sequencer::new( -// provider, -// Default::default(), -// SCROLL_GAS_LIMIT, -// 4, -// 5, // current L1 block number -// L1MessageInclusionMode::Finalized, -// 0, -// ); -// sequencer.set_l1_finalized_block_number(1); - -// // add L1 messages to database -// let wallet_lock = wallet.lock().await; -// let l1_messages = [ -// L1MessageEnvelope { -// l1_block_number: 1, -// l2_block_number: None, -// queue_hash: None, -// transaction: TxL1Message { -// queue_index: 0, -// gas_limit: SCROLL_GAS_LIMIT / 2, -// to: Address::random(), -// value: U256::from(1), -// sender: wallet_lock.inner.address(), -// input: vec![].into(), -// }, -// }, -// L1MessageEnvelope { -// l1_block_number: 1, -// l2_block_number: None, -// queue_hash: None, -// transaction: TxL1Message { -// queue_index: 1, -// gas_limit: SCROLL_GAS_LIMIT / 2 + 1, -// to: Address::random(), -// value: U256::from(1), -// sender: wallet_lock.inner.address(), -// input: vec![].into(), -// }, -// }, -// ]; -// let tx = database.tx_mut().await.unwrap(); -// for l1_message in l1_messages { -// tx.insert_l1_message(l1_message).await.unwrap(); -// } -// tx.commit().await.unwrap(); - -// // build payload, should only include first l1 message -// sequencer.start_payload_building(); -// let payload_attributes = sequencer.next().await.unwrap(); -// engine_driver.handle_build_new_payload(payload_attributes); - -// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// block -// } else { -// panic!("expected a new payload event"); -// }; - -// // verify only one L1 message is included -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.header.gas_used, 21_000); - -// // sleep 1 seconds (ethereum header timestamp has granularity of seconds and proceeding -// header // must have a greater timestamp than the last) -// tokio::time::sleep(Duration::from_secs(1)).await; - -// // build new payload -// sequencer.start_payload_building(); -// let payload_attributes = sequencer.next().await.unwrap(); -// engine_driver.handle_build_new_payload(payload_attributes); - -// let block = if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { -// block -// } else { -// panic!("expected a new payload event"); -// }; - -// // now should include the next l1 message. -// assert_eq!(block.body.transactions.len(), 1); -// assert_eq!(block.header.gas_used(), 21_000); -// } +#[tokio::test] +async fn can_build_blocks_with_finalized_l1_messages() { + reth_tracing::init_test_tracing(); + + let chain_spec = SCROLL_DEV.clone(); + // setup a test node + let (mut nodes, _tasks, wallet) = + setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) + .await + .unwrap(); + let node = nodes.pop().unwrap(); + let wallet = Arc::new(Mutex::new(wallet)); + + // create a forkchoice state + let genesis_hash = node.inner.chain_spec().genesis_hash(); + let fcs = ForkchoiceState::new( + BlockInfo { hash: genesis_hash, number: 0 }, + Default::default(), + Default::default(), + ); + + // create the engine driver connected to the node + let auth_client = node.inner.engine_http_client(); + let engine_client = ScrollAuthApiEngineClient::new(auth_client); + let mut engine = Engine::new(Arc::new(engine_client), fcs); + + // create a test database + let database = Arc::new(setup_test_db().await); + let provider = database.clone(); + + let tx = provider.tx_mut().await.unwrap(); + tx.set_latest_l1_block_number(5).await.unwrap(); + tx.commit().await.unwrap(); + + // create a sequencer + let config = SequencerConfig { + chain_spec: node.inner.chain_spec().clone(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::Finalized, + }, + block_time: 0, + payload_building_duration: 0, + allow_empty_blocks: true, + }; + let mut sequencer = Sequencer::new(provider, config); + + // set L1 finalized block number to 2 + let tx = database.tx_mut().await.unwrap(); + tx.set_finalized_l1_block_number(2).await.unwrap(); + tx.commit().await.unwrap(); + + // add L1 messages to database + let wallet_lock = wallet.lock().await; + + // this message should be included (before finalized block) + let finalized_l1_message = L1MessageEnvelope { + l1_block_number: 2, // <= 2 (finalized block) + l2_block_number: None, + queue_hash: None, + transaction: TxL1Message { + queue_index: 0, + gas_limit: 21000, + to: Address::random(), + value: U256::from(1), + sender: wallet_lock.inner.address(), + input: vec![].into(), + }, + }; + + // this message should not be included (after finalized block) + let unfinalized_l1_message = L1MessageEnvelope { + l1_block_number: 3, // > 2 (finalized block) + l2_block_number: None, + queue_hash: None, + transaction: TxL1Message { + queue_index: 1, + gas_limit: 21000, + to: Address::random(), + value: U256::from(2), + sender: wallet_lock.inner.address(), + input: vec![].into(), + }, + }; + drop(wallet_lock); + + let finalized_message_hash = finalized_l1_message.transaction.tx_hash(); + let unfinalized_message_hash = unfinalized_l1_message.transaction.tx_hash(); + + let tx = database.tx_mut().await.unwrap(); + tx.insert_l1_message(finalized_l1_message).await.unwrap(); + tx.insert_l1_message(unfinalized_l1_message).await.unwrap(); + tx.commit().await.unwrap(); + + // build payload, should only include finalized message + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() + } else { + panic!("expected a payload ready event"); + }; + + // verify only finalized L1 message is included + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &finalized_message_hash); + + // ensure unfinalized message is not included + assert!(!block.body.transactions.iter().any(|tx| tx.tx_hash() == &unfinalized_message_hash)); + + // Handle the build block with the sequencer in order to update L1 message queue index. + let tx = database.tx_mut().await.unwrap(); + tx.update_l1_messages_with_l2_block((&block).into()).await.unwrap(); + tx.commit().await.unwrap(); + + // update finalized block number to 3, now both messages should be available + let tx = database.tx_mut().await.unwrap(); + tx.set_finalized_l1_block_number(3).await.unwrap(); + tx.commit().await.unwrap(); + + // sleep 2 seconds (ethereum header timestamp has granularity of seconds and proceeding header + // must have a greater timestamp than the last) + tokio::time::sleep(std::time::Duration::from_secs(2)).await; + + // build new payload + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() + } else { + panic!("expected a payload ready event"); + }; + + // now should include the previously unfinalized message + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &unfinalized_message_hash); +} + +#[tokio::test] +async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + + // Create temporary private key file + let mut temp_file = NamedTempFile::new()?; + let private_key_hex = "0x1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; + temp_file.write_all(private_key_hex.as_bytes())?; + temp_file.flush()?; + + // Create expected signer + let expected_key_bytes = + hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef")?; + let expected_signer = alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes)?; + let expected_address = expected_signer.address(); + + let chain_spec = (*SCROLL_DEV).clone(); + let rollup_manager_args = ScrollRollupNodeConfig { + test: false, // disable test mode to enable real signing + network_args: RollupNodeNetworkArgs::default(), + database_args: RollupNodeDatabaseArgs { + rn_db_path: Some(PathBuf::from("sqlite::memory:")), + }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), + sequencer_args: SequencerArgs { + sequencer_enabled: true, + auto_start: false, + block_time: 0, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + payload_building_duration: 1000, + allow_empty_blocks: true, + ..SequencerArgs::default() + }, + blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, + signer_args: SignerArgs { + key_file: Some(temp_file.path().to_path_buf()), + aws_kms_key_id: None, + private_key: None, + }, + gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + database: None, + rpc_args: RpcArgs::default(), + }; + + let (nodes, _tasks, wallet) = + setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; + let wallet = Arc::new(Mutex::new(wallet)); + + let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); + let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + + // Generate and inject transaction + let mut wallet_lock = wallet.lock().await; + let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( + wallet_lock.chain_id, + wallet_lock.inner.clone(), + wallet_lock.inner_nonce, + ) + .await; + wallet_lock.inner_nonce += 1; + drop(wallet_lock); + let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; + + // Build block + sequencer_rnm_handle.build_block().await; + + // Verify block was successfully sequenced + if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); + } else { + panic!("Failed to receive BlockSequenced event"); + } + + // Verify signing event and signature correctness + if let Some(ChainOrchestratorEvent::SignedBlock { block: signed_block, signature }) = + sequencer_events.next().await + { + let hash = sig_encode_hash(&signed_block); + let recovered_address = signature.recover_address_from_prehash(&hash)?; + assert_eq!(recovered_address, expected_address); + } else { + panic!("Failed to receive SignerEvent with signed block"); + } + + Ok(()) +} + +#[tokio::test] +async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + + // Create temporary private key file (without 0x prefix) + let mut temp_file = NamedTempFile::new().unwrap(); + let private_key_hex = "1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef"; + temp_file.write_all(private_key_hex.as_bytes()).unwrap(); + temp_file.flush().unwrap(); + + // Create expected signer + let expected_key_bytes = + hex::decode("1234567890abcdef1234567890abcdef1234567890abcdef1234567890abcdef").unwrap(); + let expected_signer = + alloy_signer_local::PrivateKeySigner::from_slice(&expected_key_bytes).unwrap(); + let expected_address = expected_signer.address(); + + let chain_spec = (*SCROLL_DEV).clone(); + let rollup_manager_args = ScrollRollupNodeConfig { + test: false, // disable test mode to enable real signing + network_args: RollupNodeNetworkArgs::default(), + database_args: RollupNodeDatabaseArgs { + rn_db_path: Some(PathBuf::from("sqlite::memory:")), + }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), + sequencer_args: SequencerArgs { + sequencer_enabled: true, + auto_start: false, + block_time: 0, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + payload_building_duration: 1000, + allow_empty_blocks: true, + ..SequencerArgs::default() + }, + blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, + signer_args: SignerArgs { + key_file: Some(temp_file.path().to_path_buf()), + aws_kms_key_id: None, + private_key: None, + }, + gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + database: None, + rpc_args: RpcArgs::default(), + }; + + let (nodes, _tasks, wallet) = + setup_engine(rollup_manager_args, 1, chain_spec, false, false).await?; + let wallet = Arc::new(Mutex::new(wallet)); + + let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); + let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + + // Generate and inject transaction + let mut wallet_lock = wallet.lock().await; + let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( + wallet_lock.chain_id, + wallet_lock.inner.clone(), + wallet_lock.inner_nonce, + ) + .await; + wallet_lock.inner_nonce += 1; + drop(wallet_lock); + let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; + + // Build block + sequencer_rnm_handle.build_block().await; + + // Verify block was successfully sequenced + if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.body.transactions.first().unwrap().tx_hash(), &tx_hash); + } else { + panic!("Failed to receive BlockSequenced event"); + } + + // Verify signing event and signature correctness + while let Some(event) = sequencer_events.next().await { + if let ChainOrchestratorEvent::SignedBlock { block: signed_block, signature } = event { + let hash = sig_encode_hash(&signed_block); + let recovered_address = signature.recover_address_from_prehash(&hash)?; + assert_eq!(recovered_address, expected_address); + break; + } + } + + Ok(()) +} + +#[tokio::test] +async fn can_build_blocks_and_exit_at_gas_limit() { + reth_tracing::init_test_tracing(); + + let chain_spec = SCROLL_DEV.clone(); + const MIN_TRANSACTION_GAS_COST: u64 = 21_000; + const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); + const TRANSACTIONS_COUNT: usize = 2000; + + // setup a test node. use a high value for the payload building duration to be sure we don't + // exit early. + let (mut nodes, _tasks, wallet) = setup_engine( + ScrollRollupNodeConfig { + sequencer_args: SequencerArgs { payload_building_duration: 1000, ..Default::default() }, + ..default_test_scroll_rollup_node_config() + }, + 1, + chain_spec, + false, + false, + ) + .await + .unwrap(); + let node = nodes.pop().unwrap(); + let wallet = Arc::new(Mutex::new(wallet)); + + // add transactions. + let mut wallet_lock = wallet.lock().await; + for _ in 0..TRANSACTIONS_COUNT { + let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( + wallet_lock.chain_id, + wallet_lock.inner.clone(), + wallet_lock.inner_nonce, + ) + .await; + wallet_lock.inner_nonce += 1; + node.rpc.inject_tx(raw_tx).await.unwrap(); + } + drop(wallet_lock); + + // create a forkchoice state + let genesis_hash = node.inner.chain_spec().genesis_hash(); + let fcs = ForkchoiceState::new( + BlockInfo { hash: genesis_hash, number: 0 }, + Default::default(), + Default::default(), + ); + + // create the engine driver connected to the node + let auth_client = node.inner.engine_http_client(); + let engine_client = ScrollAuthApiEngineClient::new(auth_client); + let mut engine_driver = EngineDriver::new( + Arc::new(engine_client), + (*SCROLL_DEV).clone(), + None::, + fcs, + false, + BLOCK_BUILDING_DURATION, + true, + ); + + // issue a new payload to the execution layer. + let timestamp = + SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); + engine_driver.handle_build_new_payload(ScrollPayloadAttributes { + payload_attributes: PayloadAttributes { + timestamp, + prev_randao: Default::default(), + suggested_fee_recipient: Default::default(), + withdrawals: None, + parent_beacon_block_root: None, + }, + transactions: None, + no_tx_pool: false, + block_data_hint: BlockDataHint::none(), + gas_limit: None, + }); + + // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. + if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { + assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); + } else { + panic!("expected a new payload event"); + } +} + +#[tokio::test] +async fn can_build_blocks_and_exit_at_time_limit() { + reth_tracing::init_test_tracing(); + + let chain_spec = SCROLL_DEV.clone(); + const MIN_TRANSACTION_GAS_COST: u64 = 21_000; + const BLOCK_BUILDING_DURATION: Duration = Duration::from_secs(1); + const TRANSACTIONS_COUNT: usize = 2000; + + // setup a test node. use a low payload building duration in order to exit before we reach the + // gas limit. + let (mut nodes, _tasks, wallet) = setup_engine( + ScrollRollupNodeConfig { + sequencer_args: SequencerArgs { payload_building_duration: 10, ..Default::default() }, + ..default_test_scroll_rollup_node_config() + }, + 1, + chain_spec, + false, + false, + ) + .await + .unwrap(); + let node = nodes.pop().unwrap(); + let wallet = Arc::new(Mutex::new(wallet)); + + // add transactions. + let mut wallet_lock = wallet.lock().await; + for _ in 0..TRANSACTIONS_COUNT { + let raw_tx = TransactionTestContext::transfer_tx_nonce_bytes( + wallet_lock.chain_id, + wallet_lock.inner.clone(), + wallet_lock.inner_nonce, + ) + .await; + wallet_lock.inner_nonce += 1; + node.rpc.inject_tx(raw_tx).await.unwrap(); + } + drop(wallet_lock); + + // create a forkchoice state + let genesis_hash = node.inner.chain_spec().genesis_hash(); + let fcs = ForkchoiceState::new( + BlockInfo { hash: genesis_hash, number: 0 }, + Default::default(), + Default::default(), + ); + + // create the engine driver connected to the node + let auth_client = node.inner.engine_http_client(); + let engine_client = ScrollAuthApiEngineClient::new(auth_client); + let mut engine_driver = EngineDriver::new( + Arc::new(engine_client), + (*SCROLL_DEV).clone(), + None::, + fcs, + false, + BLOCK_BUILDING_DURATION, + true, + ); + + // start timer. + let start = Instant::now(); + + // issue a new payload to the execution layer. + let timestamp = + SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); + engine_driver.handle_build_new_payload(ScrollPayloadAttributes { + payload_attributes: PayloadAttributes { + timestamp, + prev_randao: Default::default(), + suggested_fee_recipient: Default::default(), + withdrawals: None, + parent_beacon_block_root: None, + }, + transactions: None, + no_tx_pool: false, + block_data_hint: BlockDataHint::none(), + gas_limit: None, + }); + + if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { + let payload_building_duration = start.elapsed(); + // verify that the block building duration is within 10% of the target (we allow for 10% + // mismatch due to slower performance of debug mode). + assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); + assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); + } else { + panic!("expected a new payload event"); + } +} + +#[tokio::test] +async fn should_limit_l1_message_cumulative_gas() { + reth_tracing::init_test_tracing(); + + let chain_spec = SCROLL_DEV.clone(); + const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); + + // setup a test node + let (mut nodes, _tasks, wallet) = + setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) + .await + .unwrap(); + let node = nodes.pop().unwrap(); + let wallet = Arc::new(Mutex::new(wallet)); + + // create a forkchoice state + let genesis_hash = node.inner.chain_spec().genesis_hash(); + let fcs = ForkchoiceState::new( + BlockInfo { hash: genesis_hash, number: 0 }, + Default::default(), + Default::default(), + ); + + // create the engine driver connected to the node + let auth_client = node.inner.engine_http_client(); + let engine_client = ScrollAuthApiEngineClient::new(auth_client); + let mut engine = Engine::new(Arc::new(engine_client), fcs); + + // create a test database + let database = Arc::new(setup_test_db().await); + let provider = database.clone(); + + // Set the latest and finalized block number + let tx = provider.tx_mut().await.unwrap(); + tx.set_latest_l1_block_number(5).await.unwrap(); + tx.set_finalized_l1_block_number(1).await.unwrap(); + tx.commit().await.unwrap(); + + // create a sequencer + let config = SequencerConfig { + chain_spec: node.inner.chain_spec().clone(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::Finalized, + }, + block_time: 0, + payload_building_duration: 0, + allow_empty_blocks: true, + }; + let mut sequencer = Sequencer::new(provider, config); + + // add L1 messages to database + let wallet_lock = wallet.lock().await; + let l1_messages = [ + L1MessageEnvelope { + l1_block_number: 1, + l2_block_number: None, + queue_hash: None, + transaction: TxL1Message { + queue_index: 0, + gas_limit: SCROLL_GAS_LIMIT / 2, + to: Address::random(), + value: U256::from(1), + sender: wallet_lock.inner.address(), + input: vec![].into(), + }, + }, + L1MessageEnvelope { + l1_block_number: 1, + l2_block_number: None, + queue_hash: None, + transaction: TxL1Message { + queue_index: 1, + gas_limit: SCROLL_GAS_LIMIT / 2 + 1, + to: Address::random(), + value: U256::from(1), + sender: wallet_lock.inner.address(), + input: vec![].into(), + }, + }, + ]; + let tx = database.tx_mut().await.unwrap(); + for l1_message in l1_messages { + tx.insert_l1_message(l1_message).await.unwrap(); + } + tx.commit().await.unwrap(); + + // build payload, should only include first l1 message + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let block = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(block.is_some(), "expected a new payload, but got: {:?}", block); + block.unwrap() + } else { + panic!("expected a payload ready event"); + }; + + // verify only one L1 message is included + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.header.gas_used, 21_000); + + // sleep 1 seconds (ethereum header timestamp has granularity of seconds and proceeding header + // must have a greater timestamp than the last) + tokio::time::sleep(Duration::from_secs(1)).await; + + // build new payload + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let block = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(block.is_some(), "expected a new payload, but got: {:?}", block); + block.unwrap() + } else { + panic!("expected a payload ready event"); + }; + + // now should include the next l1 message. + assert_eq!(block.body.transactions.len(), 1); + assert_eq!(block.header.gas_used(), 21_000); +} From 3708f68c568361d93fa49ffba95b02fc9f3b682a Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 8 Oct 2025 08:47:11 +0800 Subject: [PATCH 03/12] atomic rollup node --- Cargo.lock | 49 - Cargo.toml | 2 - crates/chain-orchestrator/Cargo.toml | 13 + crates/chain-orchestrator/src/config.rs | 8 +- .../chain-orchestrator/src/consolidation.rs | 6 +- crates/chain-orchestrator/src/event.rs | 16 + .../chain-orchestrator/src/handle/command.rs | 5 +- crates/chain-orchestrator/src/handle/mod.rs | 16 +- crates/chain-orchestrator/src/lib.rs | 303 ++++--- crates/chain-orchestrator/src/sync.rs | 17 +- crates/database/db/src/db.rs | 8 +- crates/database/db/src/operations.rs | 16 +- .../derivation-pipeline/benches/pipeline.rs | 2 +- crates/derivation-pipeline/src/lib.rs | 565 ++++-------- crates/derivation-pipeline/src/new.rs | 229 ----- crates/engine/Cargo.toml | 14 - crates/engine/src/api.rs | 89 -- crates/engine/src/driver.rs | 597 ------------- crates/engine/src/{new.rs => engine.rs} | 42 +- crates/engine/src/error.rs | 4 +- crates/engine/src/event.rs | 28 - crates/engine/src/fcs.rs | 6 +- crates/engine/src/future/mod.rs | 397 --------- crates/engine/src/future/result.rs | 51 -- crates/engine/src/lib.rs | 17 +- crates/engine/src/metrics.rs | 2 +- crates/manager/Cargo.toml | 83 -- crates/manager/src/consensus.rs | 163 ---- crates/manager/src/lib.rs | 9 - crates/manager/src/manager/budget.rs | 28 - crates/manager/src/manager/command.rs | 27 - crates/manager/src/manager/event.rs | 41 - crates/manager/src/manager/handle.rs | 75 -- crates/manager/src/manager/metrics.rs | 48 - crates/manager/src/manager/mod.rs | 758 ---------------- crates/network/Cargo.toml | 7 + crates/network/src/handle.rs | 14 +- crates/network/src/manager.rs | 17 + crates/node/Cargo.toml | 4 +- crates/node/src/args.rs | 21 +- crates/node/tests/e2e.rs | 182 ++-- crates/node/tests/sync.rs | 838 +++++++++++------- crates/primitives/src/batch.rs | 9 +- crates/sequencer/Cargo.toml | 1 - crates/sequencer/src/config.rs | 6 +- crates/sequencer/src/lib.rs | 8 +- crates/sequencer/tests/e2e.rs | 177 ++-- 47 files changed, 1226 insertions(+), 3792 deletions(-) delete mode 100644 crates/derivation-pipeline/src/new.rs delete mode 100644 crates/engine/src/api.rs delete mode 100644 crates/engine/src/driver.rs rename crates/engine/src/{new.rs => engine.rs} (69%) delete mode 100644 crates/engine/src/event.rs delete mode 100644 crates/engine/src/future/mod.rs delete mode 100644 crates/engine/src/future/result.rs delete mode 100644 crates/manager/Cargo.toml delete mode 100644 crates/manager/src/consensus.rs delete mode 100644 crates/manager/src/lib.rs delete mode 100644 crates/manager/src/manager/budget.rs delete mode 100644 crates/manager/src/manager/command.rs delete mode 100644 crates/manager/src/manager/event.rs delete mode 100644 crates/manager/src/manager/handle.rs delete mode 100644 crates/manager/src/manager/metrics.rs delete mode 100644 crates/manager/src/manager/mod.rs diff --git a/Cargo.lock b/Cargo.lock index cf4a39c7..cb7be688 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10782,7 +10782,6 @@ dependencies = [ "reth-trie-db", "rollup-node", "rollup-node-chain-orchestrator", - "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", "rollup-node-sequencer", @@ -10861,48 +10860,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "rollup-node-manager" -version = "0.0.1" -dependencies = [ - "alloy-chains", - "alloy-consensus 1.0.30", - "alloy-eips 1.0.30", - "alloy-primitives", - "alloy-provider", - "alloy-rpc-types-engine 1.0.30", - "futures", - "metrics", - "metrics-derive", - "reth-chainspec", - "reth-network", - "reth-network-api", - "reth-primitives-traits", - "reth-scroll-node", - "reth-scroll-primitives", - "reth-tasks", - "reth-tokio-util", - "rollup-node-chain-orchestrator", - "rollup-node-primitives", - "rollup-node-providers", - "rollup-node-sequencer", - "rollup-node-signer", - "rollup-node-watcher", - "scroll-alloy-consensus", - "scroll-alloy-hardforks", - "scroll-alloy-network", - "scroll-alloy-provider", - "scroll-alloy-rpc-types-engine", - "scroll-db", - "scroll-derivation-pipeline", - "scroll-engine", - "scroll-network", - "scroll-wire", - "tokio", - "tokio-stream", - "tracing", -] - [[package]] name = "rollup-node-primitives" version = "0.0.1" @@ -10975,7 +10932,6 @@ dependencies = [ "reth-tracing", "rollup-node", "rollup-node-chain-orchestrator", - "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", "rollup-node-sequencer", @@ -11626,16 +11582,11 @@ dependencies = [ "reth-scroll-primitives", "reth-testing-utils", "rollup-node-primitives", - "rollup-node-providers", - "rollup-node-signer", "scroll-alloy-consensus", "scroll-alloy-hardforks", "scroll-alloy-network", "scroll-alloy-provider", "scroll-alloy-rpc-types-engine", - "scroll-db", - "scroll-engine", - "scroll-network", "thiserror 2.0.16", "tokio", "tracing", diff --git a/Cargo.toml b/Cargo.toml index 054824ac..754d2d3e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -14,7 +14,6 @@ members = [ "crates/engine", "crates/chain-orchestrator", "crates/l1", - "crates/manager", "crates/network", "crates/node", "crates/primitives", @@ -191,7 +190,6 @@ reth-scroll-primitives = { git = "https://github.com/scroll-tech/reth.git", defa # rollup node rollup-node = { path = "crates/node" } rollup-node-chain-orchestrator = { path = "crates/chain-orchestrator" } -rollup-node-manager = { path = "crates/manager" } rollup-node-primitives = { path = "crates/primitives" } rollup-node-providers = { path = "crates/providers" } rollup-node-sequencer = { path = "crates/sequencer" } diff --git a/crates/chain-orchestrator/Cargo.toml b/crates/chain-orchestrator/Cargo.toml index 0bf42e7c..668f9c0e 100644 --- a/crates/chain-orchestrator/Cargo.toml +++ b/crates/chain-orchestrator/Cargo.toml @@ -86,3 +86,16 @@ rand.workspace = true reqwest.workspace = true serde_json.workspace = true tokio.workspace = true + +[features] +test-utils = [ + "reth-chainspec/test-utils", + "reth-network-p2p/test-utils", + "reth-primitives-traits/test-utils", + "reth-scroll-node/test-utils", + "rollup-node-providers/test-utils", + "rollup-node-watcher/test-utils", + "scroll-db/test-utils", + "scroll-engine/test-utils", + "scroll-network/test-utils", +] diff --git a/crates/chain-orchestrator/src/config.rs b/crates/chain-orchestrator/src/config.rs index 9c4eb108..5eec9ba5 100644 --- a/crates/chain-orchestrator/src/config.rs +++ b/crates/chain-orchestrator/src/config.rs @@ -14,7 +14,7 @@ pub struct ChainOrchestratorConfig { impl ChainOrchestratorConfig { /// Creates a new chain configuration. - pub fn new( + pub const fn new( chain_spec: Arc, optimistic_sync_threshold: u64, l1_v2_message_queue_start_index: u64, @@ -23,17 +23,17 @@ impl ChainOrchestratorConfig { } /// Returns a reference to the chain specification. - pub fn chain_spec(&self) -> &Arc { + pub const fn chain_spec(&self) -> &Arc { &self.chain_spec } /// Returns the optimistic sync threshold. - pub fn optimistic_sync_threshold(&self) -> u64 { + pub const fn optimistic_sync_threshold(&self) -> u64 { self.optimistic_sync_threshold } /// Returns the L1 message queue index at which the V2 L1 message queue was enabled. - pub fn l1_v2_message_queue_start_index(&self) -> u64 { + pub const fn l1_v2_message_queue_start_index(&self) -> u64 { self.l1_v2_message_queue_start_index } } diff --git a/crates/chain-orchestrator/src/consolidation.rs b/crates/chain-orchestrator/src/consolidation.rs index 6cd2f3b3..81ca043f 100644 --- a/crates/chain-orchestrator/src/consolidation.rs +++ b/crates/chain-orchestrator/src/consolidation.rs @@ -84,11 +84,11 @@ pub(crate) enum BlockConsolidationAction { impl std::fmt::Display for BlockConsolidationAction { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - BlockConsolidationAction::UpdateSafeHead(info) => { + Self::UpdateSafeHead(info) => { write!(f, "UpdateSafeHead to block {}", info.number) } - BlockConsolidationAction::Skip(info) => write!(f, "Skip block {}", info.number), - BlockConsolidationAction::Reorg(attrs) => { + Self::Skip(info) => write!(f, "Skip block {}", info.number), + Self::Reorg(attrs) => { write!(f, "Reorg to block {}", attrs.block_number) } } diff --git a/crates/chain-orchestrator/src/event.rs b/crates/chain-orchestrator/src/event.rs index aba25235..47bf267a 100644 --- a/crates/chain-orchestrator/src/event.rs +++ b/crates/chain-orchestrator/src/event.rs @@ -85,10 +85,26 @@ pub enum ChainOrchestratorEvent { }, /// An L1 message was not found in the database. L1MessageNotFoundInDatabase(L1MessageKey), + /// A block contains invalid L1 messages. + L1MessageMismatch { + /// The expected L1 messages hash. + expected: B256, + /// The actual L1 messages hash. + actual: B256, + }, /// A new block has been received from the network. NewBlockReceived(NewBlockWithPeer), /// A batch consolidation has occurred, returning the outcome of the consolidation. BatchConsolidated(BatchConsolidationOutcome), /// An L1 derived block has been consolidated, returning the outcome of the consolidation. BlockConsolidated(BlockConsolidationOutcome), + /// The chain has been consolidated from `from` block number to `to` block number (inclusive). + ChainConsolidated { + /// The starting block number of the consolidation (safe block number). + from: u64, + /// The ending block number of the consolidation (head block number). + to: u64, + }, + /// The head of the fork choice state has been updated in the engine driver. + FcsHeadUpdated(BlockInfo), } diff --git a/crates/chain-orchestrator/src/handle/command.rs b/crates/chain-orchestrator/src/handle/command.rs index 5f7ca5b4..a485c1eb 100644 --- a/crates/chain-orchestrator/src/handle/command.rs +++ b/crates/chain-orchestrator/src/handle/command.rs @@ -19,9 +19,12 @@ pub enum ChainOrchestratorCommand>), /// Update the head of the fcs in the engine driver. - UpdateFcsHead(BlockInfo), + UpdateFcsHead((BlockInfo, oneshot::Sender<()>)), /// Enable automatic sequencing. EnableAutomaticSequencing(oneshot::Sender), /// Disable automatic sequencing. DisableAutomaticSequencing(oneshot::Sender), + /// Enable gossiping of blocks to peers. + #[cfg(feature = "test-utils")] + SetGossip((bool, oneshot::Sender<()>)), } diff --git a/crates/chain-orchestrator/src/handle/mod.rs b/crates/chain-orchestrator/src/handle/mod.rs index 3d0e74ac..329e088a 100644 --- a/crates/chain-orchestrator/src/handle/mod.rs +++ b/crates/chain-orchestrator/src/handle/mod.rs @@ -39,7 +39,7 @@ impl> ChainOrchestratorHand } /// Sends a command to the rollup manager to build a block. - pub async fn build_block(&self) { + pub fn build_block(&self) { self.send_command(ChainOrchestratorCommand::BuildBlock); } @@ -63,8 +63,10 @@ impl> ChainOrchestratorHand } /// Sends a command to the rollup manager to update the head of the FCS in the engine driver. - pub async fn update_fcs_head(&self, head: BlockInfo) { - self.send_command(ChainOrchestratorCommand::UpdateFcsHead(head)); + pub async fn update_fcs_head(&self, head: BlockInfo) -> Result<(), oneshot::error::RecvError> { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::UpdateFcsHead((head, tx))); + rx.await } /// Sends a command to the rollup manager to enable automatic sequencing. @@ -87,4 +89,12 @@ impl> ChainOrchestratorHand self.send_command(ChainOrchestratorCommand::Status(tx)); rx.await } + + /// Sends a command to the rollup manager to enable or disable gossiping of blocks to peers. + #[cfg(feature = "test-utils")] + pub async fn set_gossip(&self, enabled: bool) -> Result<(), oneshot::error::RecvError> { + let (tx, rx) = oneshot::channel(); + self.send_command(ChainOrchestratorCommand::SetGossip((enabled, tx))); + rx.await + } } diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index 9b4da549..733c357b 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -30,7 +30,7 @@ use scroll_db::{ Database, DatabaseError, DatabaseReadOperations, DatabaseTransactionProvider, DatabaseWriteOperations, L1MessageKey, UnwindResult, }; -use scroll_derivation_pipeline::{BatchDerivationResult, DerivationPipelineNew}; +use scroll_derivation_pipeline::{BatchDerivationResult, DerivationPipeline}; use scroll_engine::Engine; use scroll_network::{ BlockImportOutcome, NewBlockWithPeer, ScrollNetwork, ScrollNetworkManagerEvent, @@ -80,10 +80,10 @@ use crate::consolidation::BlockConsolidationAction; const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); +/// The number of headers to fetch in each request when fetching headers from peers. const HEADER_FETCH_COUNT: u64 = 4000; -const CONSOLIDATION_BATCH_SIZE: u64 = 500; - +/// The size of the event channel used to broadcast events to listeners. const EVENT_CHANNEL_SIZE: usize = 5000; /// The [`ChainOrchestrator`] is responsible for orchestrating the progression of the L2 chain @@ -124,7 +124,7 @@ pub struct ChainOrchestrator< /// The signer used to sign messages. signer: Option, /// The derivation pipeline used to derive L2 blocks from batches. - derivation_pipeline: DerivationPipelineNew, + derivation_pipeline: DerivationPipeline, /// Optional event sender for broadcasting events to listeners. event_sender: Option>, } @@ -139,6 +139,7 @@ impl< > ChainOrchestrator { /// Creates a new chain orchestrator. + #[allow(clippy::too_many_arguments)] pub async fn new( database: Arc, config: ChainOrchestratorConfig, @@ -150,7 +151,7 @@ impl< engine: Engine, sequencer: Option>, signer: Option, - derivation_pipeline: DerivationPipelineNew, + derivation_pipeline: DerivationPipeline, ) -> Result<(Self, ChainOrchestratorHandle), ChainOrchestratorError> { let (handle_tx, handle_rx) = mpsc::unbounded_channel(); let handle = ChainOrchestratorHandle::new(handle_tx); @@ -211,7 +212,7 @@ impl< } else { unreachable!() } - }, if self.sequencer.is_some() => { + }, if self.sequencer.is_some() && self.sync_state.is_synced() => { let res = self.handle_sequencer_event(event).await; self.handle_outcome(res); } @@ -223,7 +224,7 @@ impl< let res = self.handle_network_event(event).await; self.handle_outcome(res); } - Some(notification) = self.l1_notification_rx.recv(), if self.sync_state.l2().is_synced() && self.derivation_pipeline.len().await == 0 => { + Some(notification) = self.l1_notification_rx.recv(), if self.sync_state.l2().is_synced() && self.derivation_pipeline.is_empty().await => { let res = self.handle_l1_notification(notification).await; self.handle_outcome(res); } @@ -238,18 +239,18 @@ impl< &self, outcome: Result, ChainOrchestratorError>, ) { - match (outcome, self.event_sender.as_ref()) { - (Ok(Some(event)), Some(sender)) => sender.notify(event), - (Err(err), _) => { + match outcome { + Ok(Some(event)) => self.notify(event), + Err(err) => { tracing::error!(target: "scroll::chain_orchestrator", ?err, "Encountered error in the chain orchestrator"); } - _ => {} + Ok(None) => {} } } /// Handles an event from the signer. async fn handle_signer_event( - &mut self, + &self, event: rollup_node_signer::SignerEvent, ) -> Result, ChainOrchestratorError> { tracing::info!(target: "scroll::chain_orchestrator", ?event, "Handling signer event"); @@ -337,8 +338,13 @@ impl< ChainOrchestratorCommand::NetworkHandle(tx) => { let _ = tx.send(self.network.handle().clone()); } - ChainOrchestratorCommand::UpdateFcsHead(_head) => { - todo!() + ChainOrchestratorCommand::UpdateFcsHead((head, sender)) => { + self.engine.update_fcs(Some(head), None, None).await?; + let tx = self.database.tx_mut().await?; + tx.purge_l1_message_to_l2_block_mappings(Some(head.number + 1)).await?; + tx.commit().await?; + self.notify(ChainOrchestratorEvent::FcsHeadUpdated(head)); + let _ = sender.send(()); } ChainOrchestratorCommand::EnableAutomaticSequencing(tx) => { if let Some(sequencer) = self.sequencer.as_mut() { @@ -358,6 +364,11 @@ impl< let _ = tx.send(false); } } + #[cfg(feature = "test-utils")] + ChainOrchestratorCommand::SetGossip((enabled, tx)) => { + self.network.handle().set_gossip(enabled).await; + let _ = tx.send(()); + } } Ok(()) @@ -376,6 +387,13 @@ impl< event_listener } + /// Notifies all event listeners of the given event. + fn notify(&self, event: ChainOrchestratorEvent) { + if let Some(s) = self.event_sender.as_ref() { + s.notify(event); + } + } + /// Handles a derived batch by inserting the derived blocks into the database. async fn handle_derived_batch( &mut self, @@ -435,9 +453,9 @@ impl< BlockConsolidationOutcome::Reorged(block_info) } }; - self.event_sender - .as_ref() - .map(|s| s.notify(ChainOrchestratorEvent::BlockConsolidated(outcome.clone()))); + + self.notify(ChainOrchestratorEvent::BlockConsolidated(outcome.clone())); + batch_consolidation_result.push_block(outcome); } @@ -470,12 +488,11 @@ impl< } L1Notification::Synced => { tracing::info!(target: "scroll::chain_orchestrator", "L1 is now synced"); - println!("L1 is now synced"); self.sync_state.l1_mut().set_synced(); if self.sync_state.is_synced() { self.consolidate_chain().await?; } - self.event_sender.as_ref().map(|s| s.notify(ChainOrchestratorEvent::L1Synced)); + self.notify(ChainOrchestratorEvent::L1Synced); Ok(None) } L1Notification::BatchFinalization { hash: _hash, index, block_number } => { @@ -485,7 +502,7 @@ impl< } async fn handle_l1_new_block( - &mut self, + &self, block_number: u64, ) -> Result, ChainOrchestratorError> { Retry::default() @@ -506,7 +523,6 @@ impl< &mut self, block_number: u64, ) -> Result, ChainOrchestratorError> { - println!("handling l1 reorg to block number {}", block_number); let metric = self.metrics.get(&ChainOrchestratorItem::L1Reorg).expect("metric exists"); let now = Instant::now(); let UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info } = @@ -540,31 +556,35 @@ impl< .header .hash_slow(); + // Cancel the inflight payload building job if the head has changed. + if let Some(s) = self.sequencer.as_mut() { + s.cancel_payload_building_job(); + }; + Some(BlockInfo { number: block_number, hash: block_hash }) } else { None }; + // If the L1 reorg is before the origin of the inflight payload building job, cancel it. + if Some(l1_block_number) < + self.sequencer + .as_ref() + .and_then(|s| s.payload_building_job().map(|p| p.l1_origin())) + .flatten() + { + if let Some(s) = self.sequencer.as_mut() { + s.cancel_payload_building_job(); + }; + } + // TODO: Add retry logic if l2_head_block_info.is_some() || l2_safe_block_info.is_some() { self.engine.update_fcs(l2_head_block_info, l2_safe_block_info, None).await?; } - if let Some(block_info) = l2_head_block_info { - Retry::default() - .retry("set_l2_head_block_info", || async { - let tx = self.database.tx_mut().await?; - tx.set_l2_head_block_number(block_info.number).await?; - tx.commit().await?; - Ok::<_, ChainOrchestratorError>(()) - }) - .await?; - } - metric.task_duration.record(now.elapsed().as_secs_f64()); - println!("completed the reorg notification"); - let event = ChainOrchestratorEvent::L1Reorg { l1_block_number, queue_index, @@ -572,8 +592,6 @@ impl< l2_safe_block_info, }; - println!("event {:?}", event); - Ok(Some(event)) } @@ -606,7 +624,7 @@ impl< .await?; for batch in &finalized_batches { - self.derivation_pipeline.push(Arc::new(*batch)).await; + self.derivation_pipeline.push_batch(Arc::new(*batch)).await; } metric.task_duration.record(now.elapsed().as_secs_f64()); @@ -702,7 +720,7 @@ impl< if let Ok(Some(ChainOrchestratorEvent::BatchFinalized(_, batches))) = &event { for batch in batches { - self.derivation_pipeline.push(Arc::new(*batch)).await; + self.derivation_pipeline.push_batch(Arc::new(*batch)).await; } } @@ -774,9 +792,7 @@ impl< ) -> Result, ChainOrchestratorError> { match event { ScrollNetworkManagerEvent::NewBlock(block_with_peer) => { - self.event_sender.as_ref().map(|s| { - s.notify(ChainOrchestratorEvent::NewBlockReceived(block_with_peer.clone())) - }); + self.notify(ChainOrchestratorEvent::NewBlockReceived(block_with_peer.clone())); Ok(self.handle_block_from_peer(block_with_peer).await?) } } @@ -806,11 +822,8 @@ impl< // We optimistically persist the signature upon passing consensus checks. let tx = self.database.tx_mut().await?; - tx.insert_signature( - block_with_peer.block.header.hash_slow(), - block_with_peer.signature.clone(), - ) - .await?; + tx.insert_signature(block_with_peer.block.header.hash_slow(), block_with_peer.signature) + .await?; tx.commit().await?; let received_block_number = block_with_peer.block.number; @@ -829,9 +842,7 @@ impl< self.engine.optimistic_sync(block_info).await?; self.sync_state.l2_mut().set_syncing(); - self.event_sender - .as_ref() - .map(|s| s.notify(ChainOrchestratorEvent::OptimisticSync(block_info))); + self.notify(ChainOrchestratorEvent::OptimisticSync(block_info)); // Purge all L1 message to L2 block mappings as they may be invalid after an // optimistic sync. @@ -848,7 +859,7 @@ impl< let block_number_diff = received_block_number - current_head_number; let new_headers = if received_block_number > current_head_number + 1 { self.block_client - .get_full_block_range(received_block_hash.into(), block_number_diff) + .get_full_block_range(received_block_hash, block_number_diff) .await .into_iter() .rev() @@ -869,22 +880,15 @@ impl< VecDeque::from(new_headers) } else { - // Assert that we are not reorging below the safe head. - let current_safe_info = self.engine.fcs().safe_block_info(); - if received_block_number <= current_safe_info.number { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); - return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); - } - // If the block is less than or equal to the current head check if we already have it in // the chain. - println!("checking if we already have the block in the chain"); let current_chain_block = self .l2_client .get_block_by_number(received_block_number.into()) .full() .await? .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(received_block_number))?; + if current_chain_block.header.hash_slow() == received_block_hash { tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that is already in the chain"); return Ok(Some(ChainOrchestratorEvent::BlockAlreadyKnown( @@ -893,6 +897,13 @@ impl< ))); } + // Assert that we are not reorging below the safe head. + let current_safe_info = self.engine.fcs().safe_block_info(); + if received_block_number <= current_safe_info.number { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + } + // Check to assert that we have received a newer chain. let current_head = self .l2_client @@ -939,7 +950,7 @@ impl< let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_head.number); let headers: Vec = self .block_client - .get_full_block_range(parent_hash.into(), fetch_count) + .get_full_block_range(parent_hash, fetch_count) .await .into_iter() .rev() @@ -967,14 +978,16 @@ impl< } let chain_import = self.import_chain(new_headers.into(), block_with_peer).await?; return Ok(Some(ChainOrchestratorEvent::ChainReorged(chain_import))); - } else { - for header in headers.into_iter().rev() { - new_headers.push_front(header); - } + }; + + // If we did not find a common ancestor, we add all the fetched headers to the front of + // the deque and continue fetching. + for header in headers.into_iter().rev() { + new_headers.push_front(header); } } - return Err(ChainOrchestratorError::L2SafeBlockReorgDetected) + Err(ChainOrchestratorError::L2SafeBlockReorgDetected) } /// Imports a chain of headers into the L2 chain. @@ -993,8 +1006,8 @@ impl< } // Validate the new blocks by sending them to the engine. - for block in chain.iter() { - let status = self.engine.new_payload(&block).await?; + for block in &chain { + let status = self.engine.new_payload(block).await?; tracing::debug!(target: "scroll::chain_orchestrator", block_number = block.number, block_hash = ?block.hash_slow(), ?status, "New payload status from engine"); if status.is_invalid() { @@ -1052,7 +1065,7 @@ impl< Bytes::copy_from_slice(&block_with_peer.signature.sig_as_bytes()), )); - return Ok(ChainImport { + Ok(ChainImport { chain, peer_id: block_with_peer.peer_id, signature: block_with_peer.signature, @@ -1064,39 +1077,51 @@ impl< /// /// This involves validating the L1 messages in the blocks against the expected L1 messages /// synced from L1. - async fn consolidate_chain(&mut self) -> Result<(), ChainOrchestratorError> { + async fn consolidate_chain(&self) -> Result<(), ChainOrchestratorError> { tracing::trace!(target: "scroll::chain_orchestrator", fcs = ?self.engine.fcs(), "Consolidating chain from safe to head"); - let mut start_block_number = self.engine.fcs().safe_block_info().number + 1; - let end_block_number = self.engine.fcs().head_block_info().number; - println!("consolidating chain from block {} to {}", start_block_number, end_block_number); - - while start_block_number <= end_block_number { - let batch_end = - (start_block_number + CONSOLIDATION_BATCH_SIZE - 1).min(end_block_number); - println!("consolidating batch from block {} to {}", start_block_number, batch_end); - tracing::trace!(target: "scroll::chain_orchestrator", start_block = ?start_block_number, end_block = ?batch_end, "Consolidating batch of blocks"); - - // TODO: Make fetching parallel but ensure concurrency limits are respected. - let mut blocks_to_validate = vec![]; - for block_number in start_block_number..=batch_end { - println!("fetching block {}", block_number); - let block = self - .l2_client - .get_block_by_number(block_number.into()) - .full() - .await? - .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_number))? - .into_consensus() - .map_transactions(|tx| tx.inner.into_inner()); - blocks_to_validate.push(block); - } + let safe_block_number = self.engine.fcs().safe_block_info().number; + let head_block_number = self.engine.fcs().head_block_info().number; - self.validate_l1_messages(&blocks_to_validate).await?; + if head_block_number == safe_block_number { + tracing::trace!(target: "scroll::chain_orchestrator", "No unsafe blocks to consolidate"); - start_block_number = batch_end + 1; + self.notify(ChainOrchestratorEvent::ChainConsolidated { + from: safe_block_number, + to: head_block_number, + }); + return Ok(()) } + let start_block_number = safe_block_number + 1; + // TODO: Make fetching parallel but ensure concurrency limits are respected. + let mut blocks_to_validate = vec![]; + for block_number in start_block_number..=head_block_number { + let block = self + .l2_client + .get_block_by_number(block_number.into()) + .full() + .await? + .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_number))? + .into_consensus() + .map_transactions(|tx| tx.inner.into_inner()); + blocks_to_validate.push(block); + } + + self.validate_l1_messages(&blocks_to_validate).await?; + + let tx = self.database.tx_mut().await?; + tx.update_l1_messages_from_l2_blocks( + blocks_to_validate.into_iter().map(|b| (&b).into()).collect(), + ) + .await?; + tx.commit().await?; + + self.notify(ChainOrchestratorEvent::ChainConsolidated { + from: safe_block_number, + to: head_block_number, + }); + Ok(()) } @@ -1106,44 +1131,64 @@ impl< &self, blocks: &[ScrollBlock], ) -> Result<(), ChainOrchestratorError> { - let first_block_number = if let Some(block) = blocks.first() { - block.number - } else { + let l1_message_hashes = blocks + .iter() + .flat_map(|block| { + // Get the L1 messages from the block body. + block + .body + .transactions() + .filter(|&tx| tx.is_l1_message()) + // The hash for L1 messages is the trie hash of the transaction. + .map(|tx| tx.trie_hash()) + .collect::>() + }) + .collect::>(); + + // No L1 messages in the blocks, nothing to validate. + if l1_message_hashes.is_empty() { return Ok(()); - }; + } + let first_block_number = + blocks.first().expect("at least one block exists because we have l1 messages").number; let tx = self.database.tx().await?; - let mut database_txs = - tx.get_l1_messages(Some(L1MessageKey::block_number(first_block_number))).await?; - for block in blocks { - let block_txs = - block.body.transactions().filter(|tx| tx.is_l1_message()).map(|tx| tx.trie_hash()); - - // If we have L1 messages in the block we validate them against the expected L1 - // messages from the database. - for tx in block_txs { - let expected_message = if let Some(txs) = database_txs.as_mut() { - txs.next().await.transpose()? - } else { - None - }; - if let Some(expected_message) = expected_message { - if expected_message.transaction.tx_hash() != tx { - return Err(ChainOrchestratorError::L1MessageMismatch { - expected: expected_message.transaction.tx_hash(), - actual: tx, - }) - } - } else { - self.event_sender.as_ref().map(|sender| { - sender.notify(ChainOrchestratorEvent::L1MessageNotFoundInDatabase( - L1MessageKey::TransactionHash(tx), - )); - }); - return Err(ChainOrchestratorError::L1MessageNotFound( - L1MessageKey::TransactionHash(tx), - )) - } + let mut database_messages = if let Some(database_messages) = + tx.get_l1_messages(Some(L1MessageKey::block_number(first_block_number))).await? + { + database_messages + } else { + return Err(ChainOrchestratorError::L1MessageNotFound(L1MessageKey::TransactionHash( + *l1_message_hashes.first().expect("at least one message exists"), + ))) + }; + + for message_hash in l1_message_hashes { + // Get the expected L1 message from the database. + let expected_hash = database_messages + .next() + .await + .map(|m| m.map(|msg| msg.transaction.tx_hash())) + .transpose()? + .ok_or(ChainOrchestratorError::L1MessageNotFound(L1MessageKey::TransactionHash( + message_hash, + ))) + .inspect_err(|_| { + self.notify(ChainOrchestratorEvent::L1MessageNotFoundInDatabase( + L1MessageKey::TransactionHash(message_hash), + )); + })?; + + // If the received and expected L1 messages do not match return an error. + if message_hash != expected_hash { + self.notify(ChainOrchestratorEvent::L1MessageMismatch { + expected: expected_hash, + actual: message_hash, + }); + return Err(ChainOrchestratorError::L1MessageMismatch { + expected: expected_hash, + actual: message_hash, + }); } } diff --git a/crates/chain-orchestrator/src/sync.rs b/crates/chain-orchestrator/src/sync.rs index e790844a..0d53321c 100644 --- a/crates/chain-orchestrator/src/sync.rs +++ b/crates/chain-orchestrator/src/sync.rs @@ -15,12 +15,12 @@ impl Default for SyncState { impl SyncState { /// Returns a reference to the sync mode of L1. - pub fn l1(&self) -> &SyncMode { + pub const fn l1(&self) -> &SyncMode { &self.l1 } /// Returns a reference to the sync mode of L2. - pub fn l2(&self) -> &SyncMode { + pub const fn l2(&self) -> &SyncMode { &self.l2 } @@ -35,7 +35,7 @@ impl SyncState { } /// Returns true if both L1 and L2 are synced. - pub fn is_synced(&self) -> bool { + pub const fn is_synced(&self) -> bool { self.l1.is_synced() && self.l2.is_synced() } } @@ -51,21 +51,22 @@ pub enum SyncMode { } impl SyncMode { - /// Returns true if the sync mode is [`Self::Syncing`]. - pub fn is_syncing(&self) -> bool { + /// Returns true if the sync mode is [`SyncMode::Syncing`]. + pub const fn is_syncing(&self) -> bool { matches!(self, Self::Syncing) } - pub fn is_synced(&self) -> bool { + /// Returns true if the sync mode is [`SyncMode::Synced`]. + pub const fn is_synced(&self) -> bool { matches!(self, Self::Synced) } - /// Sets the sync mode to [`Self::Synced]. + /// Sets the sync mode to [`SyncMode::Synced`]. pub fn set_synced(&mut self) { *self = Self::Synced; } - /// Sets the sync mode to [`Self::Syncing`]. + /// Sets the sync mode to [`SyncMode::Syncing`]. pub fn set_syncing(&mut self) { *self = Self::Syncing; } diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index faa4a6f2..451fa583 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -863,15 +863,15 @@ mod test { let block_1 = BlockInfo { number: 1, hash: B256::arbitrary(&mut u).unwrap() }; let block_2 = BlockInfo { number: 2, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_1.clone()).await.unwrap(); - tx.insert_block(block_1.clone(), batch_data_1.clone().into()).await.unwrap(); - tx.insert_block(block_2.clone(), batch_data_1.clone().into()).await.unwrap(); + tx.insert_block(block_1, batch_data_1.clone().into()).await.unwrap(); + tx.insert_block(block_2, batch_data_1.clone().into()).await.unwrap(); // Insert batch 2 and associate it with one block in the database let batch_data_2 = BatchCommitData { index: 2, block_number: 20, ..Arbitrary::arbitrary(&mut u).unwrap() }; let block_3 = BlockInfo { number: 3, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_2.clone()).await.unwrap(); - tx.insert_block(block_3.clone(), batch_data_2.clone().into()).await.unwrap(); + tx.insert_block(block_3, batch_data_2.clone().into()).await.unwrap(); // Insert batch 3 produced at the same block number as batch 2 and associate it with one // block @@ -879,7 +879,7 @@ mod test { BatchCommitData { index: 3, block_number: 20, ..Arbitrary::arbitrary(&mut u).unwrap() }; let block_4 = BlockInfo { number: 4, hash: B256::arbitrary(&mut u).unwrap() }; tx.insert_batch(batch_data_3.clone()).await.unwrap(); - tx.insert_block(block_4.clone(), batch_data_3.clone().into()).await.unwrap(); + tx.insert_block(block_4, batch_data_3.clone().into()).await.unwrap(); tx.set_finalized_l1_block_number(21).await.unwrap(); tx.commit().await.unwrap(); diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 47840d73..6fd25040 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -437,6 +437,9 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati genesis_hash: B256, l1_block_number: u64, ) -> Result { + // Set the latest L1 block number + self.set_latest_l1_block_number(l1_block_number).await?; + // delete batch inputs and l1 messages let batches_removed = self.delete_batches_gt_block_number(l1_block_number).await?; let deleted_messages = self.delete_l1_messages_gt(l1_block_number).await?; @@ -472,11 +475,10 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati }; // delete mapping for l1 messages that were included in unsafe blocks after the reorg point - if l2_head_block_number.is_some() { - self.purge_l1_message_to_l2_block_mappings( - l2_head_block_number.map(|x| x.saturating_add(1)), - ) - .await?; + if let Some(block_number) = l2_head_block_number { + self.purge_l1_message_to_l2_block_mappings(Some(block_number.saturating_add(1))) + .await?; + self.set_l2_head_block_number(block_number).await?; } // commit the transaction @@ -922,8 +924,8 @@ impl L1MessageKey { Self::TransactionHash(hash) } - /// Creates a new [`L1MessageStart`] for the provided block number. - pub fn block_number(number: u64) -> Self { + /// Creates a new [`L1MessageKey`] for the provided block number. + pub const fn block_number(number: u64) -> Self { Self::BlockNumber(number) } } diff --git a/crates/derivation-pipeline/benches/pipeline.rs b/crates/derivation-pipeline/benches/pipeline.rs index 88874f24..66d0cd15 100644 --- a/crates/derivation-pipeline/benches/pipeline.rs +++ b/crates/derivation-pipeline/benches/pipeline.rs @@ -87,7 +87,7 @@ fn benchmark_pipeline_derivation(c: &mut Criterion) { // commit 1000 batches. for _ in 0..1000 { - pipeline.push_batch(batch_info, 0); + pipeline.push_batch(batch_info.into()).await; } tx.send(pipeline).unwrap(); diff --git a/crates/derivation-pipeline/src/lib.rs b/crates/derivation-pipeline/src/lib.rs index d748ef32..8a8144de 100644 --- a/crates/derivation-pipeline/src/lib.rs +++ b/crates/derivation-pipeline/src/lib.rs @@ -3,6 +3,17 @@ //! This crate provides a simple implementation of a derivation pipeline that transforms a batch //! into payload attributes for block building. +use alloy_primitives::{Address, B256}; +use alloy_rpc_types_engine::PayloadAttributes; +use core::{fmt::Debug, future::Future, pin::Pin, task::Poll}; +use futures::{stream::FuturesOrdered, task::AtomicWaker, Stream, StreamExt}; +use rollup_node_primitives::{BatchCommitData, BatchInfo, L1MessageEnvelope}; +use rollup_node_providers::{BlockDataProvider, L1Provider}; +use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; +use scroll_codec::{decoding::payload::PayloadData, Codec}; +use scroll_db::{Database, DatabaseReadOperations, DatabaseTransactionProvider, L1MessageKey}; +use tokio::sync::Mutex; + mod data_source; mod error; @@ -11,89 +22,27 @@ pub use error::DerivationPipelineError; mod metrics; pub use metrics::DerivationPipelineMetrics; -mod new; -pub use new::{BatchDerivationResult, DerivationPipelineNew, DerivedAttributes}; - use crate::data_source::CodecDataSource; -use std::{boxed::Box, collections::VecDeque, fmt::Formatter, sync::Arc, time::Instant, vec::Vec}; - -use alloy_primitives::{Address, B256}; -use alloy_rpc_types_engine::PayloadAttributes; -use core::{ - fmt::Debug, - future::Future, - pin::Pin, - task::{Context, Poll, Waker}, -}; -use futures::{FutureExt, Stream}; -use rollup_node_primitives::{ - BatchCommitData, BatchInfo, L1MessageEnvelope, ScrollPayloadAttributesWithBatchInfo, - WithBlockNumber, WithFinalizedBatchInfo, WithFinalizedBlockNumber, -}; -use rollup_node_providers::{BlockDataProvider, L1Provider}; -use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; -use scroll_codec::{decoding::payload::PayloadData, Codec}; -use scroll_db::{Database, DatabaseReadOperations, DatabaseTransactionProvider, L1MessageKey}; -use tokio::time::Interval; - -/// A future that resolves to a stream of [`ScrollPayloadAttributesWithBatchInfo`]. -type DerivationPipelineFuture = Pin< - Box< - dyn Future< - Output = Result< - Vec, - (Arc, DerivationPipelineError), - >, - > + Send, - >, ->; - -/// The interval (in ms) at which the derivation pipeline should report queue size metrics. -const QUEUE_METRICS_INTERVAL: u64 = 1000; +use std::{boxed::Box, sync::Arc, time::Instant, vec::Vec}; /// A structure holding the current unresolved futures for the derivation pipeline. +#[derive(Debug)] pub struct DerivationPipeline

{ - /// The current derivation pipeline futures polled. - pipeline_future: Option>, + /// The active batch derivation futures. + futures: Arc>>, /// A reference to the database. database: Arc, /// A L1 provider. l1_provider: P, /// The L1 message queue index at which the V2 L1 message queue was enabled. l1_v2_message_queue_start_index: u64, - /// The queue of batches to handle. - batch_queue: VecDeque>>, - /// The queue of polled attributes. - attributes_queue: VecDeque>, - /// The waker for the pipeline. - waker: Option, /// The metrics of the pipeline. metrics: DerivationPipelineMetrics, - /// The interval at which the derivation pipeline should report queue size metrics. - queue_metrics_interval: Interval, + /// The waker for the stream. + waker: AtomicWaker, } -impl Debug for DerivationPipeline

{ - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("DerivationPipeline") - .field( - "pipeline_future", - &self.pipeline_future.as_ref().map(|_| "Some( ... )").unwrap_or("None"), - ) - .field("database", &self.database) - .field("l1_provider", &self.l1_provider) - .field("batch_queue", &self.batch_queue) - .field("attributes_queue", &self.attributes_queue) - .field("waker", &self.waker) - .field("metrics", &self.metrics) - .finish() - } -} - -impl

DerivationPipeline

-where - P: L1Provider + Clone + Send + Sync + 'static, -{ +impl

DerivationPipeline

{ /// Returns a new instance of the [`DerivationPipeline`]. pub fn new( l1_provider: P, @@ -101,170 +50,148 @@ where l1_v2_message_queue_start_index: u64, ) -> Self { Self { + futures: Arc::new(Mutex::new(FuturesOrdered::new())), database, l1_provider, l1_v2_message_queue_start_index, - batch_queue: Default::default(), - pipeline_future: None, - attributes_queue: Default::default(), - waker: None, metrics: DerivationPipelineMetrics::default(), - queue_metrics_interval: delayed_interval(QUEUE_METRICS_INTERVAL), + waker: AtomicWaker::new(), } } +} - /// Handles a new batch by pushing it in its internal queue. Wakes the waker in order to trigger - /// a call to poll. - pub fn push_batch(&mut self, batch_info: BatchInfo, l1_block_number: u64) { - let block_info = Arc::new(batch_info); - self.batch_queue.push_back(WithFinalizedBlockNumber::new(l1_block_number, block_info)); - if let Some(waker) = self.waker.take() { - waker.wake() - } +impl

DerivationPipeline

+where + P: L1Provider + Clone + Send + Sync + 'static, +{ + /// Pushes a new batch info to the derivation pipeline. + pub async fn push_batch(&mut self, batch_info: Arc) { + let fut = self.derivation_future(batch_info); + self.futures.lock().await.push_back(fut); + self.waker.wake(); + } + + /// Returns the number of unresolved futures in the derivation pipeline. + pub async fn len(&self) -> usize { + self.futures.lock().await.len() } - /// Handles the next batch index in the batch index queue, pushing the future in the pipeline - /// futures. - fn handle_next_batch(&mut self) -> Option> { + /// Returns true if there are no unresolved futures in the derivation pipeline. + pub async fn is_empty(&self) -> bool { + self.futures.lock().await.is_empty() + } + + fn derivation_future(&self, batch_info: Arc) -> DerivationPipelineFuture { let database = self.database.clone(); let metrics = self.metrics.clone(); let provider = self.l1_provider.clone(); let l1_v2_message_queue_start_index = self.l1_v2_message_queue_start_index; - if let Some(info) = self.batch_queue.pop_front() { - let block_number = info.number; - let index = info.inner.index; - let fut = Box::pin(async move { - let derive_start = Instant::now(); - - // get the batch commit data. - let index = info.inner.index; - let info = info.inner; - let tx = database.tx().await.map_err(|e| (info.clone(), e.into()))?; - let batch = tx - .get_batch_by_index(index) - .await - .map_err(|err| (info.clone(), err.into()))? - .ok_or((info.clone(), DerivationPipelineError::UnknownBatch(index)))?; - - // derive the attributes and attach the corresponding batch info. - let attrs = derive(batch, provider, tx, l1_v2_message_queue_start_index) - .await - .map_err(|err| (info.clone(), err))?; - - // update metrics. - metrics.derived_blocks.increment(attrs.len() as u64); - let execution_duration = derive_start.elapsed().as_secs_f64(); - metrics.blocks_per_second.set(attrs.len() as f64 / execution_duration); - - Ok(attrs.into_iter().map(|attr| (attr, *info).into()).collect()) - }); - return Some(WithFinalizedBatchInfo::new(index, block_number, fut)); - } - None - } - - /// Clear attributes, batches and future for which the associated block number > - /// `l1_block_number`. - pub fn handle_reorg(&mut self, l1_block_number: u64) { - self.batch_queue.retain(|batch| batch.number <= l1_block_number); - if let Some(fut) = &mut self.pipeline_future { - if fut.number > l1_block_number { - self.pipeline_future = None; - } - } - self.attributes_queue.retain(|attr| attr.number <= l1_block_number); - } - - /// Handles a batch revert by clearing all internal queues and futures related to a batch index - /// >= provided batch index. - pub fn handle_batch_revert(&mut self, index: u64) { - self.attributes_queue.retain(|attr| attr.inner.batch_info.index < index); - self.batch_queue.retain(|attr| attr.inner.index < index); - if let Some(fut) = &mut self.pipeline_future { - if fut.index >= index { - self.pipeline_future = None; - } - } - } + Box::pin(async move { + let derive_start = Instant::now(); - /// Emits the queue size metrics for the batch and payload attributes queues. - fn emit_queue_gauges(&self) { - self.metrics.batch_queue_size.set(self.batch_queue.len() as f64); - self.metrics.payload_attributes_queue_size.set(self.attributes_queue.len() as f64); - } + // get the batch commit data. + let tx = database.tx().await.map_err(|e| (batch_info.clone(), e.into()))?; + let batch = tx + .get_batch_by_index(batch_info.index) + .await + .map_err(|err| (batch_info.clone(), err.into()))? + .ok_or(( + batch_info.clone(), + DerivationPipelineError::UnknownBatch(batch_info.index), + ))?; + + // derive the attributes and attach the corresponding batch info. + let result = derive_new(batch, provider, tx, l1_v2_message_queue_start_index) + .await + .map_err(|err| (batch_info.clone(), err))?; - /// Returns the size of the batch queue. - pub fn batch_queue_size(&self) -> usize { - self.batch_queue.len() + // update metrics. + metrics.derived_blocks.increment(result.attributes.len() as u64); + let execution_duration = derive_start.elapsed().as_secs_f64(); + metrics.blocks_per_second.set(result.attributes.len() as f64 / execution_duration); + Ok(result) + }) } } impl

Stream for DerivationPipeline

where - P: L1Provider + Clone + Unpin + Send + Sync + 'static, + P: L1Provider + Unpin + Clone + Send + Sync + 'static, { - type Item = WithBlockNumber; + type Item = BatchDerivationResult; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_next( + self: Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { let this = self.get_mut(); - - // report queue size metrics if the interval has elapsed. - while this.queue_metrics_interval.poll_tick(cx).is_ready() { - this.emit_queue_gauges(); - } - - // return attributes from the queue if any. - if let Some(attribute) = this.attributes_queue.pop_front() { - return Poll::Ready(Some(attribute)) - } - - // if future is None and the batch queue is empty, store the waker and return. - if this.pipeline_future.is_none() && this.batch_queue.is_empty() { - this.waker = Some(cx.waker().clone()); - return Poll::Pending - } - - // if the future is None, handle the next batch. - if this.pipeline_future.is_none() { - this.pipeline_future = this.handle_next_batch() - } - - // poll the futures and handle result. - if let Some(Poll::Ready(res)) = this.pipeline_future.as_mut().map(|fut| fut.poll_unpin(cx)) - { - match res { - WithFinalizedBatchInfo { inner: Ok(attributes), number, .. } => { - let attributes = attributes - .into_iter() - .map(|attr| WithFinalizedBlockNumber::new(number, attr)); - this.attributes_queue.extend(attributes); - this.pipeline_future = None; - cx.waker().wake_by_ref(); - } - WithFinalizedBatchInfo { inner: Err((batch_info, err)), number, .. } => { - tracing::error!(target: "scroll::node::derivation_pipeline", batch_info = ?*batch_info, ?err, "failed to derive payload attributes for batch"); - // retry polling the same batch. - this.batch_queue.push_front(WithFinalizedBlockNumber::new(number, batch_info)); - let fut = this.handle_next_batch().expect("batch_queue not empty"); - this.pipeline_future = Some(fut); - // notify the waker that work can be done. - cx.waker().wake_by_ref(); + this.waker.register(cx.waker()); + + // Poll the next future in the ordered set of futures. + match this.futures.try_lock() { + Ok(mut guard) => { + let result = guard.poll_next_unpin(cx); + match result { + // If the derivation failed then push it to the front of the queue to be + // retried. + Poll::Ready(Some(Err((batch_info, err)))) => { + tracing::error!(target: "scroll::derivation_pipeline", ?batch_info, ?err, "Failed to derive payload attributes"); + guard.push_front(this.derivation_future(batch_info)); + cx.waker().wake_by_ref(); + Poll::Pending + } + // If the derivation succeeded then return the attributes. + Poll::Ready(Some(Ok(result))) => Poll::Ready(Some(result)), + // If there are no more futures then return None. + Poll::Ready(None) | Poll::Pending => Poll::Pending, } } + Err(_) => { + // Could not acquire the lock, return pending. + cx.waker().wake_by_ref(); + + Poll::Pending + } } - Poll::Pending } } +/// The result of deriving a batch. +#[derive(Debug)] +pub struct BatchDerivationResult { + /// The derived payload attributes. + pub attributes: Vec, + /// The batch info associated with the derived attributes. + pub batch_info: BatchInfo, +} + +/// The derived attributes along with the block number they correspond to. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct DerivedAttributes { + /// The block number the attributes correspond to. + pub block_number: u64, + /// The derived payload attributes. + pub attributes: ScrollPayloadAttributes, +} + +/// A future that resolves to a stream of [`BatchDerivationResult`]. +type DerivationPipelineFuture = Pin< + Box< + dyn Future< + Output = Result, DerivationPipelineError)>, + > + Send, + >, +>; + /// Returns a vector of [`ScrollPayloadAttributes`] from the [`BatchCommitData`] and a /// [`L1Provider`]. -pub async fn derive( +pub async fn derive_new( batch: BatchCommitData, l1_provider: L1P, l2_provider: L2P, l1_v2_message_queue_start_index: u64, -) -> Result, DerivationPipelineError> { +) -> Result { // fetch the blob then decode the input batch. let blob = if let Some(hash) = batch.blob_versioned_hash { l1_provider.blob(batch.block_timestamp, hash).await? @@ -284,6 +211,7 @@ pub async fn derive Interval { - let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(interval)); - interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - interval + Ok(BatchDerivationResult { + attributes, + batch_info: BatchInfo { index: batch.index, hash: batch.hash }, + }) } /// Returns an iterator over L1 messages from the `PayloadData`. If the `PayloadData` returns a @@ -464,59 +390,6 @@ mod tests { }, }; - #[tokio::test] - async fn test_should_correctly_handle_batch_revert() -> eyre::Result<()> { - // construct the pipeline. - let db = Arc::new(setup_test_db().await); - let l1_messages_provider = db.clone(); - let mock_l1_provider = MockL1Provider { l1_messages_provider, blobs: HashMap::new() }; - - let mut pipeline = DerivationPipeline { - pipeline_future: Some(WithFinalizedBatchInfo::new( - 10, - 0, - Box::pin(async { Ok(vec![]) }), - )), - database: db, - l1_provider: mock_l1_provider, - l1_v2_message_queue_start_index: u64::MAX, - batch_queue: [ - WithFinalizedBlockNumber::new( - 0, - Arc::new(BatchInfo { index: 11, hash: Default::default() }), - ), - WithFinalizedBlockNumber::new( - 0, - Arc::new(BatchInfo { index: 12, hash: Default::default() }), - ), - WithFinalizedBlockNumber::new( - 0, - Arc::new(BatchInfo { index: 13, hash: Default::default() }), - ), - ] - .into(), - attributes_queue: [WithFinalizedBlockNumber::new( - 0, - ScrollPayloadAttributesWithBatchInfo { - payload_attributes: Default::default(), - batch_info: BatchInfo { index: 9, hash: Default::default() }, - }, - )] - .into(), - waker: None, - metrics: Default::default(), - queue_metrics_interval: delayed_interval(QUEUE_METRICS_INTERVAL), - }; - - // should clear all data related to batch index >= 9. - pipeline.handle_batch_revert(9); - assert!(pipeline.pipeline_future.is_none()); - assert!(pipeline.attributes_queue.is_empty(),); - assert!(pipeline.batch_queue.is_empty(),); - - Ok(()) - } - #[tokio::test] async fn test_should_retry_on_derivation_error() -> eyre::Result<()> { // https://etherscan.io/tx/0x8f4f0fcab656aa81589db5b53255094606c4624bfd99702b56b2debaf6211f48 @@ -544,7 +417,13 @@ mod tests { let mut pipeline = DerivationPipeline::new(mock_l1_provider, db.clone(), u64::MAX); // as long as we don't call `push_batch`, pipeline should not return attributes. - pipeline.push_batch(BatchInfo { index: 12, hash: Default::default() }, 0); + pipeline.push_batch(BatchInfo { index: 12, hash: Default::default() }.into()).await; + + // wait for 5 seconds to ensure the pipeline is in a retry loop. + tokio::select! { + _ = tokio::time::sleep(tokio::time::Duration::from_secs(5)) => {} + _ = pipeline.next() => {panic!("pipeline should not yield as the transactions are not in db so it should be in a retry loop");} + } // in a separate task, add the second l1 message. tokio::task::spawn(async move { @@ -554,20 +433,14 @@ mod tests { tx.commit().await.unwrap(); }); - // pipeline should initially fail and recover once the task previously spawned loads the L1 - // message in db. - assert!(pipeline.next().await.is_some()); - // check the correctness of the last attribute. let mut attribute = ScrollPayloadAttributes::default(); - while let Some(WithBlockNumber { - inner: ScrollPayloadAttributesWithBatchInfo { payload_attributes: a, .. }, - .. - }) = pipeline.next().await - { - if a.payload_attributes.timestamp == 1696935657 { - attribute = a; - break + if let Some(BatchDerivationResult { attributes, .. }) = pipeline.next().await { + for a in attributes { + if a.attributes.payload_attributes.timestamp == 1696935657 { + attribute = a.attributes; + break; + } } } let expected = ScrollPayloadAttributes { @@ -587,6 +460,8 @@ mod tests { #[tokio::test] async fn test_should_stream_payload_attributes() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + // https://etherscan.io/tx/0x8f4f0fcab656aa81589db5b53255094606c4624bfd99702b56b2debaf6211f48 // load batch data in the db. let db = Arc::new(setup_test_db().await); @@ -615,21 +490,16 @@ mod tests { let mut pipeline = DerivationPipeline::new(mock_l1_provider, db, u64::MAX); // as long as we don't call `push_batch`, pipeline should not return attributes. - pipeline.push_batch(BatchInfo { index: 12, hash: Default::default() }, 0); - - // we should find some attributes now - assert!(pipeline.next().await.is_some()); + pipeline.push_batch(BatchInfo { index: 12, hash: Default::default() }.into()).await; // check the correctness of the last attribute. let mut attribute = ScrollPayloadAttributes::default(); - while let Some(WithBlockNumber { - inner: ScrollPayloadAttributesWithBatchInfo { payload_attributes: a, .. }, - .. - }) = pipeline.next().await - { - if a.payload_attributes.timestamp == 1696935657 { - attribute = a; - break + if let Some(BatchDerivationResult { attributes, .. }) = pipeline.next().await { + for a in attributes { + if a.attributes.payload_attributes.timestamp == 1696935657 { + attribute = a.attributes; + break + } } } let expected = ScrollPayloadAttributes { @@ -673,9 +543,12 @@ mod tests { let l1_provider = MockL1Provider { l1_messages_provider, blobs: HashMap::new() }; let l2_provider = MockL2Provider; - let attributes: Vec<_> = derive(batch_data, l1_provider, l2_provider, u64::MAX).await?; - let attribute = - attributes.iter().find(|a| a.payload_attributes.timestamp == 1696935384).unwrap(); + let result = derive_new(batch_data, l1_provider, l2_provider, u64::MAX).await?; + let attribute = result + .attributes + .iter() + .find(|a| a.attributes.payload_attributes.timestamp == 1696935384) + .unwrap(); let expected = ScrollPayloadAttributes { payload_attributes: PayloadAttributes { @@ -687,9 +560,9 @@ mod tests { block_data_hint: BlockDataHint::none(), gas_limit: Some(10_000_000), }; - assert_eq!(attribute, &expected); + assert_eq!(attribute.attributes, expected); - let attribute = attributes.last().unwrap(); + let attribute = result.attributes.last().unwrap(); let expected = ScrollPayloadAttributes { payload_attributes: PayloadAttributes { timestamp: 1696935657, @@ -700,7 +573,7 @@ mod tests { block_data_hint: BlockDataHint::none(), gas_limit: Some(10_000_000), }; - assert_eq!(attribute, &expected); + assert_eq!(attribute.attributes, expected); Ok(()) } @@ -774,10 +647,11 @@ mod tests { let l2_provider = MockL2Provider; // derive attributes and extract l1 messages. - let attributes: Vec<_> = derive(batch_data, l1_provider, l2_provider, u64::MAX).await?; + let attributes = derive_new(batch_data, l1_provider, l2_provider, u64::MAX).await?; let derived_l1_messages: Vec<_> = attributes + .attributes .into_iter() - .filter_map(|a| a.transactions) + .filter_map(|a| a.attributes.transactions) .flatten() .filter_map(|rlp| { let buf = &mut rlp.as_ref(); @@ -830,10 +704,11 @@ mod tests { let l2_provider = MockL2Provider; // derive attributes and extract l1 messages. - let attributes: Vec<_> = derive(batch_data, l1_provider, l2_provider, u64::MAX).await?; + let attributes = derive_new(batch_data, l1_provider, l2_provider, u64::MAX).await?; let derived_l1_messages: Vec<_> = attributes + .attributes .into_iter() - .filter_map(|a| a.transactions) + .filter_map(|a| a.attributes.transactions) .flatten() .filter_map(|rlp| { let buf = &mut rlp.as_ref(); @@ -946,9 +821,9 @@ mod tests { }; let l2_provider = MockL2Provider; - let attributes: Vec<_> = derive(batch_data, l1_provider, l2_provider, u64::MAX).await?; + let attributes = derive_new(batch_data, l1_provider, l2_provider, u64::MAX).await?; - let attribute = attributes.last().unwrap(); + let attribute = attributes.attributes.last().unwrap(); let expected = ScrollPayloadAttributes { payload_attributes: PayloadAttributes { timestamp: 1725455077, @@ -959,7 +834,7 @@ mod tests { block_data_hint: BlockDataHint::none(), gas_limit: Some(10_000_000), }; - assert_eq!(attribute, &expected); + assert_eq!(attribute.attributes, expected); Result::<(), eyre::Report>::Ok(()) }) @@ -969,78 +844,4 @@ mod tests { Ok(()) } - - async fn new_test_pipeline() -> DerivationPipeline>> { - let initial_block = 200; - - let batches = (initial_block - 100..initial_block) - .map(|i| WithFinalizedBlockNumber::new(i, Arc::new(BatchInfo::new(i, B256::random())))); - let attributes = (initial_block..initial_block + 100) - .zip(batches.clone()) - .map(|(i, batch)| { - WithFinalizedBlockNumber::new( - i, - ScrollPayloadAttributesWithBatchInfo { - batch_info: *batch.inner, - ..Default::default() - }, - ) - }) - .collect(); - - let db = Arc::new(setup_test_db().await); - let l1_messages_provider = db.clone(); - let mock_l1_provider = MockL1Provider { l1_messages_provider, blobs: HashMap::new() }; - - DerivationPipeline { - pipeline_future: Some(WithFinalizedBatchInfo::new( - initial_block - 100, - initial_block, - Box::pin(async { Ok(vec![]) }), - )), - database: db, - l1_provider: mock_l1_provider, - l1_v2_message_queue_start_index: u64::MAX, - batch_queue: batches.collect(), - attributes_queue: attributes, - waker: None, - metrics: Default::default(), - queue_metrics_interval: delayed_interval(QUEUE_METRICS_INTERVAL), - } - } - - #[tokio::test] - async fn test_should_handle_reorgs() -> eyre::Result<()> { - // set up pipeline. - let mut pipeline = new_test_pipeline().await; - - // reorg at block 0. - pipeline.handle_reorg(0); - // should completely clear the pipeline. - assert!(pipeline.batch_queue.is_empty()); - assert!(pipeline.pipeline_future.is_none()); - assert!(pipeline.attributes_queue.is_empty()); - - // set up pipeline. - let mut pipeline = new_test_pipeline().await; - - // reorg at block 200. - pipeline.handle_reorg(200); - // should clear all but one attribute and retain all batches and the pending future. - assert_eq!(pipeline.batch_queue.len(), 100); - assert!(pipeline.pipeline_future.is_some()); - assert_eq!(pipeline.attributes_queue.len(), 1); - - // set up pipeline. - let mut pipeline = new_test_pipeline().await; - - // reorg at block 300. - pipeline.handle_reorg(300); - // should retain all batches, attributes and the pending future. - assert_eq!(pipeline.batch_queue.len(), 100); - assert!(pipeline.pipeline_future.is_some()); - assert_eq!(pipeline.attributes_queue.len(), 100); - - Ok(()) - } } diff --git a/crates/derivation-pipeline/src/new.rs b/crates/derivation-pipeline/src/new.rs deleted file mode 100644 index 55e5c2c4..00000000 --- a/crates/derivation-pipeline/src/new.rs +++ /dev/null @@ -1,229 +0,0 @@ -use super::*; -use futures::{stream::FuturesOrdered, StreamExt}; -use tokio::sync::Mutex; - -/// A structure holding the current unresolved futures for the derivation pipeline. -#[derive(Debug)] -pub struct DerivationPipelineNew

{ - /// The active batch derivation futures. - futures: Arc>>, - /// A reference to the database. - database: Arc, - /// A L1 provider. - l1_provider: P, - /// The L1 message queue index at which the V2 L1 message queue was enabled. - l1_v2_message_queue_start_index: u64, - /// The metrics of the pipeline. - metrics: DerivationPipelineMetrics, -} - -impl

DerivationPipelineNew

{ - /// Returns a new instance of the [`DerivationPipelineNew`]. - pub fn new( - l1_provider: P, - database: Arc, - l1_v2_message_queue_start_index: u64, - ) -> Self { - Self { - futures: Arc::new(Mutex::new(FuturesOrdered::new())), - database, - l1_provider, - l1_v2_message_queue_start_index, - metrics: DerivationPipelineMetrics::default(), - } - } -} - -impl

DerivationPipelineNew

-where - P: L1Provider + Clone + Send + Sync + 'static, -{ - /// Pushes a new batch info to the derivation pipeline. - pub async fn push(&mut self, batch_info: Arc) { - let fut = self.derivation_future(batch_info); - self.futures.lock().await.push_back(fut); - } - - /// Returns the number of unresolved futures in the derivation pipeline. - pub async fn len(&self) -> usize { - self.futures.lock().await.len() - } - - fn derivation_future(&self, batch_info: Arc) -> DerivationPipelineFutureNew { - let database = self.database.clone(); - let metrics = self.metrics.clone(); - let provider = self.l1_provider.clone(); - let l1_v2_message_queue_start_index = self.l1_v2_message_queue_start_index; - - Box::pin(async move { - let derive_start = Instant::now(); - - // get the batch commit data. - let tx = database.tx().await.map_err(|e| (batch_info.clone(), e.into()))?; - let batch = tx - .get_batch_by_index(batch_info.index) - .await - .map_err(|err| (batch_info.clone(), err.into()))? - .ok_or(( - batch_info.clone(), - DerivationPipelineError::UnknownBatch(batch_info.index), - ))?; - - // derive the attributes and attach the corresponding batch info. - let result = derive_new(batch, provider, tx, l1_v2_message_queue_start_index) - .await - .map_err(|err| (batch_info.clone(), err))?; - - // update metrics. - metrics.derived_blocks.increment(result.attributes.len() as u64); - let execution_duration = derive_start.elapsed().as_secs_f64(); - metrics.blocks_per_second.set(result.attributes.len() as f64 / execution_duration); - Ok(result) - }) - } -} - -impl

Stream for DerivationPipelineNew

-where - P: L1Provider + Unpin + Clone + Send + Sync + 'static, -{ - type Item = BatchDerivationResult; - - fn poll_next( - self: Pin<&mut Self>, - cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - let this = self.get_mut(); - - // Poll the next future in the ordered set of futures. - match this.futures.try_lock() { - Ok(mut guard) => { - match guard.poll_next_unpin(cx) { - // If the derivation failed then push it to the front of the queue to be - // retried. - Poll::Ready(Some(Err((batch_info, err)))) => { - tracing::error!(target: "scroll::derivation_pipeline", ?batch_info, ?err, "Failed to derive payload attributes"); - guard.push_front(this.derivation_future(batch_info.clone())); - cx.waker().wake_by_ref(); - return Poll::Pending - } - // If the derivation succeeded then return the attributes. - Poll::Ready(Some(Ok(result))) => return Poll::Ready(Some(result)), - // If there are no more futures then return pending. - _ => return Poll::Pending, - } - } - Err(_) => { - // Could not acquire the lock, return pending. - cx.waker().wake_by_ref(); - return Poll::Pending - } - } - } -} - -/// The result of deriving a batch. -#[derive(Debug)] -pub struct BatchDerivationResult { - /// The derived payload attributes. - pub attributes: Vec, - /// The batch info associated with the derived attributes. - pub batch_info: BatchInfo, -} - -/// The derived attributes along with the block number they correspond to. -#[derive(Debug, Clone)] -pub struct DerivedAttributes { - /// The block number the attributes correspond to. - pub block_number: u64, - /// The derived payload attributes. - pub attributes: ScrollPayloadAttributes, -} - -/// A future that resolves to a stream of [`ScrollPayloadAttributesWithBatchInfo`]. -type DerivationPipelineFutureNew = Pin< - Box< - dyn Future< - Output = Result, DerivationPipelineError)>, - > + Send, - >, ->; - -/// Returns a vector of [`ScrollPayloadAttributes`] from the [`BatchCommitData`] and a -/// [`L1Provider`]. -pub async fn derive_new( - batch: BatchCommitData, - l1_provider: L1P, - l2_provider: L2P, - l1_v2_message_queue_start_index: u64, -) -> Result { - // fetch the blob then decode the input batch. - let blob = if let Some(hash) = batch.blob_versioned_hash { - l1_provider.blob(batch.block_timestamp, hash).await? - } else { - None - }; - let data = CodecDataSource { calldata: batch.calldata.as_ref(), blob: blob.as_deref() }; - let decoded = Codec::decode(&data)?; - - // set the cursor for the l1 provider. - let payload_data = &decoded.data; - let mut l1_messages_iter = - iter_l1_messages_from_payload(&l1_provider, payload_data, l1_v2_message_queue_start_index) - .await?; - - let skipped_l1_messages = decoded.data.skipped_l1_message_bitmap.clone().unwrap_or_default(); - let mut skipped_l1_messages = skipped_l1_messages.into_iter(); - let blocks = decoded.data.into_l2_blocks(); - let mut attributes = Vec::with_capacity(blocks.len()); - for mut block in blocks { - // query the appropriate amount of l1 messages. - let mut txs = Vec::with_capacity(block.context.num_transactions as usize); - for _ in 0..block.context.num_l1_messages { - // check if the next l1 message should be skipped. - if matches!(skipped_l1_messages.next(), Some(bit) if bit) { - let _ = l1_messages_iter.next(); - continue; - } - - let l1_message = l1_messages_iter - .next() - .ok_or(DerivationPipelineError::MissingL1Message(block.clone()))?; - let mut bytes = Vec::with_capacity(l1_message.transaction.eip2718_encoded_length()); - l1_message.transaction.eip2718_encode(&mut bytes); - txs.push(bytes.into()); - } - - // add the block transactions. - txs.append(&mut block.transactions); - - // get the block data for the l2 block. - let number = block.context.number; - // TODO(performance): can this be improved by adding block_data_range. - let block_data = l2_provider.block_data(number).await.map_err(Into::into)?; - - // construct the payload attributes. - let attribute = DerivedAttributes { - block_number: number, - attributes: ScrollPayloadAttributes { - payload_attributes: PayloadAttributes { - timestamp: block.context.timestamp, - suggested_fee_recipient: Address::ZERO, - prev_randao: B256::ZERO, - withdrawals: None, - parent_beacon_block_root: None, - }, - transactions: Some(txs), - no_tx_pool: true, - block_data_hint: block_data.unwrap_or_else(BlockDataHint::none), - gas_limit: Some(block.context.gas_limit), - }, - }; - attributes.push(attribute); - } - - Ok(BatchDerivationResult { - attributes, - batch_info: BatchInfo { index: batch.index, hash: batch.hash }, - }) -} diff --git a/crates/engine/Cargo.toml b/crates/engine/Cargo.toml index 551b7148..01842e73 100644 --- a/crates/engine/Cargo.toml +++ b/crates/engine/Cargo.toml @@ -36,12 +36,6 @@ reth-scroll-engine-primitives.workspace = true # rollup-node rollup-node-primitives.workspace = true -rollup-node-providers.workspace = true -rollup-node-signer.workspace = true - -# scroll -scroll-db.workspace = true -scroll-network.workspace = true # misc async-trait = { workspace = true, optional = true } @@ -58,20 +52,14 @@ alloy-consensus.workspace = true arbitrary.workspace = true async-trait.workspace = true reth-testing-utils.workspace = true -rollup-node-providers = { workspace = true, features = ["test-utils"] } scroll-alloy-consensus.workspace = true scroll-alloy-rpc-types-engine = { workspace = true, features = ["arbitrary"] } -scroll-engine = { path = "../engine", features = ["test-utils"] } [features] test-utils = [ "dep:async-trait", - "scroll-engine/test-utils", - "rollup-node-providers/test-utils", - "rollup-node-providers/test-utils", "reth-chainspec/test-utils", "reth-primitives-traits/test-utils", - "scroll-db/test-utils", ] serde = [ "alloy-eips/serde", @@ -81,8 +69,6 @@ serde = [ "scroll-alloy-rpc-types-engine/serde", "alloy-chains/serde", "reth-network-api/serde", - "scroll-network/serde", - "scroll-engine/serde", "scroll-alloy-hardforks/serde", "reth-primitives-traits/serde", "alloy-consensus/serde", diff --git a/crates/engine/src/api.rs b/crates/engine/src/api.rs deleted file mode 100644 index d96c9a99..00000000 --- a/crates/engine/src/api.rs +++ /dev/null @@ -1,89 +0,0 @@ -use super::EngineDriverError; -use alloy_rpc_types_engine::{ - ExecutionPayload, ExecutionPayloadV1, ForkchoiceState as AlloyForkchoiceState, - ForkchoiceUpdated, PayloadId, PayloadStatusEnum, -}; -use eyre::Result; -use reth_payload_primitives::PayloadTypes; -use reth_scroll_engine_primitives::ScrollEngineTypes; -use scroll_alloy_provider::ScrollEngineApi; -use std::sync::Arc; -use tracing::{debug, error, trace}; - -/// Calls `engine_newPayloadV1` and logs the result. -pub(crate) async fn new_payload( - client: Arc, - execution_payload: ExecutionPayloadV1, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, -{ - // TODO: should never enter the `Syncing`, `Accepted` or `Invalid` variants when called from - // `handle_payload_attributes`. - let response = client - .new_payload_v1(execution_payload) - .await - .map_err(|_| EngineDriverError::EngineUnavailable)?; - - match &response.status { - PayloadStatusEnum::Invalid { validation_error } => { - error!(target: "scroll::engine::driver", ?validation_error, "execution payload is invalid"); - } - PayloadStatusEnum::Syncing => { - debug!(target: "scroll::engine::driver", "execution client is syncing"); - } - PayloadStatusEnum::Accepted => { - error!(target: "scroll::engine::driver", "execution payload part of side chain"); - } - PayloadStatusEnum::Valid => { - trace!(target: "scroll::engine::driver", "execution payload valid"); - } - }; - - Ok(response.status) -} - -/// Calls `engine_forkchoiceUpdatedV1` and logs the result. -pub(crate) async fn forkchoice_updated( - client: Arc, - fcs: AlloyForkchoiceState, - attributes: Option<::PayloadAttributes>, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, -{ - let forkchoice_updated = client - .fork_choice_updated_v1(fcs, attributes) - .await - .map_err(EngineDriverError::ForkchoiceUpdateFailed)?; - - // TODO: should never enter the `Syncing`, `Accepted` or `Invalid` variants when called from - // `handle_payload_attributes`. - match &forkchoice_updated.payload_status.status { - PayloadStatusEnum::Invalid { validation_error } => { - error!(target: "scroll::engine::driver", ?validation_error, "failed to issue forkchoice"); - } - PayloadStatusEnum::Syncing => { - debug!(target: "scroll::engine::driver", "head has been seen before, but not part of the chain"); - } - PayloadStatusEnum::Accepted => { - unreachable!("forkchoice update should never return an `Accepted` status"); - } - PayloadStatusEnum::Valid => { - trace!(target: "scroll::engine::driver", "forkchoice updated"); - } - }; - - Ok(forkchoice_updated) -} - -/// Calls `engine_getPayloadV1`. -pub(crate) async fn get_payload( - client: Arc, - id: PayloadId, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, -{ - Ok(client.get_payload_v1(id).await.map_err(|_| EngineDriverError::EngineUnavailable)?.into()) -} diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs deleted file mode 100644 index e85f861f..00000000 --- a/crates/engine/src/driver.rs +++ /dev/null @@ -1,597 +0,0 @@ -use super::{future::EngineFuture, ForkchoiceState}; -use crate::{ - future::{BuildNewPayloadFuture, EngineDriverFutureResult}, - metrics::EngineDriverMetrics, - EngineDriverError, EngineDriverEvent, -}; - -use alloy_provider::Provider; -use futures::{ready, task::AtomicWaker, FutureExt, Stream}; -use rollup_node_primitives::{ - BlockInfo, ChainImport, MeteredFuture, ScrollPayloadAttributesWithBatchInfo, WithBlockNumber, -}; -use scroll_alloy_hardforks::ScrollHardforks; -use scroll_alloy_network::Scroll; -use scroll_alloy_provider::ScrollEngineApi; -use scroll_alloy_rpc_types_engine::ScrollPayloadAttributes; -use std::{ - collections::VecDeque, - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; -use tokio::time::Duration; - -/// The main interface to the Engine API of the EN. -/// Internally maintains the fork state of the chain. -pub struct EngineDriver { - /// The engine API client. - client: Arc, - /// The chain spec. - chain_spec: Arc, - /// The provider. - provider: Option

, - /// The fork choice state of the engine. - fcs: ForkchoiceState, - /// Whether the EN is syncing. - syncing: bool, - /// Block building duration. - block_building_duration: Duration, - /// The pending payload attributes derived from batches on L1. - l1_payload_attributes: VecDeque>, - /// The pending block imports received over the network. - chain_imports: VecDeque, - /// The latest optimistic sync target. - optimistic_sync_target: Option, - /// The payload attributes associated with the next block to be built. - sequencer_payload_attributes: Option, - /// The future related to engine API. - engine_future: Option>, - /// The future for the payload building job. - payload_building_future: Option, - /// The driver metrics. - metrics: EngineDriverMetrics, - /// The waker to notify when the engine driver should be polled. - waker: AtomicWaker, - /// Whether to allow empty blocks. - allow_empty_blocks: bool, -} - -impl EngineDriver -where - EC: ScrollEngineApi + Sync + 'static, - CS: ScrollHardforks + Sync + 'static, - P: Provider + Clone + Sync + 'static, -{ - /// Create a new [`EngineDriver`]. - pub fn new( - client: Arc, - chain_spec: Arc, - provider: Option

, - fcs: ForkchoiceState, - sync_at_start_up: bool, - block_building_duration: Duration, - allow_empty_blocks: bool, - ) -> Self { - Self { - client, - chain_spec, - provider, - fcs, - block_building_duration, - syncing: sync_at_start_up, - l1_payload_attributes: VecDeque::new(), - chain_imports: VecDeque::new(), - optimistic_sync_target: None, - sequencer_payload_attributes: None, - payload_building_future: None, - engine_future: None, - metrics: EngineDriverMetrics::default(), - waker: AtomicWaker::new(), - allow_empty_blocks, - } - } - - /// Returns the number of pending futures in the queue. - /// - /// This only considers the length of the L1 payload attributes and chain import queues. - pub fn pending_futures_len(&self) -> usize { - self.l1_payload_attributes.len() + self.chain_imports.len() - } - - /// Sets the finalized block info. - pub fn set_finalized_block_info(&mut self, block_info: BlockInfo) { - let _ = self.fcs.update_finalized_block_info(block_info); - } - - /// Sets the safe block info. - pub fn set_safe_block_info(&mut self, block_info: BlockInfo) { - let _ = self.fcs.update_safe_block_info(block_info); - } - - /// Sets the head block info. - pub fn set_head_block_info(&mut self, block_info: BlockInfo) { - let _ = self.fcs.update_head_block_info(block_info); - } - - /// Sets the payload building duration. - pub fn set_payload_building_duration(&mut self, block_building_duration: Duration) { - self.block_building_duration = block_building_duration; - } - - /// Clear the l1 attributes queue. - pub fn clear_l1_payload_attributes(&mut self) { - // clear the L1 attributes queue. - self.l1_payload_attributes.clear(); - - // drop the engine future if it is a L1 consolidation. - if let Some(MeteredFuture { fut: EngineFuture::L1Consolidation(_), .. }) = - self.engine_future - { - self.engine_future.take(); - } - } - - /// Handle L1 reorg, with the L1 block number reorged to, and whether this reorged the head or - /// batches. - pub fn handle_l1_reorg( - &mut self, - l1_block_number: u64, - reorged_unsafe_head: Option, - reorged_safe_head: Option, - ) { - // On an unsafe head reorg. - if let Some(l2_head_block_info) = reorged_unsafe_head { - // clear the payload building future. - self.payload_building_future = None; - - // retain only blocks from chain imports for which the block number <= L2 reorged - // number. - for chain_import in &mut self.chain_imports { - chain_import.chain.retain(|block| block.number <= l2_head_block_info.number); - } - - // reset the unsafe head. - self.set_head_block_info(l2_head_block_info); - - // drop the engine future if it's a `NewPayload` or `BlockImport` with block number > - // L2 reorged number. - if let Some(MeteredFuture { fut, .. }) = self.engine_future.as_ref() { - match fut { - EngineFuture::ChainImport(WithBlockNumber { number, .. }) - if number > &l2_head_block_info.number => - { - self.engine_future = None - } - // `NewPayload` future is ONLY instantiated when the payload building future is - // done, and we want to issue the payload to the EN. Thus, we also clear it on a - // L2 reorg. - EngineFuture::NewPayload(_) => self.engine_future = None, - _ => {} - } - } - } - - // On a safe head reorg: reset the safe head. - if let Some(safe_block_info) = reorged_safe_head { - self.set_safe_block_info(safe_block_info); - } - - // drop the engine future if it's a `L1Consolidation` future associated with a L1 block - // number > l1_block_number. - if matches!( - self.engine_future.as_ref(), - Some(MeteredFuture { - fut: EngineFuture::L1Consolidation(WithBlockNumber { number, .. }), - .. - }) if number > &l1_block_number - ) { - self.engine_future = None; - } - - // retain the L1 payload attributes with block number <= L1 block. - self.l1_payload_attributes.retain(|attribute| attribute.number <= l1_block_number); - } - - /// Handles a block import request by adding it to the queue and waking up the driver. - pub fn handle_chain_import(&mut self, chain_import: ChainImport) { - tracing::trace!(target: "scroll::engine", head = %chain_import.chain.last().unwrap().hash_slow(), "new chain import request received"); - - self.chain_imports.push_back(chain_import); - self.waker.wake(); - } - - /// Optimistically syncs the chain to the provided block info. - pub fn handle_optimistic_sync(&mut self, block_info: BlockInfo) { - tracing::info!(target: "scroll::engine", ?block_info, "optimistic sync request received"); - - // Purge all pending block imports. - self.chain_imports.clear(); - - // Update the fork choice state with the new block info. - self.optimistic_sync_target = Some(block_info); - - // Wake up the driver to process the optimistic sync. - self.waker.wake(); - } - - /// Handles a [`ScrollPayloadAttributes`] sourced from L1 by initiating a task sending the - /// attribute to the EN via the [`EngineDriver`]. - pub fn handle_l1_consolidation( - &mut self, - attributes: WithBlockNumber, - ) { - self.l1_payload_attributes.push_back(attributes); - self.waker.wake(); - } - - /// Handles a [`ScrollPayloadAttributes`] sourced from the sequencer by initiating a task - /// sending the attributes to the EN and requesting a new payload to be built. - pub fn handle_build_new_payload(&mut self, attributes: ScrollPayloadAttributes) { - tracing::info!(target: "scroll::engine", ?attributes, "new payload attributes request received"); - - if self.sequencer_payload_attributes.is_some() { - tracing::error!(target: "scroll::engine", "a payload building job is already in progress"); - return; - } - - self.sequencer_payload_attributes = Some(attributes); - self.waker.wake(); - } - - /// This function is called when a future completes and is responsible for - /// processing the result and returning an event if applicable. - fn handle_engine_future_result( - &mut self, - result: EngineDriverFutureResult, - duration: Duration, - ) -> Option { - match result { - EngineDriverFutureResult::BlockImport(result) => { - tracing::trace!(target: "scroll::engine", ?result, "handling block import result"); - - match result { - Ok((block_info, block_import_outcome, payload_status)) => { - // Update the unsafe block info - if let Some(block_info) = block_info { - tracing::trace!(target: "scroll::engine", ?block_info, "updating unsafe block info"); - let _ = self.fcs.update_head_block_info(block_info); - }; - - // Update the sync status - if !payload_status.is_syncing() { - tracing::trace!(target: "scroll::engine", "sync finished"); - self.syncing = false; - } - - // record the metric. - self.metrics.block_import_duration.record(duration.as_secs_f64()); - - // Return the block import outcome - return block_import_outcome.map(EngineDriverEvent::BlockImportOutcome); - } - Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to import block"); - } - } - } - EngineDriverFutureResult::L1Consolidation(result) => { - tracing::trace!(target: "scroll::engine", ?result, "handling L1 consolidation result"); - - match result { - Ok(consolidation_outcome) => { - let block_info = consolidation_outcome.block_info(); - - // Batches are now always considered finalized, as such we update both the - // safe and finalized block info. Update this once we implement issue #273. - // Update the safe and finalized block info and return the block info. - tracing::trace!(target: "scroll::engine", ?block_info, "updating safe and finalized block info from block derived from L1"); - let _ = self.fcs.update_safe_block_info(block_info.block_info); - let _ = self.fcs.update_finalized_block_info(block_info.block_info); - - // If we reorged, update the head block info - if consolidation_outcome.is_reorg() { - tracing::warn!(target: "scroll::engine", ?block_info, "reorging head to l1 derived block"); - let _ = self.fcs.update_head_block_info(block_info.block_info); - } - - // record the metric. - self.metrics.l1_consolidation_duration.record(duration.as_secs_f64()); - - return Some(EngineDriverEvent::L1BlockConsolidated(consolidation_outcome)); - } - Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to consolidate block derived from L1"); - if let EngineDriverError::L1ConsolidationMissingPayloadId(attributes) = err - { - tracing::info!(target: "scroll::engine", "retrying L1 consolidation job for missing payload id"); - self.l1_payload_attributes.push_front(attributes); - self.waker.wake(); - } - } - } - } - EngineDriverFutureResult::PayloadBuildingJob(result) => { - tracing::trace!(target: "scroll::engine", result = ?result.as_ref().map(|b| b.header.as_ref()), "handling payload building result"); - - match result { - Ok(block) => { - // Skip block if no transactions are present in block. - if !self.allow_empty_blocks && block.body.transactions.is_empty() { - tracing::trace!(target: "scroll::engine", "no transactions in block"); - return None; - } - - // Update the unsafe block info and return the block - let block_info = BlockInfo::new(block.number, block.hash_slow()); - tracing::trace!(target: "scroll::engine", ?block_info, "updating unsafe block info from new payload"); - let _ = self.fcs.update_head_block_info(block_info); - - // record the metrics. - self.metrics.build_new_payload_duration.record(duration.as_secs_f64()); - self.metrics.gas_per_block.record(block.gas_used as f64); - - return Some(EngineDriverEvent::NewPayload(block)); - } - Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to build new payload"); - if let EngineDriverError::PayloadBuildingMissingPayloadId(attributes) = err - { - self.sequencer_payload_attributes = Some(attributes); - } - } - } - } - EngineDriverFutureResult::OptimisticSync(result) => { - tracing::trace!(target: "scroll::engine", ?result, "handling optimistic sync result"); - - match result { - Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to perform optimistic sync") - } - Ok(fcu) => { - tracing::trace!(target: "scroll::engine", ?fcu, "optimistic sync issued successfully"); - } - } - } - } - - None - } - - /// A helper function to check if a payload building job is in progress. - pub const fn is_payload_building_in_progress(&self) -> bool { - self.sequencer_payload_attributes.is_some() || self.payload_building_future.is_some() - } - - /// Returns the sync status. - pub const fn is_syncing(&self) -> bool { - self.syncing - } - - /// Returns the forkchoice state. - pub const fn forkchoice_state(&self) -> &ForkchoiceState { - &self.fcs - } - - /// Returns the alloy forkchoice state. - pub fn alloy_forkchoice_state(&self) -> alloy_rpc_types_engine::ForkchoiceState { - if self.is_syncing() { - self.fcs.get_alloy_optimistic_fcs() - } else { - self.fcs.get_alloy_fcs() - } - } -} - -impl Stream for EngineDriver -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - CS: ScrollHardforks + Send + Sync + 'static, - P: Provider + Clone + Unpin + Send + Sync + 'static, -{ - type Item = EngineDriverEvent; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let this = self.get_mut(); - - // Register the waker such that we can wake when required. - this.waker.register(cx.waker()); - - // If we have a future, poll it. - if let Some(future) = this.engine_future.as_mut() { - let (duration, result) = ready!(future.poll_unpin(cx)); - this.engine_future = None; - if let Some(event) = this.handle_engine_future_result(result, duration) { - return Poll::Ready(Some(event)); - } - }; - - // Take the handle to the payload building job if it exists and poll it. - if let Some(mut handle) = this.payload_building_future.take() { - // If the payload build job is done, handle the result - otherwise continue to process - // another driver job. - match handle.poll_unpin(cx) { - Poll::Ready((duration, result)) => match result { - Ok(block) => { - this.engine_future = Some( - MeteredFuture::new(EngineFuture::handle_new_payload_job( - this.client.clone(), - this.alloy_forkchoice_state(), - block, - )) - .with_initial_duration(duration), - ); - this.waker.wake(); - } - Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to build new payload"); - - if let EngineDriverError::PayloadBuildingMissingPayloadId(attributes) = err - { - tracing::warn!(target: "scroll::engine", "retrying payload building job for missing payload id"); - this.sequencer_payload_attributes = Some(attributes); - this.waker.wake(); - } - } - }, - // The job is still in progress, reassign the handle and continue. - _ => { - this.payload_building_future = Some(handle); - } - } - } - - // If we have a payload building request from the sequencer, build a new payload. - if let Some(payload_attributes) = this.sequencer_payload_attributes.take() { - let fcs = this.alloy_forkchoice_state(); - let client = this.client.clone(); - let chain_spec = this.chain_spec.clone(); - let duration = this.block_building_duration; - - this.payload_building_future = - Some(MeteredFuture::new(Box::pin(super::future::build_new_payload( - client, - chain_spec, - fcs, - duration, - payload_attributes, - )))); - this.waker.wake(); - return Poll::Pending; - } - - // If we have an optimistic sync target, issue the optimistic sync. - if let Some(block_info) = this.optimistic_sync_target.take() { - let _ = this.fcs.update_head_block_info(block_info); - let fcs = this.fcs.get_alloy_optimistic_fcs(); - this.engine_future = - Some(MeteredFuture::new(EngineFuture::optimistic_sync(this.client.clone(), fcs))); - this.waker.wake(); - return Poll::Pending; - } - - // Handle the chain import requests. - if let Some(chain_import) = this.chain_imports.pop_front() { - let fcs = this.alloy_forkchoice_state(); - let client = this.client.clone(); - - this.engine_future = - Some(MeteredFuture::new(EngineFuture::chain_import(client, chain_import, fcs))); - - this.waker.wake(); - return Poll::Pending; - } - - if let Some(payload_attributes) = this.l1_payload_attributes.pop_front() { - let fcs = this.fcs.clone(); - let client = this.client.clone(); - - if let Some(provider) = this.provider.clone() { - this.engine_future = Some(MeteredFuture::new(EngineFuture::l1_consolidation( - client, - provider, - fcs, - payload_attributes, - ))); - this.waker.wake(); - } else { - tracing::error!(target: "scroll::engine", "l1 consolidation requires an execution payload provider"); - } - - return Poll::Pending; - } - - Poll::Pending - } -} - -impl std::fmt::Debug for EngineDriver { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("EngineDriver") - .field("client", &"ScrollEngineApi") - .field("provider", &"ExecutionPayloadProvider") - .field("chain_spec", &"ScrollHardforks") - .field("fcs", &self.fcs) - .field("future", &"EngineDriverFuture") - .finish() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::future::build_new_payload; - - use reth_scroll_chainspec::SCROLL_DEV; - use rollup_node_providers::ScrollRootProvider; - use scroll_engine::test_utils::PanicEngineClient; - - impl EngineDriver { - fn with_payload_future(&mut self, future: BuildNewPayloadFuture) { - self.payload_building_future = Some(future); - } - } - - #[tokio::test] - async fn test_is_payload_building_in_progress() { - let client = Arc::new(PanicEngineClient); - let chain_spec = SCROLL_DEV.clone(); - let fcs = - ForkchoiceState::from_block_info(BlockInfo { number: 0, hash: Default::default() }); - let duration = Duration::from_secs(2); - - let mut driver = EngineDriver::new( - client, - chain_spec, - None::, - fcs, - false, - duration, - true, - ); - - // Initially, it should be false - assert!(!driver.is_payload_building_in_progress()); - - // Simulate a payload building job invocation - driver.handle_build_new_payload(Default::default()); - - // Now, it should return true - assert!(driver.is_payload_building_in_progress()); - } - - #[tokio::test] - async fn test_is_payload_building_in_progress_with_future() { - let client = Arc::new(PanicEngineClient); - let chain_spec = SCROLL_DEV.clone(); - let fcs = - ForkchoiceState::from_block_info(BlockInfo { number: 0, hash: Default::default() }); - let duration = Duration::from_secs(2); - - let mut driver = EngineDriver::new( - client.clone(), - chain_spec.clone(), - None::, - fcs, - false, - duration, - true, - ); - - // Initially, it should be false - assert!(!driver.is_payload_building_in_progress()); - - // Set a future to simulate an ongoing job - driver.with_payload_future(MeteredFuture::new(Box::pin(build_new_payload( - client, - chain_spec, - Default::default(), - Default::default(), - Default::default(), - )))); - - // Now, it should return true - assert!(driver.is_payload_building_in_progress()); - } -} diff --git a/crates/engine/src/new.rs b/crates/engine/src/engine.rs similarity index 69% rename from crates/engine/src/new.rs rename to crates/engine/src/engine.rs index 3ab19af0..d7ec2816 100644 --- a/crates/engine/src/new.rs +++ b/crates/engine/src/engine.rs @@ -1,5 +1,7 @@ use super::{EngineError, ForkchoiceState}; -use alloy_rpc_types_engine::{ExecutionPayloadV1, ForkchoiceUpdated, PayloadStatus}; +use alloy_rpc_types_engine::{ + ExecutionPayloadV1, ForkchoiceUpdated, PayloadStatus, PayloadStatusEnum, +}; use reth_scroll_primitives::ScrollBlock; use rollup_node_primitives::BlockInfo; use scroll_alloy_provider::ScrollEngineApi; @@ -19,12 +21,12 @@ where EC: ScrollEngineApi + Sync + 'static, { /// Create a new [`Engine`]. - pub fn new(client: Arc, fcs: ForkchoiceState) -> Self { + pub const fn new(client: Arc, fcs: ForkchoiceState) -> Self { Self { client, fcs } } /// Get a reference to the current fork choice state. - pub fn fcs(&self) -> &ForkchoiceState { + pub const fn fcs(&self) -> &ForkchoiceState { &self.fcs } @@ -47,6 +49,21 @@ where // send the fcs update request to the engine let result = self.client.fork_choice_updated_v1(fcs.get_alloy_fcs(), None).await?; + match &result.payload_status.status { + PayloadStatusEnum::Invalid { validation_error } => { + tracing::error!(target: "scroll::engine", ?validation_error, "failed to issue forkchoice"); + } + PayloadStatusEnum::Syncing => { + tracing::debug!(target: "scroll::engine", "head has been seen before, but not part of the chain"); + } + PayloadStatusEnum::Accepted => { + unreachable!("forkchoice update should never return an `Accepted` status"); + } + PayloadStatusEnum::Valid => { + tracing::trace!(target: "scroll::engine", "forkchoice updated"); + } + }; + // update the internal fcs state if the update was successful // If the result is invalid, do not update the fcs // If the result is valid or sync, update the fcs @@ -87,6 +104,22 @@ where tracing::trace!(target: "scroll::engine", block_number = block.number, block_hash = ?block.hash_slow(), "Submitting new payload to engine"); let payload = ExecutionPayloadV1::from_block_slow(block); let result = self.client.new_payload_v1(payload).await?; + + match &result.status { + PayloadStatusEnum::Invalid { validation_error } => { + tracing::error!(target: "scroll::engine", ?validation_error, "execution payload is invalid"); + } + PayloadStatusEnum::Syncing => { + tracing::debug!(target: "scroll::engine", "execution client is syncing"); + } + PayloadStatusEnum::Accepted => { + tracing::error!(target: "scroll::engine", "execution payload part of side chain"); + } + PayloadStatusEnum::Valid => { + tracing::trace!(target: "scroll::engine", "execution payload valid"); + } + }; + Ok(result) } @@ -106,6 +139,8 @@ where let result = self.client.fork_choice_updated_v1(fcs.get_alloy_fcs(), Some(attributes)).await?; + tracing::trace!(target: "scroll::engine", ?result, "Build new payload request completed"); + Ok(result) } @@ -114,6 +149,7 @@ where &self, payload_id: alloy_rpc_types_engine::PayloadId, ) -> Result { + tracing::trace!(target: "scroll::engine", ?payload_id, "Getting payload by ID"); let payload = self.client.get_payload_v1(payload_id).await?; Ok(payload) } diff --git a/crates/engine/src/error.rs b/crates/engine/src/error.rs index f7e47a95..0baecaf4 100644 --- a/crates/engine/src/error.rs +++ b/crates/engine/src/error.rs @@ -58,12 +58,12 @@ pub enum EngineError { impl EngineError { /// Creates a new [`EngineError`] for a [`FcsError::NoUpdateProvided`]. - pub fn fcs_no_update_provided() -> Self { + pub const fn fcs_no_update_provided() -> Self { Self::FcsError(FcsError::NoUpdateProvided) } /// Creates a new [`EngineError`] for a [`FcsError::FinalizedBlockNumberNotIncreasing`]. - pub fn fcs_finalized_block_number_not_increasing() -> Self { + pub const fn fcs_finalized_block_number_not_increasing() -> Self { Self::FcsError(FcsError::FinalizedBlockNumberNotIncreasing) } } diff --git a/crates/engine/src/event.rs b/crates/engine/src/event.rs deleted file mode 100644 index 6cbfda0f..00000000 --- a/crates/engine/src/event.rs +++ /dev/null @@ -1,28 +0,0 @@ -use super::ConsolidationOutcome; -use reth_scroll_primitives::ScrollBlock; -use scroll_network::BlockImportOutcome; - -/// An enum representing the events that can be emitted by the engine driver. -#[derive(Debug)] -pub enum EngineDriverEvent { - /// A new block has been generated by the engine driver. - NewPayload(ScrollBlock), - /// The result of attempting a block import. - BlockImportOutcome(BlockImportOutcome), - /// The result of attempting a chain import. - ChainImportOutcome(ChainImportOutcome), - /// A block derived from L1 has been consolidated. - L1BlockConsolidated(ConsolidationOutcome), -} - -/// The outcome of a chain import. -/// -/// This includes the result of the final block import outcome and the chain of blocks that were -/// imported. -#[derive(Debug)] -pub struct ChainImportOutcome { - /// The outcome of the block import. - pub outcome: BlockImportOutcome, - /// The chain of blocks that were imported. - pub chain: Vec, -} diff --git a/crates/engine/src/fcs.rs b/crates/engine/src/fcs.rs index 5d07e0ee..77dd6643 100644 --- a/crates/engine/src/fcs.rs +++ b/crates/engine/src/fcs.rs @@ -83,9 +83,9 @@ impl ForkchoiceState { } // Build the candidate new state. - let new_finalized = finalized.unwrap_or(self.finalized.clone()); - let new_safe = safe.unwrap_or(self.safe.clone()); - let new_head = head.unwrap_or(self.head.clone()); + let new_finalized = finalized.unwrap_or(self.finalized); + let new_safe = safe.unwrap_or(self.safe); + let new_head = head.unwrap_or(self.head); // Check that the finalized block number is increasing. if new_finalized.number <= self.finalized.number && new_finalized != self.finalized { diff --git a/crates/engine/src/future/mod.rs b/crates/engine/src/future/mod.rs deleted file mode 100644 index 41ea27bf..00000000 --- a/crates/engine/src/future/mod.rs +++ /dev/null @@ -1,397 +0,0 @@ -use super::{payload::block_matches_attributes, EngineDriverError}; -use crate::{api::*, ForkchoiceState}; - -use alloy_primitives::bytes::Bytes; -use alloy_provider::Provider; -use alloy_rpc_types_engine::{ - ExecutionData, ExecutionPayloadV1, ForkchoiceState as AlloyForkchoiceState, ForkchoiceUpdated, - PayloadStatusEnum, -}; -use eyre::Result; -use reth_scroll_engine_primitives::try_into_block; -use reth_scroll_primitives::ScrollBlock; -use rollup_node_primitives::{ - BatchInfo, BlockInfo, ChainImport, L2BlockInfoWithL1Messages, MeteredFuture, - ScrollPayloadAttributesWithBatchInfo, WithBlockNumber, -}; -use rollup_node_signer::SignatureAsBytes; -use scroll_alloy_hardforks::ScrollHardforks; -use scroll_alloy_network::Scroll; -use scroll_alloy_provider::ScrollEngineApi; -use scroll_alloy_rpc_types_engine::ScrollPayloadAttributes; -use scroll_network::BlockImportOutcome; -use std::{ - future::Future, - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; -use tokio::time::Duration; -use tracing::instrument; - -mod result; -pub(crate) use result::EngineDriverFutureResult; - -/// A future that represents a block import job. -type ChainImportFuture = Pin< - Box< - dyn Future< - Output = Result< - (Option, Option, PayloadStatusEnum), - EngineDriverError, - >, - > + Send, - >, ->; - -/// A future that represents an L1 consolidation job. -type L1ConsolidationFuture = - Pin> + Send>>; - -/// An enum that represents the different outcomes of an L1 consolidation job. -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum ConsolidationOutcome { - /// Represents a successful consolidation outcome with the consolidated block info and batch - /// info. - Consolidation(L2BlockInfoWithL1Messages, BatchInfo), - /// Represents a reorganization outcome with the consolidated block info and batch info. - Reorg(L2BlockInfoWithL1Messages, BatchInfo), -} - -impl ConsolidationOutcome { - /// Returns the consolidated block info. - pub const fn block_info(&self) -> &L2BlockInfoWithL1Messages { - match self { - Self::Consolidation(info, _) | Self::Reorg(info, _) => info, - } - } - - /// Returns the batch info associated with the consolidation outcome. - pub const fn batch_info(&self) -> &BatchInfo { - match self { - Self::Consolidation(_, batch_info) | Self::Reorg(_, batch_info) => batch_info, - } - } - - /// Returns a boolean indicating whether the consolidation outcome is a reorg. - pub const fn is_reorg(&self) -> bool { - matches!(self, Self::Reorg(_, _)) - } - - /// Returns a boolean indicating whether the consolidation outcome is a consolidation. - pub const fn is_consolidate(&self) -> bool { - matches!(self, Self::Consolidation(_, _)) - } -} - -/// A future that represents a new payload processing. -type NewPayloadFuture = - Pin> + Send>>; - -/// A future that represents a new payload building job. -pub(crate) type BuildNewPayloadFuture = - MeteredFuture> + Send>>>; - -/// A future that represents a new payload building job. -pub(crate) type OptimisticSyncFuture = - Pin> + Send>>; - -/// An enum that represents the different types of futures that can be executed on the engine API. -/// It can be a block import job, an L1 consolidation job, or a new payload processing. -pub(crate) enum EngineFuture { - ChainImport(WithBlockNumber), - L1Consolidation(WithBlockNumber), - NewPayload(NewPayloadFuture), - OptimisticSync(OptimisticSyncFuture), -} - -impl EngineFuture { - pub(crate) fn chain_import( - client: Arc, - chain_import: ChainImport, - fcs: AlloyForkchoiceState, - ) -> Self - where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - { - let highest_block_number = chain_import.chain.last().unwrap().number; - Self::ChainImport(WithBlockNumber::new( - highest_block_number, - Box::pin(handle_chain_import(client, chain_import, fcs)), - )) - } - - pub(crate) fn optimistic_sync(client: Arc, fcs: AlloyForkchoiceState) -> Self - where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - { - Self::OptimisticSync(Box::pin(forkchoice_updated(client, fcs, None))) - } - - /// Creates a new [`EngineFuture::L1Consolidation`] future from the provided parameters. - pub(crate) fn l1_consolidation( - client: Arc, - execution_payload_provider: P, - fcs: ForkchoiceState, - payload_attributes: WithBlockNumber, - ) -> Self - where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - P: Provider + Unpin + Send + Sync + 'static, - { - Self::L1Consolidation(WithBlockNumber::new( - payload_attributes.number, - Box::pin(handle_payload_attributes( - client, - execution_payload_provider, - fcs, - payload_attributes, - )), - )) - } - - /// Creates a new [`EngineFuture::NewPayload`] future from the provided parameters. - pub(crate) fn handle_new_payload_job( - client: Arc, - fcs: AlloyForkchoiceState, - block: ScrollBlock, - ) -> Self - where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - { - Self::NewPayload(Box::pin(handle_new_payload(client, fcs, block))) - } -} - -impl Future for EngineFuture { - type Output = EngineDriverFutureResult; - - /// Polls the [`EngineFuture`] and upon completion, returns the result of the - /// corresponding future by converting it into an [`EngineDriverFutureResult`]. - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.get_mut(); - match this { - Self::ChainImport(fut) => fut.inner.as_mut().poll(cx).map(Into::into), - Self::L1Consolidation(fut) => fut.inner.as_mut().poll(cx).map(Into::into), - Self::NewPayload(fut) => fut.as_mut().poll(cx).map(Into::into), - Self::OptimisticSync(fut) => fut.as_mut().poll(cx).map(Into::into), - } - } -} - -/// Handles an execution payload: -/// - Sends the payload to the EL via `engine_newPayloadV1`. -/// - Sets the current fork choice for the EL via `engine_forkchoiceUpdatedV1`. -#[instrument(skip_all, level = "trace", - fields( - peer_id = %chain_import.peer_id, - block_hash = %chain_import.chain.last().unwrap().hash_slow(), - fcs = ?fcs - ) - )] -async fn handle_chain_import( - client: Arc, - chain_import: ChainImport, - mut fcs: AlloyForkchoiceState, -) -> Result<(Option, Option, PayloadStatusEnum), EngineDriverError> -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, -{ - tracing::trace!(target: "scroll::engine::future", ?fcs, ?chain_import.peer_id, chain = ?chain_import.chain.last().unwrap().hash_slow(), "handling execution payload"); - - let ChainImport { chain, peer_id, signature } = chain_import; - - // Extract the block info from the last payload. - let head = chain.last().unwrap().clone(); - - let mut payload_status = None; - for block in chain { - // Create the execution payload. - let payload = ExecutionPayloadV1::from_block_slow(&block); - - // Issue the new payload to the EN. - let status = new_payload(client.clone(), payload).await?; - - // Check if the payload is invalid and return early. - if let PayloadStatusEnum::Invalid { ref validation_error } = status { - tracing::error!(target: "scroll::engine", ?validation_error, "execution payload is invalid"); - - // If the payload is invalid, return early. - return Ok((None, Some(BlockImportOutcome::invalid_block(peer_id)), status)); - } - - payload_status = Some(status); - } - let payload_status = payload_status.unwrap(); - - // Update the fork choice state with the new block hash. - let block_info: BlockInfo = (&head).into(); - fcs.head_block_hash = block_info.hash; - - // Invoke the FCU with the new state. - let fcu = forkchoice_updated(client.clone(), fcs, None).await?; - - // TODO: Handle other cases appropriately. - match (&payload_status, &fcu.payload_status.status) { - (PayloadStatusEnum::Valid, PayloadStatusEnum::Valid) => Ok(( - Some(block_info), - Some(BlockImportOutcome::valid_block( - peer_id, - head, - Bytes::copy_from_slice(&signature.sig_as_bytes()), - )), - PayloadStatusEnum::Valid, - )), - _ => Ok((None, None, fcu.payload_status.status)), - } -} - -/// Handles a payload attributes: -/// - Retrieves the execution payload for block at safe head + 1. -/// - If the payload is missing or doesn't match the attributes: -/// - Starts payload building task on the EL via `engine_forkchoiceUpdatedV1`, passing the -/// provided payload attributes. -/// - Retrieve the payload with `engine_getPayloadV1`. -/// - Sends the constructed payload to the EL via `engine_newPayloadV1`. -/// - Sets the current fork choice for the EL via `engine_forkchoiceUpdatedV1`. -/// - If the execution payload matches the attributes: -/// - Sets the current fork choice for the EL via `engine_forkchoiceUpdatedV1`, advancing the -/// safe head by one. -#[instrument(skip_all, level = "trace", - fields( - fcs = ?fcs, - payload_attributes = ?payload_attributes_with_batch_info - ) - )] -async fn handle_payload_attributes( - client: Arc, - provider: P, - fcs: ForkchoiceState, - payload_attributes_with_batch_info: WithBlockNumber, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - P: Provider + Unpin + Send + Sync + 'static, -{ - tracing::trace!(target: "scroll::engine::future", ?fcs, ?payload_attributes_with_batch_info, "handling payload attributes"); - - let ScrollPayloadAttributesWithBatchInfo { mut payload_attributes, batch_info } = - payload_attributes_with_batch_info.inner.clone(); - - let maybe_execution_payload = provider - .get_block((fcs.safe_block_info().number + 1).into()) - .full() - .await - .map_err(|_| EngineDriverError::ExecutionPayloadProviderUnavailable)? - .map(|b| b.into_consensus().map_transactions(|tx| tx.inner.into_inner())) - .filter(|b| block_matches_attributes(&payload_attributes, b, fcs.safe_block_info().hash)); - - if let Some(execution_payload) = maybe_execution_payload { - // if the payload attributes match the execution payload at block safe + 1, - // this payload has already been passed to the EN in the form of a P2P gossiped - // execution payload. We can advance the safe head by one by issuing a - // forkchoiceUpdated. - // We also advance the finalized head since batches are always finalized until we implement - // issue #273. - let block_info: L2BlockInfoWithL1Messages = (&execution_payload).into(); - - // We only need to update the safe block hash if we are advancing the safe head past the - // finalized head. There is a possible edge case where on startup, - // when we reconsolidate the latest batch, the finalized head is ahead of the safe - // head. - if fcs.safe_block_info().number > fcs.finalized_block_info().number { - let mut fcs = fcs.get_alloy_fcs(); - fcs.safe_block_hash = block_info.block_info.hash; - fcs.finalized_block_hash = block_info.block_info.hash; - forkchoice_updated(client, fcs, None).await?; - } - Ok(ConsolidationOutcome::Consolidation(block_info, batch_info)) - } else { - let mut fcs = fcs.get_alloy_fcs(); - // Otherwise, we construct a block from the payload attributes on top of the current - // safe head. - fcs.head_block_hash = fcs.safe_block_hash; - - // start payload building with `no_tx_pool = true`. - payload_attributes.no_tx_pool = true; - let fc_updated = forkchoice_updated(client.clone(), fcs, Some(payload_attributes)).await?; - - // retrieve the execution payload - let execution_payload = get_payload( - client.clone(), - fc_updated.payload_id.ok_or(EngineDriverError::L1ConsolidationMissingPayloadId( - payload_attributes_with_batch_info, - ))?, - ) - .await?; - // issue the execution payload to the EL - let block_info: L2BlockInfoWithL1Messages = (&execution_payload).into(); - let result = new_payload(client.clone(), execution_payload.into_v1()).await?; - - // we should only have a valid payload when deriving from payload attributes (should not - // be syncing)! - debug_assert!(result.is_valid()); - - // update the fork choice state with the new block hash. - // We also advance the finalized head since batches are always finalized until we implement - // issue #273. - fcs.head_block_hash = block_info.block_info.hash; - fcs.safe_block_hash = block_info.block_info.hash; - fcs.finalized_block_hash = block_info.block_info.hash; - forkchoice_updated(client, fcs, None).await?; - - Ok(ConsolidationOutcome::Reorg(block_info, batch_info)) - } -} - -/// Builds a new payload from the provided fork choice state and payload attributes. -pub(crate) async fn build_new_payload( - client: Arc, - chain_spec: Arc, - fcs: AlloyForkchoiceState, - block_building_duration: Duration, - payload_attributes: ScrollPayloadAttributes, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, - CS: ScrollHardforks, -{ - tracing::trace!(target: "scroll::engine::future", ?payload_attributes, "building new payload"); - - // start a payload building job on top of the current unsafe head. - let fc_updated = - forkchoice_updated(client.clone(), fcs, Some(payload_attributes.clone())).await?; - - // wait for the payload building to take place. - tokio::time::sleep(block_building_duration).await; - - // retrieve the execution payload - let payload = get_payload( - client.clone(), - fc_updated - .payload_id - .ok_or(EngineDriverError::PayloadBuildingMissingPayloadId(payload_attributes))?, - ) - .await?; - let block = try_into_block(ExecutionData { payload, sidecar: Default::default() }, chain_spec)?; - - Ok(block) -} - -/// Handles a new payload by updating the fork choice state and returning the new block. -async fn handle_new_payload( - client: Arc, - mut fcs: AlloyForkchoiceState, - block: ScrollBlock, -) -> Result -where - EC: ScrollEngineApi + Unpin + Send + Sync + 'static, -{ - // update the head block hash to the new payload block hash. - fcs.head_block_hash = block.hash_slow(); - - // update the fork choice state with the new block hash. - forkchoice_updated(client, fcs, None).await?; - - Ok(block) -} diff --git a/crates/engine/src/future/result.rs b/crates/engine/src/future/result.rs deleted file mode 100644 index a4618bd3..00000000 --- a/crates/engine/src/future/result.rs +++ /dev/null @@ -1,51 +0,0 @@ -use super::*; - -/// A type that represents the result of the engine driver future. -#[derive(Debug)] -pub(crate) enum EngineDriverFutureResult { - BlockImport( - Result< - (Option, Option, PayloadStatusEnum), - EngineDriverError, - >, - ), - L1Consolidation(Result), - PayloadBuildingJob(Result), - OptimisticSync(Result), -} - -impl - From< - Result< - (Option, Option, PayloadStatusEnum), - EngineDriverError, - >, - > for EngineDriverFutureResult -{ - fn from( - value: Result< - (Option, Option, PayloadStatusEnum), - EngineDriverError, - >, - ) -> Self { - Self::BlockImport(value) - } -} - -impl From> for EngineDriverFutureResult { - fn from(value: Result) -> Self { - Self::L1Consolidation(value) - } -} - -impl From> for EngineDriverFutureResult { - fn from(value: Result) -> Self { - Self::PayloadBuildingJob(value) - } -} - -impl From> for EngineDriverFutureResult { - fn from(value: Result) -> Self { - Self::OptimisticSync(value) - } -} diff --git a/crates/engine/src/lib.rs b/crates/engine/src/lib.rs index d9a31277..fc87334e 100644 --- a/crates/engine/src/lib.rs +++ b/crates/engine/src/lib.rs @@ -1,31 +1,20 @@ -//! Engine Driver for the Scroll Rollup Node. The [`EngineDriver`] exposes the main interface for +//! Engine Driver for the Scroll Rollup Node. The [`Engine`] exposes the main interface for //! the Rollup Node to the Engine API. -pub(crate) mod api; - -pub use driver::EngineDriver; -mod driver; - pub use error::{EngineDriverError, EngineError, FcsError}; mod error; -pub use event::EngineDriverEvent; -mod event; - pub use fcs::{genesis_hash_from_chain_spec, ForkchoiceState}; mod fcs; -mod future; -pub use future::ConsolidationOutcome; - pub use metrics::EngineDriverMetrics; mod metrics; mod payload; pub use payload::block_matches_attributes; -mod new; -pub use new::Engine; +mod engine; +pub use engine::Engine; #[cfg(any(test, feature = "test-utils"))] pub mod test_utils; diff --git a/crates/engine/src/metrics.rs b/crates/engine/src/metrics.rs index 19b596d6..8e2f530c 100644 --- a/crates/engine/src/metrics.rs +++ b/crates/engine/src/metrics.rs @@ -1,7 +1,7 @@ use metrics::Histogram; use metrics_derive::Metrics; -/// The metrics for the [`super::EngineDriver`]. +/// The metrics for the [`super::Engine`]. #[derive(Metrics, Clone)] #[metrics(scope = "engine_driver")] pub struct EngineDriverMetrics { diff --git a/crates/manager/Cargo.toml b/crates/manager/Cargo.toml deleted file mode 100644 index f14e2e97..00000000 --- a/crates/manager/Cargo.toml +++ /dev/null @@ -1,83 +0,0 @@ -[package] -name = "rollup-node-manager" -version.workspace = true -edition.workspace = true -rust-version.workspace = true -license.workspace = true -exclude.workspace = true - -[lints] -workspace = true - -[dependencies] -# alloy -alloy-chains.workspace = true -alloy-consensus.workspace = true -alloy-eips.workspace = true -alloy-primitives.workspace = true -alloy-provider.workspace = true -alloy-rpc-types-engine.workspace = true - -# scroll-alloy -scroll-alloy-hardforks.workspace = true -scroll-alloy-network.workspace = true -scroll-alloy-provider.workspace = true -scroll-alloy-rpc-types-engine.workspace = true - -# reth -reth-network.workspace = true -reth-network-api.workspace = true -reth-tokio-util.workspace = true - -# reth-scroll -reth-chainspec.workspace = true -reth-primitives-traits.workspace = true -reth-scroll-primitives.workspace = true -reth-scroll-node.workspace = true -reth-tasks.workspace = true - -# scroll -scroll-db.workspace = true -scroll-derivation-pipeline.workspace = true -scroll-engine.workspace = true -scroll-network.workspace = true -scroll-wire.workspace = true - -# rollup node -rollup-node-chain-orchestrator.workspace = true -rollup-node-primitives.workspace = true -rollup-node-providers.workspace = true -rollup-node-sequencer.workspace = true -rollup-node-signer.workspace = true -rollup-node-watcher.workspace = true - -# misc -futures.workspace = true -tokio-stream.workspace = true -tokio.workspace = true -tracing.workspace = true -metrics.workspace = true -metrics-derive.workspace = true - -[dev-dependencies] -alloy-consensus.workspace = true -scroll-alloy-consensus.workspace = true - -[features] -serde = [ - "alloy-rpc-types-engine/serde", - "reth-scroll-primitives/serde", - "scroll-alloy-rpc-types-engine/serde", - "scroll-engine/serde", - "scroll-network/serde", - "scroll-wire/serde", - "alloy-eips/serde", - "scroll-alloy-hardforks/serde", - "alloy-chains/serde", - "alloy-consensus/serde", - "alloy-primitives/serde", - "reth-primitives-traits/serde", - "scroll-alloy-consensus/serde", - "reth-network-api/serde", - "reth-network/serde", -] diff --git a/crates/manager/src/consensus.rs b/crates/manager/src/consensus.rs deleted file mode 100644 index 5487dabf..00000000 --- a/crates/manager/src/consensus.rs +++ /dev/null @@ -1,163 +0,0 @@ -use alloy_primitives::{Address, Signature}; -use metrics::Counter; -use metrics_derive::Metrics; -use reth_primitives_traits::GotExpected; -use reth_scroll_primitives::ScrollBlock; -use rollup_node_primitives::{sig_encode_hash, ConsensusUpdate}; -use scroll_network::ConsensusError; -use std::fmt::Debug; - -/// A trait for consensus implementations. -pub trait Consensus: Send + Debug { - /// Updates the current config for the consensus. - fn update_config(&mut self, update: &ConsensusUpdate); - /// Validates a new block with the given signature. - fn validate_new_block( - &self, - block: &ScrollBlock, - signature: &Signature, - ) -> Result<(), ConsensusError>; - /// Returns a boolean indicating whether the sequencer should sequence a block. - fn should_sequence_block(&self, sequencer: &Address) -> bool; -} - -/// A no-op consensus instance. -#[non_exhaustive] -#[derive(Debug, Default)] -pub struct NoopConsensus; - -impl Consensus for NoopConsensus { - fn update_config(&mut self, _: &ConsensusUpdate) {} - - fn validate_new_block( - &self, - _block: &ScrollBlock, - _signature: &Signature, - ) -> Result<(), ConsensusError> { - Ok(()) - } - - fn should_sequence_block(&self, _sequencer: &Address) -> bool { - true - } -} - -/// The metrics for the [`SystemContractConsensus`]. -#[derive(Metrics, Clone)] -#[metrics(scope = "consensus")] -pub(crate) struct SystemContractConsensusMetrics { - /// System contract validate new block failed counter. - pub validate_new_block_failed: Counter, -} - -/// The system contract consensus. -#[derive(Debug)] -pub struct SystemContractConsensus { - authorized_signer: Address, - - /// The metrics for the [`SystemContractConsensus`]. - metrics: SystemContractConsensusMetrics, -} - -impl SystemContractConsensus { - /// Creates a new [`SystemContractConsensus`] consensus instance with the given authorized - /// signers. - pub fn new(authorized_signer: Address) -> Self { - Self { authorized_signer, metrics: SystemContractConsensusMetrics::default() } - } -} - -impl Consensus for SystemContractConsensus { - fn update_config(&mut self, update: &ConsensusUpdate) { - match update { - ConsensusUpdate::AuthorizedSigner(signer) => self.authorized_signer = *signer, - }; - } - - fn validate_new_block( - &self, - block: &ScrollBlock, - signature: &Signature, - ) -> Result<(), ConsensusError> { - let hash = sig_encode_hash(&block.header); - let signer = reth_primitives_traits::crypto::secp256k1::recover_signer(signature, hash)?; - - if self.authorized_signer != signer { - self.metrics.validate_new_block_failed.increment(1); - return Err(ConsensusError::IncorrectSigner(GotExpected { - got: signer, - expected: self.authorized_signer, - })) - } - Ok(()) - } - - fn should_sequence_block(&self, sequencer: &Address) -> bool { - sequencer == &self.authorized_signer - } -} - -#[cfg(test)] -mod tests { - use super::*; - use alloy_consensus::{Signed, TxEip1559}; - use alloy_primitives::{address, b256, bloom, bytes, TxKind, B64, U256}; - use reth_primitives_traits::Header; - use reth_scroll_primitives::ScrollBlockBody; - use std::{str::FromStr, sync::OnceLock}; - - #[test] - fn test_should_validate_block() { - let consensus = - SystemContractConsensus::new(address!("d83c4892bb5aa241b63d8c4c134920111e142a20")); - let signature = Signature::from_raw(&bytes!("6d2b8ef87f0956ea4dd10fb0725fa7196ad80c6d567a161f6b4367f95b5de6ec279142b540d3b248f08ed337bb962fa3fd83d21de622f7d6c8207272558fd15a00")).unwrap(); - - let tx_hash = OnceLock::new(); - tx_hash.get_or_init(|| { - b256!("f257edab88796a76f6d19a9fadad44b2b16c28e7aa70322cc4c6abc857128998") - }); - - let block = ScrollBlock { - header: Header { - parent_hash: b256!("3ccf36621e1f75cd1bfd2ac39ff6a00d8a5bec02e52aa7064a4860a0d02d6013"), - ommers_hash: b256!("1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347"), - beneficiary: address!("0000000000000000000000000000000000000000"), - state_root: b256!("bc6c2ccfdb3e0e78b53134f583e6d42760adcaebb23e7a6bab59503c4b98daeb"), - transactions_root: b256!("a11e1b74f0aada603d9b4e645a57d60259dc2545c0372b88e16e6b59cecac8b6"), - receipts_root: b256!("72de16699164034d2ed9930a021820e32e103ea7162b6f6a9a535d0a98f3fac0"), - logs_bloom: bloom!("0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000040000000000000000000000000000001000000000000000000000000000000000000000000000000000000000000000008000400000000000000000000000000000000004000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000008000000000010000040000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"), - difficulty: U256::ONE, - number: 14916920, - gas_limit: 10000000, - gas_used: 245760, - timestamp: 1745337938, - extra_data: bytes!("0x"), - mix_hash: b256!("0000000000000000000000000000000000000000000000000000000000000000"), - nonce: B64::from(0x0000000000000000i64), - base_fee_per_gas: Some(40306624), - withdrawals_root: None, - blob_gas_used: None, - excess_blob_gas: None, - parent_beacon_block_root: None, - requests_hash: None, - }, - body: ScrollBlockBody { - transactions: vec![ - Signed::new_unhashed(TxEip1559 { - chain_id: 534352, - nonce: 145014, - gas_limit: 600000, - max_fee_per_gas: 52355852, - max_priority_fee_per_gas: 0, - to: TxKind::Call(address!("802b65b5d9016621e66003aed0b16615093f328b")), - value: U256::ZERO, - access_list: Default::default(), - input: bytes!("a00597a00000000000000000000000000000000000000000000000000000000001826cbe0000000000000000000000000000000000005eb6831c1aa0faf2055c7d53270e00000000000000000000000006efdbff2a14a7c8e15944d1f4a48f9f95f663a40000000000000000000000000000000000000000000000000000000000000001000000000000000000000000813df550a32d4a9d42010d057386429ad2328ed9000000000000000000000000000000000000000000000000000000006807befd"), - }, Signature::new(U256::from_str("12217337930795921874768983252881296563512928283585900928219483692173266513447").unwrap(), U256::from_str("37490897792770890087946325233571758133021734266092518377537449521790010698782").unwrap(), true)).into()], - ommers: vec![], - withdrawals: None, - }, - }; - consensus.validate_new_block(&block, &signature).unwrap() - } -} diff --git a/crates/manager/src/lib.rs b/crates/manager/src/lib.rs deleted file mode 100644 index f02c2a2a..00000000 --- a/crates/manager/src/lib.rs +++ /dev/null @@ -1,9 +0,0 @@ -//! This library contains the main manager for the rollup node. - -pub use consensus::{Consensus, NoopConsensus, SystemContractConsensus}; -mod consensus; - -// mod manager; -// pub use manager::{ -// RollupManagerCommand, RollupManagerEvent, RollupManagerHandle, RollupNodeManager, -// }; diff --git a/crates/manager/src/manager/budget.rs b/crates/manager/src/manager/budget.rs deleted file mode 100644 index de29a493..00000000 --- a/crates/manager/src/manager/budget.rs +++ /dev/null @@ -1,28 +0,0 @@ -/// The budget for the L1 notification channel. -pub(crate) const L1_NOTIFICATION_CHANNEL_BUDGET: u32 = 5; - -/// Polls the given stream. Breaks with `true` if there maybe is more work. -#[macro_export] -macro_rules! poll_nested_stream_with_budget { - ($target:literal, $label:literal, $budget:ident, $poll_stream:expr, $on_ready_some:expr $(, $on_ready_none:expr;)? $(,)?) => {{ - let mut budget: u32 = $budget; - - loop { - match $poll_stream { - Poll::Ready(Some(item)) => { - $on_ready_some(item); - - budget -= 1; - if budget == 0 { - break true - } - } - Poll::Ready(None) => { - $($on_ready_none;)? // todo: handle error case with $target and $label - break false - } - Poll::Pending => break false, - } - } - }}; -} diff --git a/crates/manager/src/manager/command.rs b/crates/manager/src/manager/command.rs deleted file mode 100644 index 1d7d8d1b..00000000 --- a/crates/manager/src/manager/command.rs +++ /dev/null @@ -1,27 +0,0 @@ -use super::{RollupManagerEvent, RollupManagerStatus}; - -use reth_network_api::FullNetwork; -use reth_scroll_node::ScrollNetworkPrimitives; -use reth_tokio_util::EventStream; -use rollup_node_primitives::BlockInfo; -use scroll_network::ScrollNetworkHandle; -use tokio::sync::oneshot; - -/// The commands that can be sent to the rollup manager. -#[derive(Debug)] -pub enum RollupManagerCommand> { - /// Command to build a new block. - BuildBlock, - /// Returns an event stream for rollup manager events. - EventListener(oneshot::Sender>), - /// Report the current status of the manager via the oneshot channel. - Status(oneshot::Sender), - /// Returns the network handle. - NetworkHandle(oneshot::Sender>), - /// Update the head of the fcs in the engine driver. - UpdateFcsHead(BlockInfo), - /// Enable automatic sequencing. - EnableAutomaticSequencing(oneshot::Sender), - /// Disable automatic sequencing. - DisableAutomaticSequencing(oneshot::Sender), -} diff --git a/crates/manager/src/manager/event.rs b/crates/manager/src/manager/event.rs deleted file mode 100644 index 61749ea8..00000000 --- a/crates/manager/src/manager/event.rs +++ /dev/null @@ -1,41 +0,0 @@ -use alloy_primitives::B256; -use reth_scroll_primitives::ScrollBlock; -use rollup_node_chain_orchestrator::ChainOrchestratorEvent; -use rollup_node_signer::SignerEvent; -use rollup_node_watcher::L1Notification; -use scroll_db::L1MessageKey; -use scroll_engine::ConsolidationOutcome; -use scroll_network::NewBlockWithPeer; - -/// An event that can be emitted by the rollup node manager. -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum RollupManagerEvent { - /// A new block has been received from the network. - NewBlockReceived(NewBlockWithPeer), - /// New block sequenced. - BlockSequenced(ScrollBlock), - /// New block imported. - BlockImported(ScrollBlock), - /// Consolidated block derived from L1. - L1DerivedBlockConsolidated(ConsolidationOutcome), - /// A new event from the signer. - SignerEvent(SignerEvent), - /// A reorg event. - Reorg(u64), - /// An event from the chain orchestrator. - ChainOrchestratorEvent(ChainOrchestratorEvent), - /// An error occurred consolidating the L1 messages. - L1MessageConsolidationError { - /// The expected L1 messages hash. - expected: B256, - /// The actual L1 messages hash. - actual: B256, - }, - /// A block has been received containing an L1 message that is not in the database. - L1MessageMissingInDatabase { - /// The L1 message key. - key: L1MessageKey, - }, - /// An event was received from the L1 watcher. - L1NotificationEvent(L1Notification), -} diff --git a/crates/manager/src/manager/handle.rs b/crates/manager/src/manager/handle.rs deleted file mode 100644 index 66ff876f..00000000 --- a/crates/manager/src/manager/handle.rs +++ /dev/null @@ -1,75 +0,0 @@ -use super::{RollupManagerCommand, RollupManagerEvent}; -use crate::manager::metrics::HandleMetrics; -use reth_network_api::FullNetwork; -use reth_scroll_node::ScrollNetworkPrimitives; -use reth_tokio_util::EventStream; -use rollup_node_primitives::BlockInfo; -use scroll_network::ScrollNetworkHandle; -use tokio::sync::{mpsc, oneshot}; -use tracing::error; - -/// The handle used to send commands to the rollup manager. -#[derive(Debug, Clone)] -pub struct RollupManagerHandle> { - /// The channel used to send commands to the rollup manager. - to_manager_tx: mpsc::Sender>, - handle_metrics: HandleMetrics, -} - -impl> RollupManagerHandle { - /// Create a new rollup manager handle. - pub fn new(to_manager_tx: mpsc::Sender>) -> Self { - Self { to_manager_tx, handle_metrics: HandleMetrics::default() } - } - - /// Sends a command to the rollup manager. - pub async fn send_command(&self, command: RollupManagerCommand) { - if let Err(err) = self.to_manager_tx.send(command).await { - self.handle_metrics.handle_send_command_failed.increment(1); - error!(target: "rollup::manager::handle", "Failed to send command to rollup manager: {}", err); - } - } - - /// Sends a command to the rollup manager to build a block. - pub async fn build_block(&self) { - self.send_command(RollupManagerCommand::BuildBlock).await; - } - - /// Sends a command to the rollup manager to get the network handle. - pub async fn get_network_handle( - &self, - ) -> Result, oneshot::error::RecvError> { - let (tx, rx) = oneshot::channel(); - self.send_command(RollupManagerCommand::NetworkHandle(tx)).await; - rx.await - } - - /// Sends a command to the rollup manager to fetch an event listener for the rollup node - /// manager. - pub async fn get_event_listener( - &self, - ) -> Result, oneshot::error::RecvError> { - let (tx, rx) = oneshot::channel(); - self.send_command(RollupManagerCommand::EventListener(tx)).await; - rx.await - } - - /// Sends a command to the rollup manager to update the head of the FCS in the engine driver. - pub async fn update_fcs_head(&self, head: BlockInfo) { - self.send_command(RollupManagerCommand::UpdateFcsHead(head)).await; - } - - /// Sends a command to the rollup manager to enable automatic sequencing. - pub async fn enable_automatic_sequencing(&self) -> Result { - let (tx, rx) = oneshot::channel(); - self.send_command(RollupManagerCommand::EnableAutomaticSequencing(tx)).await; - rx.await - } - - /// Sends a command to the rollup manager to disable automatic sequencing. - pub async fn disable_automatic_sequencing(&self) -> Result { - let (tx, rx) = oneshot::channel(); - self.send_command(RollupManagerCommand::DisableAutomaticSequencing(tx)).await; - rx.await - } -} diff --git a/crates/manager/src/manager/metrics.rs b/crates/manager/src/manager/metrics.rs deleted file mode 100644 index 1e465caf..00000000 --- a/crates/manager/src/manager/metrics.rs +++ /dev/null @@ -1,48 +0,0 @@ -use metrics::{Counter, Gauge}; -use metrics_derive::Metrics; - -/// The metrics for the [`super::RollupManagerHandle`]. -#[derive(Metrics, Clone)] -#[metrics(scope = "NodeManager")] -pub(crate) struct HandleMetrics { - /// Failed to send command to rollup manager from handle counter. - pub handle_send_command_failed: Counter, -} - -/// The metrics for the [`super::RollupNodeManager`]. -#[derive(Metrics, Clone)] -#[metrics(scope = "NodeManager")] -pub(crate) struct RollupNodeManagerMetrics { - /// Manager received and handle rollup manager command counter. - pub handle_rollup_manager_command: Counter, - /// Manager received and handle engine driver event counter. - pub handle_engine_driver_event: Counter, - /// Manager received and handle new block produced counter. - pub handle_new_block_produced: Counter, - /// Manager received and handle l1 notification counter. - pub handle_l1_notification: Counter, - /// Manager received and handle chain orchestrator event counter. - pub handle_chain_orchestrator_event: Counter, - /// Manager received and handle signer event counter. - pub handle_signer_event: Counter, - /// Manager received and handle build new payload counter. - pub handle_build_new_payload: Counter, - /// Manager received and handle l1 consolidation counter. - pub handle_l1_consolidation: Counter, - /// Manager received and handle network manager event counter. - pub handle_network_manager_event: Counter, - /// Manager finalized batch index gauge. - pub handle_finalized_batch_index: Gauge, - /// Manager l1 finalized block number gauge. - pub handle_l1_finalized_block_number: Gauge, - /// Manager L1 reorg L1 block number gauge. - pub handle_l1_reorg_l1_block_number: Gauge, - /// Manager L1 reorg L2 head block number gauge. - pub handle_l1_reorg_l2_head_block_number: Gauge, - /// Manager L1 reorg L2 safe block number gauge. - pub handle_l1_reorg_l2_safe_block_number: Gauge, - /// Manager chain import block number gauge. - pub handle_chain_import_block_number: Gauge, - /// Manager optimistic syncing block number gauge. - pub handle_optimistic_syncing_block_number: Gauge, -} diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs deleted file mode 100644 index c282d02e..00000000 --- a/crates/manager/src/manager/mod.rs +++ /dev/null @@ -1,758 +0,0 @@ -//! The [`RollupNodeManager`] is the main component of the rollup node that manages the -//! [`ScrollNetworkManager`], [`EngineDriver`], [`ChainOrchestrator`] and [`Consensus`] components. -//! It is responsible for handling events from these components and coordinating their actions. - -use super::Consensus; -use crate::poll_nested_stream_with_budget; -use ::metrics::counter; -use alloy_provider::Provider; -use futures::StreamExt; -use reth_chainspec::EthChainSpec; -use reth_network::BlockDownloaderProvider; -use reth_network_api::FullNetwork; -use reth_scroll_node::ScrollNetworkPrimitives; -use reth_tasks::shutdown::GracefulShutdown; -use reth_tokio_util::{EventSender, EventStream}; -use rollup_node_chain_orchestrator::{ - ChainOrchestrator, ChainOrchestratorError, ChainOrchestratorEvent, -}; -use rollup_node_primitives::BlockInfo; -use rollup_node_sequencer::Sequencer; -use rollup_node_signer::{SignerEvent, SignerHandle}; -use rollup_node_watcher::L1Notification; -use scroll_alloy_hardforks::ScrollHardforks; -use scroll_alloy_network::Scroll; -use scroll_alloy_provider::ScrollEngineApi; -use scroll_engine::{EngineDriver, EngineDriverEvent, ForkchoiceState}; -use scroll_network::{ - BlockImportOutcome, NewBlockWithPeer, ScrollNetworkManager, ScrollNetworkManagerEvent, -}; -use std::{ - fmt::{self, Debug, Formatter}, - future::Future, - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; -use tokio::{ - sync::mpsc::{self, Receiver}, - time::Interval, -}; -use tokio_stream::wrappers::ReceiverStream; -use tracing::{error, info, trace, warn}; - -use rollup_node_providers::{L1MessageProvider, L1Provider}; -use scroll_db::{Database, DatabaseError, DatabaseTransactionProvider, DatabaseWriteOperations}; -use scroll_derivation_pipeline::DerivationPipeline; - -mod budget; -use budget::L1_NOTIFICATION_CHANNEL_BUDGET; - -mod command; -pub use command::RollupManagerCommand; - -mod event; -pub use event::RollupManagerEvent; - -mod handle; -mod metrics; - -use crate::manager::metrics::RollupNodeManagerMetrics; -pub use handle::RollupManagerHandle; - -/// The size of the event channel. -const EVENT_CHANNEL_SIZE: usize = 100; - -/// The maximum capacity of the pending futures queue in the chain orchestrator for acceptance of -/// new events from the L1 notification channel. -const CHAIN_ORCHESTRATOR_MAX_PENDING_FUTURES: usize = 20; - -/// The maximum number of pending futures in the engine driver for acceptance of new events from the -/// L1 notification channel. -const ENGINE_MAX_PENDING_FUTURES: usize = 5000; - -/// The maximum number of pending batch commits in the derivation pipeline for acceptance of new -/// events from the L1 notification channel. -const DERIVATION_PIPELINE_MAX_PENDING_BATCHES: usize = 500; - -/// The main manager for the rollup node. -/// -/// This is an endless [`Future`] that drives the state of the entire network forward and includes -/// the following components: -/// - `network`: Responsible for peer discover, managing connections between peers and operation of -/// the eth-wire protocol. -/// - `engine`: Responsible for importing blocks that have been gossiped over the scroll-wire -/// protocol. -/// - `consensus`: The consensus algorithm used by the rollup node. -/// - `new_block_rx`: Receives new blocks from the network. -/// - `forkchoice_state`: The forkchoice state of the rollup node. -/// - `pending_block_imports`: A collection of pending block imports. -/// - `event_sender`: An event sender for sending events to subscribers of the rollup node manager. -pub struct RollupNodeManager< - N: FullNetwork, - EC, - P, - L1P, - L1MP, - CS, -> { - /// The handle receiver used to receive commands. - handle_rx: Receiver>, - /// The chain orchestrator. - chain: ChainOrchestrator::Client, P>, - /// An event sender for sending events to subscribers of the rollup node manager. - event_sender: Option>, - /// The sequencer which is responsible for sequencing transactions and producing new blocks. - sequencer: Option>, - /// The signer handle used to sign artifacts. - signer: Option, - /// The trigger for the block building process. - block_building_trigger: Option, - /// A connection to the database. - database: Arc, - /// The original block time configuration for restoring automatic sequencing. - block_time_config: Option, - /// Metrics for the rollup node manager. - metrics: RollupNodeManagerMetrics, -} - -/// The current status of the rollup manager. -#[derive(Debug)] -pub struct RollupManagerStatus { - /// Whether the rollup manager is syncing. - pub syncing: bool, - /// The current FCS for the manager. - pub forkchoice_state: ForkchoiceState, -} - -impl< - N: FullNetwork, - EC: Debug, - P: Debug, - L1P: Debug, - L1MP: Debug, - CS: Debug, - > Debug for RollupNodeManager -{ - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - f.debug_struct("RollupNodeManager") - .field("network", &self.network) - .field("engine", &self.engine) - .field("derivation_pipeline", &self.derivation_pipeline) - .field("l1_notification_rx", &self.l1_notification_rx) - .field("chain_orchestrator", &self.chain) - .field("consensus", &self.consensus) - .field("eth_wire_block_rx", &"eth_wire_block_rx") - .field("event_sender", &self.event_sender) - .field("sequencer", &self.sequencer) - .field("block_building_trigger", &self.block_building_trigger) - .field("block_time_config", &self.block_time_config) - .finish() - } -} - -impl RollupNodeManager -where - N: FullNetwork, - EC: ScrollEngineApi + Unpin + Sync + Send + 'static, - P: Provider + Clone + Unpin + Send + Sync + 'static, - L1P: L1Provider + Clone + Send + Sync + Unpin + 'static, - L1MP: L1MessageProvider + Unpin + Send + Sync + 'static, - CS: ScrollHardforks + EthChainSpec + Send + Sync + 'static, -{ - /// Create a new [`RollupNodeManager`] instance. - #[allow(clippy::too_many_arguments)] - #[allow(clippy::new_ret_no_self)] - pub async fn new( - network: ScrollNetworkManager, - engine: EngineDriver, - l1_provider: L1P, - database: Arc, - l1_notification_rx: Option>>, - consensus: Box, - sequencer: Option>, - signer: Option, - block_time: Option, - auto_start: bool, - chain_orchestrator: ChainOrchestrator::Client, P>, - l1_v2_message_queue_start_index: u64, - ) -> (Self, RollupManagerHandle) { - let (handle_tx, handle_rx) = mpsc::channel(EVENT_CHANNEL_SIZE); - let derivation_pipeline = - DerivationPipeline::new(l1_provider, database.clone(), l1_v2_message_queue_start_index); - let rnm = Self { - handle_rx, - chain: chain_orchestrator, - event_sender: None, - sequencer, - signer, - block_building_trigger: if auto_start { - block_time.map(delayed_interval) - } else { - None - }, - database, - block_time_config: block_time, - metrics: RollupNodeManagerMetrics::default(), - }; - (rnm, RollupManagerHandle::new(handle_tx)) - } - - /// Returns a new event listener for the rollup node manager. - pub fn event_listener(&mut self) -> EventStream { - if let Some(event_sender) = &self.event_sender { - return event_sender.new_listener(); - }; - - let event_sender = EventSender::new(EVENT_CHANNEL_SIZE); - let event_listener = event_sender.new_listener(); - self.event_sender = Some(event_sender); - - event_listener - } - - /// Handles a new block received from the network. - /// - /// We will first validate the consensus of the block, then we will send the block to the engine - /// to validate the correctness of the block. - pub fn handle_new_block(&mut self, block_with_peer: NewBlockWithPeer) { - trace!(target: "scroll::node::manager", "Received new block from peer {:?} - hash {:?}", block_with_peer.peer_id, block_with_peer.block.hash_slow()); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::NewBlockReceived(block_with_peer.clone())); - } - - // Validate the consensus of the block. - // TODO: Should we spawn a task to validate the consensus of the block? - // Is the consensus validation blocking? - if let Err(err) = - self.consensus.validate_new_block(&block_with_peer.block, &block_with_peer.signature) - { - error!(target: "scroll::node::manager", ?err, "consensus checks failed on block {:?} from peer {:?}", block_with_peer.block.hash_slow(), block_with_peer.peer_id); - self.network.handle().block_import_outcome(BlockImportOutcome { - peer: block_with_peer.peer_id, - result: Err(err.into()), - }); - } else { - self.chain.handle_block_from_peer(block_with_peer.clone()); - - // TODO: remove this once we deprecate l2geth. - // Store the block signature in the database - let db = self.database.clone(); - let block_hash = block_with_peer.block.hash_slow(); - let signature = block_with_peer.signature; - tokio::spawn(async move { - let tx = if let Ok(tx) = db.tx_mut().await { - tx - } else { - tracing::warn!(target: "scroll::node::manager", %block_hash, sig=%signature, "Failed to create database transaction"); - return; - }; - if let Err(err) = tx.insert_signature(block_hash, signature).await { - tracing::warn!( - target: "scroll::node::manager", - %block_hash, sig=%signature, error=?err, - "Failed to store block signature; execution client already persisted the block" - ); - } else { - tracing::trace!( - target: "scroll::node::manager", - %block_hash, sig=%signature, - "Persisted block signature to database" - ); - } - if let Err(err) = tx.commit().await { - tracing::warn!(target: "scroll::node::manager", %block_hash, sig=%signature, error=?err, "Failed to commit database transaction"); - } - }); - } - } - - /// Handles a network manager event. - /// - /// Currently the network manager only emits a `NewBlock` event. - fn handle_network_manager_event(&mut self, event: ScrollNetworkManagerEvent) { - match event { - ScrollNetworkManagerEvent::NewBlock(block) => self.handle_new_block(block), - } - } - - /// Handles a chain orchestrator event. - fn handle_chain_orchestrator_event(&mut self, event: ChainOrchestratorEvent) { - trace!(target: "scroll::node::manager", ?event, "Received chain orchestrator event"); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::ChainOrchestratorEvent(event.clone())); - } - - match event { - #[allow(clippy::match_same_arms)] - ChainOrchestratorEvent::BatchCommitIndexed { .. } => { - // Uncomment once we implement issue #273. - // // if we detected a batch revert event, we reset the pipeline and the engine - // driver. if let Some(new_safe_head) = safe_head { - // self.derivation_pipeline.handle_batch_revert(batch_info.index); - // self.engine.clear_l1_payload_attributes(); - // self.engine.set_head_block_info(new_safe_head); - // self.engine.set_safe_block_info(new_safe_head); - // } - // // push the batch info into the derivation pipeline. - // self.derivation_pipeline.push_batch(batch_info, l1_block_number); - } - ChainOrchestratorEvent::BatchFinalized(block_number, finalized_batches) => { - // Uncomment once we implement issue #273. - // // update the fcs on new finalized block. - // if let Some(finalized_block) = finalized_block { - // self.engine.set_finalized_block_info(finalized_block); - // } - // Remove once we implement issue #273. - // Update the derivation pipeline on new finalized batch. - for batch_info in finalized_batches { - self.metrics.handle_finalized_batch_index.set(batch_info.index as f64); - self.derivation_pipeline.push_batch(batch_info, block_number); - } - } - ChainOrchestratorEvent::L1BlockFinalized(l1_block_number, finalized_batches, ..) => { - self.metrics.handle_l1_finalized_block_number.set(l1_block_number as f64); - // update the sequencer's l1 finalized block number. - if let Some(sequencer) = self.sequencer.as_mut() { - sequencer.set_l1_finalized_block_number(l1_block_number); - } - // Uncomment once we implement issue #273. - // // update the fcs on new finalized block. - // if let Some(finalized_block) = finalized_block { - // self.engine.set_finalized_block_info(finalized_block); - // } - // Remove once we implement issue #273. - // push all finalized batches into the derivation pipeline. - for batch_info in finalized_batches { - self.derivation_pipeline.push_batch(batch_info, l1_block_number); - } - } - ChainOrchestratorEvent::L1Reorg { - l1_block_number, - queue_index, - l2_head_block_info, - l2_safe_block_info, - } => { - self.metrics.handle_l1_reorg_l1_block_number.set(l1_block_number as f64); - self.metrics - .handle_l1_reorg_l2_head_block_number - .set(l2_head_block_info.as_ref().map_or(0, |info| info.number) as f64); - self.metrics - .handle_l1_reorg_l2_safe_block_number - .set(l2_safe_block_info.as_ref().map_or(0, |info| info.number) as f64); - - // Handle the reorg in the engine driver. - self.engine.handle_l1_reorg( - l1_block_number, - l2_head_block_info, - l2_safe_block_info, - ); - - // Update the [`Sequencer`] with the new L1 head info and queue index. - if let Some(sequencer) = self.sequencer.as_mut() { - sequencer.handle_reorg(queue_index, l1_block_number); - } - - // Handle the reorg in the derivation pipeline. - self.derivation_pipeline.handle_reorg(l1_block_number); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::Reorg(l1_block_number)); - } - } - ChainOrchestratorEvent::ChainExtended(chain_import) | - ChainOrchestratorEvent::ChainReorged(chain_import) => { - self.metrics - .handle_chain_import_block_number - .set(chain_import.chain.last().unwrap().number as f64); - - // Issue the new chain to the engine driver for processing. - self.engine.handle_chain_import(chain_import) - } - ChainOrchestratorEvent::OptimisticSync(block) => { - let block_info: BlockInfo = (&block).into(); - - self.metrics.handle_optimistic_syncing_block_number.set(block_info.number as f64); - - // Issue the new block info to the engine driver for processing. - self.engine.handle_optimistic_sync(block_info) - } - _ => {} - } - } - - /// Handles a chain orchestrator error. - fn handle_chain_orchestrator_error(&self, err: &ChainOrchestratorError) { - error!( - target: "scroll::node::manager", - error = ?err, - msg = %err, - "Error occurred in the chain orchestrator" - ); - - match err { - ChainOrchestratorError::L1MessageMismatch { expected, actual } => { - counter!( - "manager_handle_chain_orchestrator_event_failed", - "type" => "l1_message_mismatch", - ) - .increment(1); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1MessageConsolidationError { - expected: *expected, - actual: *actual, - }); - } - } - ChainOrchestratorError::DatabaseError(DatabaseError::L1MessageNotFound(start)) => { - counter!( - "manager_handle_chain_orchestrator_event_failed", - "type" => "l1_message_not_found", - ) - .increment(1); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1MessageMissingInDatabase { - key: start.clone(), - }); - } - } - _ => {} - } - } - - /// Handles an engine driver event. - fn handle_engine_driver_event(&mut self, event: EngineDriverEvent) { - trace!(target: "scroll::node::manager", ?event, "Received engine driver event"); - match event { - EngineDriverEvent::BlockImportOutcome(outcome) => { - if let Some(block) = outcome.block() { - if let Some(sequencer) = self.sequencer.as_mut() { - sequencer.handle_new_payload(&block); - } - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::BlockImported(block.clone())); - } - self.chain.consolidate_validated_l2_blocks(vec![(&block).into()]); - } - self.network.handle().block_import_outcome(outcome); - } - EngineDriverEvent::NewPayload(payload) => { - if let Some(signer) = self.signer.as_mut() { - let _ = signer.sign_block(payload.clone()).inspect_err(|err| error!(target: "scroll::node::manager", ?err, "Failed to send new payload to signer")); - } - - self.sequencer - .as_mut() - .expect("Sequencer must be enabled to build payload") - .handle_new_payload(&payload); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::BlockSequenced(payload)); - } - } - EngineDriverEvent::L1BlockConsolidated(consolidation_outcome) => { - self.chain.persist_l1_consolidated_blocks( - vec![consolidation_outcome.block_info().clone()], - *consolidation_outcome.batch_info(), - ); - - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1DerivedBlockConsolidated( - consolidation_outcome, - )); - } - } - EngineDriverEvent::ChainImportOutcome(outcome) => { - if let Some(block) = outcome.outcome.block() { - if let Some(sequencer) = self.sequencer.as_mut() { - sequencer.handle_new_payload(&block); - } - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::BlockImported(block)); - } - self.chain.consolidate_validated_l2_blocks( - outcome.chain.iter().map(|b| b.into()).collect(), - ); - } - self.network.handle().block_import_outcome(outcome.outcome); - } - } - } - - /// Handles an [`L1Notification`] from the L1 watcher. - fn handle_l1_notification(&mut self, notification: L1Notification) { - self.metrics.handle_l1_notification.increment(1); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1NotificationEvent(notification.clone())); - } - - match notification { - L1Notification::Consensus(ref update) => self.consensus.update_config(update), - L1Notification::NewBlock(new_block) => { - if let Some(sequencer) = self.sequencer.as_mut() { - sequencer.handle_new_l1_block(new_block) - } - } - // _ => self.chain.handle_l1_notification(notification), - _ => (), - } - } - - /// Returns the current status of the [`RollupNodeManager`]. - fn status(&self) -> RollupManagerStatus { - RollupManagerStatus { - syncing: self.engine.is_syncing(), - forkchoice_state: self.engine.forkchoice_state().clone(), - } - } - - /// Drives the [`RollupNodeManager`] future until a [`GracefulShutdown`] signal is received. - pub async fn run_until_graceful_shutdown(mut self, shutdown: GracefulShutdown) { - let mut graceful_guard = None; - - tokio::select! { - _ = &mut self => {}, - guard = shutdown => { - graceful_guard = Some(guard); - }, - } - - drop(graceful_guard); - } - - /// Returns true if the manager has capacity to accept new L1 notifications. - pub fn has_capacity_for_l1_notifications(&self) -> bool { - let chain_orchestrator_has_capacity = self.chain.pending_futures_len() < - CHAIN_ORCHESTRATOR_MAX_PENDING_FUTURES - L1_NOTIFICATION_CHANNEL_BUDGET as usize; - let engine_has_capacity = self.engine.pending_futures_len() < ENGINE_MAX_PENDING_FUTURES; - let derivation_pipeline_has_capacity = - self.derivation_pipeline.batch_queue_size() < DERIVATION_PIPELINE_MAX_PENDING_BATCHES; - chain_orchestrator_has_capacity && engine_has_capacity && derivation_pipeline_has_capacity - } -} - -impl Future for RollupNodeManager -where - N: FullNetwork, - EC: ScrollEngineApi + Unpin + Sync + Send + 'static, - P: Provider + Clone + Unpin + Send + Sync + 'static, - L1P: L1Provider + Clone + Unpin + Send + Sync + 'static, - L1MP: L1MessageProvider + Unpin + Send + Sync + 'static, - CS: ScrollHardforks + EthChainSpec + Unpin + Send + Sync + 'static, -{ - type Output = (); - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.get_mut(); - // Helper macro, proceeds with the $task if $proceed is true. - macro_rules! proceed_if { - ($proceed: expr, $task: expr) => { - if $proceed { - $task - } - }; - } - let en_synced = !this.engine.is_syncing(); - - // Poll the handle receiver for commands. - while let Poll::Ready(Some(command)) = this.handle_rx.poll_recv(cx) { - this.metrics.handle_rollup_manager_command.increment(1); - match command { - RollupManagerCommand::BuildBlock => { - proceed_if!( - en_synced, - if let Some(sequencer) = this.sequencer.as_mut() { - sequencer.build_payload_attributes(); - } - ); - } - RollupManagerCommand::EventListener(tx) => { - let events = this.event_listener(); - tx.send(events).expect("Failed to send event listener to handle"); - } - RollupManagerCommand::Status(tx) => { - tx.send(this.status()).expect("Failed to send status to handle"); - } - RollupManagerCommand::UpdateFcsHead(head) => { - trace!(target: "scroll::node::manager", ?head, "Updating FCS head block info"); - this.engine.set_head_block_info(head); - } - RollupManagerCommand::NetworkHandle(tx) => { - let network_handle = this.network.handle(); - tx.send(network_handle.clone()) - .expect("Failed to send network handle to handle"); - } - RollupManagerCommand::EnableAutomaticSequencing(tx) => { - let success = if let Some(block_time) = this.block_time_config { - if this.block_building_trigger.is_none() { - this.block_building_trigger = Some(delayed_interval(block_time)); - info!(target: "scroll::node::manager", "Enabled automatic sequencing with interval {}ms", block_time); - } else { - info!(target: "scroll::node::manager", "Automatic sequencing already enabled"); - } - true - } else { - warn!(target: "scroll::node::manager", "Cannot enable automatic sequencing: sequencer and block time not configured"); - false - }; - tx.send(success).expect("Failed to send enable automatic sequencing response"); - } - RollupManagerCommand::DisableAutomaticSequencing(tx) => { - let was_enabled = this.block_building_trigger.is_some(); - this.block_building_trigger = None; - info!(target: "scroll::node::manager", "Disabled automatic sequencing (was enabled: {})", was_enabled); - tx.send(true).expect("Failed to send disable automatic sequencing response"); - } - } - } - - // Drain all EngineDriver events. - while let Poll::Ready(Some(event)) = this.engine.poll_next_unpin(cx) { - this.metrics.handle_engine_driver_event.increment(1); - this.handle_engine_driver_event(event); - } - - proceed_if!( - en_synced, - // Handle new block production. - if let Some(Poll::Ready(Some(attributes))) = - this.sequencer.as_mut().map(|x| x.poll_next_unpin(cx)) - { - this.metrics.handle_new_block_produced.increment(1); - this.engine.handle_build_new_payload(attributes); - } - ); - - let mut maybe_more_l1_rx_events = false; - proceed_if!( - en_synced && this.has_capacity_for_l1_notifications(), - maybe_more_l1_rx_events = poll_nested_stream_with_budget!( - "l1_notification_rx", - "L1Notification channel", - L1_NOTIFICATION_CHANNEL_BUDGET, - this.l1_notification_rx - .as_mut() - .map(|rx| rx.poll_next_unpin(cx)) - .unwrap_or(Poll::Ready(None)), - |event: Arc| this.handle_l1_notification((*event).clone()), - ) - ); - - // Drain all chain orchestrator events. - while let Poll::Ready(Some(result)) = this.chain.poll_next_unpin(cx) { - this.metrics.handle_chain_orchestrator_event.increment(1); - match result { - Ok(event) => this.handle_chain_orchestrator_event(event), - Err(err) => { - this.handle_chain_orchestrator_error(&err); - } - } - } - - // Drain all signer events. - while let Some(Poll::Ready(Some(event))) = - this.signer.as_mut().map(|s| s.poll_next_unpin(cx)) - { - this.metrics.handle_signer_event.increment(1); - match event { - SignerEvent::SignedBlock { block, signature } => { - trace!(target: "scroll::node::manager", ?block, ?signature, "Received signed block from signer, announcing to the network"); - // Send SignerEvent for test monitoring - if let Some(event_sender) = this.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::SignerEvent( - SignerEvent::SignedBlock { block: block.clone(), signature }, - )); - } - - // TODO: remove this once we deprecate l2geth. - // Store the block signature in the database - let db = this.database.clone(); - let block_hash = block.hash_slow(); - tokio::spawn(async move { - let tx = if let Ok(tx) = db.tx_mut().await { - tx - } else { - tracing::warn!(target: "scroll::node::manager", %block_hash, sig=%signature, "Failed to create database transaction"); - return; - }; - if let Err(err) = tx.insert_signature(block_hash, signature).await { - tracing::warn!( - target: "scroll::node::manager", - %block_hash, sig=%signature, error=?err, - "Failed to store block signature; execution client already persisted the block" - ); - } else { - tracing::trace!( - target: "scroll::node::manager", - %block_hash, sig=%signature, - "Persisted block signature to database" - ); - } - if let Err(err) = tx.commit().await { - tracing::warn!(target: "scroll::node::manager", %block_hash, sig=%signature, error=?err, "Failed to commit database transaction"); - } - }); - - this.chain.handle_sequenced_block(NewBlockWithPeer { - peer_id: Default::default(), - block: block.clone(), - signature, - }); - this.network.handle().announce_block(block, signature); - } - } - } - - proceed_if!( - en_synced, - // Check if we need to trigger the build of a new payload. - if let (Some(Poll::Ready(_)), Some(sequencer)) = ( - this.block_building_trigger.as_mut().map(|trigger| trigger.poll_tick(cx)), - this.sequencer.as_mut() - ) { - this.metrics.handle_build_new_payload.increment(1); - if !this.consensus.should_sequence_block( - this.signer - .as_ref() - .map(|s| &s.address) - .expect("signer must be set if sequencer is present"), - ) { - trace!(target: "scroll::node::manager", "Signer is not authorized to sequence block for this slot"); - } else if this.engine.is_payload_building_in_progress() { - warn!(target: "scroll::node::manager", "Payload building is already in progress skipping slot"); - } else { - sequencer.build_payload_attributes(); - } - } - ); - - // Poll Derivation Pipeline and push attribute in queue if any. - while let Poll::Ready(Some(attributes)) = this.derivation_pipeline.poll_next_unpin(cx) { - this.metrics.handle_l1_consolidation.increment(1); - this.engine.handle_l1_consolidation(attributes) - } - - // Handle network manager events. - while let Poll::Ready(Some(event)) = this.network.poll_next_unpin(cx) { - this.metrics.handle_network_manager_event.increment(1); - this.handle_network_manager_event(event); - } - - if maybe_more_l1_rx_events { - cx.waker().wake_by_ref(); - } - - Poll::Pending - } -} - -/// Creates a delayed interval that will not skip ticks if the interval is missed but will delay -/// the next tick until the interval has passed. -fn delayed_interval(interval: u64) -> Interval { - let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(interval)); - interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - interval -} diff --git a/crates/network/Cargo.toml b/crates/network/Cargo.toml index 8dceb191..430a169f 100644 --- a/crates/network/Cargo.toml +++ b/crates/network/Cargo.toml @@ -51,3 +51,10 @@ serde = [ "reth-storage-api/serde", "scroll-alloy-hardforks/serde", ] +test-utils = [ + "reth-chainspec/test-utils", + "reth-network/test-utils", + "reth-network-types/test-utils", + "reth-primitives-traits/test-utils", + "reth-scroll-node/test-utils", +] diff --git a/crates/network/src/handle.rs b/crates/network/src/handle.rs index 8f988355..5fa8cfe9 100644 --- a/crates/network/src/handle.rs +++ b/crates/network/src/handle.rs @@ -76,14 +76,26 @@ impl ScrollNetworkHandle { self.send_message(NetworkHandleMessage::EventListener(tx)); rx.await.expect("network manager dropped") } + + #[cfg(feature = "test-utils")] + pub async fn set_gossip(&self, enabled: bool) { + let (tx, rx) = oneshot::channel(); + self.send_message(NetworkHandleMessage::SetGossip((enabled, tx))); + rx.await.expect("network manager dropped"); + } } /// A message type used for communication between the [`ScrollNetworkHandle`] and the /// [`super::ScrollNetworkManager`]. #[derive(Debug)] pub enum NetworkHandleMessage { - AnnounceBlock { block: ScrollBlock, signature: Signature }, + AnnounceBlock { + block: ScrollBlock, + signature: Signature, + }, BlockImportOutcome(super::BlockImportOutcome), Shutdown(oneshot::Sender<()>), EventListener(oneshot::Sender>), + #[cfg(feature = "test-utils")] + SetGossip((bool, oneshot::Sender<()>)), } diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index 28a1d203..c953958a 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -64,6 +64,9 @@ pub struct ScrollNetworkManager { td_constant: U128, /// The authorized signer for the network. authorized_signer: Option

, + /// Whether to gossip blocks to peers. + #[cfg(feature = "test-utils")] + gossip: bool, /// The event sender for network events. event_sender: EventSender, } @@ -117,6 +120,8 @@ impl td_constant, authorized_signer, event_sender, + #[cfg(feature = "test-utils")] + gossip: true, }, handle, ) @@ -162,6 +167,8 @@ impl< td_constant, authorized_signer, event_sender, + #[cfg(feature = "test-utils")] + gossip: true, }, handle, ) @@ -169,6 +176,11 @@ impl< /// Announces a new block to the network. fn announce_block(&mut self, block: NewBlock) { + #[cfg(feature = "test-utils")] + if !self.gossip { + return; + } + // Compute the block hash. let hash = block.block.hash_slow(); @@ -271,6 +283,11 @@ impl< NetworkHandleMessage::EventListener(tx) => { let _ = tx.send(self.event_sender.new_listener()); } + #[cfg(feature = "test-utils")] + NetworkHandleMessage::SetGossip((enabled, tx)) => { + self.gossip = enabled; + let _ = tx.send(()); + } } } diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index 4dc2f8ee..518d7775 100644 --- a/crates/node/Cargo.toml +++ b/crates/node/Cargo.toml @@ -64,7 +64,6 @@ reth-trie-db.workspace = true # rollup node rollup-node-chain-orchestrator.workspace = true -rollup-node-manager.workspace = true rollup-node-primitives.workspace = true rollup-node-providers.workspace = true rollup-node-sequencer.workspace = true @@ -143,4 +142,7 @@ test-utils = [ "scroll-alloy-rpc-types-engine", "alloy-rpc-types-engine", "reth-primitives-traits/test-utils", + "reth-network-p2p/test-utils", + "rollup-node-chain-orchestrator/test-utils", + "scroll-network/test-utils", ] diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 3f10e898..7a946c02 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -4,7 +4,7 @@ use crate::{ context::RollupNodeContext, }; use scroll_migration::MigratorTrait; -use std::{fs, path::PathBuf, sync::Arc, time::Duration}; +use std::{fs, path::PathBuf, sync::Arc}; use alloy_chains::NamedChain; use alloy_primitives::{hex, Address, U128}; @@ -49,8 +49,8 @@ use scroll_db::{ Database, DatabaseConnectionProvider, DatabaseReadOperations, DatabaseTransactionProvider, DatabaseWriteOperations, }; -use scroll_derivation_pipeline::DerivationPipelineNew; -use scroll_engine::{genesis_hash_from_chain_spec, Engine, EngineDriver, ForkchoiceState}; +use scroll_derivation_pipeline::DerivationPipeline; +use scroll_engine::{Engine, ForkchoiceState}; use scroll_migration::traits::ScrollMigrator; use scroll_network::ScrollNetworkManager; use scroll_wire::ScrollWireEvent; @@ -154,7 +154,7 @@ impl ScrollRollupNodeConfig { } impl ScrollRollupNodeConfig { - /// Consumes the [`ScrollRollupNodeConfig`] and builds a [`RollupNodeManager`]. + /// Consumes the [`ScrollRollupNodeConfig`] and builds a [`ChainOrchestrator`]. pub async fn build( self, ctx: RollupNodeContext, @@ -268,7 +268,7 @@ impl ScrollRollupNodeConfig { // block hash to the latest block hash associated with the previous consolidated // batch in the database. let tx = db.tx_mut().await?; - let (startup_safe_block, l1_start_block_number) = + let (_startup_safe_block, l1_start_block_number) = tx.prepare_on_startup(chain_spec.genesis_hash()).await?; tx.commit().await?; // if let Some(block_info) = startup_safe_block { @@ -378,7 +378,7 @@ impl ScrollRollupNodeConfig { // Construct the Sequencer. let chain_config = chain_spec.chain_config(); - let sequencer = if self.sequencer_args.sequencer_enabled { + let sequencer = self.sequencer_args.sequencer_enabled.then(|| { let args = &self.sequencer_args; let config = SequencerConfig { chain_spec: chain_spec.clone(), @@ -396,11 +396,8 @@ impl ScrollRollupNodeConfig { allow_empty_blocks: args.allow_empty_blocks, payload_building_duration: args.payload_building_duration, }; - let sequencer = Sequencer::new(Arc::new(l1_messages_provider), config); - Some(sequencer) - } else { - None - }; + Sequencer::new(Arc::new(l1_messages_provider), config) + }); // Instantiate the signer let chain_id = chain_spec.chain().id(); @@ -432,7 +429,7 @@ impl ScrollRollupNodeConfig { ); // Instantiate the derivation pipeline - let derivation_pipeline = DerivationPipelineNew::new( + let derivation_pipeline = DerivationPipeline::new( l1_provider.clone(), db.clone(), l1_v2_message_queue_start_index, diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 852d3512..64480f05 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -18,10 +18,7 @@ use reth_scroll_chainspec::{ScrollChainSpec, SCROLL_DEV, SCROLL_MAINNET, SCROLL_ use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; use reth_storage_api::BlockReader; -use reth_tasks::{ - shutdown::{self, signal as shutdown_signal}, - TaskExecutor, -}; +use reth_tasks::shutdown::signal as shutdown_signal; use reth_tokio_util::EventStream; use rollup_node::{ constants::SCROLL_GAS_LIMIT, @@ -34,7 +31,7 @@ use rollup_node::{ RollupNodeGasPriceOracleArgs, RollupNodeNetworkArgs as ScrollNetworkArgs, RpcArgs, ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, }; -use rollup_node_chain_orchestrator::{ChainOrchestratorEvent, ChainOrchestratorHandle}; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_primitives::{sig_encode_hash, BatchCommitData, BlockInfo, ConsensusUpdate}; use rollup_node_sequencer::L1MessageInclusionMode; use rollup_node_watcher::L1Notification; @@ -51,10 +48,7 @@ use std::{ task::{Context, Poll}, time::Duration, }; -use tokio::{ - sync::{oneshot, Mutex}, - time, -}; +use tokio::{sync::Mutex, time}; use tracing::trace; #[tokio::test] @@ -94,6 +88,9 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { let mut events = chain_orchestrator.get_event_listener().await?; let l1_watcher_tx = node.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + // Send a notification to set the L1 to synced + l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + let l1_message = TxL1Message { queue_index: 0, gas_limit: 21000, @@ -124,7 +121,7 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { ) .await; - chain_orchestrator.build_block().await; + chain_orchestrator.build_block(); wait_n_events( &mut events, @@ -191,13 +188,17 @@ async fn can_sequence_and_gossip_blocks() { // generate rollup node manager event streams for each node let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await.unwrap(); + let sequencer_l1_watcher_tx = nodes[0].inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let mut follower_events = nodes[1].inner.add_ons_handle.rollup_manager_handle.get_event_listener().await.unwrap(); + // Send a notification to set the L1 to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + // inject a transaction into the pool of the first node let tx = generate_tx(wallet).await; nodes[0].rpc.inject_tx(tx).await.unwrap(); - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // wait for the sequencer to build a block wait_n_events( @@ -354,6 +355,7 @@ async fn can_penalize_peer_for_invalid_signature() -> eyre::Result<()> { // Get handles let node0_rmn_handle = node0.inner.add_ons_handle.rollup_manager_handle.clone(); let node0_network_handle = node0_rmn_handle.get_network_handle().await.unwrap(); + let node0_l1_watcher_tx = node0.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let node0_id = node0_network_handle.inner().peer_id(); let node1_rnm_handle = node1.inner.add_ons_handle.rollup_manager_handle.clone(); @@ -363,10 +365,15 @@ async fn can_penalize_peer_for_invalid_signature() -> eyre::Result<()> { let mut node0_events = node0_rmn_handle.get_event_listener().await.unwrap(); let mut node1_events = node1_rnm_handle.get_event_listener().await.unwrap(); + // Set the L1 to synced on the sequencer node + node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + node0_events.next().await; + node0_events.next().await; + // === Phase 1: Test valid block with correct signature === // Have the legitimate sequencer build and sign a block - node0_rmn_handle.build_block().await; + node0_rmn_handle.build_block(); // Wait for the sequencer to build the block let block0 = @@ -480,7 +487,7 @@ async fn can_forward_tx_to_sequencer() { reth_tracing::init_test_tracing(); // create 2 nodes - let mut sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); + let sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); let mut follower_node_config = default_test_scroll_rollup_node_config(); // Create the chain spec for scroll mainnet with Euclid v2 activated and a test genesis. @@ -511,8 +518,15 @@ async fn can_forward_tx_to_sequencer() { .await .unwrap(); + // Send a notification to set the L1 to synced + let sequencer_l1_watcher_tx = + sequencer_node[0].inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + sequencer_events.next().await; + sequencer_events.next().await; + // have the sequencer build an empty block and gossip it to follower - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // wait for the sequencer to build a block with no transactions if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { @@ -536,7 +550,7 @@ async fn can_forward_tx_to_sequencer() { tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; // build block - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // wait for the sequencer to build a block with transactions wait_n_events( @@ -605,6 +619,8 @@ async fn can_sequence_and_gossip_transactions() { // generate rollup node manager event streams for each node let sequencer_rnm_handle = sequencer_node[0].inner.add_ons_handle.rollup_manager_handle.clone(); let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await.unwrap(); + let sequencer_l1_watcher_tx = + sequencer_node[0].inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let mut follower_events = follower_node[0] .inner .add_ons_handle @@ -613,8 +629,13 @@ async fn can_sequence_and_gossip_transactions() { .await .unwrap(); + // Send a notification to set the L1 to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + sequencer_events.next().await; + sequencer_events.next().await; + // have the sequencer build an empty block and gossip it to follower - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // wait for the sequencer to build a block with no transactions if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { @@ -638,7 +659,7 @@ async fn can_sequence_and_gossip_transactions() { tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; // build block - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // wait for the sequencer to build a block with transactions wait_n_events( @@ -730,9 +751,6 @@ async fn can_bridge_blocks() { bridge_node.network.add_peer(scroll_network_handle.local_node_record()).await; bridge_node.network.next_session_established().await; - let genesis_hash = bridge_node.inner.chain_spec().genesis_hash(); - println!("genesis hash: {genesis_hash:?}"); - // Create a standard NetworkManager to send blocks to the bridge node. let network_config = NetworkConfigBuilder::::with_rng_secret_key() .disable_discovery() @@ -792,6 +810,7 @@ async fn can_bridge_blocks() { /// Test that when the rollup node manager is shutdown, it consolidates the most recent batch /// on startup. +#[allow(clippy::large_stack_frames)] #[tokio::test] async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<()> { reth_tracing::init_test_tracing(); @@ -835,9 +854,7 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() tokio::select! { biased; - _ = shutdown => { - return; - }, + _ = shutdown => {}, _ = chain_orchestrator => {}, } }); @@ -973,9 +990,7 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() tokio::select! { biased; - _ = shutdown => { - return; - }, + _ = shutdown => {}, _ = chain_orchestrator => {}, } }); @@ -1000,7 +1015,7 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - break consolidation_outcome.block_info().clone(); + break *consolidation_outcome.block_info(); } }; @@ -1072,7 +1087,7 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( config.hydrate(node.inner.config.clone()).await?; let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (rnm, handle, _) = config + let (rnm, handle, l1_watcher_tx) = config .clone() .build( RollupNodeContext::new( @@ -1085,8 +1100,9 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( node.inner.add_ons_handle.rpc_handle.rpc_server_handles.clone(), ) .await?; - let (signal, shutdown) = shutdown_signal(); + let (_signal, shutdown) = shutdown_signal(); let mut rnm = Box::pin(rnm.run_until_shutdown(shutdown)); + let l1_watcher_tx: tokio::sync::mpsc::Sender> = l1_watcher_tx.unwrap(); // Poll the rnm until we get an event stream listener. let mut rnm_events_fut = pin!(handle.get_event_listener()); @@ -1100,10 +1116,22 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( tokio::time::sleep(Duration::from_millis(10)).await; }; + // Poll the rnm until we receive the consolidate event + l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + loop { + let _ = rnm.poll_unpin(&mut Context::from_waker(noop_waker_ref())); + if let Poll::Ready(Some(ChainOrchestratorEvent::ChainConsolidated { from: _, to: _ })) = + rnm_events.poll_next_unpin(&mut Context::from_waker(noop_waker_ref())) + { + break + } + tokio::time::sleep(Duration::from_millis(10)).await; + } + // Wait for the EN to be synced to block 10. let execution_node_provider = node.inner.provider; loop { - handle.build_block().await; + handle.build_block(); let block_number = loop { let _ = rnm.poll_unpin(&mut Context::from_waker(noop_waker_ref())); if let Poll::Ready(Some(ChainOrchestratorEvent::BlockSequenced(block))) = @@ -1281,10 +1309,13 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { let mut node1_rnm_events = node1_rnm_handle.get_event_listener().await?; let node1_l1_watcher_tx = node1.inner.add_ons_handle.l1_watcher_tx.as_ref().unwrap(); + // Set L1 synced on sequencer node + node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + // Let the sequencer build 10 blocks before performing the reorg process. let mut reorg_block = None; for i in 1..=10 { - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); let b = wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; tracing::info!(target: "scroll::test", block_number = ?b.header.number, block_hash = ?b.header.hash_slow(), "Sequenced block"); reorg_block = Some(b); @@ -1321,7 +1352,7 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Build block that contains the L1 message. let mut block11_before_reorg = None; - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_event_predicate_5s(&mut node0_rnm_events, |e| { if let ChainOrchestratorEvent::BlockSequenced(block) = e { if block.header.number == 11 && @@ -1338,7 +1369,7 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { .await?; for i in 12..=15 { - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; } @@ -1356,7 +1387,7 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Issue and wait for the reorg. node0_l1_watcher_tx.send(Arc::new(L1Notification::Reorg(9))).await?; - let reorg_block = reorg_block.as_ref().map(|b| Into::::into(b)); + let reorg_block = reorg_block.as_ref().map(Into::::into); wait_for_event_5s( &mut node0_rnm_events, ChainOrchestratorEvent::L1Reorg { @@ -1381,7 +1412,7 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { // Since the L1 reorg reverted the L1 message included in block 11, the sequencer // should produce a new block at height 11. - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_block_sequenced_5s(&mut node0_rnm_events, 11).await?; // Assert that the follower node has received the new block from the sequencer node. @@ -1491,6 +1522,7 @@ async fn test_custom_genesis_block_production_and_propagation() -> eyre::Result< // Get handles let node0_rnm_handle = node0.inner.add_ons_handle.rollup_manager_handle.clone(); let mut node0_rnm_events = node0_rnm_handle.get_event_listener().await?; + let node0_l1_watcher_tx = node0.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let node1_rnm_handle = node1.inner.add_ons_handle.rollup_manager_handle.clone(); let mut node1_rnm_events = node1_rnm_handle.get_event_listener().await?; @@ -1512,9 +1544,12 @@ async fn test_custom_genesis_block_production_and_propagation() -> eyre::Result< "Node1 should have the custom genesis hash" ); + // Set L1 synced on sequencer node + node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + // Let the sequencer build 10 blocks. for i in 1..=10 { - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); let b = wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; tracing::info!(target: "scroll::test", block_number = ?b.header.number, block_hash = ?b.header.hash_slow(), "Sequenced block"); } @@ -1551,10 +1586,14 @@ async fn can_rpc_enable_disable_sequencing() -> eyre::Result<()> { // Get handles let node0_rnm_handle = node0.inner.add_ons_handle.rollup_manager_handle.clone(); let mut node0_rnm_events = node0_rnm_handle.get_event_listener().await?; + let node0_l1_watcher_tx = node0.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let node1_rnm_handle = node1.inner.add_ons_handle.rollup_manager_handle.clone(); let mut node1_rnm_events = node1_rnm_handle.get_event_listener().await?; + // Set L1 synced + node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + // Create RPC client let client0 = node0.rpc_client().expect("RPC client should be available"); @@ -1581,7 +1620,7 @@ async fn can_rpc_enable_disable_sequencing() -> eyre::Result<()> { assert_eq!(block_num_after_wait, latest_block(&node1).await?.header.number); // Verify manual block building still works - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_block_sequenced_5s(&mut node0_rnm_events, block_num_after_wait + 1).await?; // Wait for the follower to import the block @@ -1648,13 +1687,11 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { // Let the sequencer build 10 blocks before performing the reorg process. for i in 1..=10 { - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); let b = wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; tracing::info!(target: "scroll::test", block_number = ?b.header.number, block_hash = ?b.header.hash_slow(), "Sequenced block") } - println!("Sequencer has built up to block 10"); - // Assert that the follower node has received all 10 blocks from the sequencer node. wait_for_block_imported_5s(&mut node1_rnm_events, 10).await?; @@ -1679,7 +1716,7 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { wait_for_event_5s(&mut node0_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; // Build block that contains the L1 message. - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_event_predicate_5s(&mut node0_rnm_events, |e| { if let ChainOrchestratorEvent::BlockSequenced(block) = e { if block.header.number == 11 && @@ -1695,12 +1732,10 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { .await?; for i in 12..=15 { - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_block_sequenced_5s(&mut node0_rnm_events, i).await?; } - println!("Sequencer has built up to block 15 with the L1 message included in block 11"); - wait_for_event_5s( &mut node1_rnm_events, ChainOrchestratorEvent::L1MessageNotFoundInDatabase(L1MessageKey::TransactionHash(b256!( @@ -1719,10 +1754,8 @@ async fn can_reject_l2_block_with_unknown_l1_message() -> eyre::Result<()> { wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::L1MessageCommitted(0)).await?; wait_for_event_5s(&mut node1_rnm_events, ChainOrchestratorEvent::NewL1Block(10)).await?; - println!("Follower has received the L1 message"); - // Produce another block and send to follower node. - node0_rnm_handle.build_block().await; + node0_rnm_handle.build_block(); wait_for_block_sequenced_5s(&mut node0_rnm_events, 16).await?; // Assert that the follower node has received the latest block from the sequencer node and @@ -1755,8 +1788,19 @@ async fn can_gossip_over_eth_wire() -> eyre::Result<()> { // Setup the rollup node manager. let (mut nodes, _tasks, _) = setup_engine(config, 2, chain_spec.clone(), false, false).await.unwrap(); - let _sequencer = nodes.pop().unwrap(); let follower = nodes.pop().unwrap(); + let sequencer = nodes.pop().unwrap(); + + // Set the L1 synced on the sequencer node to start block production. + let mut sequencer_events = + sequencer.inner.add_ons_handle.rollup_manager_handle.get_event_listener().await.unwrap(); + let sequencer_l1_notification_tx = + sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Set the L1 synced on the sequencer node to start block production. + sequencer_l1_notification_tx.send(Arc::new(L1Notification::Synced)).await?; + sequencer_events.next().await; + sequencer_events.next().await; let mut eth_wire_blocks = follower.inner.network.eth_wire_block_listener().await?; @@ -1818,6 +1862,10 @@ async fn signer_rotation() -> eyre::Result<()> { let sequencer_2_l1_notification_tx = sequencer_2.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + // Set the L1 synced on both nodes to start block production. + sequencer_1_l1_notification_tx.send(Arc::new(L1Notification::Synced)).await?; + sequencer_2_l1_notification_tx.send(Arc::new(L1Notification::Synced)).await?; + // Create a follower event stream. let mut follower_events = follower.inner.add_ons_handle.rollup_manager_handle.get_event_listener().await.unwrap(); @@ -1983,44 +2031,6 @@ async fn wait_for_block_imported_5s( wait_for_chain_extended(events, block_number, Duration::from_secs(5)).await } -// async fn wait_for_chain_committed_5s( -// events: &mut EventStream, -// expected_block_number: u64, -// expected_consolidated: bool, -// ) -> eyre::Result<()> { -// wait_for_chain_committed( -// events, -// expected_block_number, -// expected_consolidated, -// Duration::from_secs(5), -// ) -// .await -// } - -// async fn wait_for_chain_committed( -// events: &mut EventStream, -// expected_block_number: u64, -// expected_consolidated: bool, -// timeout: Duration, -// ) -> eyre::Result<()> { -// wait_for_event_predicate( -// events, -// |e| { -// if let RollupManagerEvent::ChainOrchestratorEvent( -// ChainOrchestratorEvent::L2ChainCommitted(block_info, _, consolidated), -// ) = e -// { -// return block_info.block_info.number == expected_block_number && -// expected_consolidated == consolidated; -// } - -// false -// }, -// timeout, -// ) -// .await -// } - async fn wait_for_event_predicate( event_stream: &mut EventStream, mut predicate: impl FnMut(ChainOrchestratorEvent) -> bool, diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 05420724..3939128a 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -4,8 +4,6 @@ use alloy_primitives::{b256, Address, U256}; use alloy_provider::{Provider, ProviderBuilder}; use futures::StreamExt; use reqwest::Url; -use reth_network::{NetworkEvent, NetworkEventListenerProvider}; -use reth_network_api::{events::PeerEvent, test_utils::PeersHandleProvider}; use reth_provider::{BlockIdReader, BlockReader}; use reth_scroll_chainspec::{SCROLL_DEV, SCROLL_SEPOLIA}; use reth_tokio_util::EventStream; @@ -115,12 +113,16 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result< .unwrap(); let mut synced = nodes.pop().unwrap(); let mut synced_events = synced.inner.rollup_manager_handle.get_event_listener().await?; + let synced_l1_watcher_tx = synced.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let (mut nodes, _tasks, _) = setup_engine(node_config.clone(), 1, chain_spec, false, false).await.unwrap(); let mut unsynced = nodes.pop().unwrap(); let mut unsynced_events = unsynced.inner.rollup_manager_handle.get_event_listener().await?; + // Set the L1 to synced on the synced node to start block production. + synced_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + // Wait for the chain to be advanced by the sequencer. let optimistic_sync_trigger = node_config.chain_orchestrator_args.optimistic_sync_trigger + 1; wait_n_events( @@ -219,6 +221,9 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { let mut follower_events = follower.inner.add_ons_handle.rollup_manager_handle.get_event_listener().await?; + // Send a notification to the sequencer node that the L1 watcher is synced. + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + // Create a sequence of L1 messages to be added to the sequencer node. const L1_MESSAGES_COUNT: usize = 200; let mut l1_messages = Vec::with_capacity(L1_MESSAGES_COUNT); @@ -234,8 +239,6 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { l1_messages.push(l1_message); } - println!("Im here"); - // Add the L1 messages to the sequencer node. for (i, l1_message) in l1_messages.iter().enumerate() { sequencer_l1_watcher_tx @@ -264,7 +267,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { 1, ) .await; - sequencer_handle.build_block().await; + sequencer_handle.build_block(); wait_n_events( &mut sequencer_events, |e: ChainOrchestratorEvent| matches!(e, ChainOrchestratorEvent::BlockSequenced(_)), @@ -273,17 +276,13 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { .await; } - println!("Im here 2"); - // Connect the nodes together. sequencer.network.add_peer(follower.network.record()).await; follower.network.next_session_established().await; sequencer.network.next_session_established().await; - println!("Im here 3"); - // trigger a new block on the sequencer node. - sequencer_handle.build_block().await; + sequencer_handle.build_block(); // Assert that the unsynced node triggers optimistic sync. wait_n_events( @@ -293,8 +292,6 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { ) .await; - println!("Im here 4"); - // Let the unsynced node process the optimistic sync. tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; @@ -321,23 +318,17 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { .await; } - println!("Im here 4.1"); - // Send a notification to the unsynced node that the L1 watcher is synced. follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); // Wait for the unsynced node to sync to the L1 watcher. wait_n_events(&mut follower_events, |e| matches!(e, ChainOrchestratorEvent::L1Synced), 1).await; - println!("Im here 5"); - // Let the unsynced node process the L1 messages. tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; // build a new block on the sequencer node to trigger consolidation on the unsynced node. - sequencer_handle.build_block().await; - - println!("Im here 6"); + sequencer_handle.build_block(); // Assert that the unsynced node consolidates the chain. wait_n_events( @@ -347,8 +338,6 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { ) .await; - println!("Im here 7"); - // Now push a L1 message to the sequencer node and build a new block. sequencer_l1_watcher_tx .send(Arc::new(L1Notification::L1Message { @@ -374,7 +363,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(201))).await.unwrap(); wait_n_events(&mut sequencer_events, |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), 1) .await; - sequencer_handle.build_block().await; + sequencer_handle.build_block(); wait_n_events( &mut follower_events, @@ -395,320 +384,495 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { Ok(()) } -// #[allow(clippy::large_stack_frames)] -// #[tokio::test] -// async fn test_consolidation() -> eyre::Result<()> { -// reth_tracing::init_test_tracing(); -// let node_config = default_test_scroll_rollup_node_config(); -// let sequencer_node_config = ScrollRollupNodeConfig { -// test: true, -// network_args: RollupNodeNetworkArgs { -// enable_eth_scroll_wire_bridge: true, -// enable_scroll_wire: true, -// sequencer_url: None, -// signer_address: None, -// }, -// database_args: RollupNodeDatabaseArgs { -// rn_db_path: Some(PathBuf::from("sqlite::memory:")), -// }, -// l1_provider_args: L1ProviderArgs::default(), -// engine_driver_args: EngineDriverArgs::default(), -// chain_orchestrator_args: ChainOrchestratorArgs::default(), -// sequencer_args: SequencerArgs { -// sequencer_enabled: true, -// auto_start: true, -// block_time: 0, -// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), -// allow_empty_blocks: true, -// ..SequencerArgs::default() -// }, -// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, -// signer_args: Default::default(), -// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), -// consensus_args: ConsensusArgs::noop(), -// database: None, -// rpc_args: RpcArgs::default(), -// }; - -// // Create the chain spec for scroll dev with Feynman activated and a test genesis. -// let chain_spec = (*SCROLL_DEV).clone(); - -// // Create a sequencer node and an unsynced node. -// let (mut nodes, _tasks, _) = -// setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); -// let mut sequencer = nodes.pop().unwrap(); -// let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); -// let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); -// let mut sequencer_events = sequencer_handle.get_event_listener().await?; - -// let (mut nodes, _tasks, _) = -// setup_engine(node_config.clone(), 1, chain_spec, false, false).await.unwrap(); -// let mut follower = nodes.pop().unwrap(); -// let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); -// let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; - -// // Connect the nodes together. -// sequencer.network.add_peer(follower.network.record()).await; -// follower.network.next_session_established().await; -// sequencer.network.next_session_established().await; - -// // Create a L1 message and send it to both nodes. -// let l1_message = TxL1Message { -// queue_index: 0, -// gas_limit: 21000, -// sender: Address::random(), -// to: Address::random(), -// value: U256::from(1), -// input: Default::default(), -// }; -// sequencer_l1_watcher_tx -// .send(Arc::new(L1Notification::L1Message { -// message: l1_message.clone(), -// block_number: 0, -// block_timestamp: 0, -// })) -// .await -// .unwrap(); -// wait_n_events( -// &mut sequencer_events, -// |e| { -// matches!( -// e, -// RollupManagerEvent::ChainOrchestratorEvent( -// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) -// ) -// ) -// }, -// 1, -// ) -// .await; -// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(2))).await.unwrap(); - -// follower_l1_watcher_tx -// .send(Arc::new(L1Notification::L1Message { -// message: l1_message, -// block_number: 0, -// block_timestamp: 0, -// })) -// .await -// .unwrap(); -// wait_n_events( -// &mut follower_events, -// |e| { -// matches!( -// e, -// RollupManagerEvent::ChainOrchestratorEvent( -// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) -// ) -// ) -// }, -// 1, -// ) -// .await; - -// // Send a notification to both nodes that the L1 watcher is synced. -// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); -// follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); - -// // Build a new block on the sequencer node. -// sequencer_handle.build_block().await; - -// // Assert that the unsynced node consolidates the chain. -// wait_n_events( -// &mut follower_events, -// |e| { -// matches!( -// e, -// RollupManagerEvent::ChainOrchestratorEvent( -// ChainOrchestratorEvent::L2ChainCommitted(_, _, true) -// ) -// ) -// }, -// 1, -// ) -// .await; - -// // Now push a L1 message to the sequencer node and build a new block. -// sequencer_l1_watcher_tx -// .send(Arc::new(L1Notification::L1Message { -// message: TxL1Message { -// queue_index: 1, -// gas_limit: 21000, -// sender: Address::random(), -// to: Address::random(), -// value: U256::from(1), -// input: Default::default(), -// }, -// block_number: 1, -// block_timestamp: 10, -// })) -// .await -// .unwrap(); -// wait_n_events( -// &mut sequencer_events, -// |e| { -// matches!( -// e, -// RollupManagerEvent::ChainOrchestratorEvent( -// rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(_) -// ) -// ) -// }, -// 1, -// ) -// .await; -// sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(5))).await.unwrap(); -// sequencer_handle.build_block().await; - -// // Assert that the follower node rejects the new block as it hasn't received the L1 message. -// wait_n_events( -// &mut follower_events, -// |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { key: _ }), -// 1, -// ) -// .await; - -// Ok(()) -// } - -// #[allow(clippy::large_stack_frames)] -// #[tokio::test] -// async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { -// reth_tracing::init_test_tracing(); -// let node_config = default_test_scroll_rollup_node_config(); -// let sequencer_node_config = ScrollRollupNodeConfig { -// test: true, -// network_args: RollupNodeNetworkArgs { -// enable_eth_scroll_wire_bridge: false, -// enable_scroll_wire: true, -// ..Default::default() -// }, -// database_args: RollupNodeDatabaseArgs { -// rn_db_path: Some(PathBuf::from("sqlite::memory:")), -// }, -// l1_provider_args: L1ProviderArgs::default(), -// engine_driver_args: EngineDriverArgs::default(), -// chain_orchestrator_args: ChainOrchestratorArgs::default(), -// sequencer_args: SequencerArgs { -// sequencer_enabled: true, -// auto_start: true, -// block_time: 0, -// l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), -// allow_empty_blocks: true, -// ..SequencerArgs::default() -// }, -// blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, -// signer_args: Default::default(), -// gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), -// consensus_args: ConsensusArgs::noop(), -// database: None, -// rpc_args: RpcArgs::default(), -// }; - -// // Create the chain spec for scroll dev with Feynman activated and a test genesis. -// let chain_spec = (*SCROLL_DEV).clone(); - -// // Create a sequencer node and an unsynced node. -// let (mut nodes, _tasks, _) = -// setup_engine(sequencer_node_config.clone(), 1, chain_spec.clone(), false, false) -// .await -// .unwrap(); -// let mut sequencer = nodes.pop().unwrap(); -// let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); -// let mut sequencer_events = sequencer_handle.get_event_listener().await?; - -// let (mut nodes, _tasks, _) = -// setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); -// let mut follower = nodes.pop().unwrap(); -// let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; - -// // Connect the nodes together. -// sequencer.connect(&mut follower).await; - -// // initially the sequencer should build 100 empty blocks and the follower should follow them -// let mut reorg_block_info = BlockInfo::default(); -// for i in 0..100 { -// sequencer_handle.build_block().await; -// wait_n_events( -// &mut sequencer_events, -// |e| { -// if let RollupManagerEvent::BlockSequenced(block) = e { -// if i == 95 { -// reorg_block_info = (&block).into(); -// } -// true -// } else { -// false -// } -// }, -// 1, -// ) -// .await; -// wait_n_events( -// &mut follower_events, -// |e| { -// matches!( -// e, -// RollupManagerEvent::ChainOrchestratorEvent( -// ChainOrchestratorEvent::L2ChainCommitted(_, _, _) -// ) -// ) -// }, -// 1, -// ) -// .await; -// } - -// // disconnect the two nodes -// let mut sequencer_network_events = sequencer.inner.network.event_listener(); -// let mut follower_network_events = follower.inner.network.event_listener(); -// sequencer.inner.network.peers_handle().remove_peer(follower.network.record().id); -// while let Some(ev) = sequencer_network_events.next().await { -// if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { -// break -// } -// } -// while let Some(ev) = sequencer_network_events.next().await { -// if let NetworkEvent::Peer(PeerEvent::PeerRemoved(_)) = ev { -// break -// } -// } -// while let Some(ev) = follower_network_events.next().await { -// if let NetworkEvent::Peer(PeerEvent::SessionClosed { peer_id: _, reason: _ }) = ev { -// break -// } -// } - -// sequencer_handle.update_fcs_head(reorg_block_info).await; - -// // Have the sequencer build 2 new blocks, one containing the L1 message. -// sequencer_handle.build_block().await; -// wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), -// 1) .await; sequencer_handle.build_block().await; wait_n_events(&mut sequencer_events, |e| -// matches!(e, RollupManagerEvent::BlockSequenced(_)), -// 1) .await; - -// // connect the two nodes again -// follower.connect(&mut sequencer).await; - -// // now build a final block -// sequencer_handle.build_block().await; - -// // Wait for the follower node to reorg to the new chain. -// wait_n_events( -// &mut follower_events, -// |e| { -// matches!( -// e, -// -// RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::ChainReorged(_)) ) -// }, -// 1, -// ) -// .await; - -// Ok(()) -// } +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_consolidation() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + let node_config = default_test_scroll_rollup_node_config(); + let sequencer_node_config = ScrollRollupNodeConfig { + test: true, + network_args: RollupNodeNetworkArgs { + enable_eth_scroll_wire_bridge: true, + enable_scroll_wire: true, + sequencer_url: None, + signer_address: None, + }, + database_args: RollupNodeDatabaseArgs { + rn_db_path: Some(PathBuf::from("sqlite::memory:")), + }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), + sequencer_args: SequencerArgs { + sequencer_enabled: true, + auto_start: false, + block_time: 10, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + allow_empty_blocks: true, + ..SequencerArgs::default() + }, + blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, + signer_args: Default::default(), + gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + database: None, + rpc_args: RpcArgs::default(), + }; + + // Create the chain spec for scroll dev with Feynman activated and a test genesis. + let chain_spec = (*SCROLL_DEV).clone(); + + // Create a sequencer node and an unsynced node. + let (mut nodes, _tasks, _) = + setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); + let mut sequencer = nodes.pop().unwrap(); + let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); + let mut sequencer_events = sequencer_handle.get_event_listener().await?; + + let (mut nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec, false, false).await.unwrap(); + let mut follower = nodes.pop().unwrap(); + let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; + + // Connect the nodes together. + sequencer.network.add_peer(follower.network.record()).await; + follower.network.next_session_established().await; + sequencer.network.next_session_established().await; + + // Create a L1 message and send it to both nodes. + let l1_message = TxL1Message { + queue_index: 0, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }; + sequencer_l1_watcher_tx + .send(Arc::new(L1Notification::L1Message { + message: l1_message.clone(), + block_number: 0, + block_timestamp: 0, + })) + .await + .unwrap(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::L1MessageCommitted(_)), + 1, + ) + .await; + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(2))).await.unwrap(); + + follower_l1_watcher_tx + .send(Arc::new(L1Notification::L1Message { + message: l1_message, + block_number: 0, + block_timestamp: 0, + })) + .await + .unwrap(); + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::L1MessageCommitted(_)), + 1, + ) + .await; + + // Send a notification to both nodes that the L1 watcher is synced. + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + + // Assert that the unsynced node consolidates the chain. + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainConsolidated { from: 0, to: 0 }), + 1, + ) + .await; + + // Build a new block on the sequencer node. + sequencer_handle.build_block(); + + // Now push a L1 message to the sequencer node and build a new block. + sequencer_l1_watcher_tx + .send(Arc::new(L1Notification::L1Message { + message: TxL1Message { + queue_index: 1, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }, + block_number: 1, + block_timestamp: 10, + })) + .await + .unwrap(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::L1MessageCommitted(_)), + 1, + ) + .await; + + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(5))).await.unwrap(); + wait_n_events(&mut sequencer_events, |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), 1) + .await; + sequencer_handle.build_block(); + + // Assert that the follower node rejects the new block as it hasn't received the L1 message. + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::L1MessageNotFoundInDatabase(_)), + 1, + ) + .await; + + Ok(()) +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_chain_orchestrator_reorg_with_gap_above_head() -> eyre::Result<()> { + test_chain_orchestrator_fork_choice(100, Some(95), 20, |e| { + matches!(e, ChainOrchestratorEvent::ChainReorged(_)) + }) + .await +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_chain_orchestrator_reorg_with_gap_below_head() -> eyre::Result<()> { + test_chain_orchestrator_fork_choice(100, Some(50), 20, |e| { + matches!(e, ChainOrchestratorEvent::ChainReorged(_)) + }) + .await +} + +#[allow(clippy::large_stack_frames)] +async fn test_chain_orchestrator_fork_choice( + initial_blocks: usize, + reorg_block_number: Option, + additional_blocks: usize, + expected_final_event_predicate: impl FnMut(ChainOrchestratorEvent) -> bool, +) -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + let node_config = default_test_scroll_rollup_node_config(); + let sequencer_node_config = ScrollRollupNodeConfig { + test: true, + network_args: RollupNodeNetworkArgs { + enable_eth_scroll_wire_bridge: false, + enable_scroll_wire: true, + ..Default::default() + }, + database_args: RollupNodeDatabaseArgs { + rn_db_path: Some(PathBuf::from("sqlite::memory:")), + }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), + sequencer_args: SequencerArgs { + sequencer_enabled: true, + auto_start: false, + block_time: 10, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + allow_empty_blocks: true, + ..SequencerArgs::default() + }, + blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, + signer_args: Default::default(), + gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + database: None, + rpc_args: RpcArgs::default(), + }; + + // Create the chain spec for scroll dev with Feynman activated and a test genesis. + let chain_spec = (*SCROLL_DEV).clone(); + + // Create a sequencer node and an unsynced node. + let (mut nodes, _tasks, _) = + setup_engine(sequencer_node_config.clone(), 1, chain_spec.clone(), false, false) + .await + .unwrap(); + let mut sequencer = nodes.pop().unwrap(); + let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); + let mut sequencer_events = sequencer_handle.get_event_listener().await?; + let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + let (mut nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); + let mut follower = nodes.pop().unwrap(); + let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; + let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Connect the nodes together. + sequencer.connect(&mut follower).await; + + // set both the sequencer and follower L1 watchers to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + + // Initially the sequencer should build 100 empty blocks in each and the follower + // should follow them + let mut reorg_block_info = BlockInfo::default(); + for i in 0..initial_blocks { + sequencer_handle.build_block(); + wait_n_events( + &mut sequencer_events, + |e| { + if let ChainOrchestratorEvent::BlockSequenced(block) = e { + if Some(i) == reorg_block_number { + reorg_block_info = (&block).into(); + } + true + } else { + false + } + }, + 1, + ) + .await; + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), + 1, + ) + .await; + } + + // Now reorg the sequencer and disable gossip so we can create fork + sequencer_handle.set_gossip(false).await.unwrap(); + sequencer_handle.update_fcs_head(reorg_block_info).await.unwrap(); + + // wait two seconds to ensure the timestamp of the new blocks is greater than the old ones + tokio::time::sleep(tokio::time::Duration::from_secs(2)).await; + + // Have the sequencer build 20 new blocks, containing new L1 messages. + for _ in 0..additional_blocks { + sequencer_handle.build_block(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::BlockSequenced(_block)), + 1, + ) + .await; + } + + // now build a final block + sequencer_handle.set_gossip(true).await.unwrap(); + sequencer_handle.build_block(); + + // Wait for the follower node to accept the new chain + wait_n_events(&mut follower_events, expected_final_event_predicate, 1).await; + + Ok(()) +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_chain_orchestrator_l1_reorg() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + let node_config = default_test_scroll_rollup_node_config(); + let sequencer_node_config = ScrollRollupNodeConfig { + test: true, + network_args: RollupNodeNetworkArgs { + enable_eth_scroll_wire_bridge: false, + enable_scroll_wire: true, + ..Default::default() + }, + database_args: RollupNodeDatabaseArgs { + rn_db_path: Some(PathBuf::from("sqlite::memory:")), + }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), + sequencer_args: SequencerArgs { + sequencer_enabled: true, + auto_start: false, + block_time: 10, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + allow_empty_blocks: true, + ..SequencerArgs::default() + }, + blob_provider_args: BlobProviderArgs { mock: true, ..Default::default() }, + signer_args: Default::default(), + gas_price_oracle_args: RollupNodeGasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + database: None, + rpc_args: RpcArgs::default(), + }; + + // Create the chain spec for scroll dev with Feynman activated and a test genesis. + let chain_spec = (*SCROLL_DEV).clone(); + + // Create a sequencer node and an unsynced node. + let (mut nodes, _tasks, _) = + setup_engine(sequencer_node_config.clone(), 1, chain_spec.clone(), false, false) + .await + .unwrap(); + let mut sequencer = nodes.pop().unwrap(); + let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); + let mut sequencer_events = sequencer_handle.get_event_listener().await?; + let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + let (mut nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); + let mut follower = nodes.pop().unwrap(); + let mut follower_events = follower.inner.rollup_manager_handle.get_event_listener().await?; + let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Connect the nodes together. + sequencer.connect(&mut follower).await; + + // set both the sequencer and follower L1 watchers to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + follower_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + + // Initially the sequencer should build 100 blocks with 1 message in each and the follower + // should follow them + for i in 0..100 { + let l1_message = Arc::new(L1Notification::L1Message { + message: TxL1Message { + queue_index: i, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }, + block_number: i, + block_timestamp: i * 10, + }); + let new_block = Arc::new(L1Notification::NewBlock(i)); + sequencer_l1_watcher_tx.send(l1_message.clone()).await.unwrap(); + sequencer_l1_watcher_tx.send(new_block.clone()).await.unwrap(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), + 1, + ) + .await; + follower_l1_watcher_tx.send(l1_message).await.unwrap(); + follower_l1_watcher_tx.send(new_block).await.unwrap(); + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), + 1, + ) + .await; + + sequencer_handle.build_block(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::BlockSequenced(_)), + 1, + ) + .await; + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), + 1, + ) + .await; + } + + // send a reorg notification to the sequencer + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Reorg(50))).await.unwrap(); + wait_n_events( + &mut sequencer_events, + |e| { + matches!( + e, + ChainOrchestratorEvent::L1Reorg { + l1_block_number: 50, + queue_index: Some(51), + l2_head_block_info: _, + l2_safe_block_info: _ + } + ) + }, + 1, + ) + .await; + + sequencer_handle.set_gossip(false).await.unwrap(); + + // Have the sequencer build 20 new blocks, containing new L1 messages. + let mut l1_notifications = vec![]; + for i in 0..20 { + let l1_message = Arc::new(L1Notification::L1Message { + message: TxL1Message { + queue_index: 51 + i, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }, + block_number: 51 + i, + block_timestamp: (51 + i) * 10, + }); + let new_block = Arc::new(L1Notification::NewBlock(51 + i)); + l1_notifications.extend([l1_message.clone(), new_block.clone()]); + sequencer_l1_watcher_tx.send(l1_message.clone()).await.unwrap(); + sequencer_l1_watcher_tx.send(new_block.clone()).await.unwrap(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), + 1, + ) + .await; + + sequencer_handle.build_block(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, ChainOrchestratorEvent::BlockSequenced(_)), + 1, + ) + .await; + } + + // wait two seconds to ensure the timestamp of the new blocks is greater than the old ones + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + + // now build a final block + sequencer_handle.set_gossip(true).await.unwrap(); + sequencer_handle.build_block(); + + // The follower node should reject the new block as it has a different view of L1 data. + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::L1MessageMismatch { .. }), + 1, + ) + .await; + + // Now update the follower node with the new L1 data + follower_l1_watcher_tx.send(Arc::new(L1Notification::Reorg(50))).await.unwrap(); + for notification in l1_notifications { + follower_l1_watcher_tx.send(notification).await.unwrap(); + } + wait_n_events(&mut follower_events, |e| matches!(e, ChainOrchestratorEvent::NewL1Block(_)), 20) + .await; + + // Now build a new block on the sequencer to trigger the reorg on the follower + sequencer_handle.build_block(); + + // Wait for the follower node to accept the new chain + wait_n_events( + &mut follower_events, + |e| matches!(e, ChainOrchestratorEvent::ChainExtended(_)), + 1, + ) + .await; + + Ok(()) +} /// Waits for n events to be emitted. async fn wait_n_events( @@ -717,7 +881,7 @@ async fn wait_n_events( mut n: u64, ) { while let Some(event) = events.next().await { - if matches(event) { + if matches(event.clone()) { n -= 1; } if n == 0 { diff --git a/crates/primitives/src/batch.rs b/crates/primitives/src/batch.rs index 6d5192b3..8a5ad282 100644 --- a/crates/primitives/src/batch.rs +++ b/crates/primitives/src/batch.rs @@ -64,7 +64,7 @@ pub struct BatchConsolidationOutcome { impl BatchConsolidationOutcome { /// Creates a new empty batch consolidation outcome for the given batch info. - pub fn new(batch_info: BatchInfo) -> Self { + pub const fn new(batch_info: BatchInfo) -> Self { Self { batch_info, blocks: Vec::new() } } @@ -87,11 +87,10 @@ pub enum BlockConsolidationOutcome { impl BlockConsolidationOutcome { /// Returns the block info for the consolidated block. - pub fn block_info(&self) -> &BlockInfo { + pub const fn block_info(&self) -> &BlockInfo { match self { - BlockConsolidationOutcome::Consolidated(info) => info, - BlockConsolidationOutcome::Skipped(info) => info, - BlockConsolidationOutcome::Reorged(info) => &info.block_info, + Self::Consolidated(info) | Self::Skipped(info) => info, + Self::Reorged(info) => &info.block_info, } } } diff --git a/crates/sequencer/Cargo.toml b/crates/sequencer/Cargo.toml index 5903959c..7a838b04 100644 --- a/crates/sequencer/Cargo.toml +++ b/crates/sequencer/Cargo.toml @@ -45,7 +45,6 @@ eyre.workspace = true alloy-consensus.workspace = true alloy-primitives.workspace = true -rollup-node-manager.workspace = true rollup-node-signer.workspace = true # scroll-alloy diff --git a/crates/sequencer/src/config.rs b/crates/sequencer/src/config.rs index b977ab79..942b803c 100644 --- a/crates/sequencer/src/config.rs +++ b/crates/sequencer/src/config.rs @@ -70,11 +70,9 @@ impl fmt::Display for L1MessageInclusionMode { impl From for L1MessageKey { fn from(mode: L1MessageInclusionMode) -> Self { match mode { - L1MessageInclusionMode::Finalized => { - L1MessageKey::NotIncluded(NotIncludedStart::Finalized) - } + L1MessageInclusionMode::Finalized => Self::NotIncluded(NotIncludedStart::Finalized), L1MessageInclusionMode::BlockDepth(depth) => { - L1MessageKey::NotIncluded(NotIncludedStart::BlockDepth(depth)) + Self::NotIncluded(NotIncludedStart::BlockDepth(depth)) } } } diff --git a/crates/sequencer/src/lib.rs b/crates/sequencer/src/lib.rs index aa28d2ac..eab6f7f1 100644 --- a/crates/sequencer/src/lib.rs +++ b/crates/sequencer/src/lib.rs @@ -72,7 +72,7 @@ where } /// Returns a reference to the payload building job. - pub fn payload_building_job(&self) -> Option<&PayloadBuildingJob> { + pub const fn payload_building_job(&self) -> Option<&PayloadBuildingJob> { self.payload_building_job.as_ref() } @@ -91,6 +91,7 @@ where /// Disables the sequencer. pub fn disable(&mut self) { self.trigger = None; + self.cancel_payload_building_job(); } /// Creates a new block using the pending transactions from the message queue and @@ -222,7 +223,7 @@ impl std::fmt::Debug for PayloadBuildingJob { impl PayloadBuildingJob { /// Returns the L1 origin block number of the first included L1 message, if any. - pub fn l1_origin(&self) -> Option { + pub const fn l1_origin(&self) -> Option { self.l1_origin } } @@ -242,9 +243,8 @@ impl Stream for Sequencer { // If there's no inflight job, emit a new slot event. if this.payload_building_job.is_none() { return Poll::Ready(Some(SequencerEvent::NewSlot)); - } else { - tracing::trace!(target: "rollup_node::sequencer", "Payload building job already in progress, skipping slot."); }; + tracing::trace!(target: "rollup_node::sequencer", "Payload building job already in progress, skipping slot."); } Poll::Pending => {} } diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 3974d701..1dcb2db5 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -2,7 +2,6 @@ use alloy_consensus::BlockHeader; use alloy_primitives::{hex, Address, U256}; -use alloy_rpc_types_engine::PayloadAttributes; use futures::stream::StreamExt; use reth_e2e_test_utils::transaction::TransactionTestContext; use reth_node_core::primitives::SignedTransaction; @@ -17,23 +16,15 @@ use rollup_node::{ }; use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_primitives::{sig_encode_hash, BlockInfo, L1MessageEnvelope}; -use rollup_node_providers::ScrollRootProvider; use rollup_node_sequencer::{ L1MessageInclusionMode, PayloadBuildingConfig, Sequencer, SequencerConfig, SequencerEvent, }; -use rollup_node_signer::SignerEvent; use rollup_node_watcher::L1Notification; use scroll_alloy_consensus::TxL1Message; use scroll_alloy_provider::ScrollAuthApiEngineClient; -use scroll_alloy_rpc_types_engine::{BlockDataHint, ScrollPayloadAttributes}; use scroll_db::{test_utils::setup_test_db, DatabaseTransactionProvider, DatabaseWriteOperations}; -use scroll_engine::{Engine, EngineDriver, EngineDriverEvent, ForkchoiceState}; -use std::{ - io::Write, - path::PathBuf, - sync::Arc, - time::{SystemTime, UNIX_EPOCH}, -}; +use scroll_engine::{Engine, ForkchoiceState}; +use std::{io::Write, path::PathBuf, sync::Arc}; use tempfile::NamedTempFile; use tokio::{ sync::Mutex, @@ -44,8 +35,6 @@ use tokio::{ async fn skip_block_with_no_transactions() { reth_tracing::init_test_tracing(); - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - // setup a test node let (mut nodes, _tasks, _wallet) = setup(1, false).await.unwrap(); let node = nodes.pop().unwrap(); @@ -74,7 +63,7 @@ async fn skip_block_with_no_transactions() { // create a sequencer let config = SequencerConfig { - chain_spec: node.inner.chain_spec().clone(), + chain_spec: node.inner.chain_spec(), fee_recipient: Address::random(), auto_start: false, payload_building_config: PayloadBuildingConfig { @@ -102,8 +91,6 @@ async fn skip_block_with_no_transactions() { async fn can_build_blocks() { reth_tracing::init_test_tracing(); - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - // setup a test node let (mut nodes, _tasks, wallet) = setup(1, false).await.unwrap(); let node = nodes.pop().unwrap(); @@ -133,7 +120,7 @@ async fn can_build_blocks() { // create a sequencer let config = SequencerConfig { - chain_spec: node.inner.chain_spec().clone(), + chain_spec: node.inner.chain_spec(), fee_recipient: Address::random(), auto_start: false, payload_building_config: PayloadBuildingConfig { @@ -263,7 +250,7 @@ async fn can_build_blocks_with_delayed_l1_messages() { // create a sequencer let config = SequencerConfig { - chain_spec: node.inner.chain_spec().clone(), + chain_spec: node.inner.chain_spec(), fee_recipient: Address::random(), auto_start: false, payload_building_config: PayloadBuildingConfig { @@ -392,7 +379,7 @@ async fn can_build_blocks_with_finalized_l1_messages() { // create a sequencer let config = SequencerConfig { - chain_spec: node.inner.chain_spec().clone(), + chain_spec: node.inner.chain_spec(), fee_recipient: Address::random(), auto_start: false, payload_building_config: PayloadBuildingConfig { @@ -552,6 +539,14 @@ async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + let sequencer_l1_watcher_tx = nodes[0].inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Send a notification to set the L1 to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + + // skip the L1 synced event and consolidated events + sequencer_events.next().await; + sequencer_events.next().await; // Generate and inject transaction let mut wallet_lock = wallet.lock().await; @@ -566,7 +561,7 @@ async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; // Build block - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // Verify block was successfully sequenced if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { @@ -644,6 +639,14 @@ async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result< let sequencer_rnm_handle = nodes[0].inner.add_ons_handle.rollup_manager_handle.clone(); let mut sequencer_events = sequencer_rnm_handle.get_event_listener().await?; + let sequencer_l1_watcher_tx = nodes[0].inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Send a notification to set the L1 to synced + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + + // skip the L1 synced event and consolidated events + sequencer_events.next().await; + sequencer_events.next().await; // Generate and inject transaction let mut wallet_lock = wallet.lock().await; @@ -658,7 +661,7 @@ async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result< let tx_hash = nodes[0].rpc.inject_tx(raw_tx).await?; // Build block - sequencer_rnm_handle.build_block().await; + sequencer_rnm_handle.build_block(); // Verify block was successfully sequenced if let Some(ChainOrchestratorEvent::BlockSequenced(block)) = sequencer_events.next().await { @@ -687,7 +690,6 @@ async fn can_build_blocks_and_exit_at_gas_limit() { let chain_spec = SCROLL_DEV.clone(); const MIN_TRANSACTION_GAS_COST: u64 = 21_000; - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); const TRANSACTIONS_COUNT: usize = 2000; // setup a test node. use a high value for the payload building duration to be sure we don't @@ -732,39 +734,39 @@ async fn can_build_blocks_and_exit_at_gas_limit() { // create the engine driver connected to the node let auth_client = node.inner.engine_http_client(); let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); + let mut engine = Engine::new(Arc::new(engine_client), fcs); - // issue a new payload to the execution layer. - let timestamp = - SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); - engine_driver.handle_build_new_payload(ScrollPayloadAttributes { - payload_attributes: PayloadAttributes { - timestamp, - prev_randao: Default::default(), - suggested_fee_recipient: Default::default(), - withdrawals: None, - parent_beacon_block_root: None, + // create a test database + let database = Arc::new(setup_test_db().await); + + // create a sequencer + let config = SequencerConfig { + chain_spec: node.inner.chain_spec(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), }, - transactions: None, - no_tx_pool: false, - block_data_hint: BlockDataHint::none(), - gas_limit: None, - }); + block_time: 1, + payload_building_duration: 0, + allow_empty_blocks: false, + }; + let mut sequencer = Sequencer::new(database, config); - // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. - if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); + // build a new payload + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); - } + panic!("expected a payload ready event"); + }; + + // verify the gas used is within MIN_TRANSACTION_GAS_COST of the gas limit. + assert!(block.header.gas_used >= block.gas_limit - MIN_TRANSACTION_GAS_COST); } #[tokio::test] @@ -818,55 +820,54 @@ async fn can_build_blocks_and_exit_at_time_limit() { // create the engine driver connected to the node let auth_client = node.inner.engine_http_client(); let engine_client = ScrollAuthApiEngineClient::new(auth_client); - let mut engine_driver = EngineDriver::new( - Arc::new(engine_client), - (*SCROLL_DEV).clone(), - None::, - fcs, - false, - BLOCK_BUILDING_DURATION, - true, - ); + let mut engine = Engine::new(Arc::new(engine_client), fcs); + + // create a test database + let database = Arc::new(setup_test_db().await); + + // create a sequencer + let config = SequencerConfig { + chain_spec: node.inner.chain_spec(), + fee_recipient: Address::random(), + auto_start: false, + payload_building_config: PayloadBuildingConfig { + block_gas_limit: SCROLL_GAS_LIMIT, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + }, + block_time: 1, + payload_building_duration: 0, + allow_empty_blocks: false, + }; + let mut sequencer = Sequencer::new(database, config); // start timer. let start = Instant::now(); // issue a new payload to the execution layer. - let timestamp = - SystemTime::now().duration_since(UNIX_EPOCH).expect("Time can't go backwards").as_secs(); - engine_driver.handle_build_new_payload(ScrollPayloadAttributes { - payload_attributes: PayloadAttributes { - timestamp, - prev_randao: Default::default(), - suggested_fee_recipient: Default::default(), - withdrawals: None, - parent_beacon_block_root: None, - }, - transactions: None, - no_tx_pool: false, - block_data_hint: BlockDataHint::none(), - gas_limit: None, - }); - - if let Some(EngineDriverEvent::NewPayload(block)) = engine_driver.next().await { - let payload_building_duration = start.elapsed(); - // verify that the block building duration is within 10% of the target (we allow for 10% - // mismatch due to slower performance of debug mode). - assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); - assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); + // build a new payload + sequencer.start_payload_building(&mut engine).await.unwrap(); + let block = if let SequencerEvent::PayloadReady(payload_id) = sequencer.next().await.unwrap() { + let result = sequencer.finalize_payload_building(payload_id, &mut engine).await.unwrap(); + assert!(result.is_some(), "expected a new payload, but got: {:?}", result); + result.unwrap() } else { - panic!("expected a new payload event"); - } + panic!("expected a payload ready event"); + }; + + let payload_building_duration = start.elapsed(); + // verify that the block building duration is within 10% of the target (we allow for 10% + // mismatch due to slower performance of debug mode). + assert!(payload_building_duration < BLOCK_BUILDING_DURATION * 110 / 100); + assert!(block.gas_used < block.gas_limit - MIN_TRANSACTION_GAS_COST); } #[tokio::test] async fn should_limit_l1_message_cumulative_gas() { reth_tracing::init_test_tracing(); - let chain_spec = SCROLL_DEV.clone(); - const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - // setup a test node + let chain_spec = SCROLL_DEV.clone(); let (mut nodes, _tasks, wallet) = setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) .await @@ -899,7 +900,7 @@ async fn should_limit_l1_message_cumulative_gas() { // create a sequencer let config = SequencerConfig { - chain_spec: node.inner.chain_spec().clone(), + chain_spec: node.inner.chain_spec(), fee_recipient: Address::random(), auto_start: false, payload_building_config: PayloadBuildingConfig { From 40b67166a0a5741c0eb1a037beee30323e605576 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 8 Oct 2025 08:54:51 +0800 Subject: [PATCH 04/12] lint --- crates/primitives/src/batch.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/primitives/src/batch.rs b/crates/primitives/src/batch.rs index 8a5ad282..3ac02aa6 100644 --- a/crates/primitives/src/batch.rs +++ b/crates/primitives/src/batch.rs @@ -1,7 +1,7 @@ -use std::sync::Arc; - use super::{BlockInfo, L2BlockInfoWithL1Messages}; + use alloy_primitives::{Bytes, B256}; +use std::{sync::Arc, vec::Vec}; /// The batch information. #[derive(Debug, Copy, Clone, Default, PartialEq, Eq)] From 5b0647541d43d51b1cc16c65f998f5e82a52db6c Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 8 Oct 2025 10:28:50 +0800 Subject: [PATCH 05/12] add status rpc --- Cargo.lock | 2 ++ Makefile | 14 ++++++++++++++ crates/chain-orchestrator/Cargo.toml | 19 +++++++++++++++++++ crates/chain-orchestrator/src/lib.rs | 14 +++++++++----- crates/chain-orchestrator/src/status.rs | 3 ++- crates/chain-orchestrator/src/sync.rs | 2 ++ crates/engine/Cargo.toml | 2 ++ crates/engine/src/fcs.rs | 1 + crates/network/src/manager.rs | 4 ++-- crates/node/Cargo.toml | 2 +- crates/node/src/add_ons/rpc.rs | 24 +++++++++++++++++++++++- 11 files changed, 77 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cb7be688..9e00f036 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10852,6 +10852,7 @@ dependencies = [ "scroll-derivation-pipeline", "scroll-engine", "scroll-network", + "serde", "serde_json", "strum 0.27.2", "thiserror 2.0.16", @@ -11587,6 +11588,7 @@ dependencies = [ "scroll-alloy-network", "scroll-alloy-provider", "scroll-alloy-rpc-types-engine", + "serde", "thiserror 2.0.16", "tokio", "tracing", diff --git a/Makefile b/Makefile index 569b6aaf..c4e5349d 100644 --- a/Makefile +++ b/Makefile @@ -71,6 +71,20 @@ test: --no-fail-fast \ -E 'not test(docker)' +.PHONY: test-docker +test-docker: + cargo nextest run \ + --workspace \ + --locked \ + --all-features \ + --no-fail-fast \ + --no-tests=pass \ + -E 'test(docker)' \ + --test-threads=1 \ + --failure-output immediate \ + --success-output never \ + --verbose + # Used to update the mainnet-sample.sql data. Provide the path to the sqlite database that should be read from # using `DB_PATH`. .PHONY: test-data diff --git a/crates/chain-orchestrator/Cargo.toml b/crates/chain-orchestrator/Cargo.toml index 668f9c0e..eef8dc9f 100644 --- a/crates/chain-orchestrator/Cargo.toml +++ b/crates/chain-orchestrator/Cargo.toml @@ -53,6 +53,7 @@ reth-tokio-util.workspace = true futures.workspace = true metrics.workspace = true metrics-derive.workspace = true +serde = { workspace = true, optional = true, features = ["derive"] } strum = "0.27.1" thiserror.workspace = true tokio = { workspace = true, features = ["full"] } @@ -99,3 +100,21 @@ test-utils = [ "scroll-engine/test-utils", "scroll-network/test-utils", ] +serde = [ + "dep:serde", + "alloy-consensus/serde", + "alloy-eips/serde", + "alloy-primitives/serde", + "alloy-rpc-types-engine/serde", + "parking_lot/serde", + "rand/serde", + "reth-eth-wire-types/serde", + "reth-network-api/serde", + "reth-primitives-traits/serde", + "reth-scroll-forks/serde", + "reth-scroll-primitives/serde", + "scroll-alloy-consensus/serde", + "scroll-alloy-hardforks/serde", + "scroll-engine/serde", + "scroll-network/serde", +] diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index 733c357b..bbbbe7fe 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -1059,11 +1059,15 @@ impl< let tx = self.database.tx_mut().await?; tx.insert_signature(chain_head_hash, block_with_peer.signature).await?; tx.commit().await?; - self.network.handle().block_import_outcome(BlockImportOutcome::valid_block( - block_with_peer.peer_id, - block_with_peer.block, - Bytes::copy_from_slice(&block_with_peer.signature.sig_as_bytes()), - )); + + // We only notify the network of valid blocks. + if result.is_valid() { + self.network.handle().block_import_outcome(BlockImportOutcome::valid_block( + block_with_peer.peer_id, + block_with_peer.block, + Bytes::copy_from_slice(&block_with_peer.signature.sig_as_bytes()), + )); + } Ok(ChainImport { chain, diff --git a/crates/chain-orchestrator/src/status.rs b/crates/chain-orchestrator/src/status.rs index 3f7ced2b..54065080 100644 --- a/crates/chain-orchestrator/src/status.rs +++ b/crates/chain-orchestrator/src/status.rs @@ -2,7 +2,8 @@ use super::SyncState; use scroll_engine::ForkchoiceState; /// The current status of the chain orchestrator. -#[derive(Debug)] +#[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct ChainOrchestratorStatus { /// The current sync state of the orchestrator. pub sync_state: SyncState, diff --git a/crates/chain-orchestrator/src/sync.rs b/crates/chain-orchestrator/src/sync.rs index 0d53321c..f23c6118 100644 --- a/crates/chain-orchestrator/src/sync.rs +++ b/crates/chain-orchestrator/src/sync.rs @@ -1,5 +1,6 @@ /// The sync state of the chain orchestrator. #[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct SyncState { /// The sync mode for L1. l1: SyncMode, @@ -42,6 +43,7 @@ impl SyncState { /// The sync mode of the chain orchestrator. #[derive(Debug, Default, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub enum SyncMode { /// Syncing mode. #[default] diff --git a/crates/engine/Cargo.toml b/crates/engine/Cargo.toml index 01842e73..a537011b 100644 --- a/crates/engine/Cargo.toml +++ b/crates/engine/Cargo.toml @@ -43,6 +43,7 @@ eyre.workspace = true futures.workspace = true metrics.workspace = true metrics-derive.workspace = true +serde = { workspace = true, optional = true, features = ["derive"] } thiserror.workspace = true tokio.workspace = true tracing.workspace = true @@ -62,6 +63,7 @@ test-utils = [ "reth-primitives-traits/test-utils", ] serde = [ + "dep:serde", "alloy-eips/serde", "alloy-primitives/serde", "alloy-rpc-types-engine/serde", diff --git a/crates/engine/src/fcs.rs b/crates/engine/src/fcs.rs index 77dd6643..90e7c500 100644 --- a/crates/engine/src/fcs.rs +++ b/crates/engine/src/fcs.rs @@ -15,6 +15,7 @@ use scroll_alloy_network::Scroll; /// The state is composed of the [`BlockInfo`] for `head`, `safe` block, and the `finalized` /// blocks. #[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct ForkchoiceState { head: BlockInfo, safe: BlockInfo, diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index c953958a..f1353ac5 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -17,7 +17,7 @@ use reth_scroll_node::ScrollNetworkPrimitives; use reth_scroll_primitives::ScrollBlock; use reth_storage_api::BlockNumReader as BlockNumReaderT; use reth_tokio_util::{EventSender, EventStream}; -use rollup_node_primitives::sig_encode_hash; +use rollup_node_primitives::{sig_encode_hash, BlockInfo}; use scroll_alloy_hardforks::ScrollHardforks; use scroll_wire::{ NewBlock, ScrollWireConfig, ScrollWireEvent, ScrollWireManager, ScrollWireProtocolHandler, @@ -297,7 +297,7 @@ impl< match result { Ok(BlockValidation::ValidBlock { new_block: msg }) | Ok(BlockValidation::ValidHeader { new_block: msg }) => { - trace!(target: "scroll::network::manager", peer_id = ?peer, block = ?msg.block, "Block import successful - announcing block to network"); + trace!(target: "scroll::network::manager", peer_id = ?peer, block = %Into::::into(&msg.block), "Block import successful - announcing block to network"); self.announce_block(msg); } Err(BlockImportError::Consensus(err)) => { diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index 518d7775..0ce13ecc 100644 --- a/crates/node/Cargo.toml +++ b/crates/node/Cargo.toml @@ -63,7 +63,7 @@ reth-transaction-pool.workspace = true reth-trie-db.workspace = true # rollup node -rollup-node-chain-orchestrator.workspace = true +rollup-node-chain-orchestrator = { workspace = true, features = ["serde"] } rollup-node-primitives.workspace = true rollup-node-providers.workspace = true rollup-node-sequencer.workspace = true diff --git a/crates/node/src/add_ons/rpc.rs b/crates/node/src/add_ons/rpc.rs index 4260b660..70ca1cb8 100644 --- a/crates/node/src/add_ons/rpc.rs +++ b/crates/node/src/add_ons/rpc.rs @@ -6,7 +6,7 @@ use jsonrpsee::{ }; use reth_network_api::FullNetwork; use reth_scroll_node::ScrollNetworkPrimitives; -use rollup_node_chain_orchestrator::ChainOrchestratorHandle; +use rollup_node_chain_orchestrator::{ChainOrchestratorHandle, ChainOrchestratorStatus}; use tokio::sync::{oneshot, Mutex, OnceCell}; /// RPC extension for rollup node management operations. @@ -75,6 +75,10 @@ pub trait RollupNodeExtApi { /// Disables automatic sequencing in the rollup node. #[method(name = "disableAutomaticSequencing")] async fn disable_automatic_sequencing(&self) -> RpcResult; + + /// Returns the current status of the rollup node. + #[method(name = "status")] + async fn status(&self) -> RpcResult; } #[async_trait] @@ -117,4 +121,22 @@ where ) }) } + + async fn status(&self) -> RpcResult { + let handle = self.rollup_manager_handle().await.map_err(|e| { + ErrorObjectOwned::owned( + error::INTERNAL_ERROR_CODE, + format!("Failed to get rollup manager handle: {}", e), + None::<()>, + ) + })?; + + handle.status().await.map_err(|e| { + ErrorObjectOwned::owned( + error::INTERNAL_ERROR_CODE, + format!("Failed to get rollup node status: {}", e), + None::<()>, + ) + }) + } } From 6351f28bb05a51bc76a38216bd4feb4476341cc4 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 8 Oct 2025 18:09:24 +0800 Subject: [PATCH 06/12] fork choice fix and test coverage --- crates/chain-orchestrator/src/lib.rs | 9 +++-- crates/node/tests/sync.rs | 54 +++++++++++++++++++++++++--- 2 files changed, 55 insertions(+), 8 deletions(-) diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index bbbbe7fe..d0cf8608 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -958,13 +958,16 @@ impl< .collect(); let mut index = None; - for (i, header) in headers.iter().enumerate() { + for (i, header) in headers.iter().enumerate().rev() { let current_block = self .l2_client .get_block_by_number(header.number.into()) .full() .await? - .expect("block must exist"); + .expect("block must exist") + .into_consensus() + .map_transactions(|tx| tx.inner.into_inner()); + if header.hash_slow() == current_block.header.hash_slow() { index = Some(i); break; @@ -973,7 +976,7 @@ impl< if let Some(index) = index { tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, common_ancestor = ?headers[index].hash_slow(), common_ancestor_number = headers[index].number, "Found common ancestor for fork - reorging to new chain"); - for header in headers.into_iter().skip(index).rev() { + for header in headers.into_iter().skip(index + 1).rev() { new_headers.push_front(header); } let chain_import = self.import_chain(new_headers.into(), block_with_peer).await?; diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 3939128a..ba0f81f0 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -539,7 +539,13 @@ async fn test_consolidation() -> eyre::Result<()> { #[tokio::test] async fn test_chain_orchestrator_reorg_with_gap_above_head() -> eyre::Result<()> { test_chain_orchestrator_fork_choice(100, Some(95), 20, |e| { - matches!(e, ChainOrchestratorEvent::ChainReorged(_)) + if let ChainOrchestratorEvent::ChainReorged(chain_import) = e { + // Assert that the chain import is as expected. + assert_eq!(chain_import.chain.len(), 21); + true + } else { + false + } }) .await } @@ -548,7 +554,43 @@ async fn test_chain_orchestrator_reorg_with_gap_above_head() -> eyre::Result<()> #[tokio::test] async fn test_chain_orchestrator_reorg_with_gap_below_head() -> eyre::Result<()> { test_chain_orchestrator_fork_choice(100, Some(50), 20, |e| { - matches!(e, ChainOrchestratorEvent::ChainReorged(_)) + if let ChainOrchestratorEvent::ChainReorged(chain_import) = e { + // Assert that the chain import is as expected. + assert_eq!(chain_import.chain.len(), 21); + true + } else { + false + } + }) + .await +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_chain_orchestrator_extension_with_gap() -> eyre::Result<()> { + test_chain_orchestrator_fork_choice(100, None, 20, |e| { + if let ChainOrchestratorEvent::ChainExtended(chain_import) = e { + // Assert that the chain import is as expected. + assert_eq!(chain_import.chain.len(), 21); + true + } else { + false + } + }) + .await +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_chain_orchestrator_extension_no_gap() -> eyre::Result<()> { + test_chain_orchestrator_fork_choice(100, None, 0, |e| { + if let ChainOrchestratorEvent::ChainExtended(chain_import) = e { + // Assert that the chain import is as expected. + assert_eq!(chain_import.chain.len(), 1); + true + } else { + false + } }) .await } @@ -619,7 +661,7 @@ async fn test_chain_orchestrator_fork_choice( // Initially the sequencer should build 100 empty blocks in each and the follower // should follow them - let mut reorg_block_info = BlockInfo::default(); + let mut reorg_block_info: Option = None; for i in 0..initial_blocks { sequencer_handle.build_block(); wait_n_events( @@ -627,7 +669,7 @@ async fn test_chain_orchestrator_fork_choice( |e| { if let ChainOrchestratorEvent::BlockSequenced(block) = e { if Some(i) == reorg_block_number { - reorg_block_info = (&block).into(); + reorg_block_info = Some((&block).into()); } true } else { @@ -647,7 +689,9 @@ async fn test_chain_orchestrator_fork_choice( // Now reorg the sequencer and disable gossip so we can create fork sequencer_handle.set_gossip(false).await.unwrap(); - sequencer_handle.update_fcs_head(reorg_block_info).await.unwrap(); + if let Some(block_info) = reorg_block_info { + sequencer_handle.update_fcs_head(block_info).await.unwrap(); + } // wait two seconds to ensure the timestamp of the new blocks is greater than the old ones tokio::time::sleep(tokio::time::Duration::from_secs(2)).await; From 355998acadad4d6172e5af87b45c417dde29d65e Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 8 Oct 2025 22:55:16 +0800 Subject: [PATCH 07/12] add logs and optimise block handling in fork choice logic --- crates/chain-orchestrator/src/lib.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index d0cf8608..bca70dc5 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -81,7 +81,7 @@ const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); /// The number of headers to fetch in each request when fetching headers from peers. -const HEADER_FETCH_COUNT: u64 = 4000; +const HEADER_FETCH_COUNT: u64 = 100; /// The size of the event channel used to broadcast events to listeners. const EVENT_CHANNEL_SIZE: usize = 5000; @@ -834,7 +834,7 @@ impl< // If the received block number has a block number greater than the current head by more // than the optimistic sync threshold, we optimistically sync the chain. if received_block_number > current_head_number + self.config.optimistic_sync_threshold() { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); let block_info = BlockInfo { number: received_block_number, hash: block_with_peer.block.header.hash_slow(), @@ -856,10 +856,11 @@ impl< { // Fetch the headers for the received block until we can reconcile it with the current // chain head. - let block_number_diff = received_block_number - current_head_number; + let fetch_count = received_block_number - current_head_number; let new_headers = if received_block_number > current_head_number + 1 { + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?current_head_number, fetch_count, "Fetching headers to extend chain"); self.block_client - .get_full_block_range(received_block_hash, block_number_diff) + .get_full_block_range(received_block_hash, fetch_count) .await .into_iter() .rev() @@ -874,6 +875,7 @@ impl< if new_headers.first().expect("at least one header exists").parent_hash == current_head_hash { + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends the current head"); let chain_import = self.import_chain(new_headers, block_with_peer).await?; return Ok(Some(ChainOrchestratorEvent::ChainExtended(chain_import))); } @@ -948,17 +950,17 @@ impl< let parent_hash = new_headers.front().expect("at least one header exists").parent_hash; let parent_number = new_headers.front().expect("at least one header exists").number - 1; let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_head.number); + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?parent_hash, ?parent_number, %current_safe_head, fetch_count, "Fetching headers to find common ancestor for fork"); let headers: Vec = self .block_client .get_full_block_range(parent_hash, fetch_count) .await .into_iter() - .rev() .map(|b| b.into_block()) .collect(); let mut index = None; - for (i, header) in headers.iter().enumerate().rev() { + for (i, header) in headers.iter().enumerate() { let current_block = self .l2_client .get_block_by_number(header.number.into()) @@ -976,7 +978,7 @@ impl< if let Some(index) = index { tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, common_ancestor = ?headers[index].hash_slow(), common_ancestor_number = headers[index].number, "Found common ancestor for fork - reorging to new chain"); - for header in headers.into_iter().skip(index + 1).rev() { + for header in headers.into_iter().take(index) { new_headers.push_front(header); } let chain_import = self.import_chain(new_headers.into(), block_with_peer).await?; @@ -985,7 +987,7 @@ impl< // If we did not find a common ancestor, we add all the fetched headers to the front of // the deque and continue fetching. - for header in headers.into_iter().rev() { + for header in headers.into_iter() { new_headers.push_front(header); } } From 62edb0b19e5f4209275471117162dc845e6bc16b Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 9 Oct 2025 02:09:06 +0800 Subject: [PATCH 08/12] update rpc and default optimistic sync threshold --- .github/workflows/lint.yaml | 2 +- Makefile | 2 +- crates/chain-orchestrator/src/lib.rs | 21 +++++----- crates/chain-orchestrator/src/status.rs | 52 ++++++++++++++++++++++--- crates/database/db/src/error.rs | 6 --- crates/database/db/src/operations.rs | 28 +++++++------ crates/node/src/constants.rs | 2 +- crates/node/tests/e2e.rs | 10 ++--- 8 files changed, 80 insertions(+), 43 deletions(-) diff --git a/.github/workflows/lint.yaml b/.github/workflows/lint.yaml index 5e3cb410..33fc377f 100644 --- a/.github/workflows/lint.yaml +++ b/.github/workflows/lint.yaml @@ -92,7 +92,7 @@ jobs: with: cache-on-failure: true - name: Run doc - run: cargo docs --document-private-items + run: cargo docs --document-private-items --exclude rollup-node-chain-orchestrator env: RUSTDOCFLAGS: --cfg docsrs --show-type-layout --generate-link-to-definition --enable-index-page -Zunstable-options -D warnings diff --git a/Makefile b/Makefile index c4e5349d..f1554d91 100644 --- a/Makefile +++ b/Makefile @@ -97,7 +97,7 @@ export-sample-test-data: .PHONY: docs docs: - cargo docs --document-private-items + cargo docs --document-private-items --exclude rollup-node-chain-orchestrator .PHONY: pr pr: lint test docs diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index bca70dc5..f563205c 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -329,10 +329,16 @@ impl< let _ = tx.send(self.event_listener()); } ChainOrchestratorCommand::Status(tx) => { - let status = ChainOrchestratorStatus { - sync_state: self.sync_state.clone(), - forkchoice_state: self.engine.fcs().clone(), - }; + let db_tx = self.database.tx_mut().await?; + let l1_latest = db_tx.get_latest_l1_block_number().await?; + let l1_finalized = db_tx.get_finalized_l1_block_number().await?; + db_tx.commit().await?; + let status = ChainOrchestratorStatus::new( + &self.sync_state, + l1_latest, + l1_finalized, + self.engine.fcs().clone(), + ); let _ = tx.send(status); } ChainOrchestratorCommand::NetworkHandle(tx) => { @@ -697,10 +703,7 @@ impl< // Get all unprocessed batches that have been finalized by this L1 block // finalization. - let finalized_block_number = tx - .get_finalized_l1_block_number() - .await? - .expect("finalized block number must exist"); + let finalized_block_number = tx.get_finalized_l1_block_number().await?; if finalized_block_number >= block_number { let finalized_batches = tx .fetch_and_update_unprocessed_finalized_batches(finalized_block_number) @@ -987,7 +990,7 @@ impl< // If we did not find a common ancestor, we add all the fetched headers to the front of // the deque and continue fetching. - for header in headers.into_iter() { + for header in headers { new_headers.push_front(header); } } diff --git a/crates/chain-orchestrator/src/status.rs b/crates/chain-orchestrator/src/status.rs index 54065080..87f7f75f 100644 --- a/crates/chain-orchestrator/src/status.rs +++ b/crates/chain-orchestrator/src/status.rs @@ -1,12 +1,54 @@ -use super::SyncState; +use crate::sync::{SyncMode, SyncState}; use scroll_engine::ForkchoiceState; /// The current status of the chain orchestrator. #[derive(Debug, Clone)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct ChainOrchestratorStatus { - /// The current sync state of the orchestrator. - pub sync_state: SyncState, - /// The current FCS for the manager. - pub forkchoice_state: ForkchoiceState, + /// The chain status for L1. + pub l1: L1ChainStatus, + /// The chain status for L2. + pub l2: L2ChainStatus, +} + +impl ChainOrchestratorStatus { + /// Creates a new [`ChainOrchestratorStatus`] from the given sync state, latest L1 block number, + pub fn new( + sync_state: &SyncState, + l1_latest: u64, + l1_finalized: u64, + l2_fcs: ForkchoiceState, + ) -> Self { + Self { + l1: L1ChainStatus { + status: sync_state.l1().clone(), + latest: l1_latest, + finalized: l1_finalized, + }, + l2: L2ChainStatus { status: sync_state.l2().clone(), fcs: l2_fcs }, + } + } +} + +/// The status of the L1 chain. +#[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub struct L1ChainStatus { + /// The sync mode of the chain. + pub status: SyncMode, + /// The latest block number of the chain. + pub latest: u64, + /// The finalized block number of the chain. + pub finalized: u64, +} + +/// The status of the L2 chain. +#[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub struct L2ChainStatus { + /// The sync mode of the chain. + pub status: SyncMode, + /// The current fork choice state of the chain. + #[cfg_attr(feature = "serde", serde(flatten))] + pub fcs: ForkchoiceState, } diff --git a/crates/database/db/src/error.rs b/crates/database/db/src/error.rs index d138438c..f7bf3f32 100644 --- a/crates/database/db/src/error.rs +++ b/crates/database/db/src/error.rs @@ -19,10 +19,4 @@ pub enum DatabaseError { /// The L1 message was not found in database. #[error("L1 message at key [{0}] not found in database")] L1MessageNotFound(L1MessageKey), - /// The finalized L1 block was not found in database. - #[error("Finalized L1 block not found in database")] - FinalizedL1BlockNotFound, - /// The latest L1 block was not found in database. - #[error("Latest L1 block not found in database")] - LatestL1BlockNotFound, } diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 6fd25040..5ef9420b 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -231,7 +231,7 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati tracing::trace!(target: "scroll::db", "Fetching startup safe block from database."); // Unwind the database to the last finalized L1 block saved in database. - let finalized_block_number = self.get_finalized_l1_block_number().await?.unwrap_or(0); + let finalized_block_number = self.get_finalized_l1_block_number().await?; self.unwind(genesis_hash, finalized_block_number).await?; // Delete all unprocessed batches from the database and return starting l2 safe head and l1 @@ -528,27 +528,31 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { } /// Get the latest L1 block number from the database. - async fn get_latest_l1_block_number(&self) -> Result, DatabaseError> { + async fn get_latest_l1_block_number(&self) -> Result { Ok(models::metadata::Entity::find() .filter(models::metadata::Column::Key.eq("l1_latest_block")) .select_only() .column(models::metadata::Column::Value) .into_tuple::() .one(self.get_connection()) - .await - .map(|x| x.and_then(|x| x.parse::().ok()))?) + .await? + .expect("l1_latest_block should always be set") + .parse::() + .expect("l1_latest_block should always be a valid u64")) } /// Get the finalized L1 block number from the database. - async fn get_finalized_l1_block_number(&self) -> Result, DatabaseError> { + async fn get_finalized_l1_block_number(&self) -> Result { Ok(models::metadata::Entity::find() .filter(models::metadata::Column::Key.eq("l1_finalized_block")) .select_only() .column(models::metadata::Column::Value) .into_tuple::() .one(self.get_connection()) - .await - .map(|x| x.and_then(|x| x.parse::().ok()))?) + .await? + .expect("l1_finalized_block should always be set") + .parse::() + .expect("l1_finalized_block should always be a valid u64")) } /// Get the latest L2 head block info. @@ -701,10 +705,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { } } Some(L1MessageKey::NotIncluded(NotIncludedStart::Finalized)) => { - let finalized_block_number = self - .get_finalized_l1_block_number() - .await? - .ok_or(DatabaseError::FinalizedL1BlockNotFound)?; + let finalized_block_number = self.get_finalized_l1_block_number().await?; let condition = Condition::all() .add( models::l1_message::Column::L1BlockNumber @@ -721,10 +722,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { )) } Some(L1MessageKey::NotIncluded(NotIncludedStart::BlockDepth(depth))) => { - let latest_block_number = self - .get_latest_l1_block_number() - .await? - .ok_or(DatabaseError::LatestL1BlockNotFound)?; + let latest_block_number = self.get_latest_l1_block_number().await?; let target_block_number = latest_block_number.checked_sub(depth); if let Some(target_block_number) = target_block_number { diff --git a/crates/node/src/constants.rs b/crates/node/src/constants.rs index 48425bbf..9c470557 100644 --- a/crates/node/src/constants.rs +++ b/crates/node/src/constants.rs @@ -27,7 +27,7 @@ pub(crate) const DEFAULT_PAYLOAD_BUILDING_DURATION: u64 = 800; pub(crate) const DEFAULT_PAYLOAD_SIZE_LIMIT: u64 = 122_880; /// The gap in blocks between the P2P and EN which triggers sync. -pub(crate) const BLOCK_GAP_TRIGGER: u64 = 100_000; +pub(crate) const BLOCK_GAP_TRIGGER: u64 = 1_000; /// The number of block headers to keep in the in-memory chain buffer in the chain orchestrator. pub(crate) const CHAIN_BUFFER_SIZE: usize = 2000; diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 64480f05..aaa4bb0f 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -1184,7 +1184,7 @@ async fn graceful_shutdown_sets_fcs_to_latest_sequenced_block_in_db_on_start_up( let status = handle.status().await?; // The fcs should be set to the database head. - assert_eq!(status.forkchoice_state.head_block_info(), &db_head_block_info); + assert_eq!(status.l2.fcs.head_block_info(), &db_head_block_info); Ok(()) } @@ -1269,8 +1269,8 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { let status = handle.status().await?; // Assert the forkchoice state is above 4 - assert!(status.forkchoice_state.head_block_info().number > 4); - assert!(status.forkchoice_state.safe_block_info().number > 4); + assert!(status.l2.fcs.head_block_info().number > 4); + assert!(status.l2.fcs.safe_block_info().number > 4); // Send the third batch which should trigger the revert. l1_watcher_tx.send(Arc::new(L1Notification::BatchCommit(revert_batch_data))).await?; @@ -1281,8 +1281,8 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { let status = handle.status().await?; // Assert the forkchoice state was reset to 4. - assert_eq!(status.forkchoice_state.head_block_info().number, 4); - assert_eq!(status.forkchoice_state.safe_block_info().number, 4); + assert_eq!(status.l2.fcs.head_block_info().number, 4); + assert_eq!(status.l2.fcs.safe_block_info().number, 4); Ok(()) } From 46ebde74cdf3349a039a4438cef83a6e7b746fda Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 9 Oct 2025 02:40:20 +0800 Subject: [PATCH 09/12] commit merge files --- Cargo.lock | 1 + crates/derivation-pipeline/Cargo.toml | 1 + crates/derivation-pipeline/benches/pipeline.rs | 4 ++-- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6452defd..a6503edb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11647,6 +11647,7 @@ dependencies = [ "metrics", "metrics-derive", "reqwest", + "reth-tracing", "rollup-node-primitives", "rollup-node-providers", "scroll-alloy-consensus", diff --git a/crates/derivation-pipeline/Cargo.toml b/crates/derivation-pipeline/Cargo.toml index 2c171372..ad20ce2a 100644 --- a/crates/derivation-pipeline/Cargo.toml +++ b/crates/derivation-pipeline/Cargo.toml @@ -39,6 +39,7 @@ async-trait.workspace = true alloy-primitives = { workspace = true, features = ["getrandom"] } criterion = { package = "codspeed-criterion-compat", version = "4.0.2", features = ["async", "async_tokio"] } eyre.workspace = true +reth-tracing.workspace = true rollup-node-providers = { workspace = true, features = ["test-utils"] } reqwest.workspace = true serde_json.workspace = true diff --git a/crates/derivation-pipeline/benches/pipeline.rs b/crates/derivation-pipeline/benches/pipeline.rs index 23c7b126..53a8094e 100644 --- a/crates/derivation-pipeline/benches/pipeline.rs +++ b/crates/derivation-pipeline/benches/pipeline.rs @@ -124,7 +124,7 @@ fn benchmark_pipeline_derivation_in_file_blobs(c: &mut Criterion) { // commit 253 batches. for index in BATCHES_START_INDEX..=BATCHES_STOP_INDEX { let batch_info = BatchInfo { index, hash: Default::default() }; - pipeline.push_batch(batch_info, 0); + pipeline.push_batch(batch_info.into()).await; } tx.send(pipeline).unwrap(); @@ -160,7 +160,7 @@ fn benchmark_pipeline_derivation_s3_blobs(c: &mut Criterion) { // commit 15 batches. for index in BATCHES_START_INDEX..=BATCHES_START_INDEX + 15 { let batch_info = BatchInfo { index, hash: Default::default() }; - pipeline.push_batch(batch_info, 0); + pipeline.push_batch(batch_info.into()).await; } tx.send(pipeline).unwrap(); From 4309fb1abd6985f6cfa3a52c53ad456b9a84567a Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 9 Oct 2025 23:54:50 +0800 Subject: [PATCH 10/12] fix derivation pipeline persisting L1 mesages and address comments --- .../chain-orchestrator/src/consolidation.rs | 97 ++++++++++-- crates/chain-orchestrator/src/error.rs | 11 ++ crates/chain-orchestrator/src/lib.rs | 148 ++++++++++-------- crates/database/db/src/operations.rs | 74 ++++++--- crates/engine/src/error.rs | 32 +--- crates/engine/src/fcs.rs | 2 +- crates/engine/src/lib.rs | 2 +- crates/node/src/args.rs | 12 -- crates/node/tests/e2e.rs | 41 +++-- crates/primitives/src/batch.rs | 38 +++-- 10 files changed, 294 insertions(+), 163 deletions(-) diff --git a/crates/chain-orchestrator/src/consolidation.rs b/crates/chain-orchestrator/src/consolidation.rs index 81ca043f..2cb91313 100644 --- a/crates/chain-orchestrator/src/consolidation.rs +++ b/crates/chain-orchestrator/src/consolidation.rs @@ -1,7 +1,7 @@ use super::ChainOrchestratorError; use alloy_provider::Provider; use futures::{stream::FuturesOrdered, TryStreamExt}; -use rollup_node_primitives::{BatchInfo, BlockInfo}; +use rollup_node_primitives::{BatchConsolidationOutcome, BatchInfo, L2BlockInfoWithL1Messages}; use scroll_alloy_network::Scroll; use scroll_derivation_pipeline::{BatchDerivationResult, DerivedAttributes}; use scroll_engine::{block_matches_attributes, ForkchoiceState}; @@ -35,17 +35,17 @@ pub(crate) async fn reconcile_batch>( ¤t_block, current_block.parent_hash, ) { + // Extract the block info with L1 messages. + let block_info: L2BlockInfoWithL1Messages = (¤t_block).into(); + // The block matches the derived attributes and the block is below or equal to the // safe current safe head. if attributes.block_number <= fcs.safe_block_info().number { - Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Skip(BlockInfo { - number: current_block.number, - hash: current_block.hash_slow(), - })) + Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::Skip(block_info)) } else { // The block matches the derived attributes, no action is needed. Ok::<_, ChainOrchestratorError>(BlockConsolidationAction::UpdateSafeHead( - BlockInfo { number: current_block.number, hash: current_block.hash_slow() }, + block_info, )) } } else { @@ -69,25 +69,102 @@ pub(crate) struct BatchReconciliationResult { pub actions: Vec, } +impl BatchReconciliationResult { + /// Aggregates the block consolidation actions into an aggregated set of actions required to + /// consolidate the L2 chain with the batch. + pub(crate) fn aggregate_actions(&self) -> AggregatedBatchConsolidationActions { + let mut actions: Vec = vec![]; + for next in &self.actions { + if let Some(last) = actions.last_mut() { + match (last, next) { + (last, next) if last.is_update_safe_head() && next.is_update_safe_head() => { + *last = next.clone(); + } + _ => { + actions.push(next.clone()); + } + } + } else if !next.is_skip() { + actions.push(next.clone()); + } + } + AggregatedBatchConsolidationActions { actions } + } + + /// Consumes the reconciliation result and produces the consolidated chain by combining + /// non-reorg block info with the reorg block results. + pub(crate) async fn into_batch_consolidation_outcome( + self, + reorg_results: Vec, + ) -> Result { + let mut consolidate_chain = BatchConsolidationOutcome::new(self.batch_info); + + // First append all non-reorg results to the consolidated chain. + self.actions.into_iter().filter(|action| !action.is_reorg()).for_each(|action| { + consolidate_chain.push_block(action.into_block_info().expect("must have block info")); + }); + + // Append the reorg results at the end of the consolidated chain. + for block in reorg_results { + consolidate_chain.push_block(block); + } + + Ok(consolidate_chain) + } +} + +/// The aggregated actions that must be performed on the L2 chain to consolidate a batch. +#[derive(Debug, Clone)] +pub(crate) struct AggregatedBatchConsolidationActions { + /// The aggregated actions that must be performed on the L2 chain to consolidate a batch. + pub actions: Vec, +} + /// An action that must be performed on the L2 chain to consolidate a block. #[derive(Debug, Clone)] pub(crate) enum BlockConsolidationAction { /// Update the safe head to the given block. - UpdateSafeHead(BlockInfo), + UpdateSafeHead(L2BlockInfoWithL1Messages), /// The derived attributes match the L2 chain and the safe head is already at or beyond the /// block, so no action is needed. - Skip(BlockInfo), + Skip(L2BlockInfoWithL1Messages), /// Reorganize the chain with the given derived attributes. Reorg(DerivedAttributes), } +impl BlockConsolidationAction { + /// Returns true if the action is to update the safe head. + pub(crate) const fn is_update_safe_head(&self) -> bool { + matches!(self, Self::UpdateSafeHead(_)) + } + + /// Returns true if the action is to skip the block. + pub(crate) const fn is_skip(&self) -> bool { + matches!(self, Self::Skip(_)) + } + + /// Returns true if the action is to perform a reorg. + pub(crate) const fn is_reorg(&self) -> bool { + matches!(self, Self::Reorg(_)) + } + + /// Consumes the action and returns the block info if the action is to update the safe head or + /// skip, returns None for reorg. + pub(crate) fn into_block_info(self) -> Option { + match self { + Self::UpdateSafeHead(info) | Self::Skip(info) => Some(info), + Self::Reorg(_attrs) => None, + } + } +} + impl std::fmt::Display for BlockConsolidationAction { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { Self::UpdateSafeHead(info) => { - write!(f, "UpdateSafeHead to block {}", info.number) + write!(f, "UpdateSafeHead to block {}", info.block_info.number) } - Self::Skip(info) => write!(f, "Skip block {}", info.number), + Self::Skip(info) => write!(f, "Skip block {}", info.block_info.number), Self::Reorg(attrs) => { write!(f, "Reorg to block {}", attrs.block_number) } diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index d7e38e2a..25a06b98 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -78,4 +78,15 @@ pub enum ChainOrchestratorError { /// The derivation pipeline found an invalid block for the given batch. #[error("The derivation pipeline found an invalid block: {0} for batch: {1}")] InvalidBatch(BlockInfo, BatchInfo), + /// Attempted to reorg a batch but the safe block number does not match the derived + /// block number - 1. + #[error("Attempted to reorg batch {batch_info:?} for derived block number {derived_block_number} but expected safe block number is {safe_block_number} - we expect `safe block number = derived block number - 1`")] + InvalidBatchReorg { + /// The batch info. + batch_info: BatchInfo, + /// The current safe block number. + safe_block_number: u64, + /// The derived block number. + derived_block_number: u64, + }, } diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index f563205c..4a51f1f1 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -15,8 +15,8 @@ use reth_scroll_primitives::ScrollBlock; use reth_tasks::shutdown::Shutdown; use reth_tokio_util::{EventSender, EventStream}; use rollup_node_primitives::{ - BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, - ChainImport, L1MessageEnvelope, L2BlockInfoWithL1Messages, + BatchCommitData, BatchInfo, BlockConsolidationOutcome, BlockInfo, ChainImport, + L1MessageEnvelope, L2BlockInfoWithL1Messages, }; use rollup_node_providers::{L1MessageProvider, L1Provider}; use rollup_node_sequencer::{Sequencer, SequencerEvent}; @@ -329,10 +329,9 @@ impl< let _ = tx.send(self.event_listener()); } ChainOrchestratorCommand::Status(tx) => { - let db_tx = self.database.tx_mut().await?; + let db_tx = self.database.tx().await?; let l1_latest = db_tx.get_latest_l1_block_number().await?; let l1_finalized = db_tx.get_finalized_l1_block_number().await?; - db_tx.commit().await?; let status = ChainOrchestratorStatus::new( &self.sync_state, l1_latest, @@ -405,28 +404,38 @@ impl< &mut self, batch: BatchDerivationResult, ) -> Result, ChainOrchestratorError> { - tracing::info!(target: "scroll::chain_orchestrator", batch_info = ?batch.batch_info, num_blocks = batch.attributes.len(), "Handling derived batch"); + let batch_info = batch.batch_info; + tracing::info!(target: "scroll::chain_orchestrator", batch_info = ?batch_info, num_blocks = batch.attributes.len(), "Handling derived batch"); let batch_reconciliation_result = reconcile_batch(&self.l2_client, batch, self.engine.fcs()).await?; - let mut batch_consolidation_result = - BatchConsolidationOutcome::new(batch_reconciliation_result.batch_info); - for action in batch_reconciliation_result.actions { + let aggregated_actions = batch_reconciliation_result.aggregate_actions(); + + let mut reorg_results = vec![]; + for action in aggregated_actions.actions { let outcome = match action { - BlockConsolidationAction::Skip(block_info) => { - tracing::info!(target: "scroll::chain_orchestrator", ?block_info, "Skipping consolidation of block as it matches the current chain and is below or equal to the safe head"); - BlockConsolidationOutcome::Skipped(block_info) + BlockConsolidationAction::Skip(_) => { + unreachable!("Skip actions have been filtered out in aggregation") } BlockConsolidationAction::UpdateSafeHead(block_info) => { tracing::info!(target: "scroll::chain_orchestrator", ?block_info, "Updating safe head to consolidated block"); - self.engine.update_fcs(None, Some(block_info), Some(block_info)).await?; - BlockConsolidationOutcome::Consolidated(block_info) + self.engine + .update_fcs(None, Some(block_info.block_info), Some(block_info.block_info)) + .await?; + BlockConsolidationOutcome::UpdateFcs(block_info) } BlockConsolidationAction::Reorg(attributes) => { tracing::info!(target: "scroll::chain_orchestrator", block_number = ?attributes.block_number, "Reorging chain to derived block"); // We reorg the head to the safe block and then build the payload for the // attributes. let head = *self.engine.fcs().safe_block_info(); + if head.number != attributes.block_number - 1 { + return Err(ChainOrchestratorError::InvalidBatchReorg { + batch_info, + safe_block_number: head.number, + derived_block_number: attributes.block_number, + }); + } let fcu = self.engine.build_payload(Some(head), attributes.attributes).await?; let payload = self .engine @@ -442,7 +451,7 @@ impl< if result.is_invalid() { return Err(ChainOrchestratorError::InvalidBatch( (&block).into(), - batch_consolidation_result.batch_info, + batch_info, )); } @@ -456,21 +465,23 @@ impl< ) .await?; + reorg_results.push(block_info.clone()); BlockConsolidationOutcome::Reorged(block_info) } }; self.notify(ChainOrchestratorEvent::BlockConsolidated(outcome.clone())); - - batch_consolidation_result.push_block(outcome); } + let batch_consolidation_outcome = + batch_reconciliation_result.into_batch_consolidation_outcome(reorg_results).await?; + // Insert the batch consolidation outcome into the database. let tx = self.database.tx_mut().await?; - tx.insert_batch_consolidation_outcome(batch_consolidation_result.clone()).await?; + tx.insert_batch_consolidation_outcome(batch_consolidation_outcome.clone()).await?; tx.commit().await?; - Ok(Some(ChainOrchestratorEvent::BatchConsolidated(batch_consolidation_result))) + Ok(Some(ChainOrchestratorEvent::BatchConsolidated(batch_consolidation_outcome))) } /// Handles an L1 notification. @@ -535,19 +546,10 @@ impl< Retry::default() .retry("unwind", || async { let txn = self.database.tx_mut().await?; - let UnwindResult { - l1_block_number, - queue_index, - l2_head_block_number, - l2_safe_block_info, - } = txn.unwind(self.config.chain_spec().genesis_hash(), block_number).await?; + let unwind_result = + txn.unwind(self.config.chain_spec().genesis_hash(), block_number).await?; txn.commit().await?; - Ok::<_, ChainOrchestratorError>(UnwindResult { - l1_block_number, - queue_index, - l2_head_block_number, - l2_safe_block_info, - }) + Ok::<_, ChainOrchestratorError>(unwind_result) }) .await?; @@ -831,13 +833,16 @@ impl< let received_block_number = block_with_peer.block.number; let received_block_hash = block_with_peer.block.header.hash_slow(); - let current_head_number = self.engine.fcs().head_block_info().number; - let current_head_hash = self.engine.fcs().head_block_info().hash; + let current_head_block_number = self.engine.fcs().head_block_info().number; + let current_head_block_hash = self.engine.fcs().head_block_info().hash; + let current_safe_block_number = self.engine.fcs().safe_block_info().number; // If the received block number has a block number greater than the current head by more // than the optimistic sync threshold, we optimistically sync the chain. - if received_block_number > current_head_number + self.config.optimistic_sync_threshold() { - tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); + if received_block_number > + current_head_block_number + self.config.optimistic_sync_threshold() + { + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_number, ?current_head_block_number, "Received new block from peer with block number greater than current head by more than the optimistic sync threshold"); let block_info = BlockInfo { number: received_block_number, hash: block_with_peer.block.header.hash_slow(), @@ -845,23 +850,22 @@ impl< self.engine.optimistic_sync(block_info).await?; self.sync_state.l2_mut().set_syncing(); - self.notify(ChainOrchestratorEvent::OptimisticSync(block_info)); - // Purge all L1 message to L2 block mappings as they may be invalid after an // optimistic sync. let tx = self.database.tx_mut().await?; tx.purge_l1_message_to_l2_block_mappings(None).await?; tx.commit().await?; + + return Ok(Some(ChainOrchestratorEvent::OptimisticSync(block_info))) } // If the block number is greater than the current head we attempt to extend the chain. - let mut new_headers = if received_block_number > self.engine.fcs().head_block_info().number - { + let mut new_headers = if received_block_number > current_head_block_number { // Fetch the headers for the received block until we can reconcile it with the current // chain head. - let fetch_count = received_block_number - current_head_number; - let new_headers = if received_block_number > current_head_number + 1 { - tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?current_head_number, fetch_count, "Fetching headers to extend chain"); + let fetch_count = received_block_number - current_head_block_number; + let new_headers = if received_block_number > current_head_block_number + 1 { + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?current_head_block_number, fetch_count, "Fetching headers to extend chain"); self.block_client .get_full_block_range(received_block_hash, fetch_count) .await @@ -876,7 +880,7 @@ impl< // If the first header in the new headers has a parent hash that matches the current // head hash, we can import the chain. if new_headers.first().expect("at least one header exists").parent_hash == - current_head_hash + current_head_block_hash { tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends the current head"); let chain_import = self.import_chain(new_headers, block_with_peer).await?; @@ -895,7 +899,7 @@ impl< .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(received_block_number))?; if current_chain_block.header.hash_slow() == received_block_hash { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that is already in the chain"); + tracing::info!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that is already in the chain"); return Ok(Some(ChainOrchestratorEvent::BlockAlreadyKnown( received_block_hash, block_with_peer.peer_id, @@ -905,14 +909,14 @@ impl< // Assert that we are not reorging below the safe head. let current_safe_info = self.engine.fcs().safe_block_info(); if received_block_number <= current_safe_info.number { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); + tracing::warn!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); } // Check to assert that we have received a newer chain. let current_head = self .l2_client - .get_block_by_number(current_head_number.into()) + .get_block_by_number(current_head_block_number.into()) .full() .await? .expect("current head block must exist"); @@ -920,7 +924,7 @@ impl< // If the timestamp of the received block is less than or equal to the current head, // we ignore it. if block_with_peer.block.header.timestamp <= current_head.header.timestamp { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_head_hash = ?current_head.header.hash_slow(), current_head_number = current_head_number, "Received block from peer that is older than the current head - ignoring"); + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_head_hash = ?current_head.header.hash_slow(), current_head_number = current_head_block_number, "Received block from peer that is older than the current head - ignoring"); return Ok(Some(ChainOrchestratorEvent::OldForkReceived { headers: vec![block_with_peer.block.header], peer_id: block_with_peer.peer_id, @@ -934,11 +938,20 @@ impl< .get_block_by_hash(block_with_peer.block.header.parent_hash) .full() .await?; - if parent_block.is_some() { - tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends an earlier part of the chain"); - let chain_import = - self.import_chain(vec![block_with_peer.block.clone()], block_with_peer).await?; - return Ok(Some(ChainOrchestratorEvent::ChainReorged(chain_import))); + if let Some(parent_block) = parent_block { + // If the parent block has a block number equal to or greater than the current safe + // head then it is safe to reorg. + if parent_block.header.number >= current_safe_block_number { + tracing::debug!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, "Received block from peer that extends an earlier part of the chain"); + let chain_import = self + .import_chain(vec![block_with_peer.block.clone()], block_with_peer) + .await?; + return Ok(Some(ChainOrchestratorEvent::ChainReorged(chain_import))); + } + // If the parent block has a block number less than the current safe head then would + // suggest a reorg of the safe head - reject it. + tracing::warn!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, current_safe_info = ?self.engine.fcs().safe_block_info(), "Received block from peer that would reorg below the safe head - ignoring"); + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); } VecDeque::from([block_with_peer.block.clone()]) @@ -946,14 +959,13 @@ impl< // If we reach this point, we have a block that is not in the current chain and does not // extend the current head. This implies a reorg. We attempt to reconcile the fork. - let current_safe_head = self.engine.fcs().safe_block_info(); - while current_safe_head.number + 1 < + while current_safe_block_number + 1 < new_headers.front().expect("at least one header exists").number { let parent_hash = new_headers.front().expect("at least one header exists").parent_hash; let parent_number = new_headers.front().expect("at least one header exists").number - 1; - let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_head.number); - tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?parent_hash, ?parent_number, %current_safe_head, fetch_count, "Fetching headers to find common ancestor for fork"); + let fetch_count = HEADER_FETCH_COUNT.min(parent_number - current_safe_block_number); + tracing::trace!(target: "scroll::chain_orchestrator", ?received_block_hash, ?received_block_number, ?parent_hash, ?parent_number, %current_safe_block_number, fetch_count, "Fetching headers to find common ancestor for fork"); let headers: Vec = self .block_client .get_full_block_range(parent_hash, fetch_count) @@ -1043,12 +1055,6 @@ impl< return Err(ChainOrchestratorError::InvalidBlock) } - // Persist the mapping of L1 messages to L2 blocks such that we can react to L1 reorgs. - let blocks = chain.iter().map(|block| block.into()).collect::>(); - let tx = self.database.tx_mut().await?; - tx.update_l1_messages_from_l2_blocks(blocks).await?; - tx.commit().await?; - // If we were previously in L2 syncing mode and the FCS update resulted in a valid state, we // transition the L2 sync state to synced and consolidate the chain. if result.is_valid() && self.sync_state.l2().is_syncing() { @@ -1062,14 +1068,20 @@ impl< } } - // Persist the signature for the block and notify the network manager of a successful - // import. - let tx = self.database.tx_mut().await?; - tx.insert_signature(chain_head_hash, block_with_peer.signature).await?; - tx.commit().await?; + // Persist the L1 message to L2 block mappings for reorg awareness, the block signature and + // handle the valid block import if we are in a synced state and the result is valid. + if self.sync_state.is_synced() && result.is_valid() { + let blocks = chain.iter().map(|block| block.into()).collect::>(); + let tx = self.database.tx_mut().await?; + tx.update_l1_messages_from_l2_blocks(blocks).await?; + tx.commit().await?; + + // Persist the signature for the block and notify the network manager of a successful + // import. + let tx = self.database.tx_mut().await?; + tx.insert_signature(chain_head_hash, block_with_peer.signature).await?; + tx.commit().await?; - // We only notify the network of valid blocks. - if result.is_valid() { self.network.handle().block_import_outcome(BlockImportOutcome::valid_block( block_with_peer.peer_id, block_with_peer.block, diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 5ef9420b..f67bee11 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -4,8 +4,8 @@ use crate::{ReadConnectionProvider, WriteConnectionProvider}; use alloy_primitives::{Signature, B256}; use futures::{Stream, StreamExt}; use rollup_node_primitives::{ - BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockConsolidationOutcome, BlockInfo, - L1MessageEnvelope, L2BlockInfoWithL1Messages, Metadata, + BatchCommitData, BatchConsolidationOutcome, BatchInfo, BlockInfo, L1MessageEnvelope, + L2BlockInfoWithL1Messages, Metadata, }; use scroll_alloy_rpc_types_engine::BlockDataHint; use sea_orm::{ @@ -412,20 +412,8 @@ pub trait DatabaseWriteOperations: WriteConnectionProvider + DatabaseReadOperati outcome: BatchConsolidationOutcome, ) -> Result<(), DatabaseError> { for block in outcome.blocks { - match block { - BlockConsolidationOutcome::Consolidated(block_info) => { - self.insert_block(block_info, outcome.batch_info).await?; - } - BlockConsolidationOutcome::Skipped(block_info) => { - // No action needed, the block has already been previously consolidated however - // we will insert it again defensively - self.insert_block(block_info, outcome.batch_info).await?; - } - BlockConsolidationOutcome::Reorged(block_info) => { - self.insert_block(block_info.block_info, outcome.batch_info).await?; - self.update_l1_messages_with_l2_block(block_info).await?; - } - } + self.insert_block(block.block_info, outcome.batch_info).await?; + self.update_l1_messages_with_l2_block(block).await?; } Ok(()) } @@ -628,6 +616,8 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { DatabaseError, > { match start { + // Provides an stream over all L1 messages with increasing queue index starting from the + // provided queue index. Some(L1MessageKey::QueueIndex(queue_index)) => Ok(Some( models::l1_message::Entity::find() .filter(models::l1_message::Column::QueueIndex.gte(queue_index)) @@ -636,8 +626,10 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .await? .map(map_l1_message_result), )), + // Provides a stream over all L1 messages with increasing queue index starting from the + // message with the provided transaction hash. Some(L1MessageKey::TransactionHash(ref h)) => { - // Lookup message by hash + // Lookup message by hash to get its queue index. let record = models::l1_message::Entity::find() .filter(models::l1_message::Column::Hash.eq(h.to_vec())) .one(self.get_connection()) @@ -645,6 +637,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .ok_or_else(|| { DatabaseError::L1MessageNotFound(L1MessageKey::TransactionHash(*h)) })?; + // Yield a stream of messages starting from the found queue index. Ok(Some( models::l1_message::Entity::find() .filter(models::l1_message::Column::QueueIndex.gte(record.queue_index)) @@ -654,8 +647,10 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .map(map_l1_message_result), )) } + // Provides a stream over all L1 messages with increasing queue index starting from the + // message with the provided queue hash. Some(L1MessageKey::QueueHash(ref h)) => { - // Lookup message by queue hash + // Lookup message by queue hash. let record = models::l1_message::Entity::find() .filter( Condition::all() @@ -665,7 +660,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .one(self.get_connection()) .await? .ok_or_else(|| DatabaseError::L1MessageNotFound(L1MessageKey::QueueHash(*h)))?; - + // Yield a stream of messages starting from the found queue index. Ok(Some( models::l1_message::Entity::find() .filter(models::l1_message::Column::QueueIndex.gte(record.queue_index)) @@ -675,7 +670,14 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .map(map_l1_message_result), )) } + // Provides a stream over all L1 messages with increasing queue index starting from the + // message included in the provided L2 block number. Some(L1MessageKey::BlockNumber(block_number)) => { + // Lookup the the latest message included in a block with a block number less than + // the provided block number. This is achieved by filtering for messages with a + // block number less than the provided block number and ordering by block number and + // queue index in descending order. This ensures that we get the latest message + // included in a block before the provided block number. if let Some(record) = models::l1_message::Entity::find() .filter(models::l1_message::Column::L2BlockNumber.lt(block_number as i64)) .order_by_desc(models::l1_message::Column::L2BlockNumber) @@ -683,6 +685,7 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .one(self.get_connection()) .await? { + // Yield a stream of messages starting from the found queue index + 1. Ok(Some( models::l1_message::Entity::find() .filter( @@ -694,7 +697,11 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .await? .map(map_l1_message_result), )) - } else { + } + // If no messages have been found then it suggests that no messages have been + // included in blocks yet and as such we should return a stream of all messages with + // increasing queue index starting from the beginning. + else { Ok(Some( models::l1_message::Entity::find() .order_by_asc(models::l1_message::Column::QueueIndex) @@ -704,14 +711,24 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { )) } } + // Provides a stream over all L1 messages with increasing queue index starting that have + // not been included in an L2 block and have a block number less than or equal to the + // finalized L1 block number (they have been finalized on L1). Some(L1MessageKey::NotIncluded(NotIncludedStart::Finalized)) => { + // Lookup the finalized L1 block number. let finalized_block_number = self.get_finalized_l1_block_number().await?; + + // Create a filter condition for messages that have an L1 block number less than or + // equal to the finalized block number and have not been included in an L2 block + // (i.e. L2BlockNumber is null). let condition = Condition::all() .add( models::l1_message::Column::L1BlockNumber .lte(finalized_block_number as i64), ) .add(models::l1_message::Column::L2BlockNumber.is_null()); + // Yield a stream of messages matching the condition ordered by increasing queue + // index. Ok(Some( models::l1_message::Entity::find() .filter(condition) @@ -721,17 +738,30 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .map(map_l1_message_result), )) } + // Provides a stream over all L1 messages with increasing queue index starting that have + // not been included in an L2 block and have a block number less than or equal to the + // latest L1 block number minus the provided depth (they have been sufficiently deep + // on L1 to be considered safe to include - reorg risk is low). Some(L1MessageKey::NotIncluded(NotIncludedStart::BlockDepth(depth))) => { + // Lookup the latest L1 block number. let latest_block_number = self.get_latest_l1_block_number().await?; + // Calculate the target block number by subtracting the depth from the latest block + // number. If the depth is greater than the latest block number, we return None as + // there are no messages that satisfy the condition. let target_block_number = latest_block_number.checked_sub(depth); if let Some(target_block_number) = target_block_number { + // Create a filter condition for messages that have an L1 block number less than + // or equal to the target block number and have not been included in an L2 block + // (i.e. L2BlockNumber is null). let condition = Condition::all() .add( models::l1_message::Column::L1BlockNumber .lte(target_block_number as i64), ) .add(models::l1_message::Column::L2BlockNumber.is_null()); + // Yield a stream of messages matching the condition ordered by increasing + // queue index. Ok(Some( models::l1_message::Entity::find() .filter(condition) @@ -741,9 +771,13 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { .map(map_l1_message_result), )) } else { + // If the depth is greater than the latest block number, return None as there + // are no messages that satisfy the condition. Ok(None) } } + // Provides a stream over all L1 messages with increasing queue index starting from the + // beginning. None => Ok(Some( models::l1_message::Entity::find() .order_by_asc(models::l1_message::Column::QueueIndex) diff --git a/crates/engine/src/error.rs b/crates/engine/src/error.rs index 0baecaf4..acaf3fe0 100644 --- a/crates/engine/src/error.rs +++ b/crates/engine/src/error.rs @@ -1,33 +1,3 @@ -use alloy_rpc_types_engine::PayloadError; -use rollup_node_primitives::{ScrollPayloadAttributesWithBatchInfo, WithBlockNumber}; -use scroll_alloy_provider::ScrollEngineApiError; -use scroll_alloy_rpc_types_engine::ScrollPayloadAttributes; - -/// The error type for the engine API. -#[derive(Debug, thiserror::Error)] -pub enum EngineDriverError { - /// The engine is unavailable. - #[error("Engine is unavailable")] - EngineUnavailable, - /// The execution payload is invalid. - #[error("Invalid execution payload: {0}")] - InvalidExecutionPayload(#[from] PayloadError), - /// The execution payload provider is unavailable. - #[error("Execution payload provider is unavailable")] - ExecutionPayloadProviderUnavailable, - /// The forkchoice update failed. - #[error("Forkchoice update failed: {0}")] - ForkchoiceUpdateFailed(ScrollEngineApiError), - /// The payload id field is missing in the forkchoice update response for an L1 consolidation - /// job. - #[error("Forkchoice update response missing payload id for L1 consolidation job")] - L1ConsolidationMissingPayloadId(WithBlockNumber), - /// The payload id field is missing in the forkchoice update response for a payload building - /// job. - #[error("Forkchoice update response missing payload id for payload building job")] - PayloadBuildingMissingPayloadId(ScrollPayloadAttributes), -} - /// The error type for the fork choice state. #[derive(Debug, thiserror::Error)] pub enum FcsError { @@ -38,7 +8,7 @@ pub enum FcsError { #[error("Finalized block number not increasing")] FinalizedBlockNumberNotIncreasing, /// Head block number cannot be below safe block number. - #[error("Safe block number can not be below the head block number")] + #[error("head block number can not be below the safe block number")] HeadBelowSafe, /// Safe block number cannot be below finalized block number. #[error("Safe block number can not be below the finalized block number")] diff --git a/crates/engine/src/fcs.rs b/crates/engine/src/fcs.rs index 90e7c500..77dc3d14 100644 --- a/crates/engine/src/fcs.rs +++ b/crates/engine/src/fcs.rs @@ -88,7 +88,7 @@ impl ForkchoiceState { let new_safe = safe.unwrap_or(self.safe); let new_head = head.unwrap_or(self.head); - // Check that the finalized block number is increasing. + // Check that the finalized block number is increasing or stays the same with the same hash. if new_finalized.number <= self.finalized.number && new_finalized != self.finalized { return Err(FcsError::FinalizedBlockNumberNotIncreasing); } diff --git a/crates/engine/src/lib.rs b/crates/engine/src/lib.rs index fc87334e..09c7463e 100644 --- a/crates/engine/src/lib.rs +++ b/crates/engine/src/lib.rs @@ -1,7 +1,7 @@ //! Engine Driver for the Scroll Rollup Node. The [`Engine`] exposes the main interface for //! the Rollup Node to the Engine API. -pub use error::{EngineDriverError, EngineError, FcsError}; +pub use error::{EngineError, FcsError}; mod error; pub use fcs::{genesis_hash_from_chain_spec, ForkchoiceState}; diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 7a946c02..b940ac31 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -31,9 +31,6 @@ use rollup_node_chain_orchestrator::{ ChainOrchestrator, ChainOrchestratorConfig, ChainOrchestratorHandle, Consensus, NoopConsensus, SystemContractConsensus, }; -// use rollup_node_manager::{ -// Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, -// }; use rollup_node_primitives::{BlockInfo, NodeConfig}; use rollup_node_providers::{ BlobProvidersBuilder, FullL1Provider, L1MessageProvider, L1Provider, SystemContractProvider, @@ -323,15 +320,6 @@ impl ScrollRollupNodeConfig { tracing::info!(target: "scroll::node::args", fcs = ?fcs, payload_building_duration = ?self.sequencer_args.payload_building_duration, "Starting engine driver"); let engine = Engine::new(Arc::new(engine_api), fcs); - // let engine = EngineDriver::new( - // Arc::new(engine_api), - // chain_spec.clone(), - // Some(l2_provider.clone()), - // fcs, - // self.engine_driver_args.sync_at_startup && !self.test && !chain_spec.is_dev_chain(), - // Duration::from_millis(self.sequencer_args.payload_building_duration), - // self.sequencer_args.allow_empty_blocks, - // ); // Create the consensus. let authorized_signer = if let Some(provider) = l1_provider.as_ref() { diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index aaa4bb0f..5e0bc5fa 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -727,6 +727,10 @@ async fn can_bridge_blocks() { .unwrap(); let mut bridge_node = nodes.pop().unwrap(); let bridge_peer_id = bridge_node.network.record().id; + let bridge_node_l1_watcher_tx = bridge_node.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Send a notification to set the L1 to synced + bridge_node_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); // Instantiate the scroll NetworkManager. let network_config = NetworkConfigBuilder::::with_rng_secret_key() @@ -888,6 +892,8 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() finalized_block_number: None, }; + println!("Sending first batch commit and finalization"); + // Send the first batch commit to the rollup node manager and finalize it. l1_notification_tx.send(Arc::new(L1Notification::BatchCommit(batch_0_data.clone()))).await?; l1_notification_tx @@ -901,15 +907,20 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() // Lets finalize the first batch l1_notification_tx.send(Arc::new(L1Notification::Finalized(batch_0_data.block_number))).await?; + println!("First batch finalized, iterating until first batch is consolidated"); + // Lets iterate over all blocks expected to be derived from the first batch commit. let consolidation_outcome = loop { let event = rnm_events.next().await; + println!("Received event: {:?}", event); if let Some(ChainOrchestratorEvent::BatchConsolidated(consolidation_outcome)) = event { break consolidation_outcome; } }; assert_eq!(consolidation_outcome.blocks.len(), 4, "Expected 4 blocks to be consolidated"); + println!("First batch consolidated, sending second batch commit and finalization"); + // Now we send the second batch commit and finalize it. l1_notification_tx.send(Arc::new(L1Notification::BatchCommit(batch_1_data.clone()))).await?; l1_notification_tx @@ -923,6 +934,8 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() // Lets finalize the second batch. l1_notification_tx.send(Arc::new(L1Notification::Finalized(batch_1_data.block_number))).await?; + println!("Second batch finalized, iterating until block 40 is consolidated"); + // The second batch commit contains 42 blocks (5-57), lets iterate until the rnm has // consolidated up to block 40. let mut i = 5; @@ -931,8 +944,8 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - assert_eq!(consolidation_outcome.block_info().number, i); - break consolidation_outcome.block_info().hash; + assert_eq!(consolidation_outcome.block_info().block_info.number, i); + break consolidation_outcome.block_info().block_info.hash; } }; if i == 40 { @@ -941,6 +954,8 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() i += 1; }; + println!("Block 40 consolidated, checking safe and head block hashes"); + // Fetch the safe and head block hashes from the EN. let rpc = node.rpc.inner.eth_api(); let safe_block_hash = @@ -1015,23 +1030,26 @@ async fn shutdown_consolidates_most_recent_batch_on_startup() -> eyre::Result<() if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - break *consolidation_outcome.block_info(); + break consolidation_outcome.block_info().clone(); } }; // One issue #273 is completed, we will again have safe blocks != finalized blocks, and this - // should be changed to 1. Assert that the consolidated block is the first block of the - // batch. - assert_eq!(l2_block.number, 5, "Consolidated block number does not match expected number"); + // should be changed to 1. Assert that the consolidated block is the first block that was not + // previously processed of the batch. + assert_eq!( + l2_block.block_info.number, 41, + "Consolidated block number does not match expected number" + ); // Lets now iterate over all remaining blocks expected to be derived from the second batch // commit. - for i in 6..=57 { + for i in 42..=57 { loop { if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - assert!(consolidation_outcome.block_info().number == i); + assert!(consolidation_outcome.block_info().block_info.number == i); break; } } @@ -1246,7 +1264,7 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - if consolidation_outcome.block_info().number == 4 { + if consolidation_outcome.block_info().block_info.number == 4 { break } } @@ -1260,7 +1278,7 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { if let Some(ChainOrchestratorEvent::BlockConsolidated(consolidation_outcome)) = rnm_events.next().await { - if consolidation_outcome.block_info().number == 46 { + if consolidation_outcome.block_info().block_info.number == 46 { break } } @@ -1309,8 +1327,9 @@ async fn can_handle_l1_message_reorg() -> eyre::Result<()> { let mut node1_rnm_events = node1_rnm_handle.get_event_listener().await?; let node1_l1_watcher_tx = node1.inner.add_ons_handle.l1_watcher_tx.as_ref().unwrap(); - // Set L1 synced on sequencer node + // Set L1 synced on both the sequencer and follower nodes. node0_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; + node1_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await?; // Let the sequencer build 10 blocks before performing the reorg process. let mut reorg_block = None; diff --git a/crates/primitives/src/batch.rs b/crates/primitives/src/batch.rs index 3ac02aa6..016c5a31 100644 --- a/crates/primitives/src/batch.rs +++ b/crates/primitives/src/batch.rs @@ -1,4 +1,4 @@ -use super::{BlockInfo, L2BlockInfoWithL1Messages}; +use super::L2BlockInfoWithL1Messages; use alloy_primitives::{Bytes, B256}; use std::{sync::Arc, vec::Vec}; @@ -59,7 +59,7 @@ pub struct BatchConsolidationOutcome { /// The batch info for the consolidated batch. pub batch_info: BatchInfo, /// The consolidation outcomes for each block in the batch. - pub blocks: Vec, + pub blocks: Vec, } impl BatchConsolidationOutcome { @@ -69,7 +69,7 @@ impl BatchConsolidationOutcome { } /// Pushes a block consolidation outcome to the batch. - pub fn push_block(&mut self, block: BlockConsolidationOutcome) { + pub fn push_block(&mut self, block: L2BlockInfoWithL1Messages) { self.blocks.push(block); } } @@ -78,19 +78,39 @@ impl BatchConsolidationOutcome { #[derive(Debug, Clone, PartialEq, Eq)] pub enum BlockConsolidationOutcome { /// The derived block was already part of the chain, update the fork choice state. - Consolidated(BlockInfo), + UpdateFcs(L2BlockInfoWithL1Messages), /// The fork choice state was already ahead of the derived block. - Skipped(BlockInfo), + Skipped(L2BlockInfoWithL1Messages), /// The derived block resulted in a reorg of the L2 chain. Reorged(L2BlockInfoWithL1Messages), } impl BlockConsolidationOutcome { - /// Returns the block info for the consolidated block. - pub const fn block_info(&self) -> &BlockInfo { + /// Returns the block info with l2 messages for the consolidated block. + pub const fn block_info(&self) -> &L2BlockInfoWithL1Messages { match self { - Self::Consolidated(info) | Self::Skipped(info) => info, - Self::Reorged(info) => &info.block_info, + Self::UpdateFcs(info) | Self::Skipped(info) | Self::Reorged(info) => info, + } + } + + /// Consumes the outcome and returns the block info with l2 messages for the consolidated block. + pub fn into_inner(self) -> L2BlockInfoWithL1Messages { + match self { + Self::UpdateFcs(info) | Self::Skipped(info) | Self::Reorged(info) => info, + } + } +} + +impl std::fmt::Display for BlockConsolidationOutcome { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::UpdateFcs(info) => { + write!(f, "Update Fcs to block {}", info.block_info.number) + } + Self::Skipped(info) => write!(f, "Skipped block {}", info.block_info.number), + Self::Reorged(attrs) => { + write!(f, "Reorged to block {}", attrs.block_info) + } } } } From d7fa9ec7ed48c4929e46cc5ca6feb0e5f5804c0d Mon Sep 17 00:00:00 2001 From: frisitano Date: Sat, 11 Oct 2025 18:40:20 +0800 Subject: [PATCH 11/12] add semaphore for database read transaction limiting --- crates/database/db/src/db.rs | 25 ++++++++++--------------- crates/database/db/src/transaction.rs | 8 +++++--- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index 451fa583..b57c4843 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -8,7 +8,7 @@ use sea_orm::{ DatabaseConnection, SqlxSqliteConnector, TransactionTrait, }; use std::sync::Arc; -use tokio::sync::Mutex; +use tokio::sync::{Mutex, Semaphore}; // TODO: make these configurable via CLI. @@ -16,7 +16,7 @@ use tokio::sync::Mutex; const BUSY_TIMEOUT_SECS: u64 = 5; /// The maximum number of connections in the database connection pool. -const MAX_CONNECTIONS: u32 = 10; +const MAX_CONNECTIONS: u32 = 32; /// The minimum number of connections in the database connection pool. const MIN_CONNECTIONS: u32 = 5; @@ -36,6 +36,8 @@ pub struct Database { connection: DatabaseConnection, /// A mutex to ensure that only one mutable transaction is active at a time. write_lock: Arc>, + /// A semaphore to limit the number of concurrent read-only transactions. + read_locks: Arc, /// The database metrics. metrics: DatabaseMetrics, /// The temporary directory used for testing. We keep it here to ensure it lives as long as the @@ -80,9 +82,13 @@ impl Database { .connect_with(options) .await?; + // We reserve one connection for write transactions. + let read_connection_limit = max_connections as usize - 1; + Ok(Self { connection: SqlxSqliteConnector::from_sqlx_sqlite_pool(sqlx_pool), write_lock: Arc::new(Mutex::new(())), + read_locks: Arc::new(Semaphore::new(read_connection_limit)), metrics: DatabaseMetrics::default(), #[cfg(feature = "test-utils")] tmp_dir: None, @@ -111,7 +117,8 @@ impl DatabaseTransactionProvider for Database { /// Creates a new [`TX`] which can be used for read-only operations. async fn tx(&self) -> Result { tracing::trace!(target: "scroll::db", "Creating new read-only transaction"); - Ok(TX::new(self.connection.clone().begin().await?)) + let permit = self.read_locks.clone().acquire_owned().await.unwrap(); + Ok(TX::new(self.connection.clone().begin().await?, permit)) } /// Creates a new [`TXMut`] which can be used for atomic read and write operations. @@ -148,18 +155,6 @@ impl DatabaseConnectionProvider for Database { } } -impl From for Database { - fn from(connection: DatabaseConnection) -> Self { - Self { - connection, - write_lock: Arc::new(Mutex::new(())), - metrics: DatabaseMetrics::default(), - #[cfg(feature = "test-utils")] - tmp_dir: None, - } - } -} - #[cfg(test)] mod test { use super::*; diff --git a/crates/database/db/src/transaction.rs b/crates/database/db/src/transaction.rs index 3d5fe1a6..c5d54433 100644 --- a/crates/database/db/src/transaction.rs +++ b/crates/database/db/src/transaction.rs @@ -1,7 +1,7 @@ use crate::DatabaseConnectionProvider; use super::{DatabaseError, ReadConnectionProvider, WriteConnectionProvider}; -use tokio::sync::OwnedMutexGuard; +use tokio::sync::{OwnedMutexGuard, OwnedSemaphorePermit}; /// A type that represents a read-only database transaction. /// @@ -10,12 +10,14 @@ use tokio::sync::OwnedMutexGuard; pub struct TX { /// The underlying database transaction. tx: sea_orm::DatabaseTransaction, + /// A permit for the read transaction semaphore. + _permit: OwnedSemaphorePermit, } impl TX { /// Creates a new [`TX`] instance associated with the provided [`sea_orm::DatabaseTransaction`]. - pub const fn new(tx: sea_orm::DatabaseTransaction) -> Self { - Self { tx } + pub const fn new(tx: sea_orm::DatabaseTransaction, permit: OwnedSemaphorePermit) -> Self { + Self { tx, _permit: permit } } } From 52f5b243af14bf59278975d33bc39b185095c9b4 Mon Sep 17 00:00:00 2001 From: frisitano Date: Sat, 11 Oct 2025 19:18:33 +0800 Subject: [PATCH 12/12] add L1 message finalized with depth inclusion rule --- crates/database/db/src/operations.rs | 75 +++++++++++++++------------- crates/sequencer/src/config.rs | 29 ++++++++--- crates/sequencer/tests/e2e.rs | 4 +- 3 files changed, 64 insertions(+), 44 deletions(-) diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index f67bee11..a711d4a9 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -714,18 +714,25 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { // Provides a stream over all L1 messages with increasing queue index starting that have // not been included in an L2 block and have a block number less than or equal to the // finalized L1 block number (they have been finalized on L1). - Some(L1MessageKey::NotIncluded(NotIncludedStart::Finalized)) => { + Some(L1MessageKey::NotIncluded(NotIncludedStart::FinalizedWithBlockDepth(depth))) => { // Lookup the finalized L1 block number. let finalized_block_number = self.get_finalized_l1_block_number().await?; + // Calculate the target block number by subtracting the depth from the finalized + // block number. If the depth is greater than the finalized block number, we return + // None as there are no messages that satisfy the condition. + let target_block_number = + if let Some(target_block_number) = finalized_block_number.checked_sub(depth) { + target_block_number + } else { + return Ok(None); + }; + // Create a filter condition for messages that have an L1 block number less than or // equal to the finalized block number and have not been included in an L2 block // (i.e. L2BlockNumber is null). let condition = Condition::all() - .add( - models::l1_message::Column::L1BlockNumber - .lte(finalized_block_number as i64), - ) + .add(models::l1_message::Column::L1BlockNumber.lte(target_block_number as i64)) .add(models::l1_message::Column::L2BlockNumber.is_null()); // Yield a stream of messages matching the condition ordered by increasing queue // index. @@ -749,32 +756,28 @@ pub trait DatabaseReadOperations: ReadConnectionProvider + Sync { // Calculate the target block number by subtracting the depth from the latest block // number. If the depth is greater than the latest block number, we return None as // there are no messages that satisfy the condition. - let target_block_number = latest_block_number.checked_sub(depth); - if let Some(target_block_number) = target_block_number { - // Create a filter condition for messages that have an L1 block number less than - // or equal to the target block number and have not been included in an L2 block - // (i.e. L2BlockNumber is null). - let condition = Condition::all() - .add( - models::l1_message::Column::L1BlockNumber - .lte(target_block_number as i64), - ) - .add(models::l1_message::Column::L2BlockNumber.is_null()); - // Yield a stream of messages matching the condition ordered by increasing - // queue index. - Ok(Some( - models::l1_message::Entity::find() - .filter(condition) - .order_by_asc(models::l1_message::Column::QueueIndex) - .stream(self.get_connection()) - .await? - .map(map_l1_message_result), - )) - } else { - // If the depth is greater than the latest block number, return None as there - // are no messages that satisfy the condition. - Ok(None) - } + let target_block_number = + if let Some(target_block_number) = latest_block_number.checked_sub(depth) { + target_block_number + } else { + return Ok(None); + }; + // Create a filter condition for messages that have an L1 block number less than + // or equal to the target block number and have not been included in an L2 block + // (i.e. L2BlockNumber is null). + let condition = Condition::all() + .add(models::l1_message::Column::L1BlockNumber.lte(target_block_number as i64)) + .add(models::l1_message::Column::L2BlockNumber.is_null()); + // Yield a stream of messages matching the condition ordered by increasing + // queue index. + Ok(Some( + models::l1_message::Entity::find() + .filter(condition) + .order_by_asc(models::l1_message::Column::QueueIndex) + .stream(self.get_connection()) + .await? + .map(map_l1_message_result), + )) } // Provides a stream over all L1 messages with increasing queue index starting from the // beginning. @@ -966,8 +969,10 @@ impl L1MessageKey { /// block yet. #[derive(Debug, Clone, PartialEq, Eq)] pub enum NotIncludedStart { - /// Start from finalized messages that have not been included in a block yet. - Finalized, + /// Start from finalized messages that have not been included in a block yet and have a L1 + /// block number that is a specified number of blocks below the current finalized L1 block + /// number. + FinalizedWithBlockDepth(u64), /// Start from unfinalized messages that are included in L1 blocks at a specific depth. BlockDepth(u64), } @@ -986,7 +991,9 @@ impl fmt::Display for L1MessageKey { Self::TransactionHash(hash) => write!(f, "TransactionHash({hash:#x})"), Self::BlockNumber(number) => write!(f, "BlockNumber({number})"), Self::NotIncluded(start) => match start { - NotIncludedStart::Finalized => write!(f, "NotIncluded(Finalized)"), + NotIncludedStart::FinalizedWithBlockDepth(depth) => { + write!(f, "NotIncluded(Finalized:{depth})") + } NotIncludedStart::BlockDepth(depth) => { write!(f, "NotIncluded(BlockDepth({depth}))") } diff --git a/crates/sequencer/src/config.rs b/crates/sequencer/src/config.rs index 942b803c..79f08cd3 100644 --- a/crates/sequencer/src/config.rs +++ b/crates/sequencer/src/config.rs @@ -33,21 +33,32 @@ pub struct PayloadBuildingConfig { } /// Configuration for L1 message inclusion strategy. -#[derive(Debug, Default, Clone, Copy)] +#[derive(Debug, Clone, Copy)] pub enum L1MessageInclusionMode { /// Include L1 messages based on block depth. BlockDepth(u64), - /// Include only finalized L1 messages. - #[default] - Finalized, + /// Include only finalized L1 messages with an additional block depth. + FinalizedWithBlockDepth(u64), +} + +// The default is to include finalized L1 messages with a depth of 2 blocks below the current +// finalized block number. +impl Default for L1MessageInclusionMode { + fn default() -> Self { + Self::FinalizedWithBlockDepth(2) + } } impl FromStr for L1MessageInclusionMode { type Err = String; fn from_str(s: &str) -> Result { - if s.eq_ignore_ascii_case("finalized") { - Ok(Self::Finalized) + if let Some(rest) = s.strip_prefix("finalized:") { + rest.parse::() + .map(Self::FinalizedWithBlockDepth) + .map_err(|_| format!("Expected a valid number after 'finalized:', got '{rest}'")) + } else if s.eq_ignore_ascii_case("finalized") { + Ok(Self::FinalizedWithBlockDepth(0)) } else if let Some(rest) = s.strip_prefix("depth:") { rest.parse::() .map(Self::BlockDepth) @@ -61,7 +72,7 @@ impl FromStr for L1MessageInclusionMode { impl fmt::Display for L1MessageInclusionMode { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Self::Finalized => write!(f, "finalized"), + Self::FinalizedWithBlockDepth(depth) => write!(f, "finalized:{depth}"), Self::BlockDepth(depth) => write!(f, "depth:{depth}"), } } @@ -70,7 +81,9 @@ impl fmt::Display for L1MessageInclusionMode { impl From for L1MessageKey { fn from(mode: L1MessageInclusionMode) -> Self { match mode { - L1MessageInclusionMode::Finalized => Self::NotIncluded(NotIncludedStart::Finalized), + L1MessageInclusionMode::FinalizedWithBlockDepth(depth) => { + Self::NotIncluded(NotIncludedStart::FinalizedWithBlockDepth(depth)) + } L1MessageInclusionMode::BlockDepth(depth) => { Self::NotIncluded(NotIncludedStart::BlockDepth(depth)) } diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 1dcb2db5..345ac0f7 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -385,7 +385,7 @@ async fn can_build_blocks_with_finalized_l1_messages() { payload_building_config: PayloadBuildingConfig { block_gas_limit: SCROLL_GAS_LIMIT, max_l1_messages_per_block: 4, - l1_message_inclusion_mode: L1MessageInclusionMode::Finalized, + l1_message_inclusion_mode: L1MessageInclusionMode::FinalizedWithBlockDepth(0), }, block_time: 0, payload_building_duration: 0, @@ -906,7 +906,7 @@ async fn should_limit_l1_message_cumulative_gas() { payload_building_config: PayloadBuildingConfig { block_gas_limit: SCROLL_GAS_LIMIT, max_l1_messages_per_block: 4, - l1_message_inclusion_mode: L1MessageInclusionMode::Finalized, + l1_message_inclusion_mode: L1MessageInclusionMode::FinalizedWithBlockDepth(0), }, block_time: 0, payload_building_duration: 0,