diff --git a/.codespellrc b/.codespellrc index ed79b8bc..1e4d8f23 100644 --- a/.codespellrc +++ b/.codespellrc @@ -1,3 +1,3 @@ [codespell] -skip = .git,target,Cargo.toml,Cargo.lock +skip = .git,target,Cargo.toml,Cargo.lock,docker-compose ignore-words-list = crate diff --git a/.github/workflows/lint.yaml b/.github/workflows/lint.yaml index dd586b46..4d36ef57 100644 --- a/.github/workflows/lint.yaml +++ b/.github/workflows/lint.yaml @@ -128,11 +128,11 @@ jobs: - type: wasm target: wasm32-unknown-unknown exclude: | - scroll-engine,scroll-wire,rollup-node,scroll-network,rollup-node-manager,rollup-node-watcher,scroll-db,scroll-migration,rollup-node-indexer,scroll-codec,scroll-derivation-pipeline,rollup-node-providers,rollup-node-sequencer,rollup-node-signer,tests + scroll-engine,scroll-wire,rollup-node,scroll-network,rollup-node-manager,rollup-node-watcher,scroll-db,scroll-migration,rollup-node-chain-orchestrator,scroll-codec,scroll-derivation-pipeline,rollup-node-providers,rollup-node-sequencer,rollup-node-signer,tests - type: riscv target: riscv32imac-unknown-none-elf exclude: | - scroll-engine,scroll-wire,rollup-node,scroll-network,rollup-node-manager,rollup-node-watcher,scroll-db,scroll-migration,rollup-node-indexer,scroll-codec,scroll-derivation-pipeline,rollup-node-providers,rollup-node-sequencer,rollup-node-signer,tests + scroll-engine,scroll-wire,rollup-node,scroll-network,rollup-node-manager,rollup-node-watcher,scroll-db,scroll-migration,rollup-node-chain-orchestrator,scroll-codec,scroll-derivation-pipeline,rollup-node-providers,rollup-node-sequencer,rollup-node-signer,tests steps: - uses: actions/checkout@v4 - uses: rui314/setup-mold@v1 diff --git a/Cargo.lock b/Cargo.lock index e32b07c9..37d66c1b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10356,6 +10356,7 @@ dependencies = [ "reth-transaction-pool", "reth-trie-db", "rollup-node", + "rollup-node-chain-orchestrator", "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", @@ -10380,26 +10381,43 @@ dependencies = [ ] [[package]] -name = "rollup-node-indexer" +name = "rollup-node-chain-orchestrator" version = "0.0.1" dependencies = [ + "alloy-consensus 1.0.23", + "alloy-eips 1.0.23", + "alloy-json-rpc", "alloy-primitives", + "alloy-provider", + "alloy-rpc-client", + "alloy-transport", "arbitrary", "futures", "metrics", "metrics-derive", + "parking_lot 0.12.4", "rand 0.9.2", + "reqwest", "reth-chainspec", + "reth-eth-wire-types", + "reth-network-p2p", + "reth-network-peers", + "reth-primitives-traits", "reth-scroll-chainspec", "reth-scroll-forks", + "reth-scroll-primitives", "rollup-node-primitives", "rollup-node-watcher", "scroll-alloy-consensus", "scroll-alloy-hardforks", + "scroll-alloy-network", "scroll-db", + "scroll-network", + "serde_json", "strum 0.27.2", "thiserror 2.0.12", "tokio", + "tracing", ] [[package]] @@ -10421,7 +10439,7 @@ dependencies = [ "reth-scroll-primitives", "reth-tasks", "reth-tokio-util", - "rollup-node-indexer", + "rollup-node-chain-orchestrator", "rollup-node-primitives", "rollup-node-providers", "rollup-node-sequencer", @@ -10453,6 +10471,7 @@ dependencies = [ "alloy-rpc-types-engine 1.0.23", "arbitrary", "derive_more", + "reth-network-peers", "reth-primitives-traits", "reth-scroll-primitives", "scroll-alloy-consensus", @@ -11177,6 +11196,7 @@ dependencies = [ "reqwest-middleware", "reqwest-retry", "reth-chainspec", + "reth-scroll-chainspec", "sea-orm", "sea-orm-migration", "sha2 0.10.9", @@ -11190,6 +11210,7 @@ dependencies = [ "alloy-primitives", "futures", "parking_lot 0.12.4", + "reth-chainspec", "reth-eth-wire-types", "reth-network", "reth-network-api", @@ -11200,6 +11221,8 @@ dependencies = [ "reth-scroll-node", "reth-scroll-primitives", "reth-storage-api", + "reth-tokio-util", + "scroll-alloy-hardforks", "scroll-wire", "thiserror 2.0.12", "tokio", diff --git a/Cargo.toml b/Cargo.toml index dc8c29d7..e175416b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -12,7 +12,7 @@ members = [ "crates/database/migration", "crates/derivation-pipeline", "crates/engine", - "crates/indexer", + "crates/chain-orchestrator", "crates/l1", "crates/manager", "crates/network", @@ -155,6 +155,7 @@ reth-primitives-traits = { git = "https://github.com/scroll-tech/reth.git", defa reth-provider = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-rpc-builder = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-rpc-server-types = { git = "https://github.com/scroll-tech/reth.git", default-features = false } +reth-storage-api = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-tasks = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-tokio-util = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-tracing = { git = "https://github.com/scroll-tech/reth.git", default-features = false } @@ -168,7 +169,7 @@ reth-scroll-primitives = { git = "https://github.com/scroll-tech/reth.git", defa # rollup node rollup-node = { path = "crates/node" } -rollup-node-indexer = { path = "crates/indexer" } +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" } diff --git a/README.md b/README.md index db7e39a2..01e38f9a 100644 --- a/README.md +++ b/README.md @@ -24,7 +24,7 @@ This repository is a modular Rust workspace for the Scroll rollup node. It is de │ │ └── migration/ │ ├── derivation-pipeline/ │ ├── engine/ -│ ├── indexer/ +│ ├── chain-orchestrator/ │ ├── l1/ │ ├── network/ │ ├── node/ @@ -46,7 +46,7 @@ This repository is a modular Rust workspace for the Scroll rollup node. It is de - **crates/database/migration/**: Database schema migrations using SeaORM. - **crates/derivation-pipeline/**: Stateless pipeline for transforming batches into block-building payloads. - **crates/engine/**: Core engine logic for block execution, fork choice, and payload management. -- **crates/indexer/**: Indexes L1 and L2 data for efficient querying and notification. +- **crates/chain-orchestrator/**: Responsible for orchestrating the L2 chain based on events from L1 and data gossiped over the P2P network. - **crates/l1/**: Primitives and ABI bindings for L1 contracts and messages. - **crates/network/**: P2P networking stack for node communication. - **crates/node/**: Node manager and orchestration logic. diff --git a/crates/indexer/Cargo.toml b/crates/chain-orchestrator/Cargo.toml similarity index 56% rename from crates/indexer/Cargo.toml rename to crates/chain-orchestrator/Cargo.toml index b761deb6..45267e7c 100644 --- a/crates/indexer/Cargo.toml +++ b/crates/chain-orchestrator/Cargo.toml @@ -1,5 +1,5 @@ [package] -name = "rollup-node-indexer" +name = "rollup-node-chain-orchestrator" version.workspace = true edition.workspace = true rust-version.workspace = true @@ -11,7 +11,12 @@ workspace = true [dependencies] # alloy +alloy-consensus = { workspace = true } +alloy-eips = { workspace = true } +alloy-json-rpc.workspace = true alloy-primitives.workspace = true +alloy-provider.workspace = true +alloy-transport.workspace = true # rollup-node scroll-db.workspace = true @@ -19,11 +24,17 @@ rollup-node-primitives.workspace = true rollup-node-watcher.workspace = true # scroll +reth-scroll-primitives.workspace = true scroll-alloy-consensus.workspace = true scroll-alloy-hardforks.workspace = true +scroll-alloy-network.workspace = true +scroll-network.workspace = true # reth reth-chainspec.workspace = true +reth-network-p2p = { git = "https://github.com/scroll-tech/reth.git", default-features = false } +reth-network-peers.workspace = true +reth-primitives-traits.workspace = true # misc futures.workspace = true @@ -31,10 +42,14 @@ metrics.workspace = true metrics-derive.workspace = true strum = "0.27.1" thiserror.workspace = true +tracing.workspace = true tokio.workspace = true [dev-dependencies] +alloy-consensus = { workspace = true, features = ["arbitrary"] } alloy-primitives = { workspace = true, features = ["arbitrary"] } +alloy-rpc-client.workspace = true +alloy-transport.workspace = true # rollup-node scroll-db = { workspace = true, features = ["test-utils"] } @@ -44,8 +59,15 @@ rollup-node-primitives = { workspace = true, features = ["arbitrary"] } reth-scroll-chainspec.workspace = true reth-scroll-forks.workspace = true +# reth +reth-eth-wire-types.workspace = true +reth-network-peers.workspace = true + # misc arbitrary.workspace = true futures.workspace = true +parking_lot.workspace = true rand.workspace = true +reqwest.workspace = true +serde_json = { version = "1.0" } tokio.workspace = true diff --git a/crates/chain-orchestrator/src/action.rs b/crates/chain-orchestrator/src/action.rs new file mode 100644 index 00000000..1baadcf7 --- /dev/null +++ b/crates/chain-orchestrator/src/action.rs @@ -0,0 +1,57 @@ +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/error.rs b/crates/chain-orchestrator/src/error.rs new file mode 100644 index 00000000..2b88a00c --- /dev/null +++ b/crates/chain-orchestrator/src/error.rs @@ -0,0 +1,44 @@ +use alloy_json_rpc::RpcError; +use alloy_primitives::B256; +use alloy_transport::TransportErrorKind; +use scroll_db::{DatabaseError, L1MessageStart}; + +/// A type that represents an error that occurred in the chain orchestrator. +#[derive(Debug, thiserror::Error)] +pub enum ChainOrchestratorError { + /// An error occurred while interacting with the database. + #[error("database error occurred: {0}")] + DatabaseError(#[from] DatabaseError), + /// An error occurred while trying to fetch the L2 block from the database. + #[error("L2 block not found - block number: {0}")] + L2BlockNotFound(u64), + /// A fork was received from the peer that is associated with a reorg of the safe chain. + #[error("L2 safe block reorg detected")] + L2SafeBlockReorgDetected, + /// A block contains invalid L1 messages. + #[error("Block contains invalid L1 message. Expected: {expected:?}, Actual: {actual:?}")] + L1MessageMismatch { + /// The expected L1 messages hash. + expected: B256, + /// The actual L1 messages hash. + actual: B256, + }, + /// An L1 message was not found in the database. + #[error("L1 message not found at {0}")] + L1MessageNotFound(L1MessageStart), + /// An inconsistency was detected when trying to consolidate the chain. + #[error("Chain inconsistency detected")] + ChainInconsistency, + /// The peer did not provide the requested block header. + #[error("A peer did not provide the requested block header")] + MissingBlockHeader { + /// The hash of the block header that was requested. + hash: B256, + }, + /// An error occurred while making a network request. + #[error("Network request error: {0}")] + NetworkRequestError(#[from] reth_network_p2p::error::RequestError), + /// 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), +} diff --git a/crates/chain-orchestrator/src/event.rs b/crates/chain-orchestrator/src/event.rs new file mode 100644 index 00000000..bbabac3b --- /dev/null +++ b/crates/chain-orchestrator/src/event.rs @@ -0,0 +1,66 @@ +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}; + +/// An event emitted by the `ChainOrchestrator`. +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum ChainOrchestratorEvent { + /// 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), + /// The block that we have received is already known. + BlockAlreadyKnown(B256, PeerId), + /// A fork of the chain that is older than the current chain has been received. + OldForkReceived { + /// The headers of the old fork. + headers: Vec
, + /// The peer that provided the old fork. + peer_id: PeerId, + /// The signature of the old fork. + signature: Signature, + }, + /// The chain should be optimistically synced to the provided block. + OptimisticSync(ScrollBlock), + /// 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. + ChainReorged(ChainImport), + /// A `BatchCommit` event has been indexed returning the batch info and the L2 block info to + /// revert to due to a batch revert. + BatchCommitIndexed { + /// The batch info. + batch_info: BatchInfo, + /// The L1 block number in which the batch was committed. + l1_block_number: u64, + /// The safe L2 block info. + safe_head: Option, + }, + /// A batch has been finalized returning the batch hash and new an optional finalized + /// L2 block. + BatchFinalized(B256, Option), + /// An L1 block has been finalized returning the L1 block number and an optional + /// finalized L2 block. + L1BlockFinalized(u64, Option), + /// A `L1Message` event has been committed returning the message queue index. + L1MessageCommitted(u64), + /// The chain has been unwound, returning the L1 block number of the new L1 head, + /// the L1 message queue index of the new L1 head, and optionally the L2 head and safe block + /// info if the unwind resulted in a new L2 head or safe block. + ChainUnwound { + /// The L1 block number of the new L1 head. + l1_block_number: u64, + /// The L1 message queue index of the new L1 head. + queue_index: Option, + /// The L2 head block info. + l2_head_block_info: Option, + /// The L2 safe block info. + l2_safe_block_info: Option, + }, + /// 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), +} diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs new file mode 100644 index 00000000..8d1c80ba --- /dev/null +++ b/crates/chain-orchestrator/src/lib.rs @@ -0,0 +1,1589 @@ +//! 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::{BlockHashOrNumber, Encodable2718}; +use alloy_primitives::{b256, keccak256, B256}; +use alloy_provider::Provider; +use futures::{task::AtomicWaker, Stream, StreamExt, TryStreamExt}; +use reth_chainspec::EthChainSpec; +use reth_network_p2p::{BlockClient, BodiesClient}; +use reth_scroll_primitives::ScrollBlock; +use rollup_node_primitives::{ + BatchCommitData, BatchInfo, BlockInfo, BoundedVec, ChainImport, L1MessageEnvelope, + L2BlockInfoWithL1Messages, +}; +use rollup_node_watcher::L1Notification; +use scroll_alloy_consensus::TxL1Message; +use scroll_alloy_hardforks::{ScrollHardfork, ScrollHardforks}; +use scroll_alloy_network::Scroll; +use scroll_db::{Database, DatabaseError, DatabaseOperations, L1MessageStart, UnwindResult}; +use scroll_network::NewBlockWithPeer; +use std::{ + collections::{HashMap, VecDeque}, + pin::Pin, + sync::{ + atomic::{AtomicU64, Ordering}, + Arc, + }, + task::{Context, Poll}, + time::Instant, +}; +use strum::IntoEnumIterator; +use tokio::sync::Mutex; + +mod action; +use action::{ChainOrchestratorFuture, PendingChainOrchestratorFuture}; + +mod event; +pub use event::ChainOrchestratorEvent; + +mod error; +pub use error::ChainOrchestratorError; + +mod metrics; +pub use metrics::{ChainOrchestratorItem, ChainOrchestratorMetrics}; + +/// The mask used to mask the L1 message queue hash. +const L1_MESSAGE_QUEUE_HASH_MASK: B256 = + b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); + +type Chain = BoundedVec
; + +/// 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 { + /// The `BlockClient` that is used to fetch blocks from peers over p2p. + network_client: Arc, + /// 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>, + /// 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 block number of the L2 finalized block. + l2_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 waker to notify when the engine driver should be polled. + waker: AtomicWaker, +} + +impl< + ChainSpec: ScrollHardforks + EthChainSpec + Send + Sync + 'static, + BC: BlockClient + Send + Sync + 'static, + P: Provider + '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, + ) -> Self { + let chain = init_chain_from_db(&database, &l2_client, chain_buffer_size).await; + 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)), + l2_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, + waker: AtomicWaker::new(), + } + } + + /// 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?; + database.update_l1_messages_with_l2_block(block_info.clone()).await?; + Ok(ChainOrchestratorEvent::L2ChainCommitted(block_info, None, true)) + } + + /// 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().unwrap().number != 0 + { + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(optimistic_headers.first().unwrap().number - 1), "fetching block"); + let parent_hash = optimistic_headers.first().unwrap().parent_hash; + let header = network_client + .get_header(BlockHashOrNumber::Hash(parent_hash)) + .await? + .into_data() + .ok_or(ChainOrchestratorError::MissingBlockHeader { hash: parent_hash })?; + optimistic_headers.push_front(header); + } + + *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, + )); + } + + // 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 = vec![received_block.header.clone()]; + + // We should never have a re-org that is deeper than the current safe head. + let (latest_safe_block, _) = + database.get_latest_safe_l2_info().await?.expect("safe block must exist"); + + // 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.last().unwrap().number < + current_chain_headers.front().unwrap().number + { + return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( + received_block.hash_slow(), + )); + } + + // 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.last().unwrap().number < + current_chain_headers.front().expect("chain can not be empty").number + { + for _ in 0..BATCH_FETCH_SIZE { + if current_chain_headers.front().unwrap().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().unwrap().number - 1), "fetching block for current chain"); + if let Some(block) = l2_client + .get_block_by_hash(current_chain_headers.front().unwrap().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().unwrap().parent_hash, + }); + } + } + } + + // 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.last().unwrap().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).unwrap().timestamp > + received_chain_headers.last().unwrap().timestamp + { + return Ok(ChainOrchestratorEvent::OldForkReceived { + headers: received_chain_headers, + peer_id, + signature, + }); + } + break pos; + } + + // If the current header block number is less than the latest safe block number then + // we should error. + if received_chain_headers.last().unwrap().number <= latest_safe_block.number { + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + } + + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_chain_headers.last().unwrap().number - 1), "fetching block"); + if let Some(header) = network_client + .get_header(BlockHashOrNumber::Hash( + received_chain_headers.last().unwrap().parent_hash, + )) + .await? + .into_data() + { + received_chain_headers.push(header.clone()); + } else { + return Err(ChainOrchestratorError::MissingBlockHeader { + hash: received_chain_headers.last().unwrap().parent_hash, + }); + } + }; + + // Reverse the new chain headers to have them in the correct order. + received_chain_headers.reverse(); + + // 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(), 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.clone()).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, + ))) + } + // 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, + ))) + } + } + } + + /// 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(); + for block in block_infos { + database.insert_block(block, batch_info).await?; + } + Result::<_, ChainOrchestratorError>::Ok(Some( + ChainOrchestratorEvent::L2ConsolidatedBlockCommitted(head), + )) + }), + ); + + self.pending_futures.push_back(ChainOrchestratorFuture::HandleDerivedBlock(fut)); + self.waker.wake(); + } + + /// 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, + ) + .await?; + *optimistic_mode.lock().await = false; + true + } else { + false + }; + + // Insert the blocks into the database. + let head = block_info.last().expect("block info must not be empty").clone(); + database.update_l1_messages_from_l2_blocks(block_info).await?; + + Result::<_, ChainOrchestratorError>::Ok(Some( + ChainOrchestratorEvent::L2ChainCommitted(head, None, consolidated), + )) + }), + ); + + self.pending_futures.push_back(ChainOrchestratorFuture::HandleDerivedBlock(fut)); + self.waker.wake(); + } + + /// 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(), + self.l2_finalized_block_number.clone(), + )), + )) + } + L1Notification::BatchCommit(batch) => { + ChainOrchestratorFuture::HandleBatchCommit(self.handle_metered( + ChainOrchestratorItem::BatchCommit, + Box::pin(Self::handle_batch_commit(self.database.clone(), batch)), + )) + } + L1Notification::L1Message { message, block_number, block_timestamp } => { + ChainOrchestratorFuture::HandleL1Message(self.handle_metered( + ChainOrchestratorItem::L1Message, + Box::pin(Self::handle_l1_message( + self.database.clone(), + self.chain_spec.clone(), + message, + block_number, + block_timestamp, + )), + )) + } + L1Notification::BatchFinalization { hash, block_number, .. } => { + ChainOrchestratorFuture::HandleBatchFinalization(self.handle_metered( + ChainOrchestratorItem::BatchFinalization, + Box::pin(Self::handle_batch_finalization( + self.database.clone(), + hash, + block_number, + self.l1_finalized_block_number.clone(), + self.l2_finalized_block_number.clone(), + )), + )) + } + L1Notification::Synced => { + self.set_l1_synced_status(true); + return + } + }; + + self.pending_futures.push_back(fut); + } + + /// 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>, + ) -> Result, ChainOrchestratorError> { + let txn = database.tx().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.commit().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 + .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 + }; + Ok(Some(ChainOrchestratorEvent::ChainUnwound { + l1_block_number, + queue_index, + l2_head_block_info, + l2_safe_block_info, + })) + } + + /// Handles a finalized event by updating the chain orchestrator L1 finalized block and + /// returning the new finalized L2 chain block. + async fn handle_finalized( + database: Arc, + block_number: u64, + l1_block_number: Arc, + l2_block_number: Arc, + ) -> Result, ChainOrchestratorError> { + // Set the latest finalized L1 block in the database. + database.set_latest_finalized_l1_block_number(block_number).await?; + + // get the newest finalized batch. + let batch_hash = database.get_finalized_batch_hash_at_height(block_number).await?; + + // get the finalized block for the batch. + let finalized_block = if let Some(hash) = batch_hash { + Self::fetch_highest_finalized_block(database, hash, l2_block_number).await? + } else { + None + }; + + // update the chain orchestrator l1 block number. + l1_block_number.store(block_number, Ordering::Relaxed); + + Ok(Some(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_block))) + } + + /// Handles an L1 message by inserting it into the database. + async fn handle_l1_message( + database: Arc, + chain_spec: Arc, + l1_message: TxL1Message, + l1_block_number: u64, + block_timestamp: u64, + ) -> Result, ChainOrchestratorError> { + let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); + + let queue_hash = if chain_spec + .scroll_fork_activation(ScrollHardfork::EuclidV2) + .active_at_timestamp_or_number(block_timestamp, l1_block_number) && + l1_message.queue_index > 0 + { + let index = l1_message.queue_index - 1; + let prev_queue_hash = database + .get_l1_message_by_index(index) + .await? + .map(|m| m.queue_hash) + .ok_or(DatabaseError::L1MessageNotFound(L1MessageStart::Index(index)))?; + + let mut input = prev_queue_hash.unwrap_or_default().to_vec(); + input.append(&mut l1_message.tx_hash().to_vec()); + Some(keccak256(input) & L1_MESSAGE_QUEUE_HASH_MASK) + } else { + None + }; + + let l1_message = L1MessageEnvelope::new(l1_message, l1_block_number, None, queue_hash); + database.insert_l1_message(l1_message).await?; + Ok(Some(event)) + } + + /// Handles a batch input by inserting it into the database. + async fn handle_batch_commit( + database: Arc, + batch: BatchCommitData, + ) -> Result, ChainOrchestratorError> { + let txn = database.tx().await?; + let prev_batch_index = batch.index - 1; + + // remove any batches with an index greater than the previous batch. + let affected = txn.delete_batches_gt_batch_index(prev_batch_index).await?; + + // handle the case of a batch revert. + let new_safe_head = if affected > 0 { + txn.delete_l2_blocks_gt_batch_index(prev_batch_index).await?; + txn.get_highest_block_for_batch_index(prev_batch_index).await? + } else { + None + }; + + let event = ChainOrchestratorEvent::BatchCommitIndexed { + batch_info: BatchInfo::new(batch.index, batch.hash), + l1_block_number: batch.block_number, + safe_head: new_safe_head, + }; + + // insert the batch and commit the transaction. + txn.insert_batch(batch).await?; + txn.commit().await?; + + Ok(Some(event)) + } + + /// Handles a batch finalization event by updating the batch input in the database. + async fn handle_batch_finalization( + database: Arc, + batch_hash: B256, + block_number: u64, + l1_block_number: Arc, + l2_block_number: Arc, + ) -> Result, ChainOrchestratorError> { + // finalized the batch. + database.finalize_batch(batch_hash, block_number).await?; + + // check if the block where the batch was finalized is finalized on L1. + let mut finalized_block = None; + let l1_block_number_value = l1_block_number.load(Ordering::Relaxed); + if l1_block_number_value > block_number { + // fetch the finalized block. + finalized_block = + Self::fetch_highest_finalized_block(database, batch_hash, l2_block_number).await?; + } + + let event = ChainOrchestratorEvent::BatchFinalized(batch_hash, finalized_block); + Ok(Some(event)) + } + + /// Returns the highest finalized block for the provided batch hash. Will return [`None`] if the + /// block number has already been seen by the indexer. + async fn fetch_highest_finalized_block( + database: Arc, + batch_hash: B256, + l2_block_number: Arc, + ) -> Result, ChainOrchestratorError> { + let finalized_block = database.get_highest_block_for_batch_hash(batch_hash).await?; + + // only return the block if the indexer hasn't seen it. + // in which case also update the `l2_finalized_block_number` value. + Ok(finalized_block.filter(|info| { + let current_l2_block_number = l2_block_number.load(Ordering::Relaxed); + if info.number > current_l2_block_number { + l2_block_number.store(info.number, Ordering::Relaxed); + true + } else { + false + } + })) + } +} + +async fn init_chain_from_db + 'static>( + database: &Arc, + l2_client: &P, + chain_buffer_size: usize, +) -> BoundedVec

{ + let blocks = { + let mut blocks = Vec::with_capacity(chain_buffer_size); + let mut blocks_stream = database.get_l2_blocks().await.unwrap().take(chain_buffer_size); + while let Some(block_info) = blocks_stream.try_next().await.unwrap() { + let header = l2_client + .get_block_by_hash(block_info.hash) + .await + .unwrap() + .unwrap() + .header + .into_consensus(); + blocks.push(header); + } + blocks.reverse(); + blocks + }; + let mut chain: Chain = Chain::new(chain_buffer_size); + chain.extend(blocks); + chain +} + +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 + } + }; + } + + Poll::Pending + } +} + +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); + } + + // 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 = database.get_latest_safe_l2_info().await?.expect("safe head must exist").0; + + // 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.clone()).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)); + } + + blocks +} + +/// 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 mut l1_message_stream = database + .get_l1_messages(l1_message_hashes.first().map(|tx| L1MessageStart::Hash(*tx))) + .await?; + + for message_hash in l1_message_hashes { + // Get the expected L1 message from the database. + let expected_hash = l1_message_stream.next().await.unwrap().unwrap().transaction.tx_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(()) +} + +#[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; + + /// 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>, + 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, + ) + .await, + db, + ) + } + + #[tokio::test] + async fn test_handle_commit_batch() { + // Instantiate indexer and db + let (mut indexer, 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); + + let batch_commit = BatchCommitData::arbitrary(&mut u).unwrap(); + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit.clone())); + + let event = indexer.next().await.unwrap().unwrap(); + + // Verify the event structure + match event { + ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { + assert_eq!(batch_info.index, batch_commit.index); + assert_eq!(batch_info.hash, batch_commit.hash); + assert_eq!(safe_head, None); // No safe head since no batch revert + } + _ => panic!("Expected BatchCommitIndexed event"), + } + + let batch_commit_result = db.get_batch_by_index(batch_commit.index).await.unwrap().unwrap(); + assert_eq!(batch_commit, batch_commit_result); + } + + #[tokio::test] + async fn test_handle_batch_commit_with_revert() { + // Instantiate indexer 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); + + // 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 batch_102 = db.get_batch_by_index(102).await.unwrap(); + assert!(batch_102.is_none()); + + // Verify batch 101 was replaced with new data + let updated_batch_101 = db.get_batch_by_index(101).await.unwrap().unwrap(); + assert_eq!(updated_batch_101, new_batch_2); + + // Verify batch 100 still exists + let batch_100 = db.get_batch_by_index(100).await.unwrap(); + assert!(batch_100.is_some()); + } + + #[tokio::test] + async fn test_handle_l1_message() { + // Instantiate indexer and db + let (mut indexer, 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); + + let message = TxL1Message { + queue_index: i64::arbitrary(&mut u).unwrap().unsigned_abs(), + ..Arbitrary::arbitrary(&mut u).unwrap() + }; + let block_number = u64::arbitrary(&mut u).unwrap(); + indexer.handle_l1_notification(L1Notification::L1Message { + message: message.clone(), + block_number, + block_timestamp: 0, + }); + + let _ = indexer.next().await; + + let l1_message_result = + db.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); + let l1_message = L1MessageEnvelope::new(message, block_number, None, None); + + assert_eq!(l1_message, l1_message_result); + } + + #[tokio::test] + async fn test_l1_message_hash_queue() { + // Instantiate indexer and db + let (mut indexer, db) = setup_test_chain_orchestrator().await; + + // insert the previous L1 message in database. + indexer.handle_l1_notification(L1Notification::L1Message { + message: TxL1Message { queue_index: 1062109, ..Default::default() }, + block_number: 1475588, + block_timestamp: 1745305199, + }); + let _ = indexer.next().await.unwrap().unwrap(); + + // + let message = TxL1Message { + queue_index: 1062110, + gas_limit: 168000, + to: address!("Ba50f5340FB9F3Bd074bD638c9BE13eCB36E603d"), + value: U256::ZERO, + sender: address!("61d8d3E7F7c656493d1d76aAA1a836CEdfCBc27b"), + input: bytes!("8ef1332e000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d6996000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d699600000000000000000000000000000000000000000000000000038d7ea4c6800000000000000000000000000000000000000000000000000000000000001034de00000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000"), + }; + indexer.handle_l1_notification(L1Notification::L1Message { + message: message.clone(), + block_number: 14755883, + block_timestamp: 1745305200, + }); + + let _ = indexer.next().await.unwrap().unwrap(); + + let l1_message_result = + db.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); + + assert_eq!( + b256!("5e48ae1092c7f912849b9935f4e66870d2034b24fb2016f506e6754900000000"), + l1_message_result.queue_hash.unwrap() + ); + } + + #[tokio::test] + async fn test_handle_reorg() { + // Instantiate indexer and db + let (mut indexer, 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); + + // 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_20 = BatchCommitData::arbitrary(&mut u).unwrap(); + batch_commit_block_20.block_number = 20; + batch_commit_block_20.index = 20; + let batch_commit_block_20 = batch_commit_block_20; + + let mut batch_commit_block_30 = BatchCommitData::arbitrary(&mut u).unwrap(); + batch_commit_block_30.block_number = 30; + batch_commit_block_30.index = 30; + let batch_commit_block_30 = batch_commit_block_30; + + // Index batch inputs + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_20.clone())); + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_30.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, + ..Arbitrary::arbitrary(&mut u).unwrap() + }; + let l1_message_block_20 = L1MessageEnvelope { + queue_hash: None, + l1_block_number: 20, + l2_block_number: None, + ..Arbitrary::arbitrary(&mut u).unwrap() + }; + let l1_message_block_30 = L1MessageEnvelope { + queue_hash: None, + l1_block_number: 30, + l2_block_number: None, + ..Arbitrary::arbitrary(&mut u).unwrap() + }; + + // Index L1 messages + indexer.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, + }); + indexer.handle_l1_notification(L1Notification::L1Message { + message: l1_message_block_20.clone().transaction, + block_number: l1_message_block_20.clone().l1_block_number, + block_timestamp: 0, + }); + indexer.handle_l1_notification(L1Notification::L1Message { + message: l1_message_block_30.clone().transaction, + block_number: l1_message_block_30.clone().l1_block_number, + block_timestamp: 0, + }); + + // Reorg at block 20 + indexer.handle_l1_notification(L1Notification::Reorg(20)); + + for _ in 0..7 { + indexer.next().await.unwrap().unwrap(); + } + + // Check that the batch input at block 30 is deleted + let batch_commits = + db.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_20)); + + // check that the L1 message at block 30 is deleted + let l1_messages = db + .get_l1_messages(None) + .await + .unwrap() + .map(|res| res.unwrap()) + .collect::>() + .await; + assert_eq!(2, l1_messages.len()); + assert!(l1_messages.contains(&l1_message_block_1)); + assert!(l1_messages.contains(&l1_message_block_20)); + } + + // 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 indexer and db + let (mut indexer, _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 + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); + indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_10.clone())); + for _ in 0..2 { + let _event = indexer.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() + }, + }; + indexer.handle_l1_notification(L1Notification::L1Message { + message: l1_message.transaction.clone(), + block_number: l1_message.l1_block_number, + block_timestamp: 0, + }); + indexer.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 { + indexer.persist_l1_consolidated_blocks(vec![l2_block.clone()], batch_info); + } else { + indexer.consolidate_validated_l2_blocks(vec![l2_block.clone()]); + } + + indexer.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. + indexer.handle_l1_notification(L1Notification::Reorg(17)); + let event = indexer.next().await.unwrap().unwrap(); + assert_eq!( + event, + ChainOrchestratorEvent::ChainUnwound { + 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. + indexer.handle_l1_notification(L1Notification::Reorg(7)); + let event = indexer.next().await.unwrap().unwrap(); + + assert_eq!( + event, + ChainOrchestratorEvent::ChainUnwound { + 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 . + indexer.handle_l1_notification(L1Notification::Reorg(3)); + let event = indexer.next().await.unwrap().unwrap(); + + assert_eq!( + event, + ChainOrchestratorEvent::ChainUnwound { + 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, indexer.chain_spec.genesis_hash())), + } + ); + } +} diff --git a/crates/indexer/src/metrics.rs b/crates/chain-orchestrator/src/metrics.rs similarity index 52% rename from crates/indexer/src/metrics.rs rename to crates/chain-orchestrator/src/metrics.rs index dc25ee0c..c71b4841 100644 --- a/crates/indexer/src/metrics.rs +++ b/crates/chain-orchestrator/src/metrics.rs @@ -2,11 +2,15 @@ use metrics::Histogram; use metrics_derive::Metrics; use strum::EnumIter; -/// An enum representing the items the indexer can handle. +/// An enum representing the items the chain orchestrator can handle. #[derive(Debug, PartialEq, Eq, Hash, EnumIter)] -pub enum IndexerItem { +pub enum ChainOrchestratorItem { + /// Handle a block received from the network. + NewBlock, + /// Insert consolidated L2 blocks into the database. + InsertConsolidatedL2Blocks, /// L2 block. - L2Block, + InsertL2Block, /// L1 reorg. L1Reorg, /// L1 finalization. @@ -19,11 +23,13 @@ pub enum IndexerItem { BatchFinalization, } -impl IndexerItem { - /// Returns the str representation of the [`IndexerItem`]. +impl ChainOrchestratorItem { + /// Returns the str representation of the [`ChainOrchestratorItem`]. pub const fn as_str(&self) -> &'static str { match self { - Self::L2Block => "l2_block", + Self::NewBlock => "new_block", + Self::InsertConsolidatedL2Blocks => "insert_consolidated_l2_blocks", + Self::InsertL2Block => "l2_block", Self::L1Reorg => "l1_reorg", Self::L1Finalization => "l1_finalization", Self::L1Message => "l1_message", @@ -33,10 +39,10 @@ impl IndexerItem { } } -/// The metrics for the [`super::Indexer`]. +/// The metrics for the [`super::ChainOrchestrator`]. #[derive(Metrics, Clone)] #[metrics(scope = "indexer")] -pub struct IndexerMetrics { - /// The duration of the task for the indexer. +pub struct ChainOrchestratorMetrics { + /// The duration of the task for the chain orchestrator. pub task_duration: Histogram, } diff --git a/crates/chain-orchestrator/testdata/genesis_block.json b/crates/chain-orchestrator/testdata/genesis_block.json new file mode 100644 index 00000000..3387b87c --- /dev/null +++ b/crates/chain-orchestrator/testdata/genesis_block.json @@ -0,0 +1,27 @@ +{ + "header": { + "hash": "0xbbc05efd412b7cd47a2ed0e5ddfcf87af251e414ea4c801d78b6784513180a80", + "parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000", + "sha3Uncles": "0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347", + "miner": "0x0000000000000000000000000000000000000000", + "stateRoot": "0x08d535cc60f40af5dd3b31e0998d7567c2d568b224bed2ba26070aeb078d1339", + "transactionsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421", + "receiptsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421", + "logsBloom": "0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + "difficulty": "0x1", + "number": "0x0", + "gasLimit": "0x989680", + "gasUsed": "0x0", + "timestamp": "0x6524e860", + "extraData": "", + "mixHash": "0x0000000000000000000000000000000000000000000000000000000000000000", + "nonce": "0x0000000000000000", + "totalDifficulty": "0x1", + "size": "0x272" + }, + "body": { + "uncles": [], + "transactions": [], + "ommers": [] + } +} \ No newline at end of file diff --git a/crates/chain-orchestrator/testdata/genesis_block_rpc.json b/crates/chain-orchestrator/testdata/genesis_block_rpc.json new file mode 100644 index 00000000..b828bd22 --- /dev/null +++ b/crates/chain-orchestrator/testdata/genesis_block_rpc.json @@ -0,0 +1,22 @@ +{ + "hash": "0xbbc05efd412b7cd47a2ed0e5ddfcf87af251e414ea4c801d78b6784513180a80", + "parentHash": "0x0000000000000000000000000000000000000000000000000000000000000000", + "sha3Uncles": "0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347", + "miner": "0x0000000000000000000000000000000000000000", + "stateRoot": "0x08d535cc60f40af5dd3b31e0998d7567c2d568b224bed2ba26070aeb078d1339", + "transactionsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421", + "receiptsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421", + "logsBloom": "0x00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + "difficulty": "0x1", + "number": "0x0", + "gasLimit": "0x989680", + "gasUsed": "0x0", + "timestamp": "0x6524e860", + "extraData": "0x4c61206573746f6e7465636f206573746173206d616c6665726d6974612e0000d2acf5d16a983db0d909d9d761b8337fabd6cbd10000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + "mixHash": "0x0000000000000000000000000000000000000000000000000000000000000000", + "nonce": "0x0000000000000000", + "totalDifficulty": "0x1", + "size": "0x272", + "uncles": [], + "transactions": [] +} \ No newline at end of file diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index db771488..08cdb016 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -168,7 +168,7 @@ mod test { }, l1_messages: vec![], }; - db.insert_block(block_info, batch_info.into()).await.unwrap(); + db.insert_block(block_info, batch_info).await.unwrap(); block_number += 1; } @@ -212,7 +212,7 @@ mod test { }, l1_messages: vec![], }; - db.insert_block(block_info, first_batch_info.into()).await.unwrap(); + db.insert_block(block_info, first_batch_info).await.unwrap(); block_number += 1; } @@ -325,8 +325,13 @@ mod test { db.insert_l1_message(l1_message_2.clone()).await.unwrap(); // collect the L1Messages - let l1_messages = - db.get_l1_messages().await.unwrap().map(|res| res.unwrap()).collect::>().await; + let l1_messages = db + .get_l1_messages(None) + .await + .unwrap() + .map(|res| res.unwrap()) + .collect::>() + .await; // Apply the assertions. assert!(l1_messages.contains(&l1_message_1)); @@ -397,7 +402,7 @@ mod test { let block_info = BlockInfo { number: i, hash: B256::arbitrary(&mut u).unwrap() }; let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; block_infos.push(block_info); - db.insert_block(l2_block, Some(batch_info)).await.unwrap(); + db.insert_block(l2_block, batch_info).await.unwrap(); } // Test getting existing blocks @@ -422,9 +427,10 @@ mod test { rand::rng().fill(bytes.as_mut_slice()); let mut u = Unstructured::new(&bytes); - // Initially should return None - let latest_safe = db.get_latest_safe_l2_info().await.unwrap(); - assert!(latest_safe.is_none()); + // Initially should return the genesis block and hash. + let (latest_safe_block, batch) = db.get_latest_safe_l2_info().await.unwrap().unwrap(); + assert_eq!(latest_safe_block.number, 0); + assert_eq!(batch.index, 0); // Generate and insert a batch let batch_data = BatchCommitData { index: 100, ..Arbitrary::arbitrary(&mut u).unwrap() }; @@ -437,23 +443,14 @@ mod test { db.insert_block( L2BlockInfoWithL1Messages { block_info: safe_block_1, l1_messages: vec![] }, - Some(batch_info), + batch_info, ) .await .unwrap(); db.insert_block( L2BlockInfoWithL1Messages { block_info: safe_block_2, l1_messages: vec![] }, - Some(batch_info), - ) - .await - .unwrap(); - - // Insert block without batch info (unsafe block) - let unsafe_block = BlockInfo { number: 202, hash: B256::arbitrary(&mut u).unwrap() }; - db.insert_block( - L2BlockInfoWithL1Messages { block_info: unsafe_block, l1_messages: vec![] }, - None, + batch_info, ) .await .unwrap(); @@ -463,32 +460,6 @@ mod test { assert_eq!(latest_safe, Some((safe_block_2, batch_info))); } - #[tokio::test] - async fn test_get_latest_l2_block() { - // Set up the test database. - let db = setup_test_db().await; - - // Generate unstructured bytes. - let mut bytes = [0u8; 1024]; - rand::rng().fill(bytes.as_mut_slice()); - let mut u = Unstructured::new(&bytes); - - // Insert multiple blocks with increasing block numbers - let mut latest_block = BlockInfo { number: 0, hash: B256::ZERO }; - for i in 300..305 { - let block_info = BlockInfo { number: i, hash: B256::arbitrary(&mut u).unwrap() }; - latest_block = block_info; - - db.insert_block(L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }, None) - .await - .unwrap(); - } - - // Should return the block with highest number - let retrieved_latest = db.get_latest_l2_block().await.unwrap(); - assert_eq!(retrieved_latest, Some(latest_block)); - } - #[tokio::test] async fn test_delete_l2_blocks_gt_block_number() { // Set up the test database. @@ -499,13 +470,17 @@ mod test { rand::rng().fill(bytes.as_mut_slice()); let mut u = Unstructured::new(&bytes); - // Insert multiple L2 blocks + // Insert multiple L2 blocks with batch info + let batch_info = BatchInfo { index: 0, hash: B256::default() }; for i in 400..410 { let block_info = BlockInfo { number: i, hash: B256::arbitrary(&mut u).unwrap() }; - db.insert_block(L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }, None) - .await - .unwrap(); + db.insert_block( + L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }, + batch_info, + ) + .await + .unwrap(); } // Delete blocks with number > 405 @@ -553,15 +528,7 @@ mod test { let block_info = BlockInfo { number: 500 + i, hash: B256::arbitrary(&mut u).unwrap() }; let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - db.insert_block(l2_block, Some(batch_info)).await.unwrap(); - } - - // Insert some blocks without batch index (should not be deleted) - for i in 0..3 { - let block_info = BlockInfo { number: 600 + i, hash: B256::arbitrary(&mut u).unwrap() }; - let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - - db.insert_block(l2_block, None).await.unwrap(); + db.insert_block(l2_block, batch_info).await.unwrap(); } // Delete L2 blocks with batch index > 105 @@ -617,7 +584,8 @@ mod test { L2BlockInfoWithL1Messages { block_info, l1_messages: l1_message_hashes.clone() }; // Insert block - db.insert_block(l2_block, Some(batch_info)).await.unwrap(); + db.insert_block(l2_block.clone(), batch_info).await.unwrap(); + db.update_l1_messages_with_l2_block(l2_block).await.unwrap(); // Verify block was inserted let retrieved_block = db.get_l2_block_info_by_number(500).await.unwrap(); @@ -653,7 +621,7 @@ mod test { let block_info = BlockInfo { number: 600, hash: B256::arbitrary(&mut u).unwrap() }; let l2_block = L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }; - db.insert_block(l2_block, Some(batch_info_1)).await.unwrap(); + db.insert_block(l2_block, batch_info_1).await.unwrap(); // Verify initial insertion let retrieved_block = db.get_l2_block_info_by_number(600).await.unwrap(); @@ -666,15 +634,15 @@ mod test { .await .unwrap() .unwrap(); - let (initial_block_info, initial_batch_info): (BlockInfo, Option) = + let (initial_block_info, initial_batch_info): (BlockInfo, BatchInfo) = initial_l2_block_model.into(); assert_eq!(initial_block_info, block_info); - assert_eq!(initial_batch_info, Some(batch_info_1)); + 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![] }; - db.insert_block(updated_l2_block, Some(batch_info_2)).await.unwrap(); + db.insert_block(updated_l2_block, batch_info_2).await.unwrap(); // Verify the block still exists and was updated let retrieved_block = db.get_l2_block_info_by_number(600).await.unwrap().unwrap(); @@ -687,9 +655,9 @@ mod test { .await .unwrap() .unwrap(); - let (updated_block_info, updated_batch_info): (BlockInfo, Option) = + let (updated_block_info, updated_batch_info): (BlockInfo, BatchInfo) = updated_l2_block_model.into(); assert_eq!(updated_block_info, block_info); - assert_eq!(updated_batch_info, Some(batch_info_2)); + assert_eq!(updated_batch_info, batch_info_2); } } diff --git a/crates/database/db/src/error.rs b/crates/database/db/src/error.rs index 22388746..a0b1d5b2 100644 --- a/crates/database/db/src/error.rs +++ b/crates/database/db/src/error.rs @@ -1,3 +1,4 @@ +use super::L1MessageStart; use alloy_eips::BlockId; use alloy_primitives::B256; @@ -15,5 +16,5 @@ pub enum DatabaseError { BlockNotFound(BlockId), /// The L1 message was not found in database. #[error("L1 message at index [{0}] not found in database")] - L1MessageNotFound(u64), + L1MessageNotFound(L1MessageStart), } diff --git a/crates/database/db/src/lib.rs b/crates/database/db/src/lib.rs index 3022048b..0184b06a 100644 --- a/crates/database/db/src/lib.rs +++ b/crates/database/db/src/lib.rs @@ -13,7 +13,7 @@ mod models; pub use models::*; mod operations; -pub use operations::{DatabaseOperations, UnwindResult}; +pub use operations::{DatabaseOperations, L1MessageStart, UnwindResult}; mod transaction; pub use transaction::DatabaseTransaction; diff --git a/crates/database/db/src/models/l2_block.rs b/crates/database/db/src/models/l2_block.rs index a336e131..21ac8ead 100644 --- a/crates/database/db/src/models/l2_block.rs +++ b/crates/database/db/src/models/l2_block.rs @@ -9,8 +9,8 @@ pub struct Model { #[sea_orm(primary_key)] block_number: i64, block_hash: Vec, - batch_index: Option, - batch_hash: Option>, + batch_index: i64, + batch_hash: Vec, } impl Model { @@ -18,12 +18,8 @@ impl Model { BlockInfo { number: self.block_number as u64, hash: B256::from_slice(&self.block_hash) } } - pub(crate) fn batch_info(&self) -> Option { - self.batch_hash.as_ref().map(|hash| BatchInfo { - index: self.batch_index.expect("batch index must be present if batch hash is present") - as u64, - hash: B256::from_slice(hash), - }) + pub(crate) fn batch_info(&self) -> BatchInfo { + BatchInfo { index: self.batch_index as u64, hash: B256::from_slice(&self.batch_hash) } } } @@ -50,32 +46,32 @@ impl Related for Entity { /// The active model behavior for the batch input model. impl ActiveModelBehavior for ActiveModel {} -impl From<(BlockInfo, Option)> for ActiveModel { - fn from((block_info, batch_info): (BlockInfo, Option)) -> Self { +impl From<(BlockInfo, BatchInfo)> for ActiveModel { + fn from((block_info, batch_info): (BlockInfo, BatchInfo)) -> Self { Self { block_number: ActiveValue::Set( block_info.number.try_into().expect("block number should fit in i64"), ), block_hash: ActiveValue::Set(block_info.hash.to_vec()), batch_index: ActiveValue::Set( - batch_info.map(|x| x.index.try_into().expect("index should fit in i64")), + batch_info.index.try_into().expect("index should fit in i64"), ), - batch_hash: ActiveValue::Set(batch_info.map(|x| x.hash.to_vec())), + batch_hash: ActiveValue::Set(batch_info.hash.to_vec()), } } } -impl From for (BlockInfo, Option) { +impl From for (BlockInfo, BatchInfo) { fn from(value: Model) -> Self { ( BlockInfo { number: value.block_number as u64, hash: B256::from_slice(&value.block_hash), }, - value.batch_hash.map(|b| BatchInfo { - index: value.batch_index.unwrap() as u64, - hash: B256::from_slice(&b), - }), + BatchInfo { + index: value.batch_index as u64, + hash: B256::from_slice(&value.batch_hash), + }, ) } } diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 65fb9a55..f647f271 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -12,6 +12,7 @@ use sea_orm::{ ActiveModelTrait, ColumnTrait, Condition, EntityTrait, QueryFilter, QueryOrder, QuerySelect, Set, }; +use std::fmt; /// The [`DatabaseOperations`] trait provides methods for interacting with the database. #[async_trait::async_trait] @@ -243,12 +244,30 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|x| x.map(Into::into))?) } - /// Gets an iterator over all [`L1MessageEnvelope`]s in the database. + /// Get an iterator over all [`L1MessageEnvelope`]s in the database starting from the provided + /// `start` point. async fn get_l1_messages<'a>( &'a self, + start: Option, ) -> Result> + 'a, DatabaseError> { + let queue_index = match start { + Some(L1MessageStart::Index(i)) => i, + Some(L1MessageStart::Hash(ref h)) => { + // Lookup message by hash + let record = models::l1_message::Entity::find() + .filter(models::l1_message::Column::Hash.eq(h.to_vec())) + .one(self.get_connection()) + .await? + .ok_or_else(|| DatabaseError::L1MessageNotFound(L1MessageStart::Hash(*h)))?; + + record.queue_index as u64 + } + None => 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)?))) @@ -266,6 +285,24 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|x| x.map(Into::into))?) } + /// Get the [`BlockInfo`] and optional [`BatchInfo`] for the provided block hash. + async fn get_l2_block_and_batch_info_by_hash( + &self, + block_hash: B256, + ) -> Result, DatabaseError> { + tracing::trace!(target: "scroll::db", ?block_hash, "Fetching L2 block and batch info by hash from database."); + Ok(models::l2_block::Entity::find() + .filter(models::l2_block::Column::BlockHash.eq(block_hash.to_vec())) + .one(self.get_connection()) + .await + .map(|x| { + x.map(|x| { + let (block_info, batch_info): (BlockInfo, BatchInfo) = x.into(); + (block_info, batch_info) + }) + })?) + } + /// Get a [`BlockInfo`] from the database by its block number. async fn get_l2_block_info_by_number( &self, @@ -277,7 +314,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .await .map(|x| { x.map(|x| { - let (block_info, _maybe_batch_info): (BlockInfo, Option) = x.into(); + let (block_info, _maybe_batch_info): (BlockInfo, BatchInfo) = x.into(); block_info }) })?) @@ -293,25 +330,19 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .order_by_desc(models::l2_block::Column::BlockNumber) .one(self.get_connection()) .await - .map(|x| { - x.map(|x| { - ( - x.block_info(), - x.batch_info() - .expect("Batch info must be present due to database query arguments"), - ) - }) - })?) + .map(|x| x.map(|x| (x.block_info(), x.batch_info())))?) } - /// Get the latest L2 [`BlockInfo`] from the database. - async fn get_latest_l2_block(&self) -> Result, DatabaseError> { - tracing::trace!(target: "scroll::db", "Fetching latest L2 block from database."); + /// Get an iterator over all L2 blocks in the database starting from the most recent one. + async fn get_l2_blocks<'a>( + &'a self, + ) -> Result> + 'a, DatabaseError> { + tracing::trace!(target: "scroll::db", "Fetching L2 blocks from database."); Ok(models::l2_block::Entity::find() .order_by_desc(models::l2_block::Column::BlockNumber) - .one(self.get_connection()) - .await - .map(|x| x.map(|x| x.block_info()))?) + .stream(self.get_connection()) + .await? + .map(|res| Ok(res.map(|res| res.block_info())?))) } /// Prepare the database on startup and return metadata used for other components in the @@ -382,16 +413,29 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|x| x.rows_affected)?) } + /// Insert multiple blocks into the database. + async fn insert_blocks( + &self, + blocks: Vec, + batch_info: BatchInfo, + ) -> Result<(), DatabaseError> { + for block in blocks { + self.insert_block(block, batch_info).await?; + } + Ok(()) + } + /// Insert a new block in the database. async fn insert_block( &self, block_info: L2BlockInfoWithL1Messages, - batch_info: Option, + batch_info: BatchInfo, ) -> Result<(), DatabaseError> { + // We only insert safe blocks into the database, we do not persist unsafe blocks. tracing::trace!( target: "scroll::db", - batch_hash = ?batch_info.as_ref().map(|b| b.hash), - batch_index = batch_info.as_ref().map(|b| b.index), + batch_hash = ?batch_info.hash, + batch_index = batch_info.index, block_number = block_info.block_info.number, block_hash = ?block_info.block_info.hash, "Inserting block into database." @@ -410,6 +454,25 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .exec(self.get_connection()) .await?; + Ok(()) + } + + /// Update the executed L1 messages from the provided L2 blocks in the database. + async fn update_l1_messages_from_l2_blocks( + &self, + blocks: Vec, + ) -> Result<(), DatabaseError> { + for block in blocks { + self.update_l1_messages_with_l2_block(block).await?; + } + Ok(()) + } + + /// Update the executed L1 messages with the provided L2 block number in the database. + async fn update_l1_messages_with_l2_block( + &self, + block_info: L2BlockInfoWithL1Messages, + ) -> Result<(), DatabaseError> { tracing::trace!( target: "scroll::db", block_number = block_info.block_info.number, @@ -488,15 +551,14 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .sort_by(|a, b| a.transaction.queue_index.cmp(&b.transaction.queue_index)); // check if we need to reorg the L2 head and delete some L2 blocks - let (queue_index, l2_head_block_info) = + let (queue_index, l2_head_block_number) = if let Some(msg) = removed_executed_l1_messages.first() { let l2_reorg_block_number = msg .l2_block_number .expect("we guarantee that this is Some(u64) due to the filter above") .saturating_sub(1); - let l2_block_info = self.get_l2_block_info_by_number(l2_reorg_block_number).await?; - self.delete_l2_blocks_gt_block_number(l2_reorg_block_number).await?; - (Some(msg.transaction.queue_index), l2_block_info) + + (Some(msg.transaction.queue_index), Some(l2_reorg_block_number)) } else { (None, None) }; @@ -513,7 +575,28 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { }; // commit the transaction - Ok(UnwindResult { l1_block_number, queue_index, l2_head_block_info, l2_safe_block_info }) + Ok(UnwindResult { l1_block_number, queue_index, l2_head_block_number, l2_safe_block_info }) + } +} + +/// This type defines the start of an L1 message stream. +/// +/// It can either be an index, which is the queue index of the first message to return, or a hash, +/// which is the hash of the first message to return. +#[derive(Debug, Clone)] +pub enum L1MessageStart { + /// Start from the provided queue index. + Index(u64), + /// Start from the provided queue hash. + Hash(B256), +} + +impl fmt::Display for L1MessageStart { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Index(index) => write!(f, "Index({index})"), + Self::Hash(hash) => write!(f, "Hash({hash:#x})"), + } } } @@ -524,8 +607,9 @@ pub struct UnwindResult { pub l1_block_number: u64, /// The latest unconsumed queue index after the uwnind. pub queue_index: Option, - /// The L2 head block info after the unwind. This is only populated if the L2 head has reorged. - pub l2_head_block_info: Option, + /// The L2 head block number after the unwind. This is only populated if the L2 head has + /// reorged. + pub l2_head_block_number: Option, /// The L2 safe block info after the unwind. This is only populated if the L2 safe has reorged. pub l2_safe_block_info: Option, } diff --git a/crates/database/migration/Cargo.toml b/crates/database/migration/Cargo.toml index 9abab627..872a070e 100644 --- a/crates/database/migration/Cargo.toml +++ b/crates/database/migration/Cargo.toml @@ -16,6 +16,7 @@ reth-chainspec.workspace = true sea-orm = { workspace = true, features = ["sqlx-sqlite", "runtime-tokio-native-tls", "macros"] } sha2 = "0.10.9" tracing.workspace = true +reth-scroll-chainspec.workspace = true [dependencies.sea-orm-migration] version = "1.1.0" diff --git a/crates/database/migration/src/lib.rs b/crates/database/migration/src/lib.rs index e4733d5b..d64688d0 100644 --- a/crates/database/migration/src/lib.rs +++ b/crates/database/migration/src/lib.rs @@ -19,7 +19,7 @@ impl MigratorTrait for Migrator { Box::new(m20250304_125946_add_l1_msg_table::Migration), Box::new(m20250408_132123_add_header_metadata::Migration), Box::new(m20250408_150338_load_header_metadata::Migration::(Default::default())), - Box::new(m20250411_072004_add_l2_block::Migration), + Box::new(m20250411_072004_add_l2_block::Migration::(Default::default())), Box::new(m20250616_223947_add_metadata::Migration), ] } diff --git a/crates/database/migration/src/m20220101_000001_create_batch_commit_table.rs b/crates/database/migration/src/m20220101_000001_create_batch_commit_table.rs index 1ee03466..e85a9d07 100644 --- a/crates/database/migration/src/m20220101_000001_create_batch_commit_table.rs +++ b/crates/database/migration/src/m20220101_000001_create_batch_commit_table.rs @@ -1,3 +1,4 @@ +use sea_orm::Statement; use sea_orm_migration::{prelude::*, schema::*}; // TODO: migrate these to a constants module @@ -23,7 +24,29 @@ impl MigrationTrait for Migration { .col(big_unsigned_null(BatchCommit::FinalizedBlockNumber)) .to_owned(), ) - .await + .await?; + + manager + .get_connection() + .execute(Statement::from_sql_and_values( + manager.get_database_backend(), + r#" + INSERT INTO batch_commit ("index", hash, block_number, block_timestamp, calldata, blob_hash, finalized_block_number) + VALUES (?, ?, ?, ?, ?, ?, ?) + "#, + vec![ + 0u64.into(), + vec![0u8; HASH_LENGTH as usize].into(), + 0u64.into(), + 0u64.into(), + vec![].into(), + None::>.into(), + 0u64.into(), + ], + )) + .await?; + + Ok(()) } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { diff --git a/crates/database/migration/src/m20250411_072004_add_l2_block.rs b/crates/database/migration/src/m20250411_072004_add_l2_block.rs index c8e1582d..82c39947 100644 --- a/crates/database/migration/src/m20250411_072004_add_l2_block.rs +++ b/crates/database/migration/src/m20250411_072004_add_l2_block.rs @@ -1,12 +1,18 @@ -use super::m20220101_000001_create_batch_commit_table::BatchCommit; +use super::{m20220101_000001_create_batch_commit_table::BatchCommit, MigrationInfo}; +use sea_orm::Statement; use sea_orm_migration::{prelude::*, schema::*}; -#[derive(DeriveMigrationName)] -pub struct Migration; +pub struct Migration(pub std::marker::PhantomData); + +impl MigrationName for Migration { + fn name(&self) -> &str { + sea_orm_migration::util::get_file_stem(file!()) + } +} #[async_trait::async_trait] -impl MigrationTrait for Migration { +impl MigrationTrait for Migration { async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { manager .create_table( @@ -15,27 +21,44 @@ impl MigrationTrait for Migration { .if_not_exists() .col(pk_auto(L2Block::BlockNumber)) .col(binary_len(L2Block::BlockHash, 32)) - .col(big_unsigned_null(L2Block::BatchIndex)) - .col(binary_len_null(L2Block::BatchHash, 32)) + .col(big_unsigned(L2Block::BatchIndex)) + .col(binary_len(L2Block::BatchHash, 32)) .foreign_key( ForeignKey::create() .name("fk_batch_index") .from(L2Block::Table, L2Block::BatchIndex) .to(BatchCommit::Table, BatchCommit::Index) - .on_delete(ForeignKeyAction::SetNull) - .on_update(ForeignKeyAction::SetNull), + .on_delete(ForeignKeyAction::Cascade) + .on_update(ForeignKeyAction::Cascade), ) .foreign_key( ForeignKey::create() .name("fk_batch_hash") .from(L2Block::Table, L2Block::BatchHash) .to(BatchCommit::Table, BatchCommit::Hash) - .on_delete(ForeignKeyAction::SetNull) - .on_update(ForeignKeyAction::SetNull), + .on_delete(ForeignKeyAction::Cascade) + .on_update(ForeignKeyAction::Cascade), ) .to_owned(), ) - .await + .await?; + + // Insert the genesis block. + let genesis_hash = MI::genesis_hash(); + + manager + .get_connection() + .execute(Statement::from_sql_and_values( + manager.get_database_backend(), + r#" + INSERT INTO l2_block (block_number, block_hash, batch_index, batch_hash) + VALUES (?, ?, ?, ?) + "#, + vec![0u64.into(), genesis_hash.to_vec().into(), 0u64.into(), vec![0u8; 32].into()], + )) + .await?; + + Ok(()) } async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { diff --git a/crates/database/migration/src/migration_info.rs b/crates/database/migration/src/migration_info.rs index 38615f2c..6ed32c46 100644 --- a/crates/database/migration/src/migration_info.rs +++ b/crates/database/migration/src/migration_info.rs @@ -1,4 +1,5 @@ use alloy_primitives::{b256, B256}; +use reth_scroll_chainspec::{SCROLL_MAINNET_GENESIS_HASH, SCROLL_SEPOLIA_GENESIS_HASH}; pub enum DataSource { Url(String), @@ -8,6 +9,7 @@ pub enum DataSource { pub trait MigrationInfo { fn data_source() -> Option; fn data_hash() -> Option; + fn genesis_hash() -> B256; } impl MigrationInfo for () { @@ -18,6 +20,11 @@ impl MigrationInfo for () { fn data_hash() -> Option { None } + + fn genesis_hash() -> B256 { + // Todo: Update + b256!("0xc77ee681dac901672fee660088df30ef11789ec89837123cdc89690ef1fef766") + } } /// The type implementing migration info for Mainnet. @@ -33,6 +40,10 @@ impl MigrationInfo for ScrollMainnetMigrationInfo { fn data_hash() -> Option { Some(b256!("fa2746026ec9590e37e495cb20046e20a38fd0e7099abd2012640dddf6c88b25")) } + + fn genesis_hash() -> B256 { + SCROLL_MAINNET_GENESIS_HASH + } } pub struct ScrollMainnetTestMigrationInfo; @@ -45,6 +56,10 @@ impl MigrationInfo for ScrollMainnetTestMigrationInfo { fn data_hash() -> Option { None } + + fn genesis_hash() -> B256 { + SCROLL_MAINNET_GENESIS_HASH + } } /// The type implementing migration info for Sepolia. @@ -60,4 +75,8 @@ impl MigrationInfo for ScrollSepoliaMigrationInfo { fn data_hash() -> Option { Some(b256!("a02354c12ca0f918bf4768255af9ed13c137db7e56252348f304b17bb4088924")) } + + fn genesis_hash() -> B256 { + SCROLL_SEPOLIA_GENESIS_HASH + } } diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs index 73d97833..3f920e7c 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -7,12 +7,13 @@ use crate::{ use alloy_provider::Provider; use futures::{ready, task::AtomicWaker, FutureExt, Stream}; -use rollup_node_primitives::{BlockInfo, MeteredFuture, ScrollPayloadAttributesWithBatchInfo}; +use rollup_node_primitives::{ + BlockInfo, ChainImport, MeteredFuture, ScrollPayloadAttributesWithBatchInfo, +}; 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::NewBlockWithPeer; use std::{ collections::VecDeque, pin::Pin, @@ -34,14 +35,14 @@ pub struct EngineDriver { fcs: ForkchoiceState, /// Whether the EN is syncing. syncing: bool, - /// The gap between EN and tip of chain which triggers optimistic sync. - block_gap_sync_trigger: u64, /// 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. - block_imports: VecDeque, + 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. @@ -67,7 +68,6 @@ where provider: Option

, fcs: ForkchoiceState, sync_at_start_up: bool, - block_gap_sync_trigger: u64, block_building_duration: Duration, ) -> Self { Self { @@ -77,9 +77,9 @@ where fcs, block_building_duration, syncing: sync_at_start_up, - block_gap_sync_trigger, l1_payload_attributes: VecDeque::new(), - block_imports: VecDeque::new(), + chain_imports: VecDeque::new(), + optimistic_sync_target: None, sequencer_payload_attributes: None, payload_building_future: None, engine_future: None, @@ -122,17 +122,24 @@ where } /// Handles a block import request by adding it to the queue and waking up the driver. - pub fn handle_block_import(&mut self, block_with_peer: NewBlockWithPeer) { - tracing::trace!(target: "scroll::engine", ?block_with_peer, "new block import request received"); - - // Check diff between EN fcs and P2P network tips. - let en_block_number = self.fcs.head_block_info().number; - let p2p_block_number = block_with_peer.block.header.number; - if p2p_block_number.saturating_sub(en_block_number) > self.block_gap_sync_trigger { - self.syncing = true - } + pub fn handle_chain_import(&mut self, chain_import: ChainImport) { + tracing::trace!(target: "scroll::engine", head = %chain_import.chain.last().unwrap().hash_slow(), "new block import request received"); - self.block_imports.push_back(block_with_peer); + 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(); } @@ -216,7 +223,13 @@ where return Some(EngineDriverEvent::L1BlockConsolidated(consolidation_outcome)) } Err(err) => { - tracing::error!(target: "scroll::engine", ?err, "failed to consolidate block derived from L1") + 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(); + } } } } @@ -238,12 +251,25 @@ where } Err(err) => { tracing::error!(target: "scroll::engine", ?err, "failed to build new payload"); - if let EngineDriverError::MissingPayloadId(attributes) = err { - self.l1_payload_attributes.push_front(attributes); + if let EngineDriverError::PayloadBuildingMissingPayloadId(attributes) = err + { + self.sequencer_payload_attributes = Some(attributes); } } } } + EngineDriverFutureResult::OptimisticSync(result) => { + tracing::info!(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 @@ -260,8 +286,8 @@ where } /// Returns the forkchoice state. - pub fn forkchoice_state(&self) -> ForkchoiceState { - self.fcs.clone() + pub const fn forkchoice_state(&self) -> &ForkchoiceState { + &self.fcs } /// Returns the alloy forkchoice state. @@ -316,6 +342,13 @@ where } Err(err) => { tracing::error!(target: "scroll::engine", ?err, "failed to build new payload"); + + if let EngineDriverError::PayloadBuildingMissingPayloadId(attributes) = err + { + tracing::info!(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. @@ -344,13 +377,23 @@ where return Poll::Pending; } - // Handle the block import requests. - if let Some(block_with_peer) = this.block_imports.pop_front() { + // 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 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::block_import(client, block_with_peer, fcs))); + Some(MeteredFuture::new(EngineFuture::chain_import(client, chain_import, fcs))); this.waker.wake(); return Poll::Pending; @@ -414,15 +457,8 @@ mod tests { 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, - 0, - duration, - ); + let mut driver = + EngineDriver::new(client, chain_spec, None::, fcs, false, duration); // Initially, it should be false assert!(!driver.is_payload_building_in_progress()); @@ -448,7 +484,6 @@ mod tests { None::, fcs, false, - 0, duration, ); diff --git a/crates/engine/src/error.rs b/crates/engine/src/error.rs index 428cb68f..72e4cca8 100644 --- a/crates/engine/src/error.rs +++ b/crates/engine/src/error.rs @@ -1,6 +1,7 @@ use alloy_rpc_types_engine::PayloadError; use rollup_node_primitives::ScrollPayloadAttributesWithBatchInfo; use scroll_alloy_provider::ScrollEngineApiError; +use scroll_alloy_rpc_types_engine::ScrollPayloadAttributes; /// The error type for the engine API. #[derive(Debug, thiserror::Error)] @@ -17,7 +18,12 @@ pub enum EngineDriverError { /// The forkchoice update failed. #[error("Forkchoice update failed: {0}")] ForkchoiceUpdateFailed(ScrollEngineApiError), - /// The payload id field is missing in the forkchoice update response. - #[error("Forkchoice update response missing payload id")] - MissingPayloadId(ScrollPayloadAttributesWithBatchInfo), + /// 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(ScrollPayloadAttributesWithBatchInfo), + /// 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), } diff --git a/crates/engine/src/event.rs b/crates/engine/src/event.rs index 1d9f1ab4..6cbfda0f 100644 --- a/crates/engine/src/event.rs +++ b/crates/engine/src/event.rs @@ -9,6 +9,20 @@ pub enum EngineDriverEvent { 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/future/mod.rs b/crates/engine/src/future/mod.rs index 8c83b097..bea06980 100644 --- a/crates/engine/src/future/mod.rs +++ b/crates/engine/src/future/mod.rs @@ -3,20 +3,21 @@ use crate::{api::*, ForkchoiceState}; use alloy_provider::Provider; use alloy_rpc_types_engine::{ - ExecutionData, ExecutionPayloadV1, ForkchoiceState as AlloyForkchoiceState, PayloadStatusEnum, + 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, L2BlockInfoWithL1Messages, MeteredFuture, + BatchInfo, BlockInfo, ChainImport, L2BlockInfoWithL1Messages, MeteredFuture, ScrollPayloadAttributesWithBatchInfo, }; 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, NewBlockWithPeer}; +use scroll_network::BlockImportOutcome; use std::{ future::Future, pin::Pin, @@ -30,7 +31,7 @@ mod result; pub(crate) use result::EngineDriverFutureResult; /// A future that represents a block import job. -type BlockImportFuture = Pin< +type ChainImportFuture = Pin< Box< dyn Future< Output = Result< @@ -41,6 +42,10 @@ type BlockImportFuture = Pin< >, >; +/// 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)] pub enum ConsolidationOutcome { @@ -77,10 +82,6 @@ impl ConsolidationOutcome { } } -/// A future that represents an L1 consolidation job. -type L1ConsolidationFuture = - Pin> + Send>>; - /// A future that represents a new payload processing. type NewPayloadFuture = Pin> + Send>>; @@ -89,25 +90,36 @@ type NewPayloadFuture = 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 { - BlockImport(BlockImportFuture), + ChainImport(ChainImportFuture), L1Consolidation(L1ConsolidationFuture), NewPayload(NewPayloadFuture), + OptimisticSync(OptimisticSyncFuture), } impl EngineFuture { - /// Creates a new [`EngineFuture::BlockImport`] future from the provided parameters. - pub(crate) fn block_import( + pub(crate) fn chain_import( client: Arc, - block_with_peer: NewBlockWithPeer, + chain_import: ChainImport, fcs: AlloyForkchoiceState, ) -> Self where EC: ScrollEngineApi + Unpin + Send + Sync + 'static, { - Self::BlockImport(Box::pin(handle_execution_payload(client, block_with_peer, fcs))) + Self::ChainImport(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. @@ -150,9 +162,10 @@ impl Future for EngineFuture { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); match this { - Self::BlockImport(fut) => fut.as_mut().poll(cx).map(Into::into), + Self::ChainImport(fut) => fut.as_mut().poll(cx).map(Into::into), Self::L1Consolidation(fut) => fut.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), } } } @@ -162,42 +175,48 @@ impl Future for EngineFuture { /// - Sets the current fork choice for the EL via `engine_forkchoiceUpdatedV1`. #[instrument(skip_all, level = "trace", fields( - peer_id = %block_with_peer.peer_id, - block_hash = %block_with_peer.block.hash_slow(), + peer_id = %chain_import.peer_id, + block_hash = %chain_import.chain.last().unwrap().hash_slow(), fcs = ?fcs ) )] -async fn handle_execution_payload( +async fn handle_chain_import( client: Arc, - block_with_peer: NewBlockWithPeer, + 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, ?block_with_peer, "handling execution payload"); + tracing::trace!(target: "scroll::engine::future", ?fcs, ?chain_import.peer_id, chain = ?chain_import.chain.last().unwrap().hash_slow(), "handling execution payload"); - // Unpack the block with peer. - let NewBlockWithPeer { peer_id, block, signature } = block_with_peer; + let ChainImport { chain, peer_id, signature } = chain_import; - // Extract the block info from the payload. - let block_info: BlockInfo = (&block).into(); + // Extract the block info from the last payload. + let head = chain.last().unwrap().clone(); - // Create the execution payload. - let payload = ExecutionPayloadV1::from_block_slow(&block); + 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 payload_status = new_payload(client.clone(), payload).await?; + // 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 { validation_error } = payload_status.clone() { - tracing::error!(target: "scroll::engine", ?validation_error, "execution payload is invalid"); + // 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)); + } - // If the payload is invalid, return early. - return Ok((None, Some(BlockImportOutcome::invalid_block(peer_id)), payload_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. @@ -209,7 +228,7 @@ where Some(block_info), Some(BlockImportOutcome::valid_block( peer_id, - block, + head, Into::>::into(signature).into(), )), PayloadStatusEnum::Valid, @@ -288,9 +307,9 @@ where // retrieve the execution payload let execution_payload = get_payload( client.clone(), - fc_updated - .payload_id - .ok_or(EngineDriverError::MissingPayloadId(payload_attributes_with_batch_info))?, + fc_updated.payload_id.ok_or(EngineDriverError::L1ConsolidationMissingPayloadId( + payload_attributes_with_batch_info, + ))?, ) .await?; // issue the execution payload to the EL @@ -325,7 +344,8 @@ where 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)).await?; + 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; @@ -333,7 +353,9 @@ where // retrieve the execution payload let payload = get_payload( client.clone(), - fc_updated.payload_id.expect("payload attributes has been set"), + fc_updated + .payload_id + .ok_or(EngineDriverError::PayloadBuildingMissingPayloadId(payload_attributes))?, ) .await?; let block = try_into_block(ExecutionData { payload, sidecar: Default::default() }, chain_spec)?; diff --git a/crates/engine/src/future/result.rs b/crates/engine/src/future/result.rs index ffd113ba..e1b966ad 100644 --- a/crates/engine/src/future/result.rs +++ b/crates/engine/src/future/result.rs @@ -10,6 +10,7 @@ pub(crate) enum EngineDriverFutureResult { ), L1Consolidation(Result), PayloadBuildingJob(Result), + OptimisticSync(Result), } impl @@ -41,3 +42,9 @@ impl From> for EngineDriverFutureResult { Self::PayloadBuildingJob(value) } } + +impl From> for EngineDriverFutureResult { + fn from(value: Result) -> Self { + Self::OptimisticSync(value) + } +} diff --git a/crates/indexer/src/action.rs b/crates/indexer/src/action.rs deleted file mode 100644 index 6c889925..00000000 --- a/crates/indexer/src/action.rs +++ /dev/null @@ -1,53 +0,0 @@ -use super::{IndexerError, IndexerEvent}; -use std::{ - fmt, - future::Future, - pin::Pin, - task::{Context, Poll}, -}; - -/// A future that resolves to a `Result`. -pub(super) type PendingIndexerFuture = - Pin> + Send>>; - -/// A type that represents a future that is being executed by the indexer. -pub(super) enum IndexerFuture { - HandleReorg(PendingIndexerFuture), - HandleFinalized(PendingIndexerFuture), - HandleBatchCommit(PendingIndexerFuture), - HandleBatchFinalization(PendingIndexerFuture), - HandleL1Message(PendingIndexerFuture), - HandleDerivedBlock(PendingIndexerFuture), -} - -impl IndexerFuture { - /// Polls the future to completion. - pub(super) fn poll( - &mut self, - cx: &mut Context<'_>, - ) -> Poll> { - match self { - Self::HandleReorg(fut) | - Self::HandleFinalized(fut) | - Self::HandleBatchCommit(fut) | - Self::HandleBatchFinalization(fut) | - Self::HandleL1Message(fut) | - Self::HandleDerivedBlock(fut) => fut.as_mut().poll(cx), - } - } -} - -// We implement the Debug trait for IndexerFuture to provide a human-readable representation of the -// enum variants. -impl fmt::Debug for IndexerFuture { - 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"), - } - } -} diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs deleted file mode 100644 index 5552440f..00000000 --- a/crates/indexer/src/error.rs +++ /dev/null @@ -1,12 +0,0 @@ -use scroll_db::DatabaseError; - -/// A type that represents an error that occurred during indexing. -#[derive(Debug, thiserror::Error)] -pub enum IndexerError { - /// An error occurred while interacting with the database. - #[error("indexing failed due to database error: {0}")] - DatabaseError(#[from] DatabaseError), - /// An error occurred while trying to fetch the L2 block from the database. - #[error("L2 block not found - block number: {0}")] - L2BlockNotFound(u64), -} diff --git a/crates/manager/Cargo.toml b/crates/manager/Cargo.toml index 6c855073..4341fd0e 100644 --- a/crates/manager/Cargo.toml +++ b/crates/manager/Cargo.toml @@ -44,7 +44,7 @@ scroll-network.workspace = true scroll-wire.workspace = true # rollup node -rollup-node-indexer.workspace = true +rollup-node-chain-orchestrator.workspace = true rollup-node-primitives.workspace = true rollup-node-providers.workspace = true rollup-node-sequencer.workspace = true diff --git a/crates/manager/src/manager/command.rs b/crates/manager/src/manager/command.rs index 2b42fc8c..2b840c8c 100644 --- a/crates/manager/src/manager/command.rs +++ b/crates/manager/src/manager/command.rs @@ -1,6 +1,7 @@ use super::{RollupManagerEvent, RollupManagerStatus}; use reth_tokio_util::EventStream; +use rollup_node_primitives::BlockInfo; use tokio::sync::oneshot; /// The commands that can be sent to the rollup manager. @@ -12,4 +13,6 @@ pub enum RollupManagerCommand { EventListener(oneshot::Sender>), /// Report the current status of the manager via the oneshot channel. Status(oneshot::Sender), + /// Update the head of the fcs in the engine driver. + UpdateFcsHead(BlockInfo), } diff --git a/crates/manager/src/manager/event.rs b/crates/manager/src/manager/event.rs index 9d2808b9..6e18c676 100644 --- a/crates/manager/src/manager/event.rs +++ b/crates/manager/src/manager/event.rs @@ -1,5 +1,9 @@ +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::L1MessageStart; use scroll_engine::ConsolidationOutcome; use scroll_network::NewBlockWithPeer; @@ -14,10 +18,24 @@ pub enum RollupManagerEvent { BlockImported(ScrollBlock), /// Consolidated block derived from L1. L1DerivedBlockConsolidated(ConsolidationOutcome), - /// An L1 message with the given index has been indexed. - L1MessageIndexed(u64), /// 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 start index or hash. + start: L1MessageStart, + }, + /// 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 index 0651add0..50ed099d 100644 --- a/crates/manager/src/manager/handle.rs +++ b/crates/manager/src/manager/handle.rs @@ -1,5 +1,6 @@ use super::{RollupManagerCommand, RollupManagerEvent}; use reth_tokio_util::EventStream; +use rollup_node_primitives::BlockInfo; use tokio::sync::{mpsc, oneshot}; /// The handle used to send commands to the rollup manager. @@ -34,4 +35,9 @@ impl RollupManagerHandle { 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; + } } diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index d4c9d3a9..5f1ed30e 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -1,18 +1,20 @@ //! The [`RollupNodeManager`] is the main component of the rollup node that manages the -//! [`ScrollNetworkManager`], [`EngineDriver`], [`Indexer`] and [`Consensus`] components. It is -//! responsible for handling events from these components and coordinating their actions. +//! [`ScrollNetworkManager`], [`EngineDriver`], [`ChainOrchestrator`] and [`Consensus`] components. +//! It is responsible for handling events from these components and coordinating their actions. use super::Consensus; -use alloy_primitives::Signature; use alloy_provider::Provider; use futures::StreamExt; use reth_chainspec::EthChainSpec; -use reth_network_api::{block::NewBlockWithPeer as RethNewBlockWithPeer, FullNetwork}; +use reth_network::BlockDownloaderProvider; +use reth_network_api::FullNetwork; use reth_scroll_node::ScrollNetworkPrimitives; -use reth_scroll_primitives::ScrollBlock; use reth_tasks::shutdown::GracefulShutdown; use reth_tokio_util::{EventSender, EventStream}; -use rollup_node_indexer::{Indexer, IndexerEvent}; +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; @@ -38,7 +40,7 @@ use tokio_stream::wrappers::ReceiverStream; use tracing::{error, trace, warn}; use rollup_node_providers::{L1MessageProvider, L1Provider}; -use scroll_db::Database; +use scroll_db::{Database, DatabaseError}; use scroll_derivation_pipeline::DerivationPipeline; mod command; @@ -53,9 +55,6 @@ pub use handle::RollupManagerHandle; /// The size of the event channel. const EVENT_CHANNEL_SIZE: usize = 100; -/// The size of the ECDSA signature in bytes. -const ECDSA_SIGNATURE_LEN: usize = 65; - /// The main manager for the rollup node. /// /// This is an endless [`Future`] that drives the state of the entire network forward and includes @@ -82,19 +81,17 @@ pub struct RollupNodeManager< /// The chain spec used by the rollup node. chain_spec: Arc, /// The network manager that manages the scroll p2p network. - network: ScrollNetworkManager, + 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>>, - /// An indexer used to index data for the rollup node. - indexer: Indexer, + /// The chain orchestrator. + chain: ChainOrchestrator::Client, P>, /// The consensus algorithm used by the rollup node. consensus: Box, - /// The receiver for new blocks received from the network (used to bridge from eth-wire). - eth_wire_block_rx: Option>>, /// 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. @@ -130,7 +127,7 @@ impl< .field("engine", &self.engine) .field("derivation_pipeline", &self.derivation_pipeline) .field("l1_notification_rx", &self.l1_notification_rx) - .field("indexer", &self.indexer) + .field("indexer", &self.chain) .field("consensus", &self.consensus) .field("eth_wire_block_rx", &"eth_wire_block_rx") .field("event_sender", &self.event_sender) @@ -152,21 +149,20 @@ where /// Create a new [`RollupNodeManager`] instance. #[allow(clippy::too_many_arguments)] #[allow(clippy::new_ret_no_self)] - pub fn new( - network: ScrollNetworkManager, + pub async fn new( + network: ScrollNetworkManager, engine: EngineDriver, l1_provider: L1P, database: Arc, l1_notification_rx: Option>>, consensus: Box, chain_spec: Arc, - eth_wire_block_rx: Option>>, sequencer: Option>, signer: Option, block_time: Option, + chain_orchestrator: ChainOrchestrator::Client, P>, ) -> (Self, RollupManagerHandle) { let (handle_tx, handle_rx) = mpsc::channel(EVENT_CHANNEL_SIZE); - let indexer = Indexer::new(database.clone(), chain_spec.clone()); let derivation_pipeline = DerivationPipeline::new(l1_provider, database); let rnm = Self { handle_rx, @@ -175,9 +171,8 @@ where engine, derivation_pipeline, l1_notification_rx: l1_notification_rx.map(Into::into), - indexer, + chain: chain_orchestrator, consensus, - eth_wire_block_rx, event_sender: None, sequencer, signer, @@ -221,7 +216,7 @@ where result: Err(err.into()), }); } else { - self.engine.handle_block_import(block_with_peer); + self.chain.handle_block_from_peer(block_with_peer); } } @@ -235,10 +230,19 @@ where } /// Handles an indexer event. - fn handle_indexer_event(&mut self, event: IndexerEvent) { - trace!(target: "scroll::node::manager", ?event, "Received indexer 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 { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head, l1_block_number } => { + ChainOrchestratorEvent::BatchCommitIndexed { + batch_info, + safe_head, + l1_block_number, + } => { // 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.flush(); @@ -249,11 +253,11 @@ where // push the batch info into the derivation pipeline. self.derivation_pipeline.handle_batch_commit(batch_info, l1_block_number); } - IndexerEvent::BatchFinalizationIndexed(_, Some(finalized_block)) => { + ChainOrchestratorEvent::BatchFinalized(_, Some(finalized_block)) => { // update the fcs on new finalized block. self.engine.set_finalized_block_info(finalized_block); } - IndexerEvent::FinalizedIndexed(l1_block_number, finalized_block) => { + ChainOrchestratorEvent::L1BlockFinalized(l1_block_number, finalized_block) => { if let Some(sequencer) = self.sequencer.as_mut() { sequencer.set_l1_finalized_block_number(l1_block_number); } @@ -262,7 +266,7 @@ where self.engine.set_finalized_block_info(finalized_block); } } - IndexerEvent::UnwindIndexed { + ChainOrchestratorEvent::ChainUnwound { l1_block_number, queue_index, l2_head_block_info, @@ -290,12 +294,25 @@ where event_sender.notify(RollupManagerEvent::Reorg(l1_block_number)); } } - IndexerEvent::L1MessageIndexed(index) => { - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1MessageIndexed(index)); - } + ChainOrchestratorEvent::ChainExtended(chain_import) => { + trace!(target: "scroll::node::manager", head = ?chain_import.chain.last().unwrap().header.clone(), peer_id = ?chain_import.peer_id.clone(), "Received chain extension from peer"); + // Issue the new chain to the engine driver for processing. + self.engine.handle_chain_import(chain_import) } - _ => (), + ChainOrchestratorEvent::ChainReorged(chain_import) => { + trace!(target: "scroll::node::manager", head = ?chain_import.chain.last().unwrap().header, ?chain_import.peer_id, "Received chain reorg from peer"); + + // 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(); + trace!(target: "scroll::node::manager", ?block_info, "Received optimistic sync from peer"); + + // Issue the new block info to the engine driver for processing. + self.engine.handle_optimistic_sync(block_info) + } + _ => {} } } @@ -308,7 +325,7 @@ where if let Some(event_sender) = self.event_sender.as_ref() { event_sender.notify(RollupManagerEvent::BlockImported(block.clone())); } - self.indexer.handle_block((&block).into(), None); + self.chain.consolidate_validated_l2_blocks(vec![(&block).into()]); } self.network.handle().block_import_outcome(outcome); } @@ -318,15 +335,13 @@ where } if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::BlockSequenced(payload.clone())); + event_sender.notify(RollupManagerEvent::BlockSequenced(payload)); } - - self.indexer.handle_block((&payload).into(), None); } EngineDriverEvent::L1BlockConsolidated(consolidation_outcome) => { - self.indexer.handle_block( - consolidation_outcome.block_info().clone(), - Some(*consolidation_outcome.batch_info()), + 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() { @@ -335,46 +350,26 @@ where )); } } + EngineDriverEvent::ChainImportOutcome(outcome) => { + if let Some(block) = outcome.outcome.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); + } } } - fn handle_eth_wire_block( - &mut self, - block: reth_network_api::block::NewBlockWithPeer, - ) { - trace!(target: "scroll::node::manager", ?block, "Received new block from eth-wire protocol"); - 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. - let extra_data = if self.chain_spec.is_euclid_v2_active_at_timestamp(block.timestamp) { - let extra_data = block.extra_data.clone(); - block.header.extra_data = Default::default(); - extra_data - } else { - block.extra_data.clone() - }; - - // If we can extract a signature from the extra data we validate consensus and then attempt - // import via the EngineAPI in the `handle_new_block` method. The signature is extracted - // from the last `ECDSA_SIGNATURE_LEN` bytes of the extra data field as specified by - // the protocol. - let block = if let Some(signature) = extra_data - .len() - .checked_sub(ECDSA_SIGNATURE_LEN) - .and_then(|i| Signature::from_raw(&extra_data[i..]).ok()) - { - trace!(target: "scroll::bridge::import", peer_id = %peer_id, block = ?block.hash_slow(), "Received new block from eth-wire protocol"); - NewBlockWithPeer { peer_id, block, signature } - } else { - warn!(target: "scroll::bridge::import", peer_id = %peer_id, "Failed to extract signature from block extra data"); - return; - }; - - self.handle_new_block(block); - } - /// Handles an [`L1Notification`] from the L1 watcher. fn handle_l1_notification(&mut self, notification: L1Notification) { + 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) => { @@ -382,7 +377,10 @@ where sequencer.handle_new_l1_block(new_block) } } - _ => self.indexer.handle_l1_notification(notification), + L1Notification::Synced => { + self.chain.handle_l1_notification(L1Notification::Synced); + } + _ => self.chain.handle_l1_notification(notification), } } @@ -390,7 +388,7 @@ where fn status(&self) -> RollupManagerStatus { RollupManagerStatus { syncing: self.engine.is_syncing(), - forkchoice_state: self.engine.forkchoice_state(), + forkchoice_state: self.engine.forkchoice_state().clone(), } } @@ -450,6 +448,10 @@ where 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); + } } } @@ -479,11 +481,40 @@ where ); // Drain all Indexer events. - while let Poll::Ready(Some(result)) = this.indexer.poll_next_unpin(cx) { + while let Poll::Ready(Some(result)) = this.chain.poll_next_unpin(cx) { match result { - Ok(event) => this.handle_indexer_event(event), + Ok(event) => this.handle_chain_orchestrator_event(event), Err(err) => { - error!(target: "scroll::node::manager", ?err, "Error occurred at indexer level") + match &err { + ChainOrchestratorError::L1MessageMismatch { expected, actual } => { + if let Some(event_sender) = this.event_sender.as_ref() { + event_sender.notify( + RollupManagerEvent::L1MessageConsolidationError { + expected: *expected, + actual: *actual, + }, + ); + } + } + ChainOrchestratorError::DatabaseError( + DatabaseError::L1MessageNotFound(start), + ) => { + if let Some(event_sender) = this.event_sender.as_ref() { + event_sender.notify( + RollupManagerEvent::L1MessageMissingInDatabase { + start: start.clone(), + }, + ); + } + } + _ => {} + } + + error!( + target: "scroll::node::manager", + ?err, + "Error occurred in the chain orchestrator" + ); } } } @@ -502,6 +533,11 @@ where )); } + this.chain.handle_sequenced_block(NewBlockWithPeer { + peer_id: Default::default(), + block: block.clone(), + signature, + }); this.network.handle().announce_block(block, signature); } } @@ -511,7 +547,7 @@ where 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(|se| se.poll_tick(cx)), + this.block_building_trigger.as_mut().map(|trigger| trigger.poll_tick(cx)), this.sequencer.as_mut() ) { if !this.consensus.should_sequence_block( @@ -534,13 +570,6 @@ where this.engine.handle_l1_consolidation(attributes) } - // Handle blocks received from the eth-wire protocol. - while let Some(Poll::Ready(Some(block))) = - this.eth_wire_block_rx.as_mut().map(|new_block_rx| new_block_rx.poll_next_unpin(cx)) - { - this.handle_eth_wire_block(block); - } - // Handle network manager events. while let Poll::Ready(Some(event)) = this.network.poll_next_unpin(cx) { this.handle_network_manager_event(event); diff --git a/crates/network/Cargo.toml b/crates/network/Cargo.toml index 1fb59c4f..ecee2d09 100644 --- a/crates/network/Cargo.toml +++ b/crates/network/Cargo.toml @@ -11,18 +11,21 @@ exclude.workspace = true alloy-primitives = { workspace = true, features = ["map-foldhash"] } # reth +reth-chainspec.workspace = true reth-eth-wire-types.workspace = true reth-network.workspace = true reth-network-api.workspace = true reth-network-types = { git = "https://github.com/scroll-tech/reth.git", default-features = false } reth-network-peers.workspace = true reth-primitives-traits.workspace = true -reth-storage-api = { git = "https://github.com/scroll-tech/reth.git", default-features = false } +reth-storage-api = { workspace = true, default-features = false } +reth-tokio-util.workspace = true # scroll reth-scroll-chainspec.workspace = true reth-scroll-node.workspace = true reth-scroll-primitives.workspace = true +scroll-alloy-hardforks.workspace = true scroll-wire.workspace = true # misc @@ -45,4 +48,5 @@ serde = [ "scroll-wire/serde", "reth-primitives-traits/serde", "reth-storage-api/serde", + "scroll-alloy-hardforks/serde", ] diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index 590d87e4..2b10d4ac 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -4,28 +4,36 @@ use super::{ BlockImportOutcome, BlockValidation, NetworkHandleMessage, NetworkManagerEvent, NewBlockWithPeer, ScrollNetworkHandle, }; -use alloy_primitives::{FixedBytes, U128}; +use alloy_primitives::{FixedBytes, Signature, U128}; use futures::{FutureExt, Stream, StreamExt}; +use reth_chainspec::EthChainSpec; use reth_eth_wire_types::NewBlock as EthWireNewBlock; use reth_network::{ cache::LruCache, NetworkConfig as RethNetworkConfig, NetworkHandle as RethNetworkHandle, NetworkManager as RethNetworkManager, }; -use reth_network_api::FullNetwork; +use reth_network_api::{block::NewBlockWithPeer as RethNewBlockWithPeer, FullNetwork}; use reth_scroll_node::ScrollNetworkPrimitives; +use reth_scroll_primitives::ScrollBlock; use reth_storage_api::BlockNumReader as BlockNumReaderT; +use reth_tokio_util::EventStream; +use scroll_alloy_hardforks::ScrollHardforks; use scroll_wire::{ NewBlock, ScrollWireConfig, ScrollWireEvent, ScrollWireManager, ScrollWireProtocolHandler, LRU_CACHE_SIZE, }; use std::{ pin::Pin, + sync::Arc, task::{Context, Poll}, }; use tokio::sync::mpsc::{self, UnboundedReceiver}; use tokio_stream::wrappers::UnboundedReceiverStream; use tracing::trace; +/// The size of the ECDSA signature in bytes. +const ECDSA_SIGNATURE_LEN: usize = 65; + /// [`ScrollNetworkManager`] manages the state of the scroll p2p network. /// /// This manager drives the state of the entire network forward and includes the following @@ -35,22 +43,33 @@ use tracing::trace; /// - `from_handle_rx`: Receives commands from the [`FullNetwork`]. /// - `scroll_wire`: The type that manages connections and state of the scroll wire protocol. #[derive(Debug)] -pub struct ScrollNetworkManager { +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, /// Receiver half of the channel set up between this type and the [`FullNetwork`], receives /// [`NetworkHandleMessage`]s. from_handle_rx: UnboundedReceiverStream, + /// The receiver for new blocks received from the network (used to bridge from eth-wire). + eth_wire_listener: Option>>, /// The scroll wire protocol manager. scroll_wire: ScrollWireManager, + /// Should blocks be announced over the eth-wire protocol. + eth_wire_gossip: bool, } -impl ScrollNetworkManager> { +impl + ScrollNetworkManager, CS> +{ /// Creates a new [`ScrollNetworkManager`] instance from the provided configuration and block /// import. pub async fn new( + chain_spec: Arc, mut network_config: RethNetworkConfig, scroll_wire_config: ScrollWireConfig, + eth_wire_listener: Option>>, + eth_wire_gossip: bool, ) -> Self { // Create the scroll-wire protocol handler. let (scroll_wire_handler, events) = ScrollWireProtocolHandler::new(scroll_wire_config); @@ -74,16 +93,33 @@ impl ScrollNetworkManager> { // Spawn the inner network manager. tokio::spawn(inner_network_manager); - Self { handle, from_handle_rx: from_handle_rx.into(), scroll_wire } + Self { + chain_spec, + handle, + from_handle_rx: from_handle_rx.into(), + scroll_wire, + eth_wire_listener, + eth_wire_gossip, + } } } -impl> ScrollNetworkManager { +impl< + N: FullNetwork, + CS: ScrollHardforks + EthChainSpec + Send + Sync + 'static, + > ScrollNetworkManager +{ /// Creates a new [`ScrollNetworkManager`] instance from the provided parts. /// /// This is used when the scroll-wire [`ScrollWireProtocolHandler`] and the inner network /// manager [`RethNetworkManager`] are instantiated externally. - pub fn from_parts(inner_network_handle: N, events: UnboundedReceiver) -> Self { + pub fn from_parts( + chain_spec: Arc, + inner_network_handle: N, + events: UnboundedReceiver, + eth_wire_listener: Option>>, + eth_wire_gossip: bool, + ) -> Self { // Create the channel for sending messages to the network manager from the network handle. let (to_manager_tx, from_handle_rx) = mpsc::unbounded_channel(); @@ -92,7 +128,14 @@ impl> ScrollNetworkManager< let handle = ScrollNetworkHandle::new(to_manager_tx, inner_network_handle); - Self { handle, from_handle_rx: from_handle_rx.into(), scroll_wire } + Self { + chain_spec, + handle, + from_handle_rx: from_handle_rx.into(), + scroll_wire, + eth_wire_listener, + eth_wire_gossip, + } } /// Returns a new [`ScrollNetworkHandle`] instance. @@ -118,13 +161,15 @@ impl> ScrollNetworkManager< .filter_map(|(peer_id, blocks)| (!blocks.contains(&hash)).then_some(*peer_id)) .collect(); - let eth_wire_new_block = { - let td = U128::from_limbs([0, block.block.header.number]); - let mut eth_wire_block = block.block.clone(); - 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); + if self.eth_wire_gossip { + let eth_wire_new_block = { + let td = U128::from_limbs([0, block.block.header.number]); + let mut eth_wire_block = block.block.clone(); + 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); + } // Announce block to the filtered set of peers for peer_id in peers { @@ -191,9 +236,53 @@ impl> ScrollNetworkManager< } } } + + /// Handles a new block received from the eth-wire protocol. + fn handle_eth_wire_block( + &mut self, + block: reth_network_api::block::NewBlockWithPeer, + ) -> Option { + let reth_network_api::block::NewBlockWithPeer { peer_id, mut block } = block; + let block_hash = block.hash_slow(); + self.scroll_wire + .state_mut() + .entry(peer_id) + .or_insert_with(|| LruCache::new(LRU_CACHE_SIZE)) + .insert(block_hash); + + trace!(target: "scroll::bridge::import", peer_id = %peer_id, block_hash = %block_hash, "Received new block from eth-wire protocol"); + + // We purge the extra data field post euclid v2 to align with protocol specification. + let extra_data = if self.chain_spec.is_euclid_v2_active_at_timestamp(block.timestamp) { + let extra_data = block.extra_data.clone(); + block.header.extra_data = Default::default(); + extra_data + } else { + block.extra_data.clone() + }; + + // If we can extract a signature from the extra data we validate consensus and then attempt + // import via the EngineAPI in the `handle_new_block` method. The signature is extracted + // from the last `ECDSA_SIGNATURE_LEN` bytes of the extra data field as specified by + // the protocol. + if let Some(signature) = extra_data + .len() + .checked_sub(ECDSA_SIGNATURE_LEN) + .and_then(|i| Signature::from_raw(&extra_data[i..]).ok()) + { + Some(NetworkManagerEvent::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"); + None + } + } } -impl> Stream for ScrollNetworkManager { +impl< + N: FullNetwork, + CS: ScrollHardforks + EthChainSpec + Send + Sync + 'static, + > Stream for ScrollNetworkManager +{ type Item = NetworkManagerEvent; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -220,6 +309,15 @@ impl> Stream for ScrollNetw return Poll::Ready(Some(this.on_scroll_wire_event(event))); } + // Handle blocks received from the eth-wire protocol. + while let Some(Poll::Ready(Some(block))) = + 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)); + } + } + Poll::Pending } } diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index 548f147e..c46a3326 100644 --- a/crates/node/Cargo.toml +++ b/crates/node/Cargo.toml @@ -56,6 +56,7 @@ reth-transaction-pool = { git = "https://github.com/scroll-tech/reth.git", defau reth-trie-db = { git = "https://github.com/scroll-tech/reth.git", default-features = false } # rollup node +rollup-node-chain-orchestrator.workspace = true rollup-node-manager.workspace = true rollup-node-primitives.workspace = true rollup-node-providers.workspace = true diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 719193e3..52f9fc92 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -19,6 +19,7 @@ use reth_node_builder::rpc::RethRpcServerHandles; use reth_node_core::primitives::BlockHeader; use reth_scroll_chainspec::SCROLL_FEE_VAULT_ADDRESS; use reth_scroll_node::ScrollNetworkPrimitives; +use rollup_node_chain_orchestrator::ChainOrchestrator; use rollup_node_manager::{ Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, }; @@ -51,6 +52,9 @@ pub struct ScrollRollupNodeConfig { /// Database args #[command(flatten)] pub database_args: DatabaseArgs, + /// Chain orchestrator args. + #[command(flatten)] + pub chain_orchestrator_args: ChainOrchestratorArgs, /// Engine driver args. #[command(flatten)] pub engine_driver_args: EngineDriverArgs, @@ -141,8 +145,6 @@ impl ScrollRollupNodeConfig { "Building rollup node with config:\n{:#?}", self ); - // Instantiate the network manager - let scroll_network_manager = ScrollNetworkManager::from_parts(network.clone(), events); // Get the rollup node config. let named_chain = chain_spec.chain().named().expect("expected named chain"); @@ -169,8 +171,8 @@ impl ScrollRollupNodeConfig { let l2_provider = rpc_server_handles .rpc .new_http_provider_for() - .map(Arc::new) .expect("failed to create payload provider"); + let l2_provider = Arc::new(l2_provider); // Instantiate the database let database_path = if let Some(database_path) = self.database_args.path { @@ -202,6 +204,19 @@ impl ScrollRollupNodeConfig { let chain_spec = Arc::new(chain_spec.clone()); + // Instantiate the network manager + let eth_wire_listener = self + .network_args + .enable_eth_scroll_wire_bridge + .then_some(network.eth_wire_block_listener().await?); + let scroll_network_manager = ScrollNetworkManager::from_parts( + chain_spec.clone(), + network.clone(), + events, + eth_wire_listener, + self.network_args.eth_wire_gossip, + ); + // On startup we replay the latest batch of blocks from the database as such we set the safe // block hash to the latest block hash associated with the previous consolidated // batch in the database. @@ -219,10 +234,9 @@ impl ScrollRollupNodeConfig { let engine = EngineDriver::new( Arc::new(engine_api), chain_spec.clone(), - Some(l2_provider), + Some(l2_provider.clone()), fcs, self.engine_driver_args.sync_at_startup && !self.test && !chain_spec.is_dev_chain(), - self.engine_driver_args.en_sync_trigger, Duration::from_millis(self.sequencer_args.payload_building_duration), ); @@ -282,12 +296,6 @@ impl ScrollRollupNodeConfig { (None, None) }; - // Instantiate the eth wire listener - let eth_wire_listener = self - .network_args - .enable_eth_scroll_wire_bridge - .then_some(network.eth_wire_block_listener().await?); - // Instantiate the signer let signer = if self.test { // Use a random private key signer for testing @@ -298,6 +306,23 @@ impl ScrollRollupNodeConfig { self.signer_args.signer(chain_id).await?.map(rollup_node_signer::Signer::spawn) }; + // Instantiate the chain orchestrator + let block_client = scroll_network_manager + .handle() + .inner() + .fetch_client() + .await + .expect("failed to fetch block client"); + let chain_orchestrator = ChainOrchestrator::new( + db.clone(), + chain_spec.clone(), + block_client, + l2_provider, + self.chain_orchestrator_args.optimistic_sync_trigger, + self.chain_orchestrator_args.chain_buffer_size, + ) + .await; + // Spawn the rollup node manager let (rnm, handle) = RollupNodeManager::new( scroll_network_manager, @@ -307,11 +332,12 @@ impl ScrollRollupNodeConfig { l1_notification_rx, consensus, chain_spec, - eth_wire_listener, sequencer, signer, block_time, - ); + chain_orchestrator, + ) + .await; Ok((rnm, handle, l1_notification_tx)) } } @@ -389,26 +415,49 @@ impl Default for ConsensusAlgorithm { } /// The engine driver args. -#[derive(Debug, Default, Clone, clap::Args)] +#[derive(Debug, Clone, clap::Args)] pub struct EngineDriverArgs { - /// The amount of block difference between the EN and the latest block received from P2P - /// at which the engine driver triggers optimistic sync. - #[arg(long = "engine.en-sync-trigger", default_value_t = constants::BLOCK_GAP_TRIGGER)] - pub en_sync_trigger: u64, /// Whether the engine driver should try to sync at start up. #[arg(long = "engine.sync-at-startup", num_args=0..=1, default_value_t = true)] pub sync_at_startup: bool, } +impl Default for EngineDriverArgs { + fn default() -> Self { + Self { sync_at_startup: true } + } +} + +/// The chain orchestrator arguments. +#[derive(Debug, Clone, clap::Args)] +pub struct ChainOrchestratorArgs { + /// The amount of block difference between the EN and the latest block received from P2P + /// at which the engine driver triggers optimistic sync. + #[arg(long = "chain.optimistic-sync-trigger", default_value_t = constants::BLOCK_GAP_TRIGGER)] + pub optimistic_sync_trigger: u64, + /// The size of the in-memory chain buffer used by the chain orchestrator. + #[arg(long = "chain.chain-buffer-size", default_value_t = constants::CHAIN_BUFFER_SIZE)] + pub chain_buffer_size: usize, +} + +impl Default for ChainOrchestratorArgs { + fn default() -> Self { + Self { + optimistic_sync_trigger: constants::BLOCK_GAP_TRIGGER, + chain_buffer_size: constants::CHAIN_BUFFER_SIZE, + } + } +} + /// The network arguments. #[derive(Debug, Clone, clap::Args)] pub struct NetworkArgs { /// A bool to represent if new blocks should be bridged from the eth wire protocol to the /// scroll wire protocol. - #[arg(long = "network.bridge", default_value_t = true)] + #[arg(long = "network.bridge")] pub enable_eth_scroll_wire_bridge: bool, /// A bool that represents if the scroll wire protocol should be enabled. - #[arg(long = "network.scroll-wire", default_value_t = true)] + #[arg(long = "network.scroll-wire")] pub enable_scroll_wire: bool, /// The URL for the Sequencer RPC. (can be both HTTP and WS) #[arg( @@ -417,11 +466,19 @@ pub struct NetworkArgs { value_name = "NETWORK_SEQUENCER_URL" )] pub sequencer_url: Option, + /// A bool that represents if blocks should be gossiped over the eth-wire protocol. + #[arg(long = "network.eth-wire-gossip")] + pub eth_wire_gossip: bool, } impl Default for NetworkArgs { fn default() -> Self { - Self { enable_eth_scroll_wire_bridge: true, enable_scroll_wire: true, sequencer_url: None } + Self { + enable_eth_scroll_wire_bridge: true, + enable_scroll_wire: true, + sequencer_url: None, + eth_wire_gossip: false, + } } } @@ -607,6 +664,7 @@ mod tests { signer_args: SignerArgs { key_file: None, aws_kms_key_id: None, private_key: None }, database_args: DatabaseArgs::default(), engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), l1_provider_args: L1ProviderArgs::default(), beacon_provider_args: BeaconProviderArgs::default(), network_args: NetworkArgs::default(), @@ -636,6 +694,7 @@ mod tests { }, database_args: DatabaseArgs::default(), engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), l1_provider_args: L1ProviderArgs::default(), beacon_provider_args: BeaconProviderArgs::default(), network_args: NetworkArgs::default(), @@ -662,6 +721,7 @@ mod tests { private_key: None, }, database_args: DatabaseArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), engine_driver_args: EngineDriverArgs::default(), l1_provider_args: L1ProviderArgs::default(), beacon_provider_args: BeaconProviderArgs::default(), @@ -685,6 +745,7 @@ mod tests { }, database_args: DatabaseArgs::default(), engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), l1_provider_args: L1ProviderArgs::default(), beacon_provider_args: BeaconProviderArgs::default(), network_args: NetworkArgs::default(), @@ -703,6 +764,7 @@ mod tests { signer_args: SignerArgs { key_file: None, aws_kms_key_id: None, private_key: None }, database_args: DatabaseArgs::default(), engine_driver_args: EngineDriverArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), l1_provider_args: L1ProviderArgs::default(), beacon_provider_args: BeaconProviderArgs::default(), network_args: NetworkArgs::default(), diff --git a/crates/node/src/constants.rs b/crates/node/src/constants.rs index 9637cef1..444595d7 100644 --- a/crates/node/src/constants.rs +++ b/crates/node/src/constants.rs @@ -20,7 +20,10 @@ pub(crate) const DEFAULT_PAYLOAD_SIZE_LIMIT: u64 = 122_880; pub(crate) const DEFAULT_MAX_L1_MESSAGES_PER_BLOCK: u64 = 4; /// The gap in blocks between the P2P and EN which triggers sync. -pub(crate) const BLOCK_GAP_TRIGGER: u64 = 500_000; +pub(crate) const BLOCK_GAP_TRIGGER: u64 = 100_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; /// The default suggested priority fee for the gas price oracle. pub(crate) const DEFAULT_SUGGEST_PRIORITY_FEE: u64 = 100; diff --git a/crates/node/src/test_utils.rs b/crates/node/src/test_utils.rs index d6212212..71aa43af 100644 --- a/crates/node/src/test_utils.rs +++ b/crates/node/src/test_utils.rs @@ -3,8 +3,8 @@ use crate::{ConsensusArgs, GasPriceOracleArgs}; use super::{ - BeaconProviderArgs, DatabaseArgs, EngineDriverArgs, L1ProviderArgs, ScrollRollupNode, - ScrollRollupNodeConfig, SequencerArgs, + BeaconProviderArgs, ChainOrchestratorArgs, DatabaseArgs, EngineDriverArgs, L1ProviderArgs, + ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, }; use alloy_primitives::Bytes; use reth_chainspec::EthChainSpec; @@ -144,7 +144,11 @@ pub fn default_test_scroll_rollup_node_config() -> ScrollRollupNodeConfig { network_args: crate::args::NetworkArgs::default(), database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs { en_sync_trigger: 100, sync_at_startup: true }, + engine_driver_args: EngineDriverArgs { sync_at_startup: true }, + chain_orchestrator_args: ChainOrchestratorArgs { + optimistic_sync_trigger: 100, + chain_buffer_size: 100, + }, sequencer_args: SequencerArgs { payload_building_duration: 1000, ..Default::default() }, beacon_provider_args: BeaconProviderArgs { blob_source: BlobSource::Mock, @@ -163,7 +167,11 @@ pub fn default_sequencer_test_scroll_rollup_node_config() -> ScrollRollupNodeCon network_args: crate::args::NetworkArgs::default(), database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, l1_provider_args: L1ProviderArgs::default(), - engine_driver_args: EngineDriverArgs { en_sync_trigger: 100, sync_at_startup: true }, + engine_driver_args: EngineDriverArgs { sync_at_startup: true }, + chain_orchestrator_args: ChainOrchestratorArgs { + optimistic_sync_trigger: 100, + chain_buffer_size: 100, + }, sequencer_args: SequencerArgs { sequencer_enabled: true, block_time: 50, diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 8c7a8fe2..f3a6fb07 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -17,10 +17,11 @@ use rollup_node::{ default_sequencer_test_scroll_rollup_node_config, default_test_scroll_rollup_node_config, generate_tx, setup_engine, }, - BeaconProviderArgs, ConsensusAlgorithm, ConsensusArgs, DatabaseArgs, EngineDriverArgs, - GasPriceOracleArgs, L1ProviderArgs, NetworkArgs as ScrollNetworkArgs, ScrollRollupNodeConfig, - SequencerArgs, + BeaconProviderArgs, ChainOrchestratorArgs, ConsensusAlgorithm, ConsensusArgs, DatabaseArgs, + EngineDriverArgs, GasPriceOracleArgs, L1ProviderArgs, NetworkArgs as ScrollNetworkArgs, + ScrollRollupNodeConfig, SequencerArgs, }; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent, RollupManagerHandle}; use rollup_node_primitives::{sig_encode_hash, BatchCommitData, ConsensusUpdate}; use rollup_node_providers::BlobSource; @@ -45,6 +46,7 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { database_args: DatabaseArgs { 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, block_time: 0, @@ -82,19 +84,43 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { block_timestamp: 1000, })) .await?; - if let Some(RollupManagerEvent::L1MessageIndexed(index)) = rnm_events.next().await { - assert_eq!(index, 0); - } else { - panic!("Incorrect index for L1 message"); - }; + + wait_n_events( + &mut rnm_events, + |e| { + if let RollupManagerEvent::ChainOrchestratorEvent( + rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted(index), + ) = e + { + assert_eq!(index, 0); + true + } else { + false + } + }, + 1, + ) + .await; rnm_handle.build_block().await; - if let Some(RollupManagerEvent::BlockSequenced(block)) = rnm_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - assert_eq!(block.body.transactions[0].as_l1_message().unwrap().inner(), &l1_message,); - } else { - panic!("Failed to receive block from rollup node"); - } + + wait_n_events( + &mut rnm_events, + |e| { + if let RollupManagerEvent::BlockSequenced(block) = e { + assert_eq!(block.body.transactions.len(), 1); + assert_eq!( + block.body.transactions[0].as_l1_message().unwrap().inner(), + &l1_message + ); + true + } else { + false + } + }, + 1, + ) + .await; Ok(()) } @@ -111,10 +137,12 @@ async fn can_sequence_and_gossip_blocks() { enable_eth_scroll_wire_bridge: true, enable_scroll_wire: true, sequencer_url: None, + eth_wire_gossip: false, }, database_args: DatabaseArgs { 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, block_time: 0, @@ -148,20 +176,49 @@ async fn can_sequence_and_gossip_blocks() { sequencer_rnm_handle.build_block().await; // wait for the sequencer to build a block - if let Some(RollupManagerEvent::BlockSequenced(block)) = sequencer_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - } else { - panic!("Failed to receive block from rollup node"); - } + wait_n_events( + &mut sequencer_events, + |e| { + if let RollupManagerEvent::BlockSequenced(block) = e { + assert_eq!(block.body.transactions.len(), 1); + true + } else { + false + } + }, + 1, + ) + .await; // assert that the follower node has received the block from the peer - if let Some(RollupManagerEvent::NewBlockReceived(block_with_peer)) = - follower_events.next().await - { - assert_eq!(block_with_peer.block.body.transactions.len(), 1); - } else { - panic!("Failed to receive block from rollup node"); - } + wait_n_events( + &mut follower_events, + |e| { + if let RollupManagerEvent::NewBlockReceived(block_with_peer) = e { + assert_eq!(block_with_peer.block.body.transactions.len(), 1); + true + } else { + false + } + }, + 1, + ) + .await; + + // 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| matches!(e, RollupManagerEvent::BlockImported(_)), 1) @@ -170,21 +227,25 @@ async fn can_sequence_and_gossip_blocks() { #[allow(clippy::large_stack_frames)] #[tokio::test] -async fn can_sequence_and_gossip_transactions() { +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(); sequencer_node_config.sequencer_args.block_time = 0; - let follower_node_config = default_test_scroll_rollup_node_config(); + 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(); let (mut sequencer_node, _tasks, _) = - setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); + setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, true).await.unwrap(); + let sequencer_url = format!("http://localhost:{}", sequencer_node[0].rpc_url().port().unwrap()); + follower_node_config.network_args.sequencer_url = Some(sequencer_url); let (mut follower_node, _tasks, wallet) = - setup_engine(follower_node_config, 1, chain_spec, false, false).await.unwrap(); + setup_engine(follower_node_config, 1, chain_spec, false, true).await.unwrap(); let wallet = Arc::new(Mutex::new(wallet)); @@ -242,6 +303,9 @@ 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 wait_n_events( &mut follower_events, @@ -257,6 +321,21 @@ async fn can_sequence_and_gossip_transactions() { ) .await; + // 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, @@ -275,23 +354,21 @@ async fn can_sequence_and_gossip_transactions() { #[allow(clippy::large_stack_frames)] #[tokio::test] -async fn can_forward_tx_to_sequencer() { +async fn can_sequence_and_gossip_transactions() { reth_tracing::init_test_tracing(); // create 2 nodes let mut sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); sequencer_node_config.sequencer_args.block_time = 0; - let mut follower_node_config = default_test_scroll_rollup_node_config(); + let follower_node_config = default_test_scroll_rollup_node_config(); // Create the chain spec for scroll mainnet with Euclid v2 activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); let (mut sequencer_node, _tasks, _) = - setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, true).await.unwrap(); + setup_engine(sequencer_node_config, 1, chain_spec.clone(), false, false).await.unwrap(); - let sequencer_url = format!("http://localhost:{}", sequencer_node[0].rpc_url().port().unwrap()); - follower_node_config.network_args.sequencer_url = Some(sequencer_url); let (mut follower_node, _tasks, wallet) = - setup_engine(follower_node_config, 1, chain_spec, false, true).await.unwrap(); + setup_engine(follower_node_config, 1, chain_spec, false, false).await.unwrap(); let wallet = Arc::new(Mutex::new(wallet)); @@ -349,13 +426,20 @@ 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| matches!(e, RollupManagerEvent::NewBlockReceived(_)), - 1, - ) - .await; + if let Some(RollupManagerEvent::NewBlockReceived(block_with_peer)) = + follower_events.next().await + { + assert_eq!(block_with_peer.block.body.transactions.len(), 1); + } else { + 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( @@ -405,8 +489,14 @@ 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(network_config, scroll_wire_config).await; + let mut scroll_network = scroll_network::ScrollNetworkManager::new( + chain_spec.clone(), + network_config, + scroll_wire_config, + None, + false, + ) + .await; let scroll_network_handle = scroll_network.handle(); // Connect the scroll-wire node to the scroll NetworkManager. @@ -679,35 +769,8 @@ async fn can_handle_batch_revert() -> eyre::Result<()> { setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec.clone(), false, false) .await?; let node = nodes.pop().unwrap(); - - // Instantiate the rollup node manager. - let mut config = default_test_scroll_rollup_node_config(); - let path = node.inner.config.datadir().db().join("scroll.db?mode=rwc"); - let path = PathBuf::from("sqlite://".to_string() + &*path.to_string_lossy()); - config.database_args.path = Some(path.clone()); - config.beacon_provider_args.url = Some( - "http://dummy:8545" - .parse() - .expect("valid url that will not be used as test batches use calldata"), - ); - - let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (rnm, handle, l1_watcher_tx) = config - .clone() - .build( - node.inner.network.clone(), - events, - node.inner.add_ons_handle.rpc_handle.rpc_server_handles.clone(), - chain_spec.clone(), - path.clone(), - ) - .await?; - let l1_watcher_tx = l1_watcher_tx.unwrap(); - - // Spawn a task that constantly polls the rnm to make progress. - tokio::spawn(async { - let _ = rnm.await; - }); + let handle = node.inner.add_ons_handle.rollup_manager_handle.clone(); + let l1_watcher_tx = node.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); // Request an event stream from the rollup node manager and manually poll rnm to process the // event stream request from the handle. @@ -805,16 +868,8 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { let chain_spec = (*SCROLL_DEV).clone(); // Launch a node - let mut config = default_test_scroll_rollup_node_config(); - config.sequencer_args.block_time = 0; - let (mut nodes, _tasks, _) = setup_engine(config, 1, chain_spec.clone(), false, false).await?; - let node = nodes.pop().unwrap(); - - // Instantiate the rollup node manager. let mut config = default_sequencer_test_scroll_rollup_node_config(); - let path = node.inner.config.datadir().db().join("scroll.db?mode=rwc"); - let path = PathBuf::from("sqlite://".to_string() + &*path.to_string_lossy()); - config.database_args.path = Some(path.clone()); + config.sequencer_args.block_time = 0; config.beacon_provider_args.url = Some( "http://dummy:8545" .parse() @@ -822,27 +877,11 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { ); config.engine_driver_args.sync_at_startup = false; config.sequencer_args.max_l1_messages_per_block = 1; + let (mut nodes, _tasks, _) = setup_engine(config, 1, chain_spec.clone(), false, false).await?; + let node = nodes.pop().unwrap(); - let (_, events) = ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - let (rnm, handle, l1_watcher_tx) = config - .clone() - .build( - node.inner.network.clone(), - events, - node.inner.add_ons_handle.rpc_handle.rpc_server_handles.clone(), - chain_spec.clone(), - path.clone(), - ) - .await?; - let l1_watcher_tx = l1_watcher_tx.unwrap(); - - // Spawn a task that constantly polls the rnm to make progress. - tokio::spawn(async { - let _ = rnm.await; - }); - - // Request an event stream from the rollup node manager and manually poll rnm to process the - // event stream request from the handle. + let handle = node.inner.add_ons_handle.rollup_manager_handle.clone(); + let l1_watcher_tx = node.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let mut rnm_events = handle.get_event_listener().await?; // Send an L1 message. @@ -856,14 +895,15 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { }; // Let the sequencer build 10 blocks before performing the reorg process. + handle.build_block().await; let mut i = 0; loop { - handle.build_block().await; if let Some(RollupManagerEvent::BlockSequenced(_)) = rnm_events.next().await { + i += 1; if i == 10 { break } - i += 1; + handle.build_block().await; } } @@ -872,7 +912,10 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { .send(Arc::new(L1Notification::L1Message { message, block_number: 10, block_timestamp: 0 })) .await?; loop { - if let Some(RollupManagerEvent::L1MessageIndexed(index)) = rnm_events.next().await { + if let Some(RollupManagerEvent::ChainOrchestratorEvent( + ChainOrchestratorEvent::L1MessageCommitted(index), + )) = rnm_events.next().await + { assert_eq!(index, 0); break } @@ -888,6 +931,7 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { l2_reorged_height = block.header.number; break } + handle.build_block().await; } } @@ -920,15 +964,10 @@ async fn can_gossip_over_eth_wire() -> eyre::Result<()> { let chain_spec = (*SCROLL_DEV).clone(); // Setup the rollup node manager. - let (mut nodes, _tasks, _) = setup_engine( - default_sequencer_test_scroll_rollup_node_config(), - 2, - chain_spec.clone(), - false, - false, - ) - .await - .unwrap(); + let mut config = default_sequencer_test_scroll_rollup_node_config(); + config.network_args.eth_wire_gossip = true; + 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(); @@ -963,18 +1002,20 @@ async fn signer_rotation() -> eyre::Result<()> { sequencer_1_config.consensus_args.algorithm = ConsensusAlgorithm::SystemContract; sequencer_1_config.consensus_args.authorized_signer = Some(signer_1_address); sequencer_1_config.signer_args.private_key = Some(signer_1); + sequencer_1_config.network_args.enable_eth_scroll_wire_bridge = false; let mut sequencer_2_config = default_sequencer_test_scroll_rollup_node_config(); sequencer_2_config.test = false; sequencer_2_config.consensus_args.algorithm = ConsensusAlgorithm::SystemContract; sequencer_2_config.consensus_args.authorized_signer = Some(signer_1_address); sequencer_2_config.signer_args.private_key = Some(signer_2); + sequencer_2_config.network_args.enable_eth_scroll_wire_bridge = false; // Setup two sequencer nodes. let (mut nodes, _tasks, _) = setup_engine(sequencer_1_config, 2, chain_spec.clone(), false, false).await.unwrap(); - let mut sequencer_1 = nodes.pop().unwrap(); let follower = nodes.pop().unwrap(); + let mut sequencer_1 = nodes.pop().unwrap(); let (mut nodes, _tasks, _) = setup_engine(sequencer_2_config, 1, chain_spec.clone(), false, false).await.unwrap(); let mut sequencer_2 = nodes.pop().unwrap(); @@ -989,23 +1030,40 @@ async fn signer_rotation() -> eyre::Result<()> { // Create a follower event stream. let mut follower_events = follower.inner.add_ons_handle.rollup_manager_handle.get_event_listener().await.unwrap(); + let mut sequencer_2_events = + sequencer_2.inner.add_ons_handle.rollup_manager_handle.get_event_listener().await.unwrap(); // connect the two sequencers sequencer_1.connect(&mut sequencer_2).await; + for _ in 0..5 { + wait_n_events( + &mut follower_events, + |event| { + if let RollupManagerEvent::NewBlockReceived(block) = event { + let signature = block.signature; + let hash = sig_encode_hash(&block.block); + // Verify that the block is signed by the first sequencer. + let recovered_address = signature.recover_address_from_prehash(&hash).unwrap(); + recovered_address == signer_1_address + } else { + false + } + }, + 1, + ) + .await; + wait_n_events( + &mut follower_events, + |event| matches!(event, RollupManagerEvent::BlockImported(_)), + 1, + ) + .await; + } + wait_n_events( - &mut follower_events, - |event| { - if let RollupManagerEvent::NewBlockReceived(block) = event { - let signature = block.signature; - let hash = sig_encode_hash(&block.block); - // Verify that the block is signed by the first sequencer. - let recovered_address = signature.recover_address_from_prehash(&hash).unwrap(); - recovered_address == signer_1_address - } else { - false - } - }, + &mut sequencer_2_events, + |e| matches!(e, RollupManagerEvent::BlockImported(_)), 5, ) .await; diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 48b4a281..74e0dbf7 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -1,23 +1,30 @@ //! Contains tests related to RN and EN sync. -use alloy_primitives::b256; +use alloy_primitives::{b256, Address, U256}; use alloy_provider::{Provider, ProviderBuilder}; use futures::StreamExt; use reqwest::Url; -use reth_e2e_test_utils::NodeHelperType; +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; use rollup_node::{ test_utils::{ default_sequencer_test_scroll_rollup_node_config, default_test_scroll_rollup_node_config, setup_engine, }, - BeaconProviderArgs, ConsensusArgs, DatabaseArgs, EngineDriverArgs, GasPriceOracleArgs, - L1ProviderArgs, NetworkArgs, ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, + BeaconProviderArgs, ChainOrchestratorArgs, ConsensusArgs, DatabaseArgs, EngineDriverArgs, + GasPriceOracleArgs, L1ProviderArgs, NetworkArgs, ScrollRollupNodeConfig, SequencerArgs, }; -use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent}; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; +use rollup_node_manager::RollupManagerEvent; +use rollup_node_primitives::BlockInfo; use rollup_node_providers::BlobSource; -use tokio::sync::oneshot; +use rollup_node_sequencer::L1MessageInclusionMode; +use rollup_node_watcher::L1Notification; +use scroll_alloy_consensus::TxL1Message; +use std::{path::PathBuf, sync::Arc}; #[tokio::test] async fn test_should_consolidate_to_block_15k() -> eyre::Result<()> { @@ -37,18 +44,20 @@ async fn test_should_consolidate_to_block_15k() -> eyre::Result<()> { enable_eth_scroll_wire_bridge: false, enable_scroll_wire: false, sequencer_url: None, + eth_wire_gossip: false, }, database_args: DatabaseArgs::default(), + chain_orchestrator_args: ChainOrchestratorArgs { + optimistic_sync_trigger: 100, + ..Default::default() + }, l1_provider_args: L1ProviderArgs { url: Some(Url::parse(&format!("https://eth-sepolia.g.alchemy.com/v2/{alchemy_key}"))?), compute_units_per_second: 500, max_retries: 10, initial_backoff: 100, }, - engine_driver_args: EngineDriverArgs { - en_sync_trigger: 10000000000, - sync_at_startup: false, - }, + engine_driver_args: EngineDriverArgs { sync_at_startup: false }, sequencer_args: SequencerArgs { sequencer_enabled: false, ..Default::default() }, beacon_provider_args: BeaconProviderArgs { url: Some(Url::parse("https://eth-beacon-chain.drpc.org/rest/")?), @@ -86,7 +95,7 @@ async fn test_should_consolidate_to_block_15k() -> eyre::Result<()> { /// We test if the syncing of the RN is correctly triggered and released when the EN reaches sync. #[allow(clippy::large_stack_frames)] #[tokio::test] -async fn test_should_trigger_pipeline_sync_for_execution_node() { +async fn test_should_trigger_pipeline_sync_for_execution_node() -> eyre::Result<()> { reth_tracing::init_test_tracing(); let node_config = default_test_scroll_rollup_node_config(); let sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); @@ -98,34 +107,41 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { .await .unwrap(); let mut synced = nodes.pop().unwrap(); + let mut synced_events = synced.inner.rollup_manager_handle.get_event_listener().await?; 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?; // Wait for the chain to be advanced by the sequencer. - let en_sync_trigger = node_config.engine_driver_args.en_sync_trigger + 1; - wait_n_events(&synced, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), en_sync_trigger) - .await; + let optimistic_sync_trigger = node_config.chain_orchestrator_args.optimistic_sync_trigger + 1; + wait_n_events( + &mut synced_events, + |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), + optimistic_sync_trigger, + ) + .await; // Connect the nodes together. synced.network.add_peer(unsynced.network.record()).await; unsynced.network.next_session_established().await; synced.network.next_session_established().await; - // Wait for the unsynced node to receive a block. - wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::NewBlockReceived(_)), 1).await; - - // Check the unsynced node enters sync mode. - let (tx, rx) = oneshot::channel(); - unsynced - .inner - .add_ons_handle - .rollup_manager_handle - .send_command(RollupManagerCommand::Status(tx)) - .await; - let status = rx.await.unwrap(); - assert!(status.syncing); + // Assert that the unsynced node triggers optimistic sync. + wait_n_events( + &mut unsynced_events, + |e| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent(ChainOrchestratorEvent::OptimisticSync( + _ + )) + ) + }, + 1, + ) + .await; // Verify the unsynced node syncs. let provider = ProviderBuilder::new().connect_http(unsynced.rpc_url()); @@ -133,7 +149,7 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { let mut num = provider.get_block_number().await.unwrap(); loop { - if retries > 10 || num > en_sync_trigger { + if retries > 10 || num > optimistic_sync_trigger { break } num = provider.get_block_number().await.unwrap(); @@ -141,28 +157,593 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { retries += 1; } - // Wait at least a single block for the driver to exit sync mode. - wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::BlockImported(_)), 1).await; + // 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( + _ + )) + ) + }, + 1, + ) + .await; + + Ok(()) +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_should_consolidate_after_optimistic_sync() -> 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: NetworkArgs { + enable_eth_scroll_wire_bridge: true, + enable_scroll_wire: true, + sequencer_url: None, + eth_wire_gossip: false, + }, + database_args: DatabaseArgs { 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, + block_time: 0, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + ..SequencerArgs::default() + }, + beacon_provider_args: BeaconProviderArgs { + blob_source: BlobSource::Mock, + ..Default::default() + }, + signer_args: Default::default(), + gas_price_oracle_args: GasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + }; + + // Create the chain spec for scroll dev with Euclid v2 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.add_ons_handle.rollup_manager_handle.get_event_listener().await?; + + // 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); + for i in 0..L1_MESSAGES_COUNT as u64 { + let l1_message = TxL1Message { + queue_index: i, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }; + l1_messages.push(l1_message); + } + + // Add the L1 messages to the sequencer node. + for (i, l1_message) in l1_messages.iter().enumerate() { + sequencer_l1_watcher_tx + .send(Arc::new(L1Notification::L1Message { + message: l1_message.clone(), + block_number: i as u64, + block_timestamp: i as u64 * 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(i as u64))).await.unwrap(); + wait_n_events( + &mut sequencer_events, + |e| matches!(e, RollupManagerEvent::L1NotificationEvent(L1Notification::NewBlock(_))), + 1, + ) + .await; + sequencer_handle.build_block().await; + wait_n_events( + &mut sequencer_events, + |e: RollupManagerEvent| matches!(e, RollupManagerEvent::BlockSequenced(_)), + 1, + ) + .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; + + // 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( + _ + )) + ) + }, + 1, + ) + .await; + + // Let the unsynced node process the optimistic sync. + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + + // Send all L1 messages to the unsynced node. + for (i, l1_message) in l1_messages.iter().enumerate() { + follower_l1_watcher_tx + .send(Arc::new(L1Notification::L1Message { + message: l1_message.clone(), + block_number: i as u64, + block_timestamp: i as u64 * 10, + })) + .await + .unwrap(); + wait_n_events( + &mut follower_events, + |e: RollupManagerEvent| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted( + _ + ) + ) + ) + }, + 1, + ) + .await; + } + + // 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; + + // 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; + + // 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: 200, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }, + block_number: 200, + block_timestamp: 2010, + })) + .await + .unwrap(); + wait_n_events( + &mut sequencer_events, + |e: RollupManagerEvent| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + rollup_node_chain_orchestrator::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; + sequencer_handle.build_block().await; + + wait_n_events( + &mut follower_events, + |e| matches!(e, RollupManagerEvent::NewBlockReceived(_)), + 1, + ) + .await; + + // Assert that the follower node does not accept the new block as it does not have the L1 + // message. + wait_n_events( + &mut follower_events, + |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { start: _ }), + 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: NetworkArgs { + enable_eth_scroll_wire_bridge: true, + enable_scroll_wire: true, + sequencer_url: None, + eth_wire_gossip: false, + }, + database_args: DatabaseArgs { 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, + block_time: 0, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + ..SequencerArgs::default() + }, + beacon_provider_args: BeaconProviderArgs { + blob_source: BlobSource::Mock, + ..Default::default() + }, + signer_args: Default::default(), + gas_price_oracle_args: GasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + }; + + // Create the chain spec for scroll dev with Euclid v2 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 { start: _ }), + 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: NetworkArgs { + enable_eth_scroll_wire_bridge: false, + enable_scroll_wire: true, + ..Default::default() + }, + database_args: DatabaseArgs { 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, + block_time: 0, + max_l1_messages_per_block: 4, + l1_message_inclusion_mode: L1MessageInclusionMode::BlockDepth(0), + ..SequencerArgs::default() + }, + beacon_provider_args: BeaconProviderArgs { + blob_source: BlobSource::Mock, + ..Default::default() + }, + signer_args: Default::default(), + gas_price_oracle_args: GasPriceOracleArgs::default(), + consensus_args: ConsensusArgs::noop(), + }; + + // Create the chain spec for scroll dev with Euclid v2 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.network.add_peer(follower.network.record()).await; + follower.network.next_session_established().await; + sequencer.network.next_session_established().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; - // Check the unsynced node exits sync mode. - let (tx, rx) = oneshot::channel(); - unsynced - .inner - .add_ons_handle - .rollup_manager_handle - .send_command(RollupManagerCommand::Status(tx)) + // 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; - let status = rx.await.unwrap(); - assert!(!status.syncing); + + // 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( - node: &NodeHelperType, - matches: impl Fn(RollupManagerEvent) -> bool, + events: &mut EventStream, + mut matches: impl FnMut(RollupManagerEvent) -> bool, mut n: u64, ) { - let mut events = node.inner.rollup_manager_handle.get_event_listener().await.unwrap(); while let Some(event) = events.next().await { if matches(event) { n -= 1; diff --git a/crates/primitives/Cargo.toml b/crates/primitives/Cargo.toml index 7dbbc3ca..d1af8a3d 100644 --- a/crates/primitives/Cargo.toml +++ b/crates/primitives/Cargo.toml @@ -22,6 +22,7 @@ scroll-alloy-consensus.workspace = true scroll-alloy-rpc-types-engine.workspace = true # reth +reth-network-peers.workspace = true reth-primitives-traits.workspace = true reth-scroll-primitives.workspace = true @@ -42,6 +43,7 @@ std = [ "reth-primitives-traits/std", "alloy-consensus/std", "alloy-chains/std", + "reth-network-peers/std", ] arbitrary = [ "std", diff --git a/crates/primitives/src/block.rs b/crates/primitives/src/block.rs index 2682ae48..0f615d2c 100644 --- a/crates/primitives/src/block.rs +++ b/crates/primitives/src/block.rs @@ -1,3 +1,4 @@ +use alloy_consensus::Header; use alloy_eips::{BlockNumHash, Decodable2718}; use alloy_primitives::{B256, U256}; use alloy_rpc_types_engine::ExecutionPayload; @@ -54,6 +55,18 @@ impl From<&ScrollBlock> for BlockInfo { } } +impl From<&Header> for BlockInfo { + fn from(value: &Header) -> Self { + Self { number: value.number, hash: value.hash_slow() } + } +} + +impl From

for BlockInfo { + fn from(value: Header) -> Self { + Self { number: value.number, hash: value.hash_slow() } + } +} + #[cfg(feature = "arbitrary")] impl arbitrary::Arbitrary<'_> for BlockInfo { fn arbitrary(u: &mut arbitrary::Unstructured<'_>) -> arbitrary::Result { diff --git a/crates/primitives/src/bounded_vec.rs b/crates/primitives/src/bounded_vec.rs index 55a185f6..9a659c3e 100644 --- a/crates/primitives/src/bounded_vec.rs +++ b/crates/primitives/src/bounded_vec.rs @@ -21,7 +21,7 @@ impl BoundedVec { /// Pushes a value at the back of the buffer. If the buffer is full, pops the data at the front /// of the buffer first. - pub fn push(&mut self, elem: T) { + pub fn push_back(&mut self, elem: T) { if self.is_full() { self.data.pop_front(); } @@ -29,6 +29,16 @@ impl BoundedVec { self.data.push_back(elem) } + /// Pushes a value at the front of the buffer. If the buffer is full, pops the data at the back + /// of the buffer first. + pub fn push_front(&mut self, elem: T) { + if self.is_full() { + self.data.pop_back(); + } + + self.data.push_front(elem) + } + /// Pops the last element from the structure and returns it if any. pub fn pop(&mut self) -> Option { self.data.pop_back() @@ -39,6 +49,11 @@ impl BoundedVec { self.data.back() } + /// Returns the first element in the vector, if any. + pub fn first(&self) -> Option<&T> { + self.data.front() + } + /// Clears the structure by removing all the elements. pub fn clear(&mut self) { self.data.clear() @@ -56,23 +71,27 @@ impl BoundedVec { fn is_full(&self) -> bool { self.data.len() == self.data.capacity() } + + /// Returns the inner `VecDeque` of the bounded vec. + pub const fn inner(&self) -> &VecDeque { + &self.data + } + + /// Returns a mutable reference to the inner `VecDeque` of the bounded vec. + pub const fn inner_mut(&mut self) -> &mut VecDeque { + &mut self.data + } + + /// Returns the inner `VecDeque` of the bounded vec. + pub fn into_inner(self) -> VecDeque { + self.data + } } impl Extend for BoundedVec { fn extend>(&mut self, iter: I) { - let iter = iter.into_iter(); - - // if size hint returns an upper bound, skip values until whole iterator can fit in the - // bounded vec. - let iter = if let (_, Some(upper_bound)) = iter.size_hint() { - iter.skip(upper_bound.saturating_sub(self.data.capacity())) - } else { - #[allow(clippy::iter_skip_zero)] - iter.skip(0) - }; - for elem in iter { - self.push(elem) + self.push_back(elem) } } } diff --git a/crates/primitives/src/chain.rs b/crates/primitives/src/chain.rs new file mode 100644 index 00000000..2568960e --- /dev/null +++ b/crates/primitives/src/chain.rs @@ -0,0 +1,24 @@ +use alloy_primitives::Signature; +use reth_network_peers::PeerId; +use reth_scroll_primitives::ScrollBlock; +use std::vec::Vec; + +/// A structure representing a chain import, which includes a vector of blocks, +/// the peer ID from which the blocks were received, and a signature for the import of the chain +/// tip. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ChainImport { + /// The blocks that are part of the chain import. + pub chain: Vec, + /// The peer ID from which the blocks were received. + pub peer_id: PeerId, + /// The signature for the import of the chain tip. + pub signature: Signature, +} + +impl ChainImport { + /// Creates a new `ChainImport` instance with the provided blocks, peer ID, and signature. + pub const fn new(blocks: Vec, peer_id: PeerId, signature: Signature) -> Self { + Self { chain: blocks, peer_id, signature } + } +} diff --git a/crates/primitives/src/lib.rs b/crates/primitives/src/lib.rs index 241c14fe..23079be0 100644 --- a/crates/primitives/src/lib.rs +++ b/crates/primitives/src/lib.rs @@ -16,6 +16,9 @@ pub use batch::{BatchCommitData, BatchInfo}; mod bounded_vec; pub use bounded_vec::BoundedVec; +mod chain; +pub use chain::ChainImport; + mod metadata; pub use metadata::Metadata; diff --git a/crates/scroll-wire/src/protocol/proto.rs b/crates/scroll-wire/src/protocol/proto.rs index 92472371..473fb837 100644 --- a/crates/scroll-wire/src/protocol/proto.rs +++ b/crates/scroll-wire/src/protocol/proto.rs @@ -30,7 +30,7 @@ pub struct NewBlock { } impl NewBlock { - /// Returns a [`NewBlock`] instance with the provided signature and block. + /// Returns a [`NewBlock`] instance with the provided signature and blocks. pub fn new(signature: Signature, block: reth_scroll_primitives::ScrollBlock) -> Self { Self { signature: Bytes::from(Into::>::into(signature)), block } } diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 6b3cb292..ac343adc 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -10,8 +10,9 @@ use reth_scroll_chainspec::SCROLL_DEV; use reth_scroll_node::test_utils::setup; use rollup_node::{ test_utils::{default_test_scroll_rollup_node_config, setup_engine}, - BeaconProviderArgs, ConsensusArgs, DatabaseArgs, EngineDriverArgs, GasPriceOracleArgs, - L1ProviderArgs, NetworkArgs, ScrollRollupNodeConfig, SequencerArgs, SignerArgs, + BeaconProviderArgs, ChainOrchestratorArgs, ConsensusArgs, DatabaseArgs, EngineDriverArgs, + GasPriceOracleArgs, L1ProviderArgs, NetworkArgs, ScrollRollupNodeConfig, SequencerArgs, + SignerArgs, }; use rollup_node_manager::RollupManagerEvent; use rollup_node_primitives::{sig_encode_hash, BlockInfo, L1MessageEnvelope}; @@ -40,7 +41,7 @@ async fn can_build_blocks() { reth_tracing::init_test_tracing(); const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - const BLOCK_GAP_TRIGGER: u64 = 100; + // const BLOCK_GAP_TRIGGER: u64 = 100; // setup a test node let (mut nodes, _tasks, wallet) = setup(1, false).await.unwrap(); @@ -64,7 +65,6 @@ async fn can_build_blocks() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -158,7 +158,6 @@ async fn can_build_blocks_with_delayed_l1_messages() { let chain_spec = SCROLL_DEV.clone(); const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(0); - const BLOCK_GAP_TRIGGER: u64 = 100; const L1_MESSAGE_DELAY: u64 = 2; // setup a test node @@ -186,7 +185,6 @@ async fn can_build_blocks_with_delayed_l1_messages() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -286,7 +284,7 @@ async fn can_build_blocks_with_finalized_l1_messages() { let chain_spec = SCROLL_DEV.clone(); const BLOCK_BUILDING_DURATION: Duration = tokio::time::Duration::from_millis(0); - const BLOCK_GAP_TRIGGER: u64 = 100; + // const BLOCK_GAP_TRIGGER: u64 = 100; // setup a test node let (mut nodes, _tasks, wallet) = @@ -313,7 +311,6 @@ async fn can_build_blocks_with_finalized_l1_messages() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -438,6 +435,7 @@ async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { database_args: DatabaseArgs { 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, block_time: 0, @@ -529,6 +527,7 @@ async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result< database_args: DatabaseArgs { 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, block_time: 0, @@ -581,16 +580,17 @@ async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result< } // 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"); + 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(()) @@ -603,7 +603,7 @@ 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 BLOCK_GAP_TRIGGER: u64 = 100; + // const BLOCK_GAP_TRIGGER: u64 = 100; const TRANSACTIONS_COUNT: usize = 2000; // setup a test node. use a high value for the payload building duration to be sure we don't @@ -654,7 +654,6 @@ async fn can_build_blocks_and_exit_at_gas_limit() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -690,7 +689,7 @@ async fn can_build_blocks_and_exit_at_time_limit() { let chain_spec = SCROLL_DEV.clone(); const MIN_TRANSACTION_GAS_COST: u64 = 21_000; const BLOCK_BUILDING_DURATION: Duration = Duration::from_secs(1); - const BLOCK_GAP_TRIGGER: u64 = 100; + // const BLOCK_GAP_TRIGGER: u64 = 100; const TRANSACTIONS_COUNT: usize = 2000; // setup a test node. use a low payload building duration in order to exit before we reach the @@ -741,7 +740,6 @@ async fn can_build_blocks_and_exit_at_time_limit() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); diff --git a/crates/watcher/src/lib.rs b/crates/watcher/src/lib.rs index b84531d0..09f027ab 100644 --- a/crates/watcher/src/lib.rs +++ b/crates/watcher/src/lib.rs @@ -82,6 +82,8 @@ pub struct L1Watcher { config: Arc, /// The metrics for the watcher. metrics: WatcherMetrics, + /// Whether the watcher is synced to the L1 head. + is_synced: bool, } /// The L1 notification type yielded by the [`L1Watcher`]. @@ -115,6 +117,8 @@ pub enum L1Notification { NewBlock(u64), /// A block has been finalized on the L1. Finalized(u64), + /// A notification that the L1 watcher is synced to the L1 head. + Synced, } impl Display for L1Notification { @@ -136,6 +140,7 @@ impl Display for L1Notification { Self::Consensus(u) => write!(f, "{u:?}"), Self::NewBlock(n) => write!(f, "NewBlock({n})"), Self::Finalized(n) => write!(f, "Finalized({n})"), + Self::Synced => write!(f, "Synced"), } } } @@ -151,6 +156,8 @@ where start_block: Option, config: Arc, ) -> mpsc::Receiver> { + tracing::trace!(target: "scroll::watcher", ?start_block, ?config, "spawning L1 watcher"); + let (tx, rx) = mpsc::channel(LOGS_QUERY_BLOCK_RANGE as usize); let fetch_block_number = async |tag: BlockNumberOrTag| { @@ -181,6 +188,7 @@ where sender: tx, config, metrics: WatcherMetrics::default(), + is_synced: false, }; // notify at spawn. @@ -212,8 +220,17 @@ where } // sleep if we are synced. - if self.is_synced() { + if self.is_synced { tokio::time::sleep(SLOW_SYNC_INTERVAL).await; + } else if self.current_block_number == self.l1_state.head { + // if we have synced to the head of the L1, notify the channel and set the + // `is_synced`` flag. + if let Err(L1WatcherError::SendError(_)) = self.notify(L1Notification::Synced).await + { + tracing::warn!(target: "scroll::watcher", "L1 watcher channel closed, stopping the watcher"); + break; + } + self.is_synced = true; } } } @@ -316,7 +333,7 @@ where } else if tail.is_some_and(|h| h.hash == latest.parent_hash) { // latest block extends the tip. tracing::trace!(target: "scroll::watcher", number = ?latest.number, hash = ?latest.hash, "block extends chain"); - self.unfinalized_blocks.push(latest.clone()); + self.unfinalized_blocks.push_back(latest.clone()); } else { // chain reorged or need to backfill. tracing::trace!(target: "scroll::watcher", number = ?latest.number, hash = ?latest.hash, "gap or reorg"); @@ -569,11 +586,6 @@ where Ok(prefix) } - /// Returns true if the [`L1Watcher`] is synced to the head of the L1. - const fn is_synced(&self) -> bool { - self.current_block_number == self.l1_state.head - } - /// Send all notifications on the channel. async fn notify_all(&self, notifications: Vec) { for notification in notifications { @@ -688,6 +700,7 @@ mod tests { sender: tx, config: Arc::new(NodeConfig::mainnet()), metrics: WatcherMetrics::default(), + is_synced: false, }, rx, ) diff --git a/crates/watcher/tests/reorg.rs b/crates/watcher/tests/reorg.rs index 975a210d..1088d1ed 100644 --- a/crates/watcher/tests/reorg.rs +++ b/crates/watcher/tests/reorg.rs @@ -91,15 +91,19 @@ async fn test_should_detect_reorg() -> eyre::Result<()> { continue } + // skip the `L1Notification::Synced` notifications + let mut notification = l1_watcher.recv().await.unwrap(); + if matches!(notification.as_ref(), L1Notification::Synced) { + notification = l1_watcher.recv().await.unwrap(); + } + // check latest for reorg or new block. if latest_number > latest.header.number { // reorg - let notification = l1_watcher.recv().await.unwrap(); assert!(matches!(notification.as_ref(), L1Notification::Reorg(_))); let notification = l1_watcher.recv().await.unwrap(); assert_eq!(notification.as_ref(), &L1Notification::NewBlock(latest.header.number)); } else { - let notification = l1_watcher.recv().await.unwrap(); assert_eq!(notification.as_ref(), &L1Notification::NewBlock(latest.header.number)); } @@ -178,7 +182,12 @@ async fn test_should_fetch_gap_in_unfinalized_blocks() -> eyre::Result<()> { continue } - let notification = l1_watcher.recv().await.unwrap(); + // skip the `L1Notification::Synced` notifications + let mut notification = l1_watcher.recv().await.unwrap(); + if matches!(notification.as_ref(), L1Notification::Synced) { + notification = l1_watcher.recv().await.unwrap(); + } + assert_eq!(notification.as_ref(), &L1Notification::NewBlock(latest.header.number)); // update finalized and latest.