From c1e5cc21db06f5ad0ae93d888708391ea988db8d Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 2 Jul 2025 19:44:32 +0100 Subject: [PATCH 01/30] feat: chain orchestrator --- Cargo.lock | 21 + crates/database/db/src/db.rs | 7 +- crates/database/db/src/operations.rs | 42 ++ crates/database/migration/Cargo.toml | 1 + crates/database/migration/src/lib.rs | 2 +- ...220101_000001_create_batch_commit_table.rs | 25 +- .../src/m20250411_072004_add_l2_block.rs | 33 +- .../database/migration/src/migration_info.rs | 19 + crates/engine/src/driver.rs | 83 ++- crates/engine/src/event.rs | 14 + crates/engine/src/future/mod.rs | 85 ++- crates/engine/src/future/result.rs | 7 + crates/indexer/Cargo.toml | 20 + crates/indexer/src/action.rs | 11 +- crates/indexer/src/error.rs | 3 + crates/indexer/src/event.rs | 62 +- crates/indexer/src/lib.rs | 664 +++++++++++++++++- crates/indexer/src/metrics.rs | 7 +- crates/indexer/testdata/genesis_block.json | 27 + .../indexer/testdata/genesis_block_rpc.json | 22 + crates/manager/src/manager/event.rs | 8 + crates/manager/src/manager/mod.rs | 153 ++-- crates/network/Cargo.toml | 3 + crates/network/src/manager.rs | 98 ++- crates/node/src/args.rs | 49 +- crates/node/tests/e2e.rs | 16 +- crates/node/tests/sync.rs | 31 +- crates/primitives/Cargo.toml | 1 + crates/primitives/src/block.rs | 13 + crates/primitives/src/bounded_vec.rs | 5 + crates/primitives/src/chain.rs | 23 + crates/primitives/src/lib.rs | 3 + crates/scroll-wire/src/protocol/proto.rs | 2 +- crates/sequencer/tests/e2e.rs | 6 - 34 files changed, 1295 insertions(+), 271 deletions(-) create mode 100644 crates/indexer/testdata/genesis_block.json create mode 100644 crates/indexer/testdata/genesis_block_rpc.json create mode 100644 crates/primitives/src/chain.rs diff --git a/Cargo.lock b/Cargo.lock index 676f89a8..fa897566 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10106,23 +10106,39 @@ dependencies = [ name = "rollup-node-indexer" version = "0.0.1" dependencies = [ + "alloy-consensus 1.0.9", + "alloy-eips 1.0.9", "alloy-primitives", + "alloy-provider", + "alloy-rpc-client", + "alloy-transport", "arbitrary", "futures", "metrics", "metrics-derive", + "parking_lot 0.12.3", "rand 0.9.1", + "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.1", "thiserror 2.0.12", "tokio", + "tracing", ] [[package]] @@ -10176,6 +10192,7 @@ dependencies = [ "alloy-rpc-types-engine 1.0.9", "arbitrary", "derive_more", + "reth-network-peers", "reth-primitives-traits", "reth-scroll-primitives", "scroll-alloy-consensus", @@ -10868,6 +10885,7 @@ dependencies = [ "reqwest-middleware", "reqwest-retry", "reth-chainspec", + "reth-scroll-chainspec", "sea-orm", "sea-orm-migration", "sha2 0.10.9", @@ -10881,6 +10899,7 @@ dependencies = [ "alloy-primitives", "futures", "parking_lot 0.12.3", + "reth-chainspec", "reth-eth-wire-types", "reth-network", "reth-network-api", @@ -10891,6 +10910,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/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index 580fa482..560823f1 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -412,9 +412,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() }; diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index 854b1942..f8b1076b 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -250,6 +250,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, Option) = 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, @@ -298,6 +316,18 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|x| x.map(|x| x.block_info()))?) } + /// 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) + .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 /// rollup-node. /// @@ -346,6 +376,18 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|x| x.rows_affected)?) } + /// Insert multiple blocks into the database. + async fn insert_blocks( + &self, + blocks: Vec, + batch_info: Option, + ) -> 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, 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..1f383f7a 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( @@ -35,7 +41,24 @@ impl MigrationTrait for Migration { ) .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 1109add6..282c2c08 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!("0xb5bd7381c6b550af0de40d6c490602574d76427c8cce17b54cb7917c323136f2") + } } /// The type implementing migration info for Mainnet. @@ -33,6 +40,10 @@ impl MigrationInfo for ScrollMainnetMigrationInfo { fn data_hash() -> Option { Some(b256!("9062e2fa1200dca63bee1d18d429572f134f5f0c98cb4852f62fc394e33cf6e6")) } + + 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!("3629f5e53250a526ffc46806c4d74b9c52c9209a6d45ecdfebdef5d596bb3f40")) } + + fn genesis_hash() -> B256 { + SCROLL_SEPOLIA_GENESIS_HASH + } } diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs index a3dca734..6fe74f2b 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -6,12 +6,13 @@ use crate::{ }; use futures::{ready, task::AtomicWaker, FutureExt, Stream}; -use rollup_node_primitives::{BlockInfo, MeteredFuture, ScrollPayloadAttributesWithBatchInfo}; +use rollup_node_primitives::{ + BlockInfo, ChainImport, MeteredFuture, ScrollPayloadAttributesWithBatchInfo, +}; use rollup_node_providers::ExecutionPayloadProvider; use scroll_alloy_hardforks::ScrollHardforks; use scroll_alloy_provider::ScrollEngineApi; use scroll_alloy_rpc_types_engine::ScrollPayloadAttributes; -use scroll_network::NewBlockWithPeer; use std::{ collections::VecDeque, pin::Pin, @@ -33,14 +34,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. @@ -66,7 +67,6 @@ where provider: Option

