From 458ee1a879230596cb0fb3e34b3b236a43233507 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 10 Apr 2024 18:03:07 +0200 Subject: [PATCH 01/53] dont set finalized on op-reth start --- bin/reth/src/optimism.rs | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/bin/reth/src/optimism.rs b/bin/reth/src/optimism.rs index 0c0a483dd41f..2bd2ffa9bffa 100644 --- a/bin/reth/src/optimism.rs +++ b/bin/reth/src/optimism.rs @@ -42,28 +42,6 @@ fn main() { .launch() .await?; - // If `enable_genesis_walkback` is set to true, the rollup client will need to - // perform the derivation pipeline from genesis, validating the data dir. - // When set to false, set the finalized, safe, and unsafe head block hashes - // on the rollup client using a fork choice update. This prevents the rollup - // client from performing the derivation pipeline from genesis, and instead - // starts syncing from the current tip in the DB. - if node.chain_spec().is_optimism() && !rollup_args.enable_genesis_walkback { - let client = node.rpc_server_handles.auth.http_client(); - if let Ok(Some(head)) = node.provider.latest_header() { - reth_rpc_api::EngineApiClient::::fork_choice_updated_v2( - &client, - reth_rpc_types::engine::ForkchoiceState { - head_block_hash: head.hash(), - safe_block_hash: head.hash(), - finalized_block_hash: head.hash(), - }, - None, - ) - .await?; - } - } - node_exit_future.await }) { eprintln!("Error: {err:?}"); From 1704948ec4cecc1d37d894b23c0f17563a352ecb Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 10 Apr 2024 18:07:11 +0200 Subject: [PATCH 02/53] make UnknownSafeOrFinalizedBlock error with upper case --- crates/rpc/rpc/src/eth/error.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/crates/rpc/rpc/src/eth/error.rs b/crates/rpc/rpc/src/eth/error.rs index d8add639726a..0e52ec8c3248 100644 --- a/crates/rpc/rpc/src/eth/error.rs +++ b/crates/rpc/rpc/src/eth/error.rs @@ -43,7 +43,12 @@ pub enum EthApiError { UnknownBlockNumber, /// Thrown when querying for `finalized` or `safe` block before the merge transition is /// finalized, - #[error("unknown block")] + /// + /// op-node uses case sensitive string comparison to parse this error: + /// https://github.com/ethereum-optimism/optimism/blob/0913776869f6cb2c1218497463d7377cf4de16de/op-service/sources/l2_client.go#L105 + /// + /// Temporary, until a version of https://github.com/ethereum-optimism/optimism/pull/10071 is pushed through that doesn't require this to figure out the EL sync status. + #[error("Unknown block")] UnknownSafeOrFinalizedBlock, /// Thrown when an unknown block or transaction index is encountered #[error("unknown block or tx index")] From 12f5ee95df964b19269d6932b75cca8ca0fda505 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 10 Apr 2024 19:07:16 +0200 Subject: [PATCH 03/53] sync to head if finalized does not exist on optimism pipeline --- crates/consensus/beacon/src/engine/mod.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index e0af48a348dd..e4783d3ed566 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -677,6 +677,10 @@ where // threshold return Some(state.finalized_block_hash) } + #[cfg(feature = "optimism")] + { + return Some(state.head_block_hash) + } } Ok(Some(_)) => { // we're fully synced to the finalized block From 7df63dbf238a49edcb7f6e80863ccd5306b89f02 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 18 Apr 2024 17:59:07 +0200 Subject: [PATCH 04/53] add some docs and scenario on finished pipeline --- crates/consensus/beacon/src/engine/mod.rs | 40 ++++++++++++++++++----- 1 file changed, 31 insertions(+), 9 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index e4783d3ed566..0274118ce219 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -677,8 +677,19 @@ where // threshold return Some(state.finalized_block_hash) } + #[cfg(feature = "optimism")] - { + if self.blockchain.chain_spec().is_optimism() { + // It can happen when the node is doing an + // optimistic sync, where the CL has no knowledge of the finalized hash, + // but is expecting the EL to sync as high + // as possible before finalizing. + // + // This usually doesn't happen on ETH mainnet since CLs use the more + // secure checkpoint syncing. + // + // However, optimism chains will do this. The risk of a reorg is however + // low. return Some(state.head_block_hash) } } @@ -1633,11 +1644,24 @@ where return Ok(()) } + #[cfg(feature = "optimism")] + let target_block_hash = if sync_target_state.finalized_block_hash.is_zero() && + self.blockchain.chain_spec().is_optimism() + { + // We are in an optimistic syncing scenario. + // + // Check fn can_pipeline_sync_to_finalized(..) for more context + sync_target_state.head_block_hash + } else { + sync_target_state.finalized_block_hash + }; + + #[cfg(not(feature = "optimism"))] + let target_block_hash = sync_target_state.finalized_block_hash; + // get the block number of the finalized block, if we have it - let newest_finalized = self - .blockchain - .buffered_header_by_hash(sync_target_state.finalized_block_hash) - .map(|header| header.number); + let newest_finalized = + self.blockchain.buffered_header_by_hash(target_block_hash).map(|header| header.number); // The block number that the pipeline finished at - if the progress or newest // finalized is None then we can't check the distance anyways. @@ -1657,9 +1681,7 @@ where if let Some(target) = pipeline_target { // run the pipeline to the target since the distance is sufficient self.sync.set_pipeline_sync_target(target); - } else if let Some(number) = - self.blockchain.block_number(sync_target_state.finalized_block_hash)? - { + } else if let Some(number) = self.blockchain.block_number(target_block_hash)? { // Finalized block is in the database, attempt to restore the tree with // the most recent canonical hashes. self.blockchain.connect_buffered_blocks_to_canonical_hashes_and_finalize(number).inspect_err(|error| { @@ -1668,7 +1690,7 @@ where } else { // We don't have the finalized block in the database, so we need to // trigger another pipeline run. - self.sync.set_pipeline_sync_target(sync_target_state.finalized_block_hash); + self.sync.set_pipeline_sync_target(target_block_hash); } Ok(()) From 42158aa642df8cd57d5853c5341b7eeaa106b5c2 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Sun, 21 Apr 2024 18:56:16 +0200 Subject: [PATCH 05/53] enable optimism feature flags on missing dep --- Cargo.lock | 2 ++ crates/consensus/beacon/Cargo.toml | 3 +++ crates/optimism/node/Cargo.toml | 3 ++- 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 6be45296b304..8a1ec322a265 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6171,6 +6171,7 @@ dependencies = [ "reth-provider", "reth-prune", "reth-revm", + "reth-rpc", "reth-rpc-types", "reth-rpc-types-compat", "reth-stages", @@ -7021,6 +7022,7 @@ dependencies = [ "reqwest 0.11.27", "reth", "reth-basic-payload-builder", + "reth-beacon-consensus", "reth-db", "reth-e2e-test-utils", "reth-evm", diff --git a/crates/consensus/beacon/Cargo.toml b/crates/consensus/beacon/Cargo.toml index f195c98bd950..dee4ec8d53b5 100644 --- a/crates/consensus/beacon/Cargo.toml +++ b/crates/consensus/beacon/Cargo.toml @@ -51,6 +51,7 @@ reth-blockchain-tree = { workspace = true, features = ["test-utils"] } reth-db = { workspace = true, features = ["test-utils"] } reth-provider = { workspace = true, features = ["test-utils"] } reth-rpc-types-compat.workspace = true +reth-rpc.workspace = true reth-tracing.workspace = true reth-revm.workspace = true reth-downloaders.workspace = true @@ -66,4 +67,6 @@ optimism = [ "reth-provider/optimism", "reth-blockchain-tree/optimism", "reth-beacon-consensus-core/optimism", + "reth-revm/optimism", + "reth-rpc/optimism" ] diff --git a/crates/optimism/node/Cargo.toml b/crates/optimism/node/Cargo.toml index d66aabd5d6a0..c760f49bfb07 100644 --- a/crates/optimism/node/Cargo.toml +++ b/crates/optimism/node/Cargo.toml @@ -28,7 +28,7 @@ reth-network.workspace = true reth-interfaces.workspace = true reth-evm.workspace = true reth-revm.workspace = true - +reth-beacon-consensus.workspace = true revm.workspace = true revm-primitives.workspace = true @@ -67,4 +67,5 @@ optimism = [ "reth-rpc/optimism", "reth-revm/optimism", "reth-optimism-payload-builder/optimism", + "reth-beacon-consensus/optimism", ] From de8928a3edf375bea868f1f8848098b80b02000b Mon Sep 17 00:00:00 2001 From: joshieDo Date: Sun, 21 Apr 2024 21:43:34 +0200 Subject: [PATCH 06/53] refactor e2e op tests --- crates/e2e-test-utils/src/engine_api.rs | 16 +++++ crates/e2e-test-utils/src/node.rs | 48 +++++++++++++- crates/e2e-test-utils/src/wallet.rs | 9 ++- crates/node-ethereum/tests/e2e/eth.rs | 4 +- crates/node-ethereum/tests/e2e/p2p.rs | 3 +- crates/optimism/node/tests/e2e/p2p.rs | 79 +++++++--------------- crates/optimism/node/tests/e2e/utils.rs | 87 ++++++++++++++++++++++++- 7 files changed, 179 insertions(+), 67 deletions(-) diff --git a/crates/e2e-test-utils/src/engine_api.rs b/crates/e2e-test-utils/src/engine_api.rs index ec8b058a3007..b810eac98214 100644 --- a/crates/e2e-test-utils/src/engine_api.rs +++ b/crates/e2e-test-utils/src/engine_api.rs @@ -64,4 +64,20 @@ impl EngineApiHelper { Ok(()) } + + /// Sends forkchoice update to the engine api with a zero finalized hash + pub async fn update_optimistic_forkchoice(&self, hash: B256) -> eyre::Result<()> { + EngineApiClient::::fork_choice_updated_v2( + &self.engine_api_client, + ForkchoiceState { + head_block_hash: hash, + safe_block_hash: B256::ZERO, + finalized_block_hash: B256::ZERO, + }, + None, + ) + .await?; + + Ok(()) + } } diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index d88a428f05a1..d04a703eff95 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -4,19 +4,21 @@ use crate::{ }; use alloy_rpc_types::BlockNumberOrTag; use eyre::Ok; +use futures_util::Future; use reth::{ api::{BuiltPayload, EngineTypes, FullNodeComponents, PayloadBuilderAttributes}, builder::FullNode, - providers::{BlockReaderIdExt, CanonStateSubscriptions}, + providers::{BlockReader, BlockReaderIdExt, CanonStateSubscriptions}, rpc::{ eth::{error::EthResult, EthTransactions}, types::engine::PayloadAttributes, }, }; use reth_payload_builder::EthPayloadBuilderAttributes; -use reth_primitives::{Address, BlockNumber, Bytes, B256}; +use reth_primitives::{Address, BlockHash, BlockNumber, Bytes, B256}; use std::{ marker::PhantomData, + pin::Pin, time::{SystemTime, UNIX_EPOCH}, }; use tokio_stream::StreamExt; @@ -52,8 +54,31 @@ where }) } - /// Advances the node forward + /// Advances the chain `length` blocks. + /// + /// Returns the added chain as a Vec of block hashes. pub async fn advance( + &mut self, + length: u64, + tx_generator: impl Fn() -> Pin>>, + attributes_generator: impl Fn(u64) -> ::PayloadBuilderAttributes + + Copy, + ) -> eyre::Result> + where + ::ExecutionPayloadV3: + From<::BuiltPayload> + PayloadEnvelopeExt, + { + let mut chain = Vec::with_capacity(length as usize); + for _ in 0..length { + let (block_hash, _) = + self.advance_block(tx_generator().await, attributes_generator).await?; + chain.push(block_hash); + } + Ok(chain) + } + + /// Advances the node forward one block + pub async fn advance_block( &mut self, raw_tx: Bytes, attributes_generator: impl Fn(u64) -> ::PayloadBuilderAttributes, @@ -92,6 +117,23 @@ where Ok((block_hash, tx_hash)) } + /// Waits for block to be available on node. + pub async fn wait_block( + &self, + number: BlockNumber, + expected_block_hash: BlockHash, + _is_pipeline: bool, + ) -> eyre::Result<()> { + loop { + tokio::time::sleep(std::time::Duration::from_millis(20)).await; + if let Some(latest_block) = self.inner.provider.block_by_number(number)? { + assert_eq!(latest_block.hash_slow(), expected_block_hash); + break + } + } + Ok(()) + } + /// Injects a raw transaction into the node tx pool via RPC server async fn inject_tx(&mut self, raw_tx: Bytes) -> EthResult { let eth_api = self.inner.rpc_registry.eth_api(); diff --git a/crates/e2e-test-utils/src/wallet.rs b/crates/e2e-test-utils/src/wallet.rs index 43fe7555dc7e..d064eede99c1 100644 --- a/crates/e2e-test-utils/src/wallet.rs +++ b/crates/e2e-test-utils/src/wallet.rs @@ -1,11 +1,11 @@ use alloy_network::{eip2718::Encodable2718, EthereumSigner, TransactionBuilder}; use alloy_rpc_types::{TransactionInput, TransactionRequest}; use alloy_signer_wallet::{coins_bip39::English, LocalWallet, MnemonicBuilder}; -use reth_primitives::{Address, Bytes, U256}; +use reth_primitives::{hex, Address, Bytes, U256}; /// One of the accounts of the genesis allocations. pub struct Wallet { inner: LocalWallet, - nonce: u64, + pub nonce: u64, chain_id: u64, } @@ -27,6 +27,11 @@ impl Wallet { self.tx(None).await } + pub async fn optimism_l1_block_info_tx(&mut self) -> Bytes { + let l1_block_info = Bytes::from_static(&hex!("7ef9015aa044bae9d41b8380d781187b426c6fe43df5fb2fb57bd4466ef6a701e1f01e015694deaddeaddeaddeaddeaddeaddeaddeaddead000194420000000000000000000000000000000000001580808408f0d18001b90104015d8eb900000000000000000000000000000000000000000000000000000000008057650000000000000000000000000000000000000000000000000000000063d96d10000000000000000000000000000000000000000000000000000000000009f35273d89754a1e0387b89520d989d3be9c37c1f32495a88faf1ea05c61121ab0d1900000000000000000000000000000000000000000000000000000000000000010000000000000000000000002d679b567db6187c0c8323fa982cfb88b74dbcc7000000000000000000000000000000000000000000000000000000000000083400000000000000000000000000000000000000000000000000000000000f4240")); + self.tx(Some(l1_block_info)).await + } + /// Creates a transaction with data and signs it pub async fn tx(&mut self, data: Option) -> Bytes { let tx = TransactionRequest { diff --git a/crates/node-ethereum/tests/e2e/eth.rs b/crates/node-ethereum/tests/e2e/eth.rs index 6f9eeb999108..e65adec92b9b 100644 --- a/crates/node-ethereum/tests/e2e/eth.rs +++ b/crates/node-ethereum/tests/e2e/eth.rs @@ -44,7 +44,7 @@ async fn can_run_eth_node() -> eyre::Result<()> { let raw_tx = wallet.transfer_tx().await; // make the node advance - node.advance(raw_tx, eth_payload_attributes).await?; + node.advance_block(raw_tx, eth_payload_attributes).await?; Ok(()) } @@ -83,7 +83,7 @@ async fn can_run_eth_node_with_auth_engine_api_over_ipc() -> eyre::Result<()> { let raw_tx = wallet.transfer_tx().await; // make the node advance - node.advance(raw_tx, crate::utils::eth_payload_attributes).await?; + node.advance_block(raw_tx, crate::utils::eth_payload_attributes).await?; Ok(()) } diff --git a/crates/node-ethereum/tests/e2e/p2p.rs b/crates/node-ethereum/tests/e2e/p2p.rs index 940096e189b8..6ec1a6e8e904 100644 --- a/crates/node-ethereum/tests/e2e/p2p.rs +++ b/crates/node-ethereum/tests/e2e/p2p.rs @@ -65,7 +65,8 @@ async fn can_sync() -> eyre::Result<()> { second_node.network.expect_session().await; // Make the first node advance - let (block_hash, tx_hash) = first_node.advance(raw_tx.clone(), eth_payload_attributes).await?; + let (block_hash, tx_hash) = + first_node.advance_block(raw_tx.clone(), eth_payload_attributes).await?; // only send forkchoice update to second node second_node.engine_api.update_forkchoice(block_hash).await?; diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 5fe4daa7bfd3..1579c781ee1c 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,58 +1,17 @@ use std::sync::Arc; -use crate::utils::optimism_payload_attributes; -use reth::{ - args::{DiscoveryArgs, NetworkArgs, RpcServerArgs}, - builder::{NodeBuilder, NodeConfig, NodeHandle}, - tasks::TaskManager, -}; -use reth_e2e_test_utils::{node::NodeHelper, wallet::Wallet}; -use reth_node_optimism::node::OptimismNode; -use reth_primitives::{hex, Bytes, ChainSpecBuilder, Genesis, BASE_MAINNET}; +use crate::utils::{node, optimism_payload_attributes, setup}; +use parking_lot::Mutex; #[tokio::test] async fn can_sync() -> eyre::Result<()> { reth_tracing::init_test_tracing(); - let tasks = TaskManager::current(); - let exec = tasks.executor(); + let (node_config, _tasks, exec, wallet) = setup(); + let wallet = Arc::new(Mutex::new(wallet)); - let genesis: Genesis = serde_json::from_str(include_str!("../assets/genesis.json")).unwrap(); - let chain_spec = Arc::new( - ChainSpecBuilder::default() - .chain(BASE_MAINNET.chain) - .genesis(genesis) - .ecotone_activated() - .build(), - ); - let mut wallet = Wallet::default().with_chain_id(chain_spec.chain.into()); - - let network_config = NetworkArgs { - discovery: DiscoveryArgs { disable_discovery: true, ..DiscoveryArgs::default() }, - ..NetworkArgs::default() - }; - - let node_config = NodeConfig::test() - .with_chain(chain_spec) - .with_network(network_config) - .with_unused_ports() - .with_rpc(RpcServerArgs::default().with_unused_ports().with_http()); - - let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config.clone()) - .testing_node(exec.clone()) - .node(OptimismNode::default()) - .launch() - .await?; - - let mut first_node = NodeHelper::new(node.clone()).await?; - - let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config) - .testing_node(exec) - .node(OptimismNode::default()) - .launch() - .await?; - - let mut second_node = NodeHelper::new(node).await?; + let mut first_node = node(node_config.clone(), exec.clone()).await?; + let mut second_node = node(node_config.clone(), exec.clone()).await?; // Make them peer first_node.network.add_peer(second_node.network.record()).await; @@ -63,18 +22,26 @@ async fn can_sync() -> eyre::Result<()> { second_node.network.expect_session().await; // Taken from optimism tests - let l1_block_info = Bytes::from_static(&hex!("7ef9015aa044bae9d41b8380d781187b426c6fe43df5fb2fb57bd4466ef6a701e1f01e015694deaddeaddeaddeaddeaddeaddeaddeaddead000194420000000000000000000000000000000000001580808408f0d18001b90104015d8eb900000000000000000000000000000000000000000000000000000000008057650000000000000000000000000000000000000000000000000000000063d96d10000000000000000000000000000000000000000000000000000000000009f35273d89754a1e0387b89520d989d3be9c37c1f32495a88faf1ea05c61121ab0d1900000000000000000000000000000000000000000000000000000000000000010000000000000000000000002d679b567db6187c0c8323fa982cfb88b74dbcc7000000000000000000000000000000000000000000000000000000000000083400000000000000000000000000000000000000000000000000000000000f4240")); - - // Make the first node advance - let raw_tx = wallet.tx(Some(l1_block_info)).await; - let (block_hash, tx_hash) = - first_node.advance(raw_tx.clone(), optimism_payload_attributes).await?; + let tip: usize = 300; + let canonical_chain = first_node + .advance( + tip as u64, + || { + let wallet = wallet.clone(); + Box::pin(async move { wallet.lock().optimism_l1_block_info_tx().await }) + }, + optimism_payload_attributes, + ) + .await?; // only send forkchoice update to second node - second_node.engine_api.update_forkchoice(block_hash).await?; + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip - 2]).await?; + second_node.wait_block(tip as u64 - 1, canonical_chain[tip - 2], true).await?; + + tokio::time::sleep(std::time::Duration::from_secs(5)).await; - // expect second node advanced via p2p gossip - second_node.assert_new_block(tx_hash, block_hash, 1).await?; + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip - 1]).await?; + second_node.wait_block(tip as u64, canonical_chain[tip - 1], false).await?; Ok(()) } diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index 1f655502e67b..fdb06ec6ef84 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -1,7 +1,55 @@ -use reth::rpc::types::engine::PayloadAttributes; -use reth_node_optimism::OptimismPayloadBuilderAttributes; +use reth::{ + args::{DiscoveryArgs, NetworkArgs, RpcServerArgs}, + rpc::types::engine::PayloadAttributes, + tasks::{TaskExecutor, TaskManager}, +}; +use reth_e2e_test_utils::{node::NodeHelper, wallet::Wallet}; +use reth_node_builder::{NodeBuilder, NodeConfig, NodeHandle}; +use reth_node_optimism::{OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; -use reth_primitives::{Address, B256}; +use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; +use std::sync::Arc; + +pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { + let tasks = TaskManager::current(); + let exec = tasks.executor(); + + let genesis: Genesis = serde_json::from_str(include_str!("../assets/genesis.json")).unwrap(); + let chain_spec = Arc::new( + ChainSpecBuilder::default() + .chain(BASE_MAINNET.chain) + .genesis(genesis) + .ecotone_activated() + .build(), + ); + let chain_id = chain_spec.chain.into(); + + let network_config = NetworkArgs { + discovery: DiscoveryArgs { disable_discovery: true, ..DiscoveryArgs::default() }, + ..NetworkArgs::default() + }; + + ( + NodeConfig::test() + .with_chain(chain_spec) + .with_network(network_config) + .with_unused_ports() + .with_rpc(RpcServerArgs::default().with_unused_ports().with_http()), + tasks, + exec, + Wallet::default().with_chain_id(chain_id), + ) +} + +pub(crate) async fn node(node_config: NodeConfig, exec: TaskExecutor) -> eyre::Result { + let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config.clone()) + .testing_node(exec.clone()) + .node(OptimismNode::default()) + .launch() + .await?; + + NodeHelper::new(node).await +} /// Helper function to create a new eth payload attributes pub(crate) fn optimism_payload_attributes(timestamp: u64) -> OptimismPayloadBuilderAttributes { @@ -20,3 +68,36 @@ pub(crate) fn optimism_payload_attributes(timestamp: u64) -> OptimismPayloadBuil gas_limit: Some(30_000_000), } } + +// Type alias +type OpNode = NodeHelper< + reth_node_api::FullNodeComponentsAdapter< + reth_node_api::FullNodeTypesAdapter< + OptimismNode, + Arc>, + reth_provider::providers::BlockchainProvider< + Arc>, + reth::blockchain_tree::ShareableBlockchainTree< + Arc>, + reth_revm::EvmProcessorFactory, + >, + >, + >, + reth_transaction_pool::Pool< + reth_transaction_pool::TransactionValidationTaskExecutor< + reth_node_optimism::txpool::OpTransactionValidator< + reth_provider::providers::BlockchainProvider< + Arc>, + reth::blockchain_tree::ShareableBlockchainTree< + Arc>, + reth_revm::EvmProcessorFactory, + >, + >, + reth_transaction_pool::EthPooledTransaction, + >, + >, + reth_transaction_pool::CoinbaseTipOrdering, + reth_transaction_pool::blobstore::DiskFileBlobStore, + >, + >, +>; From 1d51433cd1ec20707deb3ff5962d160ba7a2d448 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 00:29:59 +0200 Subject: [PATCH 07/53] smol clean-up --- crates/optimism/node/tests/e2e/p2p.rs | 31 +++++++++---------------- crates/optimism/node/tests/e2e/utils.rs | 19 ++++++++++++++- 2 files changed, 29 insertions(+), 21 deletions(-) diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 1579c781ee1c..401d95621806 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,7 +1,6 @@ +use crate::utils::{advance_chain, node, setup}; use std::sync::Arc; - -use crate::utils::{node, optimism_payload_attributes, setup}; -use parking_lot::Mutex; +use tokio::sync::Mutex; #[tokio::test] async fn can_sync() -> eyre::Result<()> { @@ -21,27 +20,19 @@ async fn can_sync() -> eyre::Result<()> { first_node.network.expect_session().await; second_node.network.expect_session().await; - // Taken from optimism tests let tip: usize = 300; - let canonical_chain = first_node - .advance( - tip as u64, - || { - let wallet = wallet.clone(); - Box::pin(async move { wallet.lock().optimism_l1_block_info_tx().await }) - }, - optimism_payload_attributes, - ) - .await?; - - // only send forkchoice update to second node - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip - 2]).await?; - second_node.wait_block(tip as u64 - 1, canonical_chain[tip - 2], true).await?; + let tip_index: usize = tip - 1; + + // Create a chain up to block number 300. + let canonical_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?; + + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 1]).await?; + second_node.wait_block(tip as u64 - 1, canonical_chain[tip_index - 1], true).await?; tokio::time::sleep(std::time::Duration::from_secs(5)).await; - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip - 1]).await?; - second_node.wait_block(tip as u64, canonical_chain[tip - 1], false).await?; + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; + second_node.wait_block(tip as u64, canonical_chain[tip_index], false).await?; Ok(()) } diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index fdb06ec6ef84..e7184c49cbec 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -7,8 +7,9 @@ use reth_e2e_test_utils::{node::NodeHelper, wallet::Wallet}; use reth_node_builder::{NodeBuilder, NodeConfig, NodeHandle}; use reth_node_optimism::{OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; -use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; +use reth_primitives::{Address, BlockHash, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; use std::sync::Arc; +use tokio::sync::Mutex; pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { let tasks = TaskManager::current(); @@ -51,6 +52,22 @@ pub(crate) async fn node(node_config: NodeConfig, exec: TaskExecutor) -> eyre::R NodeHelper::new(node).await } +pub(crate) async fn advance_chain( + length: usize, + node: &mut OpNode, + wallet: Arc>, +) -> eyre::Result> { + node.advance( + length as u64, + || { + let wallet = wallet.clone(); + Box::pin(async move { wallet.lock().await.optimism_l1_block_info_tx().await }) + }, + optimism_payload_attributes, + ) + .await +} + /// Helper function to create a new eth payload attributes pub(crate) fn optimism_payload_attributes(timestamp: u64) -> OptimismPayloadBuilderAttributes { let attributes = PayloadAttributes { From bd9a17b5bad7fd340ece4c0747361c21a70f1970 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 14:49:30 +0200 Subject: [PATCH 08/53] add StageCheckpointReader to FullProvider --- crates/storage/provider/src/traits/full.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/storage/provider/src/traits/full.rs b/crates/storage/provider/src/traits/full.rs index e73357f4a7cd..78ef740852ff 100644 --- a/crates/storage/provider/src/traits/full.rs +++ b/crates/storage/provider/src/traits/full.rs @@ -2,7 +2,7 @@ use crate::{ AccountReader, BlockReaderIdExt, CanonStateSubscriptions, ChainSpecProvider, ChangeSetReader, - DatabaseProviderFactory, EvmEnvProvider, StateProviderFactory, + DatabaseProviderFactory, EvmEnvProvider, StageCheckpointReader, StateProviderFactory, }; use reth_db::database::Database; @@ -16,6 +16,7 @@ pub trait FullProvider: + ChainSpecProvider + ChangeSetReader + CanonStateSubscriptions + + StageCheckpointReader + Clone + Unpin + 'static @@ -31,6 +32,7 @@ impl FullProvider for T where + ChainSpecProvider + ChangeSetReader + CanonStateSubscriptions + + StageCheckpointReader + Clone + Unpin + 'static From 72501dedb944e88b7983ccf433ef280d15885e9f Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 14:50:17 +0200 Subject: [PATCH 09/53] add span node identifiers on e2e --- crates/optimism/node/tests/e2e/utils.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index e7184c49cbec..8b3c5e3efdf3 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -42,7 +42,13 @@ pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { ) } -pub(crate) async fn node(node_config: NodeConfig, exec: TaskExecutor) -> eyre::Result { +pub(crate) async fn node( + node_config: NodeConfig, + exec: TaskExecutor, + id: usize, +) -> eyre::Result { + let span = span!(Level::INFO, "node", id); + let _enter = span.enter(); let NodeHandle { node, node_exit_future: _ } = NodeBuilder::new(node_config.clone()) .testing_node(exec.clone()) .node(OptimismNode::default()) From 2c96574c9d4a70cfef429d2b54b97a56f4206868 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 14:53:04 +0200 Subject: [PATCH 10/53] e2e: chain of payloads + optimism reorg --- crates/e2e-test-utils/Cargo.toml | 1 + crates/e2e-test-utils/src/engine_api.rs | 12 +++- crates/e2e-test-utils/src/node.rs | 93 +++++++++++++++---------- crates/e2e-test-utils/src/payload.rs | 2 +- crates/node-ethereum/tests/e2e/p2p.rs | 3 +- crates/optimism/node/tests/e2e/p2p.rs | 42 +++++++---- crates/optimism/node/tests/e2e/utils.rs | 7 +- 7 files changed, 101 insertions(+), 59 deletions(-) diff --git a/crates/e2e-test-utils/Cargo.toml b/crates/e2e-test-utils/Cargo.toml index f32ff029c2ed..77c67c15efce 100644 --- a/crates/e2e-test-utils/Cargo.toml +++ b/crates/e2e-test-utils/Cargo.toml @@ -17,6 +17,7 @@ reth-tracing.workspace = true reth-db.workspace = true reth-rpc.workspace = true reth-payload-builder = { workspace = true, features = ["test-utils"] } +reth-provider.workspace = true jsonrpsee.workspace = true diff --git a/crates/e2e-test-utils/src/engine_api.rs b/crates/e2e-test-utils/src/engine_api.rs index b810eac98214..fe05b0b6893f 100644 --- a/crates/e2e-test-utils/src/engine_api.rs +++ b/crates/e2e-test-utils/src/engine_api.rs @@ -3,7 +3,10 @@ use jsonrpsee::http_client::HttpClient; use reth::{ api::{EngineTypes, PayloadBuilderAttributes}, providers::CanonStateNotificationStream, - rpc::{api::EngineApiClient, types::engine::ForkchoiceState}, + rpc::{ + api::EngineApiClient, + types::engine::{ForkchoiceState, PayloadStatusEnum}, + }, }; use reth_payload_builder::PayloadId; use reth_primitives::B256; @@ -30,6 +33,7 @@ impl EngineApiHelper { &self, payload: E::BuiltPayload, payload_builder_attributes: E::PayloadBuilderAttributes, + expected_status: PayloadStatusEnum, ) -> eyre::Result where E::ExecutionPayloadV3: From + PayloadEnvelopeExt, @@ -45,8 +49,10 @@ impl EngineApiHelper { payload_builder_attributes.parent_beacon_block_root().unwrap(), ) .await?; - assert!(submission.is_valid(), "{}", submission); - Ok(submission.latest_valid_hash.unwrap()) + + assert!(submission.status == expected_status); + + Ok(submission.latest_valid_hash.unwrap_or_default()) } /// Sends forkchoice update to the engine api diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index d04a703eff95..18d147fd913a 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -8,19 +8,14 @@ use futures_util::Future; use reth::{ api::{BuiltPayload, EngineTypes, FullNodeComponents, PayloadBuilderAttributes}, builder::FullNode, - providers::{BlockReader, BlockReaderIdExt, CanonStateSubscriptions}, + providers::{BlockReader, BlockReaderIdExt, CanonStateSubscriptions, StageCheckpointReader}, rpc::{ eth::{error::EthResult, EthTransactions}, - types::engine::PayloadAttributes, + types::engine::PayloadStatusEnum, }, }; -use reth_payload_builder::EthPayloadBuilderAttributes; -use reth_primitives::{Address, BlockHash, BlockNumber, Bytes, B256}; -use std::{ - marker::PhantomData, - pin::Pin, - time::{SystemTime, UNIX_EPOCH}, -}; +use reth_primitives::{stage::StageId, BlockHash, BlockNumber, Bytes, B256}; +use std::{marker::PhantomData, pin::Pin}; use tokio_stream::StreamExt; /// An helper struct to handle node actions @@ -29,7 +24,7 @@ where Node: FullNodeComponents, { pub inner: FullNode, - payload: PayloadHelper, + pub payload: PayloadHelper, pub network: NetworkHelper, pub engine_api: EngineApiHelper, } @@ -54,6 +49,13 @@ where }) } + pub async fn connect(&mut self, node: &mut NodeHelper) { + self.network.add_peer(node.network.record()).await; + node.network.add_peer(self.network.record()).await; + node.network.expect_session().await; + self.network.expect_session().await; + } + /// Advances the chain `length` blocks. /// /// Returns the added chain as a Vec of block hashes. @@ -63,16 +65,21 @@ where tx_generator: impl Fn() -> Pin>>, attributes_generator: impl Fn(u64) -> ::PayloadBuilderAttributes + Copy, - ) -> eyre::Result> + ) -> eyre::Result< + Vec<( + ::BuiltPayload, + ::PayloadBuilderAttributes, + )>, + > where ::ExecutionPayloadV3: From<::BuiltPayload> + PayloadEnvelopeExt, { let mut chain = Vec::with_capacity(length as usize); for _ in 0..length { - let (block_hash, _) = + let (payload, _) = self.advance_block(tx_generator().await, attributes_generator).await?; - chain.push(block_hash); + chain.push(payload); } Ok(chain) } @@ -82,7 +89,13 @@ where &mut self, raw_tx: Bytes, attributes_generator: impl Fn(u64) -> ::PayloadBuilderAttributes, - ) -> eyre::Result<(B256, B256)> + ) -> eyre::Result<( + ( + ::BuiltPayload, + ::PayloadBuilderAttributes, + ), + B256, + )> where ::ExecutionPayloadV3: From<::BuiltPayload> + PayloadEnvelopeExt, @@ -106,15 +119,17 @@ where let payload = self.payload.expect_built_payload().await?; // submit payload via engine api - let block_number = payload.block().number; - let block_hash = self.engine_api.submit_payload(payload, eth_attr.clone()).await?; + let block_hash = self + .engine_api + .submit_payload(payload.clone(), eth_attr.clone(), PayloadStatusEnum::Valid) + .await?; // trigger forkchoice update via engine api to commit the block to the blockchain self.engine_api.update_forkchoice(block_hash).await?; // assert the block has been committed to the blockchain - self.assert_new_block(tx_hash, block_hash, block_number).await?; - Ok((block_hash, tx_hash)) + self.assert_new_block(tx_hash, block_hash, payload.block().number).await?; + Ok(((payload, eth_attr), tx_hash)) } /// Waits for block to be available on node. @@ -122,13 +137,33 @@ where &self, number: BlockNumber, expected_block_hash: BlockHash, - _is_pipeline: bool, + wait_finish_checkpoint: bool, ) -> eyre::Result<()> { + let mut check = !wait_finish_checkpoint; loop { tokio::time::sleep(std::time::Duration::from_millis(20)).await; - if let Some(latest_block) = self.inner.provider.block_by_number(number)? { - assert_eq!(latest_block.hash_slow(), expected_block_hash); - break + + if !check && wait_finish_checkpoint { + if let Some(checkpoint) = + self.inner.provider.get_stage_checkpoint(StageId::Finish)? + { + if checkpoint.block_number >= number { + check = true + } + } + } + + if check { + if let Some(latest_block) = self.inner.provider.block_by_number(number)? { + if latest_block.hash_slow() != expected_block_hash { + // TODO: only if its awaiting a reorg + continue + } + break + } + if wait_finish_checkpoint { + panic!("Finish checkpoint matches, but could not fetch block."); + } } } Ok(()) @@ -171,17 +206,3 @@ where Ok(()) } } - -/// Helper function to create a new eth payload attributes -pub fn eth_payload_attributes() -> EthPayloadBuilderAttributes { - let timestamp = SystemTime::now().duration_since(UNIX_EPOCH).unwrap().as_secs(); - - let attributes = PayloadAttributes { - timestamp, - prev_randao: B256::ZERO, - suggested_fee_recipient: Address::ZERO, - withdrawals: Some(vec![]), - parent_beacon_block_root: Some(B256::ZERO), - }; - EthPayloadBuilderAttributes::new(B256::ZERO, attributes) -} diff --git a/crates/e2e-test-utils/src/payload.rs b/crates/e2e-test-utils/src/payload.rs index 37138cdd3ebf..2d349721b232 100644 --- a/crates/e2e-test-utils/src/payload.rs +++ b/crates/e2e-test-utils/src/payload.rs @@ -7,7 +7,7 @@ use tokio_stream::wrappers::BroadcastStream; pub struct PayloadHelper { pub payload_event_stream: BroadcastStream>, payload_builder: PayloadBuilderHandle, - timestamp: u64, + pub timestamp: u64, } impl PayloadHelper { diff --git a/crates/node-ethereum/tests/e2e/p2p.rs b/crates/node-ethereum/tests/e2e/p2p.rs index 6ec1a6e8e904..c05cc6085370 100644 --- a/crates/node-ethereum/tests/e2e/p2p.rs +++ b/crates/node-ethereum/tests/e2e/p2p.rs @@ -65,8 +65,9 @@ async fn can_sync() -> eyre::Result<()> { second_node.network.expect_session().await; // Make the first node advance - let (block_hash, tx_hash) = + let ((payload, _), tx_hash) = first_node.advance_block(raw_tx.clone(), eth_payload_attributes).await?; + let block_hash = payload.block().hash(); // only send forkchoice update to second node second_node.engine_api.update_forkchoice(block_hash).await?; diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 401d95621806..94bf21422e99 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -9,30 +9,42 @@ async fn can_sync() -> eyre::Result<()> { let (node_config, _tasks, exec, wallet) = setup(); let wallet = Arc::new(Mutex::new(wallet)); - let mut first_node = node(node_config.clone(), exec.clone()).await?; - let mut second_node = node(node_config.clone(), exec.clone()).await?; + let mut first_node = node(node_config.clone(), exec.clone(), 1).await?; + let mut second_node = node(node_config.clone(), exec.clone(), 2).await?; + let mut third_node = node(node_config.clone(), exec.clone(), 3).await?; // Make them peer - first_node.network.add_peer(second_node.network.record()).await; - second_node.network.add_peer(first_node.network.record()).await; - - // Make sure they establish a new session - first_node.network.expect_session().await; - second_node.network.expect_session().await; + first_node.connect(&mut second_node).await; + second_node.connect(&mut third_node).await; + third_node.connect(&mut first_node).await; let tip: usize = 300; let tip_index: usize = tip - 1; - // Create a chain up to block number 300. - let canonical_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?; + // On first node, create a chain up to block number 300a + let canonical_payload_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?; + let canonical_chain = + canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); + + // On second node, sync optimistically up to block number 297a + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 3]).await?; + second_node.wait_block(tip as u64 - 3, canonical_chain[tip_index - 3], true).await?; + + // On third node, sync optimistically up to block number 300a + third_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; + third_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?; - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 1]).await?; - second_node.wait_block(tip as u64 - 1, canonical_chain[tip_index - 1], true).await?; + // On second node, create a side chain: 298b -> 299b -> 300b + wallet.lock().await.nonce -= 3; + second_node.payload.timestamp = first_node.payload.timestamp - 3; // TODO: probably want to make it node agnostic + let side_payload_chain = advance_chain(3, &mut second_node, wallet.clone()).await?; + let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - tokio::time::sleep(std::time::Duration::from_secs(5)).await; + // On third node, cause a 3 block depth re-org + assert!(side_chain[2] != canonical_chain[tip_index]); + third_node.engine_api.update_optimistic_forkchoice(dbg!(side_chain[2])).await?; + third_node.wait_block(side_payload_chain[2].0.block().number, side_chain[2], true).await?; - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; - second_node.wait_block(tip as u64, canonical_chain[tip_index], false).await?; Ok(()) } diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index 8b3c5e3efdf3..6ee2d0dc75b5 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -5,11 +5,12 @@ use reth::{ }; use reth_e2e_test_utils::{node::NodeHelper, wallet::Wallet}; use reth_node_builder::{NodeBuilder, NodeConfig, NodeHandle}; -use reth_node_optimism::{OptimismNode, OptimismPayloadBuilderAttributes}; +use reth_node_optimism::{OptimismBuiltPayload, OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; -use reth_primitives::{Address, BlockHash, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; +use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; use std::sync::Arc; use tokio::sync::Mutex; +use tracing::{span, Level}; pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { let tasks = TaskManager::current(); @@ -62,7 +63,7 @@ pub(crate) async fn advance_chain( length: usize, node: &mut OpNode, wallet: Arc>, -) -> eyre::Result> { +) -> eyre::Result> { node.advance( length as u64, || { From 522d65c56f79f7cc4db0ef4ab1d7e6b7694534bf Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 14:53:33 +0200 Subject: [PATCH 11/53] on pipeline outcome, skip handling if it's optimistic sync but update canonical tip --- crates/consensus/beacon/src/engine/mod.rs | 54 +++++++++++------------ 1 file changed, 25 insertions(+), 29 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index e5cb20038ed3..19fbad8c77ef 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1591,8 +1591,19 @@ where return Ok(()) } + let sync_target_state = match self.forkchoice_state_tracker.sync_target_state() { + Some(current_state) => current_state, + None => { + // This is only possible if the node was run with `debug.tip` + // argument and without CL. + warn!(target: "consensus::engine", "No fork choice state available"); + return Ok(()) + } + }; + // update the canon chain if continuous is enabled - if self.sync.run_pipeline_continuously() { + if self.sync.run_pipeline_continuously() || sync_target_state.finalized_block_hash.is_zero() + { let max_block = ctrl.block_number().unwrap_or_default(); let max_header = self.blockchain.sealed_header(max_block) .inspect_err(|error| { @@ -1600,17 +1611,13 @@ where })? .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; self.blockchain.set_canonical_head(max_header); + self.blockchain.connect_buffered_blocks_to_canonical_hashes_and_finalize(max_block)?; } - let sync_target_state = match self.forkchoice_state_tracker.sync_target_state() { - Some(current_state) => current_state, - None => { - // This is only possible if the node was run with `debug.tip` - // argument and without CL. - warn!(target: "consensus::engine", "No fork choice state available"); - return Ok(()) - } - }; + if sync_target_state.finalized_block_hash.is_zero() { + // We are on a optimistic syncing process, better to wait for the next FCU to handle + return Ok(()) + } // Next, we check if we need to schedule another pipeline run or transition // to live sync via tree. @@ -1644,24 +1651,11 @@ where return Ok(()) } - #[cfg(feature = "optimism")] - let target_block_hash = if sync_target_state.finalized_block_hash.is_zero() && - self.blockchain.chain_spec().is_optimism() - { - // We are in an optimistic syncing scenario. - // - // Check fn can_pipeline_sync_to_finalized(..) for more context - sync_target_state.head_block_hash - } else { - sync_target_state.finalized_block_hash - }; - - #[cfg(not(feature = "optimism"))] - let target_block_hash = sync_target_state.finalized_block_hash; - // get the block number of the finalized block, if we have it - let newest_finalized = - self.blockchain.buffered_header_by_hash(target_block_hash).map(|header| header.number); + let newest_finalized = self + .blockchain + .buffered_header_by_hash(sync_target_state.finalized_block_hash) + .map(|header| header.number); // The block number that the pipeline finished at - if the progress or newest // finalized is None then we can't check the distance anyways. @@ -1681,7 +1675,9 @@ where if let Some(target) = pipeline_target { // run the pipeline to the target since the distance is sufficient self.sync.set_pipeline_sync_target(target); - } else if let Some(number) = self.blockchain.block_number(target_block_hash)? { + } else if let Some(number) = + self.blockchain.block_number(sync_target_state.finalized_block_hash)? + { // Finalized block is in the database, attempt to restore the tree with // the most recent canonical hashes. self.blockchain.connect_buffered_blocks_to_canonical_hashes_and_finalize(number).inspect_err(|error| { @@ -1690,7 +1686,7 @@ where } else { // We don't have the finalized block in the database, so we need to // trigger another pipeline run. - self.sync.set_pipeline_sync_target(target_block_hash); + self.sync.set_pipeline_sync_target(sync_target_state.finalized_block_hash); } Ok(()) From 932043233c55f351ec3e1461f8d416b3afbae76d Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 22 Apr 2024 15:41:34 +0200 Subject: [PATCH 12/53] move node creation and peering into setup --- crates/optimism/node/tests/e2e/p2p.rs | 13 +++------ crates/optimism/node/tests/e2e/utils.rs | 39 ++++++++++++++++++------- 2 files changed, 33 insertions(+), 19 deletions(-) diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 94bf21422e99..885ae00c9ef5 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -6,17 +6,12 @@ use tokio::sync::Mutex; async fn can_sync() -> eyre::Result<()> { reth_tracing::init_test_tracing(); - let (node_config, _tasks, exec, wallet) = setup(); + let (mut nodes, _tasks, exec, wallet) = setup(3).await?; let wallet = Arc::new(Mutex::new(wallet)); - let mut first_node = node(node_config.clone(), exec.clone(), 1).await?; - let mut second_node = node(node_config.clone(), exec.clone(), 2).await?; - let mut third_node = node(node_config.clone(), exec.clone(), 3).await?; - - // Make them peer - first_node.connect(&mut second_node).await; - second_node.connect(&mut third_node).await; - third_node.connect(&mut first_node).await; + let mut third_node = nodes.pop().unwrap(); + let mut second_node = nodes.pop().unwrap(); + let mut first_node = nodes.pop().unwrap(); let tip: usize = 300; let tip_index: usize = tip - 1; diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index 6ee2d0dc75b5..009edb0bb117 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -12,7 +12,9 @@ use std::sync::Arc; use tokio::sync::Mutex; use tracing::{span, Level}; -pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { +pub(crate) async fn setup( + num_nodes: usize, +) -> eyre::Result<(Vec, TaskManager, TaskExecutor, Wallet)> { let tasks = TaskManager::current(); let exec = tasks.executor(); @@ -31,16 +33,33 @@ pub(crate) fn setup() -> (NodeConfig, TaskManager, TaskExecutor, Wallet) { ..NetworkArgs::default() }; - ( - NodeConfig::test() - .with_chain(chain_spec) - .with_network(network_config) + // Create nodes and peer them + let mut nodes: Vec = Vec::with_capacity(num_nodes); + for idx in 0..num_nodes { + let node_config = NodeConfig::test() + .with_chain(chain_spec.clone()) + .with_network(network_config.clone()) .with_unused_ports() - .with_rpc(RpcServerArgs::default().with_unused_ports().with_http()), - tasks, - exec, - Wallet::default().with_chain_id(chain_id), - ) + .with_rpc(RpcServerArgs::default().with_unused_ports().with_http()); + + let mut node = node(node_config, exec.clone(), idx + 1).await?; + + // Connect each node in a chain. + if let Some(previous_node) = nodes.last_mut() { + previous_node.connect(&mut node).await; + } + + // Connect last node with the first if there are more than two + if idx + 1 == num_nodes && num_nodes > 2 { + if let Some(first_node) = nodes.first_mut() { + node.connect(first_node).await; + } + } + + nodes.push(node); + } + + Ok((nodes, tasks, exec, Wallet::default().with_chain_id(chain_id))) } pub(crate) async fn node( From d28dc314703de23d8e51808e2fcff1e0b6515a80 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 24 Apr 2024 17:40:37 +0100 Subject: [PATCH 13/53] restore optimism wallet logic --- crates/e2e-test-utils/src/wallet.rs | 5 +++-- crates/optimism/node/tests/e2e/p2p.rs | 6 ++++-- crates/optimism/node/tests/e2e/utils.rs | 14 ++++++++++++-- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/crates/e2e-test-utils/src/wallet.rs b/crates/e2e-test-utils/src/wallet.rs index d94dec2a08c3..e841e7cd786c 100644 --- a/crates/e2e-test-utils/src/wallet.rs +++ b/crates/e2e-test-utils/src/wallet.rs @@ -4,7 +4,8 @@ use alloy_signer_wallet::{coins_bip39::English, LocalWallet, MnemonicBuilder}; /// One of the accounts of the genesis allocations. pub struct Wallet { pub inner: LocalWallet, - chain_id: u64, + pub inner_nonce: u64, + pub chain_id: u64, amount: usize, derivation_path: Option, } @@ -13,7 +14,7 @@ impl Wallet { /// Creates a new account from one of the secret/pubkeys of the genesis allocations (test.json) pub fn new(amount: usize) -> Self { let inner = MnemonicBuilder::::default().phrase(TEST_MNEMONIC).build().unwrap(); - Self { inner, chain_id: 1, amount, derivation_path: None } + Self { inner, chain_id: 1, amount, derivation_path: None, inner_nonce: 0 } } /// Sets chain id diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index e18242566270..965a8e882696 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,13 +1,15 @@ +use std::sync::Arc; use crate::utils::{advance_chain, setup}; use reth::primitives::BASE_MAINNET; use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet}; use reth_primitives::ChainId; +use tokio::sync::Mutex; #[tokio::test] async fn can_sync() -> eyre::Result<()> { reth_tracing::init_test_tracing(); - let (mut nodes, _tasks, exec, wallet) = setup(3).await?; + let (mut nodes, _tasks, wallet) = setup(3).await?; let wallet = Arc::new(Mutex::new(wallet)); let mut third_node = nodes.pop().unwrap(); @@ -31,7 +33,7 @@ async fn can_sync() -> eyre::Result<()> { third_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?; // On second node, create a side chain: 298b -> 299b -> 300b - wallet.lock().await.nonce -= 3; + wallet.lock().await.inner_nonce -= 3; second_node.payload.timestamp = first_node.payload.timestamp - 3; // TODO: probably want to make it node agnostic let side_payload_chain = advance_chain(3, &mut second_node, wallet.clone()).await?; let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index e86a7c654142..cde6d5aa22b1 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -3,7 +3,9 @@ use reth_e2e_test_utils::{wallet::Wallet, NodeHelperType}; use reth_node_optimism::{OptimismBuiltPayload, OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; +use tokio::sync::Mutex; use std::{future::Future, pin::Pin, sync::Arc}; +use reth_e2e_test_utils::transaction::TransactionTestContext; /// Optimism Node Helper type pub(crate) type OpNode = NodeHelperType; @@ -27,9 +29,17 @@ pub(crate) async fn setup(num_nodes: usize) -> eyre::Result<(Vec, TaskMa pub(crate) async fn advance_chain( length: usize, node: &mut OpNode, - tx_generator: impl Fn(u64) -> Pin>>, + wallet: Arc> ) -> eyre::Result> { - node.advance(length as u64, tx_generator, optimism_payload_attributes).await + node.advance(length as u64, |_| { + let wallet = wallet.clone(); + Box::pin(async move { + let mut wallet = wallet.lock().await; + let tx_fut = TransactionTestContext::optimism_l1_block_info_tx(wallet.chain_id, wallet.inner.clone(), wallet.inner_nonce); + wallet.inner_nonce += 1; + tx_fut.await + }) + }, optimism_payload_attributes).await } /// Helper function to create a new eth payload attributes From 57e530b99d6c7359d2c65678f4eb5b9038d26114 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 24 Apr 2024 20:36:24 +0100 Subject: [PATCH 14/53] fmt --- crates/e2e-test-utils/src/node.rs | 2 +- crates/optimism/node/tests/e2e/p2p.rs | 3 +-- crates/optimism/node/tests/e2e/utils.rs | 34 +++++++++++++++---------- 3 files changed, 23 insertions(+), 16 deletions(-) diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index e3a2576bae49..856dc531d0ce 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -186,7 +186,7 @@ where /// Asserts that a new block has been added to the blockchain /// and the tx has been included in the block. - /// + /// /// Does NOT work for for pipeline since there's no stream notification! pub async fn assert_new_block( &mut self, diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 965a8e882696..bfde47b5906e 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,8 +1,8 @@ -use std::sync::Arc; use crate::utils::{advance_chain, setup}; use reth::primitives::BASE_MAINNET; use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet}; use reth_primitives::ChainId; +use std::sync::Arc; use tokio::sync::Mutex; #[tokio::test] @@ -43,6 +43,5 @@ async fn can_sync() -> eyre::Result<()> { third_node.engine_api.update_optimistic_forkchoice(dbg!(side_chain[2])).await?; third_node.wait_block(side_payload_chain[2].0.block().number, side_chain[2], true).await?; - Ok(()) } diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index cde6d5aa22b1..b594ec9a9ab0 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -1,11 +1,10 @@ use reth::{primitives::Bytes, rpc::types::engine::PayloadAttributes, tasks::TaskManager}; -use reth_e2e_test_utils::{wallet::Wallet, NodeHelperType}; +use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet, NodeHelperType}; use reth_node_optimism::{OptimismBuiltPayload, OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; -use tokio::sync::Mutex; use std::{future::Future, pin::Pin, sync::Arc}; -use reth_e2e_test_utils::transaction::TransactionTestContext; +use tokio::sync::Mutex; /// Optimism Node Helper type pub(crate) type OpNode = NodeHelperType; @@ -29,17 +28,26 @@ pub(crate) async fn setup(num_nodes: usize) -> eyre::Result<(Vec, TaskMa pub(crate) async fn advance_chain( length: usize, node: &mut OpNode, - wallet: Arc> + wallet: Arc>, ) -> eyre::Result> { - node.advance(length as u64, |_| { - let wallet = wallet.clone(); - Box::pin(async move { - let mut wallet = wallet.lock().await; - let tx_fut = TransactionTestContext::optimism_l1_block_info_tx(wallet.chain_id, wallet.inner.clone(), wallet.inner_nonce); - wallet.inner_nonce += 1; - tx_fut.await - }) - }, optimism_payload_attributes).await + node.advance( + length as u64, + |_| { + let wallet = wallet.clone(); + Box::pin(async move { + let mut wallet = wallet.lock().await; + let tx_fut = TransactionTestContext::optimism_l1_block_info_tx( + wallet.chain_id, + wallet.inner.clone(), + wallet.inner_nonce, + ); + wallet.inner_nonce += 1; + tx_fut.await + }) + }, + optimism_payload_attributes, + ) + .await } /// Helper function to create a new eth payload attributes From e4c11eb3dd85afacd2c2773e1f337a1f7f720162 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 25 Apr 2024 01:39:15 +0200 Subject: [PATCH 15/53] fix: derank peers that responded with bad data --- crates/net/network/src/fetch/mod.rs | 74 +++++++++++++++++++++-------- 1 file changed, 55 insertions(+), 19 deletions(-) diff --git a/crates/net/network/src/fetch/mod.rs b/crates/net/network/src/fetch/mod.rs index 19c605fb9a54..af646460078c 100644 --- a/crates/net/network/src/fetch/mod.rs +++ b/crates/net/network/src/fetch/mod.rs @@ -77,8 +77,16 @@ impl StateFetcher { best_number: u64, timeout: Arc, ) { - self.peers - .insert(peer_id, Peer { state: PeerState::Idle, best_hash, best_number, timeout }); + self.peers.insert( + peer_id, + Peer { + state: PeerState::Idle, + best_hash, + best_number, + timeout, + last_response_likely_bad: false, + }, + ); } /// Removes the peer from the peer list, after which it is no longer available for future @@ -119,14 +127,27 @@ impl StateFetcher { } /// Returns the _next_ idle peer that's ready to accept a request, - /// prioritizing those with the lowest timeout/latency. - /// Once a peer has been yielded, it will be moved to the end of the map - fn next_peer(&mut self) -> Option { - self.peers - .iter() - .filter(|(_, peer)| peer.state.is_idle()) - .min_by_key(|(_, peer)| peer.timeout()) - .map(|(id, _)| *id) + /// prioritizing those with the lowest timeout/latency and those that recently responded with + /// adequate data. + fn next_best_peer(&mut self) -> Option { + let mut idle = self.peers.iter().filter(|(_, peer)| peer.state.is_idle()); + + let mut best_peer = idle.next()?; + + for maybe_better in idle { + if best_peer.1.last_response_likely_bad && !maybe_better.1.last_response_likely_bad { + best_peer = maybe_better; + continue + } + + if maybe_better.1.timeout() < best_peer.1.timeout() && + !maybe_better.1.last_response_likely_bad + { + best_peer = maybe_better; + } + } + + Some(*best_peer.0) } /// Returns the next action to return @@ -136,7 +157,7 @@ impl StateFetcher { return PollAction::NoRequests } - let Some(peer_id) = self.next_peer() else { return PollAction::NoPeersAvailable }; + let Some(peer_id) = self.next_best_peer() else { return PollAction::NoPeersAvailable }; let request = self.queued_requests.pop_front().expect("not empty"); let request = self.prepare_block_request(peer_id, request); @@ -249,6 +270,9 @@ impl StateFetcher { } if let Some(peer) = self.peers.get_mut(&peer_id) { + // update the peer's response state + peer.last_response_likely_bad = is_likely_bad_response; + // If the peer is still ready to accept new requests, we try to send a followup // request immediately. if peer.state.on_request_finished() && !is_error && !is_likely_bad_response { @@ -268,11 +292,16 @@ impl StateFetcher { peer_id: PeerId, res: RequestResult>, ) -> Option { + let is_likely_bad_response = res.as_ref().map_or(true, |bodies| bodies.is_empty()); + if let Some(resp) = self.inflight_bodies_requests.remove(&peer_id) { let _ = resp.response.send(res.map(|b| (peer_id, b).into())); } if let Some(peer) = self.peers.get_mut(&peer_id) { - if peer.state.on_request_finished() { + // update the peer's response state + peer.last_response_likely_bad = is_likely_bad_response; + + if peer.state.on_request_finished() && !is_likely_bad_response { return self.followup_request(peer_id) } } @@ -307,6 +336,13 @@ struct Peer { best_number: u64, /// Tracks the current timeout value we use for the peer. timeout: Arc, + /// Tracks whether the peer has recently responded with a likely bad response. + /// + /// This is used to de-rank the peer if there are other peers available. + /// This exists because empty responses may not be penalized (e.g. when blocks near the tip are + /// downloaded), but we still want to avoid requesting from the same peer again if it has the + /// lowest timeout. + last_response_likely_bad: bool, } impl Peer { @@ -462,17 +498,17 @@ mod tests { fetcher.new_active_peer(peer1, B256::random(), 1, Arc::new(AtomicU64::new(1))); fetcher.new_active_peer(peer2, B256::random(), 2, Arc::new(AtomicU64::new(1))); - let first_peer = fetcher.next_peer().unwrap(); + let first_peer = fetcher.next_best_peer().unwrap(); assert!(first_peer == peer1 || first_peer == peer2); // Pending disconnect for first_peer fetcher.on_pending_disconnect(&first_peer); // first_peer now isn't idle, so we should get other peer - let second_peer = fetcher.next_peer().unwrap(); + let second_peer = fetcher.next_best_peer().unwrap(); assert!(first_peer == peer1 || first_peer == peer2); assert_ne!(first_peer, second_peer); // without idle peers, returns None fetcher.on_pending_disconnect(&second_peer); - assert_eq!(fetcher.next_peer(), None); + assert_eq!(fetcher.next_best_peer(), None); } #[tokio::test] @@ -491,13 +527,13 @@ mod tests { fetcher.new_active_peer(peer3, B256::random(), 3, Arc::new(AtomicU64::new(50))); // Must always get peer1 (lowest timeout) - assert_eq!(fetcher.next_peer(), Some(peer1)); - assert_eq!(fetcher.next_peer(), Some(peer1)); + assert_eq!(fetcher.next_best_peer(), Some(peer1)); + assert_eq!(fetcher.next_best_peer(), Some(peer1)); // peer2's timeout changes below peer1's peer2_timeout.store(10, Ordering::Relaxed); // Then we get peer 2 always (now lowest) - assert_eq!(fetcher.next_peer(), Some(peer2)); - assert_eq!(fetcher.next_peer(), Some(peer2)); + assert_eq!(fetcher.next_best_peer(), Some(peer2)); + assert_eq!(fetcher.next_best_peer(), Some(peer2)); } #[tokio::test] From e377c57b9883de502ab3f21a2177426271b29161 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 12:06:14 +0100 Subject: [PATCH 16/53] reduce to 90 blocks --- crates/optimism/node/tests/e2e/p2p.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index bfde47b5906e..fd0122eebec1 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -16,23 +16,23 @@ async fn can_sync() -> eyre::Result<()> { let mut second_node = nodes.pop().unwrap(); let mut first_node = nodes.pop().unwrap(); - let tip: usize = 300; + let tip: usize = 90; let tip_index: usize = tip - 1; - // On first node, create a chain up to block number 300a + // On first node, create a chain up to block number 90a let canonical_payload_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?; let canonical_chain = canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // On second node, sync optimistically up to block number 297a + // On second node, sync optimistically up to block number 87a second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 3]).await?; second_node.wait_block(tip as u64 - 3, canonical_chain[tip_index - 3], true).await?; - // On third node, sync optimistically up to block number 300a + // On third node, sync optimistically up to block number 90a third_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; third_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?; - // On second node, create a side chain: 298b -> 299b -> 300b + // On second node, create a side chain: 88b -> 89b -> 90b wallet.lock().await.inner_nonce -= 3; second_node.payload.timestamp = first_node.payload.timestamp - 3; // TODO: probably want to make it node agnostic let side_payload_chain = advance_chain(3, &mut second_node, wallet.clone()).await?; From 13a426cf128a3cee06aa60e7ebb3578983907091 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 17:16:54 +0100 Subject: [PATCH 17/53] optimistic sync with reorg failure --- bin/reth/src/optimism.rs | 7 ++-- crates/blockchain-tree/src/blockchain_tree.rs | 33 ++++++++++++++++--- crates/blockchain-tree/src/noop.rs | 6 ++++ crates/blockchain-tree/src/shareable.rs | 9 +++++ crates/consensus/beacon/src/engine/mod.rs | 3 +- crates/interfaces/src/blockchain_tree/mod.rs | 5 +++ crates/optimism/node/tests/e2e/p2p.rs | 31 +++++++++-------- crates/optimism/node/tests/e2e/utils.rs | 5 +-- crates/storage/provider/src/providers/mod.rs | 4 +++ 9 files changed, 78 insertions(+), 25 deletions(-) diff --git a/bin/reth/src/optimism.rs b/bin/reth/src/optimism.rs index 2bd2ffa9bffa..ee267cf7d6f7 100644 --- a/bin/reth/src/optimism.rs +++ b/bin/reth/src/optimism.rs @@ -3,10 +3,7 @@ use clap::Parser; use reth::cli::Cli; use reth_node_builder::NodeHandle; -use reth_node_optimism::{ - args::RollupArgs, rpc::SequencerClient, OptimismEngineTypes, OptimismNode, -}; -use reth_provider::BlockReaderIdExt; +use reth_node_optimism::{args::RollupArgs, rpc::SequencerClient, OptimismNode}; use std::sync::Arc; // We use jemalloc for performance reasons @@ -27,7 +24,7 @@ fn main() { } if let Err(err) = Cli::::parse().run(|builder, rollup_args| async move { - let NodeHandle { node, node_exit_future } = builder + let NodeHandle { node: _node, node_exit_future } = builder .node(OptimismNode::new(rollup_args.clone())) .extend_rpc_modules(move |ctx| { // register sequencer tx forwarder diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 08f588cd2d59..52b9f4f61dc2 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -18,7 +18,7 @@ use reth_interfaces::{ }; use reth_primitives::{ BlockHash, BlockNumHash, BlockNumber, ForkBlock, GotExpected, Hardfork, PruneModes, Receipt, - SealedBlock, SealedBlockWithSenders, SealedHeader, U256, + SealedBlock, SealedBlockWithSenders, SealedHeader, StaticFileSegment, B256, U256, }; use reth_provider::{ chain::{ChainSplit, ChainSplitTarget}, @@ -320,6 +320,10 @@ where ) -> Result { debug_assert!(self.validate_block(&block).is_ok(), "Block must be validated"); + if self.is_block_hash_canonical(&block.hash())? { + return Ok(BlockStatus::Valid(BlockAttachment::Canonical)) + } + let parent = block.parent_num_hash(); // check if block parent can be found in any side chain. @@ -804,6 +808,16 @@ where ) -> RethResult<()> { self.finalize_block(last_finalized_block); + let last_canonical_hashes = self.update_block_hashes()?; + + self.connect_buffered_blocks_to_hashes(last_canonical_hashes)?; + + Ok(()) + } + + /// Update all block hashes. iterate over present and new list of canonical hashes and compare + /// them. Remove all mismatches, disconnect them and removes all chains. + pub fn update_block_hashes(&mut self) -> RethResult> { let last_canonical_hashes = self .externals .fetch_latest_canonical_hashes(self.config.num_of_canonical_hashes() as usize)?; @@ -818,9 +832,7 @@ where } } - self.connect_buffered_blocks_to_hashes(last_canonical_hashes)?; - - Ok(()) + Ok(last_canonical_hashes) } /// Reads the last `N` canonical hashes from the database and updates the block indices of the @@ -1207,6 +1219,19 @@ where &mut self, revert_until: BlockNumber, ) -> Result, CanonicalError> { + if self + .externals + .provider_factory + .static_file_provider() + .get_highest_static_file_block(StaticFileSegment::Headers) + .unwrap_or_default() > + revert_until + { + return Err(CanonicalError::CanonicalRevert( + "Cannot revert from static files.".to_string(), + )); + } + // read data that is needed for new sidechain let provider_rw = self.externals.provider_factory.provider_rw()?; diff --git a/crates/blockchain-tree/src/noop.rs b/crates/blockchain-tree/src/noop.rs index 9fa82025511a..c71042676415 100644 --- a/crates/blockchain-tree/src/noop.rs +++ b/crates/blockchain-tree/src/noop.rs @@ -59,6 +59,12 @@ impl BlockchainTreeEngine for NoopBlockchainTree { fn make_canonical(&self, block_hash: BlockHash) -> Result { Err(BlockchainTreeError::BlockHashNotFoundInChain { block_hash }.into()) } + + fn update_block_hashes_and_remove_chains( + &self, + ) -> RethResult> { + Ok(BTreeMap::new()) + } } impl BlockchainTreeViewer for NoopBlockchainTree { diff --git a/crates/blockchain-tree/src/shareable.rs b/crates/blockchain-tree/src/shareable.rs index 7a0eb36fa49b..ff9f191f3b71 100644 --- a/crates/blockchain-tree/src/shareable.rs +++ b/crates/blockchain-tree/src/shareable.rs @@ -82,6 +82,15 @@ where res } + fn update_block_hashes_and_remove_chains( + &self, + ) -> RethResult> { + let mut tree = self.tree.write(); + let res = tree.update_block_hashes(); + tree.update_chains_metrics(); + res + } + fn connect_buffered_blocks_to_canonical_hashes(&self) -> RethResult<()> { trace!(target: "blockchain_tree", "Connecting buffered blocks to canonical hashes"); let mut tree = self.tree.write(); diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 451211a76dd8..8215b2302485 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1611,10 +1611,11 @@ where })? .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; self.blockchain.set_canonical_head(max_header); - self.blockchain.connect_buffered_blocks_to_canonical_hashes_and_finalize(max_block)?; } if sync_target_state.finalized_block_hash.is_zero() { + self.blockchain.update_block_hashes_and_remove_chains()?; + self.blockchain.connect_buffered_blocks_to_canonical_hashes()?; // We are on a optimistic syncing process, better to wait for the next FCU to handle return Ok(()) } diff --git a/crates/interfaces/src/blockchain_tree/mod.rs b/crates/interfaces/src/blockchain_tree/mod.rs index d8ad667fcbbf..93d199fc21db 100644 --- a/crates/interfaces/src/blockchain_tree/mod.rs +++ b/crates/interfaces/src/blockchain_tree/mod.rs @@ -78,6 +78,11 @@ pub trait BlockchainTreeEngine: BlockchainTreeViewer + Send + Sync { last_finalized_block: BlockNumber, ) -> RethResult<()>; + /// Update all block hashes. iterate over present and new list of canonical hashes and compare + /// them. Remove all mismatches, disconnect them and removes all chains. + fn update_block_hashes_and_remove_chains(&self) + -> RethResult>; + /// Reads the last `N` canonical hashes from the database and updates the block indices of the /// tree by attempting to connect the buffered blocks to canonical hashes. /// diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index fd0122eebec1..f99f72fb9787 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,7 +1,4 @@ use crate::utils::{advance_chain, setup}; -use reth::primitives::BASE_MAINNET; -use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet}; -use reth_primitives::ChainId; use std::sync::Arc; use tokio::sync::Mutex; @@ -12,7 +9,7 @@ async fn can_sync() -> eyre::Result<()> { let (mut nodes, _tasks, wallet) = setup(3).await?; let wallet = Arc::new(Mutex::new(wallet)); - let mut third_node = nodes.pop().unwrap(); + let third_node = nodes.pop().unwrap(); let mut second_node = nodes.pop().unwrap(); let mut first_node = nodes.pop().unwrap(); @@ -25,23 +22,31 @@ async fn can_sync() -> eyre::Result<()> { canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); // On second node, sync optimistically up to block number 87a - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 3]).await?; - second_node.wait_block(tip as u64 - 3, canonical_chain[tip_index - 3], true).await?; + second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 1]).await?; + second_node.wait_block(tip as u64 - 1, canonical_chain[tip_index - 1], true).await?; // On third node, sync optimistically up to block number 90a third_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; third_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?; + let reorg_depth = 1usize; + // On second node, create a side chain: 88b -> 89b -> 90b - wallet.lock().await.inner_nonce -= 3; - second_node.payload.timestamp = first_node.payload.timestamp - 3; // TODO: probably want to make it node agnostic - let side_payload_chain = advance_chain(3, &mut second_node, wallet.clone()).await?; + wallet.lock().await.inner_nonce -= reorg_depth as u64; + second_node.payload.timestamp = first_node.payload.timestamp - reorg_depth as u64; // TODO: probably want to make it node agnostic + let side_payload_chain = advance_chain(reorg_depth, &mut second_node, wallet.clone()).await?; let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // On third node, cause a 3 block depth re-org - assert!(side_chain[2] != canonical_chain[tip_index]); - third_node.engine_api.update_optimistic_forkchoice(dbg!(side_chain[2])).await?; - third_node.wait_block(side_payload_chain[2].0.block().number, side_chain[2], true).await?; + // On third node, cause a re-org + assert!(side_chain[reorg_depth - 1] != canonical_chain[tip_index]); + third_node.engine_api.update_optimistic_forkchoice(dbg!(side_chain[reorg_depth - 1])).await?; + third_node + .wait_block( + side_payload_chain[reorg_depth - 1].0.block().number, + side_chain[reorg_depth - 1], + true, + ) + .await?; Ok(()) } diff --git a/crates/optimism/node/tests/e2e/utils.rs b/crates/optimism/node/tests/e2e/utils.rs index b594ec9a9ab0..ad19086aeda6 100644 --- a/crates/optimism/node/tests/e2e/utils.rs +++ b/crates/optimism/node/tests/e2e/utils.rs @@ -1,9 +1,9 @@ -use reth::{primitives::Bytes, rpc::types::engine::PayloadAttributes, tasks::TaskManager}; +use reth::{rpc::types::engine::PayloadAttributes, tasks::TaskManager}; use reth_e2e_test_utils::{transaction::TransactionTestContext, wallet::Wallet, NodeHelperType}; use reth_node_optimism::{OptimismBuiltPayload, OptimismNode, OptimismPayloadBuilderAttributes}; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_primitives::{Address, ChainSpecBuilder, Genesis, B256, BASE_MAINNET}; -use std::{future::Future, pin::Pin, sync::Arc}; +use std::sync::Arc; use tokio::sync::Mutex; /// Optimism Node Helper type @@ -25,6 +25,7 @@ pub(crate) async fn setup(num_nodes: usize) -> eyre::Result<(Vec, TaskMa .await } +/// Advance the chain with sequential payloads returning them in the end. pub(crate) async fn advance_chain( length: usize, node: &mut OpNode, diff --git a/crates/storage/provider/src/providers/mod.rs b/crates/storage/provider/src/providers/mod.rs index c9ebd042cdd2..d6f2c9bc98e4 100644 --- a/crates/storage/provider/src/providers/mod.rs +++ b/crates/storage/provider/src/providers/mod.rs @@ -656,6 +656,10 @@ where self.tree.finalize_block(finalized_block) } + fn update_block_hashes_and_remove_chains(&self) -> RethResult> { + self.tree.update_block_hashes_and_remove_chains() + } + fn connect_buffered_blocks_to_canonical_hashes_and_finalize( &self, last_finalized_block: BlockNumber, From 5703bbd1f203efa0364ad42ae996be756a626fff Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 17:19:17 +0100 Subject: [PATCH 18/53] doc fix --- crates/e2e-test-utils/src/node.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index 856dc531d0ce..5360ac4ad5e6 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -187,7 +187,7 @@ where /// Asserts that a new block has been added to the blockchain /// and the tx has been included in the block. /// - /// Does NOT work for for pipeline since there's no stream notification! + /// Does NOT work for pipeline since there's no stream notification! pub async fn assert_new_block( &mut self, tip_tx_hash: B256, From 63df12d3cd2ca44b783a8eb2b002cbee2f9ffa9d Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 21:19:28 +0100 Subject: [PATCH 19/53] reset download_range on is_terminated --- crates/net/downloaders/src/bodies/bodies.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index 985c545e9b39..533c02e356e6 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -354,6 +354,7 @@ where fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); if this.is_terminated() { + this.download_range = RangeInclusive::new(1, 0); return Poll::Ready(None); } // Submit new requests and poll any in progress From 3167f7fbd4480e991c55faedaea8506af32eef28 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 21:40:53 +0100 Subject: [PATCH 20/53] handle reorgs from optimistic syncing --- crates/blockchain-tree/src/blockchain_tree.rs | 4 +-- crates/consensus/beacon/src/engine/mod.rs | 29 +++++++++++---- crates/consensus/beacon/src/engine/sync.rs | 14 ++++---- crates/e2e-test-utils/src/node.rs | 11 ++++++ .../interfaces/src/blockchain_tree/error.rs | 15 +++++++- crates/optimism/node/tests/e2e/p2p.rs | 35 +++++++++++++++---- crates/primitives/src/stage/mod.rs | 33 +++++++++++++++++ crates/stages-api/src/pipeline/mod.rs | 25 ++++++++++--- 8 files changed, 136 insertions(+), 30 deletions(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index bb258638ad47..8e423ecd3d5b 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1227,9 +1227,7 @@ where .unwrap_or_default() > revert_until { - return Err(CanonicalError::CanonicalRevert( - "Cannot revert from static files.".to_string(), - )); + return Err(CanonicalError::OptimisticCanonicalRevert(revert_until)) } // read data that is needed for new sidechain diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 80fcfdbdd668..6c15ef1a0103 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -22,8 +22,9 @@ use reth_interfaces::{ }; use reth_payload_builder::PayloadBuilderHandle; use reth_primitives::{ - constants::EPOCH_SLOTS, stage::StageId, BlockNumHash, BlockNumber, Head, Header, SealedBlock, - SealedHeader, B256, + constants::EPOCH_SLOTS, + stage::{PipelineTarget, StageId}, + BlockNumHash, BlockNumber, Head, Header, SealedBlock, SealedHeader, B256, }; use reth_provider::{ BlockIdReader, BlockReader, BlockSource, CanonChainTracker, ChainSpecProvider, ProviderError, @@ -320,7 +321,7 @@ where }; if let Some(target) = maybe_pipeline_target { - this.sync.set_pipeline_sync_target(target); + this.sync.set_pipeline_sync_target(target.into()); } Ok((this, handle)) @@ -1012,6 +1013,10 @@ where // so we should not warn the user, since this will result in us attempting to sync // to a new target and is considered normal operation during sync } + CanonicalError::OptimisticCanonicalRevert(block_number) => { + self.sync.set_pipeline_sync_target(PipelineTarget::Unwind(*block_number)); + return PayloadStatus::from_status(PayloadStatusEnum::Syncing) + } _ => { warn!(target: "consensus::engine", %error, ?state, "Failed to canonicalize the head hash"); // TODO(mattsse) better error handling before attempting to sync (FCU could be @@ -1042,7 +1047,7 @@ where if self.pipeline_run_threshold == 0 { // use the pipeline to sync to the target trace!(target: "consensus::engine", %target, "Triggering pipeline run to sync missing ancestors of the new head"); - self.sync.set_pipeline_sync_target(target); + self.sync.set_pipeline_sync_target(target.into()); } else { // trigger a full block download for missing hash, or the parent of its lowest buffered // ancestor @@ -1125,6 +1130,11 @@ where return if error.is_fatal() { error!(target: "consensus::engine", %error, "Encountered fatal error"); Err(BeaconOnNewPayloadError::Internal(Box::new(error))) + } else if let Some(block_number) = error.is_optimistic_revert() { + self.sync.set_pipeline_sync_target(PipelineTarget::Unwind( + block_number, + )); + Err(BeaconOnNewPayloadError::Internal(Box::new(error))) } else { // If we could not make the sync target block canonical, we // should return the error as an invalid payload status. @@ -1398,6 +1408,9 @@ where error, hash ) + } else if let Some(block_number) = error.is_optimistic_revert() { + self.sync + .set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } } } @@ -1448,7 +1461,7 @@ where ) { // we don't have the block yet and the distance exceeds the allowed // threshold - self.sync.set_pipeline_sync_target(target); + self.sync.set_pipeline_sync_target(target.into()); // we can exit early here because the pipeline will take care of syncing return } @@ -1532,6 +1545,8 @@ where // TODO: do not ignore this let _ = self.blockchain.make_canonical(*target_hash.as_ref()); } + } else if let Some(block_number) = err.is_optimistic_revert() { + self.sync.set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } Err((target.head_block_hash, err)) @@ -1675,7 +1690,7 @@ where // the tree update from executing too many blocks and blocking. if let Some(target) = pipeline_target { // run the pipeline to the target since the distance is sufficient - self.sync.set_pipeline_sync_target(target); + self.sync.set_pipeline_sync_target(target.into()); } else if let Some(number) = self.blockchain.block_number(sync_target_state.finalized_block_hash)? { @@ -1687,7 +1702,7 @@ where } else { // We don't have the finalized block in the database, so we need to // trigger another pipeline run. - self.sync.set_pipeline_sync_target(sync_target_state.finalized_block_hash); + self.sync.set_pipeline_sync_target(sync_target_state.finalized_block_hash.into()); } Ok(()) diff --git a/crates/consensus/beacon/src/engine/sync.rs b/crates/consensus/beacon/src/engine/sync.rs index 96163e996338..1978836203d3 100644 --- a/crates/consensus/beacon/src/engine/sync.rs +++ b/crates/consensus/beacon/src/engine/sync.rs @@ -11,7 +11,7 @@ use reth_interfaces::p2p::{ full_block::{FetchFullBlockFuture, FetchFullBlockRangeFuture, FullBlockClient}, headers::client::HeadersClient, }; -use reth_primitives::{BlockNumber, ChainSpec, SealedBlock, B256}; +use reth_primitives::{stage::PipelineTarget, BlockNumber, ChainSpec, SealedBlock, B256}; use reth_stages_api::{ControlFlow, Pipeline, PipelineError, PipelineWithResult}; use reth_tasks::TaskSpawner; use reth_tokio_util::EventListeners; @@ -44,7 +44,7 @@ where /// The pipeline is used for large ranges. pipeline_state: PipelineState, /// Pending target block for the pipeline to sync - pending_pipeline_target: Option, + pending_pipeline_target: Option, /// In-flight full block requests in progress. inflight_full_block_requests: Vec>, /// In-flight full block _range_ requests in progress. @@ -216,8 +216,8 @@ where /// Sets a new target to sync the pipeline to. /// /// But ensures the target is not the zero hash. - pub(crate) fn set_pipeline_sync_target(&mut self, target: B256) { - if target.is_zero() { + pub(crate) fn set_pipeline_sync_target(&mut self, target: PipelineTarget) { + if target.sync_target().is_some_and(|target| target.is_zero()) { // precaution to never sync to the zero hash return } @@ -384,7 +384,7 @@ pub(crate) enum EngineSyncEvent { /// Pipeline started syncing /// /// This is none if the pipeline is triggered without a specific target. - PipelineStarted(Option), + PipelineStarted(Option), /// Pipeline finished /// /// If this is returned, the pipeline is idle. @@ -590,7 +590,7 @@ mod tests { .build(pipeline, chain_spec); let tip = client.highest_block().expect("there should be blocks here"); - sync_controller.set_pipeline_sync_target(tip.hash()); + sync_controller.set_pipeline_sync_target(tip.hash().into()); let sync_future = poll_fn(|cx| sync_controller.poll(cx)); let next_event = poll!(sync_future); @@ -598,7 +598,7 @@ mod tests { // can assert that the first event here is PipelineStarted because we set the sync target, // and we should get Ready because the pipeline should be spawned immediately assert_matches!(next_event, Poll::Ready(EngineSyncEvent::PipelineStarted(Some(target))) => { - assert_eq!(target, tip.hash()); + assert_eq!(target.sync_target().unwrap(), tip.hash()); }); // the next event should be the pipeline finishing in a good state diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index 5360ac4ad5e6..8a94bd1c2b85 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -14,6 +14,7 @@ use reth::{ }; use reth_node_builder::NodeTypes; use reth_primitives::{stage::StageId, BlockHash, BlockNumber, Bytes, B256}; +use reth_provider::BlockNumReader; use std::{marker::PhantomData, pin::Pin}; use tokio_stream::StreamExt; @@ -184,6 +185,16 @@ where Ok(()) } + pub async fn wait_unwind(&self, number: BlockNumber) -> eyre::Result<()> { + loop { + tokio::time::sleep(std::time::Duration::from_millis(20)).await; + if self.inner.provider.best_block_number()? == number { + break + } + } + Ok(()) + } + /// Asserts that a new block has been added to the blockchain /// and the tx has been included in the block. /// diff --git a/crates/interfaces/src/blockchain_tree/error.rs b/crates/interfaces/src/blockchain_tree/error.rs index b636985766a1..5746d54ef7e7 100644 --- a/crates/interfaces/src/blockchain_tree/error.rs +++ b/crates/interfaces/src/blockchain_tree/error.rs @@ -67,6 +67,9 @@ pub enum CanonicalError { /// Error indicating a transaction failed to commit during execution. #[error("transaction error on commit: {0}")] CanonicalCommit(String), + /// Error indicating that a previous optimistic sync target was re-orged + #[error("transaction error on revert: {0}")] + OptimisticCanonicalRevert(BlockNumber), } impl CanonicalError { @@ -83,6 +86,15 @@ impl CanonicalError { CanonicalError::BlockchainTree(BlockchainTreeError::BlockHashNotFoundInChain { .. }) ) } + + /// Returns `Some(BlockNumber)` if the underlying error matches + /// [CanonicalError::OptimisticCanonicalRevert]. + pub fn is_optimistic_revert(&self) -> Option { + match self { + CanonicalError::OptimisticCanonicalRevert(block_number) => Some(*block_number), + _ => None, + } + } } /// Error thrown when inserting a block failed because the block is considered invalid. @@ -317,7 +329,8 @@ impl InsertBlockErrorKind { InsertBlockErrorKind::Canonical(err) => match err { CanonicalError::BlockchainTree(_) | CanonicalError::CanonicalCommit(_) | - CanonicalError::CanonicalRevert(_) => false, + CanonicalError::CanonicalRevert(_) | + CanonicalError::OptimisticCanonicalRevert(_) => false, CanonicalError::Validation(_) => true, CanonicalError::Provider(_) => false, }, diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index f99f72fb9787..8dbafb050d4b 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,4 +1,5 @@ use crate::utils::{advance_chain, setup}; +use reth_rpc_types::engine::PayloadStatusEnum; use std::sync::Arc; use tokio::sync::Mutex; @@ -21,7 +22,7 @@ async fn can_sync() -> eyre::Result<()> { let canonical_chain = canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // On second node, sync optimistically up to block number 87a + // On second node, sync optimistically up to block number 89a second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 1]).await?; second_node.wait_block(tip as u64 - 1, canonical_chain[tip_index - 1], true).await?; @@ -31,19 +32,39 @@ async fn can_sync() -> eyre::Result<()> { let reorg_depth = 1usize; - // On second node, create a side chain: 88b -> 89b -> 90b + // On second node, create a side chain: 89a -> 90b wallet.lock().await.inner_nonce -= reorg_depth as u64; second_node.payload.timestamp = first_node.payload.timestamp - reorg_depth as u64; // TODO: probably want to make it node agnostic let side_payload_chain = advance_chain(reorg_depth, &mut second_node, wallet.clone()).await?; let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // On third node, cause a re-org - assert!(side_chain[reorg_depth - 1] != canonical_chain[tip_index]); - third_node.engine_api.update_optimistic_forkchoice(dbg!(side_chain[reorg_depth - 1])).await?; + // // On third node, cause a re-org, block won't be handled + // assert!(side_chain[reorg_depth - 1] != canonical_chain[tip_index]); + // third_node.engine_api.update_optimistic_forkchoice(side_chain[reorg_depth - 1]).await?; + // third_node.wait_unwind(89).await?; + + // It will create a fork chain + let _ = third_node + .engine_api + .submit_payload( + side_payload_chain[0].0.clone(), + side_payload_chain[0].1.clone(), + PayloadStatusEnum::Valid, + Default::default(), + ) + .await; + + // It will issue a pipeline reorg + third_node + .engine_api + .update_forkchoice(side_chain[reorg_depth - 1], side_chain[reorg_depth - 1]) + .await?; + + // Make sure we have the updated block third_node .wait_block( - side_payload_chain[reorg_depth - 1].0.block().number, - side_chain[reorg_depth - 1], + side_payload_chain[0].0.block().number, + side_payload_chain[0].0.block().hash(), true, ) .await?; diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index ffe52554d322..5169d8dc0f2d 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -1,6 +1,7 @@ //! Staged sync primitives. mod id; +use crate::{BlockHash, BlockNumber}; pub use id::StageId; mod checkpoints; @@ -9,3 +10,35 @@ pub use checkpoints::{ HeadersCheckpoint, IndexHistoryCheckpoint, MerkleCheckpoint, StageCheckpoint, StageUnitCheckpoint, StorageHashingCheckpoint, }; + +/// Direction and target for pipeline operations. +#[derive(Debug, Clone, Copy)] +pub enum PipelineTarget { + /// Target for forward synchronization. + Sync(BlockHash), + /// Target for backward unwinding. + Unwind(BlockNumber), +} + +impl PipelineTarget { + /// Target for forward synchronization. + pub fn sync_target(&self) -> Option { + match self { + PipelineTarget::Sync(hash) => Some(*hash), + PipelineTarget::Unwind(_) => None, + } + } + /// Target for backward unwinding. + pub fn unwind_target(&self) -> Option { + match self { + PipelineTarget::Sync(_) => None, + PipelineTarget::Unwind(number) => Some(*number), + } + } +} + +impl From for PipelineTarget { + fn from(hash: BlockHash) -> Self { + Self::Sync(hash) + } +} diff --git a/crates/stages-api/src/pipeline/mod.rs b/crates/stages-api/src/pipeline/mod.rs index 1b455a9395c2..3df70e809aac 100644 --- a/crates/stages-api/src/pipeline/mod.rs +++ b/crates/stages-api/src/pipeline/mod.rs @@ -7,7 +7,7 @@ use reth_db::database::Database; use reth_interfaces::RethResult; use reth_primitives::{ constants::BEACON_CONSENSUS_REORG_UNWIND_DEPTH, - stage::{StageCheckpoint, StageId}, + stage::{PipelineTarget, StageCheckpoint, StageId}, static_file::HighestStaticFiles, BlockNumber, B256, }; @@ -129,17 +129,32 @@ where /// Consume the pipeline and run it until it reaches the provided tip, if set. Return the /// pipeline and its result as a future. #[track_caller] - pub fn run_as_fut(mut self, tip: Option) -> PipelineFut { + pub fn run_as_fut(mut self, target: Option) -> PipelineFut { // TODO: fix this in a follow up PR. ideally, consensus engine would be responsible for // updating metrics. let _ = self.register_metrics(); // ignore error Box::pin(async move { // NOTE: the tip should only be None if we are in continuous sync mode. - if let Some(tip) = tip { - self.set_tip(tip); + if let Some(target) = target { + match target { + PipelineTarget::Sync(tip) => self.set_tip(tip), + PipelineTarget::Unwind(target) => { + if let Err(err) = self.produce_static_files() { + return (self, Err(err.into())) + } + if let Err(err) = self.unwind(target, None) { + return (self, Err(err.into())) + } + self.progress.update(target); + + // TODO: if report ControlFlow::Unwind we will issue a warning of bad block detected. Do we want? + return (self, Ok(ControlFlow::Continue { block_number: target })) + } + } } + let result = self.run_loop().await; - trace!(target: "sync::pipeline", ?tip, ?result, "Pipeline finished"); + trace!(target: "sync::pipeline", ?target, ?result, "Pipeline finished"); (self, result) }) } From 93f0f64ea10f3a1bc71c15ed30a910ddb855e7ca Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 25 Apr 2024 21:41:55 +0100 Subject: [PATCH 21/53] remove unused test code --- crates/optimism/node/tests/e2e/p2p.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 8dbafb050d4b..6230da9a0dc5 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -38,11 +38,6 @@ async fn can_sync() -> eyre::Result<()> { let side_payload_chain = advance_chain(reorg_depth, &mut second_node, wallet.clone()).await?; let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // // On third node, cause a re-org, block won't be handled - // assert!(side_chain[reorg_depth - 1] != canonical_chain[tip_index]); - // third_node.engine_api.update_optimistic_forkchoice(side_chain[reorg_depth - 1]).await?; - // third_node.wait_unwind(89).await?; - // It will create a fork chain let _ = third_node .engine_api From 3ad4cb209e80947dcf662f0a2a58e96137823ac2 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 12:56:39 +0100 Subject: [PATCH 22/53] wait_unwind more aggr and check header stage checkpoint --- crates/e2e-test-utils/src/node.rs | 17 +++++++++-------- crates/optimism/node/tests/e2e/p2p.rs | 1 + 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index 8a94bd1c2b85..5b323ebc79b3 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -171,10 +171,7 @@ where if check { if let Some(latest_block) = self.inner.provider.block_by_number(number)? { - if latest_block.hash_slow() != expected_block_hash { - // TODO: only if its awaiting a reorg - continue - } + assert_eq!(latest_block.hash_slow(), expected_block_hash); break } if wait_finish_checkpoint { @@ -187,10 +184,14 @@ where pub async fn wait_unwind(&self, number: BlockNumber) -> eyre::Result<()> { loop { - tokio::time::sleep(std::time::Duration::from_millis(20)).await; - if self.inner.provider.best_block_number()? == number { - break - } + tokio::time::sleep(std::time::Duration::from_millis(10)).await; + if let Some(checkpoint) = + self.inner.provider.get_stage_checkpoint(StageId::Headers)? + { + if checkpoint.block_number == number { + break + } + } } Ok(()) } diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 6230da9a0dc5..958f5576e550 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -56,6 +56,7 @@ async fn can_sync() -> eyre::Result<()> { .await?; // Make sure we have the updated block + third_node.wait_unwind((tip - reorg_depth) as u64).await?; third_node .wait_block( side_payload_chain[0].0.block().number, From c7ba89de4c5270d76560ac2f012f5e4ccca05a14 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 14:10:51 +0100 Subject: [PATCH 23/53] remove canonical block from buffered ones --- crates/blockchain-tree/src/block_buffer.rs | 6 +++--- crates/blockchain-tree/src/blockchain_tree.rs | 11 ++++++----- crates/blockchain-tree/src/shareable.rs | 7 +++++++ crates/e2e-test-utils/src/node.rs | 10 ++++------ crates/stages-api/src/pipeline/mod.rs | 3 ++- 5 files changed, 22 insertions(+), 15 deletions(-) diff --git a/crates/blockchain-tree/src/block_buffer.rs b/crates/blockchain-tree/src/block_buffer.rs index 23c6ca6815e0..14e89633729f 100644 --- a/crates/blockchain-tree/src/block_buffer.rs +++ b/crates/blockchain-tree/src/block_buffer.rs @@ -104,13 +104,13 @@ impl BlockBuffer { removed } - /// Discard all blocks that precede finalized block number from the buffer. - pub fn remove_old_blocks(&mut self, finalized_number: BlockNumber) { + /// Discard all blocks that precede block number from the buffer. + pub fn remove_old_blocks(&mut self, block_number: BlockNumber) { let mut block_hashes_to_remove = Vec::new(); // discard all blocks that are before the finalized number. while let Some(entry) = self.earliest_blocks.first_entry() { - if *entry.key() > finalized_number { + if *entry.key() > block_number { break } let block_hashes = entry.remove(); diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 8e423ecd3d5b..c4814857f67f 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -320,10 +320,6 @@ where ) -> Result { debug_assert!(self.validate_block(&block).is_ok(), "Block must be validated"); - if self.is_block_hash_canonical(&block.hash())? { - return Ok(BlockStatus::Valid(BlockAttachment::Canonical)) - } - let parent = block.parent_num_hash(); // check if block parent can be found in any side chain. @@ -774,6 +770,11 @@ where Ok(InsertPayloadOk::Inserted(status)) } + /// Discard all blocks that precede block number from the buffer. + pub fn remove_old_blocks(&mut self, block: BlockNumber) { + self.state.buffered_blocks.remove_old_blocks(block); + } + /// Finalize blocks up until and including `finalized_block`, and remove them from the tree. pub fn finalize_block(&mut self, finalized_block: BlockNumber) { // remove blocks @@ -788,7 +789,7 @@ where } } // clean block buffer. - self.state.buffered_blocks.remove_old_blocks(finalized_block); + self.remove_old_blocks(finalized_block); } /// Reads the last `N` canonical hashes from the database and updates the block indices of the diff --git a/crates/blockchain-tree/src/shareable.rs b/crates/blockchain-tree/src/shareable.rs index ff9f191f3b71..4e77d5096b3f 100644 --- a/crates/blockchain-tree/src/shareable.rs +++ b/crates/blockchain-tree/src/shareable.rs @@ -87,6 +87,13 @@ where ) -> RethResult> { let mut tree = self.tree.write(); let res = tree.update_block_hashes(); + + if let Ok(Some((block, _))) = + res.as_ref().map(|last_canonical_hashes| last_canonical_hashes.last_key_value()) + { + tree.remove_old_blocks(*block); + } + tree.update_chains_metrics(); res } diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index 5b323ebc79b3..3d4e5e5d2b80 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -185,13 +185,11 @@ where pub async fn wait_unwind(&self, number: BlockNumber) -> eyre::Result<()> { loop { tokio::time::sleep(std::time::Duration::from_millis(10)).await; - if let Some(checkpoint) = - self.inner.provider.get_stage_checkpoint(StageId::Headers)? - { - if checkpoint.block_number == number { - break - } + if let Some(checkpoint) = self.inner.provider.get_stage_checkpoint(StageId::Headers)? { + if checkpoint.block_number == number { + break } + } } Ok(()) } diff --git a/crates/stages-api/src/pipeline/mod.rs b/crates/stages-api/src/pipeline/mod.rs index 3df70e809aac..05f9c67e4636 100644 --- a/crates/stages-api/src/pipeline/mod.rs +++ b/crates/stages-api/src/pipeline/mod.rs @@ -147,7 +147,8 @@ where } self.progress.update(target); - // TODO: if report ControlFlow::Unwind we will issue a warning of bad block detected. Do we want? + // TODO: if report ControlFlow::Unwind we will issue a warning of bad block + // detected. Do we want? return (self, Ok(ControlFlow::Continue { block_number: target })) } } From 3dd46f79339356c7776cce10056df053bd5f3fa5 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 14:17:02 +0100 Subject: [PATCH 24/53] rename to update_block_hashes_and_clear_buffered --- crates/blockchain-tree/src/noop.rs | 2 +- crates/blockchain-tree/src/shareable.rs | 2 +- crates/consensus/beacon/src/engine/mod.rs | 2 +- crates/interfaces/src/blockchain_tree/mod.rs | 8 +++++--- crates/storage/provider/src/providers/mod.rs | 4 ++-- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/blockchain-tree/src/noop.rs b/crates/blockchain-tree/src/noop.rs index c71042676415..1d981007a92f 100644 --- a/crates/blockchain-tree/src/noop.rs +++ b/crates/blockchain-tree/src/noop.rs @@ -60,7 +60,7 @@ impl BlockchainTreeEngine for NoopBlockchainTree { Err(BlockchainTreeError::BlockHashNotFoundInChain { block_hash }.into()) } - fn update_block_hashes_and_remove_chains( + fn update_block_hashes_and_clear_buffered( &self, ) -> RethResult> { Ok(BTreeMap::new()) diff --git a/crates/blockchain-tree/src/shareable.rs b/crates/blockchain-tree/src/shareable.rs index 4e77d5096b3f..2ea8452bd2a9 100644 --- a/crates/blockchain-tree/src/shareable.rs +++ b/crates/blockchain-tree/src/shareable.rs @@ -82,7 +82,7 @@ where res } - fn update_block_hashes_and_remove_chains( + fn update_block_hashes_and_clear_buffered( &self, ) -> RethResult> { let mut tree = self.tree.write(); diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 6c15ef1a0103..8c7dcc76608f 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1629,7 +1629,7 @@ where } if sync_target_state.finalized_block_hash.is_zero() { - self.blockchain.update_block_hashes_and_remove_chains()?; + self.blockchain.update_block_hashes_and_clear_buffered()?; self.blockchain.connect_buffered_blocks_to_canonical_hashes()?; // We are on a optimistic syncing process, better to wait for the next FCU to handle return Ok(()) diff --git a/crates/interfaces/src/blockchain_tree/mod.rs b/crates/interfaces/src/blockchain_tree/mod.rs index 93d199fc21db..7d2b50e418e1 100644 --- a/crates/interfaces/src/blockchain_tree/mod.rs +++ b/crates/interfaces/src/blockchain_tree/mod.rs @@ -79,9 +79,11 @@ pub trait BlockchainTreeEngine: BlockchainTreeViewer + Send + Sync { ) -> RethResult<()>; /// Update all block hashes. iterate over present and new list of canonical hashes and compare - /// them. Remove all mismatches, disconnect them and removes all chains. - fn update_block_hashes_and_remove_chains(&self) - -> RethResult>; + /// them. Remove all mismatches, disconnect them, removes all chains and clears all buffered + /// blocks before the tip. + fn update_block_hashes_and_clear_buffered( + &self, + ) -> RethResult>; /// Reads the last `N` canonical hashes from the database and updates the block indices of the /// tree by attempting to connect the buffered blocks to canonical hashes. diff --git a/crates/storage/provider/src/providers/mod.rs b/crates/storage/provider/src/providers/mod.rs index d6f2c9bc98e4..eaf94627d22a 100644 --- a/crates/storage/provider/src/providers/mod.rs +++ b/crates/storage/provider/src/providers/mod.rs @@ -656,8 +656,8 @@ where self.tree.finalize_block(finalized_block) } - fn update_block_hashes_and_remove_chains(&self) -> RethResult> { - self.tree.update_block_hashes_and_remove_chains() + fn update_block_hashes_and_clear_buffered(&self) -> RethResult> { + self.tree.update_block_hashes_and_clear_buffered() } fn connect_buffered_blocks_to_canonical_hashes_and_finalize( From 0fcfb8dafc3f2e0bacb5985bad0ceee0cf0e0b91 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 15:00:44 +0100 Subject: [PATCH 25/53] clippy --- crates/stages-api/src/pipeline/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/stages-api/src/pipeline/mod.rs b/crates/stages-api/src/pipeline/mod.rs index 05f9c67e4636..5cb9c8d2e7d0 100644 --- a/crates/stages-api/src/pipeline/mod.rs +++ b/crates/stages-api/src/pipeline/mod.rs @@ -143,7 +143,7 @@ where return (self, Err(err.into())) } if let Err(err) = self.unwind(target, None) { - return (self, Err(err.into())) + return (self, Err(err)) } self.progress.update(target); From dab38a0aaa1ef82183ae2872c9bb8221312038be Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 15:33:07 +0100 Subject: [PATCH 26/53] remove optimistic syncing as an optimism feature flag only --- crates/consensus/beacon/src/engine/mod.rs | 28 +++++++++++----------- crates/consensus/beacon/src/engine/sync.rs | 4 ++++ 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index f818ade3f118..25f7af3e25f7 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -707,20 +707,20 @@ where return Some(state.finalized_block_hash) } - #[cfg(feature = "optimism")] - if self.blockchain.chain_spec().is_optimism() { - // It can happen when the node is doing an - // optimistic sync, where the CL has no knowledge of the finalized hash, - // but is expecting the EL to sync as high - // as possible before finalizing. - // - // This usually doesn't happen on ETH mainnet since CLs use the more - // secure checkpoint syncing. - // - // However, optimism chains will do this. The risk of a reorg is however - // low. - return Some(state.head_block_hash) - } + + // OPTIMISTIC SYNCING + // + // It can happen when the node is doing an + // optimistic sync, where the CL has no knowledge of the finalized hash, + // but is expecting the EL to sync as high + // as possible before finalizing. + // + // This usually doesn't happen on ETH mainnet since CLs use the more + // secure checkpoint syncing. + // + // However, optimism chains will do this. The risk of a reorg is however + // low. + return Some(state.head_block_hash) } Ok(Some(_)) => { // we're fully synced to the finalized block diff --git a/crates/consensus/beacon/src/engine/sync.rs b/crates/consensus/beacon/src/engine/sync.rs index 1978836203d3..6d281ccfe2b4 100644 --- a/crates/consensus/beacon/src/engine/sync.rs +++ b/crates/consensus/beacon/src/engine/sync.rs @@ -218,6 +218,10 @@ where /// But ensures the target is not the zero hash. pub(crate) fn set_pipeline_sync_target(&mut self, target: PipelineTarget) { if target.sync_target().is_some_and(|target| target.is_zero()) { + trace!( + target: "consensus::engine::sync", + "Pipeline target cannot be zero hash." + ); // precaution to never sync to the zero hash return } From e22e18727ae8bef5e6a3d24ee38a8b017b72e7c7 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 15:41:07 +0100 Subject: [PATCH 27/53] add doc to OptimisticCanonicalRevert --- crates/blockchain-tree/src/blockchain_tree.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index c4814857f67f..2f4896c5e076 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1228,6 +1228,7 @@ where .unwrap_or_default() > revert_until { + // should only happen when an optimistic sync target was re-orged. return Err(CanonicalError::OptimisticCanonicalRevert(revert_until)) } From f9c584eca78f40a874ae2a839c7f65c4b08040cc Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 15:56:27 +0100 Subject: [PATCH 28/53] clippy fmt --- crates/consensus/beacon/src/engine/mod.rs | 1 - crates/e2e-test-utils/src/node.rs | 1 - 2 files changed, 2 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 25f7af3e25f7..6fa4a9bc0a32 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -707,7 +707,6 @@ where return Some(state.finalized_block_hash) } - // OPTIMISTIC SYNCING // // It can happen when the node is doing an diff --git a/crates/e2e-test-utils/src/node.rs b/crates/e2e-test-utils/src/node.rs index 3d4e5e5d2b80..8d4341e8ec00 100644 --- a/crates/e2e-test-utils/src/node.rs +++ b/crates/e2e-test-utils/src/node.rs @@ -14,7 +14,6 @@ use reth::{ }; use reth_node_builder::NodeTypes; use reth_primitives::{stage::StageId, BlockHash, BlockNumber, Bytes, B256}; -use reth_provider::BlockNumReader; use std::{marker::PhantomData, pin::Pin}; use tokio_stream::StreamExt; From ce6de02fa2117f7301799e256c46e24b0d55f781 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 29 Apr 2024 16:01:00 +0100 Subject: [PATCH 29/53] cargo doc --- crates/rpc/rpc/src/eth/error.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/rpc/rpc/src/eth/error.rs b/crates/rpc/rpc/src/eth/error.rs index 942ee5c5a9a6..b080b51bddc6 100644 --- a/crates/rpc/rpc/src/eth/error.rs +++ b/crates/rpc/rpc/src/eth/error.rs @@ -41,9 +41,9 @@ pub enum EthApiError { /// finalized, /// /// op-node uses case sensitive string comparison to parse this error: - /// https://github.com/ethereum-optimism/optimism/blob/0913776869f6cb2c1218497463d7377cf4de16de/op-service/sources/l2_client.go#L105 + /// /// - /// Temporary, until a version of https://github.com/ethereum-optimism/optimism/pull/10071 is pushed through that doesn't require this to figure out the EL sync status. + /// Temporary, until a version of is pushed through that doesn't require this to figure out the EL sync status. #[error("Unknown block")] UnknownSafeOrFinalizedBlock, /// Thrown when an unknown block or transaction index is encountered From 0dcf372265261bebf87b38e1e656fdf638466234 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Tue, 30 Apr 2024 11:45:16 +0100 Subject: [PATCH 30/53] add more docs on revert_canonical_from_database & static --- crates/blockchain-tree/src/blockchain_tree.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 2f4896c5e076..77eff0650f7f 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1228,7 +1228,12 @@ where .unwrap_or_default() > revert_until { - // should only happen when an optimistic sync target was re-orged. + // This should only happen when an optimistic sync target was re-orged. + // + // Static files generally contain finalized data. The blockchain tree only deals + // with unfinalized data. The only scenario where canonical reverts go past the highest + // static file is when an optimistic sync occured and unfinalized data was written to + // static files. return Err(CanonicalError::OptimisticCanonicalRevert(revert_until)) } From c2f7b39c5de52d2798540656b1da7e60da1541ee Mon Sep 17 00:00:00 2001 From: joshieDo Date: Tue, 30 Apr 2024 12:10:59 +0100 Subject: [PATCH 31/53] import StaticFileProviderFactory --- crates/blockchain-tree/src/blockchain_tree.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 77eff0650f7f..b0e68ed938d6 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -25,6 +25,7 @@ use reth_provider::{ BlockExecutionWriter, BlockNumReader, BlockWriter, BundleStateWithReceipts, CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications, Chain, ChainSpecProvider, DisplayBlocksChain, ExecutorFactory, HeaderProvider, ProviderError, + StaticFileProviderFactory, }; use reth_stages_api::{MetricEvent, MetricEventsSender}; use std::{ From c0c83c54b1e0d106dfb2e591745f076265006cf7 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Tue, 30 Apr 2024 12:50:14 +0100 Subject: [PATCH 32/53] extend test --- crates/e2e-test-utils/src/engine_api.rs | 2 +- crates/optimism/node/tests/e2e/p2p.rs | 44 ++++++++++++++++++------- 2 files changed, 33 insertions(+), 13 deletions(-) diff --git a/crates/e2e-test-utils/src/engine_api.rs b/crates/e2e-test-utils/src/engine_api.rs index 9ede69e67441..58b5ff24691a 100644 --- a/crates/e2e-test-utils/src/engine_api.rs +++ b/crates/e2e-test-utils/src/engine_api.rs @@ -51,7 +51,7 @@ impl EngineApiTestContext { ) .await?; - assert!(submission.status == expected_status); + assert_eq!(submission.status, expected_status); Ok(submission.latest_valid_hash.unwrap_or_default()) } diff --git a/crates/optimism/node/tests/e2e/p2p.rs b/crates/optimism/node/tests/e2e/p2p.rs index 958f5576e550..9e3741055ab6 100644 --- a/crates/optimism/node/tests/e2e/p2p.rs +++ b/crates/optimism/node/tests/e2e/p2p.rs @@ -1,4 +1,5 @@ use crate::utils::{advance_chain, setup}; +use reth_interfaces::blockchain_tree::error::BlockchainTreeError; use reth_rpc_types::engine::PayloadStatusEnum; use std::sync::Arc; use tokio::sync::Mutex; @@ -16,29 +17,34 @@ async fn can_sync() -> eyre::Result<()> { let tip: usize = 90; let tip_index: usize = tip - 1; + let reorg_depth = 2; // On first node, create a chain up to block number 90a let canonical_payload_chain = advance_chain(tip, &mut first_node, wallet.clone()).await?; let canonical_chain = canonical_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // On second node, sync optimistically up to block number 89a - second_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index - 1]).await?; - second_node.wait_block(tip as u64 - 1, canonical_chain[tip_index - 1], true).await?; + // On second node, sync optimistically up to block number 88a + second_node + .engine_api + .update_optimistic_forkchoice(canonical_chain[tip_index - reorg_depth]) + .await?; + second_node + .wait_block((tip - reorg_depth) as u64, canonical_chain[tip_index - reorg_depth], true) + .await?; // On third node, sync optimistically up to block number 90a third_node.engine_api.update_optimistic_forkchoice(canonical_chain[tip_index]).await?; third_node.wait_block(tip as u64, canonical_chain[tip_index], true).await?; - let reorg_depth = 1usize; - - // On second node, create a side chain: 89a -> 90b + // On second node, create a side chain: 88a -> 89b -> 90b wallet.lock().await.inner_nonce -= reorg_depth as u64; second_node.payload.timestamp = first_node.payload.timestamp - reorg_depth as u64; // TODO: probably want to make it node agnostic let side_payload_chain = advance_chain(reorg_depth, &mut second_node, wallet.clone()).await?; let side_chain = side_payload_chain.iter().map(|p| p.0.block().hash()).collect::>(); - // It will create a fork chain + // Creates fork chain by submitting 89b payload. + // By returning Valid here, op-node will finally return a finalized hash let _ = third_node .engine_api .submit_payload( @@ -49,11 +55,8 @@ async fn can_sync() -> eyre::Result<()> { ) .await; - // It will issue a pipeline reorg - third_node - .engine_api - .update_forkchoice(side_chain[reorg_depth - 1], side_chain[reorg_depth - 1]) - .await?; + // It will issue a pipeline reorg to 88a, and then make 89b canonical AND finalized. + third_node.engine_api.update_forkchoice(side_chain[0], side_chain[0]).await?; // Make sure we have the updated block third_node.wait_unwind((tip - reorg_depth) as u64).await?; @@ -65,5 +68,22 @@ async fn can_sync() -> eyre::Result<()> { ) .await?; + // Make sure that trying to submit 89a again will result in an invalid payload status, since 89b + // has been set as finalized. + let _ = third_node + .engine_api + .submit_payload( + canonical_payload_chain[tip_index - reorg_depth + 1].0.clone(), + canonical_payload_chain[tip_index - reorg_depth + 1].1.clone(), + PayloadStatusEnum::Invalid { + validation_error: BlockchainTreeError::PendingBlockIsFinalized { + last_finalized: (tip - reorg_depth) as u64 + 1, + } + .to_string(), + }, + Default::default(), + ) + .await; + Ok(()) } From a3f08cd0e74c4bce7c9a6a1457f4109d038b7dfe Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 1 May 2024 21:13:11 +0100 Subject: [PATCH 33/53] allow downloading subset, if the set has terminated --- crates/net/downloaders/src/bodies/bodies.rs | 56 ++++++++++++++++++++- 1 file changed, 54 insertions(+), 2 deletions(-) diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index 533c02e356e6..4d5e627b54f6 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -317,7 +317,7 @@ where // Check if the provided range is the subset of the existing range. let is_current_range_subset = self.download_range.contains(range.start()) && *range.end() == *self.download_range.end(); - if is_current_range_subset { + if is_current_range_subset && !self.is_terminated() { tracing::trace!(target: "downloaders::bodies", ?range, "Download range already in progress"); // The current range already includes requested. return Ok(()) @@ -354,7 +354,6 @@ where fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); if this.is_terminated() { - this.download_range = RangeInclusive::new(1, 0); return Poll::Ready(None); } // Submit new requests and poll any in progress @@ -818,4 +817,57 @@ mod tests { Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().take(100), &mut bodies)) ); } + + // Check that the downloader can re-download a subset of the range + #[tokio::test] + async fn can_download_existing_subset_range_after_termination() { + // Generate some random blocks + let db = create_test_rw_db(); + let (headers, mut bodies) = generate_bodies(0..=199); + + let repeated_number = 99u64; + let repeated_header = headers.get(repeated_number as usize).expect("exists").hash(); + let repeated_body = bodies.get(&repeated_header).expect("exists").clone(); + insert_headers(db.db(), &headers); + + let client = Arc::new(TestBodiesClient::default().with_bodies(bodies.clone())); + let (_static_dir, static_dir_path) = create_test_static_files_dir(); + + let mut downloader = BodiesDownloaderBuilder::default().with_stream_batch_size(100).build( + client.clone(), + Arc::new(TestConsensus::default()), + ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), + ); + + // Set and download the first range + downloader.set_download_range(0..=99).expect("failed to set download range"); + assert_matches!( + downloader.next().await, + Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().take(100), &mut bodies)) + ); + + // Check that the stream is terminated + assert!(downloader.next().await.is_none()); + + // Set and download the second range + bodies.insert(repeated_header, repeated_body); + downloader.client = + Arc::new(Arc::new(TestBodiesClient::default().with_bodies(bodies.clone()))); + downloader + .set_download_range(repeated_number..=repeated_number) + .expect("failed to set download range"); + assert_matches!( + downloader.next().await, + Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().skip(99).take(1), &mut bodies)) + ); + + // Resume with the rest + downloader + .set_download_range(repeated_number + 1..=199) + .expect("failed to set download range"); + assert_matches!( + downloader.next().await, + Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().skip(100), &mut bodies)) + ); + } } From 0da350cb7ba4efadb7683134cc96256cb0d1858c Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 1 May 2024 21:25:52 +0100 Subject: [PATCH 34/53] dont allow subsets --- crates/net/downloaders/src/bodies/bodies.rs | 55 +-------------------- 1 file changed, 1 insertion(+), 54 deletions(-) diff --git a/crates/net/downloaders/src/bodies/bodies.rs b/crates/net/downloaders/src/bodies/bodies.rs index 4d5e627b54f6..985c545e9b39 100644 --- a/crates/net/downloaders/src/bodies/bodies.rs +++ b/crates/net/downloaders/src/bodies/bodies.rs @@ -317,7 +317,7 @@ where // Check if the provided range is the subset of the existing range. let is_current_range_subset = self.download_range.contains(range.start()) && *range.end() == *self.download_range.end(); - if is_current_range_subset && !self.is_terminated() { + if is_current_range_subset { tracing::trace!(target: "downloaders::bodies", ?range, "Download range already in progress"); // The current range already includes requested. return Ok(()) @@ -817,57 +817,4 @@ mod tests { Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().take(100), &mut bodies)) ); } - - // Check that the downloader can re-download a subset of the range - #[tokio::test] - async fn can_download_existing_subset_range_after_termination() { - // Generate some random blocks - let db = create_test_rw_db(); - let (headers, mut bodies) = generate_bodies(0..=199); - - let repeated_number = 99u64; - let repeated_header = headers.get(repeated_number as usize).expect("exists").hash(); - let repeated_body = bodies.get(&repeated_header).expect("exists").clone(); - insert_headers(db.db(), &headers); - - let client = Arc::new(TestBodiesClient::default().with_bodies(bodies.clone())); - let (_static_dir, static_dir_path) = create_test_static_files_dir(); - - let mut downloader = BodiesDownloaderBuilder::default().with_stream_batch_size(100).build( - client.clone(), - Arc::new(TestConsensus::default()), - ProviderFactory::new(db, MAINNET.clone(), static_dir_path).unwrap(), - ); - - // Set and download the first range - downloader.set_download_range(0..=99).expect("failed to set download range"); - assert_matches!( - downloader.next().await, - Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().take(100), &mut bodies)) - ); - - // Check that the stream is terminated - assert!(downloader.next().await.is_none()); - - // Set and download the second range - bodies.insert(repeated_header, repeated_body); - downloader.client = - Arc::new(Arc::new(TestBodiesClient::default().with_bodies(bodies.clone()))); - downloader - .set_download_range(repeated_number..=repeated_number) - .expect("failed to set download range"); - assert_matches!( - downloader.next().await, - Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().skip(99).take(1), &mut bodies)) - ); - - // Resume with the rest - downloader - .set_download_range(repeated_number + 1..=199) - .expect("failed to set download range"); - assert_matches!( - downloader.next().await, - Some(Ok(res)) => assert_eq!(res, zip_blocks(headers.iter().skip(100), &mut bodies)) - ); - } } From fd51d3688c8eaa8abb516728b75266b39fc1f88c Mon Sep 17 00:00:00 2001 From: joshieDo Date: Wed, 1 May 2024 21:43:07 +0100 Subject: [PATCH 35/53] only run pruner once we have a finalized block --- crates/consensus/beacon/src/engine/hooks/controller.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/crates/consensus/beacon/src/engine/hooks/controller.rs b/crates/consensus/beacon/src/engine/hooks/controller.rs index 48343d4804f5..53b1c210f924 100644 --- a/crates/consensus/beacon/src/engine/hooks/controller.rs +++ b/crates/consensus/beacon/src/engine/hooks/controller.rs @@ -138,6 +138,12 @@ impl EngineHooksController { return Poll::Pending } + // If we don't have a finalized block, we might be on an optimistic sync scenario where we + // cannot skip the FCU with the finalized hash, otherwise CL might misbehave. + if hook.db_access_level().is_read_write() && args.finalized_block_number.is_none() { + return Poll::Pending + } + if let Poll::Ready(event) = hook.poll(cx, args)? { let result = PolledHook { name: hook.name(), event, db_access_level: hook.db_access_level() }; From 17f886c721a3890d0a85a986b23de6dc4772d5fb Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 17:03:27 +0100 Subject: [PATCH 36/53] review traces and docs --- bin/reth/src/optimism.rs | 4 ++-- crates/blockchain-tree/src/blockchain_tree.rs | 5 +++++ .../beacon/src/engine/hooks/controller.rs | 18 +++++++++--------- crates/consensus/beacon/src/engine/mod.rs | 1 + crates/primitives/src/stage/mod.rs | 8 ++++---- 5 files changed, 21 insertions(+), 15 deletions(-) diff --git a/bin/reth/src/optimism.rs b/bin/reth/src/optimism.rs index ee267cf7d6f7..524860726a24 100644 --- a/bin/reth/src/optimism.rs +++ b/bin/reth/src/optimism.rs @@ -24,7 +24,7 @@ fn main() { } if let Err(err) = Cli::::parse().run(|builder, rollup_args| async move { - let NodeHandle { node: _node, node_exit_future } = builder + let handle = builder .node(OptimismNode::new(rollup_args.clone())) .extend_rpc_modules(move |ctx| { // register sequencer tx forwarder @@ -39,7 +39,7 @@ fn main() { .launch() .await?; - node_exit_future.await + handle.node_exit_future.await }) { eprintln!("Error: {err:?}"); std::process::exit(1); diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index b0e68ed938d6..e238253694c4 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1229,6 +1229,11 @@ where .unwrap_or_default() > revert_until { + trace!( + target: "blockchain_tree", + "Reverting optimistic canonical chain to block {}", + revert_until + ); // This should only happen when an optimistic sync target was re-orged. // // Static files generally contain finalized data. The blockchain tree only deals diff --git a/crates/consensus/beacon/src/engine/hooks/controller.rs b/crates/consensus/beacon/src/engine/hooks/controller.rs index 53b1c210f924..b39957d3178d 100644 --- a/crates/consensus/beacon/src/engine/hooks/controller.rs +++ b/crates/consensus/beacon/src/engine/hooks/controller.rs @@ -130,20 +130,20 @@ impl EngineHooksController { args: EngineHookContext, db_write_active: bool, ) -> Poll> { - // Hook with DB write access level is not allowed to run due to already running hook with DB - // write access level or active DB write according to passed argument + // Hook with DB write access level is not allowed to run due to any of the following + // reasons: + // - An already running hook with DB write access level + // - Active DB write according to passed argument + // - Missing a finalized block number. We might be on an optimistic sync scenario where we + // cannot skip the FCU with the finalized hash, otherwise CL might misbehave. if hook.db_access_level().is_read_write() && - (self.active_db_write_hook.is_some() || db_write_active) + (self.active_db_write_hook.is_some() || + db_write_active || + args.finalized_block_number.is_none()) { return Poll::Pending } - // If we don't have a finalized block, we might be on an optimistic sync scenario where we - // cannot skip the FCU with the finalized hash, otherwise CL might misbehave. - if hook.db_access_level().is_read_write() && args.finalized_block_number.is_none() { - return Poll::Pending - } - if let Poll::Ready(event) = hook.poll(cx, args)? { let result = PolledHook { name: hook.name(), event, db_access_level: hook.db_access_level() }; diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 6fa4a9bc0a32..3e3d8aa13847 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -719,6 +719,7 @@ where // // However, optimism chains will do this. The risk of a reorg is however // low. + debug!(target: "consensus::engine", hash=?state.head_block_hash, "Setting head hash as an optimistic pipeline target."); return Some(state.head_block_hash) } Ok(Some(_)) => { diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index 5169d8dc0f2d..ec089ce9ddbf 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -22,17 +22,17 @@ pub enum PipelineTarget { impl PipelineTarget { /// Target for forward synchronization. - pub fn sync_target(&self) -> Option { + pub fn sync_target(self) -> Option { match self { - PipelineTarget::Sync(hash) => Some(*hash), + PipelineTarget::Sync(hash) => Some(hash), PipelineTarget::Unwind(_) => None, } } /// Target for backward unwinding. - pub fn unwind_target(&self) -> Option { + pub fn unwind_target(self) -> Option { match self { PipelineTarget::Sync(_) => None, - PipelineTarget::Unwind(number) => Some(*number), + PipelineTarget::Unwind(number) => Some(number), } } } From 89077c6903d9f86ca7fd718749ce02ca17a010c9 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 17:42:08 +0100 Subject: [PATCH 37/53] clippy --- bin/reth/src/optimism.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/bin/reth/src/optimism.rs b/bin/reth/src/optimism.rs index 291965955a19..58171879774a 100644 --- a/bin/reth/src/optimism.rs +++ b/bin/reth/src/optimism.rs @@ -2,7 +2,6 @@ use clap::Parser; use reth::cli::Cli; -use reth_node_builder::NodeHandle; use reth_node_optimism::{args::RollupArgs, rpc::SequencerClient, OptimismNode}; use std::sync::Arc; From 61e5898c24cb9faa6c19fe50a69571f6ff762d1a Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 18:36:57 +0100 Subject: [PATCH 38/53] unbreak continuous pipeline --- crates/consensus/beacon/src/engine/mod.rs | 29 +++++++++++++---------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 8994a96a7a10..c4f85e4c9bb5 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1508,6 +1508,22 @@ where return Ok(()) } + let set_canonical_head = || { + let max_block = ctrl.block_number().unwrap_or_default(); + let max_header = self.blockchain.sealed_header(max_block) + .inspect_err(|error| { + error!(target: "consensus::engine", %error, "Error getting canonical header for continuous sync"); + })? + .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; + self.blockchain.set_canonical_head(max_header); + Ok::<(), RethError>(()) + }; + + // update the canon chain if continuous is enabled + if self.sync.run_pipeline_continuously() { + set_canonical_head()?; + } + let sync_target_state = match self.forkchoice_state_tracker.sync_target_state() { Some(current_state) => current_state, None => { @@ -1518,19 +1534,8 @@ where } }; - // update the canon chain if continuous is enabled - if self.sync.run_pipeline_continuously() || sync_target_state.finalized_block_hash.is_zero() - { - let max_block = ctrl.block_number().unwrap_or_default(); - let max_header = self.blockchain.sealed_header(max_block) - .inspect_err(|error| { - error!(target: "consensus::engine", %error, "Error getting canonical header for continuous sync"); - })? - .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; - self.blockchain.set_canonical_head(max_header); - } - if sync_target_state.finalized_block_hash.is_zero() { + set_canonical_head()?; self.blockchain.update_block_hashes_and_clear_buffered()?; self.blockchain.connect_buffered_blocks_to_canonical_hashes()?; // We are on a optimistic syncing process, better to wait for the next FCU to handle From 89b9645d607be099a6315a43aafa9d7f15b3583e Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 18:44:44 +0100 Subject: [PATCH 39/53] handle err if optimistic revert --- crates/consensus/beacon/src/engine/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index c4f85e4c9bb5..164702cd2247 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1687,6 +1687,9 @@ where let _ = tx.send(Err(RethError::Canonical(error.clone()))); if error.is_fatal() { return Err(RethError::Canonical(error)) + } else if let Some(block_number) = error.is_optimistic_revert() { + self.sync + .set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } } }; From 02d7d0a78496a3412d5a5b98a676559f3e6d06f5 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 18:50:26 +0100 Subject: [PATCH 40/53] add set_canonical_head --- crates/consensus/beacon/src/engine/mod.rs | 26 +++++++++++------------ 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 082cd311690e..1ead079a40f8 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1508,20 +1508,9 @@ where return Ok(()) } - let set_canonical_head = || { - let max_block = ctrl.block_number().unwrap_or_default(); - let max_header = self.blockchain.sealed_header(max_block) - .inspect_err(|error| { - error!(target: "consensus::engine", %error, "Error getting canonical header for continuous sync"); - })? - .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; - self.blockchain.set_canonical_head(max_header); - Ok::<(), RethError>(()) - }; - // update the canon chain if continuous is enabled if self.sync.run_pipeline_continuously() { - set_canonical_head()?; + self.set_canonical_head(ctrl.block_number().unwrap_or_default())?; } let sync_target_state = match self.forkchoice_state_tracker.sync_target_state() { @@ -1535,7 +1524,7 @@ where }; if sync_target_state.finalized_block_hash.is_zero() { - set_canonical_head()?; + self.set_canonical_head(ctrl.block_number().unwrap_or_default())?; self.blockchain.update_block_hashes_and_clear_buffered()?; self.blockchain.connect_buffered_blocks_to_canonical_hashes()?; // We are on a optimistic syncing process, better to wait for the next FCU to handle @@ -1615,6 +1604,17 @@ where Ok(()) } + fn set_canonical_head(&mut self, max_block: BlockNumber) -> RethResult<()> { + let max_header = self.blockchain.sealed_header(max_block) + .inspect_err(|error| { + error!(target: "consensus::engine", %error, "Error getting canonical header for continuous sync"); + })? + .ok_or_else(|| ProviderError::HeaderNotFound(max_block.into()))?; + self.blockchain.set_canonical_head(max_header); + + Ok(()) + } + fn on_hook_result(&self, polled_hook: PolledHook) -> Result<(), BeaconConsensusEngineError> { if let EngineHookEvent::Finished(Err(error)) = &polled_hook.event { error!( From 963156a2ab72e8eabab658f6700d6b8112e16379 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 19:14:58 +0100 Subject: [PATCH 41/53] dedup pipeline unwind target setting --- crates/consensus/beacon/src/engine/mod.rs | 28 +++++++++++------------ 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 1ead079a40f8..e6f86dafd755 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1310,9 +1310,6 @@ where error, hash ) - } else if let Some(block_number) = error.is_optimistic_revert() { - self.sync - .set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } } } @@ -1687,9 +1684,6 @@ where let _ = tx.send(Err(RethError::Canonical(error.clone()))); if error.is_fatal() { return Err(RethError::Canonical(error)) - } else if let Some(block_number) = error.is_optimistic_revert() { - self.sync - .set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } } }; @@ -1766,16 +1760,20 @@ where Err(BeaconOnNewPayloadError::Internal(Box::new(error.clone()))); let _ = tx.send(response); return Err(RethError::Canonical(error)) + } else if error.is_optimistic_revert().is_some() { + // engine already set the pipeline unwind target on + // `try_make_sync_target_canonical` + PayloadStatus::new(PayloadStatusEnum::Syncing, None) + } else { + // If we could not make the sync target block canonical, + // we should return the error as an invalid payload status. + PayloadStatus::new( + PayloadStatusEnum::Invalid { validation_error: error.to_string() }, + // TODO: return a proper latest valid hash + // See: + self.forkchoice_state_tracker.last_valid_head(), + ) } - - // If we could not make the sync target block canonical, - // we should return the error as an invalid payload status. - PayloadStatus::new( - PayloadStatusEnum::Invalid { validation_error: error.to_string() }, - // TODO: return a proper latest valid hash - // See: - self.forkchoice_state_tracker.last_valid_head(), - ) } }; From e18a7048b069c3895b1590d7e6d9d34deca1732d Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 19:17:51 +0100 Subject: [PATCH 42/53] renames --- crates/blockchain-tree/src/blockchain_tree.rs | 2 +- crates/consensus/beacon/src/engine/mod.rs | 6 +++--- crates/interfaces/src/blockchain_tree/error.rs | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 5796527e8bc2..fd567cb38759 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1252,7 +1252,7 @@ where // with unfinalized data. The only scenario where canonical reverts go past the highest // static file is when an optimistic sync occured and unfinalized data was written to // static files. - return Err(CanonicalError::OptimisticCanonicalRevert(revert_until)) + return Err(CanonicalError::OptimisticTargetRevert(revert_until)) } // read data that is needed for new sidechain diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index e6f86dafd755..0ff162cd403c 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -976,7 +976,7 @@ where // so we should not warn the user, since this will result in us attempting to sync // to a new target and is considered normal operation during sync } - CanonicalError::OptimisticCanonicalRevert(block_number) => { + CanonicalError::OptimisticTargetRevert(block_number) => { self.sync.set_pipeline_sync_target(PipelineTarget::Unwind(*block_number)); return PayloadStatus::from_status(PayloadStatusEnum::Syncing) } @@ -1444,7 +1444,7 @@ where // TODO: do not ignore this let _ = self.blockchain.make_canonical(*target_hash.as_ref()); } - } else if let Some(block_number) = err.is_optimistic_revert() { + } else if let Some(block_number) = err.optimistic_revert_block_number() { self.sync.set_pipeline_sync_target(PipelineTarget::Unwind(block_number)); } @@ -1760,7 +1760,7 @@ where Err(BeaconOnNewPayloadError::Internal(Box::new(error.clone()))); let _ = tx.send(response); return Err(RethError::Canonical(error)) - } else if error.is_optimistic_revert().is_some() { + } else if error.optimistic_revert_block_number().is_some() { // engine already set the pipeline unwind target on // `try_make_sync_target_canonical` PayloadStatus::new(PayloadStatusEnum::Syncing, None) diff --git a/crates/interfaces/src/blockchain_tree/error.rs b/crates/interfaces/src/blockchain_tree/error.rs index 5746d54ef7e7..9b804a91237d 100644 --- a/crates/interfaces/src/blockchain_tree/error.rs +++ b/crates/interfaces/src/blockchain_tree/error.rs @@ -69,7 +69,7 @@ pub enum CanonicalError { CanonicalCommit(String), /// Error indicating that a previous optimistic sync target was re-orged #[error("transaction error on revert: {0}")] - OptimisticCanonicalRevert(BlockNumber), + OptimisticTargetRevert(BlockNumber), } impl CanonicalError { @@ -89,9 +89,9 @@ impl CanonicalError { /// Returns `Some(BlockNumber)` if the underlying error matches /// [CanonicalError::OptimisticCanonicalRevert]. - pub fn is_optimistic_revert(&self) -> Option { + pub fn optimistic_revert_block_number(&self) -> Option { match self { - CanonicalError::OptimisticCanonicalRevert(block_number) => Some(*block_number), + CanonicalError::OptimisticTargetRevert(block_number) => Some(*block_number), _ => None, } } @@ -330,7 +330,7 @@ impl InsertBlockErrorKind { CanonicalError::BlockchainTree(_) | CanonicalError::CanonicalCommit(_) | CanonicalError::CanonicalRevert(_) | - CanonicalError::OptimisticCanonicalRevert(_) => false, + CanonicalError::OptimisticTargetRevert(_) => false, CanonicalError::Validation(_) => true, CanonicalError::Provider(_) => false, }, From a2b9ac345008bc5deb0278373956cf232cdf588d Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 19:19:42 +0100 Subject: [PATCH 43/53] use PayloadStatus::from_status instead --- crates/consensus/beacon/src/engine/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 0ff162cd403c..ab4eeeac4c64 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1763,7 +1763,7 @@ where } else if error.optimistic_revert_block_number().is_some() { // engine already set the pipeline unwind target on // `try_make_sync_target_canonical` - PayloadStatus::new(PayloadStatusEnum::Syncing, None) + PayloadStatus::from_status(PayloadStatusEnum::Syncing) } else { // If we could not make the sync target block canonical, // we should return the error as an invalid payload status. From b3e354eec14cefbdfa35b7b2a6063272027f72eb Mon Sep 17 00:00:00 2001 From: joshieDo Date: Thu, 2 May 2024 19:20:32 +0100 Subject: [PATCH 44/53] cargo docs fix --- crates/interfaces/src/blockchain_tree/error.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/interfaces/src/blockchain_tree/error.rs b/crates/interfaces/src/blockchain_tree/error.rs index 9b804a91237d..a711a92da8e1 100644 --- a/crates/interfaces/src/blockchain_tree/error.rs +++ b/crates/interfaces/src/blockchain_tree/error.rs @@ -88,7 +88,7 @@ impl CanonicalError { } /// Returns `Some(BlockNumber)` if the underlying error matches - /// [CanonicalError::OptimisticCanonicalRevert]. + /// [CanonicalError::OptimisticTargetRevert]. pub fn optimistic_revert_block_number(&self) -> Option { match self { CanonicalError::OptimisticTargetRevert(block_number) => Some(*block_number), From 6119832649d60282ca84de41625af4911cc8d60d Mon Sep 17 00:00:00 2001 From: joshieDo Date: Fri, 3 May 2024 13:45:27 +0100 Subject: [PATCH 45/53] clippy --- crates/blockchain-tree/src/blockchain_tree.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 9508de614150..af5ca073978a 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -25,7 +25,7 @@ use reth_provider::{ chain::{ChainSplit, ChainSplitTarget}, BlockExecutionWriter, BlockNumReader, BlockWriter, BundleStateWithReceipts, CanonStateNotification, CanonStateNotificationSender, CanonStateNotifications, Chain, - ChainSpecProvider, DisplayBlocksChain, ExecutorFactory, HeaderProvider, ProviderError, + ChainSpecProvider, DisplayBlocksChain, HeaderProvider, ProviderError, StaticFileProviderFactory, }; use reth_stages_api::{MetricEvent, MetricEventsSender}; From e0867c8bdd0843f290faa3b34340097f2ddaf003 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Fri, 3 May 2024 13:56:12 +0100 Subject: [PATCH 46/53] updated clippy --- crates/consensus/beacon/src/engine/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index 07e433991882..cab7b0d30b35 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1601,7 +1601,7 @@ where Ok(()) } - fn set_canonical_head(&mut self, max_block: BlockNumber) -> RethResult<()> { + fn set_canonical_head(&self, max_block: BlockNumber) -> RethResult<()> { let max_header = self.blockchain.sealed_header(max_block) .inspect_err(|error| { error!(target: "consensus::engine", %error, "Error getting canonical header for continuous sync"); From d620b131fa7f7e97b8e6053c90bca38725bfe59b Mon Sep 17 00:00:00 2001 From: joshieDo Date: Fri, 3 May 2024 14:28:41 +0100 Subject: [PATCH 47/53] move update_block_hashes_and_clear_buffered impl to blockchain tree struct --- crates/blockchain-tree/src/blockchain_tree.rs | 15 +++++++++++++++ crates/blockchain-tree/src/shareable.rs | 9 +-------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index af5ca073978a..7d0de4d37f85 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -850,6 +850,21 @@ where Ok(last_canonical_hashes) } + /// Update all block hashes. iterate over present and new list of canonical hashes and compare + /// them. Remove all mismatches, disconnect them, removes all chains and clears all buffered + /// blocks before the tip. + pub fn update_block_hashes_and_clear_buffered( + &mut self, + ) -> RethResult> { + let chain = self.update_block_hashes()?; + + if let Some((block, _)) = chain.last_key_value() { + self.remove_old_blocks(*block); + } + + Ok(chain) + } + /// Reads the last `N` canonical hashes from the database and updates the block indices of the /// tree by attempting to connect the buffered blocks to canonical hashes. /// diff --git a/crates/blockchain-tree/src/shareable.rs b/crates/blockchain-tree/src/shareable.rs index 467a97bbcb87..77cc53c2d309 100644 --- a/crates/blockchain-tree/src/shareable.rs +++ b/crates/blockchain-tree/src/shareable.rs @@ -87,14 +87,7 @@ where &self, ) -> RethResult> { let mut tree = self.tree.write(); - let res = tree.update_block_hashes(); - - if let Ok(Some((block, _))) = - res.as_ref().map(|last_canonical_hashes| last_canonical_hashes.last_key_value()) - { - tree.remove_old_blocks(*block); - } - + let res = tree.update_block_hashes_and_clear_buffered(); tree.update_chains_metrics(); res } From 643c42093b72e0a4fcad69369672e5ce80caa149 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Fri, 3 May 2024 14:31:26 +0100 Subject: [PATCH 48/53] add TODO to doc --- crates/rpc/rpc/src/eth/error.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/rpc/rpc/src/eth/error.rs b/crates/rpc/rpc/src/eth/error.rs index b080b51bddc6..7be3e2e7fad1 100644 --- a/crates/rpc/rpc/src/eth/error.rs +++ b/crates/rpc/rpc/src/eth/error.rs @@ -43,7 +43,7 @@ pub enum EthApiError { /// op-node uses case sensitive string comparison to parse this error: /// /// - /// Temporary, until a version of is pushed through that doesn't require this to figure out the EL sync status. + /// TODO(#8045): Temporary, until a version of is pushed through that doesn't require this to figure out the EL sync status. #[error("Unknown block")] UnknownSafeOrFinalizedBlock, /// Thrown when an unknown block or transaction index is encountered From 53cacf4638b8bab74b110f3e069c7a3c9a20eb34 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Fri, 3 May 2024 14:32:05 +0100 Subject: [PATCH 49/53] whitespace --- crates/rpc/rpc/src/eth/error.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/rpc/rpc/src/eth/error.rs b/crates/rpc/rpc/src/eth/error.rs index 7be3e2e7fad1..6eef1f3ff2d4 100644 --- a/crates/rpc/rpc/src/eth/error.rs +++ b/crates/rpc/rpc/src/eth/error.rs @@ -43,7 +43,7 @@ pub enum EthApiError { /// op-node uses case sensitive string comparison to parse this error: /// /// - /// TODO(#8045): Temporary, until a version of is pushed through that doesn't require this to figure out the EL sync status. + /// TODO(#8045): Temporary, until a version of is pushed through that doesn't require this to figure out the EL sync status. #[error("Unknown block")] UnknownSafeOrFinalizedBlock, /// Thrown when an unknown block or transaction index is encountered From 42df97aa5c61b19b3654944ffc6059a04bb0a08e Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 6 May 2024 20:12:48 +0100 Subject: [PATCH 50/53] review comments --- crates/blockchain-tree/src/blockchain_tree.rs | 13 +++++++------ crates/consensus/beacon/src/engine/mod.rs | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index 7d0de4d37f85..b53600451a43 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1249,6 +1249,13 @@ where &self, revert_until: BlockNumber, ) -> Result, CanonicalError> { + + // This should only happen when an optimistic sync target was re-orged. + // + // Static files generally contain finalized data. The blockchain tree only deals + // with unfinalized data. The only scenario where canonical reverts go past the highest + // static file is when an optimistic sync occured and unfinalized data was written to + // static files. if self .externals .provider_factory @@ -1262,12 +1269,6 @@ where "Reverting optimistic canonical chain to block {}", revert_until ); - // This should only happen when an optimistic sync target was re-orged. - // - // Static files generally contain finalized data. The blockchain tree only deals - // with unfinalized data. The only scenario where canonical reverts go past the highest - // static file is when an optimistic sync occured and unfinalized data was written to - // static files. return Err(CanonicalError::OptimisticTargetRevert(revert_until)) } diff --git a/crates/consensus/beacon/src/engine/mod.rs b/crates/consensus/beacon/src/engine/mod.rs index cab7b0d30b35..4290f798308f 100644 --- a/crates/consensus/beacon/src/engine/mod.rs +++ b/crates/consensus/beacon/src/engine/mod.rs @@ -1524,7 +1524,7 @@ where self.set_canonical_head(ctrl.block_number().unwrap_or_default())?; self.blockchain.update_block_hashes_and_clear_buffered()?; self.blockchain.connect_buffered_blocks_to_canonical_hashes()?; - // We are on a optimistic syncing process, better to wait for the next FCU to handle + // We are on an optimistic syncing process, better to wait for the next FCU to handle return Ok(()) } From 041920786cbc28ebde4eaca91820d149e7a22d90 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 6 May 2024 20:13:16 +0100 Subject: [PATCH 51/53] remove unnecessary todo --- crates/stages-api/src/pipeline/mod.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/stages-api/src/pipeline/mod.rs b/crates/stages-api/src/pipeline/mod.rs index 6c2ee5678bf0..199cc41e6437 100644 --- a/crates/stages-api/src/pipeline/mod.rs +++ b/crates/stages-api/src/pipeline/mod.rs @@ -148,8 +148,6 @@ where } self.progress.update(target); - // TODO: if report ControlFlow::Unwind we will issue a warning of bad block - // detected. Do we want? return (self, Ok(ControlFlow::Continue { block_number: target })) } } From 0dd1825edcc533503c99d32c21b484723241efd5 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 6 May 2024 20:18:09 +0100 Subject: [PATCH 52/53] add more docs on PipelineTarget --- crates/primitives/src/stage/mod.rs | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/crates/primitives/src/stage/mod.rs b/crates/primitives/src/stage/mod.rs index ec089ce9ddbf..3c7c972bcf6f 100644 --- a/crates/primitives/src/stage/mod.rs +++ b/crates/primitives/src/stage/mod.rs @@ -11,24 +11,35 @@ pub use checkpoints::{ StageUnitCheckpoint, StorageHashingCheckpoint, }; -/// Direction and target for pipeline operations. +/// Direction and target block for pipeline operations. #[derive(Debug, Clone, Copy)] pub enum PipelineTarget { - /// Target for forward synchronization. + /// Target for forward synchronization, indicating a block hash to sync to. Sync(BlockHash), - /// Target for backward unwinding. + /// Target for backward unwinding, indicating a block number to unwind to. Unwind(BlockNumber), } impl PipelineTarget { - /// Target for forward synchronization. + /// Returns the target block hash for forward synchronization, if applicable. + /// + /// # Returns + /// + /// - `Some(BlockHash)`: The target block hash for forward synchronization. + /// - `None`: If the target is for backward unwinding. pub fn sync_target(self) -> Option { match self { PipelineTarget::Sync(hash) => Some(hash), PipelineTarget::Unwind(_) => None, } } - /// Target for backward unwinding. + + /// Returns the target block number for backward unwinding, if applicable. + /// + /// # Returns + /// + /// - `Some(BlockNumber)`: The target block number for backward unwinding. + /// - `None`: If the target is for forward synchronization. pub fn unwind_target(self) -> Option { match self { PipelineTarget::Sync(_) => None, From c17460b6b008179ec64e30dcf03fcc3ec95ad394 Mon Sep 17 00:00:00 2001 From: joshieDo Date: Mon, 6 May 2024 20:18:27 +0100 Subject: [PATCH 53/53] fmt --- crates/blockchain-tree/src/blockchain_tree.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/blockchain-tree/src/blockchain_tree.rs b/crates/blockchain-tree/src/blockchain_tree.rs index b53600451a43..3209c81bb8f7 100644 --- a/crates/blockchain-tree/src/blockchain_tree.rs +++ b/crates/blockchain-tree/src/blockchain_tree.rs @@ -1249,7 +1249,6 @@ where &self, revert_until: BlockNumber, ) -> Result, CanonicalError> { - // This should only happen when an optimistic sync target was re-orged. // // Static files generally contain finalized data. The blockchain tree only deals