, fcs: ForkchoiceState, sync_at_start_up: bool, - block_gap_sync_trigger: u64, block_building_duration: Duration, ) -> Self { Self { @@ -76,9 +76,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, @@ -108,17 +108,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.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(); - self.block_imports.push_back(block_with_peer); + // 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(); } @@ -227,6 +234,18 @@ where } } } + 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 @@ -322,13 +341,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; @@ -392,15 +421,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()); @@ -426,7 +448,6 @@ mod tests { None::, fcs, false, - 0, duration, ); 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 2eca4977..62a85c73 100644 --- a/crates/engine/src/future/mod.rs +++ b/crates/engine/src/future/mod.rs @@ -1,20 +1,21 @@ use super::{payload::matching_payloads, EngineDriverError}; use crate::{api::*, ForkchoiceState}; 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 rollup_node_providers::ExecutionPayloadProvider; use scroll_alloy_hardforks::ScrollHardforks; 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, @@ -28,7 +29,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< @@ -39,6 +40,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 { @@ -75,10 +80,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>>; @@ -87,25 +88,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. @@ -148,9 +160,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), } } } @@ -160,42 +173,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. @@ -207,7 +226,7 @@ where Some(block_info), Some(BlockImportOutcome::valid_block( peer_id, - block, + head, Into::>::into(signature).into(), )), PayloadStatusEnum::Valid, 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/Cargo.toml b/crates/indexer/Cargo.toml index b761deb6..01fccd81 100644 --- a/crates/indexer/Cargo.toml +++ b/crates/indexer/Cargo.toml @@ -11,7 +11,10 @@ workspace = true [dependencies] # alloy +alloy-consensus = { workspace = true } +alloy-eips = { workspace = true } alloy-primitives.workspace = true +alloy-provider.workspace = true # rollup-node scroll-db.workspace = true @@ -19,11 +22,16 @@ 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 # misc futures.workspace = true @@ -31,10 +39,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 +56,16 @@ 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-primitives-traits.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/indexer/src/action.rs b/crates/indexer/src/action.rs index 6c889925..21be9f93 100644 --- a/crates/indexer/src/action.rs +++ b/crates/indexer/src/action.rs @@ -1,4 +1,4 @@ -use super::{IndexerError, IndexerEvent}; +use super::{ChainOrchestratorEvent, IndexerError}; use std::{ fmt, future::Future, @@ -8,7 +8,7 @@ use std::{ /// A future that resolves to a `Result`. pub(super) type PendingIndexerFuture = - Pin> + Send>>; + Pin> + Send>>; /// A type that represents a future that is being executed by the indexer. pub(super) enum IndexerFuture { @@ -18,6 +18,7 @@ pub(super) enum IndexerFuture { HandleBatchFinalization(PendingIndexerFuture), HandleL1Message(PendingIndexerFuture), HandleDerivedBlock(PendingIndexerFuture), + HandleL2Block(PendingIndexerFuture), } impl IndexerFuture { @@ -25,14 +26,15 @@ impl IndexerFuture { pub(super) fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll> { + ) -> 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), + Self::HandleDerivedBlock(fut) | + Self::HandleL2Block(fut) => fut.as_mut().poll(cx), } } } @@ -48,6 +50,7 @@ impl fmt::Debug for IndexerFuture { Self::HandleBatchFinalization(_) => write!(f, "HandleBatchFinalization"), Self::HandleL1Message(_) => write!(f, "HandleL1Message"), Self::HandleDerivedBlock(_) => write!(f, "HandleDerivedBlock"), + Self::HandleL2Block(_) => write!(f, "HandleL2Block"), } } } diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs index 5552440f..83a15b3c 100644 --- a/crates/indexer/src/error.rs +++ b/crates/indexer/src/error.rs @@ -9,4 +9,7 @@ pub enum IndexerError { /// 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, } diff --git a/crates/indexer/src/event.rs b/crates/indexer/src/event.rs index 008996a7..3189024f 100644 --- a/crates/indexer/src/event.rs +++ b/crates/indexer/src/event.rs @@ -1,21 +1,46 @@ -use alloy_primitives::B256; -use rollup_node_primitives::{BatchInfo, BlockInfo, L2BlockInfoWithL1Messages}; +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 indexer. +/// An event emitted by the `ChainOrchestrator`. #[derive(Debug, Clone, PartialEq, Eq)] -pub enum IndexerEvent { - /// A `BatchCommit` event has been indexed returning the batch info. - BatchCommitIndexed(BatchInfo), - /// A `BatchFinalization` event has been indexed returning the batch hash and new finalized L2 - /// block. - BatchFinalizationIndexed(B256, Option), - /// A `Finalized` event has been indexed returning the block number and new finalized L2 - /// block. - FinalizedIndexed(u64, Option), - /// A `L1Message` event has been indexed returning the message queue index. - L1MessageIndexed(u64), - /// A `Unwind` event has been indexed returning the reorg block number. - UnwindIndexed { +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 batch has been committed returning the batch info. + BatchCommitted(BatchInfo), + /// 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. @@ -25,6 +50,7 @@ pub enum IndexerEvent { /// The L2 safe block info. l2_safe_block_info: Option, }, - /// A block has been indexed returning batch and block info. - BlockIndexed(L2BlockInfoWithL1Messages, Option), + /// An L2 block has been committed returning the [`L2BlockInfoWithL1Messages`] and an + /// optional [`BatchInfo`] if the block is associated with a committed batch. + L2BlockCommitted(L2BlockInfoWithL1Messages, Option), } diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index 0827fe89..a8d13096 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -1,21 +1,29 @@ //! A library responsible for indexing data relevant to the L1. +use alloy_consensus::Header; +use alloy_eips::BlockHashOrNumber; use alloy_primitives::{b256, keccak256, B256}; -use futures::Stream; +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, L1MessageEnvelope, L2BlockInfoWithL1Messages, + 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, UnwindResult}; +use scroll_network::NewBlockWithPeer; use std::{ collections::{HashMap, VecDeque}, pin::Pin, sync::{ atomic::{AtomicU64, Ordering}, - Arc, + Arc, Mutex, }, task::{Context, Poll}, time::Instant, @@ -26,7 +34,7 @@ mod action; use action::{IndexerFuture, PendingIndexerFuture}; mod event; -pub use event::IndexerEvent; +pub use event::ChainOrchestratorEvent; mod error; pub use error::IndexerError; @@ -37,9 +45,24 @@ pub use metrics::{IndexerItem, IndexerMetrics}; const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); +/// The number of block headers we keep in memory. +const CHAIN_BUFFER_SIZE: usize = 2000; + +/// The threshold for optimistic syncing. If the received block is more than this many blocks +/// ahead of the current chain, we optimistically sync the chain. +const OPTIMISTIC_SYNC_THRESHOLD: u64 = 100; + +type Chain = BoundedVec
; + /// The indexer is responsible for indexing data relevant to the L1. #[derive(Debug)] -pub struct Indexer { +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. @@ -52,12 +75,32 @@ pub struct Indexer { chain_spec: Arc, /// The metrics for the indexer. metrics: HashMap, + /// A boolean to represent if the [`ChainOrchestrator`] is in optimistic mode. + optimistic_mode: Arc>, + /// 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 Indexer { +impl< + ChainSpec: ScrollHardforks + EthChainSpec + Send + Sync + 'static, + BC: BlockClient + Send + Sync + 'static, + P: Provider + 'static, + > ChainOrchestrator +{ /// Creates a new indexer with the given [`Database`]. - pub fn new(database: Arc, chain_spec: Arc) -> Self { + pub async fn new( + database: Arc, + chain_spec: Arc, + block_client: BC, + l2_client: P, + ) -> Self { + let chain = init_chain_from_db(&database, &l2_client).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)), @@ -69,6 +112,9 @@ impl Indexer< (i, IndexerMetrics::new_with_labels(&[("item", label)])) }) .collect(), + optimistic_mode: Arc::new(Mutex::new(false)), + l1_synced: false, + waker: AtomicWaker::new(), } } @@ -88,18 +134,288 @@ impl Indexer< fut_wrapper } - /// Handles an L2 block. - pub fn handle_block( + /// 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 chain = self.chain.clone(); + let l2_client = self.l2_client.clone(); + let optimistic_mode = self.optimistic_mode.clone(); + let network_client = self.network_client.clone(); + let database = self.database.clone(); + let fut = self.handle_metered( + IndexerItem::NewBlock, + Box::pin(async move { + Self::handle_new_block( + chain, + l2_client, + optimistic_mode, + network_client, + database, + block_with_peer, + ) + .await + }), + ); + self.pending_futures.push_back(IndexerFuture::HandleL2Block(fut)); + self.waker.wake(); + } + + /// Handles a new block received from the network. + pub async fn handle_new_block( + chain: Arc>, + l2_client: Arc

, + optimistic_mode: Arc>, + network_client: Arc, + database: Arc, + block_with_peer: NewBlockWithPeer, + ) -> Result { + let NewBlockWithPeer { block: received_block, peer_id, signature } = block_with_peer; + let mut current_chain_headers = std::mem::take(&mut *chain.lock().unwrap()); + let max_block_number = current_chain_headers.last().expect("chain can not be empty").number; + let min_block_number = + current_chain_headers.first().expect("chain can not be empty").number; + let optimistic_mode_local: bool = { + let guard = optimistic_mode.lock().unwrap(); + *guard + }; + + // 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. + if (received_block.header.number - max_block_number) >= OPTIMISTIC_SYNC_THRESHOLD { + // fetch the latest `OPTIMISTIC_CHAIN_BUFFER_SIZE` blocks from the network for the + // optimistic chain. + let mut optimistic_headers = vec![received_block.header.clone()]; + while optimistic_headers.len() < CHAIN_BUFFER_SIZE && + optimistic_headers.last().unwrap().number != 0 + { + tracing::trace!(target: "scroll::watcher", number = ?(optimistic_headers.last().unwrap().number - 1), "fetching block"); + let header = network_client + .get_header(BlockHashOrNumber::Hash( + optimistic_headers.last().unwrap().parent_hash, + )) + .await + .unwrap() + .into_data() + .unwrap(); + optimistic_headers.push(header); + } + optimistic_headers.reverse(); + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); + new_chain.extend(optimistic_headers); + *chain.lock().unwrap() = new_chain; + *optimistic_mode.lock().unwrap() = 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::watcher", block_hash = ?received_block.header.hash_slow(), "block already in chain"); + return Ok(ChainOrchestratorEvent::BlockAlreadyKnown( + received_block.header.hash_slow(), + peer_id, + )); + } + + // Perform preliminary checks on received block that are dependent on database state due to + // not being in in-memory chain. + if received_block.header.number <= min_block_number { + // If we are in optimistic mode, we return an event indicating that we have insufficient + // data to process the block. + if optimistic_mode_local { + return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( + received_block.header.hash_slow(), + )); + } + + // We check the database to see if this block is already known. + if database + .get_l2_block_and_batch_info_by_hash(received_block.header.hash_slow()) + .await? + .is_some() + { + tracing::debug!(target: "scroll::watcher", block_hash = ?received_block.header.hash_slow(), "block already in database"); + return Ok(ChainOrchestratorEvent::BlockAlreadyKnown( + received_block.header.hash_slow(), + peer_id, + )); + } + }; + + let mut new_chain_headers = vec![received_block.header.clone()]; + let mut new_header_tail = received_block.header.clone(); + + enum ReorgIndex { + Memory(usize), + Database(BlockInfo), + } + + // 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 or the database. + let reorg_index = { + loop { + // If the current header block number is greater than the in-memory chain then we + // should search the in-memory chain. + if new_header_tail.number >= min_block_number { + if let Some(pos) = current_chain_headers + .iter() + .rposition(|h| h.hash_slow() == new_header_tail.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 >= + new_header_tail.timestamp + { + return Ok(ChainOrchestratorEvent::OldForkReceived { + headers: new_chain_headers, + peer_id, + signature, + }); + } + break Some(ReorgIndex::Memory(pos)); + } + // If we are in optimistic mode, we terminate the search as we don't have the + // necessary data in the database. This is fine because very deep + // re-orgs are rare and in any case will be resolved once optimistic sync is + // completed.If the current header block number is less than the + // latest safe block number then this would suggest a reorg of a + // safe block which is not invalid - terminate the search. + } else if optimistic_mode_local || + new_header_tail.number <= latest_safe_block.number + { + break None + + // If the block is not in the in-memory chain, we search the database. + } else if let Some((l2_block, _batch_info)) = database + .get_l2_block_and_batch_info_by_hash(new_header_tail.parent_hash) + .await? + { + let diverged_block = database + .get_l2_block_info_by_number(l2_block.number + 1) + .await? + .expect("diverged block must exist"); + let diverged_block = + l2_client.get_block_by_hash(diverged_block.hash).await.unwrap().unwrap(); + + // If the received fork is older than the current chain, we return an event + // indicating that we have received an old fork. + if diverged_block.header.timestamp >= new_header_tail.timestamp { + return Ok(ChainOrchestratorEvent::OldForkReceived { + headers: new_chain_headers, + peer_id, + signature, + }); + } + + break Some(ReorgIndex::Database(l2_block)) + } + + tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); + let header = network_client + .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) + .await + .unwrap() + .into_data() + .unwrap(); + new_chain_headers.push(header.clone()); + new_header_tail = header; + } + }; + + // Reverse the new chain headers to have them in the correct order. + new_chain_headers.reverse(); + + // Fetch the blocks associated with the new chain headers. + let new_blocks = if new_chain_headers.len() == 1 { + vec![received_block] + } else { + fetch_blocks(new_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_local { + 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. + Some(ReorgIndex::Memory(index)) if index == current_chain_headers.len() - 1 => { + // Update the chain with the new blocks. + current_chain_headers.extend(new_blocks.iter().map(|b| b.header.clone())); + *chain.lock().unwrap() = current_chain_headers; + + 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. + Some(ReorgIndex::Memory(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(new_chain_headers); + *chain.lock().unwrap() = new_chain; + + Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( + new_blocks, peer_id, signature, + ))) + } + // If we have a deep reorg that impacts the database, we need to delete the blocks from + // the database, update the in-memory chain and emit a ChainReorged event. + Some(ReorgIndex::Database(l2_block)) => { + // remove old chain data from the database. + database.delete_l2_blocks_gt(l2_block.number).await?; + + // Update the in-memory chain with the new blocks. + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); + new_chain.extend(new_chain_headers); + *chain.lock().unwrap() = new_chain; + + Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( + new_blocks, peer_id, signature, + ))) + } + None => Err(IndexerError::L2SafeBlockReorgDetected), + } + } + + /// Inserts an L2 block in the database. + pub fn consolidate_l2_blocks( &mut self, - block_info: L2BlockInfoWithL1Messages, + block_info: Vec, batch_info: Option, ) { let database = self.database.clone(); + let l1_synced = self.l1_synced; + let optimistic_mode = self.optimistic_mode.clone(); + let chain = self.chain.clone(); let fut = self.handle_metered( - IndexerItem::L2Block, + IndexerItem::InsertL2Block, Box::pin(async move { - database.insert_block(block_info.clone(), batch_info).await?; - Result::<_, IndexerError>::Ok(IndexerEvent::BlockIndexed(block_info, batch_info)) + // If we are in optimistic mode and the L1 is synced, we consolidate the chain and + // disable optimistic mode. + if l1_synced && *optimistic_mode.lock().unwrap() { + consolidate_chain(database.clone(), chain).await?; + } + let head = block_info.last().expect("block info must not be empty").clone(); + database.insert_blocks(block_info, batch_info).await?; + *optimistic_mode.lock().unwrap() = false; + Result::<_, IndexerError>::Ok(ChainOrchestratorEvent::L2BlockCommitted( + head, batch_info, + )) }), ); @@ -170,12 +486,12 @@ impl Indexer< database: Arc, chain_spec: Arc, l1_block_number: u64, - ) -> Result { + ) -> Result { let txn = database.tx().await?; let UnwindResult { l1_block_number, queue_index, l2_head_block_info, l2_safe_block_info } = txn.unwind(chain_spec.genesis_hash(), l1_block_number).await?; txn.commit().await?; - Ok(IndexerEvent::UnwindIndexed { + Ok(ChainOrchestratorEvent::ChainUnwound { l1_block_number, queue_index, l2_head_block_info, @@ -190,7 +506,7 @@ impl Indexer< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result { // Set the latest finalized L1 block in the database. database.set_latest_finalized_l1_block_number(block_number).await?; @@ -207,7 +523,7 @@ impl Indexer< // update the indexer l1 block number. l1_block_number.store(block_number, Ordering::Relaxed); - Ok(IndexerEvent::FinalizedIndexed(block_number, finalized_block)) + Ok(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_block)) } /// Handles an L1 message by inserting it into the database. @@ -217,8 +533,8 @@ impl Indexer< l1_message: TxL1Message, l1_block_number: u64, block_timestamp: u64, - ) -> Result { - let event = IndexerEvent::L1MessageIndexed(l1_message.queue_index); + ) -> Result { + let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); let queue_hash = if chain_spec .scroll_fork_activation(ScrollHardfork::EuclidV2) @@ -248,8 +564,8 @@ impl Indexer< async fn handle_batch_commit( database: Arc, batch: BatchCommitData, - ) -> Result { - let event = IndexerEvent::BatchCommitIndexed(BatchInfo::new(batch.index, batch.hash)); + ) -> Result { + let event = ChainOrchestratorEvent::BatchCommitted(BatchInfo::new(batch.index, batch.hash)); database.insert_batch(batch).await?; Ok(event) } @@ -261,7 +577,7 @@ impl Indexer< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result { // finalized the batch. database.finalize_batch(batch_hash, block_number).await?; @@ -274,7 +590,7 @@ impl Indexer< Self::fetch_highest_finalized_block(database, batch_hash, l2_block_number).await?; } - let event = IndexerEvent::BatchFinalizationIndexed(batch_hash, finalized_block); + let event = ChainOrchestratorEvent::BatchFinalized(batch_hash, finalized_block); Ok(event) } @@ -301,12 +617,37 @@ impl Indexer< } } +async fn init_chain_from_db + 'static>( + database: &Arc, + l2_client: &P, +) -> 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 +} + /// Unwinds the indexer by deleting all indexed data greater than the provided L1 block number. pub async fn unwind( database: Arc, chain_spec: Arc, l1_block_number: u64, -) -> Result { +) -> Result { // create a database transaction so this operation is atomic let txn = database.tx().await?; @@ -350,7 +691,7 @@ pub async fn unwind Stream for Indexer { - type Item = Result; +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 if let Some(mut action) = self.pending_futures.pop_front() { return match action.poll(cx) { @@ -377,23 +726,266 @@ impl Stream for Indexer { } } +async fn consolidate_chain( + _database: Arc, + _chain: Arc>, +) -> Result<(), IndexerError> { + // TODO: implement the logic to consolidate the chain. + // If we are in optimistic mode, we consolidate the chain and disable optimistic + // mode. + // let mut chain = chain.lock().unwrap(); + // if !chain.is_empty() { + // database.insert_chain(chain.drain(..).collect()).await?; + // } + Ok(()) +} + +async fn fetch_blocks + 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(); + + // TODO: can we assume the bodies are in the same order as the headers? + for (header, body) in headers.into_iter().zip(bodies) { + blocks.push(ScrollBlock::new(header, body)); + } + + blocks +} + +async fn validate_l1_messages( + _blocks: &[ScrollBlock], + _database: Arc, +) -> Result<(), IndexerError> { + // TODO: implement L1 message validation logic. + Ok(()) +} + #[cfg(test)] mod test { use std::vec; use super::*; - use alloy_primitives::{address, bytes, U256}; - + 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; + + /// 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, + } + } + } - async fn setup_test_indexer() -> (Indexer, Arc) { + // impl TestScrollFullBlockClient { + // /// Insert a header and body into the client maps. + // fn insert(&self, header: SealedHeader, body: ScrollBody) { + // let hash = header.hash(); + // self.headers.lock().insert(hash, header.unseal()); + // self.bodies.lock().insert(hash, body); + // } + + // /// Set the soft response limit. + // const fn set_soft_limit(&mut self, limit: usize) { + // self.soft_limit = limit; + // } + + // /// Get the block with the highest block number. + // fn highest_block(&self) -> Option> { + // self.headers.lock().iter().max_by_key(|(_, header)| header.number).and_then( + // |(hash, header)| { + // self.bodies.lock().get(hash).map(|body| { + // SealedBlock::from_parts_unchecked(header.clone(), body.clone(), *hash) + // }) + // }, + // ) + // } + // } + + 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_indexer() -> ( + 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); - (Indexer::new(db.clone(), SCROLL_MAINNET.clone()), db) + ( + ChainOrchestrator::new( + db.clone(), + SCROLL_MAINNET.clone(), + TestScrollFullBlockClient::default(), + provider, + ) + .await, + db, + ) } #[tokio::test] @@ -561,7 +1153,7 @@ mod test { let batch_commits = db.get_batches().await.unwrap().map(|res| res.unwrap()).collect::>().await; - assert_eq!(2, batch_commits.len()); + assert_eq!(3, batch_commits.len()); assert!(batch_commits.contains(&batch_commit_block_1)); assert!(batch_commits.contains(&batch_commit_block_20)); @@ -641,7 +1233,7 @@ mod test { } else { None }; - indexer.handle_block(l2_block.clone(), batch_info); + indexer.consolidate_l2_blocks(vec![l2_block.clone()], batch_info); indexer.next().await.unwrap().unwrap(); blocks.push(l2_block); } @@ -652,7 +1244,7 @@ mod test { let event = indexer.next().await.unwrap().unwrap(); assert_eq!( event, - IndexerEvent::UnwindIndexed { + ChainOrchestratorEvent::ChainUnwound { l1_block_number: 17, queue_index: None, l2_head_block_info: None, @@ -667,7 +1259,7 @@ mod test { assert_eq!( event, - IndexerEvent::UnwindIndexed { + ChainOrchestratorEvent::ChainUnwound { l1_block_number: 7, queue_index: Some(8), l2_head_block_info: Some(blocks[7].block_info), @@ -681,7 +1273,7 @@ mod test { assert_eq!( event, - IndexerEvent::UnwindIndexed { + ChainOrchestratorEvent::ChainUnwound { l1_block_number: 3, queue_index: Some(4), l2_head_block_info: Some(blocks[3].block_info), diff --git a/crates/indexer/src/metrics.rs b/crates/indexer/src/metrics.rs index dc25ee0c..f152c7cd 100644 --- a/crates/indexer/src/metrics.rs +++ b/crates/indexer/src/metrics.rs @@ -5,8 +5,10 @@ use strum::EnumIter; /// An enum representing the items the indexer can handle. #[derive(Debug, PartialEq, Eq, Hash, EnumIter)] pub enum IndexerItem { + /// Handle a block received from the network. + NewBlock, /// L2 block. - L2Block, + InsertL2Block, /// L1 reorg. L1Reorg, /// L1 finalization. @@ -23,7 +25,8 @@ impl IndexerItem { /// Returns the str representation of the [`IndexerItem`]. pub const fn as_str(&self) -> &'static str { match self { - Self::L2Block => "l2_block", + Self::NewBlock => "new_block", + Self::InsertL2Block => "l2_block", Self::L1Reorg => "l1_reorg", Self::L1Finalization => "l1_finalization", Self::L1Message => "l1_message", diff --git a/crates/indexer/testdata/genesis_block.json b/crates/indexer/testdata/genesis_block.json new file mode 100644 index 00000000..3387b87c --- /dev/null +++ b/crates/indexer/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/indexer/testdata/genesis_block_rpc.json b/crates/indexer/testdata/genesis_block_rpc.json new file mode 100644 index 00000000..b828bd22 --- /dev/null +++ b/crates/indexer/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/manager/src/manager/event.rs b/crates/manager/src/manager/event.rs index 0fb82c76..ea708e2d 100644 --- a/crates/manager/src/manager/event.rs +++ b/crates/manager/src/manager/event.rs @@ -1,4 +1,6 @@ use reth_scroll_primitives::ScrollBlock; +use rollup_node_indexer::ChainOrchestratorEvent; +use rollup_node_primitives::ChainImport; use rollup_node_signer::SignerEvent; use scroll_engine::ConsolidationOutcome; use scroll_network::NewBlockWithPeer; @@ -18,4 +20,10 @@ pub enum RollupManagerEvent { L1MessageIndexed(u64), /// A new event from the signer. SignerEvent(SignerEvent), + /// An event from the chain orchestrator. + ChainOrchestratorEvent(ChainOrchestratorEvent), + /// An optimistic sync has been triggered by the chain orchestrator. + OptimisticSyncTriggered(ScrollBlock), + /// A chain extension has been triggered by the chain orchestrator. + ChainExtensionTriggered(ChainImport), } diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index e9051cef..103e457c 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -3,16 +3,16 @@ //! 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_indexer::{ChainOrchestrator, ChainOrchestratorEvent}; +use rollup_node_primitives::BlockInfo; use rollup_node_sequencer::Sequencer; use rollup_node_signer::{SignerEvent, SignerHandle}; use rollup_node_watcher::L1Notification; @@ -53,9 +53,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 +79,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: Option>, /// 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. @@ -128,7 +123,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) @@ -150,21 +145,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: Option, 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 = l1_provider.map(|provider| DerivationPipeline::new(provider, database)); let rnm = Self { @@ -174,9 +168,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, @@ -220,7 +213,7 @@ where result: Err(err.into()), }); } else { - self.engine.handle_block_import(block_with_peer); + self.chain.handle_block_from_peer(block_with_peer); } } @@ -234,27 +227,27 @@ where } /// Handles an indexer event. - fn handle_indexer_event(&mut self, event: IndexerEvent) { + fn handle_indexer_event(&mut self, event: ChainOrchestratorEvent) { trace!(target: "scroll::node::manager", ?event, "Received indexer event"); match event { - IndexerEvent::BatchCommitIndexed(batch_info) => { + ChainOrchestratorEvent::BatchCommitted(batch_info) => { // push the batch info into the derivation pipeline. if let Some(pipeline) = &mut self.derivation_pipeline { pipeline.handle_batch_commit(batch_info); } } - 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, Some(finalized_block)) => { + ChainOrchestratorEvent::L1BlockFinalized(l1_block_number, Some(finalized_block)) => { if let Some(sequencer) = self.sequencer.as_mut() { sequencer.set_l1_finalized_block_number(l1_block_number); } // update the fcs on new finalized block. self.engine.set_finalized_block_info(finalized_block); } - IndexerEvent::UnwindIndexed { + ChainOrchestratorEvent::ChainUnwound { l1_block_number, queue_index, l2_head_block_info, @@ -277,12 +270,52 @@ where // TODO: should clear the derivation pipeline. } - IndexerEvent::L1MessageIndexed(index) => { + ChainOrchestratorEvent::L1MessageCommitted(index) => { if let Some(event_sender) = self.event_sender.as_ref() { event_sender.notify(RollupManagerEvent::L1MessageIndexed(index)); } } - _ => (), + ChainOrchestratorEvent::OldForkReceived { ref headers, ref peer_id, signature: _ } => { + trace!(target: "scroll::node::manager", ?headers, ?peer_id, "Received old fork from peer"); + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::ChainOrchestratorEvent(event)); + } + } + 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"); + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender + .notify(RollupManagerEvent::ChainExtensionTriggered(chain_import.clone())); + } + + // 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"); + // if let Some(event_sender) = self.event_sender.as_ref() { + // event_sender.notify(RollupManagerEvent::ChainOrchestratorEvent(event)); + // } + + // 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"); + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::OptimisticSyncTriggered(block)); + } + + // Issue the new block info to the engine driver for processing. + self.engine.handle_optimistic_sync(block_info) + } + event => { + 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)); + } + } } } @@ -295,7 +328,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_l2_blocks(vec![(&block).into()], None); } self.network.handle().block_import_outcome(outcome); } @@ -308,11 +341,11 @@ where event_sender.notify(RollupManagerEvent::BlockSequenced(payload.clone())); } - self.indexer.handle_block((&payload).into(), None); + self.chain.consolidate_l2_blocks(vec![(&payload).into()], None); } EngineDriverEvent::L1BlockConsolidated(consolidation_outcome) => { - self.indexer.handle_block( - consolidation_outcome.block_info().clone(), + self.chain.consolidate_l2_blocks( + vec![consolidation_outcome.block_info().clone()], Some(*consolidation_outcome.batch_info()), ); @@ -322,50 +355,27 @@ 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_l2_blocks( + outcome.chain.iter().map(|b| b.into()).collect(), + None, + ); + } + 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 L1Notification::Consensus(ref update) = notification { self.consensus.update_config(update); } - self.indexer.handle_l1_notification(notification) + self.chain.handle_l1_notification(notification) } /// Returns the current status of the [`RollupNodeManager`]. @@ -458,7 +468,7 @@ 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), Err(err) => { @@ -514,13 +524,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 5ea6f040..4b9d7fdc 100644 --- a/crates/network/Cargo.toml +++ b/crates/network/Cargo.toml @@ -11,6 +11,7 @@ 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 @@ -18,11 +19,13 @@ reth-network-types = { git = "https://github.com/scroll-tech/reth.git", default- 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-tokio-util.workspace = true # scroll reth-scroll-chainspec.workspace = true reth-scroll-node = { workspace = true, features = ["skip-state-root-validation"] } reth-scroll-primitives.workspace = true +scroll-alloy-hardforks.workspace = true scroll-wire.workspace = true # misc diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index 7c10ceb5..badbdead 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -4,27 +4,35 @@ use super::{ BlockImportOutcome, BlockValidation, NetworkHandleMessage, NetworkManagerEvent, NewBlockWithPeer, ScrollNetworkHandle, }; -use alloy_primitives::FixedBytes; +use alloy_primitives::{FixedBytes, Signature}; use futures::{FutureExt, Stream, StreamExt}; +use reth_chainspec::EthChainSpec; 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 @@ -34,22 +42,30 @@ 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, } -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>>, ) -> Self { // Create the scroll-wire protocol handler. let (scroll_wire_handler, events) = ScrollWireProtocolHandler::new(scroll_wire_config); @@ -73,16 +89,29 @@ 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, + } } } -impl ScrollNetworkManager { +impl + 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>>, + ) -> 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(); @@ -91,7 +120,13 @@ 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, + } } /// Returns a new [`ScrollNetworkHandle`] instance. @@ -182,9 +217,45 @@ 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 { + 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. + 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"); + 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 Stream + for ScrollNetworkManager +{ type Item = NetworkManagerEvent; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { @@ -211,6 +282,15 @@ impl Stream for ScrollNetworkManager { 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/src/args.rs b/crates/node/src/args.rs index dc6d6590..df48f98d 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -17,6 +17,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_indexer::ChainOrchestrator; use rollup_node_manager::{ Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, }; @@ -111,12 +112,6 @@ impl ScrollRollupNodeConfig { RollupManagerHandle, Option>>, )> { - // Instantiate the network manager - let (scroll_wire_handler, events) = - ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); - network.add_rlpx_sub_protocol(scroll_wire_handler.into_rlpx_sub_protocol()); - 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"); let node_config = Arc::new(NodeConfig::from_named_chain(named_chain)); @@ -142,8 +137,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 { @@ -175,6 +170,21 @@ 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_wire_handler, events) = + ScrollWireProtocolHandler::new(ScrollWireConfig::new(true)); + network.add_rlpx_sub_protocol(scroll_wire_handler.into_rlpx_sub_protocol()); + let scroll_network_manager = ScrollNetworkManager::from_parts( + chain_spec.clone(), + network.clone(), + events, + eth_wire_listener, + ); + // 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. @@ -192,10 +202,9 @@ impl ScrollRollupNodeConfig { let engine = EngineDriver::new( Arc::new(engine_api), chain_spec.clone(), - Some(l2_provider), + Some(l2_provider.clone()), fcs, !self.test && !chain_spec.is_dev_chain(), - self.engine_driver_args.en_sync_trigger, Duration::from_millis(self.sequencer_args.payload_building_duration), ); @@ -258,13 +267,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 // Instantiate the signer let signer = if self.test { // Use a random private key signer for testing @@ -275,6 +277,16 @@ impl ScrollRollupNodeConfig { self.signer_args.signer(chain_id).await?.map(rollup_node_signer::Signer::spawn) }; + // Insantiate 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).await; + // Spawn the rollup node manager let (rnm, handle) = RollupNodeManager::new( scroll_network_manager, @@ -284,11 +296,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)) } } diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 2cc4a963..0bc7c2cb 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -149,6 +149,13 @@ async fn can_sequence_and_gossip_blocks() { panic!("Failed to receive block from rollup node"); } + // assert that a chain extension is triggered on the follower node + if let Some(RollupManagerEvent::ChainExtensionTriggered(_)) = follower_events.next().await { + () + } else { + panic!("Failed to receive chain extension event from rollup node"); + } + // assert that the block was successfully imported by the follower node if let Some(RollupManagerEvent::BlockImported(block)) = follower_events.next().await { assert_eq!(block.body.transactions.len(), 1); @@ -189,8 +196,13 @@ 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, + ) + .await; let scroll_network_handle = scroll_network.handle(); // Connect the scroll-wire node to the scroll NetworkManager. diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 8b315bd7..f43e0a0f 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -11,8 +11,7 @@ use rollup_node::{ }, ScrollRollupNode, }; -use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent}; -use tokio::sync::oneshot; +use rollup_node_manager::RollupManagerEvent; /// We test if the syncing of the RN is correctly triggered and released when the EN reaches sync. #[allow(clippy::large_stack_frames)] @@ -42,19 +41,9 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { 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)) + // Assert that the unsynced node triggers optimistic sync. + wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::OptimisticSyncTriggered(_)), 1) .await; - let status = rx.await.unwrap(); - assert!(status.syncing); // Verify the unsynced node syncs. let provider = ProviderBuilder::new().connect_http(unsynced.rpc_url()); @@ -70,19 +59,9 @@ 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; - - // 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)) + // Assert that the unsynced node triggers a chain extension on the optimistic chain. + wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::ChainExtensionTriggered(_)), 1) .await; - let status = rx.await.unwrap(); - assert!(!status.syncing); } /// Waits for n events to be emitted. diff --git a/crates/primitives/Cargo.toml b/crates/primitives/Cargo.toml index 7dbbc3ca..81f6a71c 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 diff --git a/crates/primitives/src/block.rs b/crates/primitives/src/block.rs index e698a314..f870f105 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; @@ -49,6 +50,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..33703214 100644 --- a/crates/primitives/src/bounded_vec.rs +++ b/crates/primitives/src/bounded_vec.rs @@ -39,6 +39,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() diff --git a/crates/primitives/src/chain.rs b/crates/primitives/src/chain.rs new file mode 100644 index 00000000..e8349e23 --- /dev/null +++ b/crates/primitives/src/chain.rs @@ -0,0 +1,23 @@ +use alloy_primitives::Signature; +use reth_network_peers::PeerId; +use reth_scroll_primitives::ScrollBlock; + +/// 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 0b3407a7..6c958348 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..6c12cbc4 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 [`NewBlocks`] 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 e79d29db..568e22ad 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -64,7 +64,6 @@ async fn can_build_blocks() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -158,7 +157,6 @@ async fn can_build_blocks_with_delayed_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 L1_MESSAGE_DELAY: u64 = 2; // setup a test node @@ -184,7 +182,6 @@ async fn can_build_blocks_with_delayed_l1_messages() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -309,7 +306,6 @@ async fn can_build_blocks_with_finalized_l1_messages() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -635,7 +631,6 @@ async fn can_build_blocks_and_exit_at_gas_limit() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); @@ -720,7 +715,6 @@ async fn can_build_blocks_and_exit_at_time_limit() { None::, fcs, false, - BLOCK_GAP_TRIGGER, BLOCK_BUILDING_DURATION, ); From 4032fa5b768a6271df65dfcedda6be3b5da9e577 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 2 Jul 2025 20:53:19 +0100 Subject: [PATCH 02/30] lint --- .codespellrc | 2 +- Cargo.lock | 1 + crates/network/Cargo.toml | 1 + crates/node/Cargo.toml | 1 + crates/node/src/args.rs | 2 +- crates/node/tests/e2e.rs | 14 +++++++++++++- crates/primitives/Cargo.toml | 1 + crates/sequencer/tests/e2e.rs | 8 ++++---- 8 files changed, 23 insertions(+), 7 deletions(-) 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/Cargo.lock b/Cargo.lock index 26827371..033e85a4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10071,6 +10071,7 @@ dependencies = [ "reth-transaction-pool", "reth-trie-db", "rollup-node", + "rollup-node-indexer", "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", diff --git a/crates/network/Cargo.toml b/crates/network/Cargo.toml index d4db4fb5..adc549fe 100644 --- a/crates/network/Cargo.toml +++ b/crates/network/Cargo.toml @@ -48,4 +48,5 @@ serde = [ "scroll-wire/serde", "reth-primitives-traits/serde", "reth-storage-api/serde", + "scroll-alloy-hardforks/serde", ] diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index 65271709..ef3d6fe3 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-indexer.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 df48f98d..9ae81ae9 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -277,7 +277,7 @@ impl ScrollRollupNodeConfig { self.signer_args.signer(chain_id).await?.map(rollup_node_signer::Signer::spawn) }; - // Insantiate the chain orchestrator + // Instantiate the chain orchestrator let block_client = scroll_network_manager .handle() .inner() diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 0bc7c2cb..9f40dba9 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -91,6 +91,19 @@ async fn can_bridge_l1_messages() -> eyre::Result<()> { Ok(()) } +#[tokio::test] +async fn follower_can_reorg() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + + // create 2 nodes + let chain_spec = (*SCROLL_DEV).clone(); + let (mut _nodes, _tasks, _) = + setup_engine(default_test_scroll_rollup_node_config(), 2, chain_spec.clone(), false) + .await?; + + Ok(()) +} + #[tokio::test] async fn can_sequence_and_gossip_blocks() { reth_tracing::init_test_tracing(); @@ -151,7 +164,6 @@ async fn can_sequence_and_gossip_blocks() { // assert that a chain extension is triggered on the follower node if let Some(RollupManagerEvent::ChainExtensionTriggered(_)) = follower_events.next().await { - () } else { panic!("Failed to receive chain extension event from rollup node"); } diff --git a/crates/primitives/Cargo.toml b/crates/primitives/Cargo.toml index 81f6a71c..d1af8a3d 100644 --- a/crates/primitives/Cargo.toml +++ b/crates/primitives/Cargo.toml @@ -43,6 +43,7 @@ std = [ "reth-primitives-traits/std", "alloy-consensus/std", "alloy-chains/std", + "reth-network-peers/std", ] arbitrary = [ "std", diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 8945f815..bc1eba94 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -40,7 +40,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(); @@ -281,7 +281,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) = @@ -581,7 +581,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 @@ -666,7 +666,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 From 39802e23f8307293f9c17abd987961f28554e5e7 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 2 Jul 2025 22:50:31 +0100 Subject: [PATCH 03/30] lint --- crates/indexer/src/metrics.rs | 2 +- crates/manager/src/manager/mod.rs | 4 ++-- crates/primitives/src/chain.rs | 1 + 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/crates/indexer/src/metrics.rs b/crates/indexer/src/metrics.rs index f152c7cd..0defdfeb 100644 --- a/crates/indexer/src/metrics.rs +++ b/crates/indexer/src/metrics.rs @@ -36,7 +36,7 @@ impl IndexerItem { } } -/// The metrics for the [`super::Indexer`]. +/// The metrics for the [`super::ChainOrchestrator`]. #[derive(Metrics, Clone)] #[metrics(scope = "indexer")] pub struct IndexerMetrics { diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 103e457c..fe17092e 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -1,6 +1,6 @@ //! 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_provider::Provider; diff --git a/crates/primitives/src/chain.rs b/crates/primitives/src/chain.rs index e8349e23..2568960e 100644 --- a/crates/primitives/src/chain.rs +++ b/crates/primitives/src/chain.rs @@ -1,6 +1,7 @@ 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 From c84dc0dca64e555167485ca46aa2e11529e8f5a0 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 2 Jul 2025 22:55:41 +0100 Subject: [PATCH 04/30] docs lint --- crates/scroll-wire/src/protocol/proto.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/scroll-wire/src/protocol/proto.rs b/crates/scroll-wire/src/protocol/proto.rs index 6c12cbc4..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 [`NewBlocks`] instance with the provided signature and blocks. + /// 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 } } From ea3140c3046123aac21f3b00e494eb2d2c605732 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 3 Jul 2025 16:05:10 +0100 Subject: [PATCH 05/30] fix: clone the in-memory chain instead of taking it --- crates/indexer/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index a8d13096..6c12221a 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -174,7 +174,7 @@ impl< block_with_peer: NewBlockWithPeer, ) -> Result { let NewBlockWithPeer { block: received_block, peer_id, signature } = block_with_peer; - let mut current_chain_headers = std::mem::take(&mut *chain.lock().unwrap()); + let mut current_chain_headers = chain.lock().unwrap().clone(); let max_block_number = current_chain_headers.last().expect("chain can not be empty").number; let min_block_number = current_chain_headers.first().expect("chain can not be empty").number; From fa6952ca4934196106b64e63d5c3dff2f139c51f Mon Sep 17 00:00:00 2001 From: frisitano Date: Fri, 4 Jul 2025 15:49:05 +0100 Subject: [PATCH 06/30] update chain consolidation and l1 message validation --- crates/database/db/src/db.rs | 9 +- crates/database/db/src/lib.rs | 2 +- crates/database/db/src/operations.rs | 32 +++++- crates/indexer/Cargo.toml | 2 +- crates/indexer/src/error.rs | 12 +++ crates/indexer/src/lib.rs | 145 +++++++++++++++++++++++---- crates/primitives/src/bounded_vec.rs | 10 ++ crates/sequencer/tests/e2e.rs | 6 ++ 8 files changed, 191 insertions(+), 27 deletions(-) diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index 560823f1..4e7179da 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -315,8 +315,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)); 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/operations.rs b/crates/database/db/src/operations.rs index f8b1076b..c5d616dc 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -227,12 +227,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(0))?; + + 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)?))) @@ -508,6 +526,18 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { } } +/// 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)] +pub enum L1MessageStart { + /// Start from the provided queue index. + Index(u64), + /// Start from the provided queue hash. + Hash(B256), +} + /// The result of [`DatabaseOperations::unwind`]. #[derive(Debug)] pub struct UnwindResult { diff --git a/crates/indexer/Cargo.toml b/crates/indexer/Cargo.toml index 01fccd81..93c62e07 100644 --- a/crates/indexer/Cargo.toml +++ b/crates/indexer/Cargo.toml @@ -32,6 +32,7 @@ scroll-network.workspace = true 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 @@ -58,7 +59,6 @@ reth-scroll-forks.workspace = true # reth reth-eth-wire-types.workspace = true -reth-primitives-traits.workspace = true reth-network-peers.workspace = true # misc diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs index 83a15b3c..0ddf8828 100644 --- a/crates/indexer/src/error.rs +++ b/crates/indexer/src/error.rs @@ -1,3 +1,4 @@ +use alloy_primitives::B256; use scroll_db::DatabaseError; /// A type that represents an error that occurred during indexing. @@ -12,4 +13,15 @@ pub enum IndexerError { /// 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 inconsistency was detected when trying to consolidate the chain. + #[error("Chain inconsistency detected")] + ChainInconsistency, } diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index 6c12221a..adcc0219 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -1,7 +1,7 @@ //! A library responsible for indexing data relevant to the L1. use alloy_consensus::Header; -use alloy_eips::BlockHashOrNumber; +use alloy_eips::{BlockHashOrNumber, Encodable2718}; use alloy_primitives::{b256, keccak256, B256}; use alloy_provider::Provider; use futures::{task::AtomicWaker, Stream, StreamExt, TryStreamExt}; @@ -16,7 +16,7 @@ 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, UnwindResult}; +use scroll_db::{Database, DatabaseError, DatabaseOperations, L1MessageStart, UnwindResult}; use scroll_network::NewBlockWithPeer; use std::{ collections::{HashMap, VecDeque}, @@ -46,7 +46,12 @@ const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); /// The number of block headers we keep in memory. -const CHAIN_BUFFER_SIZE: usize = 2000; +const OPTIMISTIC_CHAIN_BUFFER_SIZE: usize = 2000; + +/// The maximum number of blocks we keep in memory for the consolidated chain (This is the full +/// unsafe chain). Memory requirements = ~800 bytes per header * `600_000` headers = 480 MB. +// TODO: This is just a temporary hack we will transition to a VecDeque when not in synced mode. +const CONSOLIDATED_CHAIN_BUFFER_SIZE: usize = 300_000; /// The threshold for optimistic syncing. If the received block is more than this many blocks /// ahead of the current chain, we optimistically sync the chain. @@ -183,6 +188,8 @@ impl< *guard }; + // TODO: remove database lookups. + // 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. @@ -190,7 +197,7 @@ impl< // fetch the latest `OPTIMISTIC_CHAIN_BUFFER_SIZE` blocks from the network for the // optimistic chain. let mut optimistic_headers = vec![received_block.header.clone()]; - while optimistic_headers.len() < CHAIN_BUFFER_SIZE && + while optimistic_headers.len() < OPTIMISTIC_CHAIN_BUFFER_SIZE && optimistic_headers.last().unwrap().number != 0 { tracing::trace!(target: "scroll::watcher", number = ?(optimistic_headers.last().unwrap().number - 1), "fetching block"); @@ -205,7 +212,7 @@ impl< optimistic_headers.push(header); } optimistic_headers.reverse(); - let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); new_chain.extend(optimistic_headers); *chain.lock().unwrap() = new_chain; *optimistic_mode.lock().unwrap() = true; @@ -288,7 +295,7 @@ impl< // If we are in optimistic mode, we terminate the search as we don't have the // necessary data in the database. This is fine because very deep // re-orgs are rare and in any case will be resolved once optimistic sync is - // completed.If the current header block number is less than the + // completed. If the current header block number is less than the // latest safe block number then this would suggest a reorg of a // safe block which is not invalid - terminate the search. } else if optimistic_mode_local || @@ -364,7 +371,7 @@ impl< // point and extend it with the new blocks. Some(ReorgIndex::Memory(position)) => { // reorg the in-memory chain to the new chain and issue a reorg event. - let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); new_chain.extend(current_chain_headers.iter().take(position).cloned()); new_chain.extend(new_chain_headers); *chain.lock().unwrap() = new_chain; @@ -380,7 +387,7 @@ impl< database.delete_l2_blocks_gt(l2_block.number).await?; // Update the in-memory chain with the new blocks. - let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); new_chain.extend(new_chain_headers); *chain.lock().unwrap() = new_chain; @@ -402,17 +409,24 @@ impl< 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 fut = self.handle_metered( IndexerItem::InsertL2Block, Box::pin(async move { // If we are in optimistic mode and the L1 is synced, we consolidate the chain and // disable optimistic mode. if l1_synced && *optimistic_mode.lock().unwrap() { - consolidate_chain(database.clone(), chain).await?; + consolidate_chain(database.clone(), block_info.clone(), chain, l2_client) + .await?; + *optimistic_mode.lock().unwrap() = false; } + + // If we are consolidating a batch, we insert the batch info into the database. let head = block_info.last().expect("block info must not be empty").clone(); - database.insert_blocks(block_info, batch_info).await?; - *optimistic_mode.lock().unwrap() = false; + if batch_info.is_some() { + database.insert_blocks(block_info, batch_info).await?; + } + Result::<_, IndexerError>::Ok(ChainOrchestratorEvent::L2BlockCommitted( head, batch_info, )) @@ -622,8 +636,9 @@ async fn init_chain_from_db + 'static>( l2_client: &P, ) -> 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); + let mut blocks = Vec::with_capacity(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut blocks_stream = + database.get_l2_blocks().await.unwrap().take(OPTIMISTIC_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) @@ -637,7 +652,7 @@ async fn init_chain_from_db + 'static>( blocks.reverse(); blocks }; - let mut chain: Chain = Chain::new(CHAIN_BUFFER_SIZE); + let mut chain: Chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); chain.extend(blocks); chain } @@ -726,10 +741,62 @@ impl< } } -async fn consolidate_chain( - _database: Arc, +/// 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>( + database: Arc, + _block_info: Vec, _chain: Arc>, + l2_client: P, ) -> Result<(), IndexerError> { + // take the current chain. + let chain = std::mem::take(&mut *_chain.lock().unwrap()); + + // Find highest common ancestor by comparing hashes + let hca_index = chain.iter().rposition(|h| { + let h_hash = h.hash_slow(); + _block_info.iter().any(|b| b.block_info.hash == h_hash) + }); + + // This means there is a deep reorg that has just occurred however in practice this should + // never happen due to purging of expired chain. + // TODO: consider this case more carefully. + if hca_index.is_none() { + // If we do not have a common ancestor, we return an error. + *_chain.lock().unwrap() = chain; + return Err(IndexerError::ChainInconsistency); + } + + // Now reconcile back to the safe head. + let safe_head = database.get_latest_l2_block().await?.expect("safe head must exist"); + let starting_block = + l2_client.get_block_by_hash(chain.first().unwrap().hash_slow()).await.unwrap().unwrap(); + let mut consolidated_chain_blocks = + vec![starting_block.into_consensus().map_transactions(|tx| tx.inner.into_inner())]; + while consolidated_chain_blocks.last().unwrap().header.parent_hash != safe_head.hash { + // Fetch the missing blocks from the L2 client. + let block = + l2_client.get_block_by_hash(chain.last().unwrap().parent_hash).await.unwrap().unwrap(); + consolidated_chain_blocks + .push(block.into_consensus().map_transactions(|tx| tx.inner.into_inner())); + + if chain.last().unwrap().number < safe_head.number { + // If we did not consolidate back to the safe head, we return an error. + *_chain.lock().unwrap() = chain; + // TODO: should we revert to the last known safe head. + return Err(IndexerError::ChainInconsistency); + } + } + + consolidated_chain_blocks.reverse(); + validate_l1_messages(&consolidated_chain_blocks, database.clone()).await?; + + let mut consolidated_chain = BoundedVec::new(CONSOLIDATED_CHAIN_BUFFER_SIZE); + consolidated_chain.extend(consolidated_chain_blocks.iter().map(|b| b.header.clone())); + consolidated_chain.extend(chain.into_inner()); + + // let unsafe_chain = + // TODO: implement the logic to consolidate the chain. // If we are in optimistic mode, we consolidate the chain and disable optimistic // mode. @@ -752,7 +819,6 @@ async fn fetch_blocks + Send + Sync + 'stat .expect("Failed to fetch block bodies") .into_data(); - // TODO: can we assume the bodies are in the same order as the headers? for (header, body) in headers.into_iter().zip(bodies) { blocks.push(ScrollBlock::new(header, body)); } @@ -760,11 +826,41 @@ async fn fetch_blocks + Send + Sync + 'stat 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, + blocks: &[ScrollBlock], + database: Arc, ) -> Result<(), IndexerError> { - // TODO: implement L1 message validation logic. + 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::>(); + 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(IndexerError::L1MessageMismatch { + expected: expected_hash, + actual: message_hash, + }); + } + } Ok(()) } @@ -1158,8 +1254,13 @@ mod test { 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().await.unwrap().map(|res| res.unwrap()).collect::>().await; + 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)); diff --git a/crates/primitives/src/bounded_vec.rs b/crates/primitives/src/bounded_vec.rs index 33703214..eaf74526 100644 --- a/crates/primitives/src/bounded_vec.rs +++ b/crates/primitives/src/bounded_vec.rs @@ -61,6 +61,16 @@ 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 the inner `VecDeque` of the bounded vec. + pub fn into_inner(self) -> VecDeque { + self.data + } } impl Extend for BoundedVec { diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index bc1eba94..bda7b946 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -474,6 +474,9 @@ async fn can_sequence_blocks_with_private_key_file() -> eyre::Result<()> { panic!("Failed to receive BlockSequenced event"); } + // Skip the next event. + let _ = sequencer_events.next().await; + // Verify signing event and signature correctness if let Some(RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { block: signed_block, @@ -558,6 +561,9 @@ async fn can_sequence_blocks_with_hex_key_file_without_prefix() -> eyre::Result< panic!("Failed to receive BlockSequenced event"); } + // Skip the next event. + let _ = sequencer_events.next().await; + // Verify signing event and signature correctness if let Some(RollupManagerEvent::SignerEvent(SignerEvent::SignedBlock { block: signed_block, From 43cc2be6a246a591cd3f95d049519c869896bd6a Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 7 Jul 2025 14:50:14 +0100 Subject: [PATCH 07/30] add missing network block error --- crates/indexer/src/error.rs | 6 ++++++ crates/indexer/src/lib.rs | 16 ++++++++++++---- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs index 0ddf8828..b1597516 100644 --- a/crates/indexer/src/error.rs +++ b/crates/indexer/src/error.rs @@ -24,4 +24,10 @@ pub enum IndexerError { /// 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, + }, } diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index adcc0219..dceefe83 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -329,14 +329,22 @@ impl< } tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); - let header = network_client + if let Some(header) = network_client .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) .await .unwrap() .into_data() - .unwrap(); - new_chain_headers.push(header.clone()); - new_header_tail = header; + { + // TODO: what do we do when peers don't have the blocks? We can't recreate the + // chain so we should terminate here. We should be able to reconcile this gap in + // a future block. + new_chain_headers.push(header.clone()); + new_header_tail = header; + } else { + return Err(IndexerError::MissingBlockHeader { + hash: new_header_tail.parent_hash, + }); + } } }; From 5dfc813e0cd7eed330f4f357dddcdbdf33cfce40 Mon Sep 17 00:00:00 2001 From: frisitano Date: Tue, 8 Jul 2025 11:43:17 +0100 Subject: [PATCH 08/30] update fork sync / reconcilliation to use network as opossed to database --- crates/indexer/src/lib.rs | 186 ++++++++++++++++++++------------------ 1 file changed, 99 insertions(+), 87 deletions(-) diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index dceefe83..b6a48d08 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -172,17 +172,17 @@ impl< /// Handles a new block received from the network. pub async fn handle_new_block( chain: Arc>, - l2_client: Arc

, + _l2_client: Arc

, optimistic_mode: Arc>, network_client: Arc, database: Arc, block_with_peer: NewBlockWithPeer, ) -> Result { let NewBlockWithPeer { block: received_block, peer_id, signature } = block_with_peer; - let mut current_chain_headers = chain.lock().unwrap().clone(); - let max_block_number = current_chain_headers.last().expect("chain can not be empty").number; + let mut current_chain_headers = chain.lock().unwrap().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.first().expect("chain can not be empty").number; + current_chain_headers.front().expect("chain can not be empty").number; let optimistic_mode_local: bool = { let guard = optimistic_mode.lock().unwrap(); *guard @@ -231,39 +231,17 @@ impl< )); } - // Perform preliminary checks on received block that are dependent on database state due to - // not being in in-memory chain. - if received_block.header.number <= min_block_number { - // If we are in optimistic mode, we return an event indicating that we have insufficient - // data to process the block. - if optimistic_mode_local { - return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( - received_block.header.hash_slow(), - )); - } - - // We check the database to see if this block is already known. - if database - .get_l2_block_and_batch_info_by_hash(received_block.header.hash_slow()) - .await? - .is_some() - { - tracing::debug!(target: "scroll::watcher", block_hash = ?received_block.header.hash_slow(), "block already in database"); - return Ok(ChainOrchestratorEvent::BlockAlreadyKnown( - received_block.header.hash_slow(), - peer_id, - )); - } + // If we are in optimistic mode, we return an event indicating that we have insufficient + // data to process the block as we are optimistically syncing the chain. + if optimistic_mode_local && (received_block.header.number <= min_block_number) { + return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( + received_block.header.hash_slow(), + )); }; let mut new_chain_headers = vec![received_block.header.clone()]; let mut new_header_tail = received_block.header.clone(); - enum ReorgIndex { - Memory(usize), - Database(BlockInfo), - } - // 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"); @@ -271,61 +249,98 @@ impl< // We search for the re-org index in the in-memory chain or the database. let reorg_index = { loop { - // If the current header block number is greater than the in-memory chain then we - // should search the in-memory chain. - if new_header_tail.number >= min_block_number { - if let Some(pos) = current_chain_headers - .iter() - .rposition(|h| h.hash_slow() == new_header_tail.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 >= - new_header_tail.timestamp + // If the new header tail has a block number that is less than the current header + // tail then we should fetch more blocks for the current header chain to aid + // reconciliation. + if new_header_tail.number <= + current_chain_headers.back().expect("chain can not be empty").number + { + for _ in 0..50 { + if new_header_tail.number.saturating_sub(1) < latest_safe_block.number { + tracing::info!(target: "scroll::chain", hash = %latest_safe_block.hash, number = %latest_safe_block.number, "reached safe block number - terminating fetching."); + break; + } + tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); + if let Some(header) = network_client + .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) + .await + .unwrap() + .into_data() { - return Ok(ChainOrchestratorEvent::OldForkReceived { - headers: new_chain_headers, - peer_id, - signature, + new_chain_headers.push(header.clone()); + new_header_tail = header; + } else { + return Err(IndexerError::MissingBlockHeader { + hash: new_header_tail.parent_hash, }); } - break Some(ReorgIndex::Memory(pos)); } - // If we are in optimistic mode, we terminate the search as we don't have the - // necessary data in the database. This is fine because very deep - // re-orgs are rare and in any case will be resolved once optimistic sync is - // completed. If the current header block number is less than the - // latest safe block number then this would suggest a reorg of a - // safe block which is not invalid - terminate the search. - } else if optimistic_mode_local || - new_header_tail.number <= latest_safe_block.number - { - break None + } - // If the block is not in the in-memory chain, we search the database. - } else if let Some((l2_block, _batch_info)) = database - .get_l2_block_and_batch_info_by_hash(new_header_tail.parent_hash) - .await? + // If the current header block number is greater than the in-memory chain then we + // should search the in-memory chain (we keep the latest + // `OPTIMISTIC_CHAIN_BUFFER_SIZE` headers in memory). + if let Some(pos) = current_chain_headers + .iter() + .rposition(|h| h.hash_slow() == new_header_tail.parent_hash) { - let diverged_block = database - .get_l2_block_info_by_number(l2_block.number + 1) - .await? - .expect("diverged block must exist"); - let diverged_block = - l2_client.get_block_by_hash(diverged_block.hash).await.unwrap().unwrap(); - // If the received fork is older than the current chain, we return an event // indicating that we have received an old fork. - if diverged_block.header.timestamp >= new_header_tail.timestamp { + if (pos < current_chain_headers.len() - 1) && + current_chain_headers.get(pos + 1).unwrap().timestamp >= + new_header_tail.timestamp + { return Ok(ChainOrchestratorEvent::OldForkReceived { headers: new_chain_headers, peer_id, signature, }); } + break Some(pos); + } + + // If we are in optimistic mode, we terminate the search as we don't have the + // necessary data from L1 consolidation yet. This is fine because very deep + // re-orgs are rare and in any case will be resolved once optimistic sync is + // completed. If the current header block number is less than the + // latest safe block number then this would suggest a reorg of a + // safe block which is not invalid - terminate the search. + if optimistic_mode_local && + (new_header_tail.number <= + current_chain_headers + .front() + .expect("chain must not be empty") + .number) + { + if received_block.timestamp > + current_chain_headers.back().expect("chain can not be empty").timestamp + { + tracing::debug!(target: "scroll::watcher", block_hash = ?received_block.header.hash_slow(), "received block is ahead of the current chain"); + while new_chain_headers.len() < OPTIMISTIC_CHAIN_BUFFER_SIZE && + new_chain_headers.last().unwrap().number != 0 + { + tracing::trace!(target: "scroll::watcher", number = ?(new_chain_headers.last().unwrap().number - 1), "fetching block"); + let header = network_client + .get_header(BlockHashOrNumber::Hash( + new_chain_headers.last().unwrap().parent_hash, + )) + .await + .unwrap() + .into_data() + .unwrap(); + new_chain_headers.push(header); + } + break None; + } + return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( + received_block.header.hash_slow(), + )); + } - break Some(ReorgIndex::Database(l2_block)) + // If the current header block number is less than the latest safe block number then + // we should error. + if new_header_tail.number <= latest_safe_block.number { + return Err(IndexerError::L2SafeBlockReorgDetected); } tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); @@ -366,10 +381,12 @@ impl< match reorg_index { // If this is a simple chain extension, we can just extend the in-memory chain and emit // a ChainExtended event. - Some(ReorgIndex::Memory(index)) if index == current_chain_headers.len() - 1 => { + Some(index) if index == current_chain_headers.len() - 1 => { // Update the chain with the new blocks. current_chain_headers.extend(new_blocks.iter().map(|b| b.header.clone())); - *chain.lock().unwrap() = current_chain_headers; + let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); + new_chain.extend(current_chain_headers); + *chain.lock().unwrap() = new_chain; Ok(ChainOrchestratorEvent::ChainExtended(ChainImport::new( new_blocks, peer_id, signature, @@ -377,7 +394,7 @@ impl< } // 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. - Some(ReorgIndex::Memory(position)) => { + Some(position) => { // reorg the in-memory chain to the new chain and issue a reorg event. let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); new_chain.extend(current_chain_headers.iter().take(position).cloned()); @@ -388,22 +405,16 @@ impl< new_blocks, peer_id, signature, ))) } - // If we have a deep reorg that impacts the database, we need to delete the blocks from - // the database, update the in-memory chain and emit a ChainReorged event. - Some(ReorgIndex::Database(l2_block)) => { - // remove old chain data from the database. - database.delete_l2_blocks_gt(l2_block.number).await?; - - // Update the in-memory chain with the new blocks. + None => { let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); new_chain.extend(new_chain_headers); *chain.lock().unwrap() = new_chain; + *optimistic_mode.lock().unwrap() = true; - Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( - new_blocks, peer_id, signature, - ))) + Ok(ChainOrchestratorEvent::OptimisticSync( + new_blocks.last().cloned().expect("new_blocks should not be empty"), + )) } - None => Err(IndexerError::L2SafeBlockReorgDetected), } } @@ -767,7 +778,8 @@ async fn consolidate_chain>( }); // This means there is a deep reorg that has just occurred however in practice this should - // never happen due to purging of expired chain. + // never happen due to purging of expired chain. This should be reconciled upon the next + // block import. // TODO: consider this case more carefully. if hca_index.is_none() { // If we do not have a common ancestor, we return an error. From 19458f720c7eb42c3cbe7799c605bf66d9330f52 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 10 Jul 2025 15:56:19 +0100 Subject: [PATCH 09/30] fix merege --- Cargo.lock | 4 ++-- crates/indexer/src/lib.rs | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 090cba62..af29f0e4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10156,8 +10156,8 @@ dependencies = [ name = "rollup-node-indexer" version = "0.0.1" dependencies = [ - "alloy-consensus 1.0.9", - "alloy-eips 1.0.9", + "alloy-consensus 1.0.20", + "alloy-eips 1.0.20", "alloy-primitives", "alloy-provider", "alloy-rpc-client", diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index 25170e60..37f71c75 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -1085,7 +1085,7 @@ mod test { // Verify the event structure match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head } => { + 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 @@ -1128,7 +1128,7 @@ mod test { indexer.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); let event = indexer.next().await.unwrap().unwrap(); match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head } => { + ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head } => { assert_eq!(batch_info.index, 100); assert_eq!(safe_head, None); } @@ -1139,7 +1139,7 @@ mod test { indexer.handle_l1_notification(L1Notification::BatchCommit(batch_2.clone())); let event = indexer.next().await.unwrap().unwrap(); match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head } => { + ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head } => { assert_eq!(batch_info.index, 101); assert_eq!(safe_head, None); } @@ -1150,7 +1150,7 @@ mod test { indexer.handle_l1_notification(L1Notification::BatchCommit(batch_3.clone())); let event = indexer.next().await.unwrap().unwrap(); match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head } => { + ChainOrchestratorEvent::BatchCommitIndexed { batch_info, safe_head } => { assert_eq!(batch_info.index, 102); assert_eq!(safe_head, None); } @@ -1174,13 +1174,13 @@ mod test { l1_messages: vec![], }; - indexer.handle_block(block_1.clone(), Some(batch_1_info)); + indexer.consolidate_l2_blocks(vec![block_1.clone()], Some(batch_1_info)); indexer.next().await.unwrap().unwrap(); - indexer.handle_block(block_2.clone(), Some(batch_2_info)); + indexer.consolidate_l2_blocks(vec![block_2.clone()], Some(batch_2_info)); indexer.next().await.unwrap().unwrap(); - indexer.handle_block(block_3.clone(), Some(batch_2_info)); + indexer.consolidate_l2_blocks(vec![block_3.clone()], Some(batch_2_info)); indexer.next().await.unwrap().unwrap(); // Now simulate a batch revert by submitting a new batch with index 101 @@ -1196,7 +1196,7 @@ mod test { // Verify the event indicates a batch revert match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head } => { + 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 From 4578d496a743e68ff6f37ef13b7922dec68b4bca Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 16 Jul 2025 12:56:41 +0800 Subject: [PATCH 10/30] refactor and add test cases --- crates/database/db/src/db.rs | 38 +--- crates/database/db/src/error.rs | 3 +- crates/database/db/src/operations.rs | 70 ++++--- crates/indexer/src/action.rs | 8 +- crates/indexer/src/error.rs | 7 +- crates/indexer/src/event.rs | 2 +- crates/indexer/src/lib.rs | 289 +++++++++++++++------------ crates/manager/src/manager/event.rs | 20 +- crates/manager/src/manager/mod.rs | 56 +++++- crates/node/src/args.rs | 18 +- crates/node/src/constants.rs | 2 +- crates/node/tests/sync.rs | 268 ++++++++++++++++++++++++- crates/watcher/src/lib.rs | 23 ++- crates/watcher/tests/reorg.rs | 15 +- 14 files changed, 604 insertions(+), 215 deletions(-) diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index f7d0d858..d8d487bc 100644 --- a/crates/database/db/src/db.rs +++ b/crates/database/db/src/db.rs @@ -469,32 +469,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. @@ -505,13 +479,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![] }, + Some(batch_info), + ) + .await + .unwrap(); } // Delete blocks with number > 405 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/operations.rs b/crates/database/db/src/operations.rs index 77b448d0..0979c747 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] @@ -258,7 +259,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .filter(models::l1_message::Column::Hash.eq(h.to_vec())) .one(self.get_connection()) .await? - .ok_or_else(|| DatabaseError::L1MessageNotFound(0))?; + .ok_or_else(|| DatabaseError::L1MessageNotFound(L1MessageStart::Hash(*h)))?; record.queue_index as u64 } @@ -340,16 +341,6 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { })?) } - /// 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."); - 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()))?) - } - /// Get an iterator over all L2 blocks in the database starting from the most recent one. async fn get_l2_blocks<'a>( &'a self, @@ -448,27 +439,31 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { block_info: L2BlockInfoWithL1Messages, batch_info: Option, ) -> Result<(), DatabaseError> { - 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), - block_number = block_info.block_info.number, - block_hash = ?block_info.block_info.hash, - "Inserting block into database." - ); - let l2_block: models::l2_block::ActiveModel = (block_info.block_info, batch_info).into(); - models::l2_block::Entity::insert(l2_block) - .on_conflict( - OnConflict::column(models::l2_block::Column::BlockNumber) - .update_columns([ - models::l2_block::Column::BlockHash, - models::l2_block::Column::BatchHash, - models::l2_block::Column::BatchIndex, - ]) - .to_owned(), - ) - .exec(self.get_connection()) - .await?; + // We only insert safe blocks into the database, we do not persist unsafe blocks. + if batch_info.is_some() { + 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), + block_number = block_info.block_info.number, + block_hash = ?block_info.block_info.hash, + "Inserting block into database." + ); + let l2_block: models::l2_block::ActiveModel = + (block_info.block_info, batch_info).into(); + models::l2_block::Entity::insert(l2_block) + .on_conflict( + OnConflict::column(models::l2_block::Column::BlockNumber) + .update_columns([ + models::l2_block::Column::BlockHash, + models::l2_block::Column::BatchHash, + models::l2_block::Column::BatchIndex, + ]) + .to_owned(), + ) + .exec(self.get_connection()) + .await?; + } tracing::trace!( target: "scroll::db", @@ -581,7 +576,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { /// /// 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)] +#[derive(Debug, Clone)] pub enum L1MessageStart { /// Start from the provided queue index. Index(u64), @@ -589,6 +584,15 @@ pub enum L1MessageStart { 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})"), + } + } +} + /// The result of [`DatabaseOperations::unwind`]. #[derive(Debug)] pub struct UnwindResult { diff --git a/crates/indexer/src/action.rs b/crates/indexer/src/action.rs index 21be9f93..5e7bc4c6 100644 --- a/crates/indexer/src/action.rs +++ b/crates/indexer/src/action.rs @@ -1,4 +1,4 @@ -use super::{ChainOrchestratorEvent, IndexerError}; +use super::{ChainOrchestratorError, ChainOrchestratorEvent}; use std::{ fmt, future::Future, @@ -6,9 +6,9 @@ use std::{ task::{Context, Poll}, }; -/// A future that resolves to a `Result`. +/// A future that resolves to a `Result`. pub(super) type PendingIndexerFuture = - Pin> + Send>>; + Pin> + Send>>; /// A type that represents a future that is being executed by the indexer. pub(super) enum IndexerFuture { @@ -26,7 +26,7 @@ impl IndexerFuture { pub(super) fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> { match self { Self::HandleReorg(fut) | Self::HandleFinalized(fut) | diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs index b1597516..e6b7eacf 100644 --- a/crates/indexer/src/error.rs +++ b/crates/indexer/src/error.rs @@ -1,9 +1,9 @@ use alloy_primitives::B256; -use scroll_db::DatabaseError; +use scroll_db::{DatabaseError, L1MessageStart}; /// A type that represents an error that occurred during indexing. #[derive(Debug, thiserror::Error)] -pub enum IndexerError { +pub enum ChainOrchestratorError { /// An error occurred while interacting with the database. #[error("indexing failed due to database error: {0}")] DatabaseError(#[from] DatabaseError), @@ -21,6 +21,9 @@ pub enum IndexerError { /// 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, diff --git a/crates/indexer/src/event.rs b/crates/indexer/src/event.rs index 0fe75240..f97fb9ed 100644 --- a/crates/indexer/src/event.rs +++ b/crates/indexer/src/event.rs @@ -58,5 +58,5 @@ pub enum ChainOrchestratorEvent { }, /// An L2 block has been committed returning the [`L2BlockInfoWithL1Messages`] and an /// optional [`BatchInfo`] if the block is associated with a committed batch. - L2BlockCommitted(L2BlockInfoWithL1Messages, Option), + L2ChainCommitted(L2BlockInfoWithL1Messages, Option, bool), } diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs index 37f71c75..0c2bfec1 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -23,12 +23,13 @@ use std::{ pin::Pin, sync::{ atomic::{AtomicU64, Ordering}, - Arc, Mutex, + Arc, }, task::{Context, Poll}, time::Instant, }; use strum::IntoEnumIterator; +use tokio::sync::Mutex; mod action; use action::{IndexerFuture, PendingIndexerFuture}; @@ -37,25 +38,17 @@ mod event; pub use event::ChainOrchestratorEvent; mod error; -pub use error::IndexerError; +pub use error::ChainOrchestratorError; mod metrics; pub use metrics::{IndexerItem, IndexerMetrics}; +/// The mask used to mask the L1 message queue hash. const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); /// The number of block headers we keep in memory. -const OPTIMISTIC_CHAIN_BUFFER_SIZE: usize = 2000; - -/// The maximum number of blocks we keep in memory for the consolidated chain (This is the full -/// unsafe chain). Memory requirements = ~800 bytes per header * `600_000` headers = 480 MB. -// TODO: This is just a temporary hack we will transition to a VecDeque when not in synced mode. -const CONSOLIDATED_CHAIN_BUFFER_SIZE: usize = 300_000; - -/// The threshold for optimistic syncing. If the received block is more than this many blocks -/// ahead of the current chain, we optimistically sync the chain. -const OPTIMISTIC_SYNC_THRESHOLD: u64 = 100; +const CHAIN_BUFFER_SIZE: usize = 2000; type Chain = BoundedVec

; @@ -82,6 +75,9 @@ pub struct ChainOrchestrator { 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, /// A boolean to represent if the L1 has been synced. l1_synced: bool, /// The waker to notify when the engine driver should be polled. @@ -100,6 +96,7 @@ impl< chain_spec: Arc, block_client: BC, l2_client: P, + optimistic_sync_threshold: u64, ) -> Self { let chain = init_chain_from_db(&database, &l2_client).await; Self { @@ -118,6 +115,7 @@ impl< }) .collect(), optimistic_mode: Arc::new(Mutex::new(false)), + optimistic_sync_threshold, l1_synced: false, waker: AtomicWaker::new(), } @@ -149,6 +147,7 @@ impl< let chain = self.chain.clone(); let l2_client = self.l2_client.clone(); let optimistic_mode = self.optimistic_mode.clone(); + let optimistic_sync_threshold = self.optimistic_sync_threshold; let network_client = self.network_client.clone(); let database = self.database.clone(); let fut = self.handle_metered( @@ -158,6 +157,7 @@ impl< chain, l2_client, optimistic_mode, + optimistic_sync_threshold, network_client, database, block_with_peer, @@ -174,33 +174,28 @@ impl< chain: Arc>, _l2_client: Arc

, optimistic_mode: Arc>, + optimistic_sync_threshold: u64, network_client: Arc, database: Arc, block_with_peer: NewBlockWithPeer, - ) -> Result { + ) -> Result { let NewBlockWithPeer { block: received_block, peer_id, signature } = block_with_peer; - let mut current_chain_headers = chain.lock().unwrap().clone().into_inner(); + 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; - let optimistic_mode_local: bool = { - let guard = optimistic_mode.lock().unwrap(); - *guard - }; - - // TODO: remove database lookups. // 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. - if (received_block.header.number - max_block_number) >= OPTIMISTIC_SYNC_THRESHOLD { + if (received_block.header.number - max_block_number) >= optimistic_sync_threshold { // fetch the latest `OPTIMISTIC_CHAIN_BUFFER_SIZE` blocks from the network for the // optimistic chain. let mut optimistic_headers = vec![received_block.header.clone()]; - while optimistic_headers.len() < OPTIMISTIC_CHAIN_BUFFER_SIZE && + while optimistic_headers.len() < CHAIN_BUFFER_SIZE && optimistic_headers.last().unwrap().number != 0 { - tracing::trace!(target: "scroll::watcher", number = ?(optimistic_headers.last().unwrap().number - 1), "fetching block"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(optimistic_headers.last().unwrap().number - 1), "fetching block"); let header = network_client .get_header(BlockHashOrNumber::Hash( optimistic_headers.last().unwrap().parent_hash, @@ -212,10 +207,10 @@ impl< optimistic_headers.push(header); } optimistic_headers.reverse(); - let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); new_chain.extend(optimistic_headers); - *chain.lock().unwrap() = new_chain; - *optimistic_mode.lock().unwrap() = true; + *chain.lock().await = new_chain; + *optimistic_mode.lock().await = true; return Ok(ChainOrchestratorEvent::OptimisticSync(received_block)); } @@ -224,7 +219,7 @@ impl< received_block.number >= min_block_number && current_chain_headers.iter().any(|h| h == &received_block.header) { - tracing::debug!(target: "scroll::watcher", block_hash = ?received_block.header.hash_slow(), "block already in chain"); + 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, @@ -233,7 +228,7 @@ impl< // If we are in optimistic mode, we return an event indicating that we have insufficient // data to process the block as we are optimistically syncing the chain. - if optimistic_mode_local && (received_block.header.number <= min_block_number) { + if *optimistic_mode.lock().await && (received_block.header.number <= min_block_number) { return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( received_block.header.hash_slow(), )); @@ -246,7 +241,7 @@ impl< 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 or the database. + // We search for the re-org index in the in-memory chain. let reorg_index = { loop { // If the new header tail has a block number that is less than the current header @@ -257,10 +252,10 @@ impl< { for _ in 0..50 { if new_header_tail.number.saturating_sub(1) < latest_safe_block.number { - tracing::info!(target: "scroll::chain", hash = %latest_safe_block.hash, number = %latest_safe_block.number, "reached safe block number - terminating fetching."); + tracing::info!(target: "scroll::chain_orchestrator", hash = %latest_safe_block.hash, number = %latest_safe_block.number, "reached safe block number - terminating fetching."); break; } - tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_header_tail.number - 1), "fetching block"); if let Some(header) = network_client .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) .await @@ -270,7 +265,7 @@ impl< new_chain_headers.push(header.clone()); new_header_tail = header; } else { - return Err(IndexerError::MissingBlockHeader { + return Err(ChainOrchestratorError::MissingBlockHeader { hash: new_header_tail.parent_hash, }); } @@ -305,7 +300,7 @@ impl< // completed. If the current header block number is less than the // latest safe block number then this would suggest a reorg of a // safe block which is not invalid - terminate the search. - if optimistic_mode_local && + if *optimistic_mode.lock().await && (new_header_tail.number <= current_chain_headers .front() @@ -315,11 +310,11 @@ impl< if received_block.timestamp > current_chain_headers.back().expect("chain can not be empty").timestamp { - tracing::debug!(target: "scroll::watcher", block_hash = ?received_block.header.hash_slow(), "received block is ahead of the current chain"); - while new_chain_headers.len() < OPTIMISTIC_CHAIN_BUFFER_SIZE && + tracing::debug!(target: "scroll::chain_orchestrator", block_hash = ?received_block.header.hash_slow(), "received block is ahead of the current chain"); + while new_chain_headers.len() < CHAIN_BUFFER_SIZE && new_chain_headers.last().unwrap().number != 0 { - tracing::trace!(target: "scroll::watcher", number = ?(new_chain_headers.last().unwrap().number - 1), "fetching block"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_chain_headers.last().unwrap().number - 1), "fetching block"); let header = network_client .get_header(BlockHashOrNumber::Hash( new_chain_headers.last().unwrap().parent_hash, @@ -340,10 +335,10 @@ impl< // If the current header block number is less than the latest safe block number then // we should error. if new_header_tail.number <= latest_safe_block.number { - return Err(IndexerError::L2SafeBlockReorgDetected); + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); } - tracing::trace!(target: "scroll::watcher", number = ?(new_header_tail.number - 1), "fetching block"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_header_tail.number - 1), "fetching block"); if let Some(header) = network_client .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) .await @@ -356,7 +351,7 @@ impl< new_chain_headers.push(header.clone()); new_header_tail = header; } else { - return Err(IndexerError::MissingBlockHeader { + return Err(ChainOrchestratorError::MissingBlockHeader { hash: new_header_tail.parent_hash, }); } @@ -370,11 +365,11 @@ impl< let new_blocks = if new_chain_headers.len() == 1 { vec![received_block] } else { - fetch_blocks(new_chain_headers.clone(), network_client.clone()).await + fetch_blocks_from_network(new_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_local { + if !*optimistic_mode.lock().await { validate_l1_messages(&new_blocks, database.clone()).await?; } @@ -384,9 +379,9 @@ impl< Some(index) if index == 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(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); new_chain.extend(current_chain_headers); - *chain.lock().unwrap() = new_chain; + *chain.lock().await = new_chain; Ok(ChainOrchestratorEvent::ChainExtended(ChainImport::new( new_blocks, peer_id, signature, @@ -396,20 +391,20 @@ impl< // point and extend it with the new blocks. Some(position) => { // reorg the in-memory chain to the new chain and issue a reorg event. - let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); new_chain.extend(current_chain_headers.iter().take(position).cloned()); new_chain.extend(new_chain_headers); - *chain.lock().unwrap() = new_chain; + *chain.lock().await = new_chain; Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( new_blocks, peer_id, signature, ))) } None => { - let mut new_chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); new_chain.extend(new_chain_headers); - *chain.lock().unwrap() = new_chain; - *optimistic_mode.lock().unwrap() = true; + *chain.lock().await = new_chain; + *optimistic_mode.lock().await = true; Ok(ChainOrchestratorEvent::OptimisticSync( new_blocks.last().cloned().expect("new_blocks should not be empty"), @@ -429,30 +424,38 @@ impl< let optimistic_mode = self.optimistic_mode.clone(); let chain = self.chain.clone(); let l2_client = self.l2_client.clone(); - let fut = self.handle_metered( - IndexerItem::InsertL2Block, - Box::pin(async move { - // If we are in optimistic mode and the L1 is synced, we consolidate the chain and - // disable optimistic mode. - if l1_synced && *optimistic_mode.lock().unwrap() { - consolidate_chain(database.clone(), block_info.clone(), chain, l2_client) - .await?; - *optimistic_mode.lock().unwrap() = false; - } - - // If we are consolidating a batch, we insert the batch info into the database. - let head = block_info.last().expect("block info must not be empty").clone(); - if batch_info.is_some() { - database.insert_blocks(block_info, batch_info).await?; - } + let fut = + self.handle_metered( + IndexerItem::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) + .await?; + *optimistic_mode.lock().await = false; + true + } else { + false + }; + + // If we are consolidating a batch, we insert the batch info into the database. + let head = block_info.last().expect("block info must not be empty").clone(); + if batch_info.is_some() { + database.insert_blocks(block_info, batch_info).await?; + } - Result::<_, IndexerError>::Ok(ChainOrchestratorEvent::L2BlockCommitted( - head, batch_info, - )) - }), - ); + Result::<_, ChainOrchestratorError>::Ok( + ChainOrchestratorEvent::L2ChainCommitted(head, batch_info, consolidated), + ) + }), + ); - self.pending_futures.push_back(IndexerFuture::HandleDerivedBlock(fut)) + self.pending_futures.push_back(IndexerFuture::HandleDerivedBlock(fut)); + self.waker.wake(); } /// Handles an event from the L1. @@ -508,6 +511,10 @@ impl< )), )) } + L1Notification::Synced => { + self.set_l1_synced_status(true); + return + } }; self.pending_futures.push_back(fut); @@ -519,7 +526,7 @@ impl< database: Arc, chain_spec: Arc, l1_block_number: u64, - ) -> Result { + ) -> Result { let txn = database.tx().await?; let UnwindResult { l1_block_number, queue_index, l2_head_block_info, l2_safe_block_info } = txn.unwind(chain_spec.genesis_hash(), l1_block_number).await?; @@ -539,7 +546,7 @@ impl< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result { // Set the latest finalized L1 block in the database. database.set_latest_finalized_l1_block_number(block_number).await?; @@ -566,7 +573,7 @@ impl< l1_message: TxL1Message, l1_block_number: u64, block_timestamp: u64, - ) -> Result { + ) -> Result { let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); let queue_hash = if chain_spec @@ -579,7 +586,7 @@ impl< .get_l1_message_by_index(index) .await? .map(|m| m.queue_hash) - .ok_or(DatabaseError::L1MessageNotFound(index))?; + .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()); @@ -597,7 +604,7 @@ impl< async fn handle_batch_commit( database: Arc, batch: BatchCommitData, - ) -> Result { + ) -> Result { let txn = database.tx().await?; let prev_batch_index = batch.index - 1; @@ -631,7 +638,7 @@ impl< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result { // finalized the batch. database.finalize_batch(batch_hash, block_number).await?; @@ -654,7 +661,7 @@ impl< database: Arc, batch_hash: B256, l2_block_number: Arc, - ) -> Result, IndexerError> { + ) -> 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. @@ -676,9 +683,8 @@ async fn init_chain_from_db + 'static>( l2_client: &P, ) -> BoundedVec

{ let blocks = { - let mut blocks = Vec::with_capacity(OPTIMISTIC_CHAIN_BUFFER_SIZE); - let mut blocks_stream = - database.get_l2_blocks().await.unwrap().take(OPTIMISTIC_CHAIN_BUFFER_SIZE); + 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) @@ -692,7 +698,7 @@ async fn init_chain_from_db + 'static>( blocks.reverse(); blocks }; - let mut chain: Chain = Chain::new(OPTIMISTIC_CHAIN_BUFFER_SIZE); + let mut chain: Chain = Chain::new(CHAIN_BUFFER_SIZE); chain.extend(blocks); chain } @@ -703,7 +709,7 @@ impl< P: Provider + Send + Sync + 'static, > Stream for ChainOrchestrator { - type Item = Result; + 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. @@ -728,70 +734,95 @@ impl< /// This is used to ensure that the in-memory chain is consistent with the L2 chain. async fn consolidate_chain>( database: Arc, - _block_info: Vec, + validated_chain: Vec, _chain: Arc>, l2_client: P, -) -> Result<(), IndexerError> { - // take the current chain. - let chain = std::mem::take(&mut *_chain.lock().unwrap()); +) -> Result<(), ChainOrchestratorError> { + // Take the current in memory chain. + let chain = std::mem::take(&mut *_chain.lock().await); - // Find highest common ancestor by comparing hashes + // 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(); - _block_info.iter().any(|b| b.block_info.hash == h_hash) + validated_chain.iter().any(|b| b.block_info.hash == h_hash) }); - // This means there is a deep reorg that has just occurred however in practice this should - // never happen due to purging of expired chain. This should be reconciled upon the next - // block import. - // TODO: consider this case more carefully. + // 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. - *_chain.lock().unwrap() = chain; - return Err(IndexerError::ChainInconsistency); + *_chain.lock().await = chain; + return Err(ChainOrchestratorError::ChainInconsistency); } - // Now reconcile back to the safe head. - let safe_head = database.get_latest_l2_block().await?.expect("safe head must exist"); - let starting_block = - l2_client.get_block_by_hash(chain.first().unwrap().hash_slow()).await.unwrap().unwrap(); - let mut consolidated_chain_blocks = - vec![starting_block.into_consensus().map_transactions(|tx| tx.inner.into_inner())]; - while consolidated_chain_blocks.last().unwrap().header.parent_hash != safe_head.hash { - // Fetch the missing blocks from the L2 client. - let block = - l2_client.get_block_by_hash(chain.last().unwrap().parent_hash).await.unwrap().unwrap(); - consolidated_chain_blocks - .push(block.into_consensus().map_transactions(|tx| tx.inner.into_inner())); - - if chain.last().unwrap().number < safe_head.number { + // 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. - *_chain.lock().unwrap() = chain; + *_chain.lock().await = chain; // TODO: should we revert to the last known safe head. - return Err(IndexerError::ChainInconsistency); + 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); } - consolidated_chain_blocks.reverse(); - validate_l1_messages(&consolidated_chain_blocks, database.clone()).await?; + // 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 { + *_chain.lock().await = chain; + // TODO: should we revert to the last known safe head. + return Err(ChainOrchestratorError::ChainInconsistency); + } + } - let mut consolidated_chain = BoundedVec::new(CONSOLIDATED_CHAIN_BUFFER_SIZE); - consolidated_chain.extend(consolidated_chain_blocks.iter().map(|b| b.header.clone())); - consolidated_chain.extend(chain.into_inner()); + // 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?; - // let unsafe_chain = + // Set the chain which has now been consolidated. + *_chain.lock().await = chain; - // TODO: implement the logic to consolidate the chain. - // If we are in optimistic mode, we consolidate the chain and disable optimistic - // mode. - // let mut chain = chain.lock().unwrap(); - // if !chain.is_empty() { - // database.insert_chain(chain.drain(..).collect()).await?; - // } Ok(()) } -async fn fetch_blocks + Send + Sync + 'static>( +async fn fetch_blocks_from_network + Send + Sync + 'static>( headers: Vec
, client: Arc, ) -> Vec { @@ -815,7 +846,7 @@ async fn fetch_blocks + Send + Sync + 'stat async fn validate_l1_messages( blocks: &[ScrollBlock], database: Arc, -) -> Result<(), IndexerError> { +) -> Result<(), ChainOrchestratorError> { let l1_message_hashes = blocks .iter() .flat_map(|block| { @@ -829,6 +860,9 @@ async fn validate_l1_messages( .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?; @@ -839,7 +873,7 @@ async fn validate_l1_messages( // If the received and expected L1 messages do not match return an error. if message_hash != expected_hash { - return Err(IndexerError::L1MessageMismatch { + return Err(ChainOrchestratorError::L1MessageMismatch { expected: expected_hash, actual: message_hash, }); @@ -880,6 +914,8 @@ mod test { type ScrollBody = ::Body; + const TEST_OPTIMISTIC_SYNC_THRESHOLD: u64 = 100; + /// 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. /// @@ -1062,6 +1098,7 @@ mod test { SCROLL_MAINNET.clone(), TestScrollFullBlockClient::default(), provider, + TEST_OPTIMISTIC_SYNC_THRESHOLD, ) .await, db, diff --git a/crates/manager/src/manager/event.rs b/crates/manager/src/manager/event.rs index ea708e2d..947a4e01 100644 --- a/crates/manager/src/manager/event.rs +++ b/crates/manager/src/manager/event.rs @@ -1,7 +1,9 @@ +use alloy_primitives::B256; use reth_scroll_primitives::ScrollBlock; use rollup_node_indexer::ChainOrchestratorEvent; -use rollup_node_primitives::ChainImport; +use rollup_node_primitives::{BatchInfo, ChainImport, L2BlockInfoWithL1Messages}; use rollup_node_signer::SignerEvent; +use scroll_db::L1MessageStart; use scroll_engine::ConsolidationOutcome; use scroll_network::NewBlockWithPeer; @@ -22,8 +24,24 @@ pub enum RollupManagerEvent { SignerEvent(SignerEvent), /// 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 optimistic sync has been triggered by the chain orchestrator. OptimisticSyncTriggered(ScrollBlock), /// A chain extension has been triggered by the chain orchestrator. ChainExtensionTriggered(ChainImport), + /// An L2 chain has been committed. + L2ChainCommitted(L2BlockInfoWithL1Messages, Option, bool), + /// The L1 watcher has synced to the tip of the L1 chain. + L1Synced, } diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 5457ec7d..35965f39 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -11,7 +11,7 @@ use reth_network_api::FullNetwork; use reth_scroll_node::ScrollNetworkPrimitives; use reth_tasks::shutdown::GracefulShutdown; use reth_tokio_util::{EventSender, EventStream}; -use rollup_node_indexer::{ChainOrchestrator, ChainOrchestratorEvent}; +use rollup_node_indexer::{ChainOrchestrator, ChainOrchestratorError, ChainOrchestratorEvent}; use rollup_node_primitives::BlockInfo; use rollup_node_sequencer::Sequencer; use rollup_node_signer::{SignerEvent, SignerHandle}; @@ -38,7 +38,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; @@ -319,6 +319,16 @@ where // Issue the new block info to the engine driver for processing. self.engine.handle_optimistic_sync(block_info) } + ChainOrchestratorEvent::L2ChainCommitted(block_infos, batch_into, consolidated) => { + trace!(target: "scroll::node::manager", ?block_infos, ?batch_into, ?consolidated, "Received L2 chain committed event"); + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::L2ChainCommitted( + block_infos, + batch_into, + consolidated, + )); + } + } event => { trace!(target: "scroll::node::manager", ?event, "Received chain orchestrator event"); if let Some(event_sender) = self.event_sender.as_ref() { @@ -384,6 +394,17 @@ where if let L1Notification::Consensus(ref update) = notification { self.consensus.update_config(update); } + if notification == L1Notification::Synced { + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::L1Synced); + } + } + if let L1Notification::NewBlock(block) = notification { + self.sequencer + .as_mut() + .expect("Sequencer must be initialized") + .handle_new_l1_block(block); + } self.chain.handle_l1_notification(notification) } @@ -481,7 +502,36 @@ where match result { Ok(event) => this.handle_indexer_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 at indexer level" + ); } } } diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 9ae81ae9..5f9cff03 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -284,8 +284,14 @@ impl ScrollRollupNodeConfig { .fetch_client() .await .expect("failed to fetch block client"); - let chain_orchestrator = - ChainOrchestrator::new(db.clone(), chain_spec.clone(), block_client, l2_provider).await; + let chain_orchestrator = ChainOrchestrator::new( + db.clone(), + chain_spec.clone(), + block_client, + l2_provider, + self.engine_driver_args.en_sync_trigger, + ) + .await; // Spawn the rollup node manager let (rnm, handle) = RollupNodeManager::new( @@ -315,7 +321,7 @@ pub struct DatabaseArgs { } /// 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. @@ -323,6 +329,12 @@ pub struct EngineDriverArgs { pub en_sync_trigger: u64, } +impl Default for EngineDriverArgs { + fn default() -> Self { + Self { en_sync_trigger: constants::BLOCK_GAP_TRIGGER } + } +} + /// The network arguments. #[derive(Debug, Default, Clone, clap::Args)] pub struct NetworkArgs { diff --git a/crates/node/src/constants.rs b/crates/node/src/constants.rs index 5ce62ae8..1bfc2e08 100644 --- a/crates/node/src/constants.rs +++ b/crates/node/src/constants.rs @@ -20,4 +20,4 @@ pub(crate) const DEFAULT_PAYLOAD_BUILDING_DURATION: u64 = 800; 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; diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index f43e0a0f..d1caf20c 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -1,5 +1,6 @@ //! Contains tests related to RN and EN sync. +use alloy_primitives::{Address, U256}; use alloy_provider::{Provider, ProviderBuilder}; use futures::StreamExt; use reth_e2e_test_utils::NodeHelperType; @@ -9,9 +10,14 @@ use rollup_node::{ default_sequencer_test_scroll_rollup_node_config, default_test_scroll_rollup_node_config, setup_engine, }, - ScrollRollupNode, + BeaconProviderArgs, DatabaseArgs, EngineDriverArgs, L1ProviderArgs, NetworkArgs, + ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, }; use rollup_node_manager::RollupManagerEvent; +use rollup_node_sequencer::L1MessageInclusionMode; +use rollup_node_watcher::L1Notification; +use scroll_alloy_consensus::TxL1Message; +use std::{path::PathBuf, sync::Arc}; /// We test if the syncing of the RN is correctly triggered and released when the EN reaches sync. #[allow(clippy::large_stack_frames)] @@ -64,6 +70,266 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { .await; } +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_should_consolidate_after_optimistic_sync() { + 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 }, + database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::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::default(), + signer_args: Default::default(), + }; + + // 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).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 nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec, false).await.unwrap(); + let mut unsynced = nodes.pop().unwrap(); + let unsynced_l1_watcher_tx = unsynced.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // Create a sequence of L1 messages to be added to the sequencer node. + const L1_MESSAGES_COUNT: usize = 200; + let mut l1_messages = Vec::with_capacity(L1_MESSAGES_COUNT); + 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(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 1) + .await; + sequencer_l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(i as u64))).await.unwrap(); + sequencer_handle.build_block().await; + wait_n_events( + &sequencer, + |e: RollupManagerEvent| matches!(e, RollupManagerEvent::BlockSequenced(_)), + 1, + ) + .await; + } + + // Connect the nodes together. + sequencer.network.add_peer(unsynced.network.record()).await; + unsynced.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(&unsynced, |e| matches!(e, RollupManagerEvent::OptimisticSyncTriggered(_)), 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() { + unsynced_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( + &unsynced, + |e: RollupManagerEvent| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), + 1, + ) + .await; + } + + // Send a notification to the unsynced node that the L1 watcher is synced. + unsynced_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + + // Wait for the unsynced node to sync to the L1 watcher. + wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::L1Synced), 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(&unsynced, |e| matches!(e, RollupManagerEvent::L2ChainCommitted(_, _, true)), 1) + .await; + + // Now push a L1 message to the sequencer node and build a new block. + unsynced_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: 201, + block_timestamp: 2010, + })) + .await + .unwrap(); + sequencer_handle.build_block().await; + + wait_n_events(&unsynced, |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. + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; +} + +#[allow(clippy::large_stack_frames)] +#[tokio::test] +async fn test_consolidation() { + 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 }, + database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, + l1_provider_args: L1ProviderArgs::default(), + engine_driver_args: EngineDriverArgs::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::default(), + signer_args: Default::default(), + }; + + // 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).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 nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec, false).await.unwrap(); + let mut follower = nodes.pop().unwrap(); + let follower_l1_watcher_tx = follower.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + // 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(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 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(&follower, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 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(&follower, |e| matches!(e, RollupManagerEvent::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(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 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( + &follower, + |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { start: _ }), + 1, + ) + .await; +} + /// Waits for n events to be emitted. async fn wait_n_events( node: &NodeHelperType, diff --git a/crates/watcher/src/lib.rs b/crates/watcher/src/lib.rs index b84531d0..9fa6ac9e 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"), } } } @@ -181,6 +186,7 @@ where sender: tx, config, metrics: WatcherMetrics::default(), + is_synced: false, }; // notify at spawn. @@ -212,8 +218,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; } } } @@ -569,11 +584,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 +698,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. From e6248f5fa3b9ec8e83ca7ce001f12a5fbce49bfa Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 16 Jul 2025 22:10:05 +0800 Subject: [PATCH 11/30] cleanup --- Cargo.lock | 1 + crates/indexer/Cargo.toml | 2 + crates/indexer/src/error.rs | 8 + crates/indexer/src/lib.rs | 235 ++++++++++++--------------- crates/primitives/src/bounded_vec.rs | 14 +- crates/watcher/src/lib.rs | 2 +- 6 files changed, 125 insertions(+), 137 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index af29f0e4..060438ae 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10158,6 +10158,7 @@ version = "0.0.1" dependencies = [ "alloy-consensus 1.0.20", "alloy-eips 1.0.20", + "alloy-json-rpc", "alloy-primitives", "alloy-provider", "alloy-rpc-client", diff --git a/crates/indexer/Cargo.toml b/crates/indexer/Cargo.toml index 93c62e07..ecbe66d5 100644 --- a/crates/indexer/Cargo.toml +++ b/crates/indexer/Cargo.toml @@ -13,8 +13,10 @@ workspace = true # 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 diff --git a/crates/indexer/src/error.rs b/crates/indexer/src/error.rs index e6b7eacf..b08e8c81 100644 --- a/crates/indexer/src/error.rs +++ b/crates/indexer/src/error.rs @@ -1,4 +1,6 @@ +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 during indexing. @@ -33,4 +35,10 @@ pub enum ChainOrchestratorError { /// 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/indexer/src/lib.rs b/crates/indexer/src/lib.rs index 0c2bfec1..0835a41f 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/indexer/src/lib.rs @@ -172,7 +172,7 @@ impl< /// Handles a new block received from the network. pub async fn handle_new_block( chain: Arc>, - _l2_client: Arc

, + l2_client: Arc

, optimistic_mode: Arc>, optimistic_sync_threshold: u64, network_client: Arc, @@ -187,29 +187,26 @@ impl< // 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. + // update the in-memory buffer to represent the optimistic chain. if (received_block.header.number - max_block_number) >= optimistic_sync_threshold { - // fetch the latest `OPTIMISTIC_CHAIN_BUFFER_SIZE` blocks from the network for the + // fetch the latest `CHAIN_BUFFER_SIZE` headers from the network for the // optimistic chain. - let mut optimistic_headers = vec![received_block.header.clone()]; + 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.last().unwrap().number != 0 + optimistic_headers.first().unwrap().number != 0 { - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(optimistic_headers.last().unwrap().number - 1), "fetching block"); + 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( - optimistic_headers.last().unwrap().parent_hash, - )) - .await - .unwrap() + .get_header(BlockHashOrNumber::Hash(parent_hash)) + .await? .into_data() - .unwrap(); - optimistic_headers.push(header); + .ok_or(ChainOrchestratorError::MissingBlockHeader { hash: parent_hash })?; + optimistic_headers.push_front(header); } - optimistic_headers.reverse(); - let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); - new_chain.extend(optimistic_headers); - *chain.lock().await = new_chain; + + *chain.lock().await = optimistic_headers; *optimistic_mode.lock().await = true; return Ok(ChainOrchestratorEvent::OptimisticSync(received_block)); } @@ -226,146 +223,117 @@ impl< )); } - // If we are in optimistic mode, we return an event indicating that we have insufficient - // data to process the block as we are optimistically syncing the chain. + // 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(), )); }; - let mut new_chain_headers = vec![received_block.header.clone()]; - let mut new_header_tail = received_block.header.clone(); + // 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()]; + let mut received_header_tail = 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. - let reorg_index = { - loop { - // If the new header tail has a block number that is less than the current header - // tail then we should fetch more blocks for the current header chain to aid - // reconciliation. - if new_header_tail.number <= - current_chain_headers.back().expect("chain can not be empty").number - { - for _ in 0..50 { - if new_header_tail.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 - terminating fetching."); - break; - } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_header_tail.number - 1), "fetching block"); - if let Some(header) = network_client - .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) - .await - .unwrap() - .into_data() - { - new_chain_headers.push(header.clone()); - new_header_tail = header; - } else { - return Err(ChainOrchestratorError::MissingBlockHeader { - hash: new_header_tail.parent_hash, - }); - } - } - } + 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 current header block number is greater than the in-memory chain then we - // should search the in-memory chain (we keep the latest - // `OPTIMISTIC_CHAIN_BUFFER_SIZE` headers in memory). - if let Some(pos) = current_chain_headers - .iter() - .rposition(|h| h.hash_slow() == new_header_tail.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 >= - new_header_tail.timestamp + // 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_header_tail.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 { - return Ok(ChainOrchestratorEvent::OldForkReceived { - headers: new_chain_headers, - peer_id, - signature, - }); + 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; } - break Some(pos); - } - - // If we are in optimistic mode, we terminate the search as we don't have the - // necessary data from L1 consolidation yet. This is fine because very deep - // re-orgs are rare and in any case will be resolved once optimistic sync is - // completed. If the current header block number is less than the - // latest safe block number then this would suggest a reorg of a - // safe block which is not invalid - terminate the search. - if *optimistic_mode.lock().await && - (new_header_tail.number <= - current_chain_headers - .front() - .expect("chain must not be empty") - .number) - { - if received_block.timestamp > - current_chain_headers.back().expect("chain can not be empty").timestamp + 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? { - tracing::debug!(target: "scroll::chain_orchestrator", block_hash = ?received_block.header.hash_slow(), "received block is ahead of the current chain"); - while new_chain_headers.len() < CHAIN_BUFFER_SIZE && - new_chain_headers.last().unwrap().number != 0 - { - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_chain_headers.last().unwrap().number - 1), "fetching block"); - let header = network_client - .get_header(BlockHashOrNumber::Hash( - new_chain_headers.last().unwrap().parent_hash, - )) - .await - .unwrap() - .into_data() - .unwrap(); - new_chain_headers.push(header); - } - break None; + let header = block.into_consensus_header(); + current_chain_headers.push_front(header.clone()); + received_header_tail = header; + } else { + return Err(ChainOrchestratorError::MissingBlockHeader { + hash: current_chain_headers.front().unwrap().parent_hash, + }); } - return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( - received_block.header.hash_slow(), - )); - } - - // If the current header block number is less than the latest safe block number then - // we should error. - if new_header_tail.number <= latest_safe_block.number { - return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); } + } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(new_header_tail.number - 1), "fetching block"); - if let Some(header) = network_client - .get_header(BlockHashOrNumber::Hash(new_header_tail.parent_hash)) - .await - .unwrap() - .into_data() + // If the current header block number is greater than the in-memory chain then we + // should search the in-memory chain. + if let Some(pos) = current_chain_headers + .iter() + .rposition(|h| h.hash_slow() == received_header_tail.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_header_tail.timestamp { - // TODO: what do we do when peers don't have the blocks? We can't recreate the - // chain so we should terminate here. We should be able to reconcile this gap in - // a future block. - new_chain_headers.push(header.clone()); - new_header_tail = header; - } else { - return Err(ChainOrchestratorError::MissingBlockHeader { - hash: new_header_tail.parent_hash, + return Ok(ChainOrchestratorEvent::OldForkReceived { + headers: received_chain_headers, + peer_id, + signature, }); } + break Some(pos); + } + + // If the current header block number is less than the latest safe block number then + // we should error. + if received_header_tail.number <= latest_safe_block.number { + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + } + + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_header_tail.number - 1), "fetching block"); + if let Some(header) = network_client + .get_header(BlockHashOrNumber::Hash(received_header_tail.parent_hash)) + .await? + .into_data() + { + received_chain_headers.push(header.clone()); + received_header_tail = header; + } else { + return Err(ChainOrchestratorError::MissingBlockHeader { + hash: received_header_tail.parent_hash, + }); } }; // Reverse the new chain headers to have them in the correct order. - new_chain_headers.reverse(); + received_chain_headers.reverse(); // Fetch the blocks associated with the new chain headers. - let new_blocks = if new_chain_headers.len() == 1 { + let new_blocks = if received_chain_headers.len() == 1 { vec![received_block] } else { - fetch_blocks_from_network(new_chain_headers.clone(), network_client.clone()).await + 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. @@ -393,7 +361,7 @@ impl< // 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(new_chain_headers); + new_chain.extend(received_chain_headers); *chain.lock().await = new_chain; Ok(ChainOrchestratorEvent::ChainReorged(ChainImport::new( @@ -402,7 +370,7 @@ impl< } None => { let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); - new_chain.extend(new_chain_headers); + new_chain.extend(received_chain_headers); *chain.lock().await = new_chain; *optimistic_mode.lock().await = true; @@ -732,7 +700,7 @@ impl< /// 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>( +async fn consolidate_chain + 'static>( database: Arc, validated_chain: Vec, _chain: Arc>, @@ -776,8 +744,8 @@ async fn consolidate_chain>( 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. - *_chain.lock().await = chain; - // TODO: should we revert to the last known safe head. + *_chain.lock().await = init_chain_from_db(&database, &l2_client).await; + return Err(ChainOrchestratorError::ChainInconsistency); } }; @@ -805,8 +773,7 @@ async fn consolidate_chain>( // 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 { - *_chain.lock().await = chain; - // TODO: should we revert to the last known safe head. + *_chain.lock().await = init_chain_from_db(&database, &l2_client).await; return Err(ChainOrchestratorError::ChainInconsistency); } } diff --git a/crates/primitives/src/bounded_vec.rs b/crates/primitives/src/bounded_vec.rs index eaf74526..305e2281 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() @@ -87,7 +97,7 @@ impl Extend for BoundedVec { }; for elem in iter { - self.push(elem) + self.push_back(elem) } } } diff --git a/crates/watcher/src/lib.rs b/crates/watcher/src/lib.rs index 9fa6ac9e..0fcffc72 100644 --- a/crates/watcher/src/lib.rs +++ b/crates/watcher/src/lib.rs @@ -331,7 +331,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"); From 7a6ebfcc88c8c530331ee1ee1e029eac551735f8 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 16 Jul 2025 22:21:39 +0800 Subject: [PATCH 12/30] rename chain orchestrator crate --- .github/workflows/lint.yaml | 4 ++-- Cargo.lock | 6 +++--- Cargo.toml | 4 ++-- crates/{indexer => chain-orchestrator}/Cargo.toml | 2 +- crates/{indexer => chain-orchestrator}/src/action.rs | 0 crates/{indexer => chain-orchestrator}/src/error.rs | 0 crates/{indexer => chain-orchestrator}/src/event.rs | 0 crates/{indexer => chain-orchestrator}/src/lib.rs | 3 ++- crates/{indexer => chain-orchestrator}/src/metrics.rs | 0 .../testdata/genesis_block.json | 0 .../testdata/genesis_block_rpc.json | 0 crates/manager/Cargo.toml | 2 +- crates/manager/src/manager/event.rs | 2 +- crates/manager/src/manager/mod.rs | 4 +++- crates/node/Cargo.toml | 2 +- crates/node/src/args.rs | 2 +- 16 files changed, 17 insertions(+), 14 deletions(-) rename crates/{indexer => chain-orchestrator}/Cargo.toml (97%) rename crates/{indexer => chain-orchestrator}/src/action.rs (100%) rename crates/{indexer => chain-orchestrator}/src/error.rs (100%) rename crates/{indexer => chain-orchestrator}/src/event.rs (100%) rename crates/{indexer => chain-orchestrator}/src/lib.rs (99%) rename crates/{indexer => chain-orchestrator}/src/metrics.rs (100%) rename crates/{indexer => chain-orchestrator}/testdata/genesis_block.json (100%) rename crates/{indexer => chain-orchestrator}/testdata/genesis_block_rpc.json (100%) diff --git a/.github/workflows/lint.yaml b/.github/workflows/lint.yaml index ca1dc313..ee86f9a6 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 + 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 - 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 + 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 steps: - uses: actions/checkout@v4 - uses: rui314/setup-mold@v1 diff --git a/Cargo.lock b/Cargo.lock index 060438ae..8554f223 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10128,7 +10128,7 @@ dependencies = [ "reth-transaction-pool", "reth-trie-db", "rollup-node", - "rollup-node-indexer", + "rollup-node-chain-orchestrator", "rollup-node-manager", "rollup-node-primitives", "rollup-node-providers", @@ -10153,7 +10153,7 @@ dependencies = [ ] [[package]] -name = "rollup-node-indexer" +name = "rollup-node-chain-orchestrator" version = "0.0.1" dependencies = [ "alloy-consensus 1.0.20", @@ -10211,7 +10211,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", diff --git a/Cargo.toml b/Cargo.toml index e7a84e60..23e092d9 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", @@ -167,7 +167,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/crates/indexer/Cargo.toml b/crates/chain-orchestrator/Cargo.toml similarity index 97% rename from crates/indexer/Cargo.toml rename to crates/chain-orchestrator/Cargo.toml index ecbe66d5..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 diff --git a/crates/indexer/src/action.rs b/crates/chain-orchestrator/src/action.rs similarity index 100% rename from crates/indexer/src/action.rs rename to crates/chain-orchestrator/src/action.rs diff --git a/crates/indexer/src/error.rs b/crates/chain-orchestrator/src/error.rs similarity index 100% rename from crates/indexer/src/error.rs rename to crates/chain-orchestrator/src/error.rs diff --git a/crates/indexer/src/event.rs b/crates/chain-orchestrator/src/event.rs similarity index 100% rename from crates/indexer/src/event.rs rename to crates/chain-orchestrator/src/event.rs diff --git a/crates/indexer/src/lib.rs b/crates/chain-orchestrator/src/lib.rs similarity index 99% rename from crates/indexer/src/lib.rs rename to crates/chain-orchestrator/src/lib.rs index 0835a41f..e9ced04e 100644 --- a/crates/indexer/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -52,7 +52,8 @@ const CHAIN_BUFFER_SIZE: usize = 2000; type Chain = BoundedVec

; -/// The indexer is responsible for indexing data relevant to the L1. +/// 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. diff --git a/crates/indexer/src/metrics.rs b/crates/chain-orchestrator/src/metrics.rs similarity index 100% rename from crates/indexer/src/metrics.rs rename to crates/chain-orchestrator/src/metrics.rs diff --git a/crates/indexer/testdata/genesis_block.json b/crates/chain-orchestrator/testdata/genesis_block.json similarity index 100% rename from crates/indexer/testdata/genesis_block.json rename to crates/chain-orchestrator/testdata/genesis_block.json diff --git a/crates/indexer/testdata/genesis_block_rpc.json b/crates/chain-orchestrator/testdata/genesis_block_rpc.json similarity index 100% rename from crates/indexer/testdata/genesis_block_rpc.json rename to crates/chain-orchestrator/testdata/genesis_block_rpc.json 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/event.rs b/crates/manager/src/manager/event.rs index 947a4e01..9f38d12d 100644 --- a/crates/manager/src/manager/event.rs +++ b/crates/manager/src/manager/event.rs @@ -1,6 +1,6 @@ use alloy_primitives::B256; use reth_scroll_primitives::ScrollBlock; -use rollup_node_indexer::ChainOrchestratorEvent; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_primitives::{BatchInfo, ChainImport, L2BlockInfoWithL1Messages}; use rollup_node_signer::SignerEvent; use scroll_db::L1MessageStart; diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 35965f39..2de33315 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -11,7 +11,9 @@ use reth_network_api::FullNetwork; use reth_scroll_node::ScrollNetworkPrimitives; use reth_tasks::shutdown::GracefulShutdown; use reth_tokio_util::{EventSender, EventStream}; -use rollup_node_indexer::{ChainOrchestrator, ChainOrchestratorError, ChainOrchestratorEvent}; +use rollup_node_chain_orchestrator::{ + ChainOrchestrator, ChainOrchestratorError, ChainOrchestratorEvent, +}; use rollup_node_primitives::BlockInfo; use rollup_node_sequencer::Sequencer; use rollup_node_signer::{SignerEvent, SignerHandle}; diff --git a/crates/node/Cargo.toml b/crates/node/Cargo.toml index ef3d6fe3..3ddd6f98 100644 --- a/crates/node/Cargo.toml +++ b/crates/node/Cargo.toml @@ -56,7 +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-indexer.workspace = true +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 1259e59a..9641c373 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -17,7 +17,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_indexer::ChainOrchestrator; +use rollup_node_chain_orchestrator::ChainOrchestrator; use rollup_node_manager::{ Consensus, NoopConsensus, RollupManagerHandle, RollupNodeManager, SystemContractConsensus, }; From 7d351850ea17b430132dcf7bbde640250eb543b2 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 17 Jul 2025 18:01:13 +0800 Subject: [PATCH 13/30] cleanup --- crates/chain-orchestrator/src/action.rs | 14 +++--- crates/chain-orchestrator/src/error.rs | 4 +- crates/chain-orchestrator/src/lib.rs | 60 +++++++++++-------------- 3 files changed, 35 insertions(+), 43 deletions(-) diff --git a/crates/chain-orchestrator/src/action.rs b/crates/chain-orchestrator/src/action.rs index 5e7bc4c6..822e34b3 100644 --- a/crates/chain-orchestrator/src/action.rs +++ b/crates/chain-orchestrator/src/action.rs @@ -6,12 +6,12 @@ use std::{ task::{Context, Poll}, }; -/// A future that resolves to a `Result`. +/// 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 { +/// A type that represents a future that is being executed by the chain orchestrator. +pub(super) enum ChainOrchestratorFuture { HandleReorg(PendingIndexerFuture), HandleFinalized(PendingIndexerFuture), HandleBatchCommit(PendingIndexerFuture), @@ -21,7 +21,7 @@ pub(super) enum IndexerFuture { HandleL2Block(PendingIndexerFuture), } -impl IndexerFuture { +impl ChainOrchestratorFuture { /// Polls the future to completion. pub(super) fn poll( &mut self, @@ -39,9 +39,9 @@ impl IndexerFuture { } } -// We implement the Debug trait for IndexerFuture to provide a human-readable representation of the -// enum variants. -impl fmt::Debug for IndexerFuture { +// 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"), diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index b08e8c81..6eb7e69e 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -3,7 +3,7 @@ use alloy_primitives::B256; use alloy_transport::TransportErrorKind; use scroll_db::{DatabaseError, L1MessageStart}; -/// A type that represents an error that occurred during indexing. +/// 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. @@ -38,7 +38,7 @@ pub enum ChainOrchestratorError { /// 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). + /// 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/lib.rs b/crates/chain-orchestrator/src/lib.rs index e9ced04e..8dca002b 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -1,4 +1,5 @@ -//! A library responsible for indexing data relevant to the L1. +//! 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}; @@ -32,7 +33,7 @@ use strum::IntoEnumIterator; use tokio::sync::Mutex; mod action; -use action::{IndexerFuture, PendingIndexerFuture}; +use action::{ChainOrchestratorFuture, PendingIndexerFuture}; mod event; pub use event::ChainOrchestratorEvent; @@ -65,7 +66,7 @@ pub struct ChainOrchestrator { /// A reference to the database used to persist the indexed data. database: Arc, /// A queue of pending futures. - pending_futures: VecDeque, + pending_futures: VecDeque, /// The block number of the L1 finalized block. l1_finalized_block_number: Arc, /// The block number of the L2 finalized block. @@ -122,7 +123,7 @@ impl< } } - /// Wraps a pending indexer future, metering the completion of it. + /// Wraps a pending chain orchestrator future, metering the completion of it. pub fn handle_metered( &mut self, item: IndexerItem, @@ -166,12 +167,12 @@ impl< .await }), ); - self.pending_futures.push_back(IndexerFuture::HandleL2Block(fut)); + self.pending_futures.push_back(ChainOrchestratorFuture::HandleL2Block(fut)); self.waker.wake(); } /// Handles a new block received from the network. - pub async fn handle_new_block( + async fn handle_new_block( chain: Arc>, l2_client: Arc

, optimistic_mode: Arc>, @@ -285,8 +286,7 @@ impl< } } - // If the current header block number is greater than the in-memory chain then we - // should search the in-memory chain. + // 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_header_tail.parent_hash) @@ -303,7 +303,7 @@ impl< signature, }); } - break Some(pos); + break pos; } // If the current header block number is less than the latest safe block number then @@ -345,7 +345,7 @@ impl< match reorg_index { // If this is a simple chain extension, we can just extend the in-memory chain and emit // a ChainExtended event. - Some(index) if index == current_chain_headers.len() - 1 => { + 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); @@ -358,7 +358,7 @@ impl< } // 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. - Some(position) => { + 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()); @@ -369,16 +369,6 @@ impl< new_blocks, peer_id, signature, ))) } - None => { - let mut new_chain = Chain::new(CHAIN_BUFFER_SIZE); - new_chain.extend(received_chain_headers); - *chain.lock().await = new_chain; - *optimistic_mode.lock().await = true; - - Ok(ChainOrchestratorEvent::OptimisticSync( - new_blocks.last().cloned().expect("new_blocks should not be empty"), - )) - } } } @@ -423,24 +413,26 @@ impl< }), ); - self.pending_futures.push_back(IndexerFuture::HandleDerivedBlock(fut)); + 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) => IndexerFuture::HandleReorg(self.handle_metered( - IndexerItem::L1Reorg, - Box::pin(Self::handle_l1_reorg( - self.database.clone(), - self.chain_spec.clone(), - block_number, - )), - )), + L1Notification::Reorg(block_number) => { + ChainOrchestratorFuture::HandleReorg(self.handle_metered( + IndexerItem::L1Reorg, + Box::pin(Self::handle_l1_reorg( + self.database.clone(), + self.chain_spec.clone(), + block_number, + )), + )) + } L1Notification::NewBlock(_) | L1Notification::Consensus(_) => return, L1Notification::Finalized(block_number) => { - IndexerFuture::HandleFinalized(self.handle_metered( + ChainOrchestratorFuture::HandleFinalized(self.handle_metered( IndexerItem::L1Finalization, Box::pin(Self::handle_finalized( self.database.clone(), @@ -451,13 +443,13 @@ impl< )) } L1Notification::BatchCommit(batch) => { - IndexerFuture::HandleBatchCommit(self.handle_metered( + ChainOrchestratorFuture::HandleBatchCommit(self.handle_metered( IndexerItem::BatchCommit, Box::pin(Self::handle_batch_commit(self.database.clone(), batch)), )) } L1Notification::L1Message { message, block_number, block_timestamp } => { - IndexerFuture::HandleL1Message(self.handle_metered( + ChainOrchestratorFuture::HandleL1Message(self.handle_metered( IndexerItem::L1Message, Box::pin(Self::handle_l1_message( self.database.clone(), @@ -469,7 +461,7 @@ impl< )) } L1Notification::BatchFinalization { hash, block_number, .. } => { - IndexerFuture::HandleBatchFinalization(self.handle_metered( + ChainOrchestratorFuture::HandleBatchFinalization(self.handle_metered( IndexerItem::BatchFinalization, Box::pin(Self::handle_batch_finalization( self.database.clone(), From fe75ed5f22e2e13cb6fa7650587b46c30ec38f77 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 13:25:21 +0800 Subject: [PATCH 14/30] remove expect for sequencer in rnm --- crates/manager/src/manager/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 2de33315..6f47920f 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -402,10 +402,9 @@ where } } if let L1Notification::NewBlock(block) = notification { - self.sequencer - .as_mut() - .expect("Sequencer must be initialized") - .handle_new_l1_block(block); + if let Some(s) = self.sequencer.as_mut() { + s.handle_new_l1_block(block) + } } self.chain.handle_l1_notification(notification) } From f3bab0db562f4248a6cab23328864f1c11e39925 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 14:08:55 +0800 Subject: [PATCH 15/30] sync test --- .github/workflows/test.yaml | 2 +- crates/node/src/args.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 4b16e9d9..3df6c180 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -76,7 +76,7 @@ jobs: --trusted-peers enode://29cee709c400533ae038a875b9ca975c8abef9eade956dcf3585e940acd5c0ae916968f514bd37d1278775aad1b7db30f7032a70202a87fd7365bd8de3c9f5fc@44.242.39.33:30303 \ --log.stdout.format log-fmt -vvv \ --l1.url "https://eth-sepolia.g.alchemy.com/v2/$ALCHEMY_KEY" --l1.cups 500 \ - --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 --engine.en-sync-trigger 10000000000 \ + --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 --chain.optimistic-sync-trigger 10000000000 \ --engine.sync-at-startup=false --engine.always-process-payload-attributes-on-canonical-head & - name: Get hash for block 50000 run: | diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 81bcdae4..eef6c0e0 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -342,7 +342,7 @@ impl Default for EngineDriverArgs { 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 = "engine.en-sync-trigger", default_value_t = constants::BLOCK_GAP_TRIGGER)] + #[arg(long = "chain.optimistic-sync-trigger", default_value_t = constants::BLOCK_GAP_TRIGGER)] pub optimistic_sync_trigger: u64, } From 56a0ff37ad5d6c7a72bddf1e360c62fa161d5f2d Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 15:13:46 +0800 Subject: [PATCH 16/30] sync test --- .github/workflows/test.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 3df6c180..9267b5b7 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -76,8 +76,8 @@ jobs: --trusted-peers enode://29cee709c400533ae038a875b9ca975c8abef9eade956dcf3585e940acd5c0ae916968f514bd37d1278775aad1b7db30f7032a70202a87fd7365bd8de3c9f5fc@44.242.39.33:30303 \ --log.stdout.format log-fmt -vvv \ --l1.url "https://eth-sepolia.g.alchemy.com/v2/$ALCHEMY_KEY" --l1.cups 500 \ - --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 --chain.optimistic-sync-trigger 10000000000 \ - --engine.sync-at-startup=false --engine.always-process-payload-attributes-on-canonical-head & + --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 --network.bridge false \ + --engine.sync-at-startup=false & - name: Get hash for block 50000 run: | echo "Waiting for block 50000..." From 1320f14630e0b6d7518de0c4f8d076cc7802e101 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 16:02:53 +0800 Subject: [PATCH 17/30] sync test --- crates/node/src/args.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index eef6c0e0..61ca2bb8 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -357,10 +357,10 @@ impl Default for ChainOrchestratorArgs { 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", default_value_t = true, action = clap::ArgAction::Set)] 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", default_value_t = true, action = clap::ArgAction::Set)] pub enable_scroll_wire: bool, } From 0fa87fe7d840886d523d348c5c5acc90bd0475a4 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 21:38:25 +0800 Subject: [PATCH 18/30] add error handling for missing paylaod id --- crates/engine/src/driver.rs | 7 +++++-- crates/engine/src/error.rs | 4 ++++ crates/engine/src/future/mod.rs | 12 +++++++----- crates/watcher/src/lib.rs | 2 ++ 4 files changed, 18 insertions(+), 7 deletions(-) diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs index 9295dcba..829581a0 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -2,7 +2,7 @@ use super::{future::EngineFuture, ForkchoiceState}; use crate::{ future::{BuildNewPayloadFuture, EngineDriverFutureResult}, metrics::EngineDriverMetrics, - EngineDriverEvent, + EngineDriverError, EngineDriverEvent, }; use alloy_provider::Provider; @@ -223,7 +223,10 @@ 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::MissingPayloadId(attributes) = err { + self.l1_payload_attributes.push_front(attributes); + } } } } diff --git a/crates/engine/src/error.rs b/crates/engine/src/error.rs index 678e481e..606a0858 100644 --- a/crates/engine/src/error.rs +++ b/crates/engine/src/error.rs @@ -1,4 +1,5 @@ use alloy_rpc_types_engine::PayloadError; +use rollup_node_primitives::ScrollPayloadAttributesWithBatchInfo; use scroll_alloy_provider::ScrollEngineApiError; /// The error type for the engine API. @@ -19,4 +20,7 @@ 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), } diff --git a/crates/engine/src/future/mod.rs b/crates/engine/src/future/mod.rs index 630d4815..5e8950e8 100644 --- a/crates/engine/src/future/mod.rs +++ b/crates/engine/src/future/mod.rs @@ -251,23 +251,23 @@ where #[instrument(skip_all, level = "trace", fields( fcs = ?fcs, - payload_attributes = ?payload_attributes + payload_attributes = ?payload_attributes_with_batch_info ) )] async fn handle_payload_attributes( client: Arc, provider: P, fcs: ForkchoiceState, - payload_attributes: ScrollPayloadAttributesWithBatchInfo, + payload_attributes_with_batch_info: ScrollPayloadAttributesWithBatchInfo, ) -> Result where EC: ScrollEngineApi + Unpin + Send + Sync + 'static, P: Provider + Unpin + Send + Sync + 'static, { - tracing::trace!(target: "scroll::engine::future", ?fcs, ?payload_attributes, "handling payload attributes"); + tracing::trace!(target: "scroll::engine::future", ?fcs, ?payload_attributes_with_batch_info, "handling payload attributes"); let ScrollPayloadAttributesWithBatchInfo { mut payload_attributes, batch_info } = - payload_attributes; + payload_attributes_with_batch_info.clone(); let maybe_execution_payload = provider .get_block((fcs.safe_block_info().number + 1).into()) @@ -307,7 +307,9 @@ where // retrieve the execution payload let execution_payload = get_payload( client.clone(), - fc_updated.payload_id.expect("payload attributes has been set"), + fc_updated + .payload_id + .ok_or(EngineDriverError::MissingPayloadId(payload_attributes_with_batch_info))?, ) .await?; // issue the execution payload to the EL diff --git a/crates/watcher/src/lib.rs b/crates/watcher/src/lib.rs index 0fcffc72..09f027ab 100644 --- a/crates/watcher/src/lib.rs +++ b/crates/watcher/src/lib.rs @@ -156,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| { From cecbd034ad88fe9f38d1e130de542f2588dd991a Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 21 Jul 2025 23:20:24 +0800 Subject: [PATCH 19/30] remove networking for L1 consolidation sync test --- .github/workflows/test.yaml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 9267b5b7..950b05c5 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -73,10 +73,9 @@ jobs: cargo run --bin rollup-node --release -- \ node --chain scroll-sepolia --datadir=./l2reth --disable-discovery \ --http --http.addr=0.0.0.0 --http.port=8545 --http.api eth \ - --trusted-peers enode://29cee709c400533ae038a875b9ca975c8abef9eade956dcf3585e940acd5c0ae916968f514bd37d1278775aad1b7db30f7032a70202a87fd7365bd8de3c9f5fc@44.242.39.33:30303 \ --log.stdout.format log-fmt -vvv \ --l1.url "https://eth-sepolia.g.alchemy.com/v2/$ALCHEMY_KEY" --l1.cups 500 \ - --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 --network.bridge false \ + --beacon.url https://eth-beacon-chain.drpc.org/rest/ --beacon.cups 100 \ --engine.sync-at-startup=false & - name: Get hash for block 50000 run: | From 7df1a59b34e68590977479be5eba670db2957877 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 30 Jul 2025 19:05:26 +0800 Subject: [PATCH 20/30] improve test coverage and fix bugs --- README.md | 4 +- crates/chain-orchestrator/src/action.rs | 16 +- crates/chain-orchestrator/src/error.rs | 2 +- crates/chain-orchestrator/src/lib.rs | 245 +++++++------- crates/chain-orchestrator/src/metrics.rs | 12 +- crates/engine/src/driver.rs | 17 +- crates/engine/src/error.rs | 12 +- crates/engine/src/future/mod.rs | 13 +- crates/manager/src/manager/command.rs | 3 + crates/manager/src/manager/event.rs | 14 +- crates/manager/src/manager/handle.rs | 6 + crates/manager/src/manager/mod.rs | 60 +--- crates/node/src/args.rs | 9 +- crates/node/src/constants.rs | 3 + crates/node/src/test_utils.rs | 10 +- crates/node/tests/e2e.rs | 180 +++++++--- crates/node/tests/sync.rs | 411 ++++++++++++++++++++--- 17 files changed, 713 insertions(+), 304 deletions(-) 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/chain-orchestrator/src/action.rs b/crates/chain-orchestrator/src/action.rs index 822e34b3..a432f141 100644 --- a/crates/chain-orchestrator/src/action.rs +++ b/crates/chain-orchestrator/src/action.rs @@ -7,18 +7,18 @@ use std::{ }; /// A future that resolves to a `Result`. -pub(super) type PendingIndexerFuture = +pub(super) type PendingChainOrchestratorFuture = Pin> + Send>>; /// A type that represents a future that is being executed by the chain orchestrator. pub(super) enum ChainOrchestratorFuture { - HandleReorg(PendingIndexerFuture), - HandleFinalized(PendingIndexerFuture), - HandleBatchCommit(PendingIndexerFuture), - HandleBatchFinalization(PendingIndexerFuture), - HandleL1Message(PendingIndexerFuture), - HandleDerivedBlock(PendingIndexerFuture), - HandleL2Block(PendingIndexerFuture), + HandleReorg(PendingChainOrchestratorFuture), + HandleFinalized(PendingChainOrchestratorFuture), + HandleBatchCommit(PendingChainOrchestratorFuture), + HandleBatchFinalization(PendingChainOrchestratorFuture), + HandleL1Message(PendingChainOrchestratorFuture), + HandleDerivedBlock(PendingChainOrchestratorFuture), + HandleL2Block(PendingChainOrchestratorFuture), } impl ChainOrchestratorFuture { diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index 6eb7e69e..2b88a00c 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -7,7 +7,7 @@ use scroll_db::{DatabaseError, L1MessageStart}; #[derive(Debug, thiserror::Error)] pub enum ChainOrchestratorError { /// An error occurred while interacting with the database. - #[error("indexing failed due to database error: {0}")] + #[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}")] diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index b524b1db..b23b07a7 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -33,7 +33,7 @@ use strum::IntoEnumIterator; use tokio::sync::Mutex; mod action; -use action::{ChainOrchestratorFuture, PendingIndexerFuture}; +use action::{ChainOrchestratorFuture, PendingChainOrchestratorFuture}; mod event; pub use event::ChainOrchestratorEvent; @@ -42,15 +42,12 @@ mod error; pub use error::ChainOrchestratorError; mod metrics; -pub use metrics::{IndexerItem, IndexerMetrics}; +pub use metrics::{ChainOrchestratorItem, ChainOrchestratorMetrics}; /// The mask used to mask the L1 message queue hash. const L1_MESSAGE_QUEUE_HASH_MASK: B256 = b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); -/// The number of block headers we keep in memory. -const CHAIN_BUFFER_SIZE: usize = 2000; - type Chain = BoundedVec

; /// The [`ChainOrchestrator`] is responsible for orchestrating the progression of the L2 chain @@ -71,15 +68,17 @@ pub struct ChainOrchestrator { l1_finalized_block_number: Arc, /// The block number of the L2 finalized block. l2_finalized_block_number: Arc, - /// The chain specification for the indexer. + /// The chain specification for the chain orchestrator. chain_spec: Arc, - /// The metrics for the indexer. - metrics: HashMap, + /// 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. @@ -92,15 +91,16 @@ impl< P: Provider + 'static, > ChainOrchestrator { - /// Creates a new indexer with the given [`Database`]. + /// 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).await; + 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), @@ -110,14 +110,15 @@ impl< l1_finalized_block_number: Arc::new(AtomicU64::new(0)), l2_finalized_block_number: Arc::new(AtomicU64::new(0)), chain_spec, - metrics: IndexerItem::iter() + metrics: ChainOrchestratorItem::iter() .map(|i| { let label = i.as_str(); - (i, IndexerMetrics::new_with_labels(&[("item", label)])) + (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(), } @@ -126,13 +127,13 @@ impl< /// Wraps a pending chain orchestrator future, metering the completion of it. pub fn handle_metered( &mut self, - item: IndexerItem, - indexer_fut: PendingIndexerFuture, - ) -> PendingIndexerFuture { + 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 = indexer_fut.await; + let res = chain_orchestrator_fut.await; metric.task_duration.record(now.elapsed().as_secs_f64()); res }); @@ -146,26 +147,19 @@ impl< /// Handles a new block received from a peer. pub fn handle_block_from_peer(&mut self, block_with_peer: NewBlockWithPeer) { - let chain = self.chain.clone(); - let l2_client = self.l2_client.clone(); - let optimistic_mode = self.optimistic_mode.clone(); - let optimistic_sync_threshold = self.optimistic_sync_threshold; - let network_client = self.network_client.clone(); - let database = self.database.clone(); + 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( - IndexerItem::NewBlock, - Box::pin(async move { - Self::handle_new_block( - chain, - l2_client, - optimistic_mode, - optimistic_sync_threshold, - network_client, - database, - block_with_peer, - ) - .await - }), + ChainOrchestratorItem::NewBlock, + Box::pin(async move { Self::handle_new_block(ctx, block_with_peer).await }), ); self.pending_futures.push_back(ChainOrchestratorFuture::HandleL2Block(fut)); self.waker.wake(); @@ -173,14 +167,18 @@ impl< /// Handles a new block received from the network. async fn handle_new_block( - chain: Arc>, - l2_client: Arc

, - optimistic_mode: Arc>, - optimistic_sync_threshold: u64, - network_client: Arc, - database: Arc, + 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; @@ -190,12 +188,14 @@ impl< // 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 - max_block_number) >= optimistic_sync_threshold { - // fetch the latest `CHAIN_BUFFER_SIZE` headers from the network for the + 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); + let mut optimistic_headers = Chain::new(chain_buffer_size); optimistic_headers.push_front(received_block.header.clone()); - while optimistic_headers.len() < CHAIN_BUFFER_SIZE && + 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"); @@ -237,7 +237,6 @@ impl< // 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()]; - let mut received_header_tail = received_block.header.clone(); // We should never have a re-org that is deeper than the current safe head. let (latest_safe_block, _) = @@ -260,7 +259,7 @@ impl< // 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_header_tail.number < + if received_chain_headers.last().unwrap().number < current_chain_headers.front().expect("chain can not be empty").number { for _ in 0..BATCH_FETCH_SIZE { @@ -277,7 +276,6 @@ impl< { let header = block.into_consensus_header(); current_chain_headers.push_front(header.clone()); - received_header_tail = header; } else { return Err(ChainOrchestratorError::MissingBlockHeader { hash: current_chain_headers.front().unwrap().parent_hash, @@ -289,13 +287,13 @@ impl< // 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_header_tail.parent_hash) + .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_header_tail.timestamp + current_chain_headers.get(pos + 1).unwrap().timestamp > + received_chain_headers.last().unwrap().timestamp { return Ok(ChainOrchestratorEvent::OldForkReceived { headers: received_chain_headers, @@ -308,21 +306,22 @@ impl< // If the current header block number is less than the latest safe block number then // we should error. - if received_header_tail.number <= latest_safe_block.number { + if received_chain_headers.last().unwrap().number <= latest_safe_block.number { return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_header_tail.number - 1), "fetching block"); + 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_header_tail.parent_hash)) + .get_header(BlockHashOrNumber::Hash( + received_chain_headers.last().unwrap().parent_hash, + )) .await? .into_data() { received_chain_headers.push(header.clone()); - received_header_tail = header; } else { return Err(ChainOrchestratorError::MissingBlockHeader { - hash: received_header_tail.parent_hash, + hash: received_chain_headers.last().unwrap().parent_hash, }); } }; @@ -348,7 +347,7 @@ impl< 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); + let mut new_chain = Chain::new(chain_buffer_size); new_chain.extend(current_chain_headers); *chain.lock().await = new_chain; @@ -360,7 +359,7 @@ impl< // 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); + 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; @@ -383,9 +382,10 @@ impl< 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( - IndexerItem::InsertL2Block, + 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 @@ -393,8 +393,14 @@ impl< 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) - .await?; + consolidate_chain( + database.clone(), + block_info.clone(), + chain, + l2_client, + chain_buffer_size, + ) + .await?; *optimistic_mode.lock().await = false; true } else { @@ -420,7 +426,7 @@ impl< let fut = match event { L1Notification::Reorg(block_number) => { ChainOrchestratorFuture::HandleReorg(self.handle_metered( - IndexerItem::L1Reorg, + ChainOrchestratorItem::L1Reorg, Box::pin(Self::handle_l1_reorg( self.database.clone(), self.chain_spec.clone(), @@ -432,7 +438,7 @@ impl< L1Notification::NewBlock(_) | L1Notification::Consensus(_) => return, L1Notification::Finalized(block_number) => { ChainOrchestratorFuture::HandleFinalized(self.handle_metered( - IndexerItem::L1Finalization, + ChainOrchestratorItem::L1Finalization, Box::pin(Self::handle_finalized( self.database.clone(), block_number, @@ -443,13 +449,13 @@ impl< } L1Notification::BatchCommit(batch) => { ChainOrchestratorFuture::HandleBatchCommit(self.handle_metered( - IndexerItem::BatchCommit, + ChainOrchestratorItem::BatchCommit, Box::pin(Self::handle_batch_commit(self.database.clone(), batch)), )) } L1Notification::L1Message { message, block_number, block_timestamp } => { ChainOrchestratorFuture::HandleL1Message(self.handle_metered( - IndexerItem::L1Message, + ChainOrchestratorItem::L1Message, Box::pin(Self::handle_l1_message( self.database.clone(), self.chain_spec.clone(), @@ -461,7 +467,7 @@ impl< } L1Notification::BatchFinalization { hash, block_number, .. } => { ChainOrchestratorFuture::HandleBatchFinalization(self.handle_metered( - IndexerItem::BatchFinalization, + ChainOrchestratorItem::BatchFinalization, Box::pin(Self::handle_batch_finalization( self.database.clone(), hash, @@ -511,8 +517,8 @@ impl< }) } - /// Handles a finalized event by updating the indexer L1 finalized block and returning the new - /// finalized L2 chain block. + /// 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, @@ -532,7 +538,7 @@ impl< None }; - // update the indexer l1 block number. + // update the chain orchestrator l1 block number. l1_block_number.store(block_number, Ordering::Relaxed); Ok(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_block)) @@ -654,10 +660,11 @@ impl< 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); + 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) @@ -671,7 +678,7 @@ async fn init_chain_from_db + 'static>( blocks.reverse(); blocks }; - let mut chain: Chain = Chain::new(CHAIN_BUFFER_SIZE); + let mut chain: Chain = Chain::new(chain_buffer_size); chain.extend(blocks); chain } @@ -703,16 +710,27 @@ impl< } } +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, - _chain: Arc>, + current_chain: Arc>, l2_client: P, + chain_buffer_size: usize, ) -> Result<(), ChainOrchestratorError> { // Take the current in memory chain. - let chain = std::mem::take(&mut *_chain.lock().await); + 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| { @@ -727,7 +745,7 @@ async fn consolidate_chain + 'static>( // part of the in-memory chain. if hca_index.is_none() { // If we do not have a common ancestor, we return an error. - *_chain.lock().await = chain; + *current_chain.lock().await = chain; return Err(ChainOrchestratorError::ChainInconsistency); } @@ -749,7 +767,8 @@ async fn consolidate_chain + 'static>( 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. - *_chain.lock().await = init_chain_from_db(&database, &l2_client).await; + *current_chain.lock().await = + init_chain_from_db(&database, &l2_client, chain_buffer_size).await; return Err(ChainOrchestratorError::ChainInconsistency); } @@ -778,7 +797,8 @@ async fn consolidate_chain + 'static>( // 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 { - *_chain.lock().await = init_chain_from_db(&database, &l2_client).await; + *current_chain.lock().await = + init_chain_from_db(&database, &l2_client, chain_buffer_size).await; return Err(ChainOrchestratorError::ChainInconsistency); } } @@ -789,7 +809,7 @@ async fn consolidate_chain + 'static>( validate_l1_messages(&blocks_to_consolidate, database.clone()).await?; // Set the chain which has now been consolidated. - *_chain.lock().await = chain; + *current_chain.lock().await = chain; Ok(()) } @@ -887,6 +907,7 @@ mod test { 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. @@ -916,31 +937,6 @@ mod test { } } - // impl TestScrollFullBlockClient { - // /// Insert a header and body into the client maps. - // fn insert(&self, header: SealedHeader, body: ScrollBody) { - // let hash = header.hash(); - // self.headers.lock().insert(hash, header.unseal()); - // self.bodies.lock().insert(hash, body); - // } - - // /// Set the soft response limit. - // const fn set_soft_limit(&mut self, limit: usize) { - // self.soft_limit = limit; - // } - - // /// Get the block with the highest block number. - // fn highest_block(&self) -> Option> { - // self.headers.lock().iter().max_by_key(|(_, header)| header.number).and_then( - // |(hash, header)| { - // self.bodies.lock().get(hash).map(|body| { - // SealedBlock::from_parts_unchecked(header.clone(), body.clone(), *hash) - // }) - // }, - // ) - // } - // } - impl DownloadClient for TestScrollFullBlockClient { /// Reports a bad message from a specific peer. fn report_bad_message(&self, _peer_id: PeerId) {} @@ -1051,7 +1047,7 @@ mod test { type Block = ScrollBlock; } - async fn setup_test_indexer() -> ( + async fn setup_test_chain_orchestrator() -> ( ChainOrchestrator>, Arc, ) { @@ -1071,6 +1067,7 @@ mod test { TestScrollFullBlockClient::default(), provider, TEST_OPTIMISTIC_SYNC_THRESHOLD, + TEST_CHAIN_BUFFER_SIZE, ) .await, db, @@ -1080,7 +1077,7 @@ mod test { #[tokio::test] async fn test_handle_commit_batch() { // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().await; + let (mut indexer, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1109,7 +1106,7 @@ mod test { #[tokio::test] async fn test_handle_batch_commit_with_revert() { // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().await; + let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1134,8 +1131,8 @@ mod test { }; // Index first batch - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); - let event = indexer.next().await.unwrap().unwrap(); + 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); @@ -1145,8 +1142,8 @@ mod test { } // Index second batch - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_2.clone())); - let event = indexer.next().await.unwrap().unwrap(); + 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); @@ -1156,8 +1153,8 @@ mod test { } // Index third batch - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_3.clone())); - let event = indexer.next().await.unwrap().unwrap(); + 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); @@ -1183,14 +1180,14 @@ mod test { l1_messages: vec![], }; - indexer.consolidate_l2_blocks(vec![block_1.clone()], Some(batch_1_info)); - indexer.next().await.unwrap().unwrap(); + chain_orchestrator.consolidate_l2_blocks(vec![block_1.clone()], Some(batch_1_info)); + chain_orchestrator.next().await.unwrap().unwrap(); - indexer.consolidate_l2_blocks(vec![block_2.clone()], Some(batch_2_info)); - indexer.next().await.unwrap().unwrap(); + chain_orchestrator.consolidate_l2_blocks(vec![block_2.clone()], Some(batch_2_info)); + chain_orchestrator.next().await.unwrap().unwrap(); - indexer.consolidate_l2_blocks(vec![block_3.clone()], Some(batch_2_info)); - indexer.next().await.unwrap().unwrap(); + chain_orchestrator.consolidate_l2_blocks(vec![block_3.clone()], Some(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 @@ -1200,8 +1197,8 @@ mod test { ..Arbitrary::arbitrary(&mut u).unwrap() }; - indexer.handle_l1_notification(L1Notification::BatchCommit(new_batch_2.clone())); - let event = indexer.next().await.unwrap().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 { @@ -1230,7 +1227,7 @@ mod test { #[tokio::test] async fn test_handle_l1_message() { // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().await; + let (mut indexer, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1260,7 +1257,7 @@ mod test { #[tokio::test] async fn test_l1_message_hash_queue() { // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().await; + let (mut indexer, db) = setup_test_chain_orchestrator().await; // insert the previous L1 message in database. indexer.handle_l1_notification(L1Notification::L1Message { @@ -1299,7 +1296,7 @@ mod test { #[tokio::test] async fn test_handle_reorg() { // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().await; + let (mut indexer, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1398,7 +1395,7 @@ mod test { #[tokio::test] async fn test_handle_reorg_executed_l1_messages() { // Instantiate indexer and db - let (mut indexer, _database) = setup_test_indexer().await; + let (mut indexer, _database) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 8192]; diff --git a/crates/chain-orchestrator/src/metrics.rs b/crates/chain-orchestrator/src/metrics.rs index 0defdfeb..41eb9c26 100644 --- a/crates/chain-orchestrator/src/metrics.rs +++ b/crates/chain-orchestrator/src/metrics.rs @@ -2,9 +2,9 @@ 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, /// L2 block. @@ -21,8 +21,8 @@ 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::NewBlock => "new_block", @@ -39,7 +39,7 @@ impl IndexerItem { /// 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/engine/src/driver.rs b/crates/engine/src/driver.rs index e7486888..3f920e7c 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -224,8 +224,11 @@ where } Err(err) => { tracing::error!(target: "scroll::engine", ?err, "failed to consolidate block derived from L1"); - if let EngineDriverError::MissingPayloadId(attributes) = err { + 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(); } } } @@ -248,8 +251,9 @@ 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); } } } @@ -338,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. 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/future/mod.rs b/crates/engine/src/future/mod.rs index 5e8950e8..bea06980 100644 --- a/crates/engine/src/future/mod.rs +++ b/crates/engine/src/future/mod.rs @@ -307,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 @@ -344,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; @@ -352,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/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 ebe06642..6e18c676 100644 --- a/crates/manager/src/manager/event.rs +++ b/crates/manager/src/manager/event.rs @@ -1,8 +1,8 @@ use alloy_primitives::B256; use reth_scroll_primitives::ScrollBlock; use rollup_node_chain_orchestrator::ChainOrchestratorEvent; -use rollup_node_primitives::{BatchInfo, ChainImport, L2BlockInfoWithL1Messages}; use rollup_node_signer::SignerEvent; +use rollup_node_watcher::L1Notification; use scroll_db::L1MessageStart; use scroll_engine::ConsolidationOutcome; use scroll_network::NewBlockWithPeer; @@ -18,8 +18,6 @@ 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. @@ -38,12 +36,6 @@ pub enum RollupManagerEvent { /// The L1 message start index or hash. start: L1MessageStart, }, - /// An optimistic sync has been triggered by the chain orchestrator. - OptimisticSyncTriggered(ScrollBlock), - /// A chain extension has been triggered by the chain orchestrator. - ChainExtensionTriggered(ChainImport), - /// An L2 chain has been committed. - L2ChainCommitted(L2BlockInfoWithL1Messages, Option, bool), - /// The L1 watcher has synced to the tip of the L1 chain. - L1Synced, + /// 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 fbbeb6a8..1a8115be 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -230,8 +230,13 @@ where } /// Handles an indexer event. - fn handle_indexer_event(&mut self, event: ChainOrchestratorEvent) { - 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 { ChainOrchestratorEvent::BatchCommitIndexed { batch_info, @@ -289,33 +294,13 @@ where event_sender.notify(RollupManagerEvent::Reorg(l1_block_number)); } } - - ChainOrchestratorEvent::L1MessageCommitted(index) => { - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1MessageIndexed(index)); - } - } - ChainOrchestratorEvent::OldForkReceived { ref headers, ref peer_id, signature: _ } => { - trace!(target: "scroll::node::manager", ?headers, ?peer_id, "Received old fork from peer"); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::ChainOrchestratorEvent(event)); - } - } 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"); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender - .notify(RollupManagerEvent::ChainExtensionTriggered(chain_import.clone())); - } - // 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"); - // if let Some(event_sender) = self.event_sender.as_ref() { - // event_sender.notify(RollupManagerEvent::ChainOrchestratorEvent(event)); - // } // Issue the new chain to the engine driver for processing. self.engine.handle_chain_import(chain_import) @@ -323,28 +308,12 @@ where ChainOrchestratorEvent::OptimisticSync(block) => { let block_info: BlockInfo = (&block).into(); trace!(target: "scroll::node::manager", ?block_info, "Received optimistic sync from peer"); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::OptimisticSyncTriggered(block)); - } // Issue the new block info to the engine driver for processing. self.engine.handle_optimistic_sync(block_info) } - ChainOrchestratorEvent::L2ChainCommitted(block_infos, batch_into, consolidated) => { - trace!(target: "scroll::node::manager", ?block_infos, ?batch_into, ?consolidated, "Received L2 chain committed event"); - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L2ChainCommitted( - block_infos, - batch_into, - consolidated, - )); - } - } event => { 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)); - } } } } @@ -402,6 +371,10 @@ where /// 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) => { @@ -410,9 +383,6 @@ where } } L1Notification::Synced => { - if let Some(event_sender) = self.event_sender.as_ref() { - event_sender.notify(RollupManagerEvent::L1Synced); - } self.chain.handle_l1_notification(L1Notification::Synced); } _ => self.chain.handle_l1_notification(notification), @@ -483,6 +453,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); + } } } @@ -514,7 +488,7 @@ where // Drain all Indexer events. 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) => { match &err { ChainOrchestratorError::L1MessageMismatch { expected, actual } => { @@ -544,7 +518,7 @@ where error!( target: "scroll::node::manager", ?err, - "Error occurred at indexer level" + "Error occurred in the chain orchestrator" ); } } diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 46b5bbbe..c358865a 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -287,6 +287,7 @@ impl ScrollRollupNodeConfig { block_client, l2_provider, self.chain_orchestrator_args.optimistic_sync_trigger, + self.chain_orchestrator_args.chain_buffer_size, ) .await; @@ -338,11 +339,17 @@ pub struct ChainOrchestratorArgs { /// 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 } + Self { + optimistic_sync_trigger: constants::BLOCK_GAP_TRIGGER, + chain_buffer_size: constants::CHAIN_BUFFER_SIZE, + } } } diff --git a/crates/node/src/constants.rs b/crates/node/src/constants.rs index e9ae30ca..20b1f45d 100644 --- a/crates/node/src/constants.rs +++ b/crates/node/src/constants.rs @@ -21,3 +21,6 @@ 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 = 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; diff --git a/crates/node/src/test_utils.rs b/crates/node/src/test_utils.rs index d5d4242f..b43864c0 100644 --- a/crates/node/src/test_utils.rs +++ b/crates/node/src/test_utils.rs @@ -143,7 +143,10 @@ pub fn default_test_scroll_rollup_node_config() -> ScrollRollupNodeConfig { database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, l1_provider_args: L1ProviderArgs::default(), engine_driver_args: EngineDriverArgs { sync_at_startup: true }, - chain_orchestrator_args: ChainOrchestratorArgs { optimistic_sync_trigger: 100 }, + 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, @@ -161,7 +164,10 @@ pub fn default_sequencer_test_scroll_rollup_node_config() -> ScrollRollupNodeCon database_args: DatabaseArgs { path: Some(PathBuf::from("sqlite::memory:")) }, l1_provider_args: L1ProviderArgs::default(), engine_driver_args: EngineDriverArgs { sync_at_startup: true }, - chain_orchestrator_args: ChainOrchestratorArgs { optimistic_sync_trigger: 100 }, + 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 b68a54b5..188013aa 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -16,6 +16,7 @@ use rollup_node::{ BeaconProviderArgs, ChainOrchestratorArgs, DatabaseArgs, EngineDriverArgs, L1ProviderArgs, NetworkArgs as ScrollNetworkArgs, ScrollRollupNodeConfig, SequencerArgs, }; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent, RollupManagerHandle}; use rollup_node_primitives::BatchCommitData; use rollup_node_providers::BlobSource; @@ -76,19 +77,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(()) } @@ -141,33 +166,64 @@ 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 - if let Some(RollupManagerEvent::ChainExtensionTriggered(_)) = follower_events.next().await { - } else { - panic!("Failed to receive chain extension event from rollup 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 - if let Some(RollupManagerEvent::BlockImported(block)) = follower_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - } else { - panic!("Failed to receive block from rollup node"); - } + wait_n_events( + &mut follower_events, + |e| { + if let RollupManagerEvent::BlockImported(block) = e { + assert_eq!(block.body.transactions.len(), 1); + true + } else { + false + } + }, + 1, + ) + .await; } #[allow(clippy::large_stack_frames)] @@ -250,26 +306,49 @@ async fn can_forward_tx_to_sequencer() { let _ = follower_events.next().await; // assert that the follower node has received the block from the peer - if let Some(RollupManagerEvent::NewBlockReceived(block_with_peer)) = - 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 - if let Some(RollupManagerEvent::ChainExtensionTriggered(_)) = follower_events.next().await { - } else { - panic!("Failed to receive chain extension event from rollup 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 - if let Some(RollupManagerEvent::BlockImported(block)) = follower_events.next().await { - assert_eq!(block.body.transactions.len(), 1); - } else { - panic!("Failed to receive block from rollup node"); - } + wait_n_events( + &mut follower_events, + |e| { + if let RollupManagerEvent::BlockImported(block) = e { + assert_eq!(block.body.transactions.len(), 1); + true + } else { + false + } + }, + 1, + ) + .await; } #[allow(clippy::large_stack_frames)] @@ -823,7 +902,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 } diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index fe07c916..6b40de03 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -4,18 +4,20 @@ 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_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, ChainOrchestratorArgs, DatabaseArgs, EngineDriverArgs, L1ProviderArgs, - NetworkArgs, ScrollRollupNode, ScrollRollupNodeConfig, SequencerArgs, + NetworkArgs, ScrollRollupNodeConfig, SequencerArgs, }; +use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_manager::RollupManagerEvent; +use rollup_node_primitives::BlockInfo; use rollup_node_providers::BlobSource; use rollup_node_sequencer::L1MessageInclusionMode; use rollup_node_watcher::L1Notification; @@ -42,7 +44,10 @@ async fn test_should_consolidate_to_block_15k() -> eyre::Result<()> { sequencer_url: None, }, database_args: DatabaseArgs::default(), - chain_orchestrator_args: ChainOrchestratorArgs { optimistic_sync_trigger: 100 }, + 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, @@ -85,7 +90,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(); @@ -97,15 +102,17 @@ 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 optimistic_sync_trigger = node_config.chain_orchestrator_args.optimistic_sync_trigger + 1; wait_n_events( - &synced, + &mut synced_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), optimistic_sync_trigger, ) @@ -117,8 +124,19 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { synced.network.next_session_established().await; // Assert that the unsynced node triggers optimistic sync. - wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::OptimisticSyncTriggered(_)), 1) - .await; + 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()); @@ -135,13 +153,26 @@ async fn test_should_trigger_pipeline_sync_for_execution_node() { } // Assert that the unsynced node triggers a chain extension on the optimistic chain. - wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::ChainExtensionTriggered(_)), 1) - .await; + 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() { +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 { @@ -178,11 +209,14 @@ async fn test_should_consolidate_after_optimistic_sync() { 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 unsynced = nodes.pop().unwrap(); - let unsynced_l1_watcher_tx = unsynced.inner.add_ons_handle.l1_watcher_tx.clone().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; @@ -209,12 +243,31 @@ async fn test_should_consolidate_after_optimistic_sync() { })) .await .unwrap(); - wait_n_events(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 1) - .await; + 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( - &sequencer, + &mut sequencer_events, |e: RollupManagerEvent| matches!(e, RollupManagerEvent::BlockSequenced(_)), 1, ) @@ -222,23 +275,34 @@ async fn test_should_consolidate_after_optimistic_sync() { } // Connect the nodes together. - sequencer.network.add_peer(unsynced.network.record()).await; - unsynced.network.next_session_established().await; + 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(&unsynced, |e| matches!(e, RollupManagerEvent::OptimisticSyncTriggered(_)), 1) - .await; + 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() { - unsynced_l1_watcher_tx + follower_l1_watcher_tx .send(Arc::new(L1Notification::L1Message { message: l1_message.clone(), block_number: i as u64, @@ -247,22 +311,32 @@ async fn test_should_consolidate_after_optimistic_sync() { .await .unwrap(); wait_n_events( - &unsynced, - |e: RollupManagerEvent| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), + &mut follower_events, + |e: RollupManagerEvent| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + rollup_node_chain_orchestrator::ChainOrchestratorEvent::L1MessageCommitted( + _ + ) + ) + ) + }, 1, ) .await; } - println!("im here"); - // Send a notification to the unsynced node that the L1 watcher is synced. - unsynced_l1_watcher_tx.send(Arc::new(L1Notification::Synced)).await.unwrap(); + 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(&unsynced, |e| matches!(e, RollupManagerEvent::L1Synced), 1).await; - - println!("im here 2"); + 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; @@ -270,14 +344,23 @@ async fn test_should_consolidate_after_optimistic_sync() { // build a new block on the sequencer node to trigger consolidation on the unsynced node. sequencer_handle.build_block().await; - println!("im here"); - // Assert that the unsynced node consolidates the chain. - wait_n_events(&unsynced, |e| matches!(e, RollupManagerEvent::L2ChainCommitted(_, _, true)), 1) - .await; + 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. - unsynced_l1_watcher_tx + sequencer_l1_watcher_tx .send(Arc::new(L1Notification::L1Message { message: TxL1Message { queue_index: 200, @@ -287,23 +370,55 @@ async fn test_should_consolidate_after_optimistic_sync() { value: U256::from(1), input: Default::default(), }, - block_number: 201, + 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(&unsynced, |e| matches!(e, RollupManagerEvent::NewBlockReceived(_)), 1).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. - tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + 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() { +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 { @@ -340,11 +455,13 @@ async fn test_consolidation() { 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; @@ -368,7 +485,19 @@ async fn test_consolidation() { })) .await .unwrap(); - wait_n_events(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 1).await; + 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 @@ -379,7 +508,19 @@ async fn test_consolidation() { })) .await .unwrap(); - wait_n_events(&follower, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 1).await; + 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(); @@ -389,8 +530,19 @@ async fn test_consolidation() { sequencer_handle.build_block().await; // Assert that the unsynced node consolidates the chain. - wait_n_events(&follower, |e| matches!(e, RollupManagerEvent::L2ChainCommitted(_, _, true)), 1) - .await; + 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 @@ -408,26 +560,193 @@ async fn test_consolidation() { })) .await .unwrap(); - wait_n_events(&sequencer, |e| matches!(e, RollupManagerEvent::L1MessageIndexed(_)), 1).await; + 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( - &follower, + &mut follower_events, |e| matches!(e, RollupManagerEvent::L1MessageMissingInDatabase { start: _ }), 1, ) .await; + + Ok(()) +} + +#[tokio::test] +async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { + reth_tracing::init_test_tracing(); + let node_config = default_test_scroll_rollup_node_config(); + let sequencer_node_config = ScrollRollupNodeConfig { + test: true, + network_args: NetworkArgs { + enable_eth_scroll_wire_bridge: true, + 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(), + }; + + // 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 sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); + + let (mut nodes, _tasks, _) = + setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); + let mut follower = nodes.pop().unwrap(); + let 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; + + // initially the sequencer should build 100 empty blocks and the follower should follow them + let mut reorg_block = BlockInfo::default(); + for _ in 0..100 { + sequencer_handle.build_block().await; + wait_n_events( + &mut sequencer_events, + |e| { + if let RollupManagerEvent::BlockSequenced(block) = e { + if block.header.number == 97 { + reorg_block = BlockInfo { number: 97, hash: block.header.hash_slow() }; + } + true + } else { + false + } + }, + 1, + ) + .await; + wait_n_events( + &mut follower_events, + |e| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + ChainOrchestratorEvent::L2ChainCommitted(_, _, _) + ) + ) + }, + 1, + ) + .await; + } + + // Now revert the head of the sequencer node to block 97 + sequencer_handle.update_fcs_head(reorg_block).await; + + // Create a L1 message and send it to the sequencer node. + let l1_message = TxL1Message { + queue_index: 0, + gas_limit: 21000, + sender: Address::random(), + to: Address::random(), + value: U256::from(1), + input: Default::default(), + }; + let l1_message_notification = Arc::new(L1Notification::L1Message { + message: l1_message.clone(), + block_number: 0, + block_timestamp: 0, + }); + + // send the L1 message to both the sequencer and follower nodes. + sequencer_l1_watcher_tx.send(l1_message_notification.clone()).await.unwrap(); + wait_n_events( + &mut sequencer_events, + |e| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + ChainOrchestratorEvent::L1MessageCommitted(_) + ) + ) + }, + 1, + ) + .await; + follower_l1_watcher_tx.send(l1_message_notification).await.unwrap(); + wait_n_events( + &mut follower_events, + |e| { + matches!( + e, + RollupManagerEvent::ChainOrchestratorEvent( + ChainOrchestratorEvent::L1MessageCommitted(_) + ) + ) + }, + 1, + ) + .await; + + // Have the sequencer build a block with the L1 message. + 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; From fa51172b82c5739f7a81fd9fa124eb61326cec00 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 30 Jul 2025 19:11:38 +0800 Subject: [PATCH 21/30] lint --- crates/node/tests/sync.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 6b40de03..7e524cf9 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -587,6 +587,7 @@ async fn test_consolidation() -> eyre::Result<()> { Ok(()) } +#[allow(clippy::large_stack_frames)] #[tokio::test] async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { reth_tracing::init_test_tracing(); From e86446d57e8dea4731462ef12bb33302572c73b4 Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 30 Jul 2025 20:39:37 +0800 Subject: [PATCH 22/30] fix cli NetworkArgs --- crates/node/src/args.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index c358865a..04504d4c 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -358,10 +358,10 @@ impl Default for ChainOrchestratorArgs { 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, action = clap::ArgAction::Set)] + #[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, action = clap::ArgAction::Set)] + #[arg(long = "network.scroll-wire")] pub enable_scroll_wire: bool, /// The URL for the Sequencer RPC. (can be both HTTP and WS) #[arg( From 9cdbdf340e042ed51164ea05c86f93a4a5e3a2e6 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 31 Jul 2025 21:32:00 +0800 Subject: [PATCH 23/30] add block gap to reorg integration test --- crates/chain-orchestrator/src/lib.rs | 6 ++ crates/manager/src/manager/mod.rs | 4 +- crates/node/tests/sync.rs | 93 ++++++++++++---------------- crates/primitives/src/bounded_vec.rs | 5 ++ 4 files changed, 51 insertions(+), 57 deletions(-) diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index b23b07a7..bdc08ff4 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -432,6 +432,7 @@ impl< self.chain_spec.clone(), block_number, self.l2_client.clone(), + self.chain.clone(), )), )) } @@ -493,18 +494,23 @@ impl< chain_spec: Arc, l1_block_number: u64, l2_client: Arc

, + current_chain: Arc>, ) -> Result { 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 diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 1a8115be..27d1aa86 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -312,9 +312,7 @@ where // Issue the new block info to the engine driver for processing. self.engine.handle_optimistic_sync(block_info) } - event => { - trace!(target: "scroll::node::manager", ?event, "Received chain orchestrator event"); - } + _ => {} } } diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 7e524cf9..3f64b667 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -4,6 +4,8 @@ use alloy_primitives::{b256, Address, U256}; use alloy_provider::{Provider, ProviderBuilder}; use futures::StreamExt; use reqwest::Url; +use reth_network::{NetworkEvent, NetworkEventListenerProvider}; +use reth_network_api::{events::PeerEvent, test_utils::PeersHandleProvider}; use reth_provider::{BlockIdReader, BlockReader}; use reth_scroll_chainspec::{SCROLL_DEV, SCROLL_SEPOLIA}; use reth_tokio_util::EventStream; @@ -589,7 +591,7 @@ async fn test_consolidation() -> eyre::Result<()> { #[allow(clippy::large_stack_frames)] #[tokio::test] -async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { +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 { @@ -628,12 +630,10 @@ async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { let mut sequencer = nodes.pop().unwrap(); let sequencer_handle = sequencer.inner.rollup_manager_handle.clone(); let mut sequencer_events = sequencer_handle.get_event_listener().await?; - let sequencer_l1_watcher_tx = sequencer.inner.add_ons_handle.l1_watcher_tx.clone().unwrap(); let (mut nodes, _tasks, _) = setup_engine(node_config.clone(), 1, chain_spec.clone(), false, false).await.unwrap(); let mut follower = nodes.pop().unwrap(); - let 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. @@ -642,15 +642,15 @@ async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { sequencer.network.next_session_established().await; // initially the sequencer should build 100 empty blocks and the follower should follow them - let mut reorg_block = BlockInfo::default(); - for _ in 0..100 { + 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 block.header.number == 97 { - reorg_block = BlockInfo { number: 97, hash: block.header.hash_slow() }; + if i == 95 { + reorg_block_info = (&block).into(); } true } else { @@ -675,55 +675,40 @@ async fn test_chain_orchestrator_shallow_reorg() -> eyre::Result<()> { .await; } - // Now revert the head of the sequencer node to block 97 - sequencer_handle.update_fcs_head(reorg_block).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 + } + } - // Create a L1 message and send it to the sequencer node. - let l1_message = TxL1Message { - queue_index: 0, - gas_limit: 21000, - sender: Address::random(), - to: Address::random(), - value: U256::from(1), - input: Default::default(), - }; - let l1_message_notification = Arc::new(L1Notification::L1Message { - message: l1_message.clone(), - block_number: 0, - block_timestamp: 0, - }); - - // send the L1 message to both the sequencer and follower nodes. - sequencer_l1_watcher_tx.send(l1_message_notification.clone()).await.unwrap(); - wait_n_events( - &mut sequencer_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, - 1, - ) - .await; - follower_l1_watcher_tx.send(l1_message_notification).await.unwrap(); - wait_n_events( - &mut follower_events, - |e| { - matches!( - e, - RollupManagerEvent::ChainOrchestratorEvent( - ChainOrchestratorEvent::L1MessageCommitted(_) - ) - ) - }, - 1, - ) - .await; + sequencer_handle.update_fcs_head(reorg_block_info).await; + + // Have the sequencer build 2 new blocks, one containing the L1 message. + sequencer_handle.build_block().await; + wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), 1) + .await; + sequencer_handle.build_block().await; + wait_n_events(&mut sequencer_events, |e| matches!(e, RollupManagerEvent::BlockSequenced(_)), 1) + .await; + + // connect the two nodes again + follower.connect(&mut sequencer).await; - // Have the sequencer build a block with the L1 message. + // now build a final block sequencer_handle.build_block().await; // Wait for the follower node to reorg to the new chain. diff --git a/crates/primitives/src/bounded_vec.rs b/crates/primitives/src/bounded_vec.rs index 305e2281..c8f015fa 100644 --- a/crates/primitives/src/bounded_vec.rs +++ b/crates/primitives/src/bounded_vec.rs @@ -77,6 +77,11 @@ impl BoundedVec { &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 From 2f851b197913106e7a48a567b510f176d3e98705 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 31 Jul 2025 23:01:23 +0800 Subject: [PATCH 24/30] make test more robust --- crates/sequencer/tests/e2e.rs | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index 13d772bc..cfb0fa7b 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -575,16 +575,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(()) From 9a3339ad0a823cf05b45e4cbf3fcc1210f1a2df2 Mon Sep 17 00:00:00 2001 From: frisitano Date: Thu, 7 Aug 2025 12:48:29 +0800 Subject: [PATCH 25/30] merge upstream --- crates/chain-orchestrator/src/action.rs | 7 +- crates/chain-orchestrator/src/lib.rs | 146 +++++++++++++++--------- crates/manager/src/manager/mod.rs | 7 +- crates/network/src/manager.rs | 32 ++++-- crates/node/src/args.rs | 11 +- crates/node/tests/e2e.rs | 75 ++++++------ crates/node/tests/sync.rs | 11 +- crates/primitives/src/bounded_vec.rs | 11 -- 8 files changed, 186 insertions(+), 114 deletions(-) diff --git a/crates/chain-orchestrator/src/action.rs b/crates/chain-orchestrator/src/action.rs index a432f141..1baadcf7 100644 --- a/crates/chain-orchestrator/src/action.rs +++ b/crates/chain-orchestrator/src/action.rs @@ -7,8 +7,9 @@ use std::{ }; /// A future that resolves to a `Result`. -pub(super) type PendingChainOrchestratorFuture = - Pin> + Send>>; +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 { @@ -26,7 +27,7 @@ impl ChainOrchestratorFuture { pub(super) fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll, ChainOrchestratorError>> { match self { Self::HandleReorg(fut) | Self::HandleFinalized(fut) | diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index bdc08ff4..2acd3f75 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -159,14 +159,55 @@ impl< let fut = self.handle_metered( ChainOrchestratorItem::NewBlock, - Box::pin(async move { Self::handle_new_block(ctx, block_with_peer).await }), + 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.insert_block(block_info.clone(), None).await?; + Ok(ChainOrchestratorEvent::L2ChainCommitted(block_info, None, true)) + } + /// Handles a new block received from the network. - async fn handle_new_block( + async fn do_handle_block_from_peer( ctx: HandleBlockContext, block_with_peer: NewBlockWithPeer, ) -> Result { @@ -253,7 +294,7 @@ impl< { return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( received_block.hash_slow(), - )) + )); } // If the received header tail has a block number that is less than the current header @@ -383,39 +424,38 @@ impl< 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.insert_blocks(block_info, batch_info).await?; - - Result::<_, ChainOrchestratorError>::Ok( - ChainOrchestratorEvent::L2ChainCommitted(head, batch_info, consolidated), + 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.insert_blocks(block_info, batch_info).await?; + + Result::<_, ChainOrchestratorError>::Ok(Some( + ChainOrchestratorEvent::L2ChainCommitted(head, batch_info, consolidated), + )) + }), + ); self.pending_futures.push_back(ChainOrchestratorFuture::HandleDerivedBlock(fut)); self.waker.wake(); @@ -495,7 +535,7 @@ impl< l1_block_number: u64, l2_client: Arc

, current_chain: Arc>, - ) -> Result { + ) -> 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?; @@ -515,12 +555,12 @@ impl< } else { None }; - Ok(ChainOrchestratorEvent::ChainUnwound { + 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 @@ -530,7 +570,7 @@ impl< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result, ChainOrchestratorError> { // Set the latest finalized L1 block in the database. database.set_latest_finalized_l1_block_number(block_number).await?; @@ -547,7 +587,7 @@ impl< // update the chain orchestrator l1 block number. l1_block_number.store(block_number, Ordering::Relaxed); - Ok(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_block)) + Ok(Some(ChainOrchestratorEvent::L1BlockFinalized(block_number, finalized_block))) } /// Handles an L1 message by inserting it into the database. @@ -557,7 +597,7 @@ impl< l1_message: TxL1Message, l1_block_number: u64, block_timestamp: u64, - ) -> Result { + ) -> Result, ChainOrchestratorError> { let event = ChainOrchestratorEvent::L1MessageCommitted(l1_message.queue_index); let queue_hash = if chain_spec @@ -581,14 +621,14 @@ impl< let l1_message = L1MessageEnvelope::new(l1_message, l1_block_number, None, queue_hash); database.insert_l1_message(l1_message).await?; - Ok(event) + Ok(Some(event)) } /// Handles a batch input by inserting it into the database. async fn handle_batch_commit( database: Arc, batch: BatchCommitData, - ) -> Result { + ) -> Result, ChainOrchestratorError> { let txn = database.tx().await?; let prev_batch_index = batch.index - 1; @@ -613,7 +653,7 @@ impl< txn.insert_batch(batch).await?; txn.commit().await?; - Ok(event) + Ok(Some(event)) } /// Handles a batch finalization event by updating the batch input in the database. @@ -623,7 +663,7 @@ impl< block_number: u64, l1_block_number: Arc, l2_block_number: Arc, - ) -> Result { + ) -> Result, ChainOrchestratorError> { // finalized the batch. database.finalize_batch(batch_hash, block_number).await?; @@ -637,7 +677,7 @@ impl< } let event = ChainOrchestratorEvent::BatchFinalized(batch_hash, finalized_block); - Ok(event) + Ok(Some(event)) } /// Returns the highest finalized block for the provided batch hash. Will return [`None`] if the @@ -702,12 +742,16 @@ impl< self.waker.register(cx.waker()); // Remove and poll the next future in the queue - if let Some(mut action) = self.pending_futures.pop_front() { - return match action.poll(cx) { - Poll::Ready(result) => Poll::Ready(Some(result)), + 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); - Poll::Pending + return Poll::Pending } }; } @@ -918,7 +962,7 @@ mod test { /// 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. + /// This full block client can be [Cloned] and shared between multiple tasks. #[derive(Clone, Debug)] struct TestScrollFullBlockClient { headers: Arc>>, diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index a227c9ae..b2c2d6c6 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -536,6 +536,11 @@ where )); } + this.chain.handle_sequenced_block(NewBlockWithPeer { + peer_id: Default::default(), + block: block.clone(), + signature, + }); this.network.handle().announce_block(block, signature); } } @@ -545,7 +550,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( diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index ac6c692b..2b10d4ac 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -55,6 +55,8 @@ pub struct ScrollNetworkManager { 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 @@ -67,6 +69,7 @@ impl 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); @@ -96,6 +99,7 @@ impl from_handle_rx: from_handle_rx.into(), scroll_wire, eth_wire_listener, + eth_wire_gossip, } } } @@ -114,6 +118,7 @@ impl< 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(); @@ -129,6 +134,7 @@ impl< from_handle_rx: from_handle_rx.into(), scroll_wire, eth_wire_listener, + eth_wire_gossip, } } @@ -155,13 +161,15 @@ impl< .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 { @@ -234,8 +242,15 @@ impl< &mut self, block: reth_network_api::block::NewBlockWithPeer, ) -> Option { - trace!(target: "scroll::node::manager", ?block, "Received new block from eth-wire protocol"); 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) { @@ -255,7 +270,6 @@ impl< .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"); 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"); diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 05506e63..52f9fc92 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -214,6 +214,7 @@ impl ScrollRollupNodeConfig { 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 @@ -465,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, + } } } diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index b3ff5a30..f3a6fb07 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -137,6 +137,7 @@ 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(), @@ -232,7 +233,9 @@ async fn can_forward_tx_to_sequencer() { // create 2 nodes let mut sequencer_node_config = default_sequencer_test_scroll_rollup_node_config(); sequencer_node_config.sequencer_args.block_time = 0; + sequencer_node_config.network_args.enable_eth_scroll_wire_bridge = false; let mut follower_node_config = default_test_scroll_rollup_node_config(); + follower_node_config.network_args.enable_eth_scroll_wire_bridge = false; // Create the chain spec for scroll mainnet with Euclid v2 activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); @@ -347,19 +350,6 @@ async fn can_forward_tx_to_sequencer() { 1, ) .await; - wait_n_events( - &mut follower_events, - |e| { - if let RollupManagerEvent::BlockImported(block) = e { - assert_eq!(block.body.transactions.len(), 1); - true - } else { - false - } - }, - 1, - ) - .await; } #[allow(clippy::large_stack_frames)] @@ -504,6 +494,7 @@ async fn can_bridge_blocks() { network_config, scroll_wire_config, None, + false, ) .await; let scroll_network_handle = scroll_network.handle(); @@ -973,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(); @@ -1016,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(); @@ -1042,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 0ef44e9f..74e0dbf7 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -15,8 +15,7 @@ use rollup_node::{ setup_engine, }, BeaconProviderArgs, ChainOrchestratorArgs, ConsensusArgs, DatabaseArgs, EngineDriverArgs, - GasPriceOracleArgs, L1ProviderArgs, NetworkArgs, ScrollRollupNode, ScrollRollupNodeConfig, - SequencerArgs, + GasPriceOracleArgs, L1ProviderArgs, NetworkArgs, ScrollRollupNodeConfig, SequencerArgs, }; use rollup_node_chain_orchestrator::ChainOrchestratorEvent; use rollup_node_manager::RollupManagerEvent; @@ -45,6 +44,7 @@ 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 { @@ -186,6 +186,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { 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(), @@ -204,6 +205,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { }, 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. @@ -433,6 +435,7 @@ async fn test_consolidation() -> eyre::Result<()> { 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(), @@ -451,6 +454,7 @@ async fn test_consolidation() -> eyre::Result<()> { }, 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. @@ -602,7 +606,7 @@ async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { let sequencer_node_config = ScrollRollupNodeConfig { test: true, network_args: NetworkArgs { - enable_eth_scroll_wire_bridge: true, + enable_eth_scroll_wire_bridge: false, enable_scroll_wire: true, ..Default::default() }, @@ -623,6 +627,7 @@ async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { }, 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. diff --git a/crates/primitives/src/bounded_vec.rs b/crates/primitives/src/bounded_vec.rs index c8f015fa..9a659c3e 100644 --- a/crates/primitives/src/bounded_vec.rs +++ b/crates/primitives/src/bounded_vec.rs @@ -90,17 +90,6 @@ impl BoundedVec { 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_back(elem) } From 000ade7240e012aad917c53022128f8c513b5a79 Mon Sep 17 00:00:00 2001 From: frisitano Date: Fri, 8 Aug 2025 00:35:45 +0800 Subject: [PATCH 26/30] refactor --- crates/chain-orchestrator/src/event.rs | 2 + crates/chain-orchestrator/src/lib.rs | 49 ++++++++--- crates/chain-orchestrator/src/metrics.rs | 3 + crates/database/db/src/db.rs | 46 ++++------- crates/database/db/src/models/l2_block.rs | 30 +++---- crates/database/db/src/operations.rs | 82 ++++++++++--------- .../src/m20250411_072004_add_l2_block.rs | 12 +-- crates/manager/src/manager/mod.rs | 13 ++- 8 files changed, 126 insertions(+), 111 deletions(-) diff --git a/crates/chain-orchestrator/src/event.rs b/crates/chain-orchestrator/src/event.rs index 90ede951..bbabac3b 100644 --- a/crates/chain-orchestrator/src/event.rs +++ b/crates/chain-orchestrator/src/event.rs @@ -61,4 +61,6 @@ pub enum ChainOrchestratorEvent { /// 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 index 2acd3f75..8d1c80ba 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -202,7 +202,7 @@ impl< 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.insert_block(block_info.clone(), None).await?; + database.update_l1_messages_with_l2_block(block_info.clone()).await?; Ok(ChainOrchestratorEvent::L2ChainCommitted(block_info, None, true)) } @@ -412,12 +412,32 @@ impl< } } - /// Inserts an L2 block in the database. - pub fn consolidate_l2_blocks( + /// Persist L1 consolidate blocks in the database. + pub fn persist_l1_consolidated_blocks( &mut self, - block_info: Vec, - batch_info: Option, + 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(); @@ -449,10 +469,10 @@ impl< // Insert the blocks into the database. let head = block_info.last().expect("block info must not be empty").clone(); - database.insert_blocks(block_info, batch_info).await?; + database.update_l1_messages_from_l2_blocks(block_info).await?; Result::<_, ChainOrchestratorError>::Ok(Some( - ChainOrchestratorEvent::L2ChainCommitted(head, batch_info, consolidated), + ChainOrchestratorEvent::L2ChainCommitted(head, None, consolidated), )) }), ); @@ -962,7 +982,7 @@ mod test { /// 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 [Cloned] and shared between multiple tasks. + /// This full block client can be [Clone]d and shared between multiple tasks. #[derive(Clone, Debug)] struct TestScrollFullBlockClient { headers: Arc>>, @@ -1230,13 +1250,13 @@ mod test { l1_messages: vec![], }; - chain_orchestrator.consolidate_l2_blocks(vec![block_1.clone()], Some(batch_1_info)); + chain_orchestrator.persist_l1_consolidated_blocks(vec![block_1.clone()], batch_1_info); chain_orchestrator.next().await.unwrap().unwrap(); - chain_orchestrator.consolidate_l2_blocks(vec![block_2.clone()], Some(batch_2_info)); + chain_orchestrator.persist_l1_consolidated_blocks(vec![block_2.clone()], batch_2_info); chain_orchestrator.next().await.unwrap().unwrap(); - chain_orchestrator.consolidate_l2_blocks(vec![block_3.clone()], Some(batch_2_info)); + 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 @@ -1513,7 +1533,12 @@ mod test { } else { None }; - indexer.consolidate_l2_blocks(vec![l2_block.clone()], batch_info); + 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); } diff --git a/crates/chain-orchestrator/src/metrics.rs b/crates/chain-orchestrator/src/metrics.rs index 41eb9c26..c71b4841 100644 --- a/crates/chain-orchestrator/src/metrics.rs +++ b/crates/chain-orchestrator/src/metrics.rs @@ -7,6 +7,8 @@ use strum::EnumIter; pub enum ChainOrchestratorItem { /// Handle a block received from the network. NewBlock, + /// Insert consolidated L2 blocks into the database. + InsertConsolidatedL2Blocks, /// L2 block. InsertL2Block, /// L1 reorg. @@ -26,6 +28,7 @@ impl ChainOrchestratorItem { pub const fn as_str(&self) -> &'static str { match self { Self::NewBlock => "new_block", + Self::InsertConsolidatedL2Blocks => "insert_consolidated_l2_blocks", Self::InsertL2Block => "l2_block", Self::L1Reorg => "l1_reorg", Self::L1Finalization => "l1_finalization", diff --git a/crates/database/db/src/db.rs b/crates/database/db/src/db.rs index d8d487bc..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; } @@ -402,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 @@ -443,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(); @@ -486,7 +477,7 @@ mod test { db.insert_block( L2BlockInfoWithL1Messages { block_info, l1_messages: vec![] }, - Some(batch_info), + batch_info, ) .await .unwrap(); @@ -537,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 @@ -601,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(); @@ -637,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(); @@ -650,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(); @@ -671,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/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 ddcb2545..f647f271 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -289,7 +289,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { async fn get_l2_block_and_batch_info_by_hash( &self, block_hash: B256, - ) -> Result)>, DatabaseError> { + ) -> 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())) @@ -297,7 +297,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .await .map(|x| { x.map(|x| { - let (block_info, batch_info): (BlockInfo, Option) = x.into(); + let (block_info, batch_info): (BlockInfo, BatchInfo) = x.into(); (block_info, batch_info) }) })?) @@ -314,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 }) })?) @@ -330,15 +330,7 @@ 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 an iterator over all L2 blocks in the database starting from the most recent one. @@ -425,7 +417,7 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { async fn insert_blocks( &self, blocks: Vec, - batch_info: Option, + batch_info: BatchInfo, ) -> Result<(), DatabaseError> { for block in blocks { self.insert_block(block, batch_info).await?; @@ -437,34 +429,50 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { 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. - if batch_info.is_some() { - 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), - block_number = block_info.block_info.number, - block_hash = ?block_info.block_info.hash, - "Inserting block into database." - ); - let l2_block: models::l2_block::ActiveModel = - (block_info.block_info, batch_info).into(); - models::l2_block::Entity::insert(l2_block) - .on_conflict( - OnConflict::column(models::l2_block::Column::BlockNumber) - .update_columns([ - models::l2_block::Column::BlockHash, - models::l2_block::Column::BatchHash, - models::l2_block::Column::BatchIndex, - ]) - .to_owned(), - ) - .exec(self.get_connection()) - .await?; + tracing::trace!( + target: "scroll::db", + batch_hash = ?batch_info.hash, + batch_index = batch_info.index, + block_number = block_info.block_info.number, + block_hash = ?block_info.block_info.hash, + "Inserting block into database." + ); + let l2_block: models::l2_block::ActiveModel = (block_info.block_info, batch_info).into(); + models::l2_block::Entity::insert(l2_block) + .on_conflict( + OnConflict::column(models::l2_block::Column::BlockNumber) + .update_columns([ + models::l2_block::Column::BlockHash, + models::l2_block::Column::BatchHash, + models::l2_block::Column::BatchIndex, + ]) + .to_owned(), + ) + .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, 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 1f383f7a..82c39947 100644 --- a/crates/database/migration/src/m20250411_072004_add_l2_block.rs +++ b/crates/database/migration/src/m20250411_072004_add_l2_block.rs @@ -21,23 +21,23 @@ 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(), ) diff --git a/crates/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index b2c2d6c6..5f1ed30e 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -325,7 +325,7 @@ where if let Some(event_sender) = self.event_sender.as_ref() { event_sender.notify(RollupManagerEvent::BlockImported(block.clone())); } - self.chain.consolidate_l2_blocks(vec![(&block).into()], None); + self.chain.consolidate_validated_l2_blocks(vec![(&block).into()]); } self.network.handle().block_import_outcome(outcome); } @@ -335,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.chain.consolidate_l2_blocks(vec![(&payload).into()], None); } EngineDriverEvent::L1BlockConsolidated(consolidation_outcome) => { - self.chain.consolidate_l2_blocks( + self.chain.persist_l1_consolidated_blocks( vec![consolidation_outcome.block_info().clone()], - Some(*consolidation_outcome.batch_info()), + *consolidation_outcome.batch_info(), ); if let Some(event_sender) = self.event_sender.as_ref() { @@ -357,9 +355,8 @@ where if let Some(event_sender) = self.event_sender.as_ref() { event_sender.notify(RollupManagerEvent::BlockImported(block)); } - self.chain.consolidate_l2_blocks( + self.chain.consolidate_validated_l2_blocks( outcome.chain.iter().map(|b| b.into()).collect(), - None, ); } self.network.handle().block_import_outcome(outcome.outcome); From 00d18f2b2673b7ee94ded28f59756aed70fe7fa0 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 11 Aug 2025 19:09:17 +0800 Subject: [PATCH 27/30] address comments --- crates/chain-orchestrator/src/error.rs | 5 +- crates/chain-orchestrator/src/lib.rs | 89 ++++++++++++++++---------- crates/node/src/args.rs | 2 +- crates/node/tests/sync.rs | 6 +- crates/sequencer/tests/e2e.rs | 5 -- 5 files changed, 63 insertions(+), 44 deletions(-) diff --git a/crates/chain-orchestrator/src/error.rs b/crates/chain-orchestrator/src/error.rs index 2b88a00c..8fa6e7b7 100644 --- a/crates/chain-orchestrator/src/error.rs +++ b/crates/chain-orchestrator/src/error.rs @@ -11,7 +11,10 @@ pub enum ChainOrchestratorError { 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), + L2BlockNotFoundInDatabase(u64), + /// An error occurred while trying to fetch the L2 block from the L2 client. + #[error("L2 block not found in L2 client - block number: {0}")] + L2BlockNotFoundInL2Client(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, diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index 8d1c80ba..f97f2fd3 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -99,9 +99,9 @@ impl< 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 { + ) -> Result { + let chain = init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; + Ok(Self { network_client: Arc::new(block_client), l2_client: Arc::new(l2_client), chain: Arc::new(Mutex::new(chain)), @@ -121,7 +121,7 @@ impl< chain_buffer_size, l1_synced: false, waker: AtomicWaker::new(), - } + }) } /// Wraps a pending chain orchestrator future, metering the completion of it. @@ -237,10 +237,11 @@ impl< 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 + optimistic_headers.first().expect("chain can not be empty").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; + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(optimistic_headers.first().expect("chain can not be empty").number - 1), "fetching block"); + let parent_hash = + optimistic_headers.first().expect("chain can not be empty").parent_hash; let header = network_client .get_header(BlockHashOrNumber::Hash(parent_hash)) .await? @@ -289,8 +290,8 @@ impl< // 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 + received_chain_headers.last().expect("chain can not be empty").number < + current_chain_headers.front().expect("chain can not be empty").number { return Ok(ChainOrchestratorEvent::InsufficientDataForReceivedBlock( received_block.hash_slow(), @@ -300,41 +301,56 @@ impl< // 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 < + if received_chain_headers.last().expect("chain can not be empty").number < current_chain_headers.front().expect("chain can not be empty").number { for _ in 0..BATCH_FETCH_SIZE { - if current_chain_headers.front().unwrap().number.saturating_sub(1) <= + if current_chain_headers + .front() + .expect("chain can not be empty") + .number + .saturating_sub(1) <= latest_safe_block.number { tracing::info!(target: "scroll::chain_orchestrator", hash = %latest_safe_block.hash, number = %latest_safe_block.number, "reached safe block number for current chain - terminating fetching."); break; } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(current_chain_headers.front().unwrap().number - 1), "fetching block for current chain"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(current_chain_headers.front().expect("chain can not be empty").number - 1), "fetching block for current chain"); if let Some(block) = l2_client - .get_block_by_hash(current_chain_headers.front().unwrap().parent_hash) + .get_block_by_hash( + current_chain_headers + .front() + .expect("chain can not be empty") + .parent_hash, + ) .await? { let header = block.into_consensus_header(); current_chain_headers.push_front(header.clone()); } else { return Err(ChainOrchestratorError::MissingBlockHeader { - hash: current_chain_headers.front().unwrap().parent_hash, + hash: current_chain_headers + .front() + .expect("chain can not be empty") + .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 let Some(pos) = current_chain_headers.iter().rposition(|h| { + h.hash_slow() == + received_chain_headers.last().expect("chain can not be empty").parent_hash + }) { // If the received fork is older than the current chain, we return an event // indicating that we have received an old fork. if (pos < current_chain_headers.len() - 1) && - current_chain_headers.get(pos + 1).unwrap().timestamp > - received_chain_headers.last().unwrap().timestamp + current_chain_headers.get(pos + 1).expect("chain can not be empty").timestamp > + received_chain_headers + .last() + .expect("chain can not be empty") + .timestamp { return Ok(ChainOrchestratorEvent::OldForkReceived { headers: received_chain_headers, @@ -347,14 +363,16 @@ impl< // 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 { + if received_chain_headers.last().expect("chain can not be empty").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"); + tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_chain_headers.last().expect("chain can not be empty").number - 1), "fetching block"); if let Some(header) = network_client .get_header(BlockHashOrNumber::Hash( - received_chain_headers.last().unwrap().parent_hash, + received_chain_headers.last().expect("chain can not be empty").parent_hash, )) .await? .into_data() @@ -362,7 +380,10 @@ impl< received_chain_headers.push(header.clone()); } else { return Err(ChainOrchestratorError::MissingBlockHeader { - hash: received_chain_headers.last().unwrap().parent_hash, + hash: received_chain_headers + .last() + .expect("chain can not be empty") + .parent_hash, }); } }; @@ -727,16 +748,15 @@ async fn init_chain_from_db + 'static>( database: &Arc, l2_client: &P, chain_buffer_size: usize, -) -> BoundedVec

{ +) -> Result, ChainOrchestratorError> { 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 mut blocks_stream = database.get_l2_blocks().await?.take(chain_buffer_size); + while let Some(block_info) = blocks_stream.try_next().await? { let header = l2_client .get_block_by_hash(block_info.hash) - .await - .unwrap() - .unwrap() + .await? + .ok_or(ChainOrchestratorError::L2BlockNotFoundInL2Client(block_info.number))? .header .into_consensus(); blocks.push(header); @@ -746,7 +766,7 @@ async fn init_chain_from_db + 'static>( }; let mut chain: Chain = Chain::new(chain_buffer_size); chain.extend(blocks); - chain + Ok(chain) } impl< @@ -838,7 +858,7 @@ async fn consolidate_chain + 'static>( 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; + init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; return Err(ChainOrchestratorError::ChainInconsistency); } @@ -868,7 +888,7 @@ async fn consolidate_chain + 'static>( // 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; + init_chain_from_db(&database, &l2_client, chain_buffer_size).await?; return Err(ChainOrchestratorError::ChainInconsistency); } } @@ -1139,7 +1159,8 @@ mod test { TEST_OPTIMISTIC_SYNC_THRESHOLD, TEST_CHAIN_BUFFER_SIZE, ) - .await, + .await + .unwrap(), db, ) } diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index 52f9fc92..53028ee3 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -321,7 +321,7 @@ impl ScrollRollupNodeConfig { self.chain_orchestrator_args.optimistic_sync_trigger, self.chain_orchestrator_args.chain_buffer_size, ) - .await; + .await?; // Spawn the rollup node manager let (rnm, handle) = RollupNodeManager::new( diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index 74e0dbf7..9afbf38a 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -208,7 +208,7 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { consensus_args: ConsensusArgs::noop(), }; - // Create the chain spec for scroll dev with Euclid v2 activated and a test genesis. + // Create the chain spec for scroll dev with Feynman activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); // Create a sequencer node and an unsynced node. @@ -457,7 +457,7 @@ async fn test_consolidation() -> eyre::Result<()> { consensus_args: ConsensusArgs::noop(), }; - // Create the chain spec for scroll dev with Euclid v2 activated and a test genesis. + // Create the chain spec for scroll dev with Feynman activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); // Create a sequencer node and an unsynced node. @@ -630,7 +630,7 @@ async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { consensus_args: ConsensusArgs::noop(), }; - // Create the chain spec for scroll dev with Euclid v2 activated and a test genesis. + // Create the chain spec for scroll dev with Feynman activated and a test genesis. let chain_spec = (*SCROLL_DEV).clone(); // Create a sequencer node and an unsynced node. diff --git a/crates/sequencer/tests/e2e.rs b/crates/sequencer/tests/e2e.rs index ac343adc..82d267b6 100644 --- a/crates/sequencer/tests/e2e.rs +++ b/crates/sequencer/tests/e2e.rs @@ -41,7 +41,6 @@ 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; // setup a test node let (mut nodes, _tasks, wallet) = setup(1, false).await.unwrap(); @@ -284,8 +283,6 @@ 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; - // setup a test node let (mut nodes, _tasks, wallet) = setup_engine(default_test_scroll_rollup_node_config(), 1, chain_spec, false, false) @@ -603,7 +600,6 @@ async fn can_build_blocks_and_exit_at_gas_limit() { let chain_spec = SCROLL_DEV.clone(); const MIN_TRANSACTION_GAS_COST: u64 = 21_000; const BLOCK_BUILDING_DURATION: Duration = Duration::from_millis(250); - // const 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 @@ -689,7 +685,6 @@ 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 TRANSACTIONS_COUNT: usize = 2000; // setup a test node. use a low payload building duration in order to exit before we reach the From add99335af61c34e5988ed775f613ea4559dc35d Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 20 Aug 2025 15:42:48 +0800 Subject: [PATCH 28/30] address comments --- crates/chain-orchestrator/src/lib.rs | 114 +-- crates/database/db/src/operations.rs | 3 +- crates/database/db/src/test_utils.rs | 4 +- crates/database/migration/src/lib.rs | 12 +- .../database/migration/src/migration_info.rs | 5 +- crates/engine/src/driver.rs | 2 +- crates/indexer/src/event.rs | 38 - crates/indexer/src/lib.rs | 793 ------------------ crates/manager/src/manager/mod.rs | 64 +- crates/network/src/manager.rs | 26 +- crates/node/src/args.rs | 11 +- crates/node/tests/e2e.rs | 5 +- crates/node/tests/sync.rs | 3 - crates/scroll-wire/src/protocol/proto.rs | 2 +- .../resource/dashboards/rollup_node.json | 2 +- 15 files changed, 123 insertions(+), 961 deletions(-) delete mode 100644 crates/indexer/src/event.rs delete mode 100644 crates/indexer/src/lib.rs diff --git a/crates/chain-orchestrator/src/lib.rs b/crates/chain-orchestrator/src/lib.rs index f97f2fd3..93965ab9 100644 --- a/crates/chain-orchestrator/src/lib.rs +++ b/crates/chain-orchestrator/src/lib.rs @@ -298,6 +298,14 @@ impl< )); } + // If the current header block number is less than the latest safe block number then + // we should error. + if received_chain_headers.last().expect("chain can not be empty").number <= + latest_safe_block.number + { + return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); + } + // 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. @@ -361,14 +369,6 @@ impl< 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().expect("chain can not be empty").number <= - latest_safe_block.number - { - return Err(ChainOrchestratorError::L2SafeBlockReorgDetected); - } - tracing::trace!(target: "scroll::chain_orchestrator", number = ?(received_chain_headers.last().expect("chain can not be empty").number - 1), "fetching block"); if let Some(header) = network_client .get_header(BlockHashOrNumber::Hash( @@ -722,7 +722,7 @@ impl< } /// Returns the highest finalized block for the provided batch hash. Will return [`None`] if the - /// block number has already been seen by the indexer. + /// block number has already been seen by the chain orchestrator. async fn fetch_highest_finalized_block( database: Arc, batch_hash: B256, @@ -730,7 +730,7 @@ impl< ) -> 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. + // only return the block if the chain orchestrator 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); @@ -1167,8 +1167,8 @@ mod test { #[tokio::test] async fn test_handle_commit_batch() { - // Instantiate indexer and db - let (mut indexer, db) = setup_test_chain_orchestrator().await; + // Instantiate chain orchestrator and db + let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1176,9 +1176,10 @@ mod test { let mut u = Unstructured::new(&bytes); let batch_commit = BatchCommitData::arbitrary(&mut u).unwrap(); - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_commit.clone())); + chain_orchestrator + .handle_l1_notification(L1Notification::BatchCommit(batch_commit.clone())); - let event = indexer.next().await.unwrap().unwrap(); + let event = chain_orchestrator.next().await.unwrap().unwrap(); // Verify the event structure match event { @@ -1196,7 +1197,7 @@ mod test { #[tokio::test] async fn test_handle_batch_commit_with_revert() { - // Instantiate indexer and db + // Instantiate chain orchestrator and db let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. @@ -1317,8 +1318,8 @@ mod test { #[tokio::test] async fn test_handle_l1_message() { - // Instantiate indexer and db - let (mut indexer, db) = setup_test_chain_orchestrator().await; + // Instantiate chain orchestrator and db + let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1330,13 +1331,13 @@ mod test { ..Arbitrary::arbitrary(&mut u).unwrap() }; let block_number = u64::arbitrary(&mut u).unwrap(); - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.handle_l1_notification(L1Notification::L1Message { message: message.clone(), block_number, block_timestamp: 0, }); - let _ = indexer.next().await; + let _ = chain_orchestrator.next().await; let l1_message_result = db.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); @@ -1347,16 +1348,16 @@ mod test { #[tokio::test] async fn test_l1_message_hash_queue() { - // Instantiate indexer and db - let (mut indexer, db) = setup_test_chain_orchestrator().await; + // Instantiate chain orchestrator and db + let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // insert the previous L1 message in database. - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.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 _ = chain_orchestrator.next().await.unwrap().unwrap(); // let message = TxL1Message { @@ -1367,13 +1368,13 @@ mod test { sender: address!("61d8d3E7F7c656493d1d76aAA1a836CEdfCBc27b"), input: bytes!("8ef1332e000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d6996000000000000000000000000323522a8de3cddeddbb67094eecaebc2436d699600000000000000000000000000000000000000000000000000038d7ea4c6800000000000000000000000000000000000000000000000000000000000001034de00000000000000000000000000000000000000000000000000000000000000a00000000000000000000000000000000000000000000000000000000000000000"), }; - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.handle_l1_notification(L1Notification::L1Message { message: message.clone(), block_number: 14755883, block_timestamp: 1745305200, }); - let _ = indexer.next().await.unwrap().unwrap(); + let _ = chain_orchestrator.next().await.unwrap().unwrap(); let l1_message_result = db.get_l1_message_by_index(message.queue_index).await.unwrap().unwrap(); @@ -1386,8 +1387,8 @@ mod test { #[tokio::test] async fn test_handle_reorg() { - // Instantiate indexer and db - let (mut indexer, db) = setup_test_chain_orchestrator().await; + // Instantiate chain orchestrator and db + let (mut chain_orchestrator, db) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 1024]; @@ -1411,9 +1412,12 @@ mod test { 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())); + chain_orchestrator + .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); + chain_orchestrator + .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_20.clone())); + chain_orchestrator + .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 { @@ -1436,27 +1440,27 @@ mod test { }; // Index L1 messages - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.handle_l1_notification(L1Notification::L1Message { message: l1_message_block_1.clone().transaction, block_number: l1_message_block_1.clone().l1_block_number, block_timestamp: 0, }); - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.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 { + chain_orchestrator.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)); + chain_orchestrator.handle_l1_notification(L1Notification::Reorg(20)); for _ in 0..7 { - indexer.next().await.unwrap().unwrap(); + chain_orchestrator.next().await.unwrap().unwrap(); } // Check that the batch input at block 30 is deleted @@ -1485,8 +1489,8 @@ mod test { #[ignore] #[tokio::test] async fn test_handle_reorg_executed_l1_messages() { - // Instantiate indexer and db - let (mut indexer, _database) = setup_test_chain_orchestrator().await; + // Instantiate chain orchestrator and db + let (mut chain_orchestrator, _database) = setup_test_chain_orchestrator().await; // Generate unstructured bytes. let mut bytes = [0u8; 8192]; @@ -1503,10 +1507,12 @@ mod test { }; // 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())); + chain_orchestrator + .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_1.clone())); + chain_orchestrator + .handle_l1_notification(L1Notification::BatchCommit(batch_commit_block_10.clone())); for _ in 0..2 { - let _event = indexer.next().await.unwrap().unwrap(); + let _event = chain_orchestrator.next().await.unwrap().unwrap(); } let batch_1 = BatchInfo::new(batch_commit_block_1.index, batch_commit_block_1.hash); @@ -1527,12 +1533,12 @@ mod test { ..Arbitrary::arbitrary(&mut u).unwrap() }, }; - indexer.handle_l1_notification(L1Notification::L1Message { + chain_orchestrator.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(); + chain_orchestrator.next().await.unwrap().unwrap(); l1_messages.push(l1_message); } @@ -1555,19 +1561,20 @@ mod test { None }; if let Some(batch_info) = batch_info { - indexer.persist_l1_consolidated_blocks(vec![l2_block.clone()], batch_info); + chain_orchestrator + .persist_l1_consolidated_blocks(vec![l2_block.clone()], batch_info); } else { - indexer.consolidate_validated_l2_blocks(vec![l2_block.clone()]); + chain_orchestrator.consolidate_validated_l2_blocks(vec![l2_block.clone()]); } - indexer.next().await.unwrap().unwrap(); + chain_orchestrator.next().await.unwrap().unwrap(); blocks.push(l2_block); } // First we assert that we dont reorg the L2 or message queue hash for a higher block // than any of the L1 messages. - indexer.handle_l1_notification(L1Notification::Reorg(17)); - let event = indexer.next().await.unwrap().unwrap(); + chain_orchestrator.handle_l1_notification(L1Notification::Reorg(17)); + let event = chain_orchestrator.next().await.unwrap().unwrap(); assert_eq!( event, ChainOrchestratorEvent::ChainUnwound { @@ -1580,8 +1587,8 @@ mod test { // 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(); + chain_orchestrator.handle_l1_notification(L1Notification::Reorg(7)); + let event = chain_orchestrator.next().await.unwrap().unwrap(); assert_eq!( event, @@ -1594,8 +1601,8 @@ mod test { ); // 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(); + chain_orchestrator.handle_l1_notification(L1Notification::Reorg(3)); + let event = chain_orchestrator.next().await.unwrap().unwrap(); assert_eq!( event, @@ -1603,7 +1610,10 @@ mod test { 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())), + l2_safe_block_info: Some(BlockInfo::new( + 0, + chain_orchestrator.chain_spec.genesis_hash() + )), } ); } diff --git a/crates/database/db/src/operations.rs b/crates/database/db/src/operations.rs index f647f271..6b036df8 100644 --- a/crates/database/db/src/operations.rs +++ b/crates/database/db/src/operations.rs @@ -534,7 +534,8 @@ pub trait DatabaseOperations: DatabaseConnectionProvider { .map(|model| model.block_info())) } - /// Unwinds the indexer by deleting all indexed data greater than the provided L1 block number. + /// Unwinds the chain orchestrator by deleting all indexed data greater than the provided L1 + /// block number. async fn unwind( &self, genesis_hash: B256, diff --git a/crates/database/db/src/test_utils.rs b/crates/database/db/src/test_utils.rs index fd1a036c..505f9350 100644 --- a/crates/database/db/src/test_utils.rs +++ b/crates/database/db/src/test_utils.rs @@ -1,13 +1,13 @@ //! Test utilities for the database crate. use super::Database; -use scroll_migration::{Migrator, MigratorTrait}; +use scroll_migration::{Migrator, MigratorTrait, ScrollDevMigrationInfo}; /// Instantiates a new in-memory database and runs the migrations /// to set up the schema. pub async fn setup_test_db() -> Database { let database_url = "sqlite::memory:"; let connection = sea_orm::Database::connect(database_url).await.unwrap(); - Migrator::<()>::up(&connection, None).await.unwrap(); + Migrator::::up(&connection, None).await.unwrap(); connection.into() } diff --git a/crates/database/migration/src/lib.rs b/crates/database/migration/src/lib.rs index d64688d0..6172ca6f 100644 --- a/crates/database/migration/src/lib.rs +++ b/crates/database/migration/src/lib.rs @@ -7,7 +7,9 @@ mod m20250408_150338_load_header_metadata; mod m20250411_072004_add_l2_block; mod m20250616_223947_add_metadata; mod migration_info; -pub use migration_info::{MigrationInfo, ScrollMainnetMigrationInfo, ScrollSepoliaMigrationInfo}; +pub use migration_info::{ + MigrationInfo, ScrollDevMigrationInfo, ScrollMainnetMigrationInfo, ScrollSepoliaMigrationInfo, +}; pub struct Migrator(pub std::marker::PhantomData); @@ -27,8 +29,8 @@ impl MigratorTrait for Migrator { pub mod traits { use crate::{ - migration_info::ScrollMainnetTestMigrationInfo, ScrollMainnetMigrationInfo, - ScrollSepoliaMigrationInfo, + migration_info::{ScrollDevMigrationInfo, ScrollMainnetTestMigrationInfo}, + ScrollMainnetMigrationInfo, ScrollSepoliaMigrationInfo, }; use reth_chainspec::NamedChain; use sea_orm::{prelude::async_trait::async_trait, DatabaseConnection, DbErr}; @@ -54,7 +56,9 @@ pub mod traits { (NamedChain::ScrollSepolia, _) => { Ok(super::Migrator::::up(conn, None)) } - (NamedChain::Dev, _) => Ok(super::Migrator::<()>::up(conn, None)), + (NamedChain::Dev, _) => { + Ok(super::Migrator::::up(conn, None)) + } _ => Err(DbErr::Custom("expected Scroll Mainnet, Sepolia or Dev".into())), }? .await diff --git a/crates/database/migration/src/migration_info.rs b/crates/database/migration/src/migration_info.rs index 6ed32c46..ebdd03bf 100644 --- a/crates/database/migration/src/migration_info.rs +++ b/crates/database/migration/src/migration_info.rs @@ -12,7 +12,9 @@ pub trait MigrationInfo { fn genesis_hash() -> B256; } -impl MigrationInfo for () { +pub struct ScrollDevMigrationInfo; + +impl MigrationInfo for ScrollDevMigrationInfo { fn data_source() -> Option { None } @@ -22,7 +24,6 @@ impl MigrationInfo for () { } fn genesis_hash() -> B256 { - // Todo: Update b256!("0xc77ee681dac901672fee660088df30ef11789ec89837123cdc89690ef1fef766") } } diff --git a/crates/engine/src/driver.rs b/crates/engine/src/driver.rs index 3f920e7c..c934bc0c 100644 --- a/crates/engine/src/driver.rs +++ b/crates/engine/src/driver.rs @@ -123,7 +123,7 @@ where /// Handles a block import request by adding it to the queue and waking up the driver. pub fn handle_chain_import(&mut self, chain_import: ChainImport) { - tracing::trace!(target: "scroll::engine", head = %chain_import.chain.last().unwrap().hash_slow(), "new block import request received"); + tracing::trace!(target: "scroll::engine", head = %chain_import.chain.last().unwrap().hash_slow(), "new chain import request received"); self.chain_imports.push_back(chain_import); self.waker.wake(); diff --git a/crates/indexer/src/event.rs b/crates/indexer/src/event.rs deleted file mode 100644 index 64288bae..00000000 --- a/crates/indexer/src/event.rs +++ /dev/null @@ -1,38 +0,0 @@ -use alloy_primitives::B256; -use rollup_node_primitives::{BatchInfo, BlockInfo, L2BlockInfoWithL1Messages}; - -/// An event emitted by the indexer. -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum IndexerEvent { - /// 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 `BatchFinalization` event has been indexed returning the batch hash and new finalized L2 - /// block. - BatchFinalizationIndexed(B256, Option), - /// A `Finalized` event has been indexed returning the block number and new finalized L2 - /// block. - FinalizedIndexed(u64, Option), - /// A `L1Message` event has been indexed returning the message queue index. - L1MessageIndexed(u64), - /// A `Unwind` event has been indexed returning the reorg block number. - UnwindIndexed { - /// 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, - }, - /// A block has been indexed returning batch and block info. - BlockIndexed(L2BlockInfoWithL1Messages, Option), -} diff --git a/crates/indexer/src/lib.rs b/crates/indexer/src/lib.rs deleted file mode 100644 index 0a0afcc8..00000000 --- a/crates/indexer/src/lib.rs +++ /dev/null @@ -1,793 +0,0 @@ -//! A library responsible for indexing data relevant to the L1. - -use alloy_primitives::{b256, keccak256, B256}; -use futures::Stream; -use reth_chainspec::EthChainSpec; -use rollup_node_primitives::{ - BatchCommitData, BatchInfo, BlockInfo, L1MessageEnvelope, L2BlockInfoWithL1Messages, -}; -use rollup_node_watcher::L1Notification; -use scroll_alloy_consensus::TxL1Message; -use scroll_alloy_hardforks::{ScrollHardfork, ScrollHardforks}; -use scroll_db::{Database, DatabaseError, DatabaseOperations, UnwindResult}; -use std::{ - collections::{HashMap, VecDeque}, - pin::Pin, - sync::{ - atomic::{AtomicU64, Ordering}, - Arc, - }, - task::{Context, Poll}, - time::Instant, -}; -use strum::IntoEnumIterator; - -mod action; -use action::{IndexerFuture, PendingIndexerFuture}; - -mod event; -pub use event::IndexerEvent; - -mod error; -pub use error::IndexerError; - -mod metrics; -pub use metrics::{IndexerItem, IndexerMetrics}; - -const L1_MESSAGE_QUEUE_HASH_MASK: B256 = - b256!("ffffffffffffffffffffffffffffffffffffffffffffffffffffffff00000000"); - -/// The indexer is responsible for indexing data relevant to the L1. -#[derive(Debug)] -pub struct Indexer { - /// 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 indexer. - chain_spec: Arc, - /// The metrics for the indexer. - metrics: HashMap, -} - -impl Indexer { - /// Creates a new indexer with the given [`Database`]. - pub fn new(database: Arc, chain_spec: Arc) -> Self { - Self { - 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: IndexerItem::iter() - .map(|i| { - let label = i.as_str(); - (i, IndexerMetrics::new_with_labels(&[("item", label)])) - }) - .collect(), - } - } - - /// Wraps a pending indexer future, metering the completion of it. - pub fn handle_metered( - &mut self, - item: IndexerItem, - indexer_fut: PendingIndexerFuture, - ) -> PendingIndexerFuture { - let metric = self.metrics.get(&item).expect("metric exists").clone(); - let fut_wrapper = Box::pin(async move { - let now = Instant::now(); - let res = indexer_fut.await; - metric.task_duration.record(now.elapsed().as_secs_f64()); - res - }); - fut_wrapper - } - - /// Handles an L2 block. - pub fn handle_block( - &mut self, - block_info: L2BlockInfoWithL1Messages, - batch_info: Option, - ) { - let database = self.database.clone(); - let fut = self.handle_metered( - IndexerItem::L2Block, - Box::pin(async move { - database.insert_block(block_info.clone(), batch_info).await?; - Result::<_, IndexerError>::Ok(IndexerEvent::BlockIndexed(block_info, batch_info)) - }), - ); - - self.pending_futures.push_back(IndexerFuture::HandleDerivedBlock(fut)) - } - - /// Handles an event from the L1. - pub fn handle_l1_notification(&mut self, event: L1Notification) { - let fut = match event { - L1Notification::Reorg(block_number) => IndexerFuture::HandleReorg(self.handle_metered( - IndexerItem::L1Reorg, - Box::pin(Self::handle_l1_reorg( - self.database.clone(), - self.chain_spec.clone(), - block_number, - )), - )), - L1Notification::NewBlock(_) | L1Notification::Consensus(_) => return, - L1Notification::Finalized(block_number) => { - IndexerFuture::HandleFinalized(self.handle_metered( - IndexerItem::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) => { - IndexerFuture::HandleBatchCommit(self.handle_metered( - IndexerItem::BatchCommit, - Box::pin(Self::handle_batch_commit(self.database.clone(), batch)), - )) - } - L1Notification::L1Message { message, block_number, block_timestamp } => { - IndexerFuture::HandleL1Message(self.handle_metered( - IndexerItem::L1Message, - Box::pin(Self::handle_l1_message( - self.database.clone(), - self.chain_spec.clone(), - message, - block_number, - block_timestamp, - )), - )) - } - L1Notification::BatchFinalization { hash, block_number, .. } => { - IndexerFuture::HandleBatchFinalization(self.handle_metered( - IndexerItem::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(), - )), - )) - } - }; - - 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, - ) -> Result { - let txn = database.tx().await?; - let UnwindResult { l1_block_number, queue_index, l2_head_block_info, l2_safe_block_info } = - txn.unwind(chain_spec.genesis_hash(), l1_block_number).await?; - txn.commit().await?; - Ok(IndexerEvent::UnwindIndexed { - l1_block_number, - queue_index, - l2_head_block_info, - l2_safe_block_info, - }) - } - - /// Handles a finalized event by updating the indexer 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 { - // 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 indexer l1 block number. - l1_block_number.store(block_number, Ordering::Relaxed); - - Ok(IndexerEvent::FinalizedIndexed(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 { - let event = IndexerEvent::L1MessageIndexed(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(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(event) - } - - /// Handles a batch input by inserting it into the database. - async fn handle_batch_commit( - database: Arc, - batch: BatchCommitData, - ) -> Result { - 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 = IndexerEvent::BatchCommitIndexed { - batch_info: BatchInfo::new(batch.index, batch.hash), - safe_head: new_safe_head, - l1_block_number: batch.block_number, - }; - - // insert the batch and commit the transaction. - txn.insert_batch(batch).await?; - txn.commit().await?; - - Ok(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 { - // 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 = IndexerEvent::BatchFinalizationIndexed(batch_hash, finalized_block); - Ok(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, IndexerError> { - 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 - } - })) - } -} - -impl Stream for Indexer { - type Item = Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - // Remove and poll the next future in the queue - if let Some(mut action) = self.pending_futures.pop_front() { - return match action.poll(cx) { - Poll::Ready(result) => Poll::Ready(Some(result)), - Poll::Pending => { - self.pending_futures.push_front(action); - Poll::Pending - } - }; - } - - Poll::Pending - } -} - -#[cfg(test)] -mod test { - use std::vec; - - use super::*; - use alloy_primitives::{address, bytes, U256}; - - use arbitrary::{Arbitrary, Unstructured}; - use futures::StreamExt; - use rand::Rng; - use reth_scroll_chainspec::{ScrollChainSpec, SCROLL_MAINNET}; - use rollup_node_primitives::BatchCommitData; - use scroll_db::test_utils::setup_test_db; - - async fn setup_test_indexer() -> (Indexer, Arc) { - let db = Arc::new(setup_test_db().await); - (Indexer::new(db.clone(), SCROLL_MAINNET.clone()), db) - } - - #[tokio::test] - async fn test_handle_commit_batch() { - // Instantiate indexer and db - let (mut indexer, db) = setup_test_indexer().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 { - IndexerEvent::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 indexer, db) = setup_test_indexer().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 - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_1.clone())); - let event = indexer.next().await.unwrap().unwrap(); - match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 100); - assert_eq!(safe_head, None); - } - _ => panic!("Expected BatchCommitIndexed event"), - } - - // Index second batch - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_2.clone())); - let event = indexer.next().await.unwrap().unwrap(); - match event { - IndexerEvent::BatchCommitIndexed { batch_info, safe_head, .. } => { - assert_eq!(batch_info.index, 101); - assert_eq!(safe_head, None); - } - _ => panic!("Expected BatchCommitIndexed event"), - } - - // Index third batch - indexer.handle_l1_notification(L1Notification::BatchCommit(batch_3.clone())); - let event = indexer.next().await.unwrap().unwrap(); - match event { - IndexerEvent::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![], - }; - - indexer.handle_block(block_1.clone(), Some(batch_1_info)); - indexer.next().await.unwrap().unwrap(); - - indexer.handle_block(block_2.clone(), Some(batch_2_info)); - indexer.next().await.unwrap().unwrap(); - - indexer.handle_block(block_3.clone(), Some(batch_2_info)); - indexer.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() - }; - - indexer.handle_l1_notification(L1Notification::BatchCommit(new_batch_2.clone())); - let event = indexer.next().await.unwrap().unwrap(); - - // Verify the event indicates a batch revert - match event { - IndexerEvent::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_indexer().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_indexer().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_indexer().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!(2, 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().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)); - } - - #[tokio::test] - async fn test_handle_reorg_executed_l1_messages() { - // Instantiate indexer and db - let (mut indexer, _database) = setup_test_indexer().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 - }; - indexer.handle_block(l2_block.clone(), batch_info); - 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, - IndexerEvent::UnwindIndexed { - 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, - IndexerEvent::UnwindIndexed { - 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, - IndexerEvent::UnwindIndexed { - 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/manager/src/manager/mod.rs b/crates/manager/src/manager/mod.rs index 5f1ed30e..4f05073d 100644 --- a/crates/manager/src/manager/mod.rs +++ b/crates/manager/src/manager/mod.rs @@ -127,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.chain) + .field("chain_orchestrator", &self.chain) .field("consensus", &self.consensus) .field("eth_wire_block_rx", &"eth_wire_block_rx") .field("event_sender", &self.event_sender) @@ -229,7 +229,7 @@ where } } - /// Handles an indexer event. + /// Handles a chain orchestrator event. fn handle_chain_orchestrator_event(&mut self, event: ChainOrchestratorEvent) { trace!(target: "scroll::node::manager", ?event, "Received chain orchestrator event"); @@ -316,6 +316,33 @@ where } } + /// Handles a chain orchestrator error. + fn handle_chain_orchestrator_error(&self, err: &ChainOrchestratorError) { + error!( + target: "scroll::node::manager", + ?err, + "Error occurred in the chain orchestrator" + ); + match err { + ChainOrchestratorError::L1MessageMismatch { expected, actual } => { + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::L1MessageConsolidationError { + expected: *expected, + actual: *actual, + }); + } + } + ChainOrchestratorError::DatabaseError(DatabaseError::L1MessageNotFound(start)) => { + if let Some(event_sender) = self.event_sender.as_ref() { + event_sender.notify(RollupManagerEvent::L1MessageMissingInDatabase { + start: start.clone(), + }); + } + } + _ => {} + } + } + /// Handles an engine driver event. fn handle_engine_driver_event(&mut self, event: EngineDriverEvent) { trace!(target: "scroll::node::manager", ?event, "Received engine driver event"); @@ -480,41 +507,12 @@ where } ); - // Drain all Indexer events. + // Drain all chain orchestrator events. while let Poll::Ready(Some(result)) = this.chain.poll_next_unpin(cx) { match result { Ok(event) => this.handle_chain_orchestrator_event(event), Err(err) => { - 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" - ); + this.handle_chain_orchestrator_error(&err); } } } diff --git a/crates/network/src/manager.rs b/crates/network/src/manager.rs index 2b10d4ac..d0de113f 100644 --- a/crates/network/src/manager.rs +++ b/crates/network/src/manager.rs @@ -55,8 +55,6 @@ pub struct ScrollNetworkManager { 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 @@ -69,7 +67,6 @@ impl 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); @@ -99,7 +96,6 @@ impl from_handle_rx: from_handle_rx.into(), scroll_wire, eth_wire_listener, - eth_wire_gossip, } } } @@ -118,7 +114,6 @@ impl< 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(); @@ -134,7 +129,6 @@ impl< from_handle_rx: from_handle_rx.into(), scroll_wire, eth_wire_listener, - eth_wire_gossip, } } @@ -161,15 +155,13 @@ impl< .filter_map(|(peer_id, blocks)| (!blocks.contains(&hash)).then_some(*peer_id)) .collect(); - 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); - } + 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 { @@ -272,7 +264,9 @@ impl< { 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"); + tracing::warn!(target: "scroll::bridge::import", peer_id = %peer_id, "Failed to extract signature from block extra data, penalizing peer"); + self.inner_network_handle() + .reputation_change(peer_id, reth_network_api::ReputationChangeKind::BadBlock); None } } diff --git a/crates/node/src/args.rs b/crates/node/src/args.rs index e16e59bb..f0b94483 100644 --- a/crates/node/src/args.rs +++ b/crates/node/src/args.rs @@ -215,7 +215,6 @@ impl ScrollRollupNodeConfig { ctx.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 @@ -469,19 +468,11 @@ 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, - eth_wire_gossip: false, - } + Self { enable_eth_scroll_wire_bridge: true, enable_scroll_wire: true, sequencer_url: None } } } diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 0b7b89a8..6521f18a 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -137,7 +137,6 @@ 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(), @@ -493,7 +492,6 @@ async fn can_bridge_blocks() { network_config, scroll_wire_config, None, - false, ) .await; let scroll_network_handle = scroll_network.handle(); @@ -968,8 +966,7 @@ async fn can_gossip_over_eth_wire() -> eyre::Result<()> { let chain_spec = (*SCROLL_DEV).clone(); // Setup the rollup node manager. - let mut config = default_sequencer_test_scroll_rollup_node_config(); - config.network_args.eth_wire_gossip = true; + let config = default_sequencer_test_scroll_rollup_node_config(); let (mut nodes, _tasks, _) = setup_engine(config, 2, chain_spec.clone(), false, false).await.unwrap(); let _sequencer = nodes.pop().unwrap(); diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index a771cf98..e216677f 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -44,7 +44,6 @@ 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 { @@ -186,7 +185,6 @@ async fn test_should_consolidate_after_optimistic_sync() -> eyre::Result<()> { 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(), @@ -434,7 +432,6 @@ async fn test_consolidation() -> eyre::Result<()> { 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(), diff --git a/crates/scroll-wire/src/protocol/proto.rs b/crates/scroll-wire/src/protocol/proto.rs index 473fb837..92472371 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 blocks. + /// Returns a [`NewBlock`] instance with the provided signature and block. pub fn new(signature: Signature, block: reth_scroll_primitives::ScrollBlock) -> Self { Self { signature: Bytes::from(Into::>::into(signature)), block } } diff --git a/docker-compose/resource/dashboards/rollup_node.json b/docker-compose/resource/dashboards/rollup_node.json index 99cc09f6..08a1cdd8 100644 --- a/docker-compose/resource/dashboards/rollup_node.json +++ b/docker-compose/resource/dashboards/rollup_node.json @@ -287,7 +287,7 @@ "type": "prometheus", "uid": "PBFA97CFB590B2093" }, - "description": "The average duration of handling indexer tasks.", + "description": "The average duration of handling chain orchestrator tasks.", "fieldConfig": { "defaults": { "color": { From 4383231b4a249e066a92e017af7b599b835ea1df Mon Sep 17 00:00:00 2001 From: frisitano Date: Wed, 20 Aug 2025 16:59:03 +0800 Subject: [PATCH 29/30] merge changes --- Cargo.lock | 4 ++-- crates/database/migration/src/migration_info.rs | 2 +- crates/node/tests/e2e.rs | 12 ++++++++---- crates/node/tests/sync.rs | 4 +--- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 42b5bec7..288cb407 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10408,8 +10408,8 @@ dependencies = [ name = "rollup-node-chain-orchestrator" version = "0.0.1" dependencies = [ - "alloy-consensus 1.0.23", - "alloy-eips 1.0.23", + "alloy-consensus 1.0.24", + "alloy-eips 1.0.24", "alloy-json-rpc", "alloy-primitives", "alloy-provider", diff --git a/crates/database/migration/src/migration_info.rs b/crates/database/migration/src/migration_info.rs index ebdd03bf..a16749d3 100644 --- a/crates/database/migration/src/migration_info.rs +++ b/crates/database/migration/src/migration_info.rs @@ -24,7 +24,7 @@ impl MigrationInfo for ScrollDevMigrationInfo { } fn genesis_hash() -> B256 { - b256!("0xc77ee681dac901672fee660088df30ef11789ec89837123cdc89690ef1fef766") + b256!("0x14844a4fc967096c628e90df3bb0c3e98941bdd31d1982c2f3e70ed17250d98b") } } diff --git a/crates/node/tests/e2e.rs b/crates/node/tests/e2e.rs index 03ee0488..448e7d98 100644 --- a/crates/node/tests/e2e.rs +++ b/crates/node/tests/e2e.rs @@ -24,7 +24,7 @@ use rollup_node::{ RollupNodeContext, ScrollRollupNodeConfig, SequencerArgs, }; use rollup_node_chain_orchestrator::ChainOrchestratorEvent; -use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent, RollupManagerHandle}; +use rollup_node_manager::{RollupManagerCommand, RollupManagerEvent}; use rollup_node_primitives::{sig_encode_hash, BatchCommitData, ConsensusUpdate}; use rollup_node_providers::BlobSource; use rollup_node_sequencer::L1MessageInclusionMode; @@ -257,6 +257,7 @@ async fn can_penalize_peer_for_invalid_block() { signer_args: Default::default(), gas_price_oracle_args: GasPriceOracleArgs::default(), consensus_args: ConsensusArgs::noop(), + chain_orchestrator_args: ChainOrchestratorArgs::default(), }; let (nodes, _tasks, _) = @@ -275,7 +276,10 @@ async fn can_penalize_peer_for_invalid_block() { assert_eq!(initial_reputation, 0); // create invalid block - let block = ScrollBlock::default(); + let mut block = ScrollBlock::default(); + block.header.number = 1; + block.header.parent_hash = + b256!("0x14844a4fc967096c628e90df3bb0c3e98941bdd31d1982c2f3e70ed17250d98b"); // send invalid block from node0 to node1. We don't care about the signature here since we use a // NoopConsensus in the test. @@ -1023,7 +1027,7 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { l1_watcher_tx.send(Arc::new(L1Notification::NewBlock(10))).await?; // Wait for block that contains the L1 message. - sequencer_rnm_handle.build_block().await; + handle.build_block().await; let l2_reorged_height; loop { if let Some(RollupManagerEvent::BlockSequenced(block)) = rnm_events.next().await { @@ -1045,7 +1049,7 @@ async fn can_handle_reorgs_while_sequencing() -> eyre::Result<()> { } // Get the next sequenced L2 block. - sequencer_rnm_handle.build_block().await; + handle.build_block().await; loop { if let Some(RollupManagerEvent::BlockSequenced(block)) = rnm_events.next().await { assert_eq!(block.number, l2_reorged_height); diff --git a/crates/node/tests/sync.rs b/crates/node/tests/sync.rs index e216677f..f7b2487d 100644 --- a/crates/node/tests/sync.rs +++ b/crates/node/tests/sync.rs @@ -642,9 +642,7 @@ async fn test_chain_orchestrator_shallow_reorg_with_gap() -> eyre::Result<()> { 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; + sequencer.connect(&mut follower).await; // initially the sequencer should build 100 empty blocks and the follower should follow them let mut reorg_block_info = BlockInfo::default(); From 5d9aefba6a080d38b60f12a31834922f991ce791 Mon Sep 17 00:00:00 2001 From: frisitano Date: Mon, 25 Aug 2025 17:53:24 +0800 Subject: [PATCH 30/30] add migration script --- crates/database/migration/src/lib.rs | 2 ++ ...20250825_093350_remove_unsafe_l2_blocks.rs | 28 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 crates/database/migration/src/m20250825_093350_remove_unsafe_l2_blocks.rs diff --git a/crates/database/migration/src/lib.rs b/crates/database/migration/src/lib.rs index 6172ca6f..00681b49 100644 --- a/crates/database/migration/src/lib.rs +++ b/crates/database/migration/src/lib.rs @@ -6,6 +6,7 @@ mod m20250408_132123_add_header_metadata; mod m20250408_150338_load_header_metadata; mod m20250411_072004_add_l2_block; mod m20250616_223947_add_metadata; +mod m20250825_093350_remove_unsafe_l2_blocks; mod migration_info; pub use migration_info::{ MigrationInfo, ScrollDevMigrationInfo, ScrollMainnetMigrationInfo, ScrollSepoliaMigrationInfo, @@ -23,6 +24,7 @@ impl MigratorTrait for Migrator { Box::new(m20250408_150338_load_header_metadata::Migration::(Default::default())), Box::new(m20250411_072004_add_l2_block::Migration::(Default::default())), Box::new(m20250616_223947_add_metadata::Migration), + Box::new(m20250825_093350_remove_unsafe_l2_blocks::Migration), ] } } diff --git a/crates/database/migration/src/m20250825_093350_remove_unsafe_l2_blocks.rs b/crates/database/migration/src/m20250825_093350_remove_unsafe_l2_blocks.rs new file mode 100644 index 00000000..f112074d --- /dev/null +++ b/crates/database/migration/src/m20250825_093350_remove_unsafe_l2_blocks.rs @@ -0,0 +1,28 @@ +use sea_orm::Statement; +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .get_connection() + .execute(Statement::from_sql_and_values( + manager.get_database_backend(), + r#" + DELETE FROM l2_block + WHERE batch_index IS NULL + "#, + [], + )) + .await?; + + Ok(()) + } + + async fn down(&self, _manager: &SchemaManager) -> Result<(), DbErr> { + Ok(()) + } +}