From 02fe7440d04f9dbae5b41e01a4dcdcc329a843e5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 13:11:34 -0300 Subject: [PATCH 001/345] Corrupted History Fix This PR re-writes this branch's commits on top of the main branch: - Implement TrieIterator - Consume Trie when building TrieIterator - Implement `Store::iter_accounts` using TrieIterator --- crates/storage/store/storage.rs | 19 +++++++++++ crates/storage/trie/trie.rs | 13 +++++++- crates/storage/trie/trie_iter.rs | 56 ++++++++++++++++++++++++++++++++ 3 files changed, 87 insertions(+), 1 deletion(-) create mode 100644 crates/storage/trie/trie_iter.rs diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index e8d27b29c..2a93559ed 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -677,6 +677,25 @@ impl Store { Ok(trie.get_proof(&hash_key(storage_key))?) } + // Returns an iterator across all accounts in the state trie given by the state_root + // Does not check that the state_root is valid + pub fn iter_accounts( + &self, + state_root: H256, + ) -> Result, StoreError> { + Ok(self + .engine + .open_state_trie(state_root) + .into_iter() + .content() + .map_while(|(path, value)| { + Some(( + H256::decode(&path).ok()?, + AccountState::decode(&value).ok()?, + )) + })) + } + pub fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError> { self.engine.add_payload(payload_id, block) } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index b43559a16..46113c4e5 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -5,6 +5,7 @@ mod node; mod node_hash; mod rlp; mod state; +mod trie_iter; #[cfg(test)] mod test_utils; @@ -21,7 +22,7 @@ pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; pub use self::error::TrieError; -use self::{nibble::NibbleSlice, node::LeafNode, state::TrieState}; +use self::{nibble::NibbleSlice, node::LeafNode, state::TrieState, trie_iter::TrieIterator}; use lazy_static::lazy_static; @@ -206,6 +207,16 @@ impl Trie { } } +impl IntoIterator for Trie { + type Item = Node; + + type IntoIter = TrieIterator; + + fn into_iter(self) -> Self::IntoIter { + TrieIterator::new(self) + } +} + #[cfg(test)] mod test { use cita_trie::{MemoryDB as CitaMemoryDB, PatriciaTrie as CitaTrie, Trie as CitaTrieTrait}; diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs new file mode 100644 index 000000000..07a4475a0 --- /dev/null +++ b/crates/storage/trie/trie_iter.rs @@ -0,0 +1,56 @@ +use crate::{node::Node, node_hash::NodeHash, PathRLP, Trie, ValueRLP}; + +pub struct TrieIterator { + trie: Trie, + stack: Vec, +} + +impl TrieIterator { + pub(crate) fn new(trie: Trie) -> Self { + let stack = if let Some(root) = &trie.root { + vec![root.clone()] + } else { + vec![] + }; + Self { trie, stack } + } +} + +impl Iterator for TrieIterator { + type Item = Node; + + fn next(&mut self) -> Option { + if self.stack.is_empty() { + return None; + }; + // Fetch the last node in the stack + let next_node_hash = self.stack.pop()?; + let next_node = self.trie.state.get_node(next_node_hash).ok()??; + match &next_node { + Node::Branch(branch_node) => { + // Add all children to the stack (in reverse order so we process first child frist) + for child in branch_node.choices.iter().rev() { + if child.is_valid() { + self.stack.push(child.clone()) + } + } + } + Node::Extension(extension_node) => { + // Add child to the stack + self.stack.push(extension_node.child.clone()); + } + Node::Leaf(_) => {} + } + Some(next_node) + } +} + +impl TrieIterator { + pub fn content(self) -> impl Iterator { + self.filter_map(|n| match n { + Node::Branch(branch_node) => Some((branch_node.path, branch_node.value)), + Node::Extension(_) => None, + Node::Leaf(leaf_node) => Some((leaf_node.path, leaf_node.value)), + }) + } +} From a2a2aac2080b360afc4e96f0ecddf3d0fdfbf265 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 13:26:15 -0300 Subject: [PATCH 002/345] Fix slim encoding for AccountState --- crates/common/types/account.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/crates/common/types/account.rs b/crates/common/types/account.rs index 2919999f4..4e998dedb 100644 --- a/crates/common/types/account.rs +++ b/crates/common/types/account.rs @@ -139,12 +139,21 @@ impl RLPEncode for AccountState { impl RLPEncodeSlim for AccountState { fn encode(&self, buf: &mut dyn bytes::BufMut) { - // TODO: check if it's okay to use RLP_EMPTY_LIST + let root = if self.storage_root == EMPTY_TRIE_HASH { + &[] + } else { + self.storage_root.as_bytes() + }; + let code_hash = if self.code_hash == EMPTY_KECCACK_HASH { + &[] + } else { + self.code_hash.as_bytes() + }; Encoder::new(buf) .encode_field(&self.nonce) .encode_field(&self.balance) - .encode_field(&RLP_EMPTY_LIST) - .encode_field(&self.code_hash) + .encode_field(root) + .encode_field(code_hash) .finish(); } } From aef744146ffcbd4920f70fc8b2971d6da1b88fd4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 13:55:40 -0300 Subject: [PATCH 003/345] Fix slim encoding for AccountState --- crates/common/types/account.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/common/types/account.rs b/crates/common/types/account.rs index 4e998dedb..8bae95374 100644 --- a/crates/common/types/account.rs +++ b/crates/common/types/account.rs @@ -139,21 +139,21 @@ impl RLPEncode for AccountState { impl RLPEncodeSlim for AccountState { fn encode(&self, buf: &mut dyn bytes::BufMut) { - let root = if self.storage_root == EMPTY_TRIE_HASH { - &[] + let root = if self.storage_root == *EMPTY_TRIE_HASH { + Bytes::new() } else { - self.storage_root.as_bytes() + Bytes::copy_from_slice(self.storage_root.as_bytes()) }; - let code_hash = if self.code_hash == EMPTY_KECCACK_HASH { - &[] + let code_hash = if self.code_hash == *EMPTY_KECCACK_HASH { + Bytes::new() } else { - self.code_hash.as_bytes() + Bytes::copy_from_slice(self.code_hash.as_bytes()) }; Encoder::new(buf) .encode_field(&self.nonce) .encode_field(&self.balance) - .encode_field(root) - .encode_field(code_hash) + .encode_field(&root) + .encode_field(&code_hash) .finish(); } } From 13d9bc34cb168571d9ea603d91e088a158427599 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 17:10:44 -0300 Subject: [PATCH 004/345] Commit progress --- crates/networking/p2p/snap.rs | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 crates/networking/p2p/snap.rs diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs new file mode 100644 index 000000000..92812556e --- /dev/null +++ b/crates/networking/p2p/snap.rs @@ -0,0 +1,24 @@ +use ethereum_rust_storage::{error::StoreError, Store}; + +use crate::rlpx::snap::{AccountRange, GetAccountRange}; + +pub fn process_account_range_request(request: GetAccountRange, store: Store) -> Result { + let mut accounts = vec![]; + // Fetch account range + let mut iter = store.iter_accounts(request.root_hash); + let mut start_found = false; + while let Some((k, v)) = iter.next() { + if k == request.limit_hash { + break; + } + if k == request.starting_hash { + start_found = true; + } + if start_found{ + accounts.push((k, v)) + } + }; + let proof = store.get_account_range_proof(request.root_hash, request.starting_hash)?; + + Ok(AccountRange { id: request.id, accounts, proof }) +} From 0ca35898cdba20a568c7e2f0fa0bd57278e6d558 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 17:11:00 -0300 Subject: [PATCH 005/345] Commit progress --- crates/networking/p2p/net.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index b99b6eb58..a78f1806d 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -31,6 +31,7 @@ pub mod bootnode; pub(crate) mod discv4; pub(crate) mod kademlia; pub mod rlpx; +pub(crate) mod snap; pub mod types; const MAX_DISC_PACKET_SIZE: usize = 1280; From 66b6f90835af5f707439e94ed9c1cae64f52a5ce Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 18:52:28 -0300 Subject: [PATCH 006/345] Add snap as supported cap --- crates/networking/p2p/rlpx/connection.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 847a1c628..5c5ba23f2 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -21,8 +21,8 @@ use k256::{ use sha3::{Digest, Keccak256}; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tracing::{error, info}; -pub const SUPPORTED_CAPABILITIES: [(&str, u8); 2] = [("p2p", 5), ("eth", 68)]; -// pub const SUPPORTED_CAPABILITIES: [(&str, u8); 3] = [("p2p", 5), ("eth", 68), ("snap", 1)]; +// pub const SUPPORTED_CAPABILITIES: [(&str, u8); 2] = [("p2p", 5), ("eth", 68)]; +pub const SUPPORTED_CAPABILITIES: [(&str, u8); 3] = [("p2p", 5), ("eth", 68), ("snap", 1)]; pub(crate) type Aes256Ctr64BE = ctr::Ctr64BE; From 41dcaff2713bf83b7782f5ae5db573af04ccb489 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 12:27:03 -0300 Subject: [PATCH 007/345] Push missing changes --- crates/networking/p2p/rlpx/snap.rs | 35 +++++++++++++----------------- crates/networking/p2p/snap.rs | 15 +++++++++---- crates/storage/store/storage.rs | 19 ++++++++++------ 3 files changed, 38 insertions(+), 31 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index cdfafd1b6..98082e055 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -13,11 +13,11 @@ use super::{message::RLPxMessage, utils::snappy_encode}; pub(crate) struct GetAccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages - id: u64, - root_hash: H256, - starting_hash: H256, - limit_hash: H256, - response_bytes: u64, + pub id: u64, + pub root_hash: H256, + pub starting_hash: H256, + pub limit_hash: H256, + pub response_bytes: u64, } impl GetAccountRange { @@ -81,19 +81,9 @@ impl RLPxMessage for GetAccountRange { pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages - id: u64, - accounts: Vec<(H256, AccountState)>, - proof: Vec, -} - -impl AccountRange { - pub fn new(id: u64, accounts: Vec<(H256, AccountState)>, proof: Vec) -> Self { - Self { - id, - accounts, - proof, - } - } + pub id: u64, + pub accounts: Vec<(H256, AccountState)>, + pub proof: Vec>, } impl RLPxMessage for AccountRange { @@ -119,8 +109,13 @@ impl RLPxMessage for AccountRange { let (id, decoder): (u64, _) = decoder.decode_field("request-id")?; let (accounts, decoder): (Vec<(H256, AccountState)>, _) = decoder.decode_field("accounts")?; - let (proof, _): (Vec, _) = decoder.decode_field("proof")?; + let (proof, decoder) = decoder.decode_field("proof")?; + decoder.finish()?; - Ok(Self::new(id, accounts, proof)) + Ok(Self { + id, + accounts, + proof, + }) } } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 92812556e..aa47d10d4 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -2,7 +2,10 @@ use ethereum_rust_storage::{error::StoreError, Store}; use crate::rlpx::snap::{AccountRange, GetAccountRange}; -pub fn process_account_range_request(request: GetAccountRange, store: Store) -> Result { +pub fn process_account_range_request( + request: GetAccountRange, + store: Store, +) -> Result { let mut accounts = vec![]; // Fetch account range let mut iter = store.iter_accounts(request.root_hash); @@ -14,11 +17,15 @@ pub fn process_account_range_request(request: GetAccountRange, store: Store) -> if k == request.starting_hash { start_found = true; } - if start_found{ + if start_found { accounts.push((k, v)) } - }; + } let proof = store.get_account_range_proof(request.root_hash, request.starting_hash)?; - Ok(AccountRange { id: request.id, accounts, proof }) + Ok(AccountRange { + id: request.id, + accounts, + proof, + }) } diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 2a93559ed..9618d5cf3 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -679,12 +679,8 @@ impl Store { // Returns an iterator across all accounts in the state trie given by the state_root // Does not check that the state_root is valid - pub fn iter_accounts( - &self, - state_root: H256, - ) -> Result, StoreError> { - Ok(self - .engine + pub fn iter_accounts(&self, state_root: H256) -> impl Iterator { + self.engine .open_state_trie(state_root) .into_iter() .content() @@ -693,7 +689,16 @@ impl Store { H256::decode(&path).ok()?, AccountState::decode(&value).ok()?, )) - })) + }) + } + + pub fn get_account_range_proof( + &self, + state_root: H256, + starting_hash: H256, + ) -> Result>, StoreError> { + let state_trie = self.engine.open_state_trie(state_root); + Ok(state_trie.get_proof(&starting_hash.encode_to_vec())?) } pub fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError> { From a526c10cb74df6fb11b414da9ea8bd62bd454fbb Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Wed, 23 Oct 2024 15:44:13 -0300 Subject: [PATCH 008/345] Initial peer listen loop --- crates/networking/p2p/net.rs | 31 +++++++++-------- crates/networking/p2p/rlpx/connection.rs | 44 +++++++++++++++++++++--- crates/networking/p2p/rlpx/error.rs | 7 +++- crates/networking/p2p/rlpx/message.rs | 13 +++++++ 4 files changed, 74 insertions(+), 21 deletions(-) diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index b99b6eb58..1b368799e 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -714,7 +714,7 @@ async fn pong(socket: &UdpSocket, to_addr: SocketAddr, ping_hash: H256, signer: // TODO: remove this function. It's used for a hardcoded test // https://github.com/lambdaclass/lambda_ethereum_rust/issues/837 -async fn start_hardcoded_connection(tcp_addr: SocketAddr, signer: SigningKey, _storage: Store) { +async fn start_hardcoded_connection(tcp_addr: SocketAddr, signer: SigningKey, storage: Store) { let mut udp_addr = tcp_addr; udp_addr.set_port(tcp_addr.port() + 1); let udp_socket = UdpSocket::bind(udp_addr).await.unwrap(); @@ -754,48 +754,49 @@ async fn start_hardcoded_connection(tcp_addr: SocketAddr, signer: SigningKey, _s tcp_port: 30307u16, //endpoint.tcp_port, node_id, }; - handle_peer_as_initiator(signer, msg, &node).await; + handle_peer_as_initiator(signer, msg, &node, storage).await; } // TODO build a proper listen loop that receives requests from both // peers and business layer and propagate storage to use when required // https://github.com/lambdaclass/lambda_ethereum_rust/issues/840 -async fn serve_requests(tcp_addr: SocketAddr, signer: SigningKey, _storage: Store) { +async fn serve_requests(tcp_addr: SocketAddr, signer: SigningKey, storage: Store) { let tcp_socket = TcpSocket::new_v4().unwrap(); tcp_socket.bind(tcp_addr).unwrap(); let listener = tcp_socket.listen(50).unwrap(); loop { let (stream, _peer_addr) = listener.accept().await.unwrap(); - tokio::spawn(handle_peer_as_receiver(signer.clone(), stream)); + tokio::spawn(handle_peer_as_receiver( + signer.clone(), + stream, + storage.clone(), + )); } } -async fn handle_peer_as_receiver(signer: SigningKey, stream: TcpStream) { - let conn = RLPxConnection::receiver(signer, stream); +async fn handle_peer_as_receiver(signer: SigningKey, stream: TcpStream, storage: Store) { + let conn = RLPxConnection::receiver(signer, stream, storage); handle_peer(conn).await; } -async fn handle_peer_as_initiator(signer: SigningKey, msg: &[u8], node: &Node) { +async fn handle_peer_as_initiator(signer: SigningKey, msg: &[u8], node: &Node, storage: Store) { info!("Trying RLPx connection with {node:?}"); let stream = TcpSocket::new_v4() .unwrap() .connect(SocketAddr::new(node.ip, node.tcp_port)) .await .unwrap(); - let conn = RLPxConnection::initiator(signer, msg, stream).await; + let conn = RLPxConnection::initiator(signer, msg, stream, storage).await; handle_peer(conn).await; } async fn handle_peer(mut conn: RLPxConnection) { match conn.handshake().await { - Ok(_) => { - // TODO Properly build listen loop - // https://github.com/lambdaclass/lambda_ethereum_rust/issues/840 - // loop { - // conn.await_messages(); - // } - } + Ok(_) => match conn.main_loop().await { + Ok(_) => unreachable!(), + Err(e) => info!("Error during RLPx connection: ({e})"), + }, Err(e) => { // TODO propagate error to eventually discard peer from kademlia table info!("Handshake failed, discarding peer: ({e})"); diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 847a1c628..5be4136e2 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,5 +1,7 @@ use crate::{ - rlpx::{handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, + rlpx::{ + eth::StatusMessage, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey, + }, MAX_DISC_PACKET_SIZE, }; @@ -14,6 +16,7 @@ use aes::cipher::KeyIvInit; use bytes::BufMut as _; use ethereum_rust_core::{H256, H512}; use ethereum_rust_rlp::decode::RLPDecode; +use ethereum_rust_storage::Store; use k256::{ ecdsa::{RecoveryId, Signature, SigningKey, VerifyingKey}, PublicKey, SecretKey, @@ -31,20 +34,22 @@ pub(crate) struct RLPxConnection { signer: SigningKey, state: RLPxConnectionState, stream: S, + storage: Store, capabilities: Vec<(String, u8)>, } impl RLPxConnection { - fn new(signer: SigningKey, stream: S, state: RLPxConnectionState) -> Self { + fn new(signer: SigningKey, stream: S, state: RLPxConnectionState, storage: Store) -> Self { Self { signer, state, stream, + storage, capabilities: vec![], } } - pub fn receiver(signer: SigningKey, stream: S) -> Self { + pub fn receiver(signer: SigningKey, stream: S, storage: Store) -> Self { let mut rng = rand::thread_rng(); Self::new( signer, @@ -53,10 +58,11 @@ impl RLPxConnection { H256::random_using(&mut rng), SecretKey::random(&mut rng), )), + storage, ) } - pub async fn initiator(signer: SigningKey, msg: &[u8], stream: S) -> Self { + pub async fn initiator(signer: SigningKey, msg: &[u8], stream: S, storage: Store) -> Self { let mut rng = rand::thread_rng(); let digest = Keccak256::digest(&msg[65..]); let signature = &Signature::from_bytes(msg[..64].into()).unwrap(); @@ -67,7 +73,33 @@ impl RLPxConnection { SecretKey::random(&mut rng), pubkey2id(&peer_pk.into()), )); - RLPxConnection::new(signer, stream, state) + RLPxConnection::new(signer, stream, state, storage) + } + + pub async fn main_loop(&mut self) -> Result<(), RLPxError> { + match &self.state { + RLPxConnectionState::Established(_) => { + info!("Started peer main loop"); + loop { + match self.receive().await { + Message::Disconnect(_) => info!("Received Disconnect"), + Message::Ping(_) => info!("Received Ping"), + Message::Pong(_) => info!("Received Pong"), + Message::Status(_) => { + info!("Received Status"); + let message = + Message::Status(StatusMessage::new(&self.storage).unwrap()); + self.send(message).await; + info!("Sent Status"); + } + message => return Err(RLPxError::UnexpectedMessage(message)), + }; + } + } + _ => Err(RLPxError::InvalidState( + "Invalid connection state".to_string(), + )), + } } pub async fn handshake(&mut self) -> Result<(), RLPxError> { @@ -90,6 +122,8 @@ impl RLPxConnection { self.exchange_hello_messages().await?; info!("Completed Hello roundtrip!"); + // let message = Message::Status(StatusMessage::new(&self.storage).unwrap()); + // self.send(message); Ok(()) } diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index e74d4339a..4177ea10f 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,8 +1,13 @@ +use crate::rlpx::message::Message; use thiserror::Error; // TODO improve errors #[derive(Debug, Error)] -pub enum RLPxError { +pub(crate) enum RLPxError { #[error("{0}")] HandshakeError(String), + #[error("{0}")] + InvalidState(String), + #[error("Unexpected message: {0}")] + UnexpectedMessage(Message), } diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index e94b2ccb8..480e180bf 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -1,5 +1,6 @@ use bytes::BufMut; use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; +use std::fmt::Display; use super::eth::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; @@ -40,3 +41,15 @@ impl Message { } } } + +impl Display for Message { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Message::Hello(_) => "p2p:Hello".fmt(f), + Message::Disconnect(_) => "p2p:Disconnect".fmt(f), + Message::Ping(_) => "p2p:Ping".fmt(f), + Message::Pong(_) => "p2p:Pong".fmt(f), + Message::Status(_) => "eth:Status".fmt(f), + } + } +} From 562bc49336adfd16b0fe99f086816acec4e6d56a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 16:07:45 -0300 Subject: [PATCH 009/345] Deprecate `RLPEncodeSlim` trait and use `AccountStateSlim` instead --- crates/common/rlp/encode.rs | 42 ------------- crates/common/rlp/structs.rs | 9 --- crates/common/types/account.rs | 36 +---------- crates/networking/p2p/rlpx/snap.rs | 97 +++++++++++++++++++++++++++--- crates/networking/p2p/snap.rs | 2 +- 5 files changed, 93 insertions(+), 93 deletions(-) diff --git a/crates/common/rlp/encode.rs b/crates/common/rlp/encode.rs index 6b93886d1..2f25755d7 100644 --- a/crates/common/rlp/encode.rs +++ b/crates/common/rlp/encode.rs @@ -29,16 +29,6 @@ pub trait RLPEncode { } } -pub trait RLPEncodeSlim { - fn encode(&self, buf: &mut dyn BufMut); - - fn length(&self) -> usize { - let mut buf = Vec::new(); - self.encode(&mut buf); - buf.len() - } -} - impl RLPEncode for bool { #[inline(always)] fn encode(&self, buf: &mut dyn BufMut) { @@ -378,38 +368,6 @@ impl RLPEncode for ethereum_types::H256 { } } -impl RLPEncodeSlim for ethereum_types::H256 { - fn encode(&self, buf: &mut dyn BufMut) { - self.as_bytes().encode(buf) - } -} - -impl RLPEncodeSlim for Vec { - fn encode(&self, buf: &mut dyn BufMut) { - if self.is_empty() { - buf.put_u8(0xc0); - } else { - let mut total_len = 0; - for item in self { - total_len += item.length(); - } - encode_length(total_len, buf); - for item in self { - item.encode(buf); - } - } - } -} - -impl RLPEncodeSlim for (S, T) { - fn encode(&self, buf: &mut dyn BufMut) { - let total_len = self.0.length() + self.1.length(); - encode_length(total_len, buf); - self.0.encode(buf); - self.1.encode(buf); - } -} - impl RLPEncode for ethereum_types::H264 { fn encode(&self, buf: &mut dyn BufMut) { self.as_bytes().encode(buf) diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index 207545e47..01e228e51 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -1,5 +1,3 @@ -use crate::encode::RLPEncodeSlim; - use super::{ decode::{decode_rlp_item, get_item_with_prefix, RLPDecode}, encode::{encode_length, RLPEncode}, @@ -185,13 +183,6 @@ impl<'a> Encoder<'a> { self } - /// Stores a field to be encoded, but in slim format - /// https://github.com/ethereum/devp2p/blob/master/caps/snap.md#data-format - pub fn encode_slim_field(mut self, value: &T) -> Self { - ::encode(value, &mut self.temp_buf); - self - } - /// If `Some`, stores a field to be encoded, else does nothing. pub fn encode_optional_field(mut self, opt_value: &Option) -> Self { if let Some(value) = opt_value { diff --git a/crates/common/types/account.rs b/crates/common/types/account.rs index 8bae95374..cbc0a0cda 100644 --- a/crates/common/types/account.rs +++ b/crates/common/types/account.rs @@ -6,9 +6,9 @@ use ethereum_types::{H256, U256}; use sha3::{Digest as _, Keccak256}; use ethereum_rust_rlp::{ - constants::{RLP_EMPTY_LIST, RLP_NULL}, + constants::RLP_NULL, decode::RLPDecode, - encode::{RLPEncode, RLPEncodeSlim}, + encode::RLPEncode, error::RLPDecodeError, structs::{Decoder, Encoder}, }; @@ -100,17 +100,6 @@ impl RLPEncode for AccountInfo { } } -impl RLPEncodeSlim for AccountInfo { - fn encode(&self, buf: &mut dyn bytes::BufMut) { - // TODO: check if it's okay to use RLP_EMPTY_LIST - Encoder::new(buf) - .encode_field(&RLP_EMPTY_LIST) - .encode_field(&self.balance) - .encode_field(&self.nonce) - .finish(); - } -} - impl RLPDecode for AccountInfo { fn decode_unfinished(rlp: &[u8]) -> Result<(AccountInfo, &[u8]), RLPDecodeError> { let decoder = Decoder::new(rlp)?; @@ -137,27 +126,6 @@ impl RLPEncode for AccountState { } } -impl RLPEncodeSlim for AccountState { - fn encode(&self, buf: &mut dyn bytes::BufMut) { - let root = if self.storage_root == *EMPTY_TRIE_HASH { - Bytes::new() - } else { - Bytes::copy_from_slice(self.storage_root.as_bytes()) - }; - let code_hash = if self.code_hash == *EMPTY_KECCACK_HASH { - Bytes::new() - } else { - Bytes::copy_from_slice(self.code_hash.as_bytes()) - }; - Encoder::new(buf) - .encode_field(&self.nonce) - .encode_field(&self.balance) - .encode_field(&root) - .encode_field(&code_hash) - .finish(); - } -} - impl RLPDecode for AccountState { fn decode_unfinished(rlp: &[u8]) -> Result<(AccountState, &[u8]), RLPDecodeError> { let decoder = Decoder::new(rlp)?; diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 98082e055..9ed737ed2 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -1,6 +1,11 @@ -use bytes::BufMut; -use ethereum_rust_core::{types::AccountState, H256}; +use bytes::{BufMut, Bytes}; +use ethereum_rust_core::{ + types::{AccountState, EMPTY_KECCACK_HASH, EMPTY_TRIE_HASH}, + H256, U256, +}; use ethereum_rust_rlp::{ + decode::RLPDecode, + encode::RLPEncode, error::{RLPDecodeError, RLPEncodeError}, structs::{Decoder, Encoder}, }; @@ -8,6 +13,56 @@ use snap::raw::Decoder as SnappyDecoder; use super::{message::RLPxMessage, utils::snappy_encode}; +#[derive(Debug)] +pub struct AccountStateSlim { + pub nonce: u64, + pub balance: U256, + pub storage_root: Bytes, + pub code_hash: Bytes, +} + +impl From for AccountStateSlim { + fn from(value: AccountState) -> Self { + let storage_root = if value.storage_root == *EMPTY_TRIE_HASH { + Bytes::new() + } else { + Bytes::copy_from_slice(value.storage_root.as_bytes()) + }; + let code_hash = if value.code_hash == *EMPTY_KECCACK_HASH { + Bytes::new() + } else { + Bytes::copy_from_slice(value.code_hash.as_bytes()) + }; + Self { + nonce: value.nonce, + balance: value.balance, + storage_root, + code_hash, + } + } +} + +impl From for AccountState { + fn from(value: AccountStateSlim) -> Self { + let storage_root = if value.storage_root.is_empty() { + *EMPTY_TRIE_HASH + } else { + H256::from_slice(value.storage_root.as_ref()) + }; + let code_hash = if value.code_hash.is_empty() { + *EMPTY_KECCACK_HASH + } else { + H256::from_slice(value.code_hash.as_ref()) + }; + Self { + nonce: value.nonce, + balance: value.balance, + storage_root, + code_hash, + } + } +} + // https://github.com/ethereum/devp2p/blob/master/caps/snap.md#getaccountrange-0x00 #[derive(Debug)] pub(crate) struct GetAccountRange { @@ -82,7 +137,7 @@ pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, - pub accounts: Vec<(H256, AccountState)>, + pub accounts: Vec<(H256, AccountStateSlim)>, pub proof: Vec>, } @@ -91,7 +146,7 @@ impl RLPxMessage for AccountRange { let mut encoded_data = vec![]; Encoder::new(&mut encoded_data) .encode_field(&self.id) - .encode_slim_field(&self.accounts) + .encode_field(&self.accounts) .encode_field(&self.proof) .finish(); @@ -106,9 +161,8 @@ impl RLPxMessage for AccountRange { .decompress_vec(msg_data) .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; let decoder = Decoder::new(&decompressed_data)?; - let (id, decoder): (u64, _) = decoder.decode_field("request-id")?; - let (accounts, decoder): (Vec<(H256, AccountState)>, _) = - decoder.decode_field("accounts")?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (accounts, decoder) = decoder.decode_field("accounts")?; let (proof, decoder) = decoder.decode_field("proof")?; decoder.finish()?; @@ -119,3 +173,32 @@ impl RLPxMessage for AccountRange { }) } } +impl RLPEncode for AccountStateSlim { + fn encode(&self, buf: &mut dyn BufMut) { + Encoder::new(buf) + .encode_field(&self.nonce) + .encode_field(&self.balance) + .encode_field(&self.storage_root) + .encode_field(&self.code_hash) + .finish(); + } +} + +impl RLPDecode for AccountStateSlim { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let decoder = Decoder::new(rlp)?; + let (nonce, decoder) = decoder.decode_field("nonce")?; + let (balance, decoder) = decoder.decode_field("balance")?; + let (storage_root, decoder) = decoder.decode_field("storage_root")?; + let (code_hash, decoder) = decoder.decode_field("code_hash")?; + Ok(( + Self { + nonce, + balance, + storage_root, + code_hash, + }, + decoder.finish()?, + )) + } +} diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index aa47d10d4..44b5cc49f 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -18,7 +18,7 @@ pub fn process_account_range_request( start_found = true; } if start_found { - accounts.push((k, v)) + accounts.push((k, v.into())) } } let proof = store.get_account_range_proof(request.root_hash, request.starting_hash)?; From 3893b05f6619dffa7ec9b2475b6ecb01f8403cd7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 17:10:00 -0300 Subject: [PATCH 010/345] Fix logic --- crates/networking/p2p/snap.rs | 5 +++-- crates/storage/trie/trie_iter.rs | 4 +++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 44b5cc49f..307d18480 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -11,10 +11,11 @@ pub fn process_account_range_request( let mut iter = store.iter_accounts(request.root_hash); let mut start_found = false; while let Some((k, v)) = iter.next() { - if k == request.limit_hash { + dbg!(&k); + if k >= request.limit_hash { break; } - if k == request.starting_hash { + if k >= request.starting_hash { start_found = true; } if start_found { diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 07a4475a0..e60d548dd 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -48,7 +48,9 @@ impl Iterator for TrieIterator { impl TrieIterator { pub fn content(self) -> impl Iterator { self.filter_map(|n| match n { - Node::Branch(branch_node) => Some((branch_node.path, branch_node.value)), + Node::Branch(branch_node) => { + (!branch_node.path.is_empty()).then_some((branch_node.path, branch_node.value)) + } Node::Extension(_) => None, Node::Leaf(leaf_node) => Some((leaf_node.path, leaf_node.value)), }) From dbe110f28730f5addbd907ca7d5f3b2c16bcc905 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 17:10:37 -0300 Subject: [PATCH 011/345] Remove debug print --- crates/networking/p2p/snap.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 307d18480..4de78ba4a 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -11,7 +11,6 @@ pub fn process_account_range_request( let mut iter = store.iter_accounts(request.root_hash); let mut start_found = false; while let Some((k, v)) = iter.next() { - dbg!(&k); if k >= request.limit_hash { break; } From 8b8eacbef6d1ed888d8d4a9e9b84a61c64fe4c05 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 17:25:46 -0300 Subject: [PATCH 012/345] Limit response bytes --- crates/networking/p2p/snap.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 4de78ba4a..a3bd9a5a8 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,6 +1,8 @@ +use ethereum_rust_core::types::AccountState; +use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use crate::rlpx::snap::{AccountRange, GetAccountRange}; +use crate::rlpx::snap::{AccountRange, AccountStateSlim, GetAccountRange}; pub fn process_account_range_request( request: GetAccountRange, @@ -10,6 +12,7 @@ pub fn process_account_range_request( // Fetch account range let mut iter = store.iter_accounts(request.root_hash); let mut start_found = false; + let mut bytes_used = 0; while let Some((k, v)) = iter.next() { if k >= request.limit_hash { break; @@ -18,7 +21,12 @@ pub fn process_account_range_request( start_found = true; } if start_found { - accounts.push((k, v.into())) + let acc = AccountStateSlim::from(v); + bytes_used += bytes_per_entry(&acc); + accounts.push((k, acc)); + } + if bytes_used >= request.response_bytes { + break; } } let proof = store.get_account_range_proof(request.root_hash, request.starting_hash)?; From 4ee1790d81c5af07f76f262cc9b88b166c6d63b5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 17:29:34 -0300 Subject: [PATCH 013/345] Add test --- crates/networking/p2p/snap.rs | 2169 ++++++++++++++++++++++++++++++- crates/storage/store/storage.rs | 5 + 2 files changed, 2173 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index a3bd9a5a8..8b48663e0 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,4 +1,3 @@ -use ethereum_rust_core::types::AccountState; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; @@ -37,3 +36,2171 @@ pub fn process_account_range_request( proof, }) } + +// TODO: write response bytes directly here so we dont need to encode twice +fn bytes_per_entry(state: &AccountStateSlim) -> u64 { + state.encode_to_vec().len() as u64 + 32 +} + +#[cfg(test)] +mod tests { + use std::str::FromStr; + + use ethereum_rust_core::{types::AccountState, H256}; + use ethereum_rust_rlp::{decode::RLPDecode, encode::RLPEncode}; + use ethereum_rust_storage::EngineType; + + use crate::rlpx::snap::AccountStateSlim; + + use super::*; + + fn setup_initial_state() -> (Store, H256) { + // We cannot process the old blocks that hive uses for the devp2p snap tests + // So I took the state from a geth execution to run them locally + + let accounts: Vec<(&str, Vec)> = vec![ + ( + "0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6", + vec![ + 228_u8, 1, 128, 160, 223, 151, 249, 75, 196, 116, 113, 135, 6, 6, 246, 38, 251, + 122, 11, 66, 238, 210, 212, 95, 204, 132, 220, 18, 0, 206, 98, 247, 131, 29, + 169, 144, 128, + ], + ), + ( + "0x00748bacab20da9ae19dd26a33bd10bbf825e28b3de84fc8fe1d15a21645067f", + vec![196, 128, 1, 128, 128], + ), + ( + "0x00aa781aff39a8284ef43790e3a511b2caa50803613c5096bc782e8de08fa4c5", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x016d92531f4754834b0502de5b0342ceff21cde5bef386a83d2292f4445782c2", + vec![196, 128, 1, 128, 128], + ), + ( + "0x02547b56492bfe767f3d18be2aab96441c449cd945770ef7ef8555acc505b2e4", + vec![196, 128, 1, 128, 128], + ), + ( + "0x025f478d53bf78add6fa3708d9e061d59bfe14b21329b2a4cf1156d4f81b3d2d", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x0267c643f67b47cac9efacf6fcf0e4f4e1b273a727ded155db60eb9907939eb6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x0304d8eaccf0b942c468074250cbcb625ec5c4688b6b5d17d2a9bdd8dd565d5a", + vec![ + 228, 1, 128, 160, 224, 12, 73, 166, 88, 73, 208, 92, 191, 39, 164, 215, 120, + 138, 104, 188, 123, 96, 19, 174, 51, 65, 29, 64, 188, 137, 40, 47, 192, 100, + 243, 61, 128, + ], + ), + ( + "0x0463e52cda557221b0b66bd7285b043071df4c2ab146260f4e010970f3a0cccf", + vec![196, 1, 128, 128, 128], + ), + ( + "0x04d9aa4f67f8b24d70a0ffd757e82456d9184113106b7d9e8eb6c3e8a8df27ee", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x053df2c3b574026812b154a99b13b626220af85cd01bb1693b1d42591054bce6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x0579e46a5ed8a88504ac7d579b12eb346fbe4fd7e281bdd226b891f8abed4789", + vec![ + 228, 1, 128, 160, 61, 14, 43, 165, 55, 243, 89, 65, 6, 135, 9, 69, 15, 37, 254, + 228, 90, 175, 77, 198, 174, 46, 210, 42, 209, 46, 7, 67, 172, 124, 84, 167, + 128, + ], + ), + ( + "0x05f6de281d8c2b5d98e8e01cd529bd76416b248caf11e0552047c5f1d516aab6", + vec![196, 128, 1, 128, 128], + ), + ( + "0x07b49045c401bcc408f983d91a199c908cdf0d646049b5b83629a70b0117e295", + vec![ + 228, 1, 128, 160, 134, 154, 203, 146, 159, 89, 28, 84, 203, 133, 132, 42, 81, + 242, 150, 99, 94, 125, 137, 87, 152, 197, 71, 162, 147, 175, 228, 62, 123, 247, + 244, 23, 128, + ], + ), + ( + "0x0993fd5b750fe4414f93c7880b89744abb96f7af1171ed5f47026bdf01df1874", + vec![196, 128, 1, 128, 128], + ), + ( + "0x099d5081762b8b265e8ba4cd8e43f08be4715d903a0b1d96b3d9c4e811cbfb33", + vec![196, 128, 1, 128, 128], + ), + ( + "0x09d6e6745d272389182a510994e2b54d14b731fac96b9c9ef434bc1924315371", + vec![196, 128, 128, 128, 128], + ), + ( + "0x0a93a7231976ad485379a3b66c2d8983ba0b2ca87abaf0ca44836b2a06a2b102", + vec![196, 128, 1, 128, 128], + ), + ( + "0x0b564e4a0203cbcec8301709a7449e2e7371910778df64c89f48507390f2d129", + vec![196, 1, 128, 128, 128], + ), + ( + "0x0cd2a7c53c76f228ed3aa7a29644b1915fde9ec22e0433808bf5467d914e7c7a", + vec![ + 228, 1, 128, 160, 7, 84, 3, 90, 164, 7, 51, 129, 162, 17, 52, 43, 80, 125, 232, + 231, 117, 201, 124, 150, 16, 150, 230, 226, 39, 93, 240, 191, 203, 179, 160, + 28, 128, + ], + ), + ( + "0x0e0e4646090b881949ec9991e48dec768ccd1980896aefd0d51fd56fd5689790", + vec![ + 228, 1, 128, 160, 96, 252, 105, 16, 13, 142, 99, 38, 103, 200, 11, 148, 212, + 52, 0, 136, 35, 237, 117, 65, 107, 113, 203, 209, 18, 180, 208, 176, 47, 86, + 48, 39, 128, + ], + ), + ( + "0x0e27113c09de0a0cb0ff268c677aba17d39a3190fe15aec0ff7f54184955cba4", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x0e57ffa6cc6cbd96c1400150417dd9b30d958c58f63c36230a90a02b076f78b5", + vec![196, 128, 1, 128, 128], + ), + ( + "0x0f30822f90f33f1d1ba6d1521a00935630d2c81ab12fa03d4a0f4915033134f3", + vec![ + 228, 1, 128, 160, 128, 120, 243, 37, 157, 129, 153, 183, 202, 57, 213, 30, 53, + 213, 181, 141, 113, 255, 20, 134, 6, 115, 16, 96, 56, 109, 50, 60, 93, 25, 24, + 44, 128, + ], + ), + ( + "0x1017b10a7cc3732d729fe1f71ced25e5b7bc73dc62ca61309a8c7e5ac0af2f72", + vec![196, 1, 128, 128, 128], + ), + ( + "0x1098f06082dc467088ecedb143f9464ebb02f19dc10bd7491b03ba68d751ce45", + vec![196, 1, 128, 128, 128], + ), + ( + "0x11eb0304c1baa92e67239f6947cb93e485a7db05e2b477e1167a8960458fa8cc", + vec![196, 1, 128, 128, 128], + ), + ( + "0x12be3bf1f9b1dab5f908ca964115bee3bcff5371f84ede45bc60591b21117c51", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x12c1bb3dddf0f06f62d70ed5b7f7db7d89b591b3f23a838062631c4809c37196", + vec![196, 128, 1, 128, 128], + ), + ( + "0x12e394ad62e51261b4b95c431496e46a39055d7ada7dbf243f938b6d79054630", + vec![196, 1, 128, 128, 128], + ), + ( + "0x13cfc46f6bdb7a1c30448d41880d061c3b8d36c55a29f1c0c8d95a8e882b8c25", + vec![ + 228, 1, 128, 160, 148, 79, 9, 90, 251, 209, 56, 62, 93, 15, 145, 239, 2, 137, + 93, 57, 143, 79, 118, 253, 182, 216, 106, 223, 71, 101, 242, 91, 220, 48, 79, + 95, 128, + ], + ), + ( + "0x15293aec87177f6c88f58bc51274ba75f1331f5cb94f0c973b1deab8b3524dfe", + vec![196, 128, 1, 128, 128], + ), + ( + "0x170c927130fe8f1db3ae682c22b57f33f54eb987a7902ec251fe5dba358a2b25", + vec![196, 128, 1, 128, 128], + ), + ( + "0x17350c7adae7f08d7bbb8befcc97234462831638443cd6dfea186cbf5a08b7c7", + vec![ + 228, 1, 128, 160, 76, 231, 156, 217, 100, 86, 80, 240, 160, 14, 255, 168, 111, + 111, 234, 115, 60, 236, 234, 158, 162, 105, 100, 130, 143, 242, 92, 240, 87, + 123, 201, 116, 128, + ], + ), + ( + "0x174f1a19ff1d9ef72d0988653f31074cb59e2cf37cd9d2992c7b0dd3d77d84f9", + vec![196, 128, 1, 128, 128], + ), + ( + "0x17984cc4b4aac0492699d37662b53ec2acf8cbe540c968b817061e4ed27026d0", + vec![196, 128, 1, 128, 128], + ), + ( + "0x181abdd5e212171007e085fdc284a84d42d5bfc160960d881ccb6a10005ff089", + vec![196, 1, 128, 128, 128], + ), + ( + "0x188111c233bf6516bb9da8b5c4c31809a42e8604cd0158d933435cfd8e06e413", + vec![196, 1, 128, 128, 128], + ), + ( + "0x18f4256a59e1b2e01e96ac465e1d14a45d789ce49728f42082289fc25cf32b8d", + vec![196, 128, 1, 128, 128], + ), + ( + "0x1960414a11f8896c7fc4243aba7ed8179b0bc6979b7c25da7557b17f5dee7bf7", + vec![196, 1, 128, 128, 128], + ), + ( + "0x1a28912018f78f7e754df6b9fcec33bea25e5a232224db622e0c3343cf079eff", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x1bf7626cec5330a127e439e68e6ee1a1537e73b2de1aa6d6f7e06bc0f1e9d763", + vec![196, 128, 1, 128, 128], + ), + ( + "0x1c248f110218eaae2feb51bc82e9dcc2844bf93b88172c52afcb86383d262323", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x1c3f74249a4892081ba0634a819aec9ed25f34c7653f5719b9098487e65ab595", + vec![ + 228, 1, 128, 160, 175, 134, 126, 108, 186, 232, 16, 202, 169, 36, 184, 182, + 172, 61, 140, 8, 145, 131, 20, 145, 166, 144, 109, 208, 190, 122, 211, 36, 220, + 209, 83, 61, 128, + ], + ), + ( + "0x1d38ada74301c31f3fd7d92dd5ce52dc37ae633e82ac29c4ef18dfc141298e26", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x1d6ee979097e29141ad6b97ae19bb592420652b7000003c55eb52d5225c3307d", + vec![ + 228, 1, 128, 160, 247, 53, 145, 231, 145, 175, 76, 124, 95, 160, 57, 195, 61, + 217, 209, 105, 202, 177, 75, 29, 155, 12, 167, 139, 204, 78, 116, 13, 85, 59, + 26, 207, 128, + ], + ), + ( + "0x1dff76635b74ddba16bba3054cc568eed2571ea6becaabd0592b980463f157e2", + vec![196, 1, 128, 128, 128], + ), + ( + "0x1ee7e0292fba90d9733f619f976a2655c484adb30135ef0c5153b5a2f32169df", + vec![196, 1, 128, 128, 128], + ), + ( + "0x209b102e507b8dfc6acfe2cf55f4133b9209357af679a6d507e6ee87112bfe10", + vec![196, 1, 128, 128, 128], + ), + ( + "0x210ce6d692a21d75de3764b6c0356c63a51550ebec2c01f56c154c24b1cf8888", + vec![196, 1, 128, 128, 128], + ), + ( + "0x2116ab29b4cb8547af547fe472b7ce30713f234ed49cb1801ea6d3cf9c796d57", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x2290ea88cc63f09ab5e8c989a67e2e06613311801e39c84aae3badd8bb38409c", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x2369a492b6cddcc0218617a060b40df0e7dda26abe48ba4e4108c532d3f2b84f", + vec![196, 1, 128, 128, 128], + ), + ( + "0x2374954008440ca3d17b1472d34cc52a6493a94fb490d5fb427184d7d5fd1cbf", + vec![196, 1, 128, 128, 128], + ), + ( + "0x23ddaac09188c12e5d88009afa4a34041175c5531f45be53f1560a1cbfec4e8a", + vec![ + 228, 1, 128, 160, 71, 250, 72, 226, 93, 54, 105, 169, 187, 25, 12, 89, 147, + 143, 75, 228, 157, 226, 208, 131, 105, 110, 185, 57, 195, 180, 7, 46, 198, 126, + 67, 177, 128, + ], + ), + ( + "0x246cc8a2b79a30ec71390d829d0cb37cce1b953e89cb14deae4945526714a71c", + vec![196, 128, 1, 128, 128], + ), + ( + "0x255ec86eac03ba59f6dfcaa02128adbb22c561ae0c49e9e62e4fff363750626e", + vec![ + 228, 1, 128, 160, 102, 235, 22, 7, 27, 163, 121, 191, 12, 99, 47, 203, 82, 249, + 23, 90, 101, 107, 239, 98, 173, 240, 190, 245, 52, 154, 127, 90, 106, 173, 93, + 136, 128, + ], + ), + ( + "0x26ce7d83dfb0ab0e7f15c42aeb9e8c0c5dba538b07c8e64b35fb64a37267dd96", + vec![ + 228, 1, 128, 160, 36, 52, 191, 198, 67, 236, 54, 65, 22, 205, 113, 81, 154, 57, + 118, 98, 178, 12, 82, 209, 173, 207, 240, 184, 48, 232, 10, 115, 142, 25, 243, + 14, 128, + ], + ), + ( + "0x2705244734f69af78e16c74784e1dc921cb8b6a98fe76f577cc441c831e973bf", + vec![196, 1, 128, 128, 128], + ), + ( + "0x28f25652ec67d8df6a2e33730e5d0983443e3f759792a0128c06756e8eb6c37f", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x2a248c1755e977920284c8054fceeb20530dc07cd8bbe876f3ce02000818cc3a", + vec![196, 1, 128, 128, 128], + ), + ( + "0x2a39afbe88f572c23c90da2d059af3de125f1da5c3753c530dc5619a4857119f", + vec![ + 228, 1, 128, 160, 130, 137, 181, 88, 134, 95, 44, 161, 245, 76, 152, 181, 255, + 93, 249, 95, 7, 194, 78, 198, 5, 226, 71, 181, 140, 119, 152, 96, 93, 205, 121, + 79, 128, + ], + ), + ( + "0x2b8d12301a8af18405b3c826b6edcc60e8e034810f00716ca48bebb84c4ce7ab", + vec![196, 1, 128, 128, 128], + ), + ( + "0x2baa718b760c0cbd0ec40a3c6df7f2948b40ba096e6e4b116b636f0cca023bde", + vec![196, 128, 1, 128, 128], + ), + ( + "0x2e6fe1362b3e388184fd7bf08e99e74170b26361624ffd1c5f646da7067b58b6", + vec![ + 228, 128, 128, 128, 160, 142, 3, 136, 236, 246, 76, 250, 118, 179, 166, 175, + 21, 159, 119, 69, 21, 25, 167, 249, 187, 134, 46, 76, 206, 36, 23, 92, 121, 31, + 220, 176, 223, + ], + ), + ( + "0x2fe5767f605b7b821675b223a22e4e5055154f75e7f3041fdffaa02e4787fab8", + vec![196, 128, 1, 128, 128], + ), + ( + "0x303f57a0355c50bf1a0e1cf0fa8f9bdbc8d443b70f2ad93ac1c6b9c1d1fe29a2", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x30ce5b7591126d5464dfb4fc576a970b1368475ce097e244132b06d8cc8ccffe", + vec![196, 128, 1, 128, 128], + ), + ( + "0x315ccc15883d06b4e743f8252c999bf1ee994583ff6114d89c0f3ddee828302b", + vec![196, 1, 128, 128, 128], + ), + ( + "0x3197690074092fe51694bdb96aaab9ae94dac87f129785e498ab171a363d3b40", + vec![196, 128, 1, 128, 128], + ), + ( + "0x34a715e08b77afd68cde30b62e222542f3db90758370400c94d0563959a1d1a0", + vec![ + 228, 1, 128, 160, 79, 68, 99, 41, 181, 238, 61, 19, 212, 246, 181, 229, 242, + 16, 221, 194, 217, 15, 237, 186, 56, 75, 149, 14, 54, 161, 209, 154, 249, 92, + 92, 177, 128, + ], + ), + ( + "0x37310559ceaade42e45b3e3f05925aadca9e60aeeb9dd60d824875d9e9e71e26", + vec![ + 228, 1, 128, 160, 114, 200, 146, 33, 218, 237, 204, 221, 63, 187, 166, 108, 27, + 8, 27, 54, 52, 206, 137, 213, 160, 105, 190, 151, 255, 120, 50, 119, 143, 123, + 2, 58, 128, + ], + ), + ( + "0x37d65eaa92c6bc4c13a5ec45527f0c18ea8932588728769ec7aecfe6d9f32e42", + vec![ + 248, 68, 128, 42, 160, 172, 49, 98, 168, 185, 219, 180, 49, 139, 132, 33, 159, + 49, 64, 231, 169, 236, 53, 18, 98, 52, 18, 2, 151, 221, 225, 15, 81, 178, 95, + 106, 38, 160, 245, 122, 205, 64, 37, 152, 114, 96, 109, 118, 25, 126, 240, 82, + 243, 211, 85, 136, 218, 223, 145, 158, 225, 240, 227, 203, 155, 98, 211, 244, + 176, 44, + ], + ), + ( + "0x37ddfcbcb4b2498578f90e0fcfef9965dcde4d4dfabe2f2836d2257faa169947", + vec![ + 228, 1, 128, 160, 82, 214, 210, 145, 58, 228, 75, 202, 17, 181, 161, 22, 2, 29, + 185, 124, 145, 161, 62, 56, 94, 212, 139, 160, 102, 40, 231, 66, 1, 35, 29, + 186, 128, + ], + ), + ( + "0x37e51740ad994839549a56ef8606d71ace79adc5f55c988958d1c450eea5ac2d", + vec![196, 1, 128, 128, 128], + ), + ( + "0x38152bce526b7e1c2bedfc9d297250fcead02818be7806638564377af145103b", + vec![ + 228, 1, 128, 160, 108, 0, 224, 145, 218, 227, 212, 34, 111, 172, 214, 190, 128, + 44, 134, 93, 93, 176, 245, 36, 117, 77, 34, 102, 100, 6, 19, 139, 84, 250, 176, + 230, 128, + ], + ), + ( + "0x3848b7da914222540b71e398081d04e3849d2ee0d328168a3cc173a1cd4e783b", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x389093badcaa24c3a8cbb4461f262fba44c4f178a162664087924e85f3d55710", + vec![196, 1, 128, 128, 128], + ), + ( + "0x3897cb9b6f68765022f3c74f84a9f2833132858f661f4bc91ccd7a98f4e5b1ee", + vec![196, 1, 128, 128, 128], + ), + ( + "0x395b92f75f8e06b5378a84ba03379f025d785d8b626b2b6a1c84b718244b9a91", + vec![ + 228, 1, 128, 160, 84, 70, 184, 24, 244, 198, 105, 102, 156, 211, 49, 71, 38, + 255, 19, 76, 241, 140, 88, 169, 165, 54, 223, 19, 199, 0, 97, 7, 5, 168, 183, + 200, 128, + ], + ), + ( + "0x3be526914a7d688e00adca06a0c47c580cb7aa934115ca26006a1ed5455dd2ce", + vec![196, 128, 1, 128, 128], + ), + ( + "0x3e57e37bc3f588c244ffe4da1f48a360fa540b77c92f0c76919ec4ee22b63599", + vec![196, 128, 1, 128, 128], + ), + ( + "0x415ded122ff7b6fe5862f5c443ea0375e372862b9001c5fe527d276a3a420280", + vec![196, 1, 128, 128, 128], + ), + ( + "0x419809ad1512ed1ab3fb570f98ceb2f1d1b5dea39578583cd2b03e9378bbe418", + vec![196, 1, 128, 128, 128], + ), + ( + "0x4363d332a0d4df8582a84932729892387c623fe1ec42e2cfcbe85c183ed98e0e", + vec![ + 213, 130, 1, 146, 143, 192, 151, 206, 123, 201, 7, 21, 179, 73, 233, 122, 138, + 101, 46, 31, 128, 128, + ], + ), + ( + "0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099", + vec![ + 228, 1, 1, 160, 190, 61, 117, 161, 114, 155, 225, 87, 231, 156, 59, 119, 240, + 2, 6, 219, 77, 84, 227, 234, 20, 55, 90, 1, 84, 81, 200, 142, 192, 103, 199, + 144, 128, + ], + ), + ( + "0x4615e5f5df5b25349a00ad313c6cd0436b6c08ee5826e33a018661997f85ebaa", + vec![196, 1, 128, 128, 128], + ), + ( + "0x465311df0bf146d43750ed7d11b0451b5f6d5bfc69b8a216ef2f1c79c93cd848", + vec![196, 128, 1, 128, 128], + ), + ( + "0x47450e5beefbd5e3a3f80cbbac474bb3db98d5e609aa8d15485c3f0d733dea3a", + vec![ + 228, 1, 128, 160, 84, 66, 224, 39, 157, 63, 17, 73, 222, 76, 232, 217, 226, + 211, 240, 29, 24, 84, 117, 80, 56, 172, 26, 15, 174, 92, 72, 116, 155, 247, 31, + 32, 128, + ], + ), + ( + "0x482814ea8f103c39dcf6ba7e75df37145bde813964d82e81e5d7e3747b95303d", + vec![196, 128, 1, 128, 128], + ), + ( + "0x4845aac9f26fcd628b39b83d1ccb5c554450b9666b66f83aa93a1523f4db0ab6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x48e291f8a256ab15da8401c8cae555d5417a992dff3848926fa5b71655740059", + vec![ + 228, 1, 128, 160, 162, 231, 8, 75, 169, 206, 193, 121, 81, 156, 126, 137, 80, + 198, 106, 211, 203, 168, 88, 106, 96, 207, 249, 244, 214, 12, 24, 141, 214, 33, + 82, 42, 128, + ], + ), + ( + "0x4973f6aa8cf5b1190fc95379aa01cff99570ee6b670725880217237fb49e4b24", + vec![ + 228, 1, 128, 160, 174, 46, 127, 28, 147, 60, 108, 168, 76, 232, 190, 129, 30, + 244, 17, 222, 231, 115, 251, 105, 80, 128, 86, 215, 36, 72, 4, 142, 161, 219, + 92, 71, 128, + ], + ), + ( + "0x4b238e08b80378d0815e109f350a08e5d41ec4094df2cfce7bc8b9e3115bda70", + vec![ + 228, 1, 128, 160, 17, 245, 211, 153, 202, 143, 183, 169, 175, 90, 212, 129, + 190, 96, 207, 97, 212, 84, 147, 205, 32, 32, 108, 157, 10, 35, 124, 231, 215, + 87, 30, 95, 128, + ], + ), + ( + "0x4b9f335ce0bdffdd77fdb9830961c5bc7090ae94703d0392d3f0ff10e6a4fbab", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x4bd8ef9873a5e85d4805dbcb0dbf6810e558ea175167549ef80545a9cafbb0e1", + vec![ + 228, 1, 128, 160, 161, 73, 19, 213, 72, 172, 29, 63, 153, 98, 162, 26, 86, 159, + 229, 47, 20, 54, 182, 210, 245, 234, 78, 54, 222, 19, 234, 133, 94, 222, 84, + 224, 128, + ], + ), + ( + "0x4c2765139cace1d217e238cc7ccfbb751ef200e0eae7ec244e77f37e92dfaee5", + vec![196, 1, 128, 128, 128], + ), + ( + "0x4c310e1f5d2f2e03562c4a5c473ae044b9ee19411f07097ced41e85bd99c3364", + vec![196, 128, 1, 128, 128], + ), + ( + "0x4ccd31891378d2025ef58980481608f11f5b35a988e877652e7cbb0a6127287c", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x4ceaf2371fcfb54a4d8bc1c804d90b06b3c32c9f17112b57c29b30a25cf8ca12", + vec![196, 128, 1, 128, 128], + ), + ( + "0x4d67d989fdb264fa4b2524d306f7b3f70ddce0b723411581d1740407da325462", + vec![196, 128, 1, 128, 128], + ), + ( + "0x4d79fea6c7fef10cb0b5a8b3d85b66836a131bec0b04d891864e6fdb9794af75", + vec![196, 1, 128, 128, 128], + ), + ( + "0x4e0ab2902f57bf2a250c0f87f088acc325d55f2320f2e33abd8e50ba273c9244", + vec![ + 228, 1, 128, 160, 193, 104, 96, 69, 40, 138, 89, 82, 173, 87, 222, 14, 151, 27, + 210, 80, 7, 114, 60, 159, 116, 159, 73, 243, 145, 231, 21, 194, 123, 245, 38, + 200, 128, + ], + ), + ( + "0x4e258aa445a0e2a8704cbc57bbe32b859a502cd6f99190162236300fabd86c4a", + vec![196, 1, 128, 128, 128], + ), + ( + "0x4e5bab4ebd077c3bbd8239995455989ea2e95427ddeed47d0618d9773332bb05", + vec![196, 128, 1, 128, 128], + ), + ( + "0x4f458f480644b18c0e8207f405b82da7f75c7b3b5a34fe6771a0ecf644677f33", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x4fbc5fc8df4f0a578c3be3549f1cb3ef135cbcdf75f620c7a1d412462e9b3b94", + vec![196, 1, 128, 128, 128], + ), + ( + "0x4fd7c8d583447b937576211163a542d945ac8c0a6e22d0c42ac54e2cbaff9281", + vec![196, 128, 1, 128, 128], + ), + ( + "0x50d83ef5194d06752cd5594b57e809b135f24eedd124a51137feaaf049bc2efd", + vec![ + 228, 1, 128, 160, 81, 184, 41, 240, 242, 195, 222, 156, 251, 217, 78, 71, 130, + 138, 137, 148, 12, 50, 154, 73, 205, 89, 84, 12, 163, 198, 215, 81, 168, 210, + 20, 214, 128, + ], + ), + ( + "0x5162f18d40405c59ef279ad71d87fbec2bbfedc57139d56986fbf47daf8bcbf2", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x517bd5fbe28e4368b0b9fcba13d5e81fb51babdf4ed63bd83885235ee67a8fa0", + vec![ + 228, 1, 128, 160, 116, 237, 120, 235, 22, 1, 109, 127, 243, 161, 115, 171, 27, + 188, 238, 157, 170, 142, 53, 138, 157, 108, 155, 229, 232, 75, 166, 244, 163, + 76, 249, 106, 128, + ], + ), + ( + "0x519abb269c3c5710f1979ca84192e020ba5c838bdd267b2d07436a187f171232", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x5264e880ecf7b80afda6cc2a151bac470601ff8e376af91aaf913a36a30c4009", + vec![196, 128, 1, 128, 128], + ), + ( + "0x52d034ca6ebd21c7ba62a2ad3b6359aa4a1cdc88bdaa64bb2271d898777293ab", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x5380c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312a", + vec![ + 205, 128, 137, 12, 167, 152, 153, 113, 244, 250, 99, 97, 128, 128, + ], + ), + ( + "0x54c12444ede3e2567dd7f4d9a06d4db8c6ab800d5b3863f8ff22a0db6d09bf24", + vec![196, 128, 1, 128, 128], + ), + ( + "0x55cab9586acb40e66f66147ff3a059cfcbbad785dddd5c0cc31cb43edf98a5d5", + vec![196, 1, 128, 128, 128], + ), + ( + "0x55d0609468d8d4147a942e88cfc5f667daff850788d821889fbb03298924767c", + vec![196, 128, 1, 128, 128], + ), + ( + "0x5602444769b5fd1ddfca48e3c38f2ecad326fe2433f22b90f6566a38496bd426", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x5677600b2af87d21fdab2ac8ed39bd1be2f790c04600de0400c1989040d9879c", + vec![196, 128, 1, 128, 128], + ), + ( + "0x570210539713235b442bbbad50c58bee81b70efd2dad78f99e41a6c462faeb43", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x580aa878e2f92d113a12c0a3ce3c21972b03dbe80786858d49a72097e2c491a3", + vec![ + 228, 1, 128, 160, 71, 27, 248, 152, 138, 208, 215, 96, 45, 107, 213, 73, 60, 8, + 115, 48, 150, 193, 22, 172, 120, 139, 118, 242, 42, 104, 43, 196, 85, 142, 58, + 167, 128, + ], + ), + ( + "0x58e416a0dd96454bd2b1fe3138c3642f5dee52e011305c5c3416d97bc8ba5cf0", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x59312f89c13e9e24c1cb8b103aa39a9b2800348d97a92c2c9e2a78fa02b70025", + vec![196, 1, 128, 128, 128], + ), + ( + "0x5a356862c79afffd6a01af752d950e11490146e4d86dfb8ab1531e9aef4945a1", + vec![ + 228, 1, 128, 160, 58, 41, 133, 198, 173, 166, 126, 86, 4, 185, 159, 162, 252, + 26, 48, 42, 189, 13, 194, 65, 238, 127, 20, 196, 40, 250, 103, 212, 118, 134, + 139, 182, 128, + ], + ), + ( + "0x5a4a3feecfc77b402e938e28df0c4cbb874771cb3c5a92524f303cffb82a2862", + vec![196, 128, 1, 128, 128], + ), + ( + "0x5aa3b4a2ebdd402721c3953b724f4fe90900250bb4ef89ce417ec440da318cd6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x5b90bb05df9514b2d8e3a8feb3d6c8c22526b02398f289b42111426edc4fe6cf", + vec![ + 228, 1, 128, 160, 40, 122, 204, 120, 105, 66, 31, 185, 244, 154, 53, 73, 185, + 2, 251, 1, 183, 172, 204, 3, 34, 67, 189, 126, 26, 204, 216, 150, 93, 149, 217, + 21, 128, + ], + ), + ( + "0x5c1d92594d6377fe6423257781b382f94dffcde4fadbf571aa328f6eb18f8fcd", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x5c20f6ee05edbb60beeab752d87412b2f6e12c8feefa2079e6bd989f814ed4da", + vec![196, 1, 128, 128, 128], + ), + ( + "0x5d97d758e8800d37b6d452a1b1812d0afedba11f3411a17a8d51ee13a38d73f0", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x5e88e876a3af177e6daafe173b67f186a53f1771a663747f26b278c5acb4c219", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x5ec55391e89ac4c3cf9e61801cd13609e8757ab6ed08687237b789f666ea781b", + vec![ + 228, 1, 128, 160, 199, 191, 43, 52, 41, 64, 101, 175, 185, 162, 193, 95, 144, + 108, 186, 31, 122, 26, 159, 13, 163, 78, 169, 196, 102, 3, 181, 44, 174, 144, + 40, 236, 128, + ], + ), + ( + "0x5fc13d7452287b5a8e3c3be9e4f9057b5c2dd82aeaff4ed892c96fc944ec31e7", + vec![196, 128, 1, 128, 128], + ), + ( + "0x5fcd9b6fce3394ad1d44733056b3e5f6306240974a16f9de8e96ebdd14ae06b1", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x600a7a5f41a67f6f759dcb664198f1c5d9b657fb51a870ce9e234e686dff008e", + vec![ + 228, 1, 128, 160, 158, 218, 142, 182, 202, 3, 215, 196, 175, 228, 114, 121, + 172, 201, 10, 69, 209, 178, 202, 106, 17, 175, 217, 82, 6, 248, 134, 141, 32, + 82, 13, 6, 128, + ], + ), + ( + "0x60535eeb3ffb721c1688b879368c61a54e13f8881bdef6bd4a17b8b92e050e06", + vec![ + 228, 1, 128, 160, 251, 121, 2, 30, 127, 165, 75, 155, 210, 223, 100, 246, 219, + 87, 137, 125, 82, 174, 133, 247, 193, 149, 175, 81, 141, 228, 130, 0, 161, 50, + 94, 44, 128, + ], + ), + ( + "0x606059a65065e5f41347f38754e6ddb99b2d709fbff259343d399a4f9832b48f", + vec![ + 228, 1, 128, 160, 191, 186, 27, 194, 172, 66, 101, 95, 90, 151, 69, 11, 230, + 43, 148, 48, 130, 34, 50, 241, 206, 73, 152, 234, 245, 35, 155, 12, 36, 59, 43, + 132, 128, + ], + ), + ( + "0x61088707d2910974000e63c2d1a376f4480ba19dde19c4e6a757aeb3d62d5439", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x6188c4510d25576535a642b15b1dbdb8922fe572b099f504390f923c19799777", + vec![196, 128, 1, 128, 128], + ), + ( + "0x6225e8f52719d564e8217b5f5260b1d1aac2bcb959e54bc60c5f479116c321b8", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x625e5c85d5f4b6385574b572709d0f704b097527a251b7c658c0c4441aef2af6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x64bfba8a4688bdee41c4b998e101567b8b56fea53d30ab85393f2d5b70c5da90", + vec![196, 1, 128, 128, 128], + ), + ( + "0x64d0de66ea29cbcf7f237dae1c5f883fa6ff0ba52b90f696bb0348224dbc82ce", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x65cf42efacdee07ed87a1c2de0752a4e3b959f33f9f9f8c77424ba759e01fcf2", + vec![196, 1, 128, 128, 128], + ), + ( + "0x65e6b6521e4f1f97e80710581f42063392c9b33e0aeea4081a102a32238992ea", + vec![ + 228, 1, 128, 160, 17, 212, 238, 199, 223, 82, 205, 84, 231, 70, 144, 164, 135, + 136, 78, 86, 55, 25, 118, 194, 184, 196, 159, 252, 76, 143, 52, 131, 17, 102, + 191, 78, 128, + ], + ), + ( + "0x662d147a16d7c23a2ba6d3940133e65044a90985e26207501bfca9ae47a2468c", + vec![196, 128, 1, 128, 128], + ), + ( + "0x6641e3ed1f264cf275b53bb7012dabecf4c1fca700e3db989e314c24cc167074", + vec![ + 228, 1, 128, 160, 15, 216, 233, 155, 27, 74, 180, 235, 140, 108, 34, 24, 34, + 26, 230, 151, 140, 198, 116, 51, 52, 30, 216, 161, 173, 97, 133, 211, 79, 168, + 44, 97, 128, + ], + ), + ( + "0x67cc0bf5341efbb7c8e1bdbf83d812b72170e6edec0263eeebdea6f107bbef0d", + vec![ + 228, 1, 128, 160, 162, 14, 106, 33, 36, 74, 248, 255, 204, 213, 68, 34, 151, + 173, 155, 122, 118, 172, 114, 215, 216, 172, 158, 22, 241, 47, 204, 80, 233, + 11, 115, 78, 128, + ], + ), + ( + "0x68fc814efedf52ac8032da358ddcb61eab4138cb56b536884b86e229c995689c", + vec![ + 228, 1, 128, 160, 109, 43, 138, 7, 76, 120, 160, 229, 168, 9, 93, 122, 1, 13, + 73, 97, 198, 57, 197, 65, 207, 86, 251, 183, 4, 148, 128, 204, 143, 25, 151, + 101, 128, + ], + ), + ( + "0x6a2c8498657ae4f0f7b1a02492c554f7f8a077e454550727890188f7423ba014", + vec![ + 228, 1, 128, 160, 86, 34, 128, 27, 16, 17, 222, 132, 3, 228, 67, 8, 187, 248, + 154, 88, 9, 183, 173, 101, 134, 38, 140, 215, 33, 100, 82, 53, 135, 249, 176, + 228, 128, + ], + ), + ( + "0x6a5e43139d88da6cfba857e458ae0b5359c3fde36e362b6e5f782a90ce351f14", + vec![196, 1, 128, 128, 128], + ), + ( + "0x6ad3ba011e031431dc057c808b85346d58001b85b32a4b5c90ccccea0f82e170", + vec![ + 228, 1, 128, 160, 20, 249, 244, 185, 68, 92, 117, 71, 213, 164, 103, 26, 56, + 176, 177, 43, 188, 14, 113, 152, 195, 178, 147, 75, 130, 182, 149, 200, 99, 13, + 73, 114, 128, + ], + ), + ( + "0x6bd9fb206b22c76b4f9630248940855b842c684db89adff0eb9371846ea625a9", + vec![196, 1, 128, 128, 128], + ), + ( + "0x6c05d8abc81143ce7c7568c98aadfe6561635c049c07b2b4bce3019cef328cb9", + vec![196, 128, 1, 128, 128], + ), + ( + "0x6c37093a34016ae687da7aabb18e42009b71edff70a94733c904aea51a4853c1", + vec![196, 1, 128, 128, 128], + ), + ( + "0x6d1da4cf1127d654ed731a93105f481b315ecfc2f62b1ccb5f6d2717d6a40f9b", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x6d4162ce16817e46fa2ddc5e70cee790b80abc3d6f7778cfbaed327c5d2af36c", + vec![196, 1, 128, 128, 128], + ), + ( + "0x6dbe5551f50400859d14228606bf221beff07238bfa3866454304abb572f9512", + vec![196, 128, 1, 128, 128], + ), + ( + "0x6dc09fdec00aa9a30dd8db984406a33e3ca15e35222a74773071207a5e56d2c2", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x6f358b4e903d31fdd5c05cddaa174296bb30b6b2f72f1ff6410e6c1069198989", + vec![196, 128, 1, 128, 128], + ), + ( + "0x7026c939a9158beedff127a64f07a98b328c3d1770690437afdb21c34560fc57", + vec![196, 128, 1, 128, 128], + ), + ( + "0x70aae390a762a4347a4d167a2431874554edf1d77579213e55fea3ec39a1257c", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x71dee9adfef0940a36336903bd6830964865180b98c0506f9bf7ba8f2740fbf9", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x720f25b62fc39426f70eb219c9dd481c1621821c8c0fa5367a1df6e59e3edf59", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x728325587fa336e318b54298e1701d246c4f90d6094eb95635d8a47f080f4603", + vec![196, 128, 1, 128, 128], + ), + ( + "0x729953a43ed6c913df957172680a17e5735143ad767bda8f58ac84ec62fbec5e", + vec![196, 1, 128, 128, 128], + ), + ( + "0x72d91596112f9d7e61d09ffa7575f3587ad9636172ae09641882761cc369ecc0", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x72e962dfe7e2828809f5906996dedeba50950140555b193fceb94f12fd6f0a22", + vec![196, 1, 128, 128, 128], + ), + ( + "0x734ee4981754a3f1403c4e8887d35addfb31717d93de3e00ede78368c230861e", + vec![ + 228, 1, 128, 160, 44, 242, 146, 193, 227, 130, 189, 208, 231, 46, 18, 103, 1, + 215, 176, 36, 132, 230, 226, 114, 244, 192, 216, 20, 245, 166, 250, 226, 51, + 252, 121, 53, 128, + ], + ), + ( + "0x73cd1b7cd355f3f77c570a01100a616757408bb7abb78fe9ee1262b99688fcc4", + vec![196, 1, 128, 128, 128], + ), + ( + "0x74614a0c4ba7d7c70b162dad186b6cc77984ab4070534ad9757e04a5b776dcc8", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x7583557e4e3918c95965fb610dc1424976c0eee606151b6dfc13640e69e5cb15", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x75d231f57a1a9751f58769d5691f4807ab31ac0e802b1a1f6bfc77f5dff0adbf", + vec![ + 228, 1, 128, 160, 205, 49, 237, 93, 93, 167, 153, 144, 175, 237, 13, 153, 60, + 183, 37, 196, 227, 77, 217, 117, 68, 176, 52, 102, 237, 52, 33, 46, 66, 194, + 141, 104, 128, + ], + ), + ( + "0x78948842ff476b87544c189ce744d4d924ffd0907107a0dbaa4b71d0514f2225", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x792cc9f20a61c16646d5b6136693e7789549adb7d8e35503d0004130ea6528b0", + vec![ + 228, 1, 128, 160, 154, 74, 51, 249, 120, 216, 78, 10, 206, 179, 172, 54, 112, + 194, 226, 223, 108, 138, 226, 124, 24, 154, 150, 237, 0, 184, 6, 209, 14, 215, + 180, 238, 128, + ], + ), + ( + "0x7963685967117ffb6fd019663dc9e782ebb1234a38501bffc2eb5380f8dc303b", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x79afb7a5ffe6ccd537f9adff8287b78f75c37d97ea8a4dd504a08bc09926c3fa", + vec![196, 128, 1, 128, 128], + ), + ( + "0x7a08bb8417e6b18da3ba926568f1022c15553b2b0f1a32f2fd9e5a605469e54f", + vec![ + 228, 1, 128, 160, 56, 91, 132, 210, 112, 89, 163, 199, 142, 126, 166, 58, 105, + 30, 235, 156, 83, 118, 247, 122, 241, 19, 54, 118, 47, 140, 24, 136, 47, 247, + 71, 26, 128, + ], + ), + ( + "0x7a2464bc24d90557940e93a3b73308ea354ed7d988be720c545974a17959f93f", + vec![196, 1, 128, 128, 128], + ), + ( + "0x7a3870cc1ed4fc29e9ab4dd3218dbb239dd32c9bf05bff03e325b7ba68486c47", + vec![196, 1, 128, 128, 128], + ), + ( + "0x7bac5af423cb5e417fa6c103c7cb9777e80660ce3735ca830c238b0d41610186", + vec![196, 128, 1, 128, 128], + ), + ( + "0x7bff1b6b56891e66584ced453d09450c2fed9453b1644e8509bef9f9dd081bbb", + vec![196, 1, 128, 128, 128], + ), + ( + "0x7c1edabb98857d64572f03c64ac803e4a14b1698fccffffd51675d99ee3ba217", + vec![ + 228, 1, 128, 160, 97, 23, 109, 188, 5, 168, 83, 125, 141, 232, 95, 130, 160, + 59, 142, 16, 73, 206, 167, 173, 10, 159, 14, 91, 96, 238, 21, 252, 166, 254, + 13, 66, 128, + ], + ), + ( + "0x7c3e44534b1398abc786e4591364c329e976dbde3b3ed3a4d55589de84bcb9a6", + vec![196, 128, 1, 128, 128], + ), + ( + "0x7c463797c90e9ba42b45ae061ffaa6bbd0dad48bb4998f761e81859f2a904a49", + vec![196, 1, 128, 128, 128], + ), + ( + "0x7c48e400de1f24b4de94c59068fcd91a028576d13a22f900a7fcbd8f4845bcf4", + vec![196, 1, 128, 128, 128], + ), + ( + "0x7c608293e741d1eb5ae6916c249a87b6540cf0c2369e96d293b1a7b5b9bd8b31", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x7e1ef9f8d2fa6d4f8e6717c3dcccff352ea9b8b46b57f6106cdbeed109441799", + vec![196, 128, 1, 128, 128], + ), + ( + "0x7e839d9fd8a767e90a8b2f48a571f111dd2451bc5910cf2bf3ae79963e47e34d", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x7f9726a7b2f5f3a501b2d7b18ec726f25f22c86348fae0f459d882ec5fd7d0c7", + vec![196, 128, 1, 128, 128], + ), + ( + "0x80a2c1f38f8e2721079a0de39f187adedcb81b2ab5ae718ec1b8d64e4aa6930e", + vec![ + 228, 1, 128, 160, 45, 168, 110, 179, 212, 255, 221, 137, 81, 112, 188, 126, + 240, 43, 105, 161, 22, 254, 33, 172, 44, 228, 90, 62, 216, 224, 187, 138, 241, + 124, 249, 43, 128, + ], + ), + ( + "0x80cd4a7b601d4ba0cb09e527a246c2b5dd25b6dbf862ac4e87c6b189bfce82d7", + vec![196, 1, 128, 128, 128], + ), + ( + "0x81c0c51e15c9679ef12d02729c09db84220ba007efe7ced37a57132f6f0e83c9", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x84c7ee50e102d0abf5750e781c1635d60346f20ab0d5e5f9830db1a592c658ff", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x8510660ad5e3d35a30d4fb7c2615c040f9f698faae2ac48022e366deaeecbe77", + vec![ + 228, 1, 128, 160, 39, 233, 182, 165, 76, 240, 251, 24, 132, 153, 197, 8, 189, + 150, 212, 80, 148, 108, 214, 186, 28, 247, 108, 245, 52, 59, 92, 116, 69, 15, + 102, 144, 128, + ], + ), + ( + "0x8678559b30b321b0f0420a4a3e8cecfde90c6e56766b78c1723062c93c1f041f", + vec![196, 1, 128, 128, 128], + ), + ( + "0x867bc89cf8d5b39f1712fbc77414bbd93012af454c226dcee0fb34ccc0017498", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x86a73e3c668eb065ecac3402c6dc912e8eb886788ea147c770f119dcd30780c6", + vec![ + 228, 1, 128, 160, 165, 169, 28, 249, 232, 21, 251, 85, 223, 20, 179, 238, 140, + 19, 37, 169, 136, 203, 59, 109, 211, 71, 150, 201, 1, 56, 92, 60, 194, 153, 32, + 115, 128, + ], + ), + ( + "0x86d03d0f6bed220d046a4712ec4f451583b276df1aed33f96495d22569dc3485", + vec![ + 228, 1, 128, 160, 226, 161, 100, 226, 195, 12, 243, 3, 145, 200, 143, 243, 42, + 14, 32, 33, 148, 176, 143, 42, 97, 169, 205, 41, 39, 234, 94, 214, 223, 191, + 16, 86, 128, + ], + ), + ( + "0x873429def7829ff8227e4ef554591291907892fc8f3a1a0667dada3dc2a3eb84", + vec![ + 228, 1, 128, 160, 84, 171, 205, 188, 139, 4, 188, 155, 112, 233, 189, 70, 203, + 157, 185, 184, 235, 8, 207, 212, 173, 219, 164, 201, 65, 218, 204, 52, 221, 40, + 100, 142, 128, + ], + ), + ( + "0x878040f46b1b4a065e6b82abd35421eb69eededc0c9598b82e3587ae47c8a651", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x87e33f70e1dd3c6ff68e3b71757d697fbeb20daae7a3cc8a7b1b3aa894592c50", + vec![196, 128, 1, 128, 128], + ), + ( + "0x88a5635dabc83e4e021167be484b62cbed0ecdaa9ac282dab2cd9405e97ed602", + vec![ + 228, 1, 128, 160, 137, 189, 232, 157, 247, 242, 216, 51, 68, 165, 3, 148, 75, + 179, 71, 184, 71, 242, 8, 223, 131, 114, 40, 187, 44, 223, 214, 195, 34, 140, + 163, 223, 128, + ], + ), + ( + "0x88bf4121c2d189670cb4d0a16e68bdf06246034fd0a59d0d46fb5cec0209831e", + vec![ + 228, 1, 128, 160, 89, 115, 155, 163, 177, 86, 235, 120, 248, 187, 177, 75, 191, + 61, 172, 222, 191, 222, 149, 20, 15, 88, 109, 182, 111, 114, 227, 17, 123, 148, + 187, 103, 128, + ], + ), + ( + "0x8989651e80c20af78b37fdb693d74ecafc9239426ff1315e1fb7b674dcdbdb75", + vec![196, 128, 1, 128, 128], + ), + ( + "0x8a8266874b43f78d4097f27b2842132faed7e7e430469eec7354541eb97c3ea0", + vec![196, 1, 128, 128, 128], + ), + ( + "0x8b76305d3f00d33f77bd41496b4144fd3d113a2ec032983bd5830a8b73f61cf0", + vec![196, 128, 1, 128, 128], + ), + ( + "0x8c7bfaa19ea367dec5272872114c46802724a27d9b67ea3eed85431df664664e", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x8e11480987056c309d7064ebbd887f086d815353cdbaadb796891ed25f8dcf61", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0x8ee17a1ec4bae15d8650323b996c55d5fa11a14ceec17ff1d77d725183904914", + vec![196, 128, 1, 128, 128], + ), + ( + "0x903f24b3d3d45bc50c082b2e71c7339c7060f633f868db2065ef611885abe37e", + vec![196, 1, 128, 128, 128], + ), + ( + "0x910fb8b22867289cb57531ad39070ef8dbdbbe7aee941886a0e9f572b63ae9ee", + vec![ + 228, 1, 128, 160, 115, 191, 252, 104, 169, 71, 250, 25, 183, 190, 205, 69, 102, + 29, 34, 200, 112, 250, 200, 219, 242, 178, 87, 3, 225, 189, 171, 83, 103, 242, + 149, 67, 128, + ], + ), + ( + "0x913e2a02a28d71d595d7216a12311f6921a4caf40aeabf0f28edf937f1df72b4", + vec![196, 128, 1, 128, 128], + ), + ( + "0x92b13a73440c6421da22e848d23f9af80610085ab05662437d850c97a012d8d3", + vec![196, 128, 1, 128, 128], + ), + ( + "0x92d0f0954f4ec68bd32163a2bd7bc69f933c7cdbfc6f3d2457e065f841666b1c", + vec![196, 1, 128, 128, 128], + ), + ( + "0x93843d6fa1fe5709a3035573f61cc06832f0377544d16d3a0725e78a0fa0267c", + vec![196, 1, 128, 128, 128], + ), + ( + "0x943f42ad91e8019f75695946d491bb95729f0dfc5dbbb953a7239ac73f208943", + vec![ + 228, 1, 128, 160, 169, 88, 1, 9, 190, 47, 125, 53, 181, 54, 0, 80, 194, 206, + 215, 78, 93, 77, 234, 47, 130, 212, 110, 141, 38, 110, 216, 145, 87, 99, 96, 4, + 128, + ], + ), + ( + "0x946bfb429d90f1b39bb47ada75376a8d90a5778068027d4b8b8514ac13f53eca", + vec![196, 1, 128, 128, 128], + ), + ( + "0x961508ac3c93b30ee9a5a34a862c9fe1659e570546ac6c2e35da20f6d2bb5393", + vec![ + 228, 1, 128, 160, 217, 26, 207, 48, 89, 52, 166, 12, 150, 10, 147, 251, 0, 249, + 39, 236, 121, 48, 139, 138, 145, 157, 36, 73, 250, 237, 231, 34, 194, 50, 76, + 179, 128, + ], + ), + ( + "0x96c43ef9dce3410b78df97be69e7ccef8ed40d6e5bfe6582ea4cd7d577aa4569", + vec![ + 228, 1, 128, 160, 90, 130, 175, 241, 38, 255, 235, 255, 118, 0, 43, 30, 77, + 224, 60, 64, 186, 73, 75, 129, 203, 63, 188, 82, 143, 35, 228, 190, 53, 169, + 175, 230, 128, + ], + ), + ( + "0x96d7104053877823b058fd9248e0bba2a540328e52ffad9bb18805e89ff579dc", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x974a4800ec4c0e998f581c6ee8c3972530989e97a179c6b2d40b8710c036e7b1", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x97b25febb46f44607c87a3498088c605086df207c7ddcd8ee718836a516a9153", + vec![196, 1, 128, 128, 128], + ), + ( + "0x97f72ff641eb40ee1f1163544931635acb7550a0d44bfb9f4cc3aeae829b6d7d", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x98bb9ba48fda7bb8091271ab0e53d7e0022fb1f1fa8fa00814e193c7d4b91eb3", + vec![196, 128, 1, 128, 128], + ), + ( + "0x9966a8b4cd856b175855258fa7e412ffef06d9e92b519050fa7ac06d8952ac84", + vec![196, 128, 1, 128, 128], + ), + ( + "0x99ce1680f73f2adfa8e6bed135baa3360e3d17f185521918f9341fc236526321", + vec![196, 1, 128, 128, 128], + ), + ( + "0x99dba7e9230d5151cc37ff592fa1592f27c7c81d203760dfaf62ddc9f3a6b8fd", + vec![196, 128, 1, 128, 128], + ), + ( + "0x99e56541f21039c9b7c63655333841a3415de0d27b79d18ade9ec7ecde7a1139", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x9a1896e612ca43ecb7601990af0c3bc135b9012c50d132769dfb75d0038cc3be", + vec![196, 128, 1, 128, 128], + ), + ( + "0x9d42947ac5e61285567f65d4b400d90343dbd3192534c4c1f9d941c04f48f17c", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x9de451c4f48bdb56c6df198ff8e1f5e349a84a4dc11de924707718e6ac897aa6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0x9fe8b6e43098a4df56e206d479c06480801485dfd8ec3da4ccc3cebf5fba89a1", + vec![196, 128, 1, 128, 128], + ), + ( + "0x9feaf0bd45df0fbf327c964c243b2fbc2f0a3cb48fedfeea1ae87ac1e66bc02f", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xa02abeb418f26179beafd96457bda8c690c6b1f3fbabac392d0920863edddbc6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xa02c8b02efb52fad3056fc96029467937c38c96d922250f6d2c0f77b923c85aa", + vec![196, 1, 128, 128, 128], + ), + ( + "0xa03fe040e4264070290e95ffe06bf9da0006556091f17c5df5abaa041de0c2f7", + vec![196, 1, 128, 128, 128], + ), + ( + "0xa0f5dc2d18608f8e522ffffd86828e3d792b36d924d5505c614383ddff9be2eb", + vec![ + 228, 1, 128, 160, 42, 254, 147, 225, 176, 242, 110, 88, 141, 40, 9, 18, 126, + 67, 96, 173, 126, 40, 207, 85, 36, 152, 178, 188, 72, 71, 214, 188, 218, 115, + 140, 219, 128, + ], + ), + ( + "0xa13bfef92e05edee891599aa5e447ff2baa1708d9a6473a04ef66ab94f2a11e4", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xa15773c9bfabef49e9825460ed95bf67b22b67d7806c840e0eb546d73c424768", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xa248850a2e0d6fe62259d33fc498203389fa754c3bd098163e86946888e455bd", + vec![196, 128, 1, 128, 128], + ), + ( + "0xa3abdaefbb886078dc6c5c72e4bc8d12e117dbbd588236c3fa7e0c69420eb24a", + vec![196, 1, 128, 128, 128], + ), + ( + "0xa3d8baf7ae7c96b1020753d12154e28cc7206402037c28c49c332a08cf7c4b51", + vec![196, 1, 128, 128, 128], + ), + ( + "0xa5541b637a896d30688a80b7affda987d9597aac7ccd9799c15999a1d7d094e2", + vec![ + 228, 1, 128, 160, 243, 144, 38, 74, 202, 241, 67, 60, 14, 166, 112, 178, 192, + 148, 163, 0, 118, 100, 20, 105, 82, 74, 226, 79, 95, 221, 196, 78, 153, 197, + 176, 50, 128, + ], + ), + ( + "0xa601eb611972ca80636bc39087a1dae7be5a189b94bda392f84d6ce0d3c866b9", + vec![ + 228, 1, 128, 160, 156, 50, 255, 213, 5, 145, 21, 187, 169, 174, 217, 23, 79, + 90, 184, 180, 53, 46, 63, 81, 168, 93, 222, 51, 0, 15, 112, 60, 155, 159, 231, + 194, 128, + ], + ), + ( + "0xa683478d0c949580d5738b490fac8129275bb6e921dfe5eae37292be3ee281b9", + vec![ + 228, 1, 128, 160, 193, 91, 67, 229, 244, 133, 62, 200, 218, 83, 235, 222, 3, + 222, 135, 185, 74, 252, 228, 42, 156, 2, 246, 72, 173, 139, 219, 34, 70, 4, + 196, 173, 128, + ], + ), + ( + "0xa87387b50b481431c6ccdb9ae99a54d4dcdd4a3eff75d7b17b4818f7bbfc21e9", + vec![ + 228, 1, 128, 160, 226, 167, 47, 91, 251, 235, 167, 15, 201, 171, 80, 98, 55, + 186, 39, 192, 150, 164, 233, 108, 57, 104, 202, 191, 91, 27, 47, 181, 68, 49, + 181, 207, 128, + ], + ), + ( + "0xa9233a729f0468c9c309c48b82934c99ba1fd18447947b3bc0621adb7a5fc643", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xa95c88d7dc0f2373287c3b2407ba8e7419063833c424b06d8bb3b29181bb632e", + vec![196, 128, 128, 128, 128], + ), + ( + "0xa9656c0192bb27f0ef3f93ecc6cc990dd146da97ac11f3d8d0899fba68d5749a", + vec![ + 228, 1, 128, 160, 114, 23, 203, 116, 112, 84, 48, 111, 130, 110, 120, 170, 63, + 198, 143, 228, 68, 18, 153, 163, 55, 236, 234, 29, 98, 88, 47, 45, 168, 167, + 243, 54, 128, + ], + ), + ( + "0xa9970b3744a0e46b248aaf080a001441d24175b5534ad80755661d271b976d67", + vec![ + 228, 1, 128, 160, 18, 222, 69, 68, 100, 15, 200, 160, 39, 225, 169, 18, 215, + 118, 185, 6, 117, 190, 191, 213, 7, 16, 194, 135, 107, 42, 36, 236, 158, 206, + 211, 103, 128, + ], + ), + ( + "0xa9de128e7d4347403eb97f45e969cd1882dfe22c1abe8857aab3af6d0f9e9b92", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xa9fd2e3a6de5a9da5badd719bd6e048acefa6d29399d8a99e19fd9626805b60b", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xaa0ac2f707a3dc131374839d4ee969eeb1cb55adea878f56e7b5b83d187d925c", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xaa0ffaa57269b865dccce764bf412de1dff3e7bba22ce319ef09e5907317b3e7", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xaad7b91d085a94c11a2f7e77dc95cfcfc5daf4f509ca4e0c0e493b86c6cbff78", + vec![ + 228, 1, 128, 160, 160, 144, 182, 111, 188, 164, 108, 183, 26, 189, 29, 170, + 141, 65, 157, 44, 110, 41, 16, 148, 245, 40, 114, 151, 141, 252, 177, 195, 26, + 215, 169, 0, 128, + ], + ), + ( + "0xab7bdc41a80ae9c8fcb9426ba716d8d47e523f94ffb4b9823512d259c9eca8cd", + vec![196, 1, 128, 128, 128], + ), + ( + "0xabd8afe9fbf5eaa36c506d7c8a2d48a35d013472f8182816be9c833be35e50da", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xabdc44a9bc7ccf1ce76b942d25cd9d731425cd04989597d7a2e36423e2dac7ee", + vec![196, 128, 1, 128, 128], + ), + ( + "0xac7183ebb421005a660509b070d3d47fc4e134cb7379c31dc35dc03ebd02e1cf", + vec![196, 1, 128, 128, 128], + ), + ( + "0xad6a4a6ebd5166c9b5cc8cfbaec176cced40fa88c73d83c67f0c3ed426121ebc", + vec![196, 128, 1, 128, 128], + ), + ( + "0xad99b5bc38016547d5859f96be59bf18f994314116454def33ebfe9a892c508a", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xae88076d02b19c4d09cb13fca14303687417b632444f3e30fc4880c225867be3", + vec![196, 1, 128, 128, 128], + ), + ( + "0xaeaf19d38b69be4fb41cc89e4888708daa6b9b1c3f519fa28fe9a0da70cd8697", + vec![196, 1, 128, 128, 128], + ), + ( + "0xaef83ad0ab332330a20e88cd3b5a4bcf6ac6c175ee780ed4183d11340df17833", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xaf38e0e6a4a4005507b5d3e9470e8ccc0273b74b6971f768cbdf85abeab8a95b", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xaf7c37d08a73483eff9ef5054477fb5d836a184aa07c3edb4409b9eb22dd56ca", + vec![ + 228, 1, 128, 160, 197, 118, 4, 164, 97, 201, 78, 205, 172, 18, 219, 183, 6, + 165, 43, 50, 145, 61, 114, 37, 59, 175, 251, 137, 6, 231, 66, 114, 74, 225, 36, + 73, 128, + ], + ), + ( + "0xb062c716d86a832649bccd53e9b11c77fc8a2a00ef0cc0dd2f561688a69d54f7", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb17ea61d092bd5d77edd9d5214e9483607689cdcc35a30f7ea49071b3be88c64", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xb1b2c1c59637202bb0e0d21255e44e0df719fe990be05f213b1b813e3d8179d7", + vec![196, 1, 128, 128, 128], + ), + ( + "0xb1b2fd7758f73e25a2f9e72edde82995b2b32ab798bcffd2c7143f2fc8196fd8", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb31919583a759b75e83c14d00d0a89bb36adc452f73cee2933a346ccebaa8e31", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb3a33a7f35ca5d08552516f58e9f76219716f9930a3a11ce9ae5db3e7a81445d", + vec![ + 228, 1, 128, 160, 131, 71, 24, 17, 17, 33, 226, 5, 143, 219, 144, 165, 31, 68, + 128, 40, 7, 24, 87, 225, 31, 189, 85, 212, 50, 86, 23, 77, 245, 106, 240, 26, + 128, + ], + ), + ( + "0xb40cc623b26a22203675787ca05b3be2c2af34b6b565bab95d43e7057e458684", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xb4f179efc346197df9c3a1cb3e95ea743ddde97c27b31ad472d352dba09ee1f5", + vec![196, 1, 128, 128, 128], + ), + ( + "0xb58e22a9ece8f9b3fdbaa7d17fe5fc92345df11d6863db4159647d64a34ff10b", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb58e67c536550fdf7140c8333ca62128df469a7270b16d528bc778909e0ac9a5", + vec![ + 228, 1, 128, 160, 35, 168, 136, 192, 164, 100, 206, 70, 22, 81, 252, 27, 226, + 207, 160, 203, 107, 164, 209, 177, 37, 171, 229, 180, 71, 238, 173, 249, 197, + 173, 241, 241, 128, + ], + ), + ( + "0xb5bca5e9ccef948c2431372315acc3b96e098d0e962b0c99d634a0475b670dc3", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb66092bc3624d84ff94ee42b097e846baf6142197d2c31245734d56a275c8eb9", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb7c2ef96238f635f86f9950700e36368efaaa70e764865dddc43ff6e96f6b346", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xb7d9d175039df1ba52c734547844f8805252893c029f7dbba9a63f8bce3ee306", + vec![196, 128, 1, 128, 128], + ), + ( + "0xb888c9946a84be90a9e77539b5ac68a3c459761950a460f3e671b708bb39c41f", + vec![196, 1, 128, 128, 128], + ), + ( + "0xb8d9b988ed60dbf5dca3e9d169343ca667498605f34fb6c30b45b2ed0f996f1a", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xb91824b28183c95881ada12404d5ee8af8123689a98054d41aaf4dd5bec50e90", + vec![196, 1, 128, 128, 128], + ), + ( + "0xb9400acf38453fd206bc18f67ba04f55b807b20e4efc2157909d91d3a9f7bed2", + vec![196, 1, 128, 128, 128], + ), + ( + "0xb990eaca858ea15fda296f3f47baa2939e8aa8bbccc12ca0c3746d9b5d5fb2ae", + vec![ + 228, 1, 128, 160, 137, 236, 176, 206, 238, 162, 12, 205, 125, 27, 24, 207, 29, + 53, 183, 162, 253, 123, 118, 221, 200, 214, 39, 244, 51, 4, 237, 139, 49, 176, + 18, 72, 128, + ], + ), + ( + "0xb9cddc73dfdacd009e55f27bdfd1cd37eef022ded5ce686ab0ffe890e6bf311e", + vec![ + 228, 1, 128, 160, 61, 32, 254, 221, 39, 11, 55, 113, 112, 111, 224, 10, 88, 10, + 21, 84, 57, 190, 87, 232, 213, 80, 118, 45, 239, 16, 144, 110, 131, 237, 88, + 187, 128, + ], + ), + ( + "0xba1d0afdfee510e8852f24dff964afd824bf36d458cf5f5d45f02f04b7c0b35d", + vec![196, 1, 128, 128, 128], + ), + ( + "0xbaae09901e990935de19456ac6a6c8bc1e339d0b80ca129b8622d989b5c79120", + vec![ + 228, 1, 128, 160, 37, 180, 46, 197, 72, 8, 67, 160, 50, 140, 99, 188, 80, 239, + 248, 89, 93, 144, 241, 209, 176, 175, 202, 178, 244, 161, 155, 136, 140, 121, + 79, 55, 128, + ], + ), + ( + "0xbb861b82d884a70666afeb78bbf30cab7fdccf838f4d5ce5f4e5ca1be6be61b1", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xbbdc59572cc62c338fb6e027ab00c57cdeed233c8732680a56a5747141d20c7c", + vec![196, 1, 128, 128, 128], + ), + ( + "0xbccd3d2f920dfb8d70a38c9ccd5ed68c2ef6e3372199381767ce222f13f36c87", + vec![196, 128, 1, 128, 128], + ), + ( + "0xbccd85b63dba6300f84c561c5f52ce08a240564421e382e6f550ce0c12f2f632", + vec![ + 228, 1, 128, 160, 234, 131, 56, 147, 131, 21, 34, 112, 16, 64, 147, 237, 93, + 254, 52, 186, 64, 60, 117, 48, 129, 51, 170, 27, 232, 245, 26, 216, 4, 179, + 233, 238, 128, + ], + ), + ( + "0xbcebc35bfc663ecd6d4410ee2363e5b7741ee953c7d3359aa585095e503d20c8", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xbe7d987a9265c0e44e9c5736fb2eb38c41973ce96e5e8e6c3c713f9d50a079ff", + vec![ + 228, 1, 128, 160, 175, 213, 78, 129, 243, 228, 21, 64, 127, 8, 18, 166, 120, + 133, 111, 27, 64, 104, 237, 100, 160, 139, 63, 59, 245, 178, 25, 15, 207, 178, + 50, 45, 128, + ], + ), + ( + "0xbea55c1dc9f4a9fb50cbedc70448a4e162792b9502bb28b936c7e0a2fd7fe41d", + vec![ + 228, 1, 128, 160, 49, 10, 42, 200, 61, 126, 62, 77, 51, 49, 2, 177, 247, 21, + 59, 176, 65, 107, 56, 66, 126, 178, 227, 53, 220, 102, 50, 215, 121, 168, 180, + 175, 128, + ], + ), + ( + "0xbf632670b6fa18a8ad174a36180202bfef9a92c2eeda55412460491ae0f6a969", + vec![ + 228, 1, 128, 160, 207, 33, 35, 209, 16, 153, 127, 66, 104, 33, 211, 229, 65, + 51, 78, 67, 253, 214, 181, 40, 108, 60, 51, 37, 44, 36, 181, 248, 170, 252, + 122, 162, 128, + ], + ), + ( + "0xbfaac98225451c56b2f9aec858cffc1eb253909615f3d9617627c793b938694f", + vec![ + 228, 1, 128, 160, 238, 152, 33, 98, 26, 165, 236, 154, 183, 213, 135, 139, 42, + 153, 82, 40, 173, 205, 202, 203, 113, 13, 245, 34, 210, 249, 27, 67, 77, 59, + 220, 121, 128, + ], + ), + ( + "0xbfe5dee42bddd2860a8ebbcdd09f9c52a588ba38659cf5e74b07d20f396e04d4", + vec![196, 1, 128, 128, 128], + ), + ( + "0xbfe731f071443795cef55325f32e6e03c8c0d0398671548dfd5bc96b5a6555c0", + vec![ + 228, 1, 128, 160, 178, 95, 158, 79, 111, 145, 58, 74, 30, 141, 235, 247, 212, + 117, 43, 250, 82, 29, 20, 123, 182, 124, 105, 213, 133, 83, 1, 231, 109, 216, + 6, 51, 128, + ], + ), + ( + "0xc0ce77c6a355e57b89cca643e70450612c0744c9f0f8bf7dee51d6633dc850b1", + vec![ + 228, 1, 128, 160, 223, 60, 27, 250, 184, 247, 231, 10, 142, 223, 148, 121, 47, + 145, 228, 182, 178, 194, 170, 97, 202, 246, 135, 228, 246, 203, 104, 157, 24, + 10, 219, 128, 128, + ], + ), + ( + "0xc13c19f53ce8b6411d6cdaafd8480dfa462ffdf39e2eb68df90181a128d88992", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc157e0d637d64b90e2c59bc8bed2acd75696ea1ac6b633661c12ce8f2bce0d62", + vec![196, 1, 128, 128, 128], + ), + ( + "0xc192ea2d2bb89e9bb7f17f3a282ebe8d1dd672355b5555f516b99b91799b01f6", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xc1a6a0bf60ee7b3228ecf6cb7c9e5491fbf62642a3650d73314e976d9eb9a966", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc2406cbd93e511ef493ac81ebe2b6a3fbecd05a3ba52d82a23a88eeb9d8604f0", + vec![ + 228, 1, 128, 160, 130, 179, 38, 100, 24, 37, 55, 143, 170, 17, 198, 65, 201, + 22, 242, 226, 44, 1, 8, 15, 72, 125, 224, 70, 62, 48, 213, 227, 43, 150, 15, + 151, 128, + ], + ), + ( + "0xc250f30c01f4b7910c2eb8cdcd697cf493f6417bb2ed61d637d625a85a400912", + vec![ + 228, 1, 128, 160, 202, 57, 245, 244, 238, 60, 107, 51, 239, 231, 188, 72, 84, + 57, 249, 127, 157, 198, 47, 101, 133, 44, 122, 28, 223, 84, 250, 177, 227, 183, + 4, 41, 128, + ], + ), + ( + "0xc251a3acb75a90ff0cdca31da1408a27ef7dcaa42f18e648f2be1a28b35eac32", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc2c26fbc0b7893d872fa528d6c235caab9164feb5b54c48381ff3d82c8244e77", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc3791fc487a84f3731eb5a8129a7e26f357089971657813b48a821f5582514b3", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xc3ac56e9e7f2f2c2c089e966d1b83414951586c3afeb86300531dfa350e38929", + vec![ + 228, 1, 128, 160, 129, 142, 175, 90, 219, 86, 198, 114, 136, 137, 186, 102, + 182, 152, 12, 214, 107, 65, 25, 159, 0, 7, 205, 217, 5, 174, 115, 148, 5, 227, + 198, 48, 128, + ], + ), + ( + "0xc3c8e2dc64e67baa83b844263fe31bfe24de17bb72bfed790ab345b97b007816", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xc4bab059ee8f7b36c82ada44d22129671d8f47f254ca6a48fded94a8ff591c88", + vec![196, 1, 128, 128, 128], + ), + ( + "0xc54ffffcbaa5b566a7cf37386c4ce5a338d558612343caaa99788343d516aa5f", + vec![196, 1, 128, 128, 128], + ), + ( + "0xc781c7c3babeb06adfe8f09ecb61dbe0eb671e41f3a1163faac82fdfa2bc83e8", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc7fc033fe9f00d24cb9c479ddc0598e592737c305263d088001d7419d16feffa", + vec![196, 128, 1, 128, 128], + ), + ( + "0xc9ea69dc9e84712b1349c9b271956cc0cb9473106be92d7a937b29e78e7e970e", + vec![196, 1, 128, 128, 128], + ), + ( + "0xca7ad42d3c4fe14ddb81bf27d4679725a1f6c3f23b688681bb6f24262d63212f", + vec![196, 128, 1, 128, 128], + ), + ( + "0xcac96145454c46255fccca35343d9505164dabe319c17d81fda93cf1171e4c6e", + vec![196, 128, 1, 128, 128], + ), + ( + "0xcade985c7fb6d371d0c7f7cb40178e7873d623eadcc37545798ec33a04bb2173", + vec![196, 1, 128, 128, 128], + ), + ( + "0xcb54add475a18ea02ab1adf9e2e73da7f23ecd3e92c4fa8ca4e8f588258cb5d3", + vec![196, 128, 1, 128, 128], + ), + ( + "0xcb6f450b4720c6b36d3a12271e35ace27f1d527d46b073771541ad39cc59398d", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xcc74930e1ee0e71a8081f247ec47442a3e5d00897966754a5b3ee8beb2c1160c", + vec![196, 1, 128, 128, 128], + ), + ( + "0xcd07379b0120ad9a9c7fa47e77190be321ab107670f3115fec485bebb467307d", + vec![196, 128, 1, 128, 128], + ), + ( + "0xcd6b3739d4dbce17dafc156790f2a3936eb75ce95e9bba039dd76661f40ea309", + vec![ + 228, 1, 128, 160, 176, 112, 15, 225, 61, 186, 249, 75, 229, 11, 203, 236, 19, + 167, 181, 62, 108, 186, 3, 75, 41, 163, 218, 186, 152, 250, 134, 31, 88, 151, + 33, 63, 128, + ], + ), + ( + "0xce732a5e3b88ae26790aeb390a2bc02c449fdf57665c6d2c2b0dbce338c4377e", + vec![196, 128, 1, 128, 128], + ), + ( + "0xd1691564c6a5ab1391f0495634e749b9782de33756b6a058f4a9536c1b37bca6", + vec![ + 228, 1, 128, 160, 214, 14, 228, 173, 90, 187, 231, 89, 98, 47, 202, 92, 83, 97, + 9, 177, 30, 133, 170, 43, 72, 192, 190, 42, 235, 240, 29, 245, 151, 231, 77, + 186, 128, + ], + ), + ( + "0xd16e029e8c67c3f330cddaa86f82d31f523028404dfccd16d288645d718eb9da", + vec![196, 128, 1, 128, 128], + ), + ( + "0xd2501ae11a14bf0c2283a24b7e77c846c00a63e71908c6a5e1caff201bad0762", + vec![ + 228, 128, 128, 160, 73, 27, 44, 251, 169, 118, 178, 231, 139, 217, 190, 59, + 193, 92, 153, 100, 146, 114, 5, 252, 52, 201, 149, 74, 77, 97, 187, 232, 23, + 11, 165, 51, 128, + ], + ), + ( + "0xd2f394b4549b085fb9b9a8b313a874ea660808a4323ab2598ee15ddd1eb7e897", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd3443fa37ee617edc09a9c930be4873c21af2c47c99601d5e20483ce6d01960a", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xd352b05571154d9a2061143fe6df190a740a2d321c59eb94a54acb7f3054e489", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd37b6f5e5f0fa6a1b3fd15c9b3cf0fb595ba245ab912ad8059e672fa55f061b8", + vec![ + 228, 1, 128, 160, 89, 147, 108, 21, 196, 84, 147, 62, 188, 73, 137, 175, 167, + 126, 53, 15, 118, 64, 48, 27, 7, 52, 26, 234, 213, 241, 178, 102, 142, 235, 29, + 173, 128, + ], + ), + ( + "0xd52564daf6d32a6ae29470732726859261f5a7409b4858101bd233ed5cc2f662", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xd57eafe6d4c5b91fe7114e199318ab640e55d67a1e9e3c7833253808b7dca75f", + vec![ + 228, 1, 128, 160, 224, 163, 211, 184, 57, 252, 160, 245, 71, 69, 208, 197, 10, + 4, 142, 66, 76, 146, 89, 240, 99, 183, 65, 100, 16, 164, 66, 46, 235, 127, 131, + 126, 128, + ], + ), + ( + "0xd5e252ab2fba10107258010f154445cf7dffc42b7d8c5476de9a7adb533d73f1", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd5e5e7be8a61bb5bfa271dfc265aa9744dea85de957b6cffff0ecb403f9697db", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd623b1845175b206c127c08046281c013e4a3316402a771f1b3b77a9831143f5", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd63070208c85e91c4c8c942cf52c416f0f3004c392a15f579350168f178dba2e", + vec![196, 128, 1, 128, 128], + ), + ( + "0xd72e318c1cea7baf503950c9b1bd67cf7caf2f663061fcde48d379047a38d075", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xd8489fd0ce5e1806b24d1a7ce0e4ba8f0856b87696456539fcbb625a9bed2ccc", + vec![ + 228, 1, 128, 160, 52, 55, 128, 49, 1, 168, 4, 10, 202, 39, 63, 183, 52, 215, + 150, 90, 135, 248, 35, 255, 30, 247, 140, 126, 220, 170, 211, 88, 235, 152, + 222, 227, 128, + ], + ), + ( + "0xd84f7711be2f8eca69c742153230995afb483855b7c555b08da330139cdb9579", + vec![ + 228, 1, 128, 160, 158, 83, 240, 162, 221, 180, 48, 210, 127, 111, 255, 160, + 166, 139, 95, 117, 219, 29, 104, 226, 65, 19, 220, 202, 110, 51, 145, 140, 218, + 232, 8, 70, 128, + ], + ), + ( + "0xd9f987fec216556304eba05bcdae47bb736eea5a4183eb3e2c3a5045734ae8c7", + vec![196, 1, 128, 128, 128], + ), + ( + "0xd9fa858992bc92386a7cebcd748eedd602bf432cb4b31607566bc92b85179624", + vec![196, 128, 1, 128, 128], + ), + ( + "0xda81833ff053aff243d305449775c3fb1bd7f62c4a3c95dc9fb91b85e032faee", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xdbd66b6a89e01c76ae5f8cb0dcd8a24e787f58f015c9b08972bfabefa2eae0d5", + vec![196, 1, 128, 128, 128], + ), + ( + "0xdbea1fd70fe1c93dfef412ce5d8565d87d6843aac044d3a015fc3db4d20a351b", + vec![ + 228, 1, 128, 160, 190, 254, 85, 182, 6, 168, 101, 195, 137, 142, 194, 9, 59, + 209, 96, 179, 124, 57, 118, 1, 21, 22, 244, 55, 54, 202, 194, 169, 167, 236, + 212, 202, 128, + ], + ), + ( + "0xdc9ea08bdea052acab7c990edbb85551f2af3e1f1a236356ab345ac5bcc84562", + vec![ + 228, 128, 128, 160, 32, 127, 108, 62, 69, 5, 70, 176, 209, 243, 188, 106, 111, + 175, 91, 250, 11, 255, 128, 57, 108, 85, 213, 103, 184, 52, 207, 14, 124, 118, + 3, 71, 128, + ], + ), + ( + "0xdcda5b5203c2257997a574bdf85b2bea6d04829e8d7e048a709badc0fb99288c", + vec![ + 228, 1, 128, 160, 174, 68, 1, 67, 210, 30, 36, 169, 49, 182, 117, 111, 107, 61, + 80, 211, 55, 234, 240, 219, 62, 108, 52, 227, 106, 180, 111, 226, 217, 158, + 248, 62, 128, + ], + ), + ( + "0xdce547cc70c79575ef72c061502d6066db1cbce200bd904d5d2b20d4f1cb5963", + vec![ + 228, 1, 128, 160, 38, 37, 248, 162, 61, 36, 165, 223, 246, 167, 159, 99, 43, + 16, 32, 89, 51, 98, 166, 172, 98, 47, 165, 35, 116, 96, 188, 103, 176, 170, 14, + 211, 128, + ], + ), + ( + "0xdd1589b1fe1d9b4ca947f98ff324de7887af299d5490ed92ae40e95eec944118", + vec![196, 128, 1, 128, 128], + ), + ( + "0xdef989cb85107747de11222bd7418411f8f3264855e1939ef6bef9447e42076d", + vec![196, 128, 1, 128, 128], + ), + ( + "0xe02ec497b66cb57679eb01de1bed2ad385a3d18130441a9d337bd14897e85d39", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe04fdefc4f2eefd22721d5944411b282d0fcb1f9ac218f54793a35bca8199c25", + vec![196, 128, 1, 128, 128], + ), + ( + "0xe09e5f27b8a7bf61805df6e5fefc24eb6894281550c2d06250adecfe1e6581d7", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe0c5acf66bda927704953fdf7fb4b99e116857121c069eca7fb9bd8acfc25434", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe1068e9986da7636501d8893f67aa94f5d73df849feab36505fd990e2d6240e9", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe1b86a365b0f1583a07fc014602efc3f7dedfa90c66e738e9850719d34ac194e", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe1eb1e18ae510d0066d60db5c2752e8c33604d4da24c38d2bda07c0cb6ad19e4", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe31747e6542bf4351087edfbeb23e225e4217b5fa25d385f33cd024df0c9ae12", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe333845edc60ed469a894c43ed8c06ec807dafd079b3c948077da56e18436290", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe3c2e12be28e2e36dc852e76dd32e091954f99f2a6480853cd7b9e01ec6cd889", + vec![ + 228, 1, 128, 160, 204, 72, 248, 209, 192, 221, 110, 200, 171, 123, 189, 121, + 45, 148, 246, 167, 76, 136, 118, 180, 27, 200, 89, 206, 226, 34, 142, 141, 173, + 130, 7, 164, 128, + ], + ), + ( + "0xe3c79e424fd3a7e5bf8e0426383abd518604272fda87ecd94e1633d36f55bbb6", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe3d7213321be060ae2e1ff70871131ab3e4c9f4214a17fe9441453745c29365b", + vec![196, 128, 1, 128, 128], + ), + ( + "0xe42a85d04a1d0d9fe0703020ef98fa89ecdeb241a48de2db73f2feeaa2e49b0f", + vec![ + 228, 1, 128, 160, 251, 0, 114, 154, 95, 79, 154, 36, 54, 185, 153, 170, 113, + 89, 73, 122, 156, 216, 141, 21, 87, 112, 248, 115, 168, 24, 181, 80, 82, 197, + 240, 103, 128, + ], + ), + ( + "0xe4d9c31cc9b4a9050bbbf77cc08ac26d134253dcb6fd994275c5c3468f5b7810", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe5302e42ca6111d3515cbbb2225265077da41d997f069a6c492fa3fcb0fdf284", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xe6388bfcbbd6000e90a10633c72c43b0b0fed7cf38eab785a71e6f0c5b80a26a", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe69f40f00148bf0d4dfa28b3f3f5a0297790555eca01a00e49517c6645096a6c", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe6c5edf6a0fbdcff100e5ceafb63cba9aea355ba397a93fdb42a1a67b91375f8", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe6d72f72fd2fc8af227f75ab3ab199f12dfb939bdcff5f0acdac06a90084def8", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe73b3367629c8cb991f244ac073c0863ad1d8d88c2e180dd582cefda2de4415e", + vec![196, 1, 128, 128, 128], + ), + ( + "0xe74ac72f03e8c514c2c75f3c4f54ba31e920374ea7744ef1c33937e64c7d54f1", + vec![196, 128, 1, 128, 128], + ), + ( + "0xe7c6828e1fe8c586b263a81aafc9587d313c609c6db8665a42ae1267cd9ade59", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xe99460a483f3369006e3edeb356b3653699f246ec71f30568617ebc702058f59", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xea810ea64a420acfa917346a4a02580a50483890cba1d8d1d158d11f1c59ed02", + vec![ + 228, 1, 128, 160, 147, 106, 198, 37, 24, 72, 218, 105, 161, 145, 204, 145, 23, + 78, 75, 117, 131, 161, 42, 67, 216, 150, 226, 67, 132, 30, 169, 139, 101, 242, + 100, 173, 128, + ], + ), + ( + "0xeba984db32038d7f4d71859a9a2fc6e19dde2e23f34b7cedf0c4bf228c319f17", + vec![196, 128, 1, 128, 128], + ), + ( + "0xec3e92967d10ac66eff64a5697258b8acf87e661962b2938a0edcd78788f360d", + vec![ + 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, + 128, 128, + ], + ), + ( + "0xed263a22f0e8be37bcc1873e589c54fe37fdde92902dc75d656997a7158a9d8c", + vec![ + 228, 1, 128, 160, 229, 71, 192, 5, 2, 83, 7, 91, 27, 228, 33, 6, 8, 188, 99, + 156, 255, 231, 1, 16, 25, 76, 49, 100, 129, 35, 94, 115, 139, 233, 97, 231, + 128, + ], + ), + ( + "0xedd9b1f966f1dfe50234523b479a45e95a1a8ec4a057ba5bfa7b69a13768197c", + vec![196, 1, 128, 128, 128], + ), + ( + "0xee9186a01e5e1122b61223b0e6acc6a069c9dcdb7307b0a296421272275f821b", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xefaff7acc3ad3417517b21a92187d2e63d7a77bc284290ed406d1bc07ab3d885", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xf0877d51b7712e08f2a3c96cddf50ff61b8b90f80b8b9817ea613a8a157b0c45", + vec![196, 1, 128, 128, 128], + ), + ( + "0xf0a51b55aadfa3cafdd214b0676816e574931a683f51218207c625375884e785", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf164775805f47d8970d3282188009d4d7a2da1574fe97e5d7bc9836a2eed1d5b", + vec![196, 1, 128, 128, 128], + ), + ( + "0xf16522fc36907ee1e9948240b0c1d1d105a75cc63b71006f16c20d79ad469bd7", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf19ee923ed66b7b9264c2644aa20e5268a251b4914ca81b1dffee96ecb074cb1", + vec![ + 228, 1, 128, 160, 205, 62, 117, 41, 158, 150, 125, 95, 136, 211, 6, 190, 144, + 90, 19, 67, 67, 178, 36, 211, 253, 90, 134, 27, 26, 105, 13, 224, 226, 223, + 225, 186, 128, + ], + ), + ( + "0xf2b9bc1163840284f3eb15c539972edad583cda91946f344f4cb57be15af9c8f", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf33a7b66489679fa665dbfb4e6dd4b673495f853850eedc81d5f28bd2f4bd3b5", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xf462aaa112b195c148974ff796a81c0e7f9a972d04e60c178ac109102d593a88", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf4a1c4554b186a354b3e0c467eef03df9907cd5a5d96086c1a542b9e5160ca78", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf63360f8bb23f88b0a564f9e07631c38c73b4074ba4192d6131336ef02ee9cf2", + vec![196, 128, 1, 128, 128], + ), + ( + "0xf84223f460140ad56af9836cfa6c1c58c1397abf599c214689bc881066020ff7", + vec![196, 1, 128, 128, 128], + ), + ( + "0xfab4c6889992a3f4e96b005dfd851021e9e1ec2631a7ccd2a001433e35077968", + vec![196, 128, 1, 128, 128], + ), + ( + "0xfb2ab315988de92dcf6ba848e756676265b56e4b84778a2c955fb2b3c848c51c", + vec![196, 1, 128, 128, 128], + ), + ( + "0xfb5a31c5cfd33dce2c80a30c5efc28e5f4025624adcc2205a2504a78c57bdd1c", + vec![ + 228, 1, 128, 160, 73, 63, 144, 67, 84, 2, 223, 9, 7, 1, 155, 255, 198, 221, 37, + 161, 124, 228, 172, 214, 235, 96, 119, 239, 148, 193, 98, 111, 13, 119, 201, + 240, 128, + ], + ), + ( + "0xfb9474d0e5538fcd99e8d8d024db335b4e057f4bcd359e85d78f4a5226b33272", + vec![196, 128, 1, 128, 128], + ), + ( + "0xfc3d2e27841c0913d10aa11fc4af4793bf376efe3d90ce8360aa392d0ecefa24", + vec![ + 228, 1, 128, 160, 123, 245, 66, 189, 175, 245, 191, 227, 211, 60, 38, 168, 135, + 119, 119, 59, 94, 82, 84, 97, 9, 60, 54, 172, 176, 218, 181, 145, 163, 25, 229, + 9, 128, + ], + ), + ( + "0xfc4870c3cd21d694424c88f0f31f75b2426e1530fdea26a14031ccf9baed84c4", + vec![196, 128, 1, 128, 128], + ), + ( + "0xfc8d513d1615c763865b984ea9c381032c14a983f80e5b2bd90b20b518329ed7", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xfcc08928955d4e5e17e17e46d5adbb8011e0a8a74cabbdd3e138c367e89a4428", + vec![196, 128, 1, 128, 128], + ), + ( + "0xfd3a8bacd3b2061cbe54f8d38cf13c5c87a92816937683652886dee936dfae10", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xfdaf2549ea901a469b3e91cd1c4290fab376ef687547046751e10b7b461ff297", + vec![196, 1, 128, 128, 128], + ), + ( + "0xfdbb8ddca8cecfe275da1ea1c36e494536f581d64ddf0c4f2e6dae9c7d891427", + vec![ + 228, 1, 128, 160, 211, 217, 131, 159, 135, 194, 159, 176, 7, 253, 153, 40, 211, + 139, 191, 132, 239, 8, 159, 12, 214, 64, 200, 56, 244, 164, 38, 49, 232, 40, + 198, 103, 128, + ], + ), + ( + "0xfe2149c5c256a5eb2578c013d33e3af6a87a514965c7ddf4a8131e2d978f09f9", + vec![196, 128, 1, 128, 128], + ), + ( + "0xfe2511e8a33ac9973b773aaedcb4daa73ae82481fe5a1bf78b41281924260cf5", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ( + "0xfe6e594c507ec0ac14917f7a8032f83cd0c3c58b461d459b822190290852c0e1", + vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], + ), + ]; + + // Create a store and load it up with the accounts + let store = Store::new("null", EngineType::InMemory).unwrap(); + let mut state_trie = store.new_state_trie_for_test(); + for (address, account) in accounts { + let hashed_address = H256::from_str(address).unwrap(); + let account = AccountState::from(AccountStateSlim::decode(&account).unwrap()); + state_trie + .insert(hashed_address.encode_to_vec(), account.encode_to_vec()) + .unwrap(); + } + (store, state_trie.hash().unwrap()) + } + + #[test] + fn hive_account_range_a() { + let (store, root) = setup_initial_state(); + // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 86); + assert_eq!(res.accounts.first().unwrap().0, H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6").unwrap()); + assert_eq!(res.accounts.last().unwrap().0, H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099").unwrap()); + // Check proofs against geth values + } +} diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 9618d5cf3..82a88a3c9 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -708,6 +708,11 @@ impl Store { pub fn get_payload(&self, payload_id: u64) -> Result, StoreError> { self.engine.get_payload(payload_id) } + + /// Creates a new state trie with an empty state root, for testing purposes only + pub fn new_state_trie_for_test(&self) -> Trie { + self.engine.open_state_trie(*EMPTY_TRIE_HASH) + } } fn hash_address(address: &Address) -> Vec { From d87b1cae6a76af260d29dff0748518857e12596e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 17:36:39 -0300 Subject: [PATCH 014/345] Add more tests --- crates/networking/p2p/snap.rs | 128 +++++++++++++++++++++++++++++++++- 1 file changed, 126 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 8b48663e0..ce3c93429 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -2199,8 +2199,132 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 86); - assert_eq!(res.accounts.first().unwrap().0, H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6").unwrap()); - assert_eq!(res.accounts.last().unwrap().0, H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099").unwrap()); + assert_eq!( + res.accounts.first().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") + .unwrap() + ); // Check proofs against geth values } + + #[test] + fn hive_account_range_b() { + let (store, root) = setup_initial_state(); + // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 3000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 65); + assert_eq!( + res.accounts.first().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x2e6fe1362b3e388184fd7bf08e99e74170b26361624ffd1c5f646da7067b58b6") + .unwrap() + ); + } + + #[test] + fn hive_account_range_c() { + let (store, root) = setup_initial_state(); + // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 2000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 44); + assert_eq!( + res.accounts.first().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x1c3f74249a4892081ba0634a819aec9ed25f34c7653f5719b9098487e65ab595") + .unwrap() + ); + } + + #[test] + fn hive_account_range_d() { + let (store, root) = setup_initial_state(); + // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 1, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!( + res.accounts.first().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + } + + #[test] + fn hive_account_range_e() { + let (store, root) = setup_initial_state(); + // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 0, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!( + res.accounts.first().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap() + ); + } } From 89621bf73239dae0b7260b1c7fea72e34ff4b950 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:00:43 -0300 Subject: [PATCH 015/345] Cleanup test code + Fix logic + add test --- crates/networking/p2p/Cargo.toml | 1 + crates/networking/p2p/snap.rs | 294 +++++++++++++++---------------- 2 files changed, 145 insertions(+), 150 deletions(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index 53e6f0dba..128cff7e7 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -15,6 +15,7 @@ tokio.workspace = true bytes.workspace = true hex.workspace = true thiserror.workspace = true +lazy_static.workspace = true k256 = { version = "0.13.3", features = ["ecdh"] } sha3 = "0.10.8" diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index ce3c93429..92af49222 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -13,9 +13,6 @@ pub fn process_account_range_request( let mut start_found = false; let mut bytes_used = 0; while let Some((k, v)) = iter.next() { - if k >= request.limit_hash { - break; - } if k >= request.starting_hash { start_found = true; } @@ -24,6 +21,9 @@ pub fn process_account_range_request( bytes_used += bytes_per_entry(&acc); accounts.push((k, acc)); } + if k >= request.limit_hash { + break; + } if bytes_used >= request.response_bytes { break; } @@ -46,7 +46,7 @@ fn bytes_per_entry(state: &AccountStateSlim) -> u64 { mod tests { use std::str::FromStr; - use ethereum_rust_core::{types::AccountState, H256}; + use ethereum_rust_core::{types::AccountState, BigEndianHash, H256}; use ethereum_rust_rlp::{decode::RLPDecode, encode::RLPEncode}; use ethereum_rust_storage::EngineType; @@ -54,6 +54,146 @@ mod tests { use super::*; + // Hive `AccounRange` Tests + // Requests & invariantes taken from https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L69 + + use lazy_static::lazy_static; + + lazy_static! { + static ref HASH_MIN: H256 = H256::zero(); + static ref HASH_MAX: H256 = + H256::from_str("0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff",) + .unwrap(); + static ref HASH_FIRST: H256 = + H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") + .unwrap(); + static ref HASH_SECOND: H256 = + H256::from_str("0x00748bacab20da9ae19dd26a33bd10bbf825e28b3de84fc8fe1d15a21645067f") + .unwrap(); + static ref HASH_FIRST_MINUS_500: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 500)); + static ref HASH_FIRST_PLUS_ONE: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() + 1)); + } + + #[test] + fn hive_account_range_a() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MAX, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 86); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") + .unwrap() + ); + // Check proofs against geth values + } + + #[test] + fn hive_account_range_b() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MAX, + response_bytes: 3000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 65); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x2e6fe1362b3e388184fd7bf08e99e74170b26361624ffd1c5f646da7067b58b6") + .unwrap() + ); + } + + #[test] + fn hive_account_range_c() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MAX, + response_bytes: 2000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 44); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x1c3f74249a4892081ba0634a819aec9ed25f34c7653f5719b9098487e65ab595") + .unwrap() + ); + } + + #[test] + fn hive_account_range_d() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MAX, + response_bytes: 1, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + + #[test] + fn hive_account_range_e() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MAX, + response_bytes: 0, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + + #[test] + fn hive_account_range_f() { + // In this test, we request a range where startingHash is before the first available + // account key, and limitHash is after. The server should return the first and second + // account of the state (because the second account is the 'next available'). + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST_MINUS_500, + limit_hash: *HASH_FIRST_PLUS_ONE, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 2); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_SECOND); + } + + // Initial state setup for hive snap tests + fn setup_initial_state() -> (Store, H256) { // We cannot process the old blocks that hive uses for the devp2p snap tests // So I took the state from a geth execution to run them locally @@ -2181,150 +2321,4 @@ mod tests { } (store, state_trie.hash().unwrap()) } - - #[test] - fn hive_account_range_a() { - let (store, root) = setup_initial_state(); - // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 - let request = GetAccountRange { - id: 0, - root_hash: root, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 4000, - }; - let res = process_account_range_request(request, store).unwrap(); - // Check test invariants - assert_eq!(res.accounts.len(), 86); - assert_eq!( - res.accounts.first().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - assert_eq!( - res.accounts.last().unwrap().0, - H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") - .unwrap() - ); - // Check proofs against geth values - } - - #[test] - fn hive_account_range_b() { - let (store, root) = setup_initial_state(); - // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 - let request = GetAccountRange { - id: 0, - root_hash: root, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 3000, - }; - let res = process_account_range_request(request, store).unwrap(); - // Check test invariants - assert_eq!(res.accounts.len(), 65); - assert_eq!( - res.accounts.first().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - assert_eq!( - res.accounts.last().unwrap().0, - H256::from_str("0x2e6fe1362b3e388184fd7bf08e99e74170b26361624ffd1c5f646da7067b58b6") - .unwrap() - ); - } - - #[test] - fn hive_account_range_c() { - let (store, root) = setup_initial_state(); - // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 - let request = GetAccountRange { - id: 0, - root_hash: root, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 2000, - }; - let res = process_account_range_request(request, store).unwrap(); - // Check test invariants - assert_eq!(res.accounts.len(), 44); - assert_eq!( - res.accounts.first().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - assert_eq!( - res.accounts.last().unwrap().0, - H256::from_str("0x1c3f74249a4892081ba0634a819aec9ed25f34c7653f5719b9098487e65ab595") - .unwrap() - ); - } - - #[test] - fn hive_account_range_d() { - let (store, root) = setup_initial_state(); - // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 - let request = GetAccountRange { - id: 0, - root_hash: root, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 1, - }; - let res = process_account_range_request(request, store).unwrap(); - // Check test invariants - assert_eq!(res.accounts.len(), 1); - assert_eq!( - res.accounts.first().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - assert_eq!( - res.accounts.last().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - } - - #[test] - fn hive_account_range_e() { - let (store, root) = setup_initial_state(); - // First request in test list: https://github.com/ethereum/go-ethereum/blob/3e567b8b2901611f004b5a6070a9b6d286be128d/cmd/devp2p/internal/ethtest/snap.go#L84 - let request = GetAccountRange { - id: 0, - root_hash: root, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 0, - }; - let res = process_account_range_request(request, store).unwrap(); - // Check test invariants - assert_eq!(res.accounts.len(), 1); - assert_eq!( - res.accounts.first().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - assert_eq!( - res.accounts.last().unwrap().0, - H256::from_str("0x005e94bf632e80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6") - .unwrap() - ); - } } From 3a14cdda40fc67370e05dd7cf9e84009f23c818c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:04:04 -0300 Subject: [PATCH 016/345] Add test --- crates/networking/p2p/snap.rs | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 92af49222..86badd36d 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -71,6 +71,7 @@ mod tests { H256::from_str("0x00748bacab20da9ae19dd26a33bd10bbf825e28b3de84fc8fe1d15a21645067f") .unwrap(); static ref HASH_FIRST_MINUS_500: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 500)); + static ref HASH_FIRST_MINUS_450: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 450)); static ref HASH_FIRST_PLUS_ONE: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() + 1)); } @@ -192,6 +193,25 @@ mod tests { assert_eq!(res.accounts.last().unwrap().0, *HASH_SECOND); } + #[test] + fn hive_account_range_g() { + // Here we request range where both bounds are before the first available account key. + // This should return the first account (even though it's out of bounds).` + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST_MINUS_500, + limit_hash: *HASH_FIRST_MINUS_450, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + // Initial state setup for hive snap tests fn setup_initial_state() -> (Store, H256) { From c354321b53a71705552be5dd81a15c0c0b2d0752 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:05:45 -0300 Subject: [PATCH 017/345] Add test --- crates/networking/p2p/snap.rs | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 86badd36d..b012c00a3 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -196,7 +196,7 @@ mod tests { #[test] fn hive_account_range_g() { // Here we request range where both bounds are before the first available account key. - // This should return the first account (even though it's out of bounds).` + // This should return the first account (even though it's out of bounds). let (store, root) = setup_initial_state(); let request = GetAccountRange { id: 0, @@ -212,6 +212,25 @@ mod tests { assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); } + #[test] + fn hive_account_range_h() { + // In this test, both startingHash and limitHash are zero. + // The server should return the first available account. + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_MIN, + limit_hash: *HASH_MIN, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + // Initial state setup for hive snap tests fn setup_initial_state() -> (Store, H256) { From 7abd4f7ba502ceee1fdc9f76290756b351569791 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:10:52 -0300 Subject: [PATCH 018/345] Add test --- crates/networking/p2p/snap.rs | 66 +++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index b012c00a3..0406ed9ee 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -72,6 +72,7 @@ mod tests { .unwrap(); static ref HASH_FIRST_MINUS_500: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 500)); static ref HASH_FIRST_MINUS_450: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 450)); + static ref HASH_FIRST_MINUS_ONE: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() - 1)); static ref HASH_FIRST_PLUS_ONE: H256 = H256::from_uint(&((*HASH_FIRST).into_uint() + 1)); } @@ -231,6 +232,71 @@ mod tests { assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); } + #[test] + fn hive_account_range_i() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST, + limit_hash: *HASH_MAX, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 86); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") + .unwrap() + ); + } + + #[test] + fn hive_account_range_j() { + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST_PLUS_ONE, + limit_hash: *HASH_MAX, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 86); + assert_eq!(res.accounts.first().unwrap().0, *HASH_SECOND); + assert_eq!( + res.accounts.last().unwrap().0, + H256::from_str("0x4615e5f5df5b25349a00ad313c6cd0436b6c08ee5826e33a018661997f85ebaa") + .unwrap() + ); + } + + // Tests for different roots skipped (we don't have other state's data loaded) + + // Non-sensical requests + + #[test] + fn hive_account_range_k() { + // In this test, the startingHash is the first available key, and limitHash is + // a key before startingHash (wrong order). The server should return the first available key. + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST, + limit_hash: *HASH_FIRST_MINUS_ONE, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + // Initial state setup for hive snap tests fn setup_initial_state() -> (Store, H256) { From 7cee2b70a09080d992688bef1723b69d53b70bba Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:12:24 -0300 Subject: [PATCH 019/345] Add test --- crates/networking/p2p/snap.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 0406ed9ee..f6458a452 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -297,6 +297,25 @@ mod tests { assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); } + #[test] + fn hive_account_range_m() { + // In this test, the startingHash is the first available key and limitHash is zero. + // (wrong order). The server should return the first available key. + let (store, root) = setup_initial_state(); + let request = GetAccountRange { + id: 0, + root_hash: root, + starting_hash: *HASH_FIRST, + limit_hash: *HASH_MIN, + response_bytes: 4000, + }; + let res = process_account_range_request(request, store).unwrap(); + // Check test invariants + assert_eq!(res.accounts.len(), 1); + assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + } + // Initial state setup for hive snap tests fn setup_initial_state() -> (Store, H256) { From 579309f0fb2297e4547311263f7d00131d0365df Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 23 Oct 2024 18:21:41 -0300 Subject: [PATCH 020/345] Also fetch limit proof --- crates/networking/p2p/snap.rs | 6 +++++- crates/storage/store/storage.rs | 5 ++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index f6458a452..3ba9fbaed 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -28,7 +28,11 @@ pub fn process_account_range_request( break; } } - let proof = store.get_account_range_proof(request.root_hash, request.starting_hash)?; + let proof = store.get_account_range_proof( + request.root_hash, + request.starting_hash, + request.limit_hash, + )?; Ok(AccountRange { id: request.id, diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 82a88a3c9..145f00b74 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -696,9 +696,12 @@ impl Store { &self, state_root: H256, starting_hash: H256, + limit_hash: H256, ) -> Result>, StoreError> { let state_trie = self.engine.open_state_trie(state_root); - Ok(state_trie.get_proof(&starting_hash.encode_to_vec())?) + let mut proof = state_trie.get_proof(&starting_hash.encode_to_vec())?; + proof.extend_from_slice(&state_trie.get_proof(&limit_hash.encode_to_vec())?); + Ok(proof) } pub fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError> { From 9fccb078bffda631abc702fced7b600e1119a35f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 10:40:11 -0300 Subject: [PATCH 021/345] Trim test state 408 -> 100 --- crates/networking/p2p/snap.rs | 1590 +-------------------------------- 1 file changed, 4 insertions(+), 1586 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 3ba9fbaed..2816c998d 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -324,7 +324,10 @@ mod tests { fn setup_initial_state() -> (Store, H256) { // We cannot process the old blocks that hive uses for the devp2p snap tests - // So I took the state from a geth execution to run them locally + // So I copied the state from a geth execution of the test suite + + // State was trimmed to only the first 100 accounts (as the furthest account used by the tests is account 87) + // If the full 408 account state is needed check out previous commits the PR that added this code let accounts: Vec<(&str, Vec)> = vec![ ( @@ -850,1591 +853,6 @@ mod tests { "0x4ceaf2371fcfb54a4d8bc1c804d90b06b3c32c9f17112b57c29b30a25cf8ca12", vec![196, 128, 1, 128, 128], ), - ( - "0x4d67d989fdb264fa4b2524d306f7b3f70ddce0b723411581d1740407da325462", - vec![196, 128, 1, 128, 128], - ), - ( - "0x4d79fea6c7fef10cb0b5a8b3d85b66836a131bec0b04d891864e6fdb9794af75", - vec![196, 1, 128, 128, 128], - ), - ( - "0x4e0ab2902f57bf2a250c0f87f088acc325d55f2320f2e33abd8e50ba273c9244", - vec![ - 228, 1, 128, 160, 193, 104, 96, 69, 40, 138, 89, 82, 173, 87, 222, 14, 151, 27, - 210, 80, 7, 114, 60, 159, 116, 159, 73, 243, 145, 231, 21, 194, 123, 245, 38, - 200, 128, - ], - ), - ( - "0x4e258aa445a0e2a8704cbc57bbe32b859a502cd6f99190162236300fabd86c4a", - vec![196, 1, 128, 128, 128], - ), - ( - "0x4e5bab4ebd077c3bbd8239995455989ea2e95427ddeed47d0618d9773332bb05", - vec![196, 128, 1, 128, 128], - ), - ( - "0x4f458f480644b18c0e8207f405b82da7f75c7b3b5a34fe6771a0ecf644677f33", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x4fbc5fc8df4f0a578c3be3549f1cb3ef135cbcdf75f620c7a1d412462e9b3b94", - vec![196, 1, 128, 128, 128], - ), - ( - "0x4fd7c8d583447b937576211163a542d945ac8c0a6e22d0c42ac54e2cbaff9281", - vec![196, 128, 1, 128, 128], - ), - ( - "0x50d83ef5194d06752cd5594b57e809b135f24eedd124a51137feaaf049bc2efd", - vec![ - 228, 1, 128, 160, 81, 184, 41, 240, 242, 195, 222, 156, 251, 217, 78, 71, 130, - 138, 137, 148, 12, 50, 154, 73, 205, 89, 84, 12, 163, 198, 215, 81, 168, 210, - 20, 214, 128, - ], - ), - ( - "0x5162f18d40405c59ef279ad71d87fbec2bbfedc57139d56986fbf47daf8bcbf2", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x517bd5fbe28e4368b0b9fcba13d5e81fb51babdf4ed63bd83885235ee67a8fa0", - vec![ - 228, 1, 128, 160, 116, 237, 120, 235, 22, 1, 109, 127, 243, 161, 115, 171, 27, - 188, 238, 157, 170, 142, 53, 138, 157, 108, 155, 229, 232, 75, 166, 244, 163, - 76, 249, 106, 128, - ], - ), - ( - "0x519abb269c3c5710f1979ca84192e020ba5c838bdd267b2d07436a187f171232", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x5264e880ecf7b80afda6cc2a151bac470601ff8e376af91aaf913a36a30c4009", - vec![196, 128, 1, 128, 128], - ), - ( - "0x52d034ca6ebd21c7ba62a2ad3b6359aa4a1cdc88bdaa64bb2271d898777293ab", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x5380c7b7ae81a58eb98d9c78de4a1fd7fd9535fc953ed2be602daaa41767312a", - vec![ - 205, 128, 137, 12, 167, 152, 153, 113, 244, 250, 99, 97, 128, 128, - ], - ), - ( - "0x54c12444ede3e2567dd7f4d9a06d4db8c6ab800d5b3863f8ff22a0db6d09bf24", - vec![196, 128, 1, 128, 128], - ), - ( - "0x55cab9586acb40e66f66147ff3a059cfcbbad785dddd5c0cc31cb43edf98a5d5", - vec![196, 1, 128, 128, 128], - ), - ( - "0x55d0609468d8d4147a942e88cfc5f667daff850788d821889fbb03298924767c", - vec![196, 128, 1, 128, 128], - ), - ( - "0x5602444769b5fd1ddfca48e3c38f2ecad326fe2433f22b90f6566a38496bd426", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x5677600b2af87d21fdab2ac8ed39bd1be2f790c04600de0400c1989040d9879c", - vec![196, 128, 1, 128, 128], - ), - ( - "0x570210539713235b442bbbad50c58bee81b70efd2dad78f99e41a6c462faeb43", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x580aa878e2f92d113a12c0a3ce3c21972b03dbe80786858d49a72097e2c491a3", - vec![ - 228, 1, 128, 160, 71, 27, 248, 152, 138, 208, 215, 96, 45, 107, 213, 73, 60, 8, - 115, 48, 150, 193, 22, 172, 120, 139, 118, 242, 42, 104, 43, 196, 85, 142, 58, - 167, 128, - ], - ), - ( - "0x58e416a0dd96454bd2b1fe3138c3642f5dee52e011305c5c3416d97bc8ba5cf0", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x59312f89c13e9e24c1cb8b103aa39a9b2800348d97a92c2c9e2a78fa02b70025", - vec![196, 1, 128, 128, 128], - ), - ( - "0x5a356862c79afffd6a01af752d950e11490146e4d86dfb8ab1531e9aef4945a1", - vec![ - 228, 1, 128, 160, 58, 41, 133, 198, 173, 166, 126, 86, 4, 185, 159, 162, 252, - 26, 48, 42, 189, 13, 194, 65, 238, 127, 20, 196, 40, 250, 103, 212, 118, 134, - 139, 182, 128, - ], - ), - ( - "0x5a4a3feecfc77b402e938e28df0c4cbb874771cb3c5a92524f303cffb82a2862", - vec![196, 128, 1, 128, 128], - ), - ( - "0x5aa3b4a2ebdd402721c3953b724f4fe90900250bb4ef89ce417ec440da318cd6", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x5b90bb05df9514b2d8e3a8feb3d6c8c22526b02398f289b42111426edc4fe6cf", - vec![ - 228, 1, 128, 160, 40, 122, 204, 120, 105, 66, 31, 185, 244, 154, 53, 73, 185, - 2, 251, 1, 183, 172, 204, 3, 34, 67, 189, 126, 26, 204, 216, 150, 93, 149, 217, - 21, 128, - ], - ), - ( - "0x5c1d92594d6377fe6423257781b382f94dffcde4fadbf571aa328f6eb18f8fcd", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x5c20f6ee05edbb60beeab752d87412b2f6e12c8feefa2079e6bd989f814ed4da", - vec![196, 1, 128, 128, 128], - ), - ( - "0x5d97d758e8800d37b6d452a1b1812d0afedba11f3411a17a8d51ee13a38d73f0", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x5e88e876a3af177e6daafe173b67f186a53f1771a663747f26b278c5acb4c219", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x5ec55391e89ac4c3cf9e61801cd13609e8757ab6ed08687237b789f666ea781b", - vec![ - 228, 1, 128, 160, 199, 191, 43, 52, 41, 64, 101, 175, 185, 162, 193, 95, 144, - 108, 186, 31, 122, 26, 159, 13, 163, 78, 169, 196, 102, 3, 181, 44, 174, 144, - 40, 236, 128, - ], - ), - ( - "0x5fc13d7452287b5a8e3c3be9e4f9057b5c2dd82aeaff4ed892c96fc944ec31e7", - vec![196, 128, 1, 128, 128], - ), - ( - "0x5fcd9b6fce3394ad1d44733056b3e5f6306240974a16f9de8e96ebdd14ae06b1", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x600a7a5f41a67f6f759dcb664198f1c5d9b657fb51a870ce9e234e686dff008e", - vec![ - 228, 1, 128, 160, 158, 218, 142, 182, 202, 3, 215, 196, 175, 228, 114, 121, - 172, 201, 10, 69, 209, 178, 202, 106, 17, 175, 217, 82, 6, 248, 134, 141, 32, - 82, 13, 6, 128, - ], - ), - ( - "0x60535eeb3ffb721c1688b879368c61a54e13f8881bdef6bd4a17b8b92e050e06", - vec![ - 228, 1, 128, 160, 251, 121, 2, 30, 127, 165, 75, 155, 210, 223, 100, 246, 219, - 87, 137, 125, 82, 174, 133, 247, 193, 149, 175, 81, 141, 228, 130, 0, 161, 50, - 94, 44, 128, - ], - ), - ( - "0x606059a65065e5f41347f38754e6ddb99b2d709fbff259343d399a4f9832b48f", - vec![ - 228, 1, 128, 160, 191, 186, 27, 194, 172, 66, 101, 95, 90, 151, 69, 11, 230, - 43, 148, 48, 130, 34, 50, 241, 206, 73, 152, 234, 245, 35, 155, 12, 36, 59, 43, - 132, 128, - ], - ), - ( - "0x61088707d2910974000e63c2d1a376f4480ba19dde19c4e6a757aeb3d62d5439", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x6188c4510d25576535a642b15b1dbdb8922fe572b099f504390f923c19799777", - vec![196, 128, 1, 128, 128], - ), - ( - "0x6225e8f52719d564e8217b5f5260b1d1aac2bcb959e54bc60c5f479116c321b8", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x625e5c85d5f4b6385574b572709d0f704b097527a251b7c658c0c4441aef2af6", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x64bfba8a4688bdee41c4b998e101567b8b56fea53d30ab85393f2d5b70c5da90", - vec![196, 1, 128, 128, 128], - ), - ( - "0x64d0de66ea29cbcf7f237dae1c5f883fa6ff0ba52b90f696bb0348224dbc82ce", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x65cf42efacdee07ed87a1c2de0752a4e3b959f33f9f9f8c77424ba759e01fcf2", - vec![196, 1, 128, 128, 128], - ), - ( - "0x65e6b6521e4f1f97e80710581f42063392c9b33e0aeea4081a102a32238992ea", - vec![ - 228, 1, 128, 160, 17, 212, 238, 199, 223, 82, 205, 84, 231, 70, 144, 164, 135, - 136, 78, 86, 55, 25, 118, 194, 184, 196, 159, 252, 76, 143, 52, 131, 17, 102, - 191, 78, 128, - ], - ), - ( - "0x662d147a16d7c23a2ba6d3940133e65044a90985e26207501bfca9ae47a2468c", - vec![196, 128, 1, 128, 128], - ), - ( - "0x6641e3ed1f264cf275b53bb7012dabecf4c1fca700e3db989e314c24cc167074", - vec![ - 228, 1, 128, 160, 15, 216, 233, 155, 27, 74, 180, 235, 140, 108, 34, 24, 34, - 26, 230, 151, 140, 198, 116, 51, 52, 30, 216, 161, 173, 97, 133, 211, 79, 168, - 44, 97, 128, - ], - ), - ( - "0x67cc0bf5341efbb7c8e1bdbf83d812b72170e6edec0263eeebdea6f107bbef0d", - vec![ - 228, 1, 128, 160, 162, 14, 106, 33, 36, 74, 248, 255, 204, 213, 68, 34, 151, - 173, 155, 122, 118, 172, 114, 215, 216, 172, 158, 22, 241, 47, 204, 80, 233, - 11, 115, 78, 128, - ], - ), - ( - "0x68fc814efedf52ac8032da358ddcb61eab4138cb56b536884b86e229c995689c", - vec![ - 228, 1, 128, 160, 109, 43, 138, 7, 76, 120, 160, 229, 168, 9, 93, 122, 1, 13, - 73, 97, 198, 57, 197, 65, 207, 86, 251, 183, 4, 148, 128, 204, 143, 25, 151, - 101, 128, - ], - ), - ( - "0x6a2c8498657ae4f0f7b1a02492c554f7f8a077e454550727890188f7423ba014", - vec![ - 228, 1, 128, 160, 86, 34, 128, 27, 16, 17, 222, 132, 3, 228, 67, 8, 187, 248, - 154, 88, 9, 183, 173, 101, 134, 38, 140, 215, 33, 100, 82, 53, 135, 249, 176, - 228, 128, - ], - ), - ( - "0x6a5e43139d88da6cfba857e458ae0b5359c3fde36e362b6e5f782a90ce351f14", - vec![196, 1, 128, 128, 128], - ), - ( - "0x6ad3ba011e031431dc057c808b85346d58001b85b32a4b5c90ccccea0f82e170", - vec![ - 228, 1, 128, 160, 20, 249, 244, 185, 68, 92, 117, 71, 213, 164, 103, 26, 56, - 176, 177, 43, 188, 14, 113, 152, 195, 178, 147, 75, 130, 182, 149, 200, 99, 13, - 73, 114, 128, - ], - ), - ( - "0x6bd9fb206b22c76b4f9630248940855b842c684db89adff0eb9371846ea625a9", - vec![196, 1, 128, 128, 128], - ), - ( - "0x6c05d8abc81143ce7c7568c98aadfe6561635c049c07b2b4bce3019cef328cb9", - vec![196, 128, 1, 128, 128], - ), - ( - "0x6c37093a34016ae687da7aabb18e42009b71edff70a94733c904aea51a4853c1", - vec![196, 1, 128, 128, 128], - ), - ( - "0x6d1da4cf1127d654ed731a93105f481b315ecfc2f62b1ccb5f6d2717d6a40f9b", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x6d4162ce16817e46fa2ddc5e70cee790b80abc3d6f7778cfbaed327c5d2af36c", - vec![196, 1, 128, 128, 128], - ), - ( - "0x6dbe5551f50400859d14228606bf221beff07238bfa3866454304abb572f9512", - vec![196, 128, 1, 128, 128], - ), - ( - "0x6dc09fdec00aa9a30dd8db984406a33e3ca15e35222a74773071207a5e56d2c2", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x6f358b4e903d31fdd5c05cddaa174296bb30b6b2f72f1ff6410e6c1069198989", - vec![196, 128, 1, 128, 128], - ), - ( - "0x7026c939a9158beedff127a64f07a98b328c3d1770690437afdb21c34560fc57", - vec![196, 128, 1, 128, 128], - ), - ( - "0x70aae390a762a4347a4d167a2431874554edf1d77579213e55fea3ec39a1257c", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x71dee9adfef0940a36336903bd6830964865180b98c0506f9bf7ba8f2740fbf9", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x720f25b62fc39426f70eb219c9dd481c1621821c8c0fa5367a1df6e59e3edf59", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x728325587fa336e318b54298e1701d246c4f90d6094eb95635d8a47f080f4603", - vec![196, 128, 1, 128, 128], - ), - ( - "0x729953a43ed6c913df957172680a17e5735143ad767bda8f58ac84ec62fbec5e", - vec![196, 1, 128, 128, 128], - ), - ( - "0x72d91596112f9d7e61d09ffa7575f3587ad9636172ae09641882761cc369ecc0", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x72e962dfe7e2828809f5906996dedeba50950140555b193fceb94f12fd6f0a22", - vec![196, 1, 128, 128, 128], - ), - ( - "0x734ee4981754a3f1403c4e8887d35addfb31717d93de3e00ede78368c230861e", - vec![ - 228, 1, 128, 160, 44, 242, 146, 193, 227, 130, 189, 208, 231, 46, 18, 103, 1, - 215, 176, 36, 132, 230, 226, 114, 244, 192, 216, 20, 245, 166, 250, 226, 51, - 252, 121, 53, 128, - ], - ), - ( - "0x73cd1b7cd355f3f77c570a01100a616757408bb7abb78fe9ee1262b99688fcc4", - vec![196, 1, 128, 128, 128], - ), - ( - "0x74614a0c4ba7d7c70b162dad186b6cc77984ab4070534ad9757e04a5b776dcc8", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x7583557e4e3918c95965fb610dc1424976c0eee606151b6dfc13640e69e5cb15", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x75d231f57a1a9751f58769d5691f4807ab31ac0e802b1a1f6bfc77f5dff0adbf", - vec![ - 228, 1, 128, 160, 205, 49, 237, 93, 93, 167, 153, 144, 175, 237, 13, 153, 60, - 183, 37, 196, 227, 77, 217, 117, 68, 176, 52, 102, 237, 52, 33, 46, 66, 194, - 141, 104, 128, - ], - ), - ( - "0x78948842ff476b87544c189ce744d4d924ffd0907107a0dbaa4b71d0514f2225", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x792cc9f20a61c16646d5b6136693e7789549adb7d8e35503d0004130ea6528b0", - vec![ - 228, 1, 128, 160, 154, 74, 51, 249, 120, 216, 78, 10, 206, 179, 172, 54, 112, - 194, 226, 223, 108, 138, 226, 124, 24, 154, 150, 237, 0, 184, 6, 209, 14, 215, - 180, 238, 128, - ], - ), - ( - "0x7963685967117ffb6fd019663dc9e782ebb1234a38501bffc2eb5380f8dc303b", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x79afb7a5ffe6ccd537f9adff8287b78f75c37d97ea8a4dd504a08bc09926c3fa", - vec![196, 128, 1, 128, 128], - ), - ( - "0x7a08bb8417e6b18da3ba926568f1022c15553b2b0f1a32f2fd9e5a605469e54f", - vec![ - 228, 1, 128, 160, 56, 91, 132, 210, 112, 89, 163, 199, 142, 126, 166, 58, 105, - 30, 235, 156, 83, 118, 247, 122, 241, 19, 54, 118, 47, 140, 24, 136, 47, 247, - 71, 26, 128, - ], - ), - ( - "0x7a2464bc24d90557940e93a3b73308ea354ed7d988be720c545974a17959f93f", - vec![196, 1, 128, 128, 128], - ), - ( - "0x7a3870cc1ed4fc29e9ab4dd3218dbb239dd32c9bf05bff03e325b7ba68486c47", - vec![196, 1, 128, 128, 128], - ), - ( - "0x7bac5af423cb5e417fa6c103c7cb9777e80660ce3735ca830c238b0d41610186", - vec![196, 128, 1, 128, 128], - ), - ( - "0x7bff1b6b56891e66584ced453d09450c2fed9453b1644e8509bef9f9dd081bbb", - vec![196, 1, 128, 128, 128], - ), - ( - "0x7c1edabb98857d64572f03c64ac803e4a14b1698fccffffd51675d99ee3ba217", - vec![ - 228, 1, 128, 160, 97, 23, 109, 188, 5, 168, 83, 125, 141, 232, 95, 130, 160, - 59, 142, 16, 73, 206, 167, 173, 10, 159, 14, 91, 96, 238, 21, 252, 166, 254, - 13, 66, 128, - ], - ), - ( - "0x7c3e44534b1398abc786e4591364c329e976dbde3b3ed3a4d55589de84bcb9a6", - vec![196, 128, 1, 128, 128], - ), - ( - "0x7c463797c90e9ba42b45ae061ffaa6bbd0dad48bb4998f761e81859f2a904a49", - vec![196, 1, 128, 128, 128], - ), - ( - "0x7c48e400de1f24b4de94c59068fcd91a028576d13a22f900a7fcbd8f4845bcf4", - vec![196, 1, 128, 128, 128], - ), - ( - "0x7c608293e741d1eb5ae6916c249a87b6540cf0c2369e96d293b1a7b5b9bd8b31", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x7e1ef9f8d2fa6d4f8e6717c3dcccff352ea9b8b46b57f6106cdbeed109441799", - vec![196, 128, 1, 128, 128], - ), - ( - "0x7e839d9fd8a767e90a8b2f48a571f111dd2451bc5910cf2bf3ae79963e47e34d", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x7f9726a7b2f5f3a501b2d7b18ec726f25f22c86348fae0f459d882ec5fd7d0c7", - vec![196, 128, 1, 128, 128], - ), - ( - "0x80a2c1f38f8e2721079a0de39f187adedcb81b2ab5ae718ec1b8d64e4aa6930e", - vec![ - 228, 1, 128, 160, 45, 168, 110, 179, 212, 255, 221, 137, 81, 112, 188, 126, - 240, 43, 105, 161, 22, 254, 33, 172, 44, 228, 90, 62, 216, 224, 187, 138, 241, - 124, 249, 43, 128, - ], - ), - ( - "0x80cd4a7b601d4ba0cb09e527a246c2b5dd25b6dbf862ac4e87c6b189bfce82d7", - vec![196, 1, 128, 128, 128], - ), - ( - "0x81c0c51e15c9679ef12d02729c09db84220ba007efe7ced37a57132f6f0e83c9", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x84c7ee50e102d0abf5750e781c1635d60346f20ab0d5e5f9830db1a592c658ff", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x8510660ad5e3d35a30d4fb7c2615c040f9f698faae2ac48022e366deaeecbe77", - vec![ - 228, 1, 128, 160, 39, 233, 182, 165, 76, 240, 251, 24, 132, 153, 197, 8, 189, - 150, 212, 80, 148, 108, 214, 186, 28, 247, 108, 245, 52, 59, 92, 116, 69, 15, - 102, 144, 128, - ], - ), - ( - "0x8678559b30b321b0f0420a4a3e8cecfde90c6e56766b78c1723062c93c1f041f", - vec![196, 1, 128, 128, 128], - ), - ( - "0x867bc89cf8d5b39f1712fbc77414bbd93012af454c226dcee0fb34ccc0017498", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x86a73e3c668eb065ecac3402c6dc912e8eb886788ea147c770f119dcd30780c6", - vec![ - 228, 1, 128, 160, 165, 169, 28, 249, 232, 21, 251, 85, 223, 20, 179, 238, 140, - 19, 37, 169, 136, 203, 59, 109, 211, 71, 150, 201, 1, 56, 92, 60, 194, 153, 32, - 115, 128, - ], - ), - ( - "0x86d03d0f6bed220d046a4712ec4f451583b276df1aed33f96495d22569dc3485", - vec![ - 228, 1, 128, 160, 226, 161, 100, 226, 195, 12, 243, 3, 145, 200, 143, 243, 42, - 14, 32, 33, 148, 176, 143, 42, 97, 169, 205, 41, 39, 234, 94, 214, 223, 191, - 16, 86, 128, - ], - ), - ( - "0x873429def7829ff8227e4ef554591291907892fc8f3a1a0667dada3dc2a3eb84", - vec![ - 228, 1, 128, 160, 84, 171, 205, 188, 139, 4, 188, 155, 112, 233, 189, 70, 203, - 157, 185, 184, 235, 8, 207, 212, 173, 219, 164, 201, 65, 218, 204, 52, 221, 40, - 100, 142, 128, - ], - ), - ( - "0x878040f46b1b4a065e6b82abd35421eb69eededc0c9598b82e3587ae47c8a651", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x87e33f70e1dd3c6ff68e3b71757d697fbeb20daae7a3cc8a7b1b3aa894592c50", - vec![196, 128, 1, 128, 128], - ), - ( - "0x88a5635dabc83e4e021167be484b62cbed0ecdaa9ac282dab2cd9405e97ed602", - vec![ - 228, 1, 128, 160, 137, 189, 232, 157, 247, 242, 216, 51, 68, 165, 3, 148, 75, - 179, 71, 184, 71, 242, 8, 223, 131, 114, 40, 187, 44, 223, 214, 195, 34, 140, - 163, 223, 128, - ], - ), - ( - "0x88bf4121c2d189670cb4d0a16e68bdf06246034fd0a59d0d46fb5cec0209831e", - vec![ - 228, 1, 128, 160, 89, 115, 155, 163, 177, 86, 235, 120, 248, 187, 177, 75, 191, - 61, 172, 222, 191, 222, 149, 20, 15, 88, 109, 182, 111, 114, 227, 17, 123, 148, - 187, 103, 128, - ], - ), - ( - "0x8989651e80c20af78b37fdb693d74ecafc9239426ff1315e1fb7b674dcdbdb75", - vec![196, 128, 1, 128, 128], - ), - ( - "0x8a8266874b43f78d4097f27b2842132faed7e7e430469eec7354541eb97c3ea0", - vec![196, 1, 128, 128, 128], - ), - ( - "0x8b76305d3f00d33f77bd41496b4144fd3d113a2ec032983bd5830a8b73f61cf0", - vec![196, 128, 1, 128, 128], - ), - ( - "0x8c7bfaa19ea367dec5272872114c46802724a27d9b67ea3eed85431df664664e", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x8e11480987056c309d7064ebbd887f086d815353cdbaadb796891ed25f8dcf61", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0x8ee17a1ec4bae15d8650323b996c55d5fa11a14ceec17ff1d77d725183904914", - vec![196, 128, 1, 128, 128], - ), - ( - "0x903f24b3d3d45bc50c082b2e71c7339c7060f633f868db2065ef611885abe37e", - vec![196, 1, 128, 128, 128], - ), - ( - "0x910fb8b22867289cb57531ad39070ef8dbdbbe7aee941886a0e9f572b63ae9ee", - vec![ - 228, 1, 128, 160, 115, 191, 252, 104, 169, 71, 250, 25, 183, 190, 205, 69, 102, - 29, 34, 200, 112, 250, 200, 219, 242, 178, 87, 3, 225, 189, 171, 83, 103, 242, - 149, 67, 128, - ], - ), - ( - "0x913e2a02a28d71d595d7216a12311f6921a4caf40aeabf0f28edf937f1df72b4", - vec![196, 128, 1, 128, 128], - ), - ( - "0x92b13a73440c6421da22e848d23f9af80610085ab05662437d850c97a012d8d3", - vec![196, 128, 1, 128, 128], - ), - ( - "0x92d0f0954f4ec68bd32163a2bd7bc69f933c7cdbfc6f3d2457e065f841666b1c", - vec![196, 1, 128, 128, 128], - ), - ( - "0x93843d6fa1fe5709a3035573f61cc06832f0377544d16d3a0725e78a0fa0267c", - vec![196, 1, 128, 128, 128], - ), - ( - "0x943f42ad91e8019f75695946d491bb95729f0dfc5dbbb953a7239ac73f208943", - vec![ - 228, 1, 128, 160, 169, 88, 1, 9, 190, 47, 125, 53, 181, 54, 0, 80, 194, 206, - 215, 78, 93, 77, 234, 47, 130, 212, 110, 141, 38, 110, 216, 145, 87, 99, 96, 4, - 128, - ], - ), - ( - "0x946bfb429d90f1b39bb47ada75376a8d90a5778068027d4b8b8514ac13f53eca", - vec![196, 1, 128, 128, 128], - ), - ( - "0x961508ac3c93b30ee9a5a34a862c9fe1659e570546ac6c2e35da20f6d2bb5393", - vec![ - 228, 1, 128, 160, 217, 26, 207, 48, 89, 52, 166, 12, 150, 10, 147, 251, 0, 249, - 39, 236, 121, 48, 139, 138, 145, 157, 36, 73, 250, 237, 231, 34, 194, 50, 76, - 179, 128, - ], - ), - ( - "0x96c43ef9dce3410b78df97be69e7ccef8ed40d6e5bfe6582ea4cd7d577aa4569", - vec![ - 228, 1, 128, 160, 90, 130, 175, 241, 38, 255, 235, 255, 118, 0, 43, 30, 77, - 224, 60, 64, 186, 73, 75, 129, 203, 63, 188, 82, 143, 35, 228, 190, 53, 169, - 175, 230, 128, - ], - ), - ( - "0x96d7104053877823b058fd9248e0bba2a540328e52ffad9bb18805e89ff579dc", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x974a4800ec4c0e998f581c6ee8c3972530989e97a179c6b2d40b8710c036e7b1", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x97b25febb46f44607c87a3498088c605086df207c7ddcd8ee718836a516a9153", - vec![196, 1, 128, 128, 128], - ), - ( - "0x97f72ff641eb40ee1f1163544931635acb7550a0d44bfb9f4cc3aeae829b6d7d", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x98bb9ba48fda7bb8091271ab0e53d7e0022fb1f1fa8fa00814e193c7d4b91eb3", - vec![196, 128, 1, 128, 128], - ), - ( - "0x9966a8b4cd856b175855258fa7e412ffef06d9e92b519050fa7ac06d8952ac84", - vec![196, 128, 1, 128, 128], - ), - ( - "0x99ce1680f73f2adfa8e6bed135baa3360e3d17f185521918f9341fc236526321", - vec![196, 1, 128, 128, 128], - ), - ( - "0x99dba7e9230d5151cc37ff592fa1592f27c7c81d203760dfaf62ddc9f3a6b8fd", - vec![196, 128, 1, 128, 128], - ), - ( - "0x99e56541f21039c9b7c63655333841a3415de0d27b79d18ade9ec7ecde7a1139", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x9a1896e612ca43ecb7601990af0c3bc135b9012c50d132769dfb75d0038cc3be", - vec![196, 128, 1, 128, 128], - ), - ( - "0x9d42947ac5e61285567f65d4b400d90343dbd3192534c4c1f9d941c04f48f17c", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x9de451c4f48bdb56c6df198ff8e1f5e349a84a4dc11de924707718e6ac897aa6", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0x9fe8b6e43098a4df56e206d479c06480801485dfd8ec3da4ccc3cebf5fba89a1", - vec![196, 128, 1, 128, 128], - ), - ( - "0x9feaf0bd45df0fbf327c964c243b2fbc2f0a3cb48fedfeea1ae87ac1e66bc02f", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xa02abeb418f26179beafd96457bda8c690c6b1f3fbabac392d0920863edddbc6", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xa02c8b02efb52fad3056fc96029467937c38c96d922250f6d2c0f77b923c85aa", - vec![196, 1, 128, 128, 128], - ), - ( - "0xa03fe040e4264070290e95ffe06bf9da0006556091f17c5df5abaa041de0c2f7", - vec![196, 1, 128, 128, 128], - ), - ( - "0xa0f5dc2d18608f8e522ffffd86828e3d792b36d924d5505c614383ddff9be2eb", - vec![ - 228, 1, 128, 160, 42, 254, 147, 225, 176, 242, 110, 88, 141, 40, 9, 18, 126, - 67, 96, 173, 126, 40, 207, 85, 36, 152, 178, 188, 72, 71, 214, 188, 218, 115, - 140, 219, 128, - ], - ), - ( - "0xa13bfef92e05edee891599aa5e447ff2baa1708d9a6473a04ef66ab94f2a11e4", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xa15773c9bfabef49e9825460ed95bf67b22b67d7806c840e0eb546d73c424768", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xa248850a2e0d6fe62259d33fc498203389fa754c3bd098163e86946888e455bd", - vec![196, 128, 1, 128, 128], - ), - ( - "0xa3abdaefbb886078dc6c5c72e4bc8d12e117dbbd588236c3fa7e0c69420eb24a", - vec![196, 1, 128, 128, 128], - ), - ( - "0xa3d8baf7ae7c96b1020753d12154e28cc7206402037c28c49c332a08cf7c4b51", - vec![196, 1, 128, 128, 128], - ), - ( - "0xa5541b637a896d30688a80b7affda987d9597aac7ccd9799c15999a1d7d094e2", - vec![ - 228, 1, 128, 160, 243, 144, 38, 74, 202, 241, 67, 60, 14, 166, 112, 178, 192, - 148, 163, 0, 118, 100, 20, 105, 82, 74, 226, 79, 95, 221, 196, 78, 153, 197, - 176, 50, 128, - ], - ), - ( - "0xa601eb611972ca80636bc39087a1dae7be5a189b94bda392f84d6ce0d3c866b9", - vec![ - 228, 1, 128, 160, 156, 50, 255, 213, 5, 145, 21, 187, 169, 174, 217, 23, 79, - 90, 184, 180, 53, 46, 63, 81, 168, 93, 222, 51, 0, 15, 112, 60, 155, 159, 231, - 194, 128, - ], - ), - ( - "0xa683478d0c949580d5738b490fac8129275bb6e921dfe5eae37292be3ee281b9", - vec![ - 228, 1, 128, 160, 193, 91, 67, 229, 244, 133, 62, 200, 218, 83, 235, 222, 3, - 222, 135, 185, 74, 252, 228, 42, 156, 2, 246, 72, 173, 139, 219, 34, 70, 4, - 196, 173, 128, - ], - ), - ( - "0xa87387b50b481431c6ccdb9ae99a54d4dcdd4a3eff75d7b17b4818f7bbfc21e9", - vec![ - 228, 1, 128, 160, 226, 167, 47, 91, 251, 235, 167, 15, 201, 171, 80, 98, 55, - 186, 39, 192, 150, 164, 233, 108, 57, 104, 202, 191, 91, 27, 47, 181, 68, 49, - 181, 207, 128, - ], - ), - ( - "0xa9233a729f0468c9c309c48b82934c99ba1fd18447947b3bc0621adb7a5fc643", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xa95c88d7dc0f2373287c3b2407ba8e7419063833c424b06d8bb3b29181bb632e", - vec![196, 128, 128, 128, 128], - ), - ( - "0xa9656c0192bb27f0ef3f93ecc6cc990dd146da97ac11f3d8d0899fba68d5749a", - vec![ - 228, 1, 128, 160, 114, 23, 203, 116, 112, 84, 48, 111, 130, 110, 120, 170, 63, - 198, 143, 228, 68, 18, 153, 163, 55, 236, 234, 29, 98, 88, 47, 45, 168, 167, - 243, 54, 128, - ], - ), - ( - "0xa9970b3744a0e46b248aaf080a001441d24175b5534ad80755661d271b976d67", - vec![ - 228, 1, 128, 160, 18, 222, 69, 68, 100, 15, 200, 160, 39, 225, 169, 18, 215, - 118, 185, 6, 117, 190, 191, 213, 7, 16, 194, 135, 107, 42, 36, 236, 158, 206, - 211, 103, 128, - ], - ), - ( - "0xa9de128e7d4347403eb97f45e969cd1882dfe22c1abe8857aab3af6d0f9e9b92", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xa9fd2e3a6de5a9da5badd719bd6e048acefa6d29399d8a99e19fd9626805b60b", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xaa0ac2f707a3dc131374839d4ee969eeb1cb55adea878f56e7b5b83d187d925c", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xaa0ffaa57269b865dccce764bf412de1dff3e7bba22ce319ef09e5907317b3e7", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xaad7b91d085a94c11a2f7e77dc95cfcfc5daf4f509ca4e0c0e493b86c6cbff78", - vec![ - 228, 1, 128, 160, 160, 144, 182, 111, 188, 164, 108, 183, 26, 189, 29, 170, - 141, 65, 157, 44, 110, 41, 16, 148, 245, 40, 114, 151, 141, 252, 177, 195, 26, - 215, 169, 0, 128, - ], - ), - ( - "0xab7bdc41a80ae9c8fcb9426ba716d8d47e523f94ffb4b9823512d259c9eca8cd", - vec![196, 1, 128, 128, 128], - ), - ( - "0xabd8afe9fbf5eaa36c506d7c8a2d48a35d013472f8182816be9c833be35e50da", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xabdc44a9bc7ccf1ce76b942d25cd9d731425cd04989597d7a2e36423e2dac7ee", - vec![196, 128, 1, 128, 128], - ), - ( - "0xac7183ebb421005a660509b070d3d47fc4e134cb7379c31dc35dc03ebd02e1cf", - vec![196, 1, 128, 128, 128], - ), - ( - "0xad6a4a6ebd5166c9b5cc8cfbaec176cced40fa88c73d83c67f0c3ed426121ebc", - vec![196, 128, 1, 128, 128], - ), - ( - "0xad99b5bc38016547d5859f96be59bf18f994314116454def33ebfe9a892c508a", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xae88076d02b19c4d09cb13fca14303687417b632444f3e30fc4880c225867be3", - vec![196, 1, 128, 128, 128], - ), - ( - "0xaeaf19d38b69be4fb41cc89e4888708daa6b9b1c3f519fa28fe9a0da70cd8697", - vec![196, 1, 128, 128, 128], - ), - ( - "0xaef83ad0ab332330a20e88cd3b5a4bcf6ac6c175ee780ed4183d11340df17833", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xaf38e0e6a4a4005507b5d3e9470e8ccc0273b74b6971f768cbdf85abeab8a95b", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xaf7c37d08a73483eff9ef5054477fb5d836a184aa07c3edb4409b9eb22dd56ca", - vec![ - 228, 1, 128, 160, 197, 118, 4, 164, 97, 201, 78, 205, 172, 18, 219, 183, 6, - 165, 43, 50, 145, 61, 114, 37, 59, 175, 251, 137, 6, 231, 66, 114, 74, 225, 36, - 73, 128, - ], - ), - ( - "0xb062c716d86a832649bccd53e9b11c77fc8a2a00ef0cc0dd2f561688a69d54f7", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb17ea61d092bd5d77edd9d5214e9483607689cdcc35a30f7ea49071b3be88c64", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xb1b2c1c59637202bb0e0d21255e44e0df719fe990be05f213b1b813e3d8179d7", - vec![196, 1, 128, 128, 128], - ), - ( - "0xb1b2fd7758f73e25a2f9e72edde82995b2b32ab798bcffd2c7143f2fc8196fd8", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb31919583a759b75e83c14d00d0a89bb36adc452f73cee2933a346ccebaa8e31", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb3a33a7f35ca5d08552516f58e9f76219716f9930a3a11ce9ae5db3e7a81445d", - vec![ - 228, 1, 128, 160, 131, 71, 24, 17, 17, 33, 226, 5, 143, 219, 144, 165, 31, 68, - 128, 40, 7, 24, 87, 225, 31, 189, 85, 212, 50, 86, 23, 77, 245, 106, 240, 26, - 128, - ], - ), - ( - "0xb40cc623b26a22203675787ca05b3be2c2af34b6b565bab95d43e7057e458684", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xb4f179efc346197df9c3a1cb3e95ea743ddde97c27b31ad472d352dba09ee1f5", - vec![196, 1, 128, 128, 128], - ), - ( - "0xb58e22a9ece8f9b3fdbaa7d17fe5fc92345df11d6863db4159647d64a34ff10b", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb58e67c536550fdf7140c8333ca62128df469a7270b16d528bc778909e0ac9a5", - vec![ - 228, 1, 128, 160, 35, 168, 136, 192, 164, 100, 206, 70, 22, 81, 252, 27, 226, - 207, 160, 203, 107, 164, 209, 177, 37, 171, 229, 180, 71, 238, 173, 249, 197, - 173, 241, 241, 128, - ], - ), - ( - "0xb5bca5e9ccef948c2431372315acc3b96e098d0e962b0c99d634a0475b670dc3", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb66092bc3624d84ff94ee42b097e846baf6142197d2c31245734d56a275c8eb9", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb7c2ef96238f635f86f9950700e36368efaaa70e764865dddc43ff6e96f6b346", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xb7d9d175039df1ba52c734547844f8805252893c029f7dbba9a63f8bce3ee306", - vec![196, 128, 1, 128, 128], - ), - ( - "0xb888c9946a84be90a9e77539b5ac68a3c459761950a460f3e671b708bb39c41f", - vec![196, 1, 128, 128, 128], - ), - ( - "0xb8d9b988ed60dbf5dca3e9d169343ca667498605f34fb6c30b45b2ed0f996f1a", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xb91824b28183c95881ada12404d5ee8af8123689a98054d41aaf4dd5bec50e90", - vec![196, 1, 128, 128, 128], - ), - ( - "0xb9400acf38453fd206bc18f67ba04f55b807b20e4efc2157909d91d3a9f7bed2", - vec![196, 1, 128, 128, 128], - ), - ( - "0xb990eaca858ea15fda296f3f47baa2939e8aa8bbccc12ca0c3746d9b5d5fb2ae", - vec![ - 228, 1, 128, 160, 137, 236, 176, 206, 238, 162, 12, 205, 125, 27, 24, 207, 29, - 53, 183, 162, 253, 123, 118, 221, 200, 214, 39, 244, 51, 4, 237, 139, 49, 176, - 18, 72, 128, - ], - ), - ( - "0xb9cddc73dfdacd009e55f27bdfd1cd37eef022ded5ce686ab0ffe890e6bf311e", - vec![ - 228, 1, 128, 160, 61, 32, 254, 221, 39, 11, 55, 113, 112, 111, 224, 10, 88, 10, - 21, 84, 57, 190, 87, 232, 213, 80, 118, 45, 239, 16, 144, 110, 131, 237, 88, - 187, 128, - ], - ), - ( - "0xba1d0afdfee510e8852f24dff964afd824bf36d458cf5f5d45f02f04b7c0b35d", - vec![196, 1, 128, 128, 128], - ), - ( - "0xbaae09901e990935de19456ac6a6c8bc1e339d0b80ca129b8622d989b5c79120", - vec![ - 228, 1, 128, 160, 37, 180, 46, 197, 72, 8, 67, 160, 50, 140, 99, 188, 80, 239, - 248, 89, 93, 144, 241, 209, 176, 175, 202, 178, 244, 161, 155, 136, 140, 121, - 79, 55, 128, - ], - ), - ( - "0xbb861b82d884a70666afeb78bbf30cab7fdccf838f4d5ce5f4e5ca1be6be61b1", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xbbdc59572cc62c338fb6e027ab00c57cdeed233c8732680a56a5747141d20c7c", - vec![196, 1, 128, 128, 128], - ), - ( - "0xbccd3d2f920dfb8d70a38c9ccd5ed68c2ef6e3372199381767ce222f13f36c87", - vec![196, 128, 1, 128, 128], - ), - ( - "0xbccd85b63dba6300f84c561c5f52ce08a240564421e382e6f550ce0c12f2f632", - vec![ - 228, 1, 128, 160, 234, 131, 56, 147, 131, 21, 34, 112, 16, 64, 147, 237, 93, - 254, 52, 186, 64, 60, 117, 48, 129, 51, 170, 27, 232, 245, 26, 216, 4, 179, - 233, 238, 128, - ], - ), - ( - "0xbcebc35bfc663ecd6d4410ee2363e5b7741ee953c7d3359aa585095e503d20c8", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xbe7d987a9265c0e44e9c5736fb2eb38c41973ce96e5e8e6c3c713f9d50a079ff", - vec![ - 228, 1, 128, 160, 175, 213, 78, 129, 243, 228, 21, 64, 127, 8, 18, 166, 120, - 133, 111, 27, 64, 104, 237, 100, 160, 139, 63, 59, 245, 178, 25, 15, 207, 178, - 50, 45, 128, - ], - ), - ( - "0xbea55c1dc9f4a9fb50cbedc70448a4e162792b9502bb28b936c7e0a2fd7fe41d", - vec![ - 228, 1, 128, 160, 49, 10, 42, 200, 61, 126, 62, 77, 51, 49, 2, 177, 247, 21, - 59, 176, 65, 107, 56, 66, 126, 178, 227, 53, 220, 102, 50, 215, 121, 168, 180, - 175, 128, - ], - ), - ( - "0xbf632670b6fa18a8ad174a36180202bfef9a92c2eeda55412460491ae0f6a969", - vec![ - 228, 1, 128, 160, 207, 33, 35, 209, 16, 153, 127, 66, 104, 33, 211, 229, 65, - 51, 78, 67, 253, 214, 181, 40, 108, 60, 51, 37, 44, 36, 181, 248, 170, 252, - 122, 162, 128, - ], - ), - ( - "0xbfaac98225451c56b2f9aec858cffc1eb253909615f3d9617627c793b938694f", - vec![ - 228, 1, 128, 160, 238, 152, 33, 98, 26, 165, 236, 154, 183, 213, 135, 139, 42, - 153, 82, 40, 173, 205, 202, 203, 113, 13, 245, 34, 210, 249, 27, 67, 77, 59, - 220, 121, 128, - ], - ), - ( - "0xbfe5dee42bddd2860a8ebbcdd09f9c52a588ba38659cf5e74b07d20f396e04d4", - vec![196, 1, 128, 128, 128], - ), - ( - "0xbfe731f071443795cef55325f32e6e03c8c0d0398671548dfd5bc96b5a6555c0", - vec![ - 228, 1, 128, 160, 178, 95, 158, 79, 111, 145, 58, 74, 30, 141, 235, 247, 212, - 117, 43, 250, 82, 29, 20, 123, 182, 124, 105, 213, 133, 83, 1, 231, 109, 216, - 6, 51, 128, - ], - ), - ( - "0xc0ce77c6a355e57b89cca643e70450612c0744c9f0f8bf7dee51d6633dc850b1", - vec![ - 228, 1, 128, 160, 223, 60, 27, 250, 184, 247, 231, 10, 142, 223, 148, 121, 47, - 145, 228, 182, 178, 194, 170, 97, 202, 246, 135, 228, 246, 203, 104, 157, 24, - 10, 219, 128, 128, - ], - ), - ( - "0xc13c19f53ce8b6411d6cdaafd8480dfa462ffdf39e2eb68df90181a128d88992", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc157e0d637d64b90e2c59bc8bed2acd75696ea1ac6b633661c12ce8f2bce0d62", - vec![196, 1, 128, 128, 128], - ), - ( - "0xc192ea2d2bb89e9bb7f17f3a282ebe8d1dd672355b5555f516b99b91799b01f6", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xc1a6a0bf60ee7b3228ecf6cb7c9e5491fbf62642a3650d73314e976d9eb9a966", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc2406cbd93e511ef493ac81ebe2b6a3fbecd05a3ba52d82a23a88eeb9d8604f0", - vec![ - 228, 1, 128, 160, 130, 179, 38, 100, 24, 37, 55, 143, 170, 17, 198, 65, 201, - 22, 242, 226, 44, 1, 8, 15, 72, 125, 224, 70, 62, 48, 213, 227, 43, 150, 15, - 151, 128, - ], - ), - ( - "0xc250f30c01f4b7910c2eb8cdcd697cf493f6417bb2ed61d637d625a85a400912", - vec![ - 228, 1, 128, 160, 202, 57, 245, 244, 238, 60, 107, 51, 239, 231, 188, 72, 84, - 57, 249, 127, 157, 198, 47, 101, 133, 44, 122, 28, 223, 84, 250, 177, 227, 183, - 4, 41, 128, - ], - ), - ( - "0xc251a3acb75a90ff0cdca31da1408a27ef7dcaa42f18e648f2be1a28b35eac32", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc2c26fbc0b7893d872fa528d6c235caab9164feb5b54c48381ff3d82c8244e77", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc3791fc487a84f3731eb5a8129a7e26f357089971657813b48a821f5582514b3", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xc3ac56e9e7f2f2c2c089e966d1b83414951586c3afeb86300531dfa350e38929", - vec![ - 228, 1, 128, 160, 129, 142, 175, 90, 219, 86, 198, 114, 136, 137, 186, 102, - 182, 152, 12, 214, 107, 65, 25, 159, 0, 7, 205, 217, 5, 174, 115, 148, 5, 227, - 198, 48, 128, - ], - ), - ( - "0xc3c8e2dc64e67baa83b844263fe31bfe24de17bb72bfed790ab345b97b007816", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xc4bab059ee8f7b36c82ada44d22129671d8f47f254ca6a48fded94a8ff591c88", - vec![196, 1, 128, 128, 128], - ), - ( - "0xc54ffffcbaa5b566a7cf37386c4ce5a338d558612343caaa99788343d516aa5f", - vec![196, 1, 128, 128, 128], - ), - ( - "0xc781c7c3babeb06adfe8f09ecb61dbe0eb671e41f3a1163faac82fdfa2bc83e8", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc7fc033fe9f00d24cb9c479ddc0598e592737c305263d088001d7419d16feffa", - vec![196, 128, 1, 128, 128], - ), - ( - "0xc9ea69dc9e84712b1349c9b271956cc0cb9473106be92d7a937b29e78e7e970e", - vec![196, 1, 128, 128, 128], - ), - ( - "0xca7ad42d3c4fe14ddb81bf27d4679725a1f6c3f23b688681bb6f24262d63212f", - vec![196, 128, 1, 128, 128], - ), - ( - "0xcac96145454c46255fccca35343d9505164dabe319c17d81fda93cf1171e4c6e", - vec![196, 128, 1, 128, 128], - ), - ( - "0xcade985c7fb6d371d0c7f7cb40178e7873d623eadcc37545798ec33a04bb2173", - vec![196, 1, 128, 128, 128], - ), - ( - "0xcb54add475a18ea02ab1adf9e2e73da7f23ecd3e92c4fa8ca4e8f588258cb5d3", - vec![196, 128, 1, 128, 128], - ), - ( - "0xcb6f450b4720c6b36d3a12271e35ace27f1d527d46b073771541ad39cc59398d", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xcc74930e1ee0e71a8081f247ec47442a3e5d00897966754a5b3ee8beb2c1160c", - vec![196, 1, 128, 128, 128], - ), - ( - "0xcd07379b0120ad9a9c7fa47e77190be321ab107670f3115fec485bebb467307d", - vec![196, 128, 1, 128, 128], - ), - ( - "0xcd6b3739d4dbce17dafc156790f2a3936eb75ce95e9bba039dd76661f40ea309", - vec![ - 228, 1, 128, 160, 176, 112, 15, 225, 61, 186, 249, 75, 229, 11, 203, 236, 19, - 167, 181, 62, 108, 186, 3, 75, 41, 163, 218, 186, 152, 250, 134, 31, 88, 151, - 33, 63, 128, - ], - ), - ( - "0xce732a5e3b88ae26790aeb390a2bc02c449fdf57665c6d2c2b0dbce338c4377e", - vec![196, 128, 1, 128, 128], - ), - ( - "0xd1691564c6a5ab1391f0495634e749b9782de33756b6a058f4a9536c1b37bca6", - vec![ - 228, 1, 128, 160, 214, 14, 228, 173, 90, 187, 231, 89, 98, 47, 202, 92, 83, 97, - 9, 177, 30, 133, 170, 43, 72, 192, 190, 42, 235, 240, 29, 245, 151, 231, 77, - 186, 128, - ], - ), - ( - "0xd16e029e8c67c3f330cddaa86f82d31f523028404dfccd16d288645d718eb9da", - vec![196, 128, 1, 128, 128], - ), - ( - "0xd2501ae11a14bf0c2283a24b7e77c846c00a63e71908c6a5e1caff201bad0762", - vec![ - 228, 128, 128, 160, 73, 27, 44, 251, 169, 118, 178, 231, 139, 217, 190, 59, - 193, 92, 153, 100, 146, 114, 5, 252, 52, 201, 149, 74, 77, 97, 187, 232, 23, - 11, 165, 51, 128, - ], - ), - ( - "0xd2f394b4549b085fb9b9a8b313a874ea660808a4323ab2598ee15ddd1eb7e897", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd3443fa37ee617edc09a9c930be4873c21af2c47c99601d5e20483ce6d01960a", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xd352b05571154d9a2061143fe6df190a740a2d321c59eb94a54acb7f3054e489", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd37b6f5e5f0fa6a1b3fd15c9b3cf0fb595ba245ab912ad8059e672fa55f061b8", - vec![ - 228, 1, 128, 160, 89, 147, 108, 21, 196, 84, 147, 62, 188, 73, 137, 175, 167, - 126, 53, 15, 118, 64, 48, 27, 7, 52, 26, 234, 213, 241, 178, 102, 142, 235, 29, - 173, 128, - ], - ), - ( - "0xd52564daf6d32a6ae29470732726859261f5a7409b4858101bd233ed5cc2f662", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xd57eafe6d4c5b91fe7114e199318ab640e55d67a1e9e3c7833253808b7dca75f", - vec![ - 228, 1, 128, 160, 224, 163, 211, 184, 57, 252, 160, 245, 71, 69, 208, 197, 10, - 4, 142, 66, 76, 146, 89, 240, 99, 183, 65, 100, 16, 164, 66, 46, 235, 127, 131, - 126, 128, - ], - ), - ( - "0xd5e252ab2fba10107258010f154445cf7dffc42b7d8c5476de9a7adb533d73f1", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd5e5e7be8a61bb5bfa271dfc265aa9744dea85de957b6cffff0ecb403f9697db", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd623b1845175b206c127c08046281c013e4a3316402a771f1b3b77a9831143f5", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd63070208c85e91c4c8c942cf52c416f0f3004c392a15f579350168f178dba2e", - vec![196, 128, 1, 128, 128], - ), - ( - "0xd72e318c1cea7baf503950c9b1bd67cf7caf2f663061fcde48d379047a38d075", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xd8489fd0ce5e1806b24d1a7ce0e4ba8f0856b87696456539fcbb625a9bed2ccc", - vec![ - 228, 1, 128, 160, 52, 55, 128, 49, 1, 168, 4, 10, 202, 39, 63, 183, 52, 215, - 150, 90, 135, 248, 35, 255, 30, 247, 140, 126, 220, 170, 211, 88, 235, 152, - 222, 227, 128, - ], - ), - ( - "0xd84f7711be2f8eca69c742153230995afb483855b7c555b08da330139cdb9579", - vec![ - 228, 1, 128, 160, 158, 83, 240, 162, 221, 180, 48, 210, 127, 111, 255, 160, - 166, 139, 95, 117, 219, 29, 104, 226, 65, 19, 220, 202, 110, 51, 145, 140, 218, - 232, 8, 70, 128, - ], - ), - ( - "0xd9f987fec216556304eba05bcdae47bb736eea5a4183eb3e2c3a5045734ae8c7", - vec![196, 1, 128, 128, 128], - ), - ( - "0xd9fa858992bc92386a7cebcd748eedd602bf432cb4b31607566bc92b85179624", - vec![196, 128, 1, 128, 128], - ), - ( - "0xda81833ff053aff243d305449775c3fb1bd7f62c4a3c95dc9fb91b85e032faee", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xdbd66b6a89e01c76ae5f8cb0dcd8a24e787f58f015c9b08972bfabefa2eae0d5", - vec![196, 1, 128, 128, 128], - ), - ( - "0xdbea1fd70fe1c93dfef412ce5d8565d87d6843aac044d3a015fc3db4d20a351b", - vec![ - 228, 1, 128, 160, 190, 254, 85, 182, 6, 168, 101, 195, 137, 142, 194, 9, 59, - 209, 96, 179, 124, 57, 118, 1, 21, 22, 244, 55, 54, 202, 194, 169, 167, 236, - 212, 202, 128, - ], - ), - ( - "0xdc9ea08bdea052acab7c990edbb85551f2af3e1f1a236356ab345ac5bcc84562", - vec![ - 228, 128, 128, 160, 32, 127, 108, 62, 69, 5, 70, 176, 209, 243, 188, 106, 111, - 175, 91, 250, 11, 255, 128, 57, 108, 85, 213, 103, 184, 52, 207, 14, 124, 118, - 3, 71, 128, - ], - ), - ( - "0xdcda5b5203c2257997a574bdf85b2bea6d04829e8d7e048a709badc0fb99288c", - vec![ - 228, 1, 128, 160, 174, 68, 1, 67, 210, 30, 36, 169, 49, 182, 117, 111, 107, 61, - 80, 211, 55, 234, 240, 219, 62, 108, 52, 227, 106, 180, 111, 226, 217, 158, - 248, 62, 128, - ], - ), - ( - "0xdce547cc70c79575ef72c061502d6066db1cbce200bd904d5d2b20d4f1cb5963", - vec![ - 228, 1, 128, 160, 38, 37, 248, 162, 61, 36, 165, 223, 246, 167, 159, 99, 43, - 16, 32, 89, 51, 98, 166, 172, 98, 47, 165, 35, 116, 96, 188, 103, 176, 170, 14, - 211, 128, - ], - ), - ( - "0xdd1589b1fe1d9b4ca947f98ff324de7887af299d5490ed92ae40e95eec944118", - vec![196, 128, 1, 128, 128], - ), - ( - "0xdef989cb85107747de11222bd7418411f8f3264855e1939ef6bef9447e42076d", - vec![196, 128, 1, 128, 128], - ), - ( - "0xe02ec497b66cb57679eb01de1bed2ad385a3d18130441a9d337bd14897e85d39", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe04fdefc4f2eefd22721d5944411b282d0fcb1f9ac218f54793a35bca8199c25", - vec![196, 128, 1, 128, 128], - ), - ( - "0xe09e5f27b8a7bf61805df6e5fefc24eb6894281550c2d06250adecfe1e6581d7", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe0c5acf66bda927704953fdf7fb4b99e116857121c069eca7fb9bd8acfc25434", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe1068e9986da7636501d8893f67aa94f5d73df849feab36505fd990e2d6240e9", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe1b86a365b0f1583a07fc014602efc3f7dedfa90c66e738e9850719d34ac194e", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe1eb1e18ae510d0066d60db5c2752e8c33604d4da24c38d2bda07c0cb6ad19e4", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe31747e6542bf4351087edfbeb23e225e4217b5fa25d385f33cd024df0c9ae12", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe333845edc60ed469a894c43ed8c06ec807dafd079b3c948077da56e18436290", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe3c2e12be28e2e36dc852e76dd32e091954f99f2a6480853cd7b9e01ec6cd889", - vec![ - 228, 1, 128, 160, 204, 72, 248, 209, 192, 221, 110, 200, 171, 123, 189, 121, - 45, 148, 246, 167, 76, 136, 118, 180, 27, 200, 89, 206, 226, 34, 142, 141, 173, - 130, 7, 164, 128, - ], - ), - ( - "0xe3c79e424fd3a7e5bf8e0426383abd518604272fda87ecd94e1633d36f55bbb6", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe3d7213321be060ae2e1ff70871131ab3e4c9f4214a17fe9441453745c29365b", - vec![196, 128, 1, 128, 128], - ), - ( - "0xe42a85d04a1d0d9fe0703020ef98fa89ecdeb241a48de2db73f2feeaa2e49b0f", - vec![ - 228, 1, 128, 160, 251, 0, 114, 154, 95, 79, 154, 36, 54, 185, 153, 170, 113, - 89, 73, 122, 156, 216, 141, 21, 87, 112, 248, 115, 168, 24, 181, 80, 82, 197, - 240, 103, 128, - ], - ), - ( - "0xe4d9c31cc9b4a9050bbbf77cc08ac26d134253dcb6fd994275c5c3468f5b7810", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe5302e42ca6111d3515cbbb2225265077da41d997f069a6c492fa3fcb0fdf284", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xe6388bfcbbd6000e90a10633c72c43b0b0fed7cf38eab785a71e6f0c5b80a26a", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe69f40f00148bf0d4dfa28b3f3f5a0297790555eca01a00e49517c6645096a6c", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe6c5edf6a0fbdcff100e5ceafb63cba9aea355ba397a93fdb42a1a67b91375f8", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe6d72f72fd2fc8af227f75ab3ab199f12dfb939bdcff5f0acdac06a90084def8", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe73b3367629c8cb991f244ac073c0863ad1d8d88c2e180dd582cefda2de4415e", - vec![196, 1, 128, 128, 128], - ), - ( - "0xe74ac72f03e8c514c2c75f3c4f54ba31e920374ea7744ef1c33937e64c7d54f1", - vec![196, 128, 1, 128, 128], - ), - ( - "0xe7c6828e1fe8c586b263a81aafc9587d313c609c6db8665a42ae1267cd9ade59", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xe99460a483f3369006e3edeb356b3653699f246ec71f30568617ebc702058f59", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xea810ea64a420acfa917346a4a02580a50483890cba1d8d1d158d11f1c59ed02", - vec![ - 228, 1, 128, 160, 147, 106, 198, 37, 24, 72, 218, 105, 161, 145, 204, 145, 23, - 78, 75, 117, 131, 161, 42, 67, 216, 150, 226, 67, 132, 30, 169, 139, 101, 242, - 100, 173, 128, - ], - ), - ( - "0xeba984db32038d7f4d71859a9a2fc6e19dde2e23f34b7cedf0c4bf228c319f17", - vec![196, 128, 1, 128, 128], - ), - ( - "0xec3e92967d10ac66eff64a5697258b8acf87e661962b2938a0edcd78788f360d", - vec![ - 211, 128, 143, 192, 151, 206, 123, 201, 7, 21, 179, 75, 159, 16, 0, 0, 0, 0, - 128, 128, - ], - ), - ( - "0xed263a22f0e8be37bcc1873e589c54fe37fdde92902dc75d656997a7158a9d8c", - vec![ - 228, 1, 128, 160, 229, 71, 192, 5, 2, 83, 7, 91, 27, 228, 33, 6, 8, 188, 99, - 156, 255, 231, 1, 16, 25, 76, 49, 100, 129, 35, 94, 115, 139, 233, 97, 231, - 128, - ], - ), - ( - "0xedd9b1f966f1dfe50234523b479a45e95a1a8ec4a057ba5bfa7b69a13768197c", - vec![196, 1, 128, 128, 128], - ), - ( - "0xee9186a01e5e1122b61223b0e6acc6a069c9dcdb7307b0a296421272275f821b", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xefaff7acc3ad3417517b21a92187d2e63d7a77bc284290ed406d1bc07ab3d885", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xf0877d51b7712e08f2a3c96cddf50ff61b8b90f80b8b9817ea613a8a157b0c45", - vec![196, 1, 128, 128, 128], - ), - ( - "0xf0a51b55aadfa3cafdd214b0676816e574931a683f51218207c625375884e785", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf164775805f47d8970d3282188009d4d7a2da1574fe97e5d7bc9836a2eed1d5b", - vec![196, 1, 128, 128, 128], - ), - ( - "0xf16522fc36907ee1e9948240b0c1d1d105a75cc63b71006f16c20d79ad469bd7", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf19ee923ed66b7b9264c2644aa20e5268a251b4914ca81b1dffee96ecb074cb1", - vec![ - 228, 1, 128, 160, 205, 62, 117, 41, 158, 150, 125, 95, 136, 211, 6, 190, 144, - 90, 19, 67, 67, 178, 36, 211, 253, 90, 134, 27, 26, 105, 13, 224, 226, 223, - 225, 186, 128, - ], - ), - ( - "0xf2b9bc1163840284f3eb15c539972edad583cda91946f344f4cb57be15af9c8f", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf33a7b66489679fa665dbfb4e6dd4b673495f853850eedc81d5f28bd2f4bd3b5", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xf462aaa112b195c148974ff796a81c0e7f9a972d04e60c178ac109102d593a88", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf4a1c4554b186a354b3e0c467eef03df9907cd5a5d96086c1a542b9e5160ca78", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf63360f8bb23f88b0a564f9e07631c38c73b4074ba4192d6131336ef02ee9cf2", - vec![196, 128, 1, 128, 128], - ), - ( - "0xf84223f460140ad56af9836cfa6c1c58c1397abf599c214689bc881066020ff7", - vec![196, 1, 128, 128, 128], - ), - ( - "0xfab4c6889992a3f4e96b005dfd851021e9e1ec2631a7ccd2a001433e35077968", - vec![196, 128, 1, 128, 128], - ), - ( - "0xfb2ab315988de92dcf6ba848e756676265b56e4b84778a2c955fb2b3c848c51c", - vec![196, 1, 128, 128, 128], - ), - ( - "0xfb5a31c5cfd33dce2c80a30c5efc28e5f4025624adcc2205a2504a78c57bdd1c", - vec![ - 228, 1, 128, 160, 73, 63, 144, 67, 84, 2, 223, 9, 7, 1, 155, 255, 198, 221, 37, - 161, 124, 228, 172, 214, 235, 96, 119, 239, 148, 193, 98, 111, 13, 119, 201, - 240, 128, - ], - ), - ( - "0xfb9474d0e5538fcd99e8d8d024db335b4e057f4bcd359e85d78f4a5226b33272", - vec![196, 128, 1, 128, 128], - ), - ( - "0xfc3d2e27841c0913d10aa11fc4af4793bf376efe3d90ce8360aa392d0ecefa24", - vec![ - 228, 1, 128, 160, 123, 245, 66, 189, 175, 245, 191, 227, 211, 60, 38, 168, 135, - 119, 119, 59, 94, 82, 84, 97, 9, 60, 54, 172, 176, 218, 181, 145, 163, 25, 229, - 9, 128, - ], - ), - ( - "0xfc4870c3cd21d694424c88f0f31f75b2426e1530fdea26a14031ccf9baed84c4", - vec![196, 128, 1, 128, 128], - ), - ( - "0xfc8d513d1615c763865b984ea9c381032c14a983f80e5b2bd90b20b518329ed7", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xfcc08928955d4e5e17e17e46d5adbb8011e0a8a74cabbdd3e138c367e89a4428", - vec![196, 128, 1, 128, 128], - ), - ( - "0xfd3a8bacd3b2061cbe54f8d38cf13c5c87a92816937683652886dee936dfae10", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xfdaf2549ea901a469b3e91cd1c4290fab376ef687547046751e10b7b461ff297", - vec![196, 1, 128, 128, 128], - ), - ( - "0xfdbb8ddca8cecfe275da1ea1c36e494536f581d64ddf0c4f2e6dae9c7d891427", - vec![ - 228, 1, 128, 160, 211, 217, 131, 159, 135, 194, 159, 176, 7, 253, 153, 40, 211, - 139, 191, 132, 239, 8, 159, 12, 214, 64, 200, 56, 244, 164, 38, 49, 232, 40, - 198, 103, 128, - ], - ), - ( - "0xfe2149c5c256a5eb2578c013d33e3af6a87a514965c7ddf4a8131e2d978f09f9", - vec![196, 128, 1, 128, 128], - ), - ( - "0xfe2511e8a33ac9973b773aaedcb4daa73ae82481fe5a1bf78b41281924260cf5", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), - ( - "0xfe6e594c507ec0ac14917f7a8032f83cd0c3c58b461d459b822190290852c0e1", - vec![201, 128, 133, 23, 72, 118, 232, 0, 128, 128], - ), ]; // Create a store and load it up with the accounts From 38a7d5bbae8f43cdeca6a8226f9d2bf886014d8f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 10:43:50 -0300 Subject: [PATCH 022/345] Simplify logic --- crates/networking/p2p/snap.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 2816c998d..6b4b172c1 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -10,21 +10,14 @@ pub fn process_account_range_request( let mut accounts = vec![]; // Fetch account range let mut iter = store.iter_accounts(request.root_hash); - let mut start_found = false; let mut bytes_used = 0; while let Some((k, v)) = iter.next() { if k >= request.starting_hash { - start_found = true; - } - if start_found { let acc = AccountStateSlim::from(v); bytes_used += bytes_per_entry(&acc); accounts.push((k, acc)); } - if k >= request.limit_hash { - break; - } - if bytes_used >= request.response_bytes { + if k >= request.limit_hash || bytes_used >= request.response_bytes { break; } } @@ -64,6 +57,7 @@ mod tests { use lazy_static::lazy_static; lazy_static! { + // Constant values for hive `AccountRange` tests static ref HASH_MIN: H256 = H256::zero(); static ref HASH_MAX: H256 = H256::from_str("0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff",) From 704da4cff865478a9d9c66caab52f94ddb4c307a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 12:49:24 -0300 Subject: [PATCH 023/345] Encode accounts while building range so we do not encode twice --- crates/networking/p2p/rlpx/snap.rs | 3 ++- crates/networking/p2p/snap.rs | 13 ++++--------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 9ed737ed2..62f86cacb 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -137,7 +137,8 @@ pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, - pub accounts: Vec<(H256, AccountStateSlim)>, + // List of (hash, account) pairs, accounts consis of RLP-encoded slim accounts + pub accounts: Vec<(H256, Vec)>, pub proof: Vec>, } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 6b4b172c1..0601c9fca 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -13,9 +13,10 @@ pub fn process_account_range_request( let mut bytes_used = 0; while let Some((k, v)) = iter.next() { if k >= request.starting_hash { - let acc = AccountStateSlim::from(v); - bytes_used += bytes_per_entry(&acc); - accounts.push((k, acc)); + let account = AccountStateSlim::from(v).encode_to_vec(); + // size of hash + size of account + bytes_used += 32 + account.len() as u64; + accounts.push((k, account)); } if k >= request.limit_hash || bytes_used >= request.response_bytes { break; @@ -34,11 +35,6 @@ pub fn process_account_range_request( }) } -// TODO: write response bytes directly here so we dont need to encode twice -fn bytes_per_entry(state: &AccountStateSlim) -> u64 { - state.encode_to_vec().len() as u64 + 32 -} - #[cfg(test)] mod tests { use std::str::FromStr; @@ -93,7 +89,6 @@ mod tests { H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") .unwrap() ); - // Check proofs against geth values } #[test] From 6d5cc45ae85819da6993fdb6a7825852245033dc Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 13:06:28 -0300 Subject: [PATCH 024/345] Clippy --- crates/networking/p2p/snap.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 0601c9fca..93cc43234 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -8,10 +8,8 @@ pub fn process_account_range_request( store: Store, ) -> Result { let mut accounts = vec![]; - // Fetch account range - let mut iter = store.iter_accounts(request.root_hash); let mut bytes_used = 0; - while let Some((k, v)) = iter.next() { + for (k, v) in store.iter_accounts(request.root_hash) { if k >= request.starting_hash { let account = AccountStateSlim::from(v).encode_to_vec(); // size of hash + size of account From 95140370dce44c41851ddf09610157fa9d887ad5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 16:26:46 -0300 Subject: [PATCH 025/345] Add allow tag --- crates/networking/p2p/snap.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 93cc43234..6ef8c5c10 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -3,6 +3,7 @@ use ethereum_rust_storage::{error::StoreError, Store}; use crate::rlpx::snap::{AccountRange, AccountStateSlim, GetAccountRange}; +#[allow(unused)] pub fn process_account_range_request( request: GetAccountRange, store: Store, From a5b3934c99dc835b33d5347e391f1ea37326bec4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 16:34:12 -0300 Subject: [PATCH 026/345] Remove comment --- crates/networking/p2p/rlpx/connection.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index d69914db2..f47d4e342 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -21,7 +21,6 @@ use k256::{ use sha3::{Digest, Keccak256}; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tracing::{error, info}; -// pub const SUPPORTED_CAPABILITIES: [(&str, u8); 2] = [("p2p", 5), ("eth", 68)]; pub const SUPPORTED_CAPABILITIES: [(&str, u8); 3] = [("p2p", 5), ("eth", 68), ("snap", 1)]; pub(crate) type Aes256Ctr64BE = ctr::Ctr64BE; From 68dcef63fe134613936ec76c0fbcb6660bc8cd93 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 24 Oct 2024 16:39:18 -0300 Subject: [PATCH 027/345] Fix typo --- crates/networking/p2p/rlpx/snap.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 62f86cacb..c1ab07374 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -137,7 +137,7 @@ pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, - // List of (hash, account) pairs, accounts consis of RLP-encoded slim accounts + // List of (hash, account) pairs, accounts consists of RLP-encoded slim accounts pub accounts: Vec<(H256, Vec)>, pub proof: Vec>, } From f23077eb328731cf3bfdcf07e2f6a07de523b5d6 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Thu, 24 Oct 2024 16:46:41 -0300 Subject: [PATCH 028/345] Sending eth Status message first --- cmd/ethereum_rust/ethereum_rust.rs | 8 +- crates/networking/p2p/net.rs | 2 +- crates/networking/p2p/rlpx/connection.rs | 163 ++++++++++++----------- crates/networking/p2p/rlpx/eth.rs | 6 +- crates/networking/p2p/rlpx/handshake.rs | 22 +-- 5 files changed, 94 insertions(+), 107 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index af94e955f..0ae420469 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -1,3 +1,4 @@ +use anyhow::Context; use bytes::Bytes; use directories::ProjectDirs; use ethereum_rust_blockchain::add_block; @@ -128,10 +129,11 @@ async fn main() { "Adding block {} with hash {:#x}.", block.header.number, hash ); - if add_block(&block, &store).is_err() { + let result = add_block(&block, &store); + if result.is_err() { warn!( - "Failed to add block {} with hash {:#x}.", - block.header.number, hash + "Failed to add block {} with hash {:#x}: {:?}.", + block.header.number, hash, result ); } } diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index 7f6e59fd1..fd512c138 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -779,7 +779,7 @@ async fn handle_peer_as_initiator( async fn handle_peer(mut conn: RLPxConnection, table: Arc>) { match conn.handshake().await { - Ok(_) => match conn.main_loop().await { + Ok(_) => match conn.handle_peer().await { Ok(_) => unreachable!(), Err(e) => info!("Error during RLPx connection: ({e})"), }, diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 69a9af5d5..bb678c7c7 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -76,7 +76,70 @@ impl RLPxConnection { RLPxConnection::new(signer, stream, state, storage) } - pub async fn main_loop(&mut self) -> Result<(), RLPxError> { + pub async fn handshake(&mut self) -> Result<(), RLPxError> { + match &self.state { + RLPxConnectionState::Initiator(_) => { + self.send_auth().await; + self.receive_ack().await; + } + RLPxConnectionState::Receiver(_) => { + self.receive_auth().await; + self.send_ack().await; + } + _ => { + return Err(RLPxError::HandshakeError( + "Invalid connection state for handshake".to_string(), + )) + } + }; + info!("Completed handshake!"); + + self.exchange_hello_messages().await?; + info!("Completed Hello roundtrip!"); + Ok(()) + } + + pub async fn exchange_hello_messages(&mut self) -> Result<(), RLPxError> { + let supported_capabilities: Vec<(String, u8)> = SUPPORTED_CAPABILITIES + .into_iter() + .map(|(name, version)| (name.to_string(), version)) + .collect(); + let hello_msg = Message::Hello(p2p::HelloMessage::new( + supported_capabilities.clone(), + PublicKey::from(self.signer.verifying_key()), + )); + + self.send(hello_msg).await; + info!("Hello message sent!"); + + // Receive Hello message + match self.receive().await { + Message::Hello(hello_message) => { + info!("Hello message received {hello_message:?}"); + self.capabilities = hello_message.capabilities; + + // Check if we have any capability in common + for cap in self.capabilities.clone() { + if supported_capabilities.contains(&cap) { + return Ok(()); + } + } + // Return error if not + Err(RLPxError::HandshakeError( + "No matching capabilities".to_string(), + )) + } + _ => { + // Fail if it is not a hello message + Err(RLPxError::HandshakeError( + "Expected Hello message".to_string(), + )) + } + } + } + + pub async fn handle_peer(&mut self) -> Result<(), RLPxError> { + self.start_capabilities().await?; match &self.state { RLPxConnectionState::Established(_) => { info!("Started peer main loop"); @@ -87,9 +150,6 @@ impl RLPxConnection { Message::Pong(_) => info!("Received Pong"), Message::Status(_) => { info!("Received Status"); - let message = - Message::Status(StatusMessage::new(&self.storage).unwrap()); - self.send(message).await; info!("Sent Status"); } message => return Err(RLPxError::UnexpectedMessage(message)), @@ -102,32 +162,25 @@ impl RLPxConnection { } } - pub async fn handshake(&mut self) -> Result<(), RLPxError> { + pub fn get_remote_node_id(&self) -> H512 { match &self.state { - RLPxConnectionState::Initiator(_) => { - self.send_auth().await; - self.receive_ack().await; - } - RLPxConnectionState::Receiver(_) => { - self.receive_auth().await; - self.send_ack().await; - } - _ => { - return Err(RLPxError::HandshakeError( - "Invalid connection state for handshake".to_string(), - )) - } - }; - info!("Completed handshake!"); + RLPxConnectionState::Established(state) => state.remote_node_id, + // TODO proper error + _ => panic!("Invalid state"), + } + } - self.exchange_hello_messages().await?; - info!("Completed Hello roundtrip!"); - // let message = Message::Status(StatusMessage::new(&self.storage).unwrap()); - // self.send(message); + async fn start_capabilities(&mut self) -> Result<(), RLPxError> { + // Sending eth Status if peer supports it + if self.capabilities.contains(&("eth".to_string(), 68u8)) { + let status = StatusMessage::new(&self.storage).unwrap(); + info!("Status message sent: {status:?}"); + self.send(Message::Status(status)).await; + } Ok(()) } - pub async fn send_auth(&mut self) { + async fn send_auth(&mut self) { match &self.state { RLPxConnectionState::Initiator(initiator_state) => { let secret_key: SecretKey = self.signer.clone().into(); @@ -155,19 +208,16 @@ impl RLPxConnection { }; } - pub async fn send_ack(&mut self) { + async fn send_ack(&mut self) { match &self.state { RLPxConnectionState::ReceivedAuth(received_auth_state) => { - let secret_key: SecretKey = self.signer.clone().into(); let peer_pk = id2pubkey(received_auth_state.remote_node_id).unwrap(); let mut ack_message = vec![]; let msg = encode_ack_message( - &secret_key, &received_auth_state.local_ephemeral_key, received_auth_state.local_nonce, &peer_pk, - &received_auth_state.remote_ephemeral_key, ); ack_message.put_slice(&msg); @@ -184,7 +234,7 @@ impl RLPxConnection { }; } - pub async fn receive_auth(&mut self) { + async fn receive_auth(&mut self) { match &self.state { RLPxConnectionState::Receiver(receiver_state) => { let secret_key: SecretKey = self.signer.clone().into(); @@ -219,7 +269,7 @@ impl RLPxConnection { }; } - pub async fn receive_ack(&mut self) { + async fn receive_ack(&mut self) { match &self.state { RLPxConnectionState::InitiatedAuth(initiated_auth_state) => { let secret_key: SecretKey = self.signer.clone().into(); @@ -254,46 +304,7 @@ impl RLPxConnection { }; } - pub async fn exchange_hello_messages(&mut self) -> Result<(), RLPxError> { - let supported_capabilities: Vec<(String, u8)> = SUPPORTED_CAPABILITIES - .into_iter() - .map(|(name, version)| (name.to_string(), version)) - .collect(); - let hello_msg = Message::Hello(p2p::HelloMessage::new( - supported_capabilities.clone(), - PublicKey::from(self.signer.verifying_key()), - )); - - self.send(hello_msg).await; - info!("Hello message sent!"); - - // Receive Hello message - match self.receive().await { - Message::Hello(hello_message) => { - info!("Hello message received {hello_message:?}"); - self.capabilities = hello_message.capabilities; - - // Check if we have any capability in common - for cap in self.capabilities.clone() { - if supported_capabilities.contains(&cap) { - return Ok(()); - } - } - // Return error if not - Err(RLPxError::HandshakeError( - "No matching capabilities".to_string(), - )) - } - _ => { - // Fail if it is not a hello message - Err(RLPxError::HandshakeError( - "Expected Hello message".to_string(), - )) - } - } - } - - pub async fn send(&mut self, message: rlpx::Message) { + async fn send(&mut self, message: rlpx::Message) { match &mut self.state { RLPxConnectionState::Established(state) => { let mut frame_buffer = vec![]; @@ -311,7 +322,7 @@ impl RLPxConnection { } } - pub async fn receive(&mut self) -> rlpx::Message { + async fn receive(&mut self) -> rlpx::Message { match &mut self.state { RLPxConnectionState::Established(state) => { let frame_data = frame::read(state, &mut self.stream).await; @@ -323,14 +334,6 @@ impl RLPxConnection { _ => panic!("Received an unexpected message"), } } - - pub fn get_remote_node_id(&self) -> H512 { - match &self.state { - RLPxConnectionState::Established(state) => state.remote_node_id, - // TODO proper error - _ => panic!("Invalid state"), - } - } } enum RLPxConnectionState { diff --git a/crates/networking/p2p/rlpx/eth.rs b/crates/networking/p2p/rlpx/eth.rs index 52387e6aa..84480d6c7 100644 --- a/crates/networking/p2p/rlpx/eth.rs +++ b/crates/networking/p2p/rlpx/eth.rs @@ -2,7 +2,7 @@ use super::{message::RLPxMessage, utils::snappy_encode}; use bytes::BufMut; use ethereum_rust_core::{ types::{BlockHash, ForkId}, - U256, + H256, U256, }; use ethereum_rust_rlp::{ encode::RLPEncode, @@ -11,6 +11,7 @@ use ethereum_rust_rlp::{ }; use ethereum_rust_storage::{error::StoreError, Store}; use snap::raw::Decoder as SnappyDecoder; +use tracing::info; pub const ETH_VERSION: u32 = 68; pub const HASH_FIRST_BYTE_DECODER: u8 = 160; @@ -41,7 +42,8 @@ impl StatusMessage { // These blocks must always be available let genesis_header = storage.get_block_header(0)?.unwrap(); let block_number = storage.get_latest_block_number()?.unwrap(); - let block_header = storage.get_block_header(block_number)?.unwrap(); + info!("last block number {block_number}"); + let block_header = storage.get_block_header(500)?.unwrap(); let genesis = genesis_header.compute_block_hash(); let block_hash = block_header.compute_block_hash(); diff --git a/crates/networking/p2p/rlpx/handshake.rs b/crates/networking/p2p/rlpx/handshake.rs index 679a0f8d8..c88941ca5 100644 --- a/crates/networking/p2p/rlpx/handshake.rs +++ b/crates/networking/p2p/rlpx/handshake.rs @@ -13,7 +13,6 @@ use k256::{ PublicKey, SecretKey, }; use rand::Rng; -use tracing::info; type Aes128Ctr64BE = ctr::Ctr64BE; @@ -56,39 +55,20 @@ pub(crate) fn decode_auth_message( // RLP-decode the message. let (auth, _padding) = AuthMessage::decode_unfinished(&payload).unwrap(); - info!( - "signature: {:?} node_id: {:?} nonce: {:?}", - &auth.signature, &auth.node_id, &auth.nonce - ); - - let peer_pk = id2pubkey(auth.node_id).unwrap(); - // Derive a shared secret from the static keys. + let peer_pk = id2pubkey(auth.node_id).unwrap(); let static_shared_secret = ecdh_xchng(static_key, &peer_pk); - info!("token {static_shared_secret:?}"); - let remote_ephemeral_key = retrieve_remote_ephemeral_key(static_shared_secret.into(), auth.nonce, auth.signature); - - info!("remote pub key {remote_ephemeral_key:?}"); - (auth, remote_ephemeral_key) } /// Encodes an Ack message, to complete a handshake pub fn encode_ack_message( - static_key: &SecretKey, local_ephemeral_key: &SecretKey, local_nonce: H256, remote_static_pubkey: &PublicKey, - remote_ephemeral_key: &PublicKey, ) -> Vec { - // Derive a shared secret from the static keys. - let static_shared_secret = ecdh_xchng(static_key, remote_static_pubkey); - info!("token {static_shared_secret:?}"); - - info!("remote pub key {remote_ephemeral_key:?}"); - // Compose the ack message. let ack_msg = AckMessage::new(pubkey2id(&local_ephemeral_key.public_key()), local_nonce); From 122778dd4daa8e769d0eae129e82010a2eefb6b9 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Fri, 25 Oct 2024 11:35:59 -0300 Subject: [PATCH 029/345] Small fixes --- cmd/ethereum_rust/ethereum_rust.rs | 1 - crates/networking/p2p/rlpx/eth.rs | 6 ++---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 0ae420469..32ad9e3b3 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -1,4 +1,3 @@ -use anyhow::Context; use bytes::Bytes; use directories::ProjectDirs; use ethereum_rust_blockchain::add_block; diff --git a/crates/networking/p2p/rlpx/eth.rs b/crates/networking/p2p/rlpx/eth.rs index 84480d6c7..7474a661a 100644 --- a/crates/networking/p2p/rlpx/eth.rs +++ b/crates/networking/p2p/rlpx/eth.rs @@ -2,7 +2,7 @@ use super::{message::RLPxMessage, utils::snappy_encode}; use bytes::BufMut; use ethereum_rust_core::{ types::{BlockHash, ForkId}, - H256, U256, + U256, }; use ethereum_rust_rlp::{ encode::RLPEncode, @@ -30,8 +30,6 @@ pub(crate) struct StatusMessage { fork_id: ForkId, } -// TODO remove this allow once we construct StatusMessages -#[allow(unused)] impl StatusMessage { pub fn new(storage: &Store) -> Result { let chain_config = storage.get_chain_config()?; @@ -43,7 +41,7 @@ impl StatusMessage { let genesis_header = storage.get_block_header(0)?.unwrap(); let block_number = storage.get_latest_block_number()?.unwrap(); info!("last block number {block_number}"); - let block_header = storage.get_block_header(500)?.unwrap(); + let block_header = storage.get_block_header(block_number)?.unwrap(); let genesis = genesis_header.compute_block_hash(); let block_hash = block_header.compute_block_hash(); From 5ae41fedb5f0922273e9cb4ae8b85216ef8f603a Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Fri, 25 Oct 2024 11:53:13 -0300 Subject: [PATCH 030/345] Small fixes --- crates/networking/p2p/rlpx/eth.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/eth.rs b/crates/networking/p2p/rlpx/eth.rs index 7474a661a..2074ec85c 100644 --- a/crates/networking/p2p/rlpx/eth.rs +++ b/crates/networking/p2p/rlpx/eth.rs @@ -11,7 +11,6 @@ use ethereum_rust_rlp::{ }; use ethereum_rust_storage::{error::StoreError, Store}; use snap::raw::Decoder as SnappyDecoder; -use tracing::info; pub const ETH_VERSION: u32 = 68; pub const HASH_FIRST_BYTE_DECODER: u8 = 160; @@ -40,7 +39,6 @@ impl StatusMessage { // These blocks must always be available let genesis_header = storage.get_block_header(0)?.unwrap(); let block_number = storage.get_latest_block_number()?.unwrap(); - info!("last block number {block_number}"); let block_header = storage.get_block_header(block_number)?.unwrap(); let genesis = genesis_header.compute_block_hash(); From 66b0bd828c0ec361fa2f21f4c5dfb9b8b26c15eb Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Fri, 25 Oct 2024 11:58:55 -0300 Subject: [PATCH 031/345] Added TODO comments for pending tasks --- crates/networking/p2p/rlpx/connection.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index bb678c7c7..9310e44ca 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -145,13 +145,12 @@ impl RLPxConnection { info!("Started peer main loop"); loop { match self.receive().await { + // TODO: implement handlers for each message type Message::Disconnect(_) => info!("Received Disconnect"), Message::Ping(_) => info!("Received Ping"), Message::Pong(_) => info!("Received Pong"), - Message::Status(_) => { - info!("Received Status"); - info!("Sent Status"); - } + Message::Status(_) => info!("Received Status"), + // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; } @@ -177,6 +176,7 @@ impl RLPxConnection { info!("Status message sent: {status:?}"); self.send(Message::Status(status)).await; } + // TODO: add new capabilities startup when required (eg. snap) Ok(()) } From 6a5f58065707342782a6d68d50fcf8f3fb751554 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Fri, 25 Oct 2024 12:52:52 -0300 Subject: [PATCH 032/345] Refactored code to separate encoding/decoding from backend logic --- crates/networking/p2p/rlpx/connection.rs | 6 +- crates/networking/p2p/rlpx/eth.rs | 114 +----------------- crates/networking/p2p/rlpx/eth/backend.rs | 29 +++++ crates/networking/p2p/rlpx/eth/receipts.rs | 4 +- crates/networking/p2p/rlpx/eth/status.rs | 96 +++++++++++++++ .../networking/p2p/rlpx/eth/transactions.rs | 4 +- crates/networking/p2p/rlpx/message.rs | 2 +- 7 files changed, 135 insertions(+), 120 deletions(-) create mode 100644 crates/networking/p2p/rlpx/eth/backend.rs create mode 100644 crates/networking/p2p/rlpx/eth/status.rs diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 9310e44ca..0ce82a83c 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,7 +1,5 @@ use crate::{ - rlpx::{ - eth::StatusMessage, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey, - }, + rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, MAX_DISC_PACKET_SIZE, }; @@ -172,7 +170,7 @@ impl RLPxConnection { async fn start_capabilities(&mut self) -> Result<(), RLPxError> { // Sending eth Status if peer supports it if self.capabilities.contains(&("eth".to_string(), 68u8)) { - let status = StatusMessage::new(&self.storage).unwrap(); + let status = backend::get_status(&self.storage).unwrap(); info!("Status message sent: {status:?}"); self.send(Message::Status(status)).await; } diff --git a/crates/networking/p2p/rlpx/eth.rs b/crates/networking/p2p/rlpx/eth.rs index 2074ec85c..a03ff256c 100644 --- a/crates/networking/p2p/rlpx/eth.rs +++ b/crates/networking/p2p/rlpx/eth.rs @@ -1,109 +1,5 @@ -use super::{message::RLPxMessage, utils::snappy_encode}; -use bytes::BufMut; -use ethereum_rust_core::{ - types::{BlockHash, ForkId}, - U256, -}; -use ethereum_rust_rlp::{ - encode::RLPEncode, - error::{RLPDecodeError, RLPEncodeError}, - structs::{Decoder, Encoder}, -}; -use ethereum_rust_storage::{error::StoreError, Store}; -use snap::raw::Decoder as SnappyDecoder; - -pub const ETH_VERSION: u32 = 68; -pub const HASH_FIRST_BYTE_DECODER: u8 = 160; - -mod blocks; -mod receipts; -mod transactions; - -#[derive(Debug)] -pub(crate) struct StatusMessage { - eth_version: u32, - network_id: u64, - total_difficulty: U256, - block_hash: BlockHash, - genesis: BlockHash, - fork_id: ForkId, -} - -impl StatusMessage { - pub fn new(storage: &Store) -> Result { - let chain_config = storage.get_chain_config()?; - let total_difficulty = - U256::from(chain_config.terminal_total_difficulty.unwrap_or_default()); - let network_id = chain_config.chain_id; - - // These blocks must always be available - let genesis_header = storage.get_block_header(0)?.unwrap(); - let block_number = storage.get_latest_block_number()?.unwrap(); - let block_header = storage.get_block_header(block_number)?.unwrap(); - - let genesis = genesis_header.compute_block_hash(); - let block_hash = block_header.compute_block_hash(); - let fork_id = ForkId::new(chain_config, genesis, block_header.timestamp, block_number); - Ok(Self { - eth_version: ETH_VERSION, - network_id, - total_difficulty, - block_hash, - genesis, - fork_id, - }) - } -} - -impl RLPxMessage for StatusMessage { - fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { - 16_u8.encode(buf); // msg_id - - let mut encoded_data = vec![]; - Encoder::new(&mut encoded_data) - .encode_field(&self.eth_version) - .encode_field(&self.network_id) - .encode_field(&self.total_difficulty) - .encode_field(&self.block_hash) - .encode_field(&self.genesis) - .encode_field(&self.fork_id) - .finish(); - - let msg_data = snappy_encode(encoded_data)?; - buf.put_slice(&msg_data); - Ok(()) - } - - fn decode(msg_data: &[u8]) -> Result { - let mut snappy_decoder = SnappyDecoder::new(); - let decompressed_data = snappy_decoder - .decompress_vec(msg_data) - .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; - let decoder = Decoder::new(&decompressed_data)?; - let (eth_version, decoder): (u32, _) = decoder.decode_field("protocolVersion")?; - - assert_eq!(eth_version, 68, "only eth version 68 is supported"); - - let (network_id, decoder): (u64, _) = decoder.decode_field("networkId")?; - - let (total_difficulty, decoder): (U256, _) = decoder.decode_field("totalDifficulty")?; - - let (block_hash, decoder): (BlockHash, _) = decoder.decode_field("blockHash")?; - - let (genesis, decoder): (BlockHash, _) = decoder.decode_field("genesis")?; - - let (fork_id, decoder): (ForkId, _) = decoder.decode_field("forkId")?; - - // Implementations must ignore any additional list elements - let _padding = decoder.finish_unchecked(); - - Ok(Self { - eth_version, - network_id, - total_difficulty, - block_hash, - genesis, - fork_id, - }) - } -} +pub(crate) mod backend; +pub(crate) mod blocks; +pub(crate) mod receipts; +pub(crate) mod status; +pub(crate) mod transactions; diff --git a/crates/networking/p2p/rlpx/eth/backend.rs b/crates/networking/p2p/rlpx/eth/backend.rs new file mode 100644 index 000000000..1af62214a --- /dev/null +++ b/crates/networking/p2p/rlpx/eth/backend.rs @@ -0,0 +1,29 @@ +use ethereum_rust_core::{types::ForkId, U256}; +use ethereum_rust_storage::{error::StoreError, Store}; + +use super::status::StatusMessage; + +pub const ETH_VERSION: u32 = 68; + +pub fn get_status(storage: &Store) -> Result { + let chain_config = storage.get_chain_config()?; + let total_difficulty = U256::from(chain_config.terminal_total_difficulty.unwrap_or_default()); + let network_id = chain_config.chain_id; + + // These blocks must always be available + let genesis_header = storage.get_block_header(0)?.unwrap(); + let block_number = storage.get_latest_block_number()?.unwrap(); + let block_header = storage.get_block_header(block_number)?.unwrap(); + + let genesis = genesis_header.compute_block_hash(); + let block_hash = block_header.compute_block_hash(); + let fork_id = ForkId::new(chain_config, genesis, block_header.timestamp, block_number); + Ok(StatusMessage::new( + ETH_VERSION, + network_id, + total_difficulty, + block_hash, + genesis, + fork_id, + )) +} diff --git a/crates/networking/p2p/rlpx/eth/receipts.rs b/crates/networking/p2p/rlpx/eth/receipts.rs index 496273341..5d76a2f27 100644 --- a/crates/networking/p2p/rlpx/eth/receipts.rs +++ b/crates/networking/p2p/rlpx/eth/receipts.rs @@ -6,9 +6,7 @@ use ethereum_rust_rlp::{ }; use snap::raw::Decoder as SnappyDecoder; -use crate::rlpx::message::RLPxMessage; - -use super::snappy_encode; +use crate::rlpx::{message::RLPxMessage, utils::snappy_encode}; // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#getreceipts-0x0f #[derive(Debug)] diff --git a/crates/networking/p2p/rlpx/eth/status.rs b/crates/networking/p2p/rlpx/eth/status.rs new file mode 100644 index 000000000..9050ac433 --- /dev/null +++ b/crates/networking/p2p/rlpx/eth/status.rs @@ -0,0 +1,96 @@ +use bytes::BufMut; +use ethereum_rust_core::{ + types::{BlockHash, ForkId}, + U256, +}; +use ethereum_rust_rlp::{ + encode::RLPEncode, + error::{RLPDecodeError, RLPEncodeError}, + structs::{Decoder, Encoder}, +}; +use snap::raw::Decoder as SnappyDecoder; + +use crate::rlpx::{message::RLPxMessage, utils::snappy_encode}; + +#[derive(Debug)] +pub(crate) struct StatusMessage { + eth_version: u32, + network_id: u64, + total_difficulty: U256, + block_hash: BlockHash, + genesis: BlockHash, + fork_id: ForkId, +} + +impl StatusMessage { + pub fn new( + eth_version: u32, + network_id: u64, + total_difficulty: U256, + block_hash: BlockHash, + genesis: BlockHash, + fork_id: ForkId, + ) -> Self { + Self { + eth_version, + network_id, + total_difficulty, + block_hash, + genesis, + fork_id, + } + } +} + +impl RLPxMessage for StatusMessage { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + 16_u8.encode(buf); // msg_id + + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.eth_version) + .encode_field(&self.network_id) + .encode_field(&self.total_difficulty) + .encode_field(&self.block_hash) + .encode_field(&self.genesis) + .encode_field(&self.fork_id) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (eth_version, decoder): (u32, _) = decoder.decode_field("protocolVersion")?; + + assert_eq!(eth_version, 68, "only eth version 68 is supported"); + + let (network_id, decoder): (u64, _) = decoder.decode_field("networkId")?; + + let (total_difficulty, decoder): (U256, _) = decoder.decode_field("totalDifficulty")?; + + let (block_hash, decoder): (BlockHash, _) = decoder.decode_field("blockHash")?; + + let (genesis, decoder): (BlockHash, _) = decoder.decode_field("genesis")?; + + let (fork_id, decoder): (ForkId, _) = decoder.decode_field("forkId")?; + + // Implementations must ignore any additional list elements + let _padding = decoder.finish_unchecked(); + + Ok(Self::new( + eth_version, + network_id, + total_difficulty, + block_hash, + genesis, + fork_id, + )) + } +} diff --git a/crates/networking/p2p/rlpx/eth/transactions.rs b/crates/networking/p2p/rlpx/eth/transactions.rs index 4923e3b19..d84e9c228 100644 --- a/crates/networking/p2p/rlpx/eth/transactions.rs +++ b/crates/networking/p2p/rlpx/eth/transactions.rs @@ -6,9 +6,7 @@ use ethereum_rust_rlp::{ }; use snap::raw::Decoder as SnappyDecoder; -use crate::rlpx::message::RLPxMessage; - -use super::snappy_encode; +use crate::rlpx::{message::RLPxMessage, utils::snappy_encode}; // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#transactions-0x02 // Broadcast message diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 480e180bf..8f06159be 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -2,7 +2,7 @@ use bytes::BufMut; use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; use std::fmt::Display; -use super::eth::StatusMessage; +use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; pub trait RLPxMessage: Sized { From 05c9a5efaa31bdf3d74ff10ee27aaf17dbcc65a2 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Fri, 25 Oct 2024 15:42:38 -0300 Subject: [PATCH 033/345] Replaced hardcoded capabilities strings with enums --- crates/networking/p2p/rlpx/connection.rs | 20 ++++++------- crates/networking/p2p/rlpx/p2p.rs | 38 +++++++++++++++++++++--- 2 files changed, 44 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 0ce82a83c..06de21c3c 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -8,6 +8,7 @@ use super::{ frame, handshake::{decode_ack_message, decode_auth_message, encode_auth_message}, message as rlpx, + p2p::Capability, utils::{ecdh_xchng, pubkey2id}, }; use aes::cipher::KeyIvInit; @@ -22,8 +23,11 @@ use k256::{ use sha3::{Digest, Keccak256}; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}; use tracing::{error, info}; -pub const SUPPORTED_CAPABILITIES: [(&str, u8); 2] = [("p2p", 5), ("eth", 68)]; -// pub const SUPPORTED_CAPABILITIES: [(&str, u8); 3] = [("p2p", 5), ("eth", 68), ("snap", 1)]; +const CAP_P2P: (Capability, u8) = (Capability::P2p, 5); +const CAP_ETH: (Capability, u8) = (Capability::Eth, 68); +//const CAP_SNAP: (Capability, u8) = (Capability::Snap, 1); +const SUPPORTED_CAPABILITIES: [(Capability, u8); 2] = [CAP_P2P, CAP_ETH]; +// pub const SUPPORTED_CAPABILITIES: [(&str, u8); 3] = [CAP_P2P, CAP_ETH, CAP_SNAP)]; pub(crate) type Aes256Ctr64BE = ctr::Ctr64BE; @@ -33,7 +37,7 @@ pub(crate) struct RLPxConnection { state: RLPxConnectionState, stream: S, storage: Store, - capabilities: Vec<(String, u8)>, + capabilities: Vec<(Capability, u8)>, } impl RLPxConnection { @@ -98,12 +102,8 @@ impl RLPxConnection { } pub async fn exchange_hello_messages(&mut self) -> Result<(), RLPxError> { - let supported_capabilities: Vec<(String, u8)> = SUPPORTED_CAPABILITIES - .into_iter() - .map(|(name, version)| (name.to_string(), version)) - .collect(); let hello_msg = Message::Hello(p2p::HelloMessage::new( - supported_capabilities.clone(), + SUPPORTED_CAPABILITIES.to_vec(), PublicKey::from(self.signer.verifying_key()), )); @@ -118,7 +118,7 @@ impl RLPxConnection { // Check if we have any capability in common for cap in self.capabilities.clone() { - if supported_capabilities.contains(&cap) { + if SUPPORTED_CAPABILITIES.contains(&cap) { return Ok(()); } } @@ -169,7 +169,7 @@ impl RLPxConnection { async fn start_capabilities(&mut self) -> Result<(), RLPxError> { // Sending eth Status if peer supports it - if self.capabilities.contains(&("eth".to_string(), 68u8)) { + if self.capabilities.contains(&CAP_ETH) { let status = backend::get_status(&self.storage).unwrap(); info!("Status message sent: {status:?}"); self.send(Message::Status(status)).await; diff --git a/crates/networking/p2p/rlpx/p2p.rs b/crates/networking/p2p/rlpx/p2p.rs index 852e66545..7795f5fb5 100644 --- a/crates/networking/p2p/rlpx/p2p.rs +++ b/crates/networking/p2p/rlpx/p2p.rs @@ -1,7 +1,8 @@ use bytes::BufMut; use ethereum_rust_core::H512; use ethereum_rust_rlp::{ - encode::RLPEncode as _, + decode::RLPDecode, + encode::RLPEncode, error::{RLPDecodeError, RLPEncodeError}, structs::{Decoder, Encoder}, }; @@ -15,14 +16,43 @@ use super::{ utils::{pubkey2id, snappy_encode}, }; +#[derive(Debug, Clone, PartialEq)] +pub(crate) enum Capability { + P2p, + Eth, + Snap, +} + +impl RLPEncode for Capability { + fn encode(&self, buf: &mut dyn BufMut) { + match self { + Self::P2p => "p2p".encode(buf), + Self::Eth => "eth".encode(buf), + Self::Snap => "snap".encode(buf), + } + } +} + +impl RLPDecode for Capability { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let (cap_string, rest) = String::decode_unfinished(rlp)?; + match cap_string.as_str() { + "p2p" => Ok((Capability::P2p, rest)), + "eth" => Ok((Capability::Eth, rest)), + "snap" => Ok((Capability::Snap, rest)), + _ => return Err(RLPDecodeError::UnexpectedString), + } + } +} + #[derive(Debug)] pub(crate) struct HelloMessage { - pub(crate) capabilities: Vec<(String, u8)>, + pub(crate) capabilities: Vec<(Capability, u8)>, pub(crate) node_id: PublicKey, } impl HelloMessage { - pub fn new(capabilities: Vec<(String, u8)>, node_id: PublicKey) -> Self { + pub fn new(capabilities: Vec<(Capability, u8)>, node_id: PublicKey) -> Self { Self { capabilities, node_id, @@ -55,7 +85,7 @@ impl RLPxMessage for HelloMessage { // TODO: store client id for debugging purposes // [[cap1, capVersion1], [cap2, capVersion2], ...] - let (capabilities, decoder): (Vec<(String, u8)>, _) = + let (capabilities, decoder): (Vec<(Capability, u8)>, _) = decoder.decode_field("capabilities").unwrap(); // This field should be ignored From 8ea26440150415a22a49987eb1d22f3b9bc813b1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 25 Oct 2024 18:24:13 -0300 Subject: [PATCH 034/345] Add snap messages to liste loop --- crates/networking/p2p/rlpx/connection.rs | 7 +++++-- crates/networking/p2p/rlpx/error.rs | 3 +++ crates/networking/p2p/rlpx/frame.rs | 1 - crates/networking/p2p/rlpx/message.rs | 14 +++++++++++++- 4 files changed, 21 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index bb34a2bc7..222bd0e2f 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,6 +1,5 @@ use crate::{ - rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, - MAX_DISC_PACKET_SIZE, + rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, snap::process_account_range_request, MAX_DISC_PACKET_SIZE }; use super::{ @@ -147,6 +146,10 @@ impl RLPxConnection { Message::Ping(_) => info!("Received Ping"), Message::Pong(_) => info!("Received Pong"), Message::Status(_) => info!("Received Status"), + Message::GetAccountRange(req) => { + let response = process_account_range_request(req, self.storage.clone())?; + dbg!(self.send(Message::AccountRange(response)).await) + }, // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 4177ea10f..83b158762 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,4 +1,5 @@ use crate::rlpx::message::Message; +use ethereum_rust_storage::error::StoreError; use thiserror::Error; // TODO improve errors @@ -10,4 +11,6 @@ pub(crate) enum RLPxError { InvalidState(String), #[error("Unexpected message: {0}")] UnexpectedMessage(Message), + #[error(transparent)] + Store(#[from] StoreError), } diff --git a/crates/networking/p2p/rlpx/frame.rs b/crates/networking/p2p/rlpx/frame.rs index d2b007704..9c5c8d266 100644 --- a/crates/networking/p2p/rlpx/frame.rs +++ b/crates/networking/p2p/rlpx/frame.rs @@ -65,7 +65,6 @@ pub(crate) async fn write( }; state.egress_mac.update(frame_mac_seed); let frame_mac = state.egress_mac.clone().finalize(); - // Send frame-mac stream.write_all(&frame_mac[..16]).await.unwrap(); } diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 8f06159be..9ed648118 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -4,6 +4,9 @@ use std::fmt::Display; use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; +use super::snap::{AccountRange, GetAccountRange}; + +use ethereum_rust_rlp::encode::RLPEncode; pub trait RLPxMessage: Sized { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError>; @@ -17,16 +20,21 @@ pub(crate) enum Message { Ping(PingMessage), Pong(PongMessage), Status(StatusMessage), + // snap capability + GetAccountRange(GetAccountRange), + AccountRange(AccountRange) } impl Message { pub fn decode(msg_id: u8, msg_data: &[u8]) -> Result { - match msg_id { + match dbg!(msg_id) { 0x00 => Ok(Message::Hello(HelloMessage::decode(msg_data)?)), 0x01 => Ok(Message::Disconnect(DisconnectMessage::decode(msg_data)?)), 0x02 => Ok(Message::Ping(PingMessage::decode(msg_data)?)), 0x03 => Ok(Message::Pong(PongMessage::decode(msg_data)?)), 0x10 => Ok(Message::Status(StatusMessage::decode(msg_data)?)), + 0x21 => Ok(Message::GetAccountRange(GetAccountRange::decode(msg_data)?)), + 0x22 => Ok(Message::AccountRange(AccountRange::decode(msg_data)?)), _ => Err(RLPDecodeError::MalformedData), } } @@ -38,6 +46,8 @@ impl Message { Message::Ping(msg) => msg.encode(buf), Message::Pong(msg) => msg.encode(buf), Message::Status(msg) => msg.encode(buf), + Message::GetAccountRange(msg) => {0x21_u8.encode(buf); msg.encode(buf)}, + Message::AccountRange(msg) => {0x22_u8.encode(buf); msg.encode(buf)}, } } } @@ -50,6 +60,8 @@ impl Display for Message { Message::Ping(_) => "p2p:Ping".fmt(f), Message::Pong(_) => "p2p:Pong".fmt(f), Message::Status(_) => "eth:Status".fmt(f), + Message::GetAccountRange(_) => "snap::GetAccountRange".fmt(f), + Message::AccountRange(_) => "snap::AccountRange".fmt(f), } } } From 42049abc7f6613ee4e81e07bd6e869d6ab27d7b3 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 09:29:23 -0300 Subject: [PATCH 035/345] Made Status test to pass --- Makefile | 2 +- cmd/ethereum_rust/ethereum_rust.rs | 4 ++++ crates/networking/p2p/rlpx/p2p.rs | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/Makefile b/Makefile index e50324028..743f10df1 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := 3be4465a45c421651d765f4a28702962567b40e6 +HIVE_REVISION := ccf28e5c3e940b2bc4b4f387317ee6a46f5d15c8 # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 32ad9e3b3..538389ec4 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -135,6 +135,10 @@ async fn main() { block.header.number, hash, result ); } + store + .set_canonical_block(block.header.number, hash) + .unwrap(); + store.update_latest_block_number(block.header.number).unwrap(); } info!("Added {} blocks to blockchain", size); } diff --git a/crates/networking/p2p/rlpx/p2p.rs b/crates/networking/p2p/rlpx/p2p.rs index 7795f5fb5..4521c5bef 100644 --- a/crates/networking/p2p/rlpx/p2p.rs +++ b/crates/networking/p2p/rlpx/p2p.rs @@ -40,7 +40,7 @@ impl RLPDecode for Capability { "p2p" => Ok((Capability::P2p, rest)), "eth" => Ok((Capability::Eth, rest)), "snap" => Ok((Capability::Snap, rest)), - _ => return Err(RLPDecodeError::UnexpectedString), + _ => Err(RLPDecodeError::UnexpectedString), } } } From 940b6f39148977b2243a831108e771c48815869b Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 09:40:31 -0300 Subject: [PATCH 036/345] Fixed format --- cmd/ethereum_rust/ethereum_rust.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 538389ec4..eb59ef27b 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -138,7 +138,9 @@ async fn main() { store .set_canonical_block(block.header.number, hash) .unwrap(); - store.update_latest_block_number(block.header.number).unwrap(); + store + .update_latest_block_number(block.header.number) + .unwrap(); } info!("Added {} blocks to blockchain", size); } From 8047f415b7c99dd80a93d87712a2556deee59cb8 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 09:50:24 -0300 Subject: [PATCH 037/345] Removed required Debug format on error printing --- cmd/ethereum_rust/ethereum_rust.rs | 6 +++--- crates/blockchain/error.rs | 10 ++++------ 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index eb59ef27b..d06c485fe 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -129,10 +129,10 @@ async fn main() { block.header.number, hash ); let result = add_block(&block, &store); - if result.is_err() { + if let Some(error) = result.err() { warn!( - "Failed to add block {} with hash {:#x}: {:?}.", - block.header.number, hash, result + "Failed to add block {} with hash {:#x}: {}.", + block.header.number, hash, error ); } store diff --git a/crates/blockchain/error.rs b/crates/blockchain/error.rs index e2e7c57ec..9446795b4 100644 --- a/crates/blockchain/error.rs +++ b/crates/blockchain/error.rs @@ -1,10 +1,8 @@ -use thiserror::Error; - use ethereum_rust_core::types::InvalidBlockHeaderError; use ethereum_rust_storage::error::StoreError; use ethereum_rust_vm::EvmError; -#[derive(Debug, Error)] +#[derive(Debug, thiserror::Error)] pub enum ChainError { #[error("Invalid Block: {0}")] InvalidBlock(#[from] InvalidBlockError), @@ -20,7 +18,7 @@ pub enum ChainError { EvmError(#[from] EvmError), } -#[derive(Debug, Error)] +#[derive(Debug, thiserror::Error)] pub enum InvalidBlockError { #[error("World State Root does not match the one in the header after executing")] StateRootMismatch, @@ -36,7 +34,7 @@ pub enum InvalidBlockError { BlobGasUsedMismatch, } -#[derive(Debug, Error)] +#[derive(Debug, thiserror::Error)] pub enum MempoolError { #[error("No block header")] NoBlockHeaderError, @@ -67,7 +65,7 @@ pub enum ForkChoiceElement { Finalized, } -#[derive(Debug, Error)] +#[derive(Debug, thiserror::Error)] pub enum InvalidForkChoice { #[error("DB error: {0}")] StoreError(#[from] StoreError), From f4746170f33030e91e20c1c2ec2f52446a275968 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 11:56:17 -0300 Subject: [PATCH 038/345] Trying previous commit from hive to see if CI works --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 743f10df1..0b58411fd 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := ccf28e5c3e940b2bc4b4f387317ee6a46f5d15c8 +HIVE_REVISION := 5c9e4cc3f43ab5f7401f3d7e7a79c3b86044ee47 # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). From 1d5223be4741b7d780c9b506b1e448a7db22d73e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 12:07:28 -0300 Subject: [PATCH 039/345] Fix `AccountRange message decoding --- crates/networking/p2p/rlpx/connection.rs | 9 ++- crates/networking/p2p/rlpx/message.rs | 12 +++- crates/networking/p2p/rlpx/snap.rs | 11 +++- crates/networking/p2p/snap.rs | 82 +++++++++++++----------- 4 files changed, 68 insertions(+), 46 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 222bd0e2f..da6eccb31 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,5 +1,7 @@ use crate::{ - rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, snap::process_account_range_request, MAX_DISC_PACKET_SIZE + rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, + snap::process_account_range_request, + MAX_DISC_PACKET_SIZE, }; use super::{ @@ -147,9 +149,10 @@ impl RLPxConnection { Message::Pong(_) => info!("Received Pong"), Message::Status(_) => info!("Received Status"), Message::GetAccountRange(req) => { - let response = process_account_range_request(req, self.storage.clone())?; + let response = + process_account_range_request(req, self.storage.clone())?; dbg!(self.send(Message::AccountRange(response)).await) - }, + } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 9ed648118..5ae609cbd 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -22,7 +22,7 @@ pub(crate) enum Message { Status(StatusMessage), // snap capability GetAccountRange(GetAccountRange), - AccountRange(AccountRange) + AccountRange(AccountRange), } impl Message { @@ -46,8 +46,14 @@ impl Message { Message::Ping(msg) => msg.encode(buf), Message::Pong(msg) => msg.encode(buf), Message::Status(msg) => msg.encode(buf), - Message::GetAccountRange(msg) => {0x21_u8.encode(buf); msg.encode(buf)}, - Message::AccountRange(msg) => {0x22_u8.encode(buf); msg.encode(buf)}, + Message::GetAccountRange(msg) => { + 0x21_u8.encode(buf); + msg.encode(buf) + } + Message::AccountRange(msg) => { + 0x22_u8.encode(buf); + msg.encode(buf) + } } } } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index c1ab07374..0dfbf3750 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -21,6 +21,12 @@ pub struct AccountStateSlim { pub code_hash: Bytes, } +#[derive(Debug)] +pub struct AccountRangeUnit { + pub hash: H256, + pub account: AccountStateSlim, +} + impl From for AccountStateSlim { fn from(value: AccountState) -> Self { let storage_root = if value.storage_root == *EMPTY_TRIE_HASH { @@ -138,8 +144,8 @@ pub(crate) struct AccountRange { // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, // List of (hash, account) pairs, accounts consists of RLP-encoded slim accounts - pub accounts: Vec<(H256, Vec)>, - pub proof: Vec>, + pub accounts: Vec, + pub proof: Vec, } impl RLPxMessage for AccountRange { @@ -174,6 +180,7 @@ impl RLPxMessage for AccountRange { }) } } + impl RLPEncode for AccountStateSlim { fn encode(&self, buf: &mut dyn BufMut) { Encoder::new(buf) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 6ef8c5c10..881abbe1a 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,32 +1,31 @@ +use bytes::Bytes; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use crate::rlpx::snap::{AccountRange, AccountStateSlim, GetAccountRange}; +use crate::rlpx::snap::{AccountRange, AccountRangeUnit, AccountStateSlim, GetAccountRange}; -#[allow(unused)] pub fn process_account_range_request( request: GetAccountRange, store: Store, ) -> Result { let mut accounts = vec![]; let mut bytes_used = 0; - for (k, v) in store.iter_accounts(request.root_hash) { - if k >= request.starting_hash { - let account = AccountStateSlim::from(v).encode_to_vec(); + for (hash, account) in store.iter_accounts(request.root_hash) { + if hash >= request.starting_hash { + let account = AccountStateSlim::from(account); // size of hash + size of account - bytes_used += 32 + account.len() as u64; - accounts.push((k, account)); + bytes_used += 32 + account.encoded_len() as u64; + accounts.push(AccountRangeUnit { hash, account }); } - if k >= request.limit_hash || bytes_used >= request.response_bytes { + if hash >= request.limit_hash || bytes_used >= request.response_bytes { break; } } - let proof = store.get_account_range_proof( - request.root_hash, - request.starting_hash, - request.limit_hash, - )?; - + let proof = store + .get_account_range_proof(request.root_hash, request.starting_hash, request.limit_hash)? + .iter() + .map(|bytes| Bytes::copy_from_slice(bytes)) + .collect(); Ok(AccountRange { id: request.id, accounts, @@ -34,6 +33,13 @@ pub fn process_account_range_request( }) } +impl AccountStateSlim { + // TODO: calculate this without encoding + fn encoded_len(&self) -> usize { + self.encode_to_vec().len() + } +} + #[cfg(test)] mod tests { use std::str::FromStr; @@ -82,9 +88,9 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 86); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); assert_eq!( - res.accounts.last().unwrap().0, + res.accounts.last().unwrap().hash, H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") .unwrap() ); @@ -103,9 +109,9 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 65); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); assert_eq!( - res.accounts.last().unwrap().0, + res.accounts.last().unwrap().hash, H256::from_str("0x2e6fe1362b3e388184fd7bf08e99e74170b26361624ffd1c5f646da7067b58b6") .unwrap() ); @@ -124,9 +130,9 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 44); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); assert_eq!( - res.accounts.last().unwrap().0, + res.accounts.last().unwrap().hash, H256::from_str("0x1c3f74249a4892081ba0634a819aec9ed25f34c7653f5719b9098487e65ab595") .unwrap() ); @@ -145,8 +151,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } #[test] @@ -162,8 +168,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } #[test] @@ -182,8 +188,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 2); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_SECOND); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_SECOND); } #[test] @@ -201,8 +207,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } #[test] @@ -220,8 +226,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } #[test] @@ -237,9 +243,9 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 86); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); assert_eq!( - res.accounts.last().unwrap().0, + res.accounts.last().unwrap().hash, H256::from_str("0x445cb5c1278fdce2f9cbdb681bdd76c52f8e50e41dbd9e220242a69ba99ac099") .unwrap() ); @@ -258,9 +264,9 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 86); - assert_eq!(res.accounts.first().unwrap().0, *HASH_SECOND); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_SECOND); assert_eq!( - res.accounts.last().unwrap().0, + res.accounts.last().unwrap().hash, H256::from_str("0x4615e5f5df5b25349a00ad313c6cd0436b6c08ee5826e33a018661997f85ebaa") .unwrap() ); @@ -285,8 +291,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } #[test] @@ -304,8 +310,8 @@ mod tests { let res = process_account_range_request(request, store).unwrap(); // Check test invariants assert_eq!(res.accounts.len(), 1); - assert_eq!(res.accounts.first().unwrap().0, *HASH_FIRST); - assert_eq!(res.accounts.last().unwrap().0, *HASH_FIRST); + assert_eq!(res.accounts.first().unwrap().hash, *HASH_FIRST); + assert_eq!(res.accounts.last().unwrap().hash, *HASH_FIRST); } // Initial state setup for hive snap tests From 80329c76cf9c4ca87d4feb8d1c121bee705fb5e7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 13:01:03 -0300 Subject: [PATCH 040/345] Fix --- crates/networking/p2p/snap.rs | 4 ++-- crates/storage/store/storage.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 881abbe1a..549bf11f2 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -853,10 +853,10 @@ mod tests { let store = Store::new("null", EngineType::InMemory).unwrap(); let mut state_trie = store.new_state_trie_for_test(); for (address, account) in accounts { - let hashed_address = H256::from_str(address).unwrap(); + let hashed_address = H256::from_str(address).unwrap().as_bytes().to_vec(); let account = AccountState::from(AccountStateSlim::decode(&account).unwrap()); state_trie - .insert(hashed_address.encode_to_vec(), account.encode_to_vec()) + .insert(hashed_address, account.encode_to_vec()) .unwrap(); } (store, state_trie.hash().unwrap()) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 145f00b74..93ab11cfe 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -686,7 +686,7 @@ impl Store { .content() .map_while(|(path, value)| { Some(( - H256::decode(&path).ok()?, + H256::from_slice(&path), AccountState::decode(&value).ok()?, )) }) From 297a37b70d669900dae8147881b23f433972ead0 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 13:19:00 -0300 Subject: [PATCH 041/345] fix --- crates/storage/store/storage.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 93ab11cfe..897c5177b 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -699,8 +699,8 @@ impl Store { limit_hash: H256, ) -> Result>, StoreError> { let state_trie = self.engine.open_state_trie(state_root); - let mut proof = state_trie.get_proof(&starting_hash.encode_to_vec())?; - proof.extend_from_slice(&state_trie.get_proof(&limit_hash.encode_to_vec())?); + let mut proof = state_trie.get_proof(&starting_hash.as_bytes().to_vec())?; + proof.extend_from_slice(&state_trie.get_proof(&limit_hash.as_bytes().to_vec())?); Ok(proof) } From f25fb1a0552baf955db9bb5f282eaaf7955f4806 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 15:07:36 -0300 Subject: [PATCH 042/345] Fix: build proof for last account not limit hash --- crates/networking/p2p/snap.rs | 2 +- crates/storage/store/storage.rs | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 549bf11f2..5c985fea7 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -22,7 +22,7 @@ pub fn process_account_range_request( } } let proof = store - .get_account_range_proof(request.root_hash, request.starting_hash, request.limit_hash)? + .get_account_range_proof(request.root_hash, request.starting_hash, accounts.last().map(|acc| acc.hash))? .iter() .map(|bytes| Bytes::copy_from_slice(bytes)) .collect(); diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 897c5177b..f14d57d6f 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -696,11 +696,13 @@ impl Store { &self, state_root: H256, starting_hash: H256, - limit_hash: H256, + last_hash: Option, ) -> Result>, StoreError> { let state_trie = self.engine.open_state_trie(state_root); let mut proof = state_trie.get_proof(&starting_hash.as_bytes().to_vec())?; - proof.extend_from_slice(&state_trie.get_proof(&limit_hash.as_bytes().to_vec())?); + if let Some(last_hash) = last_hash { + proof.extend_from_slice(&state_trie.get_proof(&last_hash.as_bytes().to_vec())?); + } Ok(proof) } From 0fdf562f1c2a6256e3cd4f094d04df59dd959295 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 15:39:24 -0300 Subject: [PATCH 043/345] Using apply_fork_choice to set last block number and make blocks cannonical --- cmd/ethereum_rust/ethereum_rust.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index d06c485fe..ea463f320 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -1,6 +1,7 @@ use bytes::Bytes; use directories::ProjectDirs; use ethereum_rust_blockchain::add_block; +use ethereum_rust_blockchain::fork_choice::apply_fork_choice; use ethereum_rust_core::types::{Block, Genesis}; use ethereum_rust_core::H256; use ethereum_rust_net::bootnode::BootNode; @@ -135,13 +136,9 @@ async fn main() { block.header.number, hash, error ); } - store - .set_canonical_block(block.header.number, hash) - .unwrap(); - store - .update_latest_block_number(block.header.number) - .unwrap(); + apply_fork_choice(&store, hash, hash, hash).unwrap(); } + info!("Added {} blocks to blockchain", size); } let jwt_secret = read_jwtsecret_file(authrpc_jwtsecret); From 12010b3e34b3f5624327a4c5056774c87f1bda63 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 16:07:49 -0300 Subject: [PATCH 044/345] Fmt + Return error instead of pancking if invalid root is used --- crates/networking/p2p/rlpx/connection.rs | 2 +- crates/networking/p2p/rlpx/message.rs | 2 +- crates/networking/p2p/rlpx/snap.rs | 18 ++++++++++++++++++ crates/networking/rpc/eth/filter.rs | 1 - crates/storage/trie/error.rs | 2 ++ crates/storage/trie/trie.rs | 6 +++--- crates/vm/levm/tests/tests.rs | 2 -- 7 files changed, 25 insertions(+), 8 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index da6eccb31..86d547a68 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -151,7 +151,7 @@ impl RLPxConnection { Message::GetAccountRange(req) => { let response = process_account_range_request(req, self.storage.clone())?; - dbg!(self.send(Message::AccountRange(response)).await) + self.send(Message::AccountRange(response)).await } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 5ae609cbd..9eaaecdff 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -27,7 +27,7 @@ pub(crate) enum Message { impl Message { pub fn decode(msg_id: u8, msg_data: &[u8]) -> Result { - match dbg!(msg_id) { + match msg_id { 0x00 => Ok(Message::Hello(HelloMessage::decode(msg_data)?)), 0x01 => Ok(Message::Disconnect(DisconnectMessage::decode(msg_data)?)), 0x02 => Ok(Message::Ping(PingMessage::decode(msg_data)?)), diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 0dfbf3750..a2995bae8 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -210,3 +210,21 @@ impl RLPDecode for AccountStateSlim { )) } } + +impl RLPEncode for AccountRangeUnit { + fn encode(&self, buf: &mut dyn BufMut) { + Encoder::new(buf) + .encode_field(&self.hash) + .encode_field(&self.account) + .finish(); + } +} + +impl RLPDecode for AccountRangeUnit { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let decoder = Decoder::new(rlp)?; + let (hash, decoder) = decoder.decode_field("hash")?; + let (account, decoder) = decoder.decode_field("account")?; + Ok((Self { hash, account }, decoder.finish()?)) + } +} \ No newline at end of file diff --git a/crates/networking/rpc/eth/filter.rs b/crates/networking/rpc/eth/filter.rs index bbeb2f9b7..a5e8c8f13 100644 --- a/crates/networking/rpc/eth/filter.rs +++ b/crates/networking/rpc/eth/filter.rs @@ -558,7 +558,6 @@ mod tests { .await .unwrap(); - dbg!(&response); assert!( response.get("result").is_some(), "Response should have a 'result' field" diff --git a/crates/storage/trie/error.rs b/crates/storage/trie/error.rs index f7cdf60e9..b451d8324 100644 --- a/crates/storage/trie/error.rs +++ b/crates/storage/trie/error.rs @@ -7,4 +7,6 @@ pub enum TrieError { LibmdbxError(anyhow::Error), #[error(transparent)] RLPDecode(#[from] RLPDecodeError), + #[error("Trie root doesn't match trie structure")] + InconsistentRoot } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 46113c4e5..e90f420b1 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -74,7 +74,7 @@ impl Trie { let root_node = self .state .get_node(root.clone())? - .expect("inconsistent internal tree structure"); + .ok_or(TrieError::InconsistentRoot)?; root_node.get(&self.state, NibbleSlice::new(path)) } else { Ok(None) @@ -109,7 +109,7 @@ impl Trie { let root_node = self .state .get_node(root)? - .expect("inconsistent internal tree structure"); + .ok_or(TrieError::InconsistentRoot)?; let (root_node, old_value) = root_node.remove(&mut self.state, NibbleSlice::new(&path))?; self.root = root_node @@ -151,7 +151,7 @@ impl Trie { let root_node = self .state .get_node(root.clone())? - .expect("inconsistent tree structure"); + .ok_or(TrieError::InconsistentRoot)?; root_node.get_path(&self.state, NibbleSlice::new(path), &mut node_path)?; Ok(node_path) } diff --git a/crates/vm/levm/tests/tests.rs b/crates/vm/levm/tests/tests.rs index 4ddd2e8ee..7aedbc923 100644 --- a/crates/vm/levm/tests/tests.rs +++ b/crates/vm/levm/tests/tests.rs @@ -3934,8 +3934,6 @@ fn balance_op() { let mut current_call_frame = vm.call_frames.pop().unwrap(); vm.execute(&mut current_call_frame); - dbg!(&vm); - assert_eq!( vm.current_call_frame_mut().stack.pop().unwrap(), U256::from(1234) From ed0186f514079d5b42c5c821c5745903381045b9 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 16:12:08 -0300 Subject: [PATCH 045/345] Calling apply_fork_choice only once for the whole chain --- cmd/ethereum_rust/ethereum_rust.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index ea463f320..5acad2428 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -136,9 +136,9 @@ async fn main() { block.header.number, hash, error ); } - apply_fork_choice(&store, hash, hash, hash).unwrap(); } + apply_fork_choice(&store, hash, hash, hash).unwrap(); info!("Added {} blocks to blockchain", size); } let jwt_secret = read_jwtsecret_file(authrpc_jwtsecret); From 42f2af3e053f3c85a25f7b188e19f3b9fb5587b6 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 16:13:04 -0300 Subject: [PATCH 046/345] Revert change --- crates/storage/trie/error.rs | 2 -- crates/storage/trie/trie.rs | 6 +++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/crates/storage/trie/error.rs b/crates/storage/trie/error.rs index b451d8324..f7cdf60e9 100644 --- a/crates/storage/trie/error.rs +++ b/crates/storage/trie/error.rs @@ -7,6 +7,4 @@ pub enum TrieError { LibmdbxError(anyhow::Error), #[error(transparent)] RLPDecode(#[from] RLPDecodeError), - #[error("Trie root doesn't match trie structure")] - InconsistentRoot } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index e90f420b1..46113c4e5 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -74,7 +74,7 @@ impl Trie { let root_node = self .state .get_node(root.clone())? - .ok_or(TrieError::InconsistentRoot)?; + .expect("inconsistent internal tree structure"); root_node.get(&self.state, NibbleSlice::new(path)) } else { Ok(None) @@ -109,7 +109,7 @@ impl Trie { let root_node = self .state .get_node(root)? - .ok_or(TrieError::InconsistentRoot)?; + .expect("inconsistent internal tree structure"); let (root_node, old_value) = root_node.remove(&mut self.state, NibbleSlice::new(&path))?; self.root = root_node @@ -151,7 +151,7 @@ impl Trie { let root_node = self .state .get_node(root.clone())? - .ok_or(TrieError::InconsistentRoot)?; + .expect("inconsistent tree structure"); root_node.get_path(&self.state, NibbleSlice::new(path), &mut node_path)?; Ok(node_path) } From 61ad32954e8cffe739fde83f89f177fab0f2fe40 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 16:15:13 -0300 Subject: [PATCH 047/345] Return empty proof if root is missing --- crates/networking/p2p/rlpx/connection.rs | 2 +- crates/networking/p2p/rlpx/snap.rs | 2 +- crates/networking/p2p/snap.rs | 6 +++++- crates/storage/store/storage.rs | 5 +---- crates/storage/trie/trie.rs | 8 +++----- 5 files changed, 11 insertions(+), 12 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 86d547a68..b3c1deafb 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -151,7 +151,7 @@ impl RLPxConnection { Message::GetAccountRange(req) => { let response = process_account_range_request(req, self.storage.clone())?; - self.send(Message::AccountRange(response)).await + self.send(Message::AccountRange(response)).await } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index a2995bae8..82a03cdc4 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -227,4 +227,4 @@ impl RLPDecode for AccountRangeUnit { let (account, decoder) = decoder.decode_field("account")?; Ok((Self { hash, account }, decoder.finish()?)) } -} \ No newline at end of file +} diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 5c985fea7..4fe66c8c3 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -22,7 +22,11 @@ pub fn process_account_range_request( } } let proof = store - .get_account_range_proof(request.root_hash, request.starting_hash, accounts.last().map(|acc| acc.hash))? + .get_account_range_proof( + request.root_hash, + request.starting_hash, + accounts.last().map(|acc| acc.hash), + )? .iter() .map(|bytes| Bytes::copy_from_slice(bytes)) .collect(); diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index f14d57d6f..e3611f8e1 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -685,10 +685,7 @@ impl Store { .into_iter() .content() .map_while(|(path, value)| { - Some(( - H256::from_slice(&path), - AccountState::decode(&value).ok()?, - )) + Some((H256::from_slice(&path), AccountState::decode(&value).ok()?)) }) } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 46113c4e5..45b098f03 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -148,11 +148,9 @@ impl Trie { if let NodeHash::Inline(node) = root { node_path.push(node.to_vec()); } - let root_node = self - .state - .get_node(root.clone())? - .expect("inconsistent tree structure"); - root_node.get_path(&self.state, NibbleSlice::new(path), &mut node_path)?; + if let Some(root_node) = self.state.get_node(root.clone())? { + root_node.get_path(&self.state, NibbleSlice::new(path), &mut node_path)?; + } Ok(node_path) } From 13c44cfe1b38c6379875955aa08723f326bfd8a9 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 16:22:42 -0300 Subject: [PATCH 048/345] Calling apply_fork_choice only once for the whole chain --- cmd/ethereum_rust/ethereum_rust.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 5acad2428..4ed1ebfcc 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -123,7 +123,7 @@ async fn main() { if let Some(chain_rlp_path) = matches.get_one::("import") { let blocks = read_chain_file(chain_rlp_path); let size = blocks.len(); - for block in blocks { + for block in &blocks { let hash = block.header.compute_block_hash(); info!( "Adding block {} with hash {:#x}.", @@ -137,8 +137,10 @@ async fn main() { ); } } - - apply_fork_choice(&store, hash, hash, hash).unwrap(); + if let Some(last_block) = blocks.last() { + let hash = last_block.header.compute_block_hash(); + apply_fork_choice(&store, hash, hash, hash).unwrap(); + } info!("Added {} blocks to blockchain", size); } let jwt_secret = read_jwtsecret_file(authrpc_jwtsecret); From ecb682f6010ef6cbc530043dd87df42729e8190a Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Mon, 28 Oct 2024 16:24:54 -0300 Subject: [PATCH 049/345] Clippy fix --- cmd/ethereum_rust/ethereum_rust.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 4ed1ebfcc..ce4903ec0 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -129,7 +129,7 @@ async fn main() { "Adding block {} with hash {:#x}.", block.header.number, hash ); - let result = add_block(&block, &store); + let result = add_block(block, &store); if let Some(error) = result.err() { warn!( "Failed to add block {} with hash {:#x}: {}.", From f8bd1dc853f25d7325a143f0bc09061972910f3a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 16:27:46 -0300 Subject: [PATCH 050/345] use trait method --- crates/networking/p2p/snap.rs | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 4fe66c8c3..8efa3f5f1 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -12,10 +12,12 @@ pub fn process_account_range_request( let mut bytes_used = 0; for (hash, account) in store.iter_accounts(request.root_hash) { if hash >= request.starting_hash { - let account = AccountStateSlim::from(account); - // size of hash + size of account - bytes_used += 32 + account.encoded_len() as u64; - accounts.push(AccountRangeUnit { hash, account }); + let account = AccountRangeUnit { + hash, + account: AccountStateSlim::from(account), + }; + bytes_used += account.length() as u64; + accounts.push(account); } if hash >= request.limit_hash || bytes_used >= request.response_bytes { break; @@ -37,13 +39,6 @@ pub fn process_account_range_request( }) } -impl AccountStateSlim { - // TODO: calculate this without encoding - fn encoded_len(&self) -> usize { - self.encode_to_vec().len() - } -} - #[cfg(test)] mod tests { use std::str::FromStr; From 31aa8aced7d651e656eedaef3d8a5892009bae10 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 18:21:42 -0300 Subject: [PATCH 051/345] Update hive revision --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 0b58411fd..95f8b62cc 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := 5c9e4cc3f43ab5f7401f3d7e7a79c3b86044ee47 +HIVE_REVISION := fdc6a6d04caf3e339b4b5409c63e567903769fa8 #TODO: move to main branch once the PR is merged # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). From 34d8d846e3d3773c50fe7e84af5c81ece457a5cc Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 28 Oct 2024 18:51:06 -0300 Subject: [PATCH 052/345] Reorganize module --- crates/networking/p2p/rlpx/snap.rs | 177 +++++++++++++---------------- 1 file changed, 79 insertions(+), 98 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 82a03cdc4..12007ac82 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -13,67 +13,11 @@ use snap::raw::Decoder as SnappyDecoder; use super::{message::RLPxMessage, utils::snappy_encode}; -#[derive(Debug)] -pub struct AccountStateSlim { - pub nonce: u64, - pub balance: U256, - pub storage_root: Bytes, - pub code_hash: Bytes, -} +// Snap Capability Messages -#[derive(Debug)] -pub struct AccountRangeUnit { - pub hash: H256, - pub account: AccountStateSlim, -} - -impl From for AccountStateSlim { - fn from(value: AccountState) -> Self { - let storage_root = if value.storage_root == *EMPTY_TRIE_HASH { - Bytes::new() - } else { - Bytes::copy_from_slice(value.storage_root.as_bytes()) - }; - let code_hash = if value.code_hash == *EMPTY_KECCACK_HASH { - Bytes::new() - } else { - Bytes::copy_from_slice(value.code_hash.as_bytes()) - }; - Self { - nonce: value.nonce, - balance: value.balance, - storage_root, - code_hash, - } - } -} - -impl From for AccountState { - fn from(value: AccountStateSlim) -> Self { - let storage_root = if value.storage_root.is_empty() { - *EMPTY_TRIE_HASH - } else { - H256::from_slice(value.storage_root.as_ref()) - }; - let code_hash = if value.code_hash.is_empty() { - *EMPTY_KECCACK_HASH - } else { - H256::from_slice(value.code_hash.as_ref()) - }; - Self { - nonce: value.nonce, - balance: value.balance, - storage_root, - code_hash, - } - } -} - -// https://github.com/ethereum/devp2p/blob/master/caps/snap.md#getaccountrange-0x00 #[derive(Debug)] pub(crate) struct GetAccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response - // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, pub root_hash: H256, pub starting_hash: H256, @@ -81,22 +25,12 @@ pub(crate) struct GetAccountRange { pub response_bytes: u64, } -impl GetAccountRange { - pub fn new( - id: u64, - root_hash: H256, - starting_hash: H256, - limit_hash: H256, - response_bytes: u64, - ) -> Self { - Self { - id, - root_hash, - starting_hash, - limit_hash, - response_bytes, - } - } +#[derive(Debug)] +pub(crate) struct AccountRange { + // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response + pub id: u64, + pub accounts: Vec, + pub proof: Vec, } impl RLPxMessage for GetAccountRange { @@ -127,27 +61,16 @@ impl RLPxMessage for GetAccountRange { let (limit_hash, decoder): (H256, _) = decoder.decode_field("limitHash")?; let (response_bytes, _): (u64, _) = decoder.decode_field("responseBytes")?; - Ok(Self::new( + Ok(Self { id, root_hash, starting_hash, limit_hash, response_bytes, - )) + }) } } -// https://github.com/ethereum/devp2p/blob/master/caps/snap.md#accountrange-0x01 -#[derive(Debug)] -pub(crate) struct AccountRange { - // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response - // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages - pub id: u64, - // List of (hash, account) pairs, accounts consists of RLP-encoded slim accounts - pub accounts: Vec, - pub proof: Vec, -} - impl RLPxMessage for AccountRange { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { let mut encoded_data = vec![]; @@ -181,6 +104,40 @@ impl RLPxMessage for AccountRange { } } +// Intermediate structures + +#[derive(Debug)] +pub struct AccountRangeUnit { + pub hash: H256, + pub account: AccountStateSlim, +} + +#[derive(Debug)] +pub struct AccountStateSlim { + pub nonce: u64, + pub balance: U256, + pub storage_root: Bytes, + pub code_hash: Bytes, +} + +impl RLPEncode for AccountRangeUnit { + fn encode(&self, buf: &mut dyn BufMut) { + Encoder::new(buf) + .encode_field(&self.hash) + .encode_field(&self.account) + .finish(); + } +} + +impl RLPDecode for AccountRangeUnit { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let decoder = Decoder::new(rlp)?; + let (hash, decoder) = decoder.decode_field("hash")?; + let (account, decoder) = decoder.decode_field("account")?; + Ok((Self { hash, account }, decoder.finish()?)) + } +} + impl RLPEncode for AccountStateSlim { fn encode(&self, buf: &mut dyn BufMut) { Encoder::new(buf) @@ -211,20 +168,44 @@ impl RLPDecode for AccountStateSlim { } } -impl RLPEncode for AccountRangeUnit { - fn encode(&self, buf: &mut dyn BufMut) { - Encoder::new(buf) - .encode_field(&self.hash) - .encode_field(&self.account) - .finish(); +impl From for AccountStateSlim { + fn from(value: AccountState) -> Self { + let storage_root = if value.storage_root == *EMPTY_TRIE_HASH { + Bytes::new() + } else { + Bytes::copy_from_slice(value.storage_root.as_bytes()) + }; + let code_hash = if value.code_hash == *EMPTY_KECCACK_HASH { + Bytes::new() + } else { + Bytes::copy_from_slice(value.code_hash.as_bytes()) + }; + Self { + nonce: value.nonce, + balance: value.balance, + storage_root, + code_hash, + } } } -impl RLPDecode for AccountRangeUnit { - fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - let decoder = Decoder::new(rlp)?; - let (hash, decoder) = decoder.decode_field("hash")?; - let (account, decoder) = decoder.decode_field("account")?; - Ok((Self { hash, account }, decoder.finish()?)) +impl From for AccountState { + fn from(value: AccountStateSlim) -> Self { + let storage_root = if value.storage_root.is_empty() { + *EMPTY_TRIE_HASH + } else { + H256::from_slice(value.storage_root.as_ref()) + }; + let code_hash = if value.code_hash.is_empty() { + *EMPTY_KECCACK_HASH + } else { + H256::from_slice(value.code_hash.as_ref()) + }; + Self { + nonce: value.nonce, + balance: value.balance, + storage_root, + code_hash, + } } } From 013a977ce94d52595b32f1d5f68bf05d12e69777 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 10:41:49 -0300 Subject: [PATCH 053/345] Update hive revision --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 95f8b62cc..a34ba2394 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := fdc6a6d04caf3e339b4b5409c63e567903769fa8 #TODO: move to main branch once the PR is merged +HIVE_REVISION := 448926bb93f6c025236a3b81c08d7c437d64852 #TODO: move to main branch once the PR is merged # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). From 66f759a56fdde5c715d15b110962abc3bc56db28 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 10:53:30 -0300 Subject: [PATCH 054/345] Add hive snap worflow --- .github/workflows/hive.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 7d3f90850..1f0fe044b 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -36,6 +36,9 @@ jobs: - simulation: discv4 name: "Devp2p discv4 tests" run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" + - simulation: snap + name: "Devp2p snap tests" + run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange" - simulation: engine name: "Engine tests" run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" From b1ea882a08c89ad9e0dbb595a64afea43870f95f Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Tue, 29 Oct 2024 14:32:02 -0300 Subject: [PATCH 055/345] Reverting hive version --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 0b58411fd..e50324028 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := 5c9e4cc3f43ab5f7401f3d7e7a79c3b86044ee47 +HIVE_REVISION := 3be4465a45c421651d765f4a28702962567b40e6 # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). From fe2fb778cf5d50d5ec86902f5f5413d63d8b07a7 Mon Sep 17 00:00:00 2001 From: Esteban Dimitroff Hodi Date: Tue, 29 Oct 2024 14:37:57 -0300 Subject: [PATCH 056/345] Removed unnecessary info! messages --- crates/networking/p2p/rlpx/connection.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 06de21c3c..642535904 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -97,7 +97,6 @@ impl RLPxConnection { info!("Completed handshake!"); self.exchange_hello_messages().await?; - info!("Completed Hello roundtrip!"); Ok(()) } @@ -108,12 +107,10 @@ impl RLPxConnection { )); self.send(hello_msg).await; - info!("Hello message sent!"); // Receive Hello message match self.receive().await { Message::Hello(hello_message) => { - info!("Hello message received {hello_message:?}"); self.capabilities = hello_message.capabilities; // Check if we have any capability in common @@ -171,7 +168,6 @@ impl RLPxConnection { // Sending eth Status if peer supports it if self.capabilities.contains(&CAP_ETH) { let status = backend::get_status(&self.storage).unwrap(); - info!("Status message sent: {status:?}"); self.send(Message::Status(status)).await; } // TODO: add new capabilities startup when required (eg. snap) @@ -194,7 +190,6 @@ impl RLPxConnection { auth_message.put_slice(&msg); self.stream.write_all(&auth_message).await.unwrap(); - info!("Sent auth message correctly!"); self.state = RLPxConnectionState::InitiatedAuth(InitiatedAuth::new( initiator_state, @@ -220,7 +215,6 @@ impl RLPxConnection { ack_message.put_slice(&msg); self.stream.write_all(&ack_message).await.unwrap(); - info!("Sent ack message correctly!"); self.state = RLPxConnectionState::Established(Box::new(Established::for_receiver( received_auth_state, @@ -251,7 +245,6 @@ impl RLPxConnection { let auth_bytes = &buf[..msg_size + 2]; let msg = &buf[2..msg_size + 2]; let (auth, remote_ephemeral_key) = decode_auth_message(&secret_key, msg, auth_data); - info!("Received auth message correctly!"); // Build next state self.state = RLPxConnectionState::ReceivedAuth(ReceivedAuth::new( @@ -287,7 +280,6 @@ impl RLPxConnection { let msg = &buf[2..msg_size + 2]; let ack = decode_ack_message(&secret_key, msg, ack_data); let remote_ephemeral_key = ack.get_ephemeral_pubkey().unwrap(); - info!("Received ack message correctly!"); // Build next state self.state = RLPxConnectionState::Established(Box::new(Established::for_initiator( From 7062b1bd06aaae31a7efc6637a04ef692117af2a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 15:18:34 -0300 Subject: [PATCH 057/345] Add messages + inner structs --- crates/networking/p2p/rlpx/message.rs | 18 +++- crates/networking/p2p/rlpx/snap.rs | 123 ++++++++++++++++++++++++-- 2 files changed, 135 insertions(+), 6 deletions(-) diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 9eaaecdff..766e66cdc 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -4,7 +4,7 @@ use std::fmt::Display; use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; -use super::snap::{AccountRange, GetAccountRange}; +use super::snap::{AccountRange, GetAccountRange, GetStorageRanges, StorageRanges}; use ethereum_rust_rlp::encode::RLPEncode; @@ -23,6 +23,8 @@ pub(crate) enum Message { // snap capability GetAccountRange(GetAccountRange), AccountRange(AccountRange), + GetStorageRanges(GetStorageRanges), + StorageRanges(StorageRanges), } impl Message { @@ -35,6 +37,10 @@ impl Message { 0x10 => Ok(Message::Status(StatusMessage::decode(msg_data)?)), 0x21 => Ok(Message::GetAccountRange(GetAccountRange::decode(msg_data)?)), 0x22 => Ok(Message::AccountRange(AccountRange::decode(msg_data)?)), + 0x23 => Ok(Message::GetStorageRanges(GetStorageRanges::decode( + msg_data, + )?)), + 0x24 => Ok(Message::StorageRanges(StorageRanges::decode(msg_data)?)), _ => Err(RLPDecodeError::MalformedData), } } @@ -54,6 +60,14 @@ impl Message { 0x22_u8.encode(buf); msg.encode(buf) } + Message::GetStorageRanges(msg) => { + 0x23_u8.encode(buf); + msg.encode(buf) + } + Message::StorageRanges(msg) => { + 0x24_u8.encode(buf); + msg.encode(buf) + } } } } @@ -68,6 +82,8 @@ impl Display for Message { Message::Status(_) => "eth:Status".fmt(f), Message::GetAccountRange(_) => "snap::GetAccountRange".fmt(f), Message::AccountRange(_) => "snap::AccountRange".fmt(f), + Message::GetStorageRanges(_) => "snap::GetStorageRanges".fmt(f), + Message::StorageRanges(_) => "snap::StorageRanges".fmt(f), } } } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 12007ac82..be6151273 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -33,6 +33,23 @@ pub(crate) struct AccountRange { pub proof: Vec, } +#[derive(Debug)] +pub(crate) struct GetStorageRanges { + pub id: u64, + pub root_hash: H256, + pub account_hashes: Vec, + pub starting_hash: H256, + pub limit_hash: H256, + pub response_bytes: u64, +} + +#[derive(Debug)] +pub(crate) struct StorageRanges { + pub id: u64, + pub slots: Vec, + pub proof: Vec, +} + impl RLPxMessage for GetAccountRange { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { let mut encoded_data = vec![]; @@ -55,11 +72,12 @@ impl RLPxMessage for GetAccountRange { .decompress_vec(msg_data) .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; let decoder = Decoder::new(&decompressed_data)?; - let (id, decoder): (u64, _) = decoder.decode_field("request-id")?; - let (root_hash, decoder): (H256, _) = decoder.decode_field("rootHash")?; - let (starting_hash, decoder): (H256, _) = decoder.decode_field("startingHash")?; - let (limit_hash, decoder): (H256, _) = decoder.decode_field("limitHash")?; - let (response_bytes, _): (u64, _) = decoder.decode_field("responseBytes")?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (root_hash, decoder) = decoder.decode_field("rootHash")?; + let (starting_hash, decoder) = decoder.decode_field("startingHash")?; + let (limit_hash, decoder) = decoder.decode_field("limitHash")?; + let (response_bytes, decoder) = decoder.decode_field("responseBytes")?; + decoder.finish()?; Ok(Self { id, @@ -104,6 +122,77 @@ impl RLPxMessage for AccountRange { } } +impl RLPxMessage for GetStorageRanges { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.root_hash) + .encode_field(&self.account_hashes) + .encode_field(&self.starting_hash) + .encode_field(&self.limit_hash) + .encode_field(&self.response_bytes) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (root_hash, decoder) = decoder.decode_field("rootHash")?; + let (account_hashes, decoder) = decoder.decode_field("accountHashes")?; + let (starting_hash, decoder) = decoder.decode_field("startingHash")?; + let (limit_hash, decoder) = decoder.decode_field("limitHash")?; + let (response_bytes, decoder) = decoder.decode_field("responseBytes")?; + decoder.finish()?; + + Ok(Self { + id, + root_hash, + starting_hash, + account_hashes, + limit_hash, + response_bytes, + }) + } +} + +impl RLPxMessage for StorageRanges { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.slots) + .encode_field(&self.proof) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (slots, decoder) = decoder.decode_field("slots")?; + let (proof, decoder) = decoder.decode_field("proof")?; + decoder.finish()?; + + Ok(Self { id, slots, proof }) + } +} + // Intermediate structures #[derive(Debug)] @@ -120,6 +209,12 @@ pub struct AccountStateSlim { pub code_hash: Bytes, } +#[derive(Debug)] +pub struct StorageSlot { + hash: H256, + data: U256, +} + impl RLPEncode for AccountRangeUnit { fn encode(&self, buf: &mut dyn BufMut) { Encoder::new(buf) @@ -209,3 +304,21 @@ impl From for AccountState { } } } + +impl RLPEncode for StorageSlot { + fn encode(&self, buf: &mut dyn BufMut) { + Encoder::new(buf) + .encode_field(&self.hash) + .encode_field(&self.data) + .finish(); + } +} + +impl RLPDecode for StorageSlot { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let decoder = Decoder::new(rlp)?; + let (hash, decoder) = decoder.decode_field("hash")?; + let (data, decoder) = decoder.decode_field("data")?; + Ok((Self { hash, data }, decoder.finish()?)) + } +} From d609cae745185814d21691932b30f38fa6f2c12d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 16:45:12 -0300 Subject: [PATCH 058/345] progress --- crates/storage/store/storage.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 90076c429..52330863b 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -689,6 +689,24 @@ impl Store { }) } + // Returns an iterator across all accounts in the state trie given by the state_root + // Does not check that the state_root is valid + pub fn iter_storage(&self, state_root: H256, hashed_address: H256) -> Result>, StoreError> { + let state_trie = self.engine.open_state_trie(state_root); + let Some(account_rlp) = state_trie.get(&hashed_address.as_bytes().to_vec())? else { + return Ok(None) + }; + let storage_root = AccountState::decode(&account_rlp)?.storage_root; + Ok(Some(self.engine + // On no! + .open_storage_trie(hashed_address, storage_root) + .into_iter() + .content() + .map_while(|(path, value)| { + Some((H256::from_slice(&path), U256::decode(&value).ok()?)) + }))) + } + pub fn get_account_range_proof( &self, state_root: H256, From c69690d00479d8e5a91c44ddcdec2f226a2d4184 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 16:46:07 -0300 Subject: [PATCH 059/345] Bump hive version --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 3c7cff1b3..39440f534 100644 --- a/Makefile +++ b/Makefile @@ -71,7 +71,7 @@ stop-localnet-silent: @kurtosis enclave stop lambdanet >/dev/null 2>&1 || true @kurtosis enclave rm lambdanet --force >/dev/null 2>&1 || true -HIVE_REVISION := 448926bb93f6c025236a3b81c08d7c437d64852 #TODO: move to main branch once the PR is merged +HIVE_REVISION := 421852ec25e4e608fe5460656f4bf0637649619e # Shallow clones can't specify a single revision, but at least we avoid working # the whole history by making it shallow since a given date (one day before our # target revision). From b7c0211b865e56a8685f78a267fb4035f380b06d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 17:39:51 -0300 Subject: [PATCH 060/345] Fix bug in response bytes length calculation --- crates/networking/p2p/snap.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 8efa3f5f1..2c40620cd 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -12,12 +12,9 @@ pub fn process_account_range_request( let mut bytes_used = 0; for (hash, account) in store.iter_accounts(request.root_hash) { if hash >= request.starting_hash { - let account = AccountRangeUnit { - hash, - account: AccountStateSlim::from(account), - }; - bytes_used += account.length() as u64; - accounts.push(account); + let account = AccountStateSlim::from(account); + bytes_used += 32 + account.length() as u64; + accounts.push(AccountRangeUnit { hash, account }); } if hash >= request.limit_hash || bytes_used >= request.response_bytes { break; From 74173d2c614c0a8e058e42b65de36057cee9b5a5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 17:40:36 -0300 Subject: [PATCH 061/345] Restore Makefile --- Makefile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Makefile b/Makefile index 39eb36b52..39440f534 100644 --- a/Makefile +++ b/Makefile @@ -94,10 +94,10 @@ TEST_PATTERN ?= / # The endpoints tested may be limited by supplying a test pattern in the form "/endpoint_1|enpoint_2|..|enpoint_n" # For example, to run the rpc-compat suites for eth_chainId & eth_blockNumber you should run: # `make run-hive SIMULATION=ethereum/rpc-compat TEST_PATTERN="/eth_chainId|eth_blockNumber"` -run-hive: build-image ## 🧪 Run Hive testing suite +run-hive: build-image setup-hive ## 🧪 Run Hive testing suite cd hive && ./hive --sim $(SIMULATION) --client ethereumrust --sim.limit "$(TEST_PATTERN)" -run-hive-debug: build-image ## 🐞 Run Hive testing suite in debug mode +run-hive-debug: build-image setup-hive ## 🐞 Run Hive testing suite in debug mode cd hive && ./hive --sim $(SIMULATION) --client ethereumrust --sim.limit "$(TEST_PATTERN)" --docker.output clean-hive-logs: ## 🧹 Clean Hive logs From 777732a79677a07987b323d7215772b77380b085 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 18:40:33 -0300 Subject: [PATCH 062/345] Identify storage tries by hashed account address --- crates/storage/store/engines/api.rs | 2 +- crates/storage/store/engines/in_memory.rs | 7 +-- crates/storage/store/engines/libmdbx.rs | 10 ++--- crates/storage/store/storage.rs | 53 +++++++++++++++-------- 4 files changed, 46 insertions(+), 26 deletions(-) diff --git a/crates/storage/store/engines/api.rs b/crates/storage/store/engines/api.rs index d67385de4..5582d1f98 100644 --- a/crates/storage/store/engines/api.rs +++ b/crates/storage/store/engines/api.rs @@ -232,7 +232,7 @@ pub trait StoreEngine: Debug + Send + Sync + RefUnwindSafe { // Obtain a storage trie from the given address and storage_root // Doesn't check if the account is stored // Used for internal store operations - fn open_storage_trie(&self, address: Address, storage_root: H256) -> Trie; + fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie; // Obtain a state trie from the given state root // Doesn't check if the state root is valid diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 46039ea3c..7dcab8eca 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -36,7 +36,8 @@ struct StoreInner { blobs_bundle_pool: HashMap, receipts: HashMap>, state_trie_nodes: NodeMap, - storage_trie_nodes: HashMap, + // A storage trie for each hashed account address + storage_trie_nodes: HashMap<[u8; 32], NodeMap>, // TODO (#307): Remove TotalDifficulty. block_total_difficulties: HashMap, // Stores local blocks by payload id @@ -339,9 +340,9 @@ impl StoreEngine for Store { Ok(self.inner().chain_data.pending_block_number) } - fn open_storage_trie(&self, address: Address, storage_root: H256) -> Trie { + fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie { let mut store = self.inner(); - let trie_backend = store.storage_trie_nodes.entry(address).or_default(); + let trie_backend = store.storage_trie_nodes.entry(hashed_address).or_default(); let db = Box::new(InMemoryTrieDB::new(trie_backend.clone())); Trie::open(db, storage_root) } diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 0d2f75515..25cefdf45 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -394,10 +394,10 @@ impl StoreEngine for Store { } } - fn open_storage_trie(&self, address: Address, storage_root: H256) -> Trie { - let db = Box::new(LibmdbxDupsortTrieDB::::new( + fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie { + let db = Box::new(LibmdbxDupsortTrieDB::::new( self.db.clone(), - address.0, + hashed_address, )); Trie::open(db, storage_root) } @@ -523,8 +523,8 @@ dupsort!( dupsort!( /// Table containing all storage trie's nodes - /// Each node is stored by address and node hash in order to keep different storage trie's nodes separate - ( StorageTriesNodes ) ([u8;20], [u8;33])[[u8;20]] => Vec + /// Each node is stored by hashed account address and node hash in order to keep different storage trie's nodes separate + ( StorageTriesNodes ) ([u8;32], [u8;33])[[u8;32]] => Vec ); dupsort!( diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 52330863b..553f325de 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -335,9 +335,10 @@ impl Store { } // Store the added storage in the account's storage trie and compute its new root if !update.added_storage.is_empty() { - let mut storage_trie = self - .engine - .open_storage_trie(update.address, account_state.storage_root); + let mut storage_trie = self.engine.open_storage_trie( + hashed_address.clone().try_into().unwrap(), + account_state.storage_root, + ); for (storage_key, storage_value) in &update.added_storage { let hashed_key = hash_key(storage_key); if storage_value.is_zero() { @@ -361,11 +362,14 @@ impl Store { ) -> Result { let mut genesis_state_trie = self.engine.open_state_trie(*EMPTY_TRIE_HASH); for (address, account) in genesis_accounts { + let hashed_address = hash_address(&address); // Store account code (as this won't be stored in the trie) let code_hash = code_hash(&account.code); self.add_account_code(code_hash, account.code)?; // Store the account's storage in a clean storage trie and compute its root - let mut storage_trie = self.engine.open_storage_trie(address, *EMPTY_TRIE_HASH); + let mut storage_trie = self + .engine + .open_storage_trie(hashed_address.clone().try_into().unwrap(), *EMPTY_TRIE_HASH); for (storage_key, storage_value) in account.storage { if !storage_value.is_zero() { let hashed_key = hash_key(&storage_key); @@ -380,7 +384,6 @@ impl Store { storage_root, code_hash, }; - let hashed_address = hash_address(&address); genesis_state_trie.insert(hashed_address, account_state.encode_to_vec())?; } Ok(genesis_state_trie.hash()?) @@ -631,7 +634,10 @@ impl Store { let account = AccountState::decode(&encoded_account)?; // Open storage_trie let storage_root = account.storage_root; - Ok(Some(self.engine.open_storage_trie(address, storage_root))) + Ok(Some(self.engine.open_storage_trie( + hashed_address.try_into().unwrap(), + storage_root, + ))) } pub fn get_account_state( @@ -673,7 +679,9 @@ impl Store { storage_root: H256, storage_key: &H256, ) -> Result>, StoreError> { - let trie = self.engine.open_storage_trie(address, storage_root); + let trie = self + .engine + .open_storage_trie(hash_address_fixed(&address), storage_root); Ok(trie.get_proof(&hash_key(storage_key))?) } @@ -691,20 +699,25 @@ impl Store { // Returns an iterator across all accounts in the state trie given by the state_root // Does not check that the state_root is valid - pub fn iter_storage(&self, state_root: H256, hashed_address: H256) -> Result>, StoreError> { + pub fn iter_storage( + &self, + state_root: H256, + hashed_address: H256, + ) -> Result>, StoreError> { let state_trie = self.engine.open_state_trie(state_root); let Some(account_rlp) = state_trie.get(&hashed_address.as_bytes().to_vec())? else { - return Ok(None) + return Ok(None); }; let storage_root = AccountState::decode(&account_rlp)?.storage_root; - Ok(Some(self.engine - // On no! - .open_storage_trie(hashed_address, storage_root) - .into_iter() - .content() - .map_while(|(path, value)| { - Some((H256::from_slice(&path), U256::decode(&value).ok()?)) - }))) + Ok(Some( + self.engine + .open_storage_trie(hashed_address.0, storage_root) + .into_iter() + .content() + .map_while(|(path, value)| { + Some((H256::from_slice(&path), U256::decode(&value).ok()?)) + }), + )) } pub fn get_account_range_proof( @@ -740,6 +753,12 @@ fn hash_address(address: &Address) -> Vec { .finalize() .to_vec() } +fn hash_address_fixed(address: &Address) -> [u8; 32] { + Keccak256::new_with_prefix(address.to_fixed_bytes()) + .finalize() + .try_into() + .unwrap() +} fn hash_key(key: &H256) -> Vec { Keccak256::new_with_prefix(key.to_fixed_bytes()) From abb28cfa78d69d121e62768bf6d39d49b9f5148f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 29 Oct 2024 18:52:53 -0300 Subject: [PATCH 063/345] Impl logic --- crates/networking/p2p/rlpx/snap.rs | 6 ++--- crates/networking/p2p/snap.rs | 38 +++++++++++++++++++++++++++++- 2 files changed, 40 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index be6151273..b63a88c7f 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -46,7 +46,7 @@ pub(crate) struct GetStorageRanges { #[derive(Debug)] pub(crate) struct StorageRanges { pub id: u64, - pub slots: Vec, + pub slots: Vec>, pub proof: Vec, } @@ -211,8 +211,8 @@ pub struct AccountStateSlim { #[derive(Debug)] pub struct StorageSlot { - hash: H256, - data: U256, + pub hash: H256, + pub data: U256, } impl RLPEncode for AccountRangeUnit { diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 2c40620cd..1590922f9 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -2,7 +2,10 @@ use bytes::Bytes; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use crate::rlpx::snap::{AccountRange, AccountRangeUnit, AccountStateSlim, GetAccountRange}; +use crate::rlpx::snap::{ + AccountRange, AccountRangeUnit, AccountStateSlim, GetAccountRange, GetStorageRanges, + StorageRanges, StorageSlot, +}; pub fn process_account_range_request( request: GetAccountRange, @@ -36,6 +39,39 @@ pub fn process_account_range_request( }) } +pub fn process_storage_ranges_request( + request: GetStorageRanges, + store: Store, +) -> Result { + let mut slots = vec![]; + let mut bytes_used = 0; + for hashed_address in request.account_hashes { + let mut account_slots = vec![]; + if let Some(storage_iter) = store.iter_storage(request.root_hash, hashed_address)? { + for (hash, data) in storage_iter { + if hash >= request.starting_hash { + bytes_used += 64_u64; // slot size + account_slots.push(StorageSlot {hash, data}); + } + if hash >= request.limit_hash || bytes_used >= request.response_bytes { + break; + } + } + } + slots.push(account_slots); + // TODO: check if this break is consistent with spec + if bytes_used >= request.response_bytes { + break; + } + } + let proof = vec![]; + Ok(StorageRanges { + id: request.id, + slots, + proof, + }) +} + #[cfg(test)] mod tests { use std::str::FromStr; From fea174dd0ebce8f20d4357bbf65f106a26050a4d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 13:01:39 -0300 Subject: [PATCH 064/345] Connect to main loop --- crates/networking/p2p/rlpx/connection.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 22b7389d4..a6c65c20c 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,6 +1,6 @@ use crate::{ rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, - snap::process_account_range_request, + snap::{process_account_range_request, process_storage_ranges_request}, MAX_DISC_PACKET_SIZE, }; @@ -150,6 +150,11 @@ impl RLPxConnection { process_account_range_request(req, self.storage.clone())?; self.send(Message::AccountRange(response)).await } + Message::GetStorageRanges(req) => { + let response = + process_storage_ranges_request(req, self.storage.clone())?; + self.send(Message::StorageRanges(response)).await + } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; From dc6752c8f7e319de2361bb01c6ef956d8286185d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 15:54:34 -0300 Subject: [PATCH 065/345] Compute proofs --- crates/networking/p2p/snap.rs | 34 +++++++++++++++++++++++++++++---- crates/storage/store/storage.rs | 22 +++++++++++++++++++++ 2 files changed, 52 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 1590922f9..7ee6e9332 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -44,27 +44,53 @@ pub fn process_storage_ranges_request( store: Store, ) -> Result { let mut slots = vec![]; + let mut proof = vec![]; let mut bytes_used = 0; + for hashed_address in request.account_hashes { let mut account_slots = vec![]; + let mut res_capped = false; + if let Some(storage_iter) = store.iter_storage(request.root_hash, hashed_address)? { for (hash, data) in storage_iter { if hash >= request.starting_hash { bytes_used += 64_u64; // slot size - account_slots.push(StorageSlot {hash, data}); + account_slots.push(StorageSlot { hash, data }); } if hash >= request.limit_hash || bytes_used >= request.response_bytes { + if bytes_used >= request.response_bytes { + res_capped = true; + } break; } } } - slots.push(account_slots); - // TODO: check if this break is consistent with spec + + // Generate proofs only if the response doesn't contain the full storage range for the account + // Aka if the starting hash is not zero or if the response was capped due to byte limit + if !request.starting_hash.is_zero() || res_capped && !!account_slots.is_empty() { + proof.extend( + store + .get_storage_range_proof( + request.root_hash, + hashed_address, + request.starting_hash, + account_slots.last().map(|acc| acc.hash), + )? + .unwrap_or_default() + .iter() + .map(|bytes| Bytes::copy_from_slice(bytes)), + ); + } + + if !account_slots.is_empty() { + slots.push(account_slots); + } + if bytes_used >= request.response_bytes { break; } } - let proof = vec![]; Ok(StorageRanges { id: request.id, slots, diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 553f325de..b14e95181 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -734,6 +734,28 @@ impl Store { Ok(proof) } + pub fn get_storage_range_proof( + &self, + state_root: H256, + hashed_address: H256, + starting_hash: H256, + last_hash: Option, + ) -> Result>>, StoreError> { + let state_trie = self.engine.open_state_trie(state_root); + let Some(account_rlp) = state_trie.get(&hashed_address.as_bytes().to_vec())? else { + return Ok(None); + }; + let storage_root = AccountState::decode(&account_rlp)?.storage_root; + let storage_trie = self + .engine + .open_storage_trie(hashed_address.0, storage_root); + let mut proof = storage_trie.get_proof(&starting_hash.as_bytes().to_vec())?; + if let Some(last_hash) = last_hash { + proof.extend_from_slice(&storage_trie.get_proof(&last_hash.as_bytes().to_vec())?); + } + Ok(Some(proof)) + } + pub fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError> { self.engine.add_payload(payload_id, block) } From 03d9928e3bc2fbeaded9a4290cf5270b7fe3adb2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 15:58:21 -0300 Subject: [PATCH 066/345] Clippy --- crates/networking/p2p/snap.rs | 2 +- crates/storage/store/storage.rs | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 7ee6e9332..256a348cc 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -68,7 +68,7 @@ pub fn process_storage_ranges_request( // Generate proofs only if the response doesn't contain the full storage range for the account // Aka if the starting hash is not zero or if the response was capped due to byte limit - if !request.starting_hash.is_zero() || res_capped && !!account_slots.is_empty() { + if !request.starting_hash.is_zero() || res_capped && !account_slots.is_empty() { proof.extend( store .get_storage_range_proof( diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index b14e95181..cd56e10f7 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -778,8 +778,7 @@ fn hash_address(address: &Address) -> Vec { fn hash_address_fixed(address: &Address) -> [u8; 32] { Keccak256::new_with_prefix(address.to_fixed_bytes()) .finalize() - .try_into() - .unwrap() + .into() } fn hash_key(key: &H256) -> Vec { From c791a890a493a9f2d989bf3570abf3f91cb902c9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 16:02:41 -0300 Subject: [PATCH 067/345] Chamge type --- crates/storage/store/engines/api.rs | 2 +- crates/storage/store/engines/in_memory.rs | 4 ++-- crates/storage/store/engines/libmdbx.rs | 4 ++-- crates/storage/store/storage.rs | 22 +++++++++++----------- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/crates/storage/store/engines/api.rs b/crates/storage/store/engines/api.rs index 5582d1f98..2824176e3 100644 --- a/crates/storage/store/engines/api.rs +++ b/crates/storage/store/engines/api.rs @@ -232,7 +232,7 @@ pub trait StoreEngine: Debug + Send + Sync + RefUnwindSafe { // Obtain a storage trie from the given address and storage_root // Doesn't check if the account is stored // Used for internal store operations - fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie; + fn open_storage_trie(&self, hashed_address: H256, storage_root: H256) -> Trie; // Obtain a state trie from the given state root // Doesn't check if the state root is valid diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 7dcab8eca..15e5ca080 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -37,7 +37,7 @@ struct StoreInner { receipts: HashMap>, state_trie_nodes: NodeMap, // A storage trie for each hashed account address - storage_trie_nodes: HashMap<[u8; 32], NodeMap>, + storage_trie_nodes: HashMap, // TODO (#307): Remove TotalDifficulty. block_total_difficulties: HashMap, // Stores local blocks by payload id @@ -340,7 +340,7 @@ impl StoreEngine for Store { Ok(self.inner().chain_data.pending_block_number) } - fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie { + fn open_storage_trie(&self, hashed_address: H256, storage_root: H256) -> Trie { let mut store = self.inner(); let trie_backend = store.storage_trie_nodes.entry(hashed_address).or_default(); let db = Box::new(InMemoryTrieDB::new(trie_backend.clone())); diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 25cefdf45..9001a7007 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -394,10 +394,10 @@ impl StoreEngine for Store { } } - fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie { + fn open_storage_trie(&self, hashed_address: H256, storage_root: H256) -> Trie { let db = Box::new(LibmdbxDupsortTrieDB::::new( self.db.clone(), - hashed_address, + hashed_address.0, )); Trie::open(db, storage_root) } diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index cd56e10f7..4a211db85 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -336,7 +336,7 @@ impl Store { // Store the added storage in the account's storage trie and compute its new root if !update.added_storage.is_empty() { let mut storage_trie = self.engine.open_storage_trie( - hashed_address.clone().try_into().unwrap(), + H256::from_slice(&hashed_address), account_state.storage_root, ); for (storage_key, storage_value) in &update.added_storage { @@ -369,7 +369,7 @@ impl Store { // Store the account's storage in a clean storage trie and compute its root let mut storage_trie = self .engine - .open_storage_trie(hashed_address.clone().try_into().unwrap(), *EMPTY_TRIE_HASH); + .open_storage_trie(H256::from_slice(&hashed_address), *EMPTY_TRIE_HASH); for (storage_key, storage_value) in account.storage { if !storage_value.is_zero() { let hashed_key = hash_key(&storage_key); @@ -635,7 +635,7 @@ impl Store { // Open storage_trie let storage_root = account.storage_root; Ok(Some(self.engine.open_storage_trie( - hashed_address.try_into().unwrap(), + H256::from_slice(&hashed_address), storage_root, ))) } @@ -711,7 +711,7 @@ impl Store { let storage_root = AccountState::decode(&account_rlp)?.storage_root; Ok(Some( self.engine - .open_storage_trie(hashed_address.0, storage_root) + .open_storage_trie(hashed_address, storage_root) .into_iter() .content() .map_while(|(path, value)| { @@ -746,9 +746,7 @@ impl Store { return Ok(None); }; let storage_root = AccountState::decode(&account_rlp)?.storage_root; - let storage_trie = self - .engine - .open_storage_trie(hashed_address.0, storage_root); + let storage_trie = self.engine.open_storage_trie(hashed_address, storage_root); let mut proof = storage_trie.get_proof(&starting_hash.as_bytes().to_vec())?; if let Some(last_hash) = last_hash { proof.extend_from_slice(&storage_trie.get_proof(&last_hash.as_bytes().to_vec())?); @@ -775,10 +773,12 @@ fn hash_address(address: &Address) -> Vec { .finalize() .to_vec() } -fn hash_address_fixed(address: &Address) -> [u8; 32] { - Keccak256::new_with_prefix(address.to_fixed_bytes()) - .finalize() - .into() +fn hash_address_fixed(address: &Address) -> H256 { + H256( + Keccak256::new_with_prefix(address.to_fixed_bytes()) + .finalize() + .into(), + ) } fn hash_key(key: &H256) -> Vec { From 9fd88d322e8005b3c660bb357994e3093164f0ad Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 16:06:01 -0300 Subject: [PATCH 068/345] Add test --- .github/workflows/hive.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 1f0fe044b..86e934eb3 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -38,7 +38,7 @@ jobs: run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" - simulation: snap name: "Devp2p snap tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange" + run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges" - simulation: engine name: "Engine tests" run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" From 2b9d2071d76393c63a7613a970b594069012eebf Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 17:55:41 -0300 Subject: [PATCH 069/345] Add messages --- crates/networking/p2p/rlpx/message.rs | 16 +++++- crates/networking/p2p/rlpx/snap.rs | 76 +++++++++++++++++++++++++++ 2 files changed, 91 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 766e66cdc..cbd1449d1 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -4,7 +4,7 @@ use std::fmt::Display; use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; -use super::snap::{AccountRange, GetAccountRange, GetStorageRanges, StorageRanges}; +use super::snap::{AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges}; use ethereum_rust_rlp::encode::RLPEncode; @@ -25,6 +25,8 @@ pub(crate) enum Message { AccountRange(AccountRange), GetStorageRanges(GetStorageRanges), StorageRanges(StorageRanges), + GetByteCodes(GetByteCodes), + ByteCodes(ByteCodes), } impl Message { @@ -41,6 +43,8 @@ impl Message { msg_data, )?)), 0x24 => Ok(Message::StorageRanges(StorageRanges::decode(msg_data)?)), + 0x25 => Ok(Message::GetByteCodes(GetByteCodes::decode(msg_data)?)), + 0x26 => Ok(Message::ByteCodes(ByteCodes::decode(msg_data)?)), _ => Err(RLPDecodeError::MalformedData), } } @@ -67,6 +71,14 @@ impl Message { Message::StorageRanges(msg) => { 0x24_u8.encode(buf); msg.encode(buf) + }, + Message::GetByteCodes(msg) => { + 0x25_u8.encode(buf); + msg.encode(buf) + } + Message::ByteCodes(msg) => { + 0x26_u8.encode(buf); + msg.encode(buf) } } } @@ -84,6 +96,8 @@ impl Display for Message { Message::AccountRange(_) => "snap::AccountRange".fmt(f), Message::GetStorageRanges(_) => "snap::GetStorageRanges".fmt(f), Message::StorageRanges(_) => "snap::StorageRanges".fmt(f), + Message::GetByteCodes(_) => "snap::GetByteCodes".fmt(f), + Message::ByteCodes(_) => "snap::ByteCodes".fmt(f), } } } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index b63a88c7f..84506c09a 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -50,6 +50,19 @@ pub(crate) struct StorageRanges { pub proof: Vec, } +#[derive(Debug)] +pub(crate) struct GetByteCodes { + pub id: u64, + pub hashes: Vec, + pub bytes: u64, +} + +#[derive(Debug)] +pub(crate) struct ByteCodes { + pub id: u64, + pub codes: Vec, +} + impl RLPxMessage for GetAccountRange { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { let mut encoded_data = vec![]; @@ -193,6 +206,69 @@ impl RLPxMessage for StorageRanges { } } +impl RLPxMessage for GetByteCodes { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.hashes) + .encode_field(&self.bytes) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (hashes, decoder) = decoder.decode_field("hashes")?; + let (bytes, decoder) = decoder.decode_field("bytes")?; + decoder.finish()?; + + Ok(Self { + id, + hashes, + bytes, + }) + } +} + +impl RLPxMessage for ByteCodes { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.codes) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (codes, decoder) = decoder.decode_field("codes")?; + decoder.finish()?; + + Ok(Self { + id, + codes, + }) + } +} + // Intermediate structures #[derive(Debug)] From 37becaae939d2a231db601b46e139f980dcf0d78 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 18:25:30 -0300 Subject: [PATCH 070/345] Add hive test --- .github/workflows/hive.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 86e934eb3..041ed4355 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -38,7 +38,7 @@ jobs: run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" - simulation: snap name: "Devp2p snap tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges" + run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges|ByteCodes" - simulation: engine name: "Engine tests" run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" From a6d2f0808d77a5e2c0e1fd77d30e7c4500d8af3f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 30 Oct 2024 18:26:22 -0300 Subject: [PATCH 071/345] fmt --- crates/networking/p2p/rlpx/connection.rs | 8 +++++++- crates/networking/p2p/rlpx/message.rs | 6 ++++-- crates/networking/p2p/rlpx/snap.rs | 11 ++--------- crates/networking/p2p/snap.rs | 25 ++++++++++++++++++++++-- 4 files changed, 36 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index a6c65c20c..9b945d37e 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,6 +1,8 @@ use crate::{ rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, - snap::{process_account_range_request, process_storage_ranges_request}, + snap::{ + process_account_range_request, process_byte_codes_request, process_storage_ranges_request, + }, MAX_DISC_PACKET_SIZE, }; @@ -155,6 +157,10 @@ impl RLPxConnection { process_storage_ranges_request(req, self.storage.clone())?; self.send(Message::StorageRanges(response)).await } + Message::GetByteCodes(req) => { + let response = process_byte_codes_request(req, self.storage.clone())?; + self.send(Message::ByteCodes(response)).await + } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index cbd1449d1..9a3cc5d76 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -4,7 +4,9 @@ use std::fmt::Display; use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; -use super::snap::{AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges}; +use super::snap::{ + AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges, +}; use ethereum_rust_rlp::encode::RLPEncode; @@ -71,7 +73,7 @@ impl Message { Message::StorageRanges(msg) => { 0x24_u8.encode(buf); msg.encode(buf) - }, + } Message::GetByteCodes(msg) => { 0x25_u8.encode(buf); msg.encode(buf) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 84506c09a..6126db6dc 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -231,11 +231,7 @@ impl RLPxMessage for GetByteCodes { let (bytes, decoder) = decoder.decode_field("bytes")?; decoder.finish()?; - Ok(Self { - id, - hashes, - bytes, - }) + Ok(Self { id, hashes, bytes }) } } @@ -262,10 +258,7 @@ impl RLPxMessage for ByteCodes { let (codes, decoder) = decoder.decode_field("codes")?; decoder.finish()?; - Ok(Self { - id, - codes, - }) + Ok(Self { id, codes }) } } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 256a348cc..d32562d37 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -3,8 +3,8 @@ use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; use crate::rlpx::snap::{ - AccountRange, AccountRangeUnit, AccountStateSlim, GetAccountRange, GetStorageRanges, - StorageRanges, StorageSlot, + AccountRange, AccountRangeUnit, AccountStateSlim, ByteCodes, GetAccountRange, GetByteCodes, + GetStorageRanges, StorageRanges, StorageSlot, }; pub fn process_account_range_request( @@ -98,6 +98,27 @@ pub fn process_storage_ranges_request( }) } +pub fn process_byte_codes_request( + request: GetByteCodes, + store: Store, +) -> Result { + let mut codes = vec![]; + let mut bytes_used = 0; + for code_hash in request.hashes { + if let Some(code) = store.get_account_code(code_hash)? { + bytes_used += code.len() as u64; + codes.push(code); + } + if bytes_used >= request.bytes { + break; + } + } + Ok(ByteCodes { + id: request.id, + codes, + }) +} + #[cfg(test)] mod tests { use std::str::FromStr; From c502f06dc8292c8d6440b4068000fa6dc25cb8e9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 10:36:48 -0300 Subject: [PATCH 072/345] Impl messages --- crates/networking/p2p/rlpx/message.rs | 17 +++++- crates/networking/p2p/rlpx/snap.rs | 78 +++++++++++++++++++++++++++ 2 files changed, 94 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 9a3cc5d76..0e42ceb35 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -5,7 +5,8 @@ use std::fmt::Display; use super::eth::status::StatusMessage; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; use super::snap::{ - AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges, + AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, GetTrieNodes, + StorageRanges, TrieNodes, }; use ethereum_rust_rlp::encode::RLPEncode; @@ -29,6 +30,8 @@ pub(crate) enum Message { StorageRanges(StorageRanges), GetByteCodes(GetByteCodes), ByteCodes(ByteCodes), + GetTrieNodes(GetTrieNodes), + TrieNodes(TrieNodes), } impl Message { @@ -47,6 +50,8 @@ impl Message { 0x24 => Ok(Message::StorageRanges(StorageRanges::decode(msg_data)?)), 0x25 => Ok(Message::GetByteCodes(GetByteCodes::decode(msg_data)?)), 0x26 => Ok(Message::ByteCodes(ByteCodes::decode(msg_data)?)), + 0x27 => Ok(Message::GetTrieNodes(GetTrieNodes::decode(msg_data)?)), + 0x28 => Ok(Message::TrieNodes(TrieNodes::decode(msg_data)?)), _ => Err(RLPDecodeError::MalformedData), } } @@ -82,6 +87,14 @@ impl Message { 0x26_u8.encode(buf); msg.encode(buf) } + Message::GetTrieNodes(msg) => { + 0x27_u8.encode(buf); + msg.encode(buf) + } + Message::TrieNodes(msg) => { + 0x28_u8.encode(buf); + msg.encode(buf) + } } } } @@ -100,6 +113,8 @@ impl Display for Message { Message::StorageRanges(_) => "snap::StorageRanges".fmt(f), Message::GetByteCodes(_) => "snap::GetByteCodes".fmt(f), Message::ByteCodes(_) => "snap::ByteCodes".fmt(f), + Message::GetTrieNodes(_) => "snap::GetTrieNodes".fmt(f), + Message::TrieNodes(_) => "snap::TrieNodes".fmt(f), } } } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 6126db6dc..8da84ca16 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -63,6 +63,21 @@ pub(crate) struct ByteCodes { pub codes: Vec, } +#[derive(Debug)] +pub(crate) struct GetTrieNodes { + pub id: u64, + pub root_hash: H256, + // [[acc_path, slot_path_1, slot_path_2,...]...] + pub paths: Vec>, + pub bytes: u64, +} + +#[derive(Debug)] +pub(crate) struct TrieNodes { + pub id: u64, + pub nodes: Vec, +} + impl RLPxMessage for GetAccountRange { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { let mut encoded_data = vec![]; @@ -262,6 +277,69 @@ impl RLPxMessage for ByteCodes { } } +impl RLPxMessage for GetTrieNodes { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.root_hash) + .encode_field(&self.paths) + .encode_field(&self.bytes) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (root_hash, decoder) = decoder.decode_field("root_hash")?; + let (paths, decoder) = decoder.decode_field("paths")?; + let (bytes, decoder) = decoder.decode_field("bytes")?; + decoder.finish()?; + + Ok(Self { + id, + root_hash, + paths, + bytes, + }) + } +} + +impl RLPxMessage for TrieNodes { + fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { + let mut encoded_data = vec![]; + Encoder::new(&mut encoded_data) + .encode_field(&self.id) + .encode_field(&self.nodes) + .finish(); + + let msg_data = snappy_encode(encoded_data)?; + buf.put_slice(&msg_data); + Ok(()) + } + + fn decode(msg_data: &[u8]) -> Result { + let mut snappy_decoder = SnappyDecoder::new(); + let decompressed_data = snappy_decoder + .decompress_vec(msg_data) + .map_err(|e| RLPDecodeError::Custom(e.to_string()))?; + let decoder = Decoder::new(&decompressed_data)?; + let (id, decoder) = decoder.decode_field("request-id")?; + let (nodes, decoder) = decoder.decode_field("nodes")?; + decoder.finish()?; + + Ok(Self { id, nodes }) + } +} + // Intermediate structures #[derive(Debug)] From 242502eefed183209b3ce1d8230c7c3c4dcbfc5b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 11:49:51 -0300 Subject: [PATCH 073/345] Impl Trie::get_node --- crates/storage/trie/node.rs | 13 +++++++++++++ crates/storage/trie/node/branch.rs | 25 +++++++++++++++++++++++++ crates/storage/trie/node/extension.rs | 19 +++++++++++++++++++ crates/storage/trie/node/leaf.rs | 10 ++++++++++ crates/storage/trie/trie.rs | 15 +++++++++++++++ 5 files changed, 82 insertions(+) diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index b19a0111b..cd8bacc40 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -101,4 +101,17 @@ impl Node { Node::Leaf(n) => n.insert_self(path_offset, state), } } + + /// Obtain the encoded node given its path. + pub fn get_node( + &self, + state: &TrieState, + path: NibbleSlice, + ) -> Result>, TrieError> { + match self { + Node::Branch(n) => n.get_node(state, path), + Node::Extension(n) => n.get_node(state, path), + Node::Leaf(n) => n.get_node(path), + } + } } diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index a5279bc2e..2ee12f38c 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -341,6 +341,31 @@ impl BranchNode { } Ok(()) } + + /// Obtain the encoded node given its path. + pub fn get_node( + &self, + state: &TrieState, + mut path: NibbleSlice, + ) -> Result>, TrieError> { + // If path is at the end, then path must be at self or not exist in the trie. + // Otherwise, check the corresponding choice and delegate accordingly if present. + if let Some(choice) = path.next().map(usize::from) { + // Delegate to children if present + let child_hash = &self.choices[choice]; + if child_hash.is_valid() { + let child_node = state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + child_node.get_node(state, path) + } else { + Ok(None) + } + } else { + // If self is not devoid of path then it must be the node we are looking for + Ok((!self.path.is_empty()).then_some(self.encode_raw())) + } + } } #[cfg(test)] diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 4e0c0d0e4..ecdc4b276 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -215,6 +215,25 @@ impl ExtensionNode { } Ok(()) } + + /// Obtain the encoded node given its path. + pub fn get_node( + &self, + state: &TrieState, + mut path: NibbleSlice, + ) -> Result>, TrieError> { + // If the path is prefixed by this node's prefix, delegate to its child. + // Otherwise, the path doesn't belong to the trie. + if path.skip_prefix(&self.prefix) { + let child_node = state + .get_node(self.child.clone())? + .expect("inconsistent internal tree structure"); + + child_node.get_node(state, path) + } else { + Ok(None) + } + } } #[cfg(test)] diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index b6684e61d..05064a4a9 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -166,6 +166,16 @@ impl LeafNode { } Ok(()) } + + /// Obtain the encoded node given its path. + pub fn get_node(&self, path: NibbleSlice) -> Result>, TrieError> { + if path.cmp_rest(&self.path) { + // TODO: check if this is ok or if ot should be offset + 1 + Ok(Some(self.encode_raw(path.offset()))) + } else { + Ok(None) + } + } } #[cfg(test)] diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 45b098f03..bff2db163 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -182,6 +182,21 @@ impl Trie { .unwrap_or(*EMPTY_TRIE_HASH) } + /// Obtain the encoded node given its path. + pub fn get_node(&self, path: &PathRLP) -> Result>, TrieError> { + if let Some(root_node) = self + .root + .as_ref() + .map(|root| self.state.get_node(root.clone())) + .transpose()? + .flatten() + { + root_node.get_node(&self.state, NibbleSlice::new(path)) + } else { + Ok(None) + } + } + #[cfg(all(test, feature = "libmdbx"))] /// Creates a new Trie based on a temporary Libmdbx DB fn new_temp() -> Self { From 3b7d70e70fbd6a9b36526a2d941477b4d1ce214a Mon Sep 17 00:00:00 2001 From: fmoletta <99273364+fmoletta@users.noreply.github.com> Date: Thu, 31 Oct 2024 12:09:54 -0300 Subject: [PATCH 074/345] Fix --- crates/networking/p2p/rlpx/message.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 9eaaecdff..2d5edffa3 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -66,8 +66,8 @@ impl Display for Message { Message::Ping(_) => "p2p:Ping".fmt(f), Message::Pong(_) => "p2p:Pong".fmt(f), Message::Status(_) => "eth:Status".fmt(f), - Message::GetAccountRange(_) => "snap::GetAccountRange".fmt(f), - Message::AccountRange(_) => "snap::AccountRange".fmt(f), + Message::GetAccountRange(_) => "snap:GetAccountRange".fmt(f), + Message::AccountRange(_) => "snap:AccountRange".fmt(f), } } } From 1cf1e9b950ebec3559eb369dfe0163f94a9e8117 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 12:30:03 -0300 Subject: [PATCH 075/345] implement get_trie_nodes for storage --- crates/storage/store/storage.rs | 40 +++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 4a211db85..b016cd65c 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -754,6 +754,46 @@ impl Store { Ok(Some(proof)) } + /// Receives the root of the state trie and a list of paths where the first path will correspond to a path in the state trie + /// (aka a hashed account address) and the following paths will be paths in the accoun's storage trie (aka hashed storage keys) + /// Returns a list of encoded nodes where the first one will be the state trie's node where the account is stored and the following + /// ones will be the nodes of the storage trie where each storage key is stored. Missing nodes will be skipped. + /// For more information check out snap capability message [`GetTrieNodes`](https://github.com/ethereum/devp2p/blob/master/caps/snap.md#gettrienodes-0x06) + pub fn get_trie_nodes( + &self, + state_root: H256, + paths: Vec, + ) -> Result>, StoreError> { + let Some(account_path) = paths.first() else { + return Ok(vec![]); + }; + let state_trie = self.engine.open_state_trie(state_root); + // Fetch state trie node + let Some(node) = state_trie.get_node(&account_path.0.to_vec())? else { + return Ok(vec![]); + }; + let mut nodes = vec![node]; + + let Some(account_state) = state_trie + .get(&account_path.0.to_vec())? + .map(|ref rlp| AccountState::decode(rlp)) + .transpose()? + else { + // We already fetched the node containing the account so we should be able to fetch the account + return Ok(vec![]); + }; + let storage_trie = self + .engine + .open_storage_trie(*account_path, account_state.storage_root); + // Fetch storage trie nodes + for path in paths.iter().skip(1) { + if let Some(node) = storage_trie.get_node(&path.0.to_vec())? { + nodes.push(node); + } + } + Ok(nodes) + } + pub fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError> { self.engine.add_payload(payload_id, block) } From 349d05fb2fce2d00a0675330ea13a5fcadb4b680 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 12:33:42 -0300 Subject: [PATCH 076/345] Add byte limit --- crates/storage/store/storage.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index b016cd65c..98246dc5b 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -763,15 +763,18 @@ impl Store { &self, state_root: H256, paths: Vec, + byte_limit: u64, ) -> Result>, StoreError> { let Some(account_path) = paths.first() else { return Ok(vec![]); }; + let mut bytes_used = 0; let state_trie = self.engine.open_state_trie(state_root); // Fetch state trie node let Some(node) = state_trie.get_node(&account_path.0.to_vec())? else { return Ok(vec![]); }; + bytes_used += node.len() as u64; let mut nodes = vec![node]; let Some(account_state) = state_trie @@ -787,7 +790,11 @@ impl Store { .open_storage_trie(*account_path, account_state.storage_root); // Fetch storage trie nodes for path in paths.iter().skip(1) { + if bytes_used >= byte_limit { + break; + } if let Some(node) = storage_trie.get_node(&path.0.to_vec())? { + bytes_used += node.len() as u64; nodes.push(node); } } From 4fd0ee5a5586b108ac0218658e36bb6ba9b22de2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 14:42:32 -0300 Subject: [PATCH 077/345] Add request processing --- crates/networking/p2p/rlpx/connection.rs | 5 +++++ crates/networking/p2p/snap.rs | 24 +++++++++++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 9b945d37e..bfbb89a41 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -2,6 +2,7 @@ use crate::{ rlpx::{eth::backend, handshake::encode_ack_message, message::Message, p2p, utils::id2pubkey}, snap::{ process_account_range_request, process_byte_codes_request, process_storage_ranges_request, + process_trie_nodes_request, }, MAX_DISC_PACKET_SIZE, }; @@ -161,6 +162,10 @@ impl RLPxConnection { let response = process_byte_codes_request(req, self.storage.clone())?; self.send(Message::ByteCodes(response)).await } + Message::GetTrieNodes(req) => { + let response = process_trie_nodes_request(req, self.storage.clone())?; + self.send(Message::TrieNodes(response)).await + } // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::UnexpectedMessage(message)), }; diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index d32562d37..b50c4481d 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -4,7 +4,7 @@ use ethereum_rust_storage::{error::StoreError, Store}; use crate::rlpx::snap::{ AccountRange, AccountRangeUnit, AccountStateSlim, ByteCodes, GetAccountRange, GetByteCodes, - GetStorageRanges, StorageRanges, StorageSlot, + GetStorageRanges, GetTrieNodes, StorageRanges, StorageSlot, TrieNodes, }; pub fn process_account_range_request( @@ -119,6 +119,28 @@ pub fn process_byte_codes_request( }) } +pub fn process_trie_nodes_request( + request: GetTrieNodes, + store: Store, +) -> Result { + let mut nodes = vec![]; + let mut remaining_bytes = request.bytes; + for paths in request.paths { + let trie_nodes = store.get_trie_nodes(request.root_hash, paths, remaining_bytes)?; + nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); + remaining_bytes = remaining_bytes + .saturating_sub(trie_nodes.iter().fold(0, |acc, nodes| acc + nodes.len()) as u64); + if remaining_bytes == 0 { + break; + } + } + + Ok(TrieNodes { + id: request.id, + nodes, + }) +} + #[cfg(test)] mod tests { use std::str::FromStr; From aa29e5fb4683a518f16c198cca4e68772ca04d3a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 16:43:47 -0300 Subject: [PATCH 078/345] Fix lingering conflicts --- crates/networking/p2p/rlpx/snap.rs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 5c093188a..b63a88c7f 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -31,7 +31,6 @@ pub(crate) struct AccountRange { pub id: u64, pub accounts: Vec, pub proof: Vec, -<<<<<<< HEAD } #[derive(Debug)] @@ -49,8 +48,6 @@ pub(crate) struct StorageRanges { pub id: u64, pub slots: Vec>, pub proof: Vec, -======= ->>>>>>> ef9c51143dda041e0c8e4ca2a8833b0857e73bf2 } impl RLPxMessage for GetAccountRange { @@ -125,7 +122,6 @@ impl RLPxMessage for AccountRange { } } -<<<<<<< HEAD impl RLPxMessage for GetStorageRanges { fn encode(&self, buf: &mut dyn BufMut) -> Result<(), RLPEncodeError> { let mut encoded_data = vec![]; @@ -197,8 +193,6 @@ impl RLPxMessage for StorageRanges { } } -======= ->>>>>>> ef9c51143dda041e0c8e4ca2a8833b0857e73bf2 // Intermediate structures #[derive(Debug)] @@ -215,15 +209,12 @@ pub struct AccountStateSlim { pub code_hash: Bytes, } -<<<<<<< HEAD #[derive(Debug)] pub struct StorageSlot { pub hash: H256, pub data: U256, } -======= ->>>>>>> ef9c51143dda041e0c8e4ca2a8833b0857e73bf2 impl RLPEncode for AccountRangeUnit { fn encode(&self, buf: &mut dyn BufMut) { Encoder::new(buf) @@ -311,7 +302,6 @@ impl From for AccountState { storage_root, code_hash, } -<<<<<<< HEAD } } @@ -330,7 +320,5 @@ impl RLPDecode for StorageSlot { let (hash, decoder) = decoder.decode_field("hash")?; let (data, decoder) = decoder.decode_field("data")?; Ok((Self { hash, data }, decoder.finish()?)) -======= ->>>>>>> ef9c51143dda041e0c8e4ca2a8833b0857e73bf2 } } From 89e144aabd16624d98bd91dc591f8ab8bc3233ce Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 31 Oct 2024 19:11:21 -0300 Subject: [PATCH 079/345] Save progress --- Makefile | 3 ++ cmd/ethereum_rust/ethereum_rust.rs | 8 ++--- crates/networking/p2p/rlpx/snap.rs | 3 +- crates/networking/p2p/snap.rs | 57 +++++++++++++++++++++++++++++- crates/storage/store/storage.rs | 36 +++++++++++-------- crates/storage/trie/node/branch.rs | 7 ++-- crates/storage/trie/node/leaf.rs | 1 - crates/storage/trie/trie.rs | 15 +++++++- 8 files changed, 106 insertions(+), 24 deletions(-) diff --git a/Makefile b/Makefile index 39440f534..6468711f0 100644 --- a/Makefile +++ b/Makefile @@ -100,5 +100,8 @@ run-hive: build-image setup-hive ## 🧪 Run Hive testing suite run-hive-debug: build-image setup-hive ## 🐞 Run Hive testing suite in debug mode cd hive && ./hive --sim $(SIMULATION) --client ethereumrust --sim.limit "$(TEST_PATTERN)" --docker.output +run-hive-geth: build-image setup-hive ## 🧪 Run Hive testing suite + cd hive && ./hive --sim $(SIMULATION) --client go-ethereum --sim.limit "$(TEST_PATTERN)" --docker.output + clean-hive-logs: ## 🧹 Clean Hive logs rm -rf ./hive/workspace/logs diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index ff1e49803..bd4441092 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -126,10 +126,10 @@ async fn main() { let size = blocks.len(); for block in &blocks { let hash = block.header.compute_block_hash(); - info!( - "Adding block {} with hash {:#x}.", - block.header.number, hash - ); + // info!( + // "Adding block {} with hash {:#x}.", + // block.header.number, hash + // ); let result = add_block(block, &store); if let Some(error) = result.err() { warn!( diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 8da84ca16..dbfe1d587 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -68,7 +68,8 @@ pub(crate) struct GetTrieNodes { pub id: u64, pub root_hash: H256, // [[acc_path, slot_path_1, slot_path_2,...]...] - pub paths: Vec>, + // The paths can be either full paths (hash) or only the partial path (bytes) + pub paths: Vec>, pub bytes: u64, } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index b50c4481d..b4ab972aa 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -126,7 +126,11 @@ pub fn process_trie_nodes_request( let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { - let trie_nodes = store.get_trie_nodes(request.root_hash, paths, remaining_bytes)?; + let trie_nodes = store.get_trie_nodes( + request.root_hash, + paths.into_iter().map(|bytes| bytes.to_vec()).collect(), + remaining_bytes, + )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); remaining_bytes = remaining_bytes .saturating_sub(trie_nodes.iter().fold(0, |acc, nodes| acc + nodes.len()) as u64); @@ -141,6 +145,57 @@ pub fn process_trie_nodes_request( }) } +// // func keybytesToHex(str []byte) []byte { +// // l := len(str)*2 + 1 +// // var nibbles = make([]byte, l) +// // for i, b := range str { +// // nibbles[i*2] = b / 16 +// // nibbles[i*2+1] = b % 16 +// // } +// // nibbles[l-1] = 16 +// // return nibbles +// // } + +// fn keybytes_to_hex(keybytes: Bytes) -> Vec { +// let l = keybytes.len()*2+1; +// let mut nibbles = vec![0;l]; +// for (i, b) in keybytes.into_iter().enumerate() { +// nibbles[i*2] = b / 16; +// nibbles[i*2+1] = b % 16; +// } +// nibbles[l - 1] = 16; +// nibbles +// } + +// // func compactToHex(compact []byte) []byte { +// // if len(compact) == 0 { +// // return compact +// // } +// // base := keybytesToHex(compact) +// // // delete terminator flag +// // if base[0] < 2 { +// // base = base[:len(base)-1] +// // } +// // // apply odd flag +// // chop := 2 - base[0]&1 +// // return base[chop:] +// // } + +// fn compact_to_hex(compact: Bytes) -> Vec { +// if compact.is_empty() { +// return vec![] +// } +// let mut base = keybytes_to_hex(compact); +// // delete terminator flag +// if base[0] < 2 { +// base = base[..base.len() - 1].to_vec(); +// } +// // apply odd flag +// let chop = 2 - (base[0]&1) as usize; +// base[chop..].to_vec() + +// } + #[cfg(test)] mod tests { use std::str::FromStr; diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 98246dc5b..026bc42df 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -756,13 +756,15 @@ impl Store { /// Receives the root of the state trie and a list of paths where the first path will correspond to a path in the state trie /// (aka a hashed account address) and the following paths will be paths in the accoun's storage trie (aka hashed storage keys) - /// Returns a list of encoded nodes where the first one will be the state trie's node where the account is stored and the following - /// ones will be the nodes of the storage trie where each storage key is stored. Missing nodes will be skipped. + /// If only one hash (account) is received, then the state trie node containing the account will be returned. + /// If more than one hash is received, then the storage trie nodes where each storage key is stored will be returned + /// Missing nodes will be skipped. /// For more information check out snap capability message [`GetTrieNodes`](https://github.com/ethereum/devp2p/blob/master/caps/snap.md#gettrienodes-0x06) + /// The paths can be either full paths (hash) or partial paths (bytes), if a partial path is given for the account this method will not return storage nodes for it (TODO: FIX) pub fn get_trie_nodes( &self, state_root: H256, - paths: Vec, + paths: Vec>, byte_limit: u64, ) -> Result>, StoreError> { let Some(account_path) = paths.first() else { @@ -770,30 +772,36 @@ impl Store { }; let mut bytes_used = 0; let state_trie = self.engine.open_state_trie(state_root); - // Fetch state trie node - let Some(node) = state_trie.get_node(&account_path.0.to_vec())? else { - return Ok(vec![]); - }; - bytes_used += node.len() as u64; - let mut nodes = vec![node]; - + // State Trie Nodes Request + if paths.len() == 1 { + // Fetch state trie node + let Some(node) = state_trie.get_node(&account_path)? else { + return Ok(vec![]); + }; + return Ok(vec![node]); + } + // Storage Trie Nodes Request + let mut nodes = vec![]; let Some(account_state) = state_trie - .get(&account_path.0.to_vec())? + .get(&account_path)? .map(|ref rlp| AccountState::decode(rlp)) .transpose()? else { - // We already fetched the node containing the account so we should be able to fetch the account return Ok(vec![]); }; + // We can't access the storage trie without the account's address hash (TODO: FIX THIS) + let Ok(hashed_address) = account_path.clone().try_into().map(H256) else { + return Ok(nodes); + }; let storage_trie = self .engine - .open_storage_trie(*account_path, account_state.storage_root); + .open_storage_trie(hashed_address, account_state.storage_root); // Fetch storage trie nodes for path in paths.iter().skip(1) { if bytes_used >= byte_limit { break; } - if let Some(node) = storage_trie.get_node(&path.0.to_vec())? { + if let Some(node) = storage_trie.get_node(path)? { bytes_used += node.len() as u64; nodes.push(node); } diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 2ee12f38c..3b9c1962f 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -362,8 +362,11 @@ impl BranchNode { Ok(None) } } else { - // If self is not devoid of path then it must be the node we are looking for - Ok((!self.path.is_empty()).then_some(self.encode_raw())) + if path.len() == 0 || self.path == path.data() { + Ok(Some(self.encode_raw())) + } else { + Ok(None) + } } } } diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 05064a4a9..0cb86dd8f 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -170,7 +170,6 @@ impl LeafNode { /// Obtain the encoded node given its path. pub fn get_node(&self, path: NibbleSlice) -> Result>, TrieError> { if path.cmp_rest(&self.path) { - // TODO: check if this is ok or if ot should be offset + 1 Ok(Some(self.encode_raw(path.offset()))) } else { Ok(None) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index bff2db163..0de9df58c 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -183,7 +183,17 @@ impl Trie { } /// Obtain the encoded node given its path. + /// Allows usage of partial paths pub fn get_node(&self, path: &PathRLP) -> Result>, TrieError> { + if path.is_empty() { + return Ok(None); + } + println!("Getting node with path: {:?}", path); + let path = if path == &vec![0] { + NibbleSlice::new(&[]) + } else { + NibbleSlice::new(path) + }; if let Some(root_node) = self .root .as_ref() @@ -191,7 +201,10 @@ impl Trie { .transpose()? .flatten() { - root_node.get_node(&self.state, NibbleSlice::new(path)) + println!("Node: {:?}", root_node); + let res = root_node.get_node(&self.state, path)?; + println!("Node: {:?}", res); + Ok(res) } else { Ok(None) } From 1d8c3f93dc5d370f7d84edcca974af105d7c2a48 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 11:22:38 -0300 Subject: [PATCH 080/345] Progress --- crates/networking/p2p/snap.rs | 76 +++++++++++---------------------- crates/storage/store/storage.rs | 2 +- crates/storage/trie/node.rs | 11 +++++ crates/storage/trie/trie.rs | 52 ++++++++++++++++++++++ 4 files changed, 89 insertions(+), 52 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index b4ab972aa..409e26703 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -128,7 +128,7 @@ pub fn process_trie_nodes_request( for paths in request.paths { let trie_nodes = store.get_trie_nodes( request.root_hash, - paths.into_iter().map(|bytes| bytes.to_vec()).collect(), + paths.into_iter().map(|bytes| compact_to_hex(bytes)).collect(), remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); @@ -145,56 +145,30 @@ pub fn process_trie_nodes_request( }) } -// // func keybytesToHex(str []byte) []byte { -// // l := len(str)*2 + 1 -// // var nibbles = make([]byte, l) -// // for i, b := range str { -// // nibbles[i*2] = b / 16 -// // nibbles[i*2+1] = b % 16 -// // } -// // nibbles[l-1] = 16 -// // return nibbles -// // } - -// fn keybytes_to_hex(keybytes: Bytes) -> Vec { -// let l = keybytes.len()*2+1; -// let mut nibbles = vec![0;l]; -// for (i, b) in keybytes.into_iter().enumerate() { -// nibbles[i*2] = b / 16; -// nibbles[i*2+1] = b % 16; -// } -// nibbles[l - 1] = 16; -// nibbles -// } - -// // func compactToHex(compact []byte) []byte { -// // if len(compact) == 0 { -// // return compact -// // } -// // base := keybytesToHex(compact) -// // // delete terminator flag -// // if base[0] < 2 { -// // base = base[:len(base)-1] -// // } -// // // apply odd flag -// // chop := 2 - base[0]&1 -// // return base[chop:] -// // } - -// fn compact_to_hex(compact: Bytes) -> Vec { -// if compact.is_empty() { -// return vec![] -// } -// let mut base = keybytes_to_hex(compact); -// // delete terminator flag -// if base[0] < 2 { -// base = base[..base.len() - 1].to_vec(); -// } -// // apply odd flag -// let chop = 2 - (base[0]&1) as usize; -// base[chop..].to_vec() - -// } +fn keybytes_to_hex(keybytes: Bytes) -> Vec { + let l = keybytes.len() * 2 + 1; + let mut nibbles = vec![0; l]; + for (i, b) in keybytes.into_iter().enumerate() { + nibbles[i * 2] = b / 16; + nibbles[i * 2 + 1] = b % 16; + } + nibbles[l - 1] = 16; + nibbles +} + +fn compact_to_hex(compact: Bytes) -> Vec { + if compact.is_empty() || compact.len() == 32 { + return compact.to_vec(); + } + let mut base = keybytes_to_hex(compact); + // delete terminator flag + if base[0] < 2 { + base = base[..base.len() - 1].to_vec(); + } + // apply odd flag + let chop = 2 - (base[0] & 1) as usize; + base[chop..].to_vec() +} #[cfg(test)] mod tests { diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 026bc42df..c70a3d6e4 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -801,7 +801,7 @@ impl Store { if bytes_used >= byte_limit { break; } - if let Some(node) = storage_trie.get_node(path)? { + if let Some(node) = storage_trie.get_node_partial(path)? { bytes_used += node.len() as u64; nodes.push(node); } diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index cd8bacc40..604ea4cc4 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -114,4 +114,15 @@ impl Node { Node::Leaf(n) => n.get_node(path), } } + + pub fn encode_raw( + self, + path_offset: usize, + ) -> Vec { + match self { + Node::Branch(n) => n.encode_raw(), + Node::Extension(n) => n.encode_raw(), + Node::Leaf(n) => n.encode_raw(path_offset), + } + } } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 0de9df58c..0c2626106 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -210,6 +210,58 @@ impl Trie { } } + pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result>, TrieError> { + println!("Getting node with partial path: {:?}", partial_path); + let Some(root_node) = self + .root + .as_ref() + .map(|root| self.state.get_node(root.clone())) + .transpose()? + .flatten() + else { + return Ok(None); + }; + + println!("Root Node: {:?}", root_node); + self.get_node_inner(root_node, partial_path, 0) + } + + fn get_node_inner(&self, node: Node, partial_path: &Vec, pos: usize) -> Result>, TrieError> { + if pos == partial_path.len() { + return Ok(Some(node.encode_raw(pos))) + } + match node { + Node::Branch(branch_node) => { + match partial_path.get(pos) { + Some(idx) if *idx <= 16 => { + let child_hash = &branch_node.choices[*idx as usize]; + if child_hash.is_valid() { + let child_node = self.state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + self.get_node_inner(child_node, partial_path, pos+1) + } else { + Ok(None) + } + }, + _=> Ok(Some(branch_node.encode_raw())) + } + }, + Node::Extension(extension_node) => { + if extension_node.child.is_valid() { + let child_node = self.state + .get_node(extension_node.child.clone())? + .expect("inconsistent internal tree structure"); + self.get_node_inner(child_node, partial_path, pos+1) + } else { + Ok(None) + } + }, + Node::Leaf(_) => Ok(None), + } + + } + #[cfg(all(test, feature = "libmdbx"))] /// Creates a new Trie based on a temporary Libmdbx DB fn new_temp() -> Self { From ea34fbca1d704710f906900e82d50a2ac580b634 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 11:49:03 -0300 Subject: [PATCH 081/345] Progress --- crates/networking/p2p/snap.rs | 6 ++++- crates/storage/store/storage.rs | 11 +++----- crates/storage/trie/node.rs | 5 +--- crates/storage/trie/trie.rs | 47 ++++++++++++++++++--------------- 4 files changed, 36 insertions(+), 33 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 409e26703..49ffc4ce6 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -123,12 +123,16 @@ pub fn process_trie_nodes_request( request: GetTrieNodes, store: Store, ) -> Result { + println!("PROCESSING REQUEST"); let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { let trie_nodes = store.get_trie_nodes( request.root_hash, - paths.into_iter().map(|bytes| compact_to_hex(bytes)).collect(), + paths + .into_iter() + .map(|bytes| compact_to_hex(bytes)) + .collect(), remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index c70a3d6e4..30fbc02bd 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -775,9 +775,7 @@ impl Store { // State Trie Nodes Request if paths.len() == 1 { // Fetch state trie node - let Some(node) = state_trie.get_node(&account_path)? else { - return Ok(vec![]); - }; + let node = state_trie.get_node_partial(&account_path)?; return Ok(vec![node]); } // Storage Trie Nodes Request @@ -801,10 +799,9 @@ impl Store { if bytes_used >= byte_limit { break; } - if let Some(node) = storage_trie.get_node_partial(path)? { - bytes_used += node.len() as u64; - nodes.push(node); - } + let node = storage_trie.get_node_partial(path)?; + bytes_used += node.len() as u64; + nodes.push(node); } Ok(nodes) } diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 604ea4cc4..55084ce58 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -115,10 +115,7 @@ impl Node { } } - pub fn encode_raw( - self, - path_offset: usize, - ) -> Vec { + pub fn encode_raw(self, path_offset: usize) -> Vec { match self { Node::Branch(n) => n.encode_raw(), Node::Extension(n) => n.encode_raw(), diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 0c2626106..a83364480 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -210,7 +210,7 @@ impl Trie { } } - pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result>, TrieError> { + pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result, TrieError> { println!("Getting node with partial path: {:?}", partial_path); let Some(root_node) = self .root @@ -219,47 +219,52 @@ impl Trie { .transpose()? .flatten() else { - return Ok(None); + return Ok(vec![]); }; - println!("Root Node: {:?}", root_node); - self.get_node_inner(root_node, partial_path, 0) + let node = self.get_node_inner(root_node, partial_path, 0)?; + println!("Node: {:?}", node); + Ok(node) } - fn get_node_inner(&self, node: Node, partial_path: &Vec, pos: usize) -> Result>, TrieError> { + fn get_node_inner( + &self, + node: Node, + partial_path: &Vec, + pos: usize, + ) -> Result, TrieError> { if pos == partial_path.len() { - return Ok(Some(node.encode_raw(pos))) + return Ok(node.encode_raw(pos)); } match node { - Node::Branch(branch_node) => { - match partial_path.get(pos) { - Some(idx) if *idx <= 16 => { - let child_hash = &branch_node.choices[*idx as usize]; + Node::Branch(branch_node) => match partial_path.get(pos) { + Some(idx) if *idx <= 16 => { + let child_hash = &branch_node.choices[*idx as usize]; if child_hash.is_valid() { - let child_node = self.state + let child_node = self + .state .get_node(child_hash.clone())? .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, pos+1) + self.get_node_inner(child_node, partial_path, pos + 1) } else { - Ok(None) + Ok(vec![]) } - }, - _=> Ok(Some(branch_node.encode_raw())) } + _ => Ok(branch_node.encode_raw()), }, Node::Extension(extension_node) => { if extension_node.child.is_valid() { - let child_node = self.state + let child_node = self + .state .get_node(extension_node.child.clone())? .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, pos+1) + self.get_node_inner(child_node, partial_path, pos + 1) } else { - Ok(None) + Ok(vec![]) } - }, - Node::Leaf(_) => Ok(None), + } + Node::Leaf(_) => Ok(vec![]), } - } #[cfg(all(test, feature = "libmdbx"))] From e57f3861a165243c854217ba3abc4b97e419a4ac Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 11:51:06 -0300 Subject: [PATCH 082/345] remove unused code --- crates/storage/trie/node.rs | 13 ------------- crates/storage/trie/node/branch.rs | 28 --------------------------- crates/storage/trie/node/extension.rs | 19 ------------------ crates/storage/trie/node/leaf.rs | 9 --------- crates/storage/trie/trie.rs | 26 ------------------------- 5 files changed, 95 deletions(-) diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 55084ce58..0c6468478 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -102,19 +102,6 @@ impl Node { } } - /// Obtain the encoded node given its path. - pub fn get_node( - &self, - state: &TrieState, - path: NibbleSlice, - ) -> Result>, TrieError> { - match self { - Node::Branch(n) => n.get_node(state, path), - Node::Extension(n) => n.get_node(state, path), - Node::Leaf(n) => n.get_node(path), - } - } - pub fn encode_raw(self, path_offset: usize) -> Vec { match self { Node::Branch(n) => n.encode_raw(), diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 3b9c1962f..a5279bc2e 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -341,34 +341,6 @@ impl BranchNode { } Ok(()) } - - /// Obtain the encoded node given its path. - pub fn get_node( - &self, - state: &TrieState, - mut path: NibbleSlice, - ) -> Result>, TrieError> { - // If path is at the end, then path must be at self or not exist in the trie. - // Otherwise, check the corresponding choice and delegate accordingly if present. - if let Some(choice) = path.next().map(usize::from) { - // Delegate to children if present - let child_hash = &self.choices[choice]; - if child_hash.is_valid() { - let child_node = state - .get_node(child_hash.clone())? - .expect("inconsistent internal tree structure"); - child_node.get_node(state, path) - } else { - Ok(None) - } - } else { - if path.len() == 0 || self.path == path.data() { - Ok(Some(self.encode_raw())) - } else { - Ok(None) - } - } - } } #[cfg(test)] diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index ecdc4b276..4e0c0d0e4 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -215,25 +215,6 @@ impl ExtensionNode { } Ok(()) } - - /// Obtain the encoded node given its path. - pub fn get_node( - &self, - state: &TrieState, - mut path: NibbleSlice, - ) -> Result>, TrieError> { - // If the path is prefixed by this node's prefix, delegate to its child. - // Otherwise, the path doesn't belong to the trie. - if path.skip_prefix(&self.prefix) { - let child_node = state - .get_node(self.child.clone())? - .expect("inconsistent internal tree structure"); - - child_node.get_node(state, path) - } else { - Ok(None) - } - } } #[cfg(test)] diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 0cb86dd8f..b6684e61d 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -166,15 +166,6 @@ impl LeafNode { } Ok(()) } - - /// Obtain the encoded node given its path. - pub fn get_node(&self, path: NibbleSlice) -> Result>, TrieError> { - if path.cmp_rest(&self.path) { - Ok(Some(self.encode_raw(path.offset()))) - } else { - Ok(None) - } - } } #[cfg(test)] diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index a83364480..6c69399f5 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -184,32 +184,6 @@ impl Trie { /// Obtain the encoded node given its path. /// Allows usage of partial paths - pub fn get_node(&self, path: &PathRLP) -> Result>, TrieError> { - if path.is_empty() { - return Ok(None); - } - println!("Getting node with path: {:?}", path); - let path = if path == &vec![0] { - NibbleSlice::new(&[]) - } else { - NibbleSlice::new(path) - }; - if let Some(root_node) = self - .root - .as_ref() - .map(|root| self.state.get_node(root.clone())) - .transpose()? - .flatten() - { - println!("Node: {:?}", root_node); - let res = root_node.get_node(&self.state, path)?; - println!("Node: {:?}", res); - Ok(res) - } else { - Ok(None) - } - } - pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result, TrieError> { println!("Getting node with partial path: {:?}", partial_path); let Some(root_node) = self From 0a31fc9c0e1774c756b9bbde94157546e82868ac Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 12:13:56 -0300 Subject: [PATCH 083/345] Handle wrong path input lens --- crates/networking/p2p/rlpx/error.rs | 2 ++ crates/networking/p2p/snap.rs | 46 ++++++++++++++++++----------- crates/storage/trie/trie.rs | 3 ++ 3 files changed, 33 insertions(+), 18 deletions(-) diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 83b158762..cd31aecdb 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -13,4 +13,6 @@ pub(crate) enum RLPxError { UnexpectedMessage(Message), #[error(transparent)] Store(#[from] StoreError), + #[error("Bad Request: {0}")] + BadRequest(String), } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 49ffc4ce6..7ec8d2c3b 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -2,9 +2,12 @@ use bytes::Bytes; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use crate::rlpx::snap::{ - AccountRange, AccountRangeUnit, AccountStateSlim, ByteCodes, GetAccountRange, GetByteCodes, - GetStorageRanges, GetTrieNodes, StorageRanges, StorageSlot, TrieNodes, +use crate::rlpx::{ + error::RLPxError, + snap::{ + AccountRange, AccountRangeUnit, AccountStateSlim, ByteCodes, GetAccountRange, GetByteCodes, + GetStorageRanges, GetTrieNodes, StorageRanges, StorageSlot, TrieNodes, + }, }; pub fn process_account_range_request( @@ -122,8 +125,7 @@ pub fn process_byte_codes_request( pub fn process_trie_nodes_request( request: GetTrieNodes, store: Store, -) -> Result { - println!("PROCESSING REQUEST"); +) -> Result { let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { @@ -131,8 +133,8 @@ pub fn process_trie_nodes_request( request.root_hash, paths .into_iter() - .map(|bytes| compact_to_hex(bytes)) - .collect(), + .map(|bytes| process_path_input(bytes)) + .collect::>()?, remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); @@ -149,21 +151,18 @@ pub fn process_trie_nodes_request( }) } -fn keybytes_to_hex(keybytes: Bytes) -> Vec { - let l = keybytes.len() * 2 + 1; - let mut nibbles = vec![0; l]; - for (i, b) in keybytes.into_iter().enumerate() { - nibbles[i * 2] = b / 16; - nibbles[i * 2 + 1] = b % 16; +fn process_path_input(bytes: Bytes) -> Result, RLPxError> { + match bytes.len() { + 0 => Err(RLPxError::BadRequest( + "zero-item pathset requested".to_string(), + )), + n if n < 32 => Ok(compact_to_hex(bytes)), + _ => Ok(bytes.to_vec()), } - nibbles[l - 1] = 16; - nibbles } fn compact_to_hex(compact: Bytes) -> Vec { - if compact.is_empty() || compact.len() == 32 { - return compact.to_vec(); - } + // We already checked that compact is not empty let mut base = keybytes_to_hex(compact); // delete terminator flag if base[0] < 2 { @@ -174,6 +173,17 @@ fn compact_to_hex(compact: Bytes) -> Vec { base[chop..].to_vec() } +fn keybytes_to_hex(keybytes: Bytes) -> Vec { + let l = keybytes.len() * 2 + 1; + let mut nibbles = vec![0; l]; + for (i, b) in keybytes.into_iter().enumerate() { + nibbles[i * 2] = b / 16; + nibbles[i * 2 + 1] = b % 16; + } + nibbles[l - 1] = 16; + nibbles +} + #[cfg(test)] mod tests { use std::str::FromStr; diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 6c69399f5..194e94468 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -185,6 +185,9 @@ impl Trie { /// Obtain the encoded node given its path. /// Allows usage of partial paths pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result, TrieError> { + if partial_path.len() > 32 { + return Ok(vec![]) + } println!("Getting node with partial path: {:?}", partial_path); let Some(root_node) = self .root From 2c0e30eec7b6c863ff095a157adbd315205996c1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 14:29:56 -0300 Subject: [PATCH 084/345] Add restrictions --- crates/storage/trie/nibble.rs | 2 +- crates/storage/trie/trie.rs | 18 ++++++++++++++++-- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/nibble.rs b/crates/storage/trie/nibble.rs index 537f9d29a..dd9f93946 100644 --- a/crates/storage/trie/nibble.rs +++ b/crates/storage/trie/nibble.rs @@ -322,7 +322,7 @@ impl NibbleVec { } } - #[cfg(test)] + //#[cfg(test)] /// Create a NibbleVec from an iterator of nibbles pub fn from_nibbles( data_iter: impl Iterator, diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 194e94468..e33a33ff5 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -12,6 +12,7 @@ mod test_utils; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; +use nibble::{Nibble, NibbleVec}; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; @@ -227,15 +228,28 @@ impl Trie { Ok(vec![]) } } - _ => Ok(branch_node.encode_raw()), + _ => if &branch_node.path == partial_path { + Ok(branch_node.encode_raw()) + } else { + Ok(vec![]) + } }, Node::Extension(extension_node) => { + // len(path)-pos < len(n.Key) || !bytes.Equal(n.Key, path[pos:pos+len(n.Key)]) + if partial_path.len()-pos < extension_node.prefix.len() { + return Ok(vec![]) + } + // Compare prefix + let nibble_vec = NibbleVec::from_nibbles(partial_path[pos..pos+extension_node.prefix.len()].iter().map(|b| Nibble::try_from(*b).unwrap()), false); + if extension_node.prefix != nibble_vec { + return Ok(vec![]) + } if extension_node.child.is_valid() { let child_node = self .state .get_node(extension_node.child.clone())? .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, pos + 1) + self.get_node_inner(child_node, partial_path, pos + extension_node.prefix.len()) } else { Ok(vec![]) } From b3c4816bd6903cc910d83343800a950530fcc65f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 15:14:34 -0300 Subject: [PATCH 085/345] Fix error handling --- crates/networking/p2p/snap.rs | 20 ++++++++++++-------- crates/storage/trie/trie.rs | 2 +- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 7ec8d2c3b..abb295e84 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -129,12 +129,17 @@ pub fn process_trie_nodes_request( let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { + if paths.is_empty() { + return Err(RLPxError::BadRequest( + "zero-item pathset requested".to_string(), + )); + } let trie_nodes = store.get_trie_nodes( request.root_hash, paths .into_iter() .map(|bytes| process_path_input(bytes)) - .collect::>()?, + .collect(), remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); @@ -151,18 +156,17 @@ pub fn process_trie_nodes_request( }) } -fn process_path_input(bytes: Bytes) -> Result, RLPxError> { +fn process_path_input(bytes: Bytes) -> Vec { match bytes.len() { - 0 => Err(RLPxError::BadRequest( - "zero-item pathset requested".to_string(), - )), - n if n < 32 => Ok(compact_to_hex(bytes)), - _ => Ok(bytes.to_vec()), + n if n < 32 => compact_to_hex(bytes), + _ => bytes.to_vec(), } } fn compact_to_hex(compact: Bytes) -> Vec { - // We already checked that compact is not empty + if compact.is_empty() { + return vec![]; + } let mut base = keybytes_to_hex(compact); // delete terminator flag if base[0] < 2 { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index e33a33ff5..bcb449e68 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -187,7 +187,7 @@ impl Trie { /// Allows usage of partial paths pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result, TrieError> { if partial_path.len() > 32 { - return Ok(vec![]) + return Ok(vec![]); } println!("Getting node with partial path: {:?}", partial_path); let Some(root_node) = self From 80bbc30f2f99dcf6be420d96d1aa149537811d6d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 15:16:29 -0300 Subject: [PATCH 086/345] clippy+fmt --- crates/networking/p2p/snap.rs | 2 +- crates/storage/store/storage.rs | 4 ++-- crates/storage/trie/trie.rs | 23 +++++++++++++++-------- 3 files changed, 18 insertions(+), 11 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index abb295e84..c43bdbaea 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -138,7 +138,7 @@ pub fn process_trie_nodes_request( request.root_hash, paths .into_iter() - .map(|bytes| process_path_input(bytes)) + .map(process_path_input) .collect(), remaining_bytes, )?; diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 30fbc02bd..5c59fa122 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -775,13 +775,13 @@ impl Store { // State Trie Nodes Request if paths.len() == 1 { // Fetch state trie node - let node = state_trie.get_node_partial(&account_path)?; + let node = state_trie.get_node_partial(account_path)?; return Ok(vec![node]); } // Storage Trie Nodes Request let mut nodes = vec![]; let Some(account_state) = state_trie - .get(&account_path)? + .get(account_path)? .map(|ref rlp| AccountState::decode(rlp)) .transpose()? else { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index bcb449e68..88af9513e 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -228,21 +228,28 @@ impl Trie { Ok(vec![]) } } - _ => if &branch_node.path == partial_path { - Ok(branch_node.encode_raw()) - } else { - Ok(vec![]) + _ => { + if &branch_node.path == partial_path { + Ok(branch_node.encode_raw()) + } else { + Ok(vec![]) + } } }, Node::Extension(extension_node) => { // len(path)-pos < len(n.Key) || !bytes.Equal(n.Key, path[pos:pos+len(n.Key)]) - if partial_path.len()-pos < extension_node.prefix.len() { - return Ok(vec![]) + if partial_path.len() - pos < extension_node.prefix.len() { + return Ok(vec![]); } // Compare prefix - let nibble_vec = NibbleVec::from_nibbles(partial_path[pos..pos+extension_node.prefix.len()].iter().map(|b| Nibble::try_from(*b).unwrap()), false); + let nibble_vec = NibbleVec::from_nibbles( + partial_path[pos..pos + extension_node.prefix.len()] + .iter() + .map(|b| Nibble::try_from(*b).unwrap()), + false, + ); if extension_node.prefix != nibble_vec { - return Ok(vec![]) + return Ok(vec![]); } if extension_node.child.is_valid() { let child_node = self From b7b978a09b2fc88c4019de4e10d172067a015659 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 15:17:16 -0300 Subject: [PATCH 087/345] Add hive test --- .github/workflows/hive.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 041ed4355..82614c56d 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -38,7 +38,7 @@ jobs: run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" - simulation: snap name: "Devp2p snap tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges|ByteCodes" + run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges|ByteCodes|TrieNodes" - simulation: engine name: "Engine tests" run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" From f02d29a6fd3d9e5444cfe7d0cd32e526a91f496b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 15:41:09 -0300 Subject: [PATCH 088/345] Cleanup --- crates/networking/p2p/snap.rs | 5 +---- crates/storage/store/storage.rs | 8 ++++---- crates/storage/trie/trie.rs | 5 +---- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index c43bdbaea..9cf9cca7e 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -136,10 +136,7 @@ pub fn process_trie_nodes_request( } let trie_nodes = store.get_trie_nodes( request.root_hash, - paths - .into_iter() - .map(process_path_input) - .collect(), + paths.into_iter().map(process_path_input).collect(), remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 5c59fa122..b9f295d56 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -760,7 +760,7 @@ impl Store { /// If more than one hash is received, then the storage trie nodes where each storage key is stored will be returned /// Missing nodes will be skipped. /// For more information check out snap capability message [`GetTrieNodes`](https://github.com/ethereum/devp2p/blob/master/caps/snap.md#gettrienodes-0x06) - /// The paths can be either full paths (hash) or partial paths (bytes), if a partial path is given for the account this method will not return storage nodes for it (TODO: FIX) + /// The paths can be either full paths (hash) or partial paths (bytes), if a partial path is given for the account this method will not return storage nodes for it pub fn get_trie_nodes( &self, state_root: H256, @@ -775,7 +775,7 @@ impl Store { // State Trie Nodes Request if paths.len() == 1 { // Fetch state trie node - let node = state_trie.get_node_partial(account_path)?; + let node = state_trie.get_node(account_path)?; return Ok(vec![node]); } // Storage Trie Nodes Request @@ -787,7 +787,7 @@ impl Store { else { return Ok(vec![]); }; - // We can't access the storage trie without the account's address hash (TODO: FIX THIS) + // We can't access the storage trie without the account's address hash let Ok(hashed_address) = account_path.clone().try_into().map(H256) else { return Ok(nodes); }; @@ -799,7 +799,7 @@ impl Store { if bytes_used >= byte_limit { break; } - let node = storage_trie.get_node_partial(path)?; + let node = storage_trie.get_node(path)?; bytes_used += node.len() as u64; nodes.push(node); } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 88af9513e..e55ec1825 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -185,11 +185,10 @@ impl Trie { /// Obtain the encoded node given its path. /// Allows usage of partial paths - pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result, TrieError> { + pub fn get_node(&self, partial_path: &PathRLP) -> Result, TrieError> { if partial_path.len() > 32 { return Ok(vec![]); } - println!("Getting node with partial path: {:?}", partial_path); let Some(root_node) = self .root .as_ref() @@ -201,7 +200,6 @@ impl Trie { }; let node = self.get_node_inner(root_node, partial_path, 0)?; - println!("Node: {:?}", node); Ok(node) } @@ -237,7 +235,6 @@ impl Trie { } }, Node::Extension(extension_node) => { - // len(path)-pos < len(n.Key) || !bytes.Equal(n.Key, path[pos:pos+len(n.Key)]) if partial_path.len() - pos < extension_node.prefix.len() { return Ok(vec![]); } From 0e5a00547070c09064b99ea8a9f9b945ce085be7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 16:56:10 -0300 Subject: [PATCH 089/345] Draft --- crates/networking/p2p/snap.rs | 9 ++++++++- crates/storage/trie/trie.rs | 38 +++++++++++------------------------ 2 files changed, 20 insertions(+), 27 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 9cf9cca7e..885ddab50 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -155,11 +155,18 @@ pub fn process_trie_nodes_request( fn process_path_input(bytes: Bytes) -> Vec { match bytes.len() { - n if n < 32 => compact_to_hex(bytes), + n if n < 32 => nibbles_to_bytes(compact_to_hex(bytes)), _ => bytes.to_vec(), } } +fn nibbles_to_bytes(nibbles: Vec) -> Vec { + nibbles.chunks(2).map(|chunk| match chunk.len() { + 1 => chunk[0] << 4, + _ /* 2 */ => chunk[0] << 4 | chunk[1] + }).collect::>() +} + fn compact_to_hex(compact: Bytes) -> Vec { if compact.is_empty() { return vec![]; diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index e55ec1825..262c1ccfc 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -12,7 +12,6 @@ mod test_utils; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; -use nibble::{Nibble, NibbleVec}; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; @@ -199,35 +198,34 @@ impl Trie { return Ok(vec![]); }; - let node = self.get_node_inner(root_node, partial_path, 0)?; + let node = self.get_node_inner(root_node, NibbleSlice::new(partial_path))?; Ok(node) } fn get_node_inner( &self, node: Node, - partial_path: &Vec, - pos: usize, + mut partial_path: NibbleSlice, ) -> Result, TrieError> { - if pos == partial_path.len() { - return Ok(node.encode_raw(pos)); + if partial_path.len() == 0 { + return Ok(node.encode_raw(partial_path.offset())); } match node { - Node::Branch(branch_node) => match partial_path.get(pos) { - Some(idx) if *idx <= 16 => { - let child_hash = &branch_node.choices[*idx as usize]; + Node::Branch(branch_node) => match partial_path.next().map(usize::from) { + Some(idx) if idx <= 16 => { + let child_hash = &branch_node.choices[idx as usize]; if child_hash.is_valid() { let child_node = self .state .get_node(child_hash.clone())? .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, pos + 1) + self.get_node_inner(child_node, partial_path) } else { Ok(vec![]) } } _ => { - if &branch_node.path == partial_path { + if partial_path.cmp_rest(&branch_node.path) { Ok(branch_node.encode_raw()) } else { Ok(vec![]) @@ -235,25 +233,13 @@ impl Trie { } }, Node::Extension(extension_node) => { - if partial_path.len() - pos < extension_node.prefix.len() { - return Ok(vec![]); - } - // Compare prefix - let nibble_vec = NibbleVec::from_nibbles( - partial_path[pos..pos + extension_node.prefix.len()] - .iter() - .map(|b| Nibble::try_from(*b).unwrap()), - false, - ); - if extension_node.prefix != nibble_vec { - return Ok(vec![]); - } - if extension_node.child.is_valid() { + if partial_path.skip_prefix(&extension_node.prefix) && extension_node.child.is_valid() { let child_node = self .state .get_node(extension_node.child.clone())? .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, pos + extension_node.prefix.len()) + dbg!(&child_node); + self.get_node_inner(child_node, partial_path) } else { Ok(vec![]) } From 003d445b168603d67d2443764c3c843791401342 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 17:17:29 -0300 Subject: [PATCH 090/345] :( --- crates/networking/p2p/snap.rs | 1 + crates/storage/trie/trie.rs | 9 ++++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 885ddab50..97eaf63d1 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -126,6 +126,7 @@ pub fn process_trie_nodes_request( request: GetTrieNodes, store: Store, ) -> Result { + println!("Request: {request:?}"); let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 262c1ccfc..fcb061ee0 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -185,6 +185,7 @@ impl Trie { /// Obtain the encoded node given its path. /// Allows usage of partial paths pub fn get_node(&self, partial_path: &PathRLP) -> Result, TrieError> { + println!("Get node from partial path: {partial_path:?}"); if partial_path.len() > 32 { return Ok(vec![]); } @@ -199,6 +200,7 @@ impl Trie { }; let node = self.get_node_inner(root_node, NibbleSlice::new(partial_path))?; + println!("Node got: {node:?}"); Ok(node) } @@ -207,11 +209,16 @@ impl Trie { node: Node, mut partial_path: NibbleSlice, ) -> Result, TrieError> { + println!("Partial path: {partial_path:?}, offset {}, len {}", partial_path.offset(), partial_path.len()); + // PROBLEM: We may have an odd number of nibbles here that we are not taking into account with NibbleSlice if partial_path.len() == 0 { return Ok(node.encode_raw(partial_path.offset())); } match node { - Node::Branch(branch_node) => match partial_path.next().map(usize::from) { + Node::Branch(branch_node) => { + let next = partial_path.next().map(usize::from); + println!("BR Next: {next:?}"); + match next { Some(idx) if idx <= 16 => { let child_hash = &branch_node.choices[idx as usize]; if child_hash.is_valid() { From 23974bb846bec7941f56b3ca0c67b274c31714dc Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:02:03 -0300 Subject: [PATCH 091/345] Fix code to handle nibbles properly --- crates/networking/p2p/snap.rs | 41 +----------- crates/storage/trie/trie.rs | 118 +++++++++++++++++++++++++++------- 2 files changed, 96 insertions(+), 63 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 97eaf63d1..ab08e076f 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -137,7 +137,7 @@ pub fn process_trie_nodes_request( } let trie_nodes = store.get_trie_nodes( request.root_hash, - paths.into_iter().map(process_path_input).collect(), + paths.into_iter().map(|bytes| bytes.to_vec()).collect(), remaining_bytes, )?; nodes.extend(trie_nodes.iter().map(|nodes| Bytes::copy_from_slice(nodes))); @@ -154,45 +154,6 @@ pub fn process_trie_nodes_request( }) } -fn process_path_input(bytes: Bytes) -> Vec { - match bytes.len() { - n if n < 32 => nibbles_to_bytes(compact_to_hex(bytes)), - _ => bytes.to_vec(), - } -} - -fn nibbles_to_bytes(nibbles: Vec) -> Vec { - nibbles.chunks(2).map(|chunk| match chunk.len() { - 1 => chunk[0] << 4, - _ /* 2 */ => chunk[0] << 4 | chunk[1] - }).collect::>() -} - -fn compact_to_hex(compact: Bytes) -> Vec { - if compact.is_empty() { - return vec![]; - } - let mut base = keybytes_to_hex(compact); - // delete terminator flag - if base[0] < 2 { - base = base[..base.len() - 1].to_vec(); - } - // apply odd flag - let chop = 2 - (base[0] & 1) as usize; - base[chop..].to_vec() -} - -fn keybytes_to_hex(keybytes: Bytes) -> Vec { - let l = keybytes.len() * 2 + 1; - let mut nibbles = vec![0; l]; - for (i, b) in keybytes.into_iter().enumerate() { - nibbles[i * 2] = b / 16; - nibbles[i * 2 + 1] = b % 16; - } - nibbles[l - 1] = 16; - nibbles -} - #[cfg(test)] mod tests { use std::str::FromStr; diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index fcb061ee0..09b18168e 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -189,6 +189,8 @@ impl Trie { if partial_path.len() > 32 { return Ok(vec![]); } + let partial_path = PartialPath::new(partial_path.clone()); + let (partial_path, last_byte_is_half) = partial_path.to_bytes(); let Some(root_node) = self .root .as_ref() @@ -199,7 +201,11 @@ impl Trie { return Ok(vec![]); }; - let node = self.get_node_inner(root_node, NibbleSlice::new(partial_path))?; + let node = self.get_node_inner( + root_node, + NibbleSlice::new(&partial_path), + last_byte_is_half, + )?; println!("Node got: {node:?}"); Ok(node) } @@ -208,10 +214,15 @@ impl Trie { &self, node: Node, mut partial_path: NibbleSlice, + last_byte_is_half: bool, ) -> Result, TrieError> { - println!("Partial path: {partial_path:?}, offset {}, len {}", partial_path.offset(), partial_path.len()); + println!( + "Partial path: {partial_path:?}, offset {}, len {}", + partial_path.offset(), + partial_path.len() + ); // PROBLEM: We may have an odd number of nibbles here that we are not taking into account with NibbleSlice - if partial_path.len() == 0 { + if partial_path.len() == last_byte_is_half as usize { return Ok(node.encode_raw(partial_path.offset())); } match node { @@ -219,34 +230,31 @@ impl Trie { let next = partial_path.next().map(usize::from); println!("BR Next: {next:?}"); match next { - Some(idx) if idx <= 16 => { - let child_hash = &branch_node.choices[idx as usize]; - if child_hash.is_valid() { - let child_node = self - .state - .get_node(child_hash.clone())? - .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path) - } else { - Ok(vec![]) - } - } - _ => { - if partial_path.cmp_rest(&branch_node.path) { - Ok(branch_node.encode_raw()) - } else { - Ok(vec![]) + Some(idx) if idx <= 16 => { + let child_hash = &branch_node.choices[idx as usize]; + if child_hash.is_valid() { + let child_node = self + .state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + self.get_node_inner(child_node, partial_path, last_byte_is_half) + } else { + Ok(vec![]) + } } + _ => Ok(vec![]), } - }, + } Node::Extension(extension_node) => { - if partial_path.skip_prefix(&extension_node.prefix) && extension_node.child.is_valid() { + if partial_path.skip_prefix(&extension_node.prefix) + && extension_node.child.is_valid() + { let child_node = self .state .get_node(extension_node.child.clone())? .expect("inconsistent internal tree structure"); dbg!(&child_node); - self.get_node_inner(child_node, partial_path) + self.get_node_inner(child_node, partial_path, last_byte_is_half) } else { Ok(vec![]) } @@ -278,6 +286,70 @@ impl Trie { } } +/// Struct representing a partial path, either in the form of bytes (full path) or compact-encoded bytes +pub enum PartialPath { + // Full 32-byte path expressed in bytes + Bytes(Vec), + // Partial path expressed as compact nibbles + Compact(Vec), +} + +impl PartialPath { + /// Returns the partial path represented as a byte slice and a boolean representing if the last byte is only a half byte + pub fn to_bytes(self) -> (Vec, bool) { + match self { + PartialPath::Bytes(bytes) => (bytes, false), + PartialPath::Compact(compact) => { + let nibbles = compact_to_hex(compact); + let mut last_is_half = false; + let bytes = nibbles + .chunks(2) + .map(|chunk| match chunk.len() { + 1 => { + last_is_half = true; + chunk[0] << 4 + } + // 2 + _ => chunk[0] << 4 | chunk[1], + }) + .collect::>(); + (bytes, last_is_half) + } + } + } + pub fn new(maybe_compact: Vec) -> Self { + match maybe_compact.len() { + n if n < 32 => Self::Compact(maybe_compact), + _ => Self::Bytes(maybe_compact), + } + } +} + +fn compact_to_hex(compact: Vec) -> Vec { + if compact.is_empty() { + return vec![]; + } + let mut base = keybytes_to_hex(compact); + // delete terminator flag + if base[0] < 2 { + base = base[..base.len() - 1].to_vec(); + } + // apply odd flag + let chop = 2 - (base[0] & 1) as usize; + base[chop..].to_vec() +} + +fn keybytes_to_hex(keybytes: Vec) -> Vec { + let l = keybytes.len() * 2 + 1; + let mut nibbles = vec![0; l]; + for (i, b) in keybytes.into_iter().enumerate() { + nibbles[i * 2] = b / 16; + nibbles[i * 2 + 1] = b % 16; + } + nibbles[l - 1] = 16; + nibbles +} + impl IntoIterator for Trie { type Item = Node; From b159f7d8554cd64892ae130839711908bee04a8b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:13:19 -0300 Subject: [PATCH 092/345] Simplify code --- crates/storage/trie/trie.rs | 74 ++++++++++++++----------------------- 1 file changed, 28 insertions(+), 46 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 09b18168e..ea0b027c0 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -10,6 +10,8 @@ mod trie_iter; #[cfg(test)] mod test_utils; +use std::borrow::Cow; + use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; use node::Node; @@ -183,14 +185,13 @@ impl Trie { } /// Obtain the encoded node given its path. - /// Allows usage of partial paths + /// Allows usage of full paths (byte slice of 32 bytes) or compact-encoded nibble slices (with length lower than 32) pub fn get_node(&self, partial_path: &PathRLP) -> Result, TrieError> { println!("Get node from partial path: {partial_path:?}"); if partial_path.len() > 32 { return Ok(vec![]); } - let partial_path = PartialPath::new(partial_path.clone()); - let (partial_path, last_byte_is_half) = partial_path.to_bytes(); + let (partial_path, last_byte_is_half) = maybe_compact_to_bytes(partial_path); let Some(root_node) = self .root .as_ref() @@ -216,12 +217,7 @@ impl Trie { mut partial_path: NibbleSlice, last_byte_is_half: bool, ) -> Result, TrieError> { - println!( - "Partial path: {partial_path:?}, offset {}, len {}", - partial_path.offset(), - partial_path.len() - ); - // PROBLEM: We may have an odd number of nibbles here that we are not taking into account with NibbleSlice + // If we reached the end of the partial path, return the current node if partial_path.len() == last_byte_is_half as usize { return Ok(node.encode_raw(partial_path.offset())); } @@ -253,7 +249,6 @@ impl Trie { .state .get_node(extension_node.child.clone())? .expect("inconsistent internal tree structure"); - dbg!(&child_node); self.get_node_inner(child_node, partial_path, last_byte_is_half) } else { Ok(vec![]) @@ -286,42 +281,29 @@ impl Trie { } } -/// Struct representing a partial path, either in the form of bytes (full path) or compact-encoded bytes -pub enum PartialPath { - // Full 32-byte path expressed in bytes - Bytes(Vec), - // Partial path expressed as compact nibbles - Compact(Vec), -} - -impl PartialPath { - /// Returns the partial path represented as a byte slice and a boolean representing if the last byte is only a half byte - pub fn to_bytes(self) -> (Vec, bool) { - match self { - PartialPath::Bytes(bytes) => (bytes, false), - PartialPath::Compact(compact) => { - let nibbles = compact_to_hex(compact); - let mut last_is_half = false; - let bytes = nibbles - .chunks(2) - .map(|chunk| match chunk.len() { - 1 => { - last_is_half = true; - chunk[0] << 4 - } - // 2 - _ => chunk[0] << 4 | chunk[1], - }) - .collect::>(); - (bytes, last_is_half) - } - } - } - pub fn new(maybe_compact: Vec) -> Self { - match maybe_compact.len() { - n if n < 32 => Self::Compact(maybe_compact), - _ => Self::Bytes(maybe_compact), - } +fn maybe_compact_to_bytes(maybe_compact: &Vec) -> (Cow>, bool) { + match maybe_compact.len() { + // Partial path is represented as compact nibbles + n if n < 32 => { + // Convert compact nibbles to nibbles + let nibbles = compact_to_hex(maybe_compact.clone()); + // Convert nibbles to bytes, accouning for odd number of bytes + let mut last_is_half = false; + let bytes = nibbles + .chunks(2) + .map(|chunk| match chunk.len() { + 1 => { + last_is_half = true; + chunk[0] << 4 + } + // 2 + _ => chunk[0] << 4 | chunk[1], + }) + .collect::>(); + (Cow::Owned(bytes), last_is_half) + }, + // Full path already represented as bytes + _ => (Cow::Borrowed(maybe_compact), false) } } From 12b026874bb27ded8a815de80b4be23648ef237a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:30:27 -0300 Subject: [PATCH 093/345] refactor --- crates/storage/trie/trie.rs | 64 +++++++++++++++++++------------------ 1 file changed, 33 insertions(+), 31 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index ea0b027c0..b7416bbab 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -187,11 +187,18 @@ impl Trie { /// Obtain the encoded node given its path. /// Allows usage of full paths (byte slice of 32 bytes) or compact-encoded nibble slices (with length lower than 32) pub fn get_node(&self, partial_path: &PathRLP) -> Result, TrieError> { - println!("Get node from partial path: {partial_path:?}"); - if partial_path.len() > 32 { - return Ok(vec![]); - } - let (partial_path, last_byte_is_half) = maybe_compact_to_bytes(partial_path); + // Convert compact-encoded nibbles into a byte slice if necessary + let (partial_path, last_byte_is_half) = match partial_path.len() { + // Compact-encoded nibbles + n if n < 32 => { + let (p, l) = compact_nibbles_to_bytes(partial_path); + (Cow::Owned(p), l) + } + // Full path (No conversion needed) + 32 => (Cow::Borrowed(partial_path), false), + // We won't handle paths with length over 32 + _ => return Ok(vec![]), + }; let Some(root_node) = self .root .as_ref() @@ -281,33 +288,28 @@ impl Trie { } } -fn maybe_compact_to_bytes(maybe_compact: &Vec) -> (Cow>, bool) { - match maybe_compact.len() { - // Partial path is represented as compact nibbles - n if n < 32 => { - // Convert compact nibbles to nibbles - let nibbles = compact_to_hex(maybe_compact.clone()); - // Convert nibbles to bytes, accouning for odd number of bytes - let mut last_is_half = false; - let bytes = nibbles - .chunks(2) - .map(|chunk| match chunk.len() { - 1 => { - last_is_half = true; - chunk[0] << 4 - } - // 2 - _ => chunk[0] << 4 | chunk[1], - }) - .collect::>(); - (Cow::Owned(bytes), last_is_half) - }, - // Full path already represented as bytes - _ => (Cow::Borrowed(maybe_compact), false) - } +/// Converts a slice of compact-encoded nibbles into a byte slice +/// If the nibble slice has odd-length (aka the last byte will be a half byte) returns true else false +fn compact_nibbles_to_bytes(compact: &Vec) -> (Vec, bool) { + // Convert compact nibbles to nibbles + let nibbles = compact_to_hex(compact); + // Convert nibbles to bytes, accouning for odd number of bytes + let mut last_is_half = false; + let bytes = nibbles + .chunks(2) + .map(|chunk| match chunk.len() { + 1 => { + last_is_half = true; + chunk[0] << 4 + } + // 2 + _ => chunk[0] << 4 | chunk[1], + }) + .collect::>(); + (bytes, last_is_half) } -fn compact_to_hex(compact: Vec) -> Vec { +fn compact_to_hex(compact: &Vec) -> Vec { if compact.is_empty() { return vec![]; } @@ -321,7 +323,7 @@ fn compact_to_hex(compact: Vec) -> Vec { base[chop..].to_vec() } -fn keybytes_to_hex(keybytes: Vec) -> Vec { +fn keybytes_to_hex(keybytes: &Vec) -> Vec { let l = keybytes.len() * 2 + 1; let mut nibbles = vec![0; l]; for (i, b) in keybytes.into_iter().enumerate() { From 8e4c884752ee7e5332ce02a30c808b1e20271ac7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:33:35 -0300 Subject: [PATCH 094/345] Remove debug code --- crates/storage/trie/trie.rs | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index b7416bbab..7e923a641 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -199,6 +199,8 @@ impl Trie { // We won't handle paths with length over 32 _ => return Ok(vec![]), }; + + // Fetch node let Some(root_node) = self .root .as_ref() @@ -208,14 +210,11 @@ impl Trie { else { return Ok(vec![]); }; - - let node = self.get_node_inner( + self.get_node_inner( root_node, NibbleSlice::new(&partial_path), last_byte_is_half, - )?; - println!("Node got: {node:?}"); - Ok(node) + ) } fn get_node_inner( @@ -230,11 +229,9 @@ impl Trie { } match node { Node::Branch(branch_node) => { - let next = partial_path.next().map(usize::from); - println!("BR Next: {next:?}"); - match next { - Some(idx) if idx <= 16 => { - let child_hash = &branch_node.choices[idx as usize]; + match partial_path.next().map(usize::from) { + Some(idx) if idx < 16 => { + let child_hash = &branch_node.choices[idx]; if child_hash.is_valid() { let child_node = self .state From b80f19fe9fd9b8957435e7d99560f5e5dcdb2d4a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:36:59 -0300 Subject: [PATCH 095/345] Cleanup code --- crates/networking/p2p/snap.rs | 1 - crates/storage/store/storage.rs | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index ab08e076f..fefd7374b 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -126,7 +126,6 @@ pub fn process_trie_nodes_request( request: GetTrieNodes, store: Store, ) -> Result { - println!("Request: {request:?}"); let mut nodes = vec![]; let mut remaining_bytes = request.bytes; for paths in request.paths { diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index b9f295d56..96a10cabd 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -758,9 +758,8 @@ impl Store { /// (aka a hashed account address) and the following paths will be paths in the accoun's storage trie (aka hashed storage keys) /// If only one hash (account) is received, then the state trie node containing the account will be returned. /// If more than one hash is received, then the storage trie nodes where each storage key is stored will be returned - /// Missing nodes will be skipped. /// For more information check out snap capability message [`GetTrieNodes`](https://github.com/ethereum/devp2p/blob/master/caps/snap.md#gettrienodes-0x06) - /// The paths can be either full paths (hash) or partial paths (bytes), if a partial path is given for the account this method will not return storage nodes for it + /// The paths can be either full paths (hash) or partial paths (compact-encoded nibbles), if a partial path is given for the account this method will not return storage nodes for it pub fn get_trie_nodes( &self, state_root: H256, @@ -779,7 +778,6 @@ impl Store { return Ok(vec![node]); } // Storage Trie Nodes Request - let mut nodes = vec![]; let Some(account_state) = state_trie .get(account_path)? .map(|ref rlp| AccountState::decode(rlp)) @@ -789,12 +787,13 @@ impl Store { }; // We can't access the storage trie without the account's address hash let Ok(hashed_address) = account_path.clone().try_into().map(H256) else { - return Ok(nodes); + return Ok(vec![]); }; let storage_trie = self .engine .open_storage_trie(hashed_address, account_state.storage_root); // Fetch storage trie nodes + let mut nodes = vec![]; for path in paths.iter().skip(1) { if bytes_used >= byte_limit { break; From 098a584ef869bd3d74c5af8f1f66e7871a6233ca Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:48:05 -0300 Subject: [PATCH 096/345] Remove debug command --- Makefile | 3 --- 1 file changed, 3 deletions(-) diff --git a/Makefile b/Makefile index 6468711f0..39440f534 100644 --- a/Makefile +++ b/Makefile @@ -100,8 +100,5 @@ run-hive: build-image setup-hive ## 🧪 Run Hive testing suite run-hive-debug: build-image setup-hive ## 🐞 Run Hive testing suite in debug mode cd hive && ./hive --sim $(SIMULATION) --client ethereumrust --sim.limit "$(TEST_PATTERN)" --docker.output -run-hive-geth: build-image setup-hive ## 🧪 Run Hive testing suite - cd hive && ./hive --sim $(SIMULATION) --client go-ethereum --sim.limit "$(TEST_PATTERN)" --docker.output - clean-hive-logs: ## 🧹 Clean Hive logs rm -rf ./hive/workspace/logs From 9a57e371239a1b5e9367792e873557d363388d9e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:48:48 -0300 Subject: [PATCH 097/345] Un-comment noisy info --- cmd/ethereum_rust/ethereum_rust.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index bd4441092..ff1e49803 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -126,10 +126,10 @@ async fn main() { let size = blocks.len(); for block in &blocks { let hash = block.header.compute_block_hash(); - // info!( - // "Adding block {} with hash {:#x}.", - // block.header.number, hash - // ); + info!( + "Adding block {} with hash {:#x}.", + block.header.number, hash + ); let result = add_block(block, &store); if let Some(error) = result.err() { warn!( From f54af1df26113ad732e4f9174c9b88a2c6516e31 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:49:57 -0300 Subject: [PATCH 098/345] Update comment --- crates/networking/p2p/rlpx/snap.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index dbfe1d587..10597994d 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -68,7 +68,7 @@ pub(crate) struct GetTrieNodes { pub id: u64, pub root_hash: H256, // [[acc_path, slot_path_1, slot_path_2,...]...] - // The paths can be either full paths (hash) or only the partial path (bytes) + // The paths can be either full paths (hash) or only the partial path (compact-encoded nibbles) pub paths: Vec>, pub bytes: u64, } From b307261f0881f7803fec6d7931bf715f2c8b8657 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:53:59 -0300 Subject: [PATCH 099/345] reorder --- crates/storage/store/storage.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 96a10cabd..62447c720 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -769,7 +769,6 @@ impl Store { let Some(account_path) = paths.first() else { return Ok(vec![]); }; - let mut bytes_used = 0; let state_trie = self.engine.open_state_trie(state_root); // State Trie Nodes Request if paths.len() == 1 { @@ -794,6 +793,7 @@ impl Store { .open_storage_trie(hashed_address, account_state.storage_root); // Fetch storage trie nodes let mut nodes = vec![]; + let mut bytes_used = 0; for path in paths.iter().skip(1) { if bytes_used >= byte_limit { break; From 9da6ec982cea74c7da8261a02968b999e96cf527 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 18:57:41 -0300 Subject: [PATCH 100/345] Cite sources --- crates/storage/trie/trie.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 7e923a641..675702130 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -306,6 +306,7 @@ fn compact_nibbles_to_bytes(compact: &Vec) -> (Vec, bool) { (bytes, last_is_half) } +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L82 fn compact_to_hex(compact: &Vec) -> Vec { if compact.is_empty() { return vec![]; @@ -320,6 +321,7 @@ fn compact_to_hex(compact: &Vec) -> Vec { base[chop..].to_vec() } +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L96 fn keybytes_to_hex(keybytes: &Vec) -> Vec { let l = keybytes.len() * 2 + 1; let mut nibbles = vec![0; l]; From 8a0fbbbae23d22adaf79fce24014653c9287885a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 1 Nov 2024 19:01:20 -0300 Subject: [PATCH 101/345] Move funcs to encodings module --- crates/storage/trie/encoding.rs | 47 ++++++++++++++++++++ crates/storage/trie/trie.rs | 78 +++++++-------------------------- 2 files changed, 62 insertions(+), 63 deletions(-) create mode 100644 crates/storage/trie/encoding.rs diff --git a/crates/storage/trie/encoding.rs b/crates/storage/trie/encoding.rs new file mode 100644 index 000000000..ff3dba7b6 --- /dev/null +++ b/crates/storage/trie/encoding.rs @@ -0,0 +1,47 @@ +/// Converts a slice of compact-encoded nibbles into a byte slice +/// If the nibble slice has odd-length (aka the last byte will be a half byte) returns true else false +pub fn compact_nibbles_to_bytes(compact: &[u8]) -> (Vec, bool) { + // Convert compact nibbles to nibbles + let nibbles = compact_to_hex(compact); + // Convert nibbles to bytes, accouning for odd number of bytes + let mut last_is_half = false; + let bytes = nibbles + .chunks(2) + .map(|chunk| match chunk.len() { + 1 => { + last_is_half = true; + chunk[0] << 4 + } + // 2 + _ => chunk[0] << 4 | chunk[1], + }) + .collect::>(); + (bytes, last_is_half) +} + +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L82 +fn compact_to_hex(compact: &[u8]) -> Vec { + if compact.is_empty() { + return vec![]; + } + let mut base = keybytes_to_hex(compact); + // delete terminator flag + if base[0] < 2 { + base = base[..base.len() - 1].to_vec(); + } + // apply odd flag + let chop = 2 - (base[0] & 1) as usize; + base[chop..].to_vec() +} + +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L96 +fn keybytes_to_hex(keybytes: &[u8]) -> Vec { + let l = keybytes.len() * 2 + 1; + let mut nibbles = vec![0; l]; + for (i, b) in keybytes.iter().enumerate() { + nibbles[i * 2] = b / 16; + nibbles[i * 2 + 1] = b % 16; + } + nibbles[l - 1] = 16; + nibbles +} diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 675702130..da498f798 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -1,4 +1,5 @@ mod db; +mod encoding; mod error; mod nibble; mod node; @@ -12,6 +13,7 @@ mod test_utils; use std::borrow::Cow; +use encoding::compact_nibbles_to_bytes; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; use node::Node; @@ -228,23 +230,21 @@ impl Trie { return Ok(node.encode_raw(partial_path.offset())); } match node { - Node::Branch(branch_node) => { - match partial_path.next().map(usize::from) { - Some(idx) if idx < 16 => { - let child_hash = &branch_node.choices[idx]; - if child_hash.is_valid() { - let child_node = self - .state - .get_node(child_hash.clone())? - .expect("inconsistent internal tree structure"); - self.get_node_inner(child_node, partial_path, last_byte_is_half) - } else { - Ok(vec![]) - } + Node::Branch(branch_node) => match partial_path.next().map(usize::from) { + Some(idx) if idx < 16 => { + let child_hash = &branch_node.choices[idx]; + if child_hash.is_valid() { + let child_node = self + .state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + self.get_node_inner(child_node, partial_path, last_byte_is_half) + } else { + Ok(vec![]) } - _ => Ok(vec![]), } - } + _ => Ok(vec![]), + }, Node::Extension(extension_node) => { if partial_path.skip_prefix(&extension_node.prefix) && extension_node.child.is_valid() @@ -285,54 +285,6 @@ impl Trie { } } -/// Converts a slice of compact-encoded nibbles into a byte slice -/// If the nibble slice has odd-length (aka the last byte will be a half byte) returns true else false -fn compact_nibbles_to_bytes(compact: &Vec) -> (Vec, bool) { - // Convert compact nibbles to nibbles - let nibbles = compact_to_hex(compact); - // Convert nibbles to bytes, accouning for odd number of bytes - let mut last_is_half = false; - let bytes = nibbles - .chunks(2) - .map(|chunk| match chunk.len() { - 1 => { - last_is_half = true; - chunk[0] << 4 - } - // 2 - _ => chunk[0] << 4 | chunk[1], - }) - .collect::>(); - (bytes, last_is_half) -} - -// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L82 -fn compact_to_hex(compact: &Vec) -> Vec { - if compact.is_empty() { - return vec![]; - } - let mut base = keybytes_to_hex(compact); - // delete terminator flag - if base[0] < 2 { - base = base[..base.len() - 1].to_vec(); - } - // apply odd flag - let chop = 2 - (base[0] & 1) as usize; - base[chop..].to_vec() -} - -// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L96 -fn keybytes_to_hex(keybytes: &Vec) -> Vec { - let l = keybytes.len() * 2 + 1; - let mut nibbles = vec![0; l]; - for (i, b) in keybytes.into_iter().enumerate() { - nibbles[i * 2] = b / 16; - nibbles[i * 2 + 1] = b % 16; - } - nibbles[l - 1] = 16; - nibbles -} - impl IntoIterator for Trie { type Item = Node; From 1805ef6eba0cecff00fd2d5b7ad2382c652ad619 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 4 Nov 2024 16:42:35 -0300 Subject: [PATCH 102/345] Update workflow --- .github/workflows/hive.yaml | 67 ----------------------- .github/workflows/hive_and_assertoor.yaml | 2 +- 2 files changed, 1 insertion(+), 68 deletions(-) delete mode 100644 .github/workflows/hive.yaml diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml deleted file mode 100644 index 041ed4355..000000000 --- a/.github/workflows/hive.yaml +++ /dev/null @@ -1,67 +0,0 @@ -# Runs the specified hive testing suites -name: Hive -on: - merge_group: - push: - branches: [main] - pull_request: - branches: ["*"] - paths-ignore: - - "crates/l2/**" - - 'README.md' - - 'LICENSE' - - "**/README.md" - - "**/docs/**" - -concurrency: - group: ${{ github.workflow }}-${{ github.head_ref || github.run_id }} - cancel-in-progress: true - -env: - RUST_VERSION: 1.80.1 - -jobs: - run-hive: - name: ${{ matrix.name }} - runs-on: ubuntu-latest - strategy: - matrix: - include: - - simulation: rpc-compat - name: "Rpc Compat tests" - run_command: make run-hive SIMULATION=ethereum/rpc-compat TEST_PATTERN="/eth_chainId|eth_getTransactionByBlockHashAndIndex|eth_getTransactionByBlockNumberAndIndex|eth_getCode|eth_getStorageAt|eth_call|eth_getTransactionByHash|eth_getBlockByHash|eth_getBlockByNumber|eth_createAccessList|eth_getBlockTransactionCountByNumber|eth_getBlockTransactionCountByHash|eth_getBlockReceipts|eth_getTransactionReceipt|eth_blobGasPrice|eth_blockNumber|ethGetTransactionCount|debug_getRawHeader|debug_getRawBlock|debug_getRawTransaction|debug_getRawReceipts|eth_estimateGas|eth_getBalance|eth_sendRawTransaction|eth_getProof|eth_getLogs" - - simulation: rpc-auth - name: "Rpc Auth tests" - run_command: make run-hive SIMULATION=ethereum/rpc-compat TEST_PATTERN="/engine-auth" - - simulation: discv4 - name: "Devp2p discv4 tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" - - simulation: snap - name: "Devp2p snap tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges|ByteCodes" - - simulation: engine - name: "Engine tests" - run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" - - simulation: engine-cancun - name: "Cancun Engine tests" - run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="cancun/Unique Payload ID|ParentHash equals BlockHash on NewPayload|Re-Execute Payload|Payload Build after New Invalid Payload|RPC|Build Payload with Invalid ChainID|Invalid PayloadAttributes, Zero timestamp, Syncing=False|Invalid PayloadAttributes, Parent timestamp, Syncing=False|Invalid PayloadAttributes, Missing BeaconRoot, Syncing=False|Suggested Fee Recipient Test|PrevRandao Opcode Transactions Test|Invalid Missing Ancestor ReOrg, StateRoot" - steps: - - name: Checkout sources - uses: actions/checkout@v3 - - - name: Rustup toolchain install - uses: dtolnay/rust-toolchain@stable - with: - toolchain: ${{ env.RUST_VERSION }} - - - name: Setup Go - uses: actions/setup-go@v3 - - - name: Set up Docker Buildx - uses: docker/setup-buildx-action@v3 - - - name: Download Hive - run: make setup-hive - - - name: Run Hive Simulation - run: ${{ matrix.run_command }} diff --git a/.github/workflows/hive_and_assertoor.yaml b/.github/workflows/hive_and_assertoor.yaml index 998440206..64bcd3665 100644 --- a/.github/workflows/hive_and_assertoor.yaml +++ b/.github/workflows/hive_and_assertoor.yaml @@ -66,7 +66,7 @@ jobs: run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="discv4" - simulation: snap name: "Devp2p snap tests" - run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges" + run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges|ByteCodes" - simulation: engine name: "Engine tests" run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" From b270580043d4cbc877f2ebcb5e01871fa854f180 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 4 Nov 2024 16:43:28 -0300 Subject: [PATCH 103/345] Update workflow --- .github/workflows/hive.yaml | 67 ------------------------------------- 1 file changed, 67 deletions(-) delete mode 100644 .github/workflows/hive.yaml diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml deleted file mode 100644 index 86e934eb3..000000000 --- a/.github/workflows/hive.yaml +++ /dev/null @@ -1,67 +0,0 @@ -# Runs the specified hive testing suites -name: Hive -on: - merge_group: - push: - branches: [main] - pull_request: - branches: ["*"] - paths-ignore: - - "crates/l2/**" - - 'README.md' - - 'LICENSE' - - "**/README.md" - - "**/docs/**" - -concurrency: - group: ${{ github.workflow }}-${{ github.head_ref || github.run_id }} - cancel-in-progress: true - -env: - RUST_VERSION: 1.80.1 - -jobs: - run-hive: - name: ${{ matrix.name }} - runs-on: ubuntu-latest - strategy: - matrix: - include: - - simulation: rpc-compat - name: "Rpc Compat tests" - run_command: make run-hive SIMULATION=ethereum/rpc-compat TEST_PATTERN="/eth_chainId|eth_getTransactionByBlockHashAndIndex|eth_getTransactionByBlockNumberAndIndex|eth_getCode|eth_getStorageAt|eth_call|eth_getTransactionByHash|eth_getBlockByHash|eth_getBlockByNumber|eth_createAccessList|eth_getBlockTransactionCountByNumber|eth_getBlockTransactionCountByHash|eth_getBlockReceipts|eth_getTransactionReceipt|eth_blobGasPrice|eth_blockNumber|ethGetTransactionCount|debug_getRawHeader|debug_getRawBlock|debug_getRawTransaction|debug_getRawReceipts|eth_estimateGas|eth_getBalance|eth_sendRawTransaction|eth_getProof|eth_getLogs" - - simulation: rpc-auth - name: "Rpc Auth tests" - run_command: make run-hive SIMULATION=ethereum/rpc-compat TEST_PATTERN="/engine-auth" - - simulation: discv4 - name: "Devp2p discv4 tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="discv4" - - simulation: snap - name: "Devp2p snap tests" - run_command: make run-hive SIMULATION=devp2p TEST_PATTERN="/AccountRange|StorageRanges" - - simulation: engine - name: "Engine tests" - run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="/Blob Transactions On Block 1, Cancun Genesis|Blob Transactions On Block 1, Shanghai Genesis|Blob Transaction Ordering, Single Account, Single Blob|Blob Transaction Ordering, Single Account, Dual Blob|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|ForkchoiceUpdated Version on Payload Request" - - simulation: engine-cancun - name: "Cancun Engine tests" - run_command: make run-hive SIMULATION=ethereum/engine TEST_PATTERN="cancun/Unique Payload ID|ParentHash equals BlockHash on NewPayload|Re-Execute Payload|Payload Build after New Invalid Payload|RPC|Build Payload with Invalid ChainID|Invalid PayloadAttributes, Zero timestamp, Syncing=False|Invalid PayloadAttributes, Parent timestamp, Syncing=False|Invalid PayloadAttributes, Missing BeaconRoot, Syncing=False|Suggested Fee Recipient Test|PrevRandao Opcode Transactions Test|Invalid Missing Ancestor ReOrg, StateRoot" - steps: - - name: Checkout sources - uses: actions/checkout@v3 - - - name: Rustup toolchain install - uses: dtolnay/rust-toolchain@stable - with: - toolchain: ${{ env.RUST_VERSION }} - - - name: Setup Go - uses: actions/setup-go@v3 - - - name: Set up Docker Buildx - uses: docker/setup-buildx-action@v3 - - - name: Download Hive - run: make setup-hive - - - name: Run Hive Simulation - run: ${{ matrix.run_command }} From 2be03ee64477cf50be1f1b2e7e7c1784b61d9655 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 10:15:59 -0300 Subject: [PATCH 104/345] Progress --- crates/networking/p2p/snap.rs | 11 +++++++++ crates/storage/trie/error.rs | 2 ++ crates/storage/trie/proof.rs | 44 +++++++++++++++++++++++++++++++++++ crates/storage/trie/trie.rs | 35 +++++++++++++++++----------- 4 files changed, 78 insertions(+), 14 deletions(-) create mode 100644 crates/storage/trie/proof.rs diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index fefd7374b..2eebffda6 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -10,6 +10,8 @@ use crate::rlpx::{ }, }; +// Request Processing + pub fn process_account_range_request( request: GetAccountRange, store: Store, @@ -153,6 +155,15 @@ pub fn process_trie_nodes_request( }) } +// Response Processing + +fn validate_account_range_response(request: &GetAccountRange, response: &AccountRange) -> Result<(), RLPxError> { + // Reconstruct a partial trie from the response and verify it + let (keys, accounts)= response.accounts.iter().map(|unit| (unit.hash, unit.account)).unzip(); + Ok(()) + // verify_range(origin, keys, accounts, nodes) +} + #[cfg(test)] mod tests { use std::str::FromStr; diff --git a/crates/storage/trie/error.rs b/crates/storage/trie/error.rs index f7cdf60e9..fdefd2c04 100644 --- a/crates/storage/trie/error.rs +++ b/crates/storage/trie/error.rs @@ -7,4 +7,6 @@ pub enum TrieError { LibmdbxError(anyhow::Error), #[error(transparent)] RLPDecode(#[from] RLPDecodeError), + #[error("Verification Error: {0}")] + Verify(String) } diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs new file mode 100644 index 000000000..0896ebc27 --- /dev/null +++ b/crates/storage/trie/proof.rs @@ -0,0 +1,44 @@ +use ethereum_types::H256; + +use crate::{PathRLP, Trie, TrieError, ValueRLP}; + +/// The boolead indicates if there is more state to be fetched +fn verify_range_proof(root: H256, first_key: H256, keys: Vec, values: Vec, proof: Vec>) -> Result { + if keys.len() != values.len() { + return Err(TrieError::Verify(format!("inconsistent proof data, got {} keys and {} values", keys.len(), values.len()))); + } + // Check that the key range is monotonically increasing + for keys in keys.windows(2) { + if keys[0] >= keys[1] { + return Err(TrieError::Verify(String::from("key range is not monotonically increasing"))); + } + } + // Check for empty values + if values.iter().find(|value| value.is_empty()).is_some() { + return Err(TrieError::Verify(String::from("value range contains empty value"))); + } + + // Verify ranges depending on the given proof + + // Case A) No proofs given, the range is expected to be the full set of leaves + if proof.is_empty() { + let mut trie = Trie::stateless(); + for (index, key) in keys.iter().enumerate() { + // Ignore the error as we don't rely on a DB + let _ = trie.insert(key.0.to_vec(), values[index].clone()); + } + let hash = trie.hash().unwrap_or_default(); + if hash != root { + return Err(TrieError::Verify(format!("invalid proof, expected root hash {}, got {}", root, hash))); + } + return Ok(false) + } + + // Case B) One edge proof no range given, there are no more values in the trie + if keys.is_empty() { + + } + + + Ok(true) +} \ No newline at end of file diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index da498f798..79887b445 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -7,6 +7,7 @@ mod node_hash; mod rlp; mod state; mod trie_iter; +mod proof; #[cfg(test)] mod test_utils; @@ -162,20 +163,7 @@ impl Trie { pub fn compute_hash_from_unsorted_iter( iter: impl Iterator, ) -> H256 { - // We will only be using the trie's cache so we don't need a working DB - struct NullTrieDB; - - impl TrieDB for NullTrieDB { - fn get(&self, _key: Vec) -> Result>, TrieError> { - Ok(None) - } - - fn put(&self, _key: Vec, _value: Vec) -> Result<(), TrieError> { - Ok(()) - } - } - - let mut trie = Trie::new(Box::new(NullTrieDB)); + let mut trie = Trie::stateless(); for (path, value) in iter { // Unwraping here won't panic as our in_memory trie DB won't fail trie.insert(path, value).unwrap(); @@ -262,6 +250,25 @@ impl Trie { } } + /// Creates a new stateless trie. This trie won't be able to store any nodes so all data will be lost after calculating the hash + /// Only use it for proof verification or computing a hash from an iterator + pub(crate) fn stateless() -> Trie { + // We will only be using the trie's cache so we don't need a working DB + struct NullTrieDB; + + impl TrieDB for NullTrieDB { + fn get(&self, _key: Vec) -> Result>, TrieError> { + Ok(None) + } + + fn put(&self, _key: Vec, _value: Vec) -> Result<(), TrieError> { + Ok(()) + } + } + + Trie::new(Box::new(NullTrieDB)) + } + #[cfg(all(test, feature = "libmdbx"))] /// Creates a new Trie based on a temporary Libmdbx DB fn new_temp() -> Self { From 09f688b67a6982fe7547182a8fdcf795e0f85b12 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 10:20:18 -0300 Subject: [PATCH 105/345] Clippy --- crates/storage/store/engines/api.rs | 2 +- crates/storage/store/engines/in_memory.rs | 2 +- crates/storage/store/engines/libmdbx.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/store/engines/api.rs b/crates/storage/store/engines/api.rs index 48a1e4ff0..ce1c898b9 100644 --- a/crates/storage/store/engines/api.rs +++ b/crates/storage/store/engines/api.rs @@ -2,7 +2,7 @@ use bytes::Bytes; use ethereum_rust_core::types::{ Block, BlockBody, BlockHash, BlockHeader, BlockNumber, ChainConfig, Index, Receipt, Transaction, }; -use ethereum_types::{Address, H256, U256}; +use ethereum_types::{H256, U256}; use std::{fmt::Debug, panic::RefUnwindSafe}; use crate::error::StoreError; diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index a4a57600b..afaff9cfa 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -4,7 +4,7 @@ use ethereum_rust_core::types::{ Block, BlockBody, BlockHash, BlockHeader, BlockNumber, ChainConfig, Index, Receipt, }; use ethereum_rust_trie::{InMemoryTrieDB, Trie}; -use ethereum_types::{Address, H256, U256}; +use ethereum_types::{H256, U256}; use std::{ collections::HashMap, fmt::Debug, diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 20c58f8af..5b4bfa817 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -12,7 +12,7 @@ use ethereum_rust_core::types::{ use ethereum_rust_rlp::decode::RLPDecode; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_trie::{LibmdbxDupsortTrieDB, LibmdbxTrieDB, Trie}; -use ethereum_types::{Address, H256, U256}; +use ethereum_types::{H256, U256}; use libmdbx::orm::{Decodable, Encodable, Table}; use libmdbx::{ dupsort, From 824afbc579f5ce6650c95d6804c6ff8c279c7b6b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 12:29:16 -0300 Subject: [PATCH 106/345] progress --- crates/common/rlp/decode.rs | 2 +- crates/storage/trie/node.rs | 2 +- crates/storage/trie/node/branch.rs | 2 +- crates/storage/trie/node/extension.rs | 2 +- crates/storage/trie/node/leaf.rs | 2 +- crates/storage/trie/node_hash.rs | 57 ++++++++++++++++++++++++++- 6 files changed, 61 insertions(+), 6 deletions(-) diff --git a/crates/common/rlp/decode.rs b/crates/common/rlp/decode.rs index b5061c3e6..0760b2ee0 100644 --- a/crates/common/rlp/decode.rs +++ b/crates/common/rlp/decode.rs @@ -472,7 +472,7 @@ pub fn get_rlp_bytes_item_payload(rlp: &[u8]) -> &[u8] { /// It returns a 2-element tuple with the following elements: /// - The payload of the item. /// - The remaining bytes after the item. -fn decode_bytes(data: &[u8]) -> Result<(&[u8], &[u8]), RLPDecodeError> { +pub fn decode_bytes(data: &[u8]) -> Result<(&[u8], &[u8]), RLPDecodeError> { let (is_list, payload, rest) = decode_rlp_item(data)?; if is_list { return Err(RLPDecodeError::UnexpectedList); diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 0c6468478..db4f1d254 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -11,7 +11,7 @@ use crate::error::TrieError; use super::{nibble::NibbleSlice, node_hash::NodeHash, state::TrieState, ValueRLP}; /// A Node in an Ethereum Compatible Patricia Merkle Trie -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub enum Node { Branch(BranchNode), Extension(ExtensionNode), diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index a5279bc2e..1d0bdb5b3 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -10,7 +10,7 @@ use super::{ExtensionNode, LeafNode, Node}; /// Branch Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's hash, value, path, and the hash of its children nodes -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct BranchNode { // TODO: check if switching to hashmap is a better solution pub choices: Box<[NodeHash; 16]>, diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 4e0c0d0e4..f9440b132 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -9,7 +9,7 @@ use super::{BranchNode, LeafNode, Node}; /// Extension Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's prefix and a its child node hash, doesn't store any value -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct ExtensionNode { pub prefix: NibbleVec, pub child: NodeHash, diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index b6684e61d..cccb4685c 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -10,7 +10,7 @@ use crate::{ use super::{ExtensionNode, Node}; /// Leaf Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's hash, value & path -#[derive(Debug, Clone, Default)] +#[derive(Debug, Clone, Default, PartialEq)] pub struct LeafNode { pub path: PathRLP, pub value: ValueRLP, diff --git a/crates/storage/trie/node_hash.rs b/crates/storage/trie/node_hash.rs index 868b3479b..ea80adef1 100644 --- a/crates/storage/trie/node_hash.rs +++ b/crates/storage/trie/node_hash.rs @@ -1,9 +1,12 @@ -use ethereum_rust_rlp::{decode::RLPDecode, encode::RLPEncode}; +use ethereum_rust_rlp::{decode::{self, RLPDecode}, encode::RLPEncode, error::RLPDecodeError}; use ethereum_types::H256; +use hex::decode; #[cfg(feature = "libmdbx")] use libmdbx::orm::{Decodable, Encodable}; use sha3::{Digest, Keccak256}; +use crate::node::{LeafNode, Node}; + use super::nibble::{NibbleSlice, NibbleVec}; /// Struct representing a trie node hash @@ -259,3 +262,55 @@ impl NodeEncoder { self.encoded } } + +pub fn decode_raw(encoded_node: Vec) -> Result { + // Decode List + if let Ok((true, list, &[])) = decode::decode_rlp_item(&encoded_node) { + // Decode inner values + // Try to decode as leaf/extension + if let (false, path_slice, rest) = decode::decode_rlp_item(&list)? { + // Check the path slice + match path_slice.first() { + // Extension + Some(0x00) => todo!(), + // Leaf Node + Some(_) => { + let (value, _) = decode::decode_bytes(rest)?; + Ok(LeafNode { + path: path_slice[1..].to_vec(), + value: value.to_vec(), + }.into()) + }, + // Branch? + _ => todo!(), + } + } else { + todo!() + } + } else { + todo!() + } + } + +#[cfg(test)] +mod tests { + use crate::node::LeafNode; + + use super::*; + + #[test] + fn decode_raw_leaf() { + let leaf_node = LeafNode::new([6;32].to_vec(), [7;32].to_vec()); + let encoded_leaf_node = leaf_node.encode_raw(0); + let decoded_leaf_node = decode_raw(encoded_leaf_node).unwrap(); + assert_eq!(Into::::into(leaf_node), decoded_leaf_node); + } + + #[test] + fn decode_raw_leaf_with_offset() { + let leaf_node = LeafNode::new([6;32].to_vec(), [7;32].to_vec()); + let encoded_leaf_node = leaf_node.encode_raw(7); + let decoded_leaf_node = decode_raw(encoded_leaf_node).unwrap(); + assert_eq!(Into::::into(leaf_node), decoded_leaf_node); + } +} From 19c0bf09ff264fd61a1a725135eb46bbe3ed0e9c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 12:54:10 -0300 Subject: [PATCH 107/345] Remove path field from BranchNode --- crates/storage/trie/node/branch.rs | 27 +++++++++------------------ crates/storage/trie/node/extension.rs | 2 +- crates/storage/trie/node/leaf.rs | 4 ++-- crates/storage/trie/rlp.rs | 11 +---------- crates/storage/trie/test_utils.rs | 2 +- crates/storage/trie/trie_iter.rs | 3 ++- 6 files changed, 16 insertions(+), 33 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index a5279bc2e..3fb91b6ae 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -3,7 +3,7 @@ use crate::{ nibble::{Nibble, NibbleSlice, NibbleVec}, node_hash::{NodeEncoder, NodeHash}, state::TrieState, - PathRLP, ValueRLP, + ValueRLP, }; use super::{ExtensionNode, LeafNode, Node}; @@ -14,7 +14,6 @@ use super::{ExtensionNode, LeafNode, Node}; pub struct BranchNode { // TODO: check if switching to hashmap is a better solution pub choices: Box<[NodeHash; 16]>, - pub path: PathRLP, pub value: ValueRLP, } @@ -43,23 +42,17 @@ impl BranchNode { pub fn new(choices: Box<[NodeHash; 16]>) -> Self { Self { choices, - path: Default::default(), value: Default::default(), } } /// Creates a new branch node given its children and stores the given (path, value) pair - pub fn new_with_value(choices: Box<[NodeHash; 16]>, path: PathRLP, value: ValueRLP) -> Self { - Self { - choices, - path, - value, - } + pub fn new_with_value(choices: Box<[NodeHash; 16]>, value: ValueRLP) -> Self { + Self { choices, value } } /// Updates the node's path and value - pub fn update(&mut self, new_path: PathRLP, new_value: ValueRLP) { - self.path = new_path; + pub fn update(&mut self, new_value: ValueRLP) { self.value = new_value; } @@ -117,7 +110,7 @@ impl BranchNode { }, None => { // Insert into self - self.update(path.data(), value); + self.update(value); } }; @@ -176,9 +169,8 @@ impl BranchNode { } None => { // Remove own value (if it has one) and return it - if !self.path.is_empty() { + if !self.value.is_empty() { let value = self.value; - self.path = Default::default(); self.value = Default::default(); (!value.is_empty()).then_some(value) @@ -227,7 +219,7 @@ impl BranchNode { } // Replace self with the child extension node, updating its path in the process Node::Extension(mut extension_node) => { - debug_assert!(self.path.is_empty()); // Sanity check + debug_assert!(self.value.is_empty()); // Sanity check extension_node.prefix.prepend(choice_index); // Return node here so we don't have to update it in the state and then fetch it return Ok((Some(extension_node.into()), value)); @@ -240,11 +232,11 @@ impl BranchNode { _ => None, }; - let new_node = match (child_hash, !self.path.is_empty()) { + let new_node = match (child_hash, !self.value.is_empty()) { // If this node still has a child and value return the updated node (Some(_), true) => Some(self.into()), // If this node still has a value but no longer has children, convert it into a leaf node - (None, true) => Some(LeafNode::new(self.path, self.value).into()), + (None, true) => Some(LeafNode::new(path.data(), self.value).into()), // If this node doesn't have a value, replace it with its child node (Some(x), false) => Some( state @@ -488,7 +480,6 @@ mod test { }; assert_eq!(new_node.choices, node.choices); - assert_eq!(new_node.path, path.data()); assert_eq!(new_node.value, value); } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 4e0c0d0e4..33903d014 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -98,7 +98,7 @@ impl ExtensionNode { choices[c as usize] = new_leaf.insert_self(child_offset, state)?; BranchNode::new(Box::new(choices)) } else { - BranchNode::new_with_value(Box::new(choices), path.data(), value) + BranchNode::new_with_value(Box::new(choices), value) }; // Create a final node, then return it: diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index b6684e61d..c3c495c46 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -69,7 +69,7 @@ impl LeafNode { .nth(absolute_offset) .unwrap() as usize] = self.clone().insert_self(leaf_offset, state)?; - BranchNode::new_with_value(Box::new(choices), path.data(), value) + BranchNode::new_with_value(Box::new(choices), value) } else if absolute_offset == 2 * self.path.len() { // Create a new leaf node and store the path and value in it // Create a new branch node with the leaf as a child and store self's path and value @@ -79,7 +79,7 @@ impl LeafNode { choices[path_branch.next().unwrap() as usize] = new_leaf.insert_self(leaf_offset, state)?; - BranchNode::new_with_value(Box::new(choices), self.path, self.value) + BranchNode::new_with_value(Box::new(choices), self.value) } else { // Create a new leaf node and store the path and value in it // Create a new branch node with the leaf and self as children diff --git a/crates/storage/trie/rlp.rs b/crates/storage/trie/rlp.rs index a19a70d3c..da121526b 100644 --- a/crates/storage/trie/rlp.rs +++ b/crates/storage/trie/rlp.rs @@ -31,7 +31,6 @@ impl RLPEncode for BranchNode { // TODO: choices encoded as vec due to conflicting trait impls for [T;N] & [u8;N], check if we can fix this later Encoder::new(buf) .encode_field(&self.choices.to_vec()) - .encode_field(&self.path) .encode_field(&self.value) .finish() } @@ -64,16 +63,8 @@ impl RLPDecode for BranchNode { let choices = choices .try_into() .map_err(|_| RLPDecodeError::Custom(CHOICES_LEN_ERROR_MSG.to_string()))?; - let (path, decoder) = decoder.decode_field("path")?; let (value, decoder) = decoder.decode_field("value")?; - Ok(( - Self { - choices, - path, - value, - }, - decoder.finish()?, - )) + Ok((Self { choices, value }, decoder.finish()?)) } } diff --git a/crates/storage/trie/test_utils.rs b/crates/storage/trie/test_utils.rs index ab0a211fb..26459ebad 100644 --- a/crates/storage/trie/test_utils.rs +++ b/crates/storage/trie/test_utils.rs @@ -71,7 +71,7 @@ macro_rules! pmt_node { }).insert_self(1, &mut $trie.state).unwrap(); )* Box::new(choices) - }, $path, $value) + }, $value) }}; ( diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index e60d548dd..f8d6c2ef0 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -46,10 +46,11 @@ impl Iterator for TrieIterator { } impl TrieIterator { + // TODO: construct path from nibbles pub fn content(self) -> impl Iterator { self.filter_map(|n| match n { Node::Branch(branch_node) => { - (!branch_node.path.is_empty()).then_some((branch_node.path, branch_node.value)) + (!branch_node.value.is_empty()).then_some((vec![], branch_node.value)) } Node::Extension(_) => None, Node::Leaf(leaf_node) => Some((leaf_node.path, leaf_node.value)), From d32bb4e0f105c1f225bd0599dabe2cf6c52b64f5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 14:47:13 -0300 Subject: [PATCH 108/345] Start dumb nibbles impl --- crates/storage/trie/dumb_nibbles.rs | 81 +++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 crates/storage/trie/dumb_nibbles.rs diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs new file mode 100644 index 000000000..4221e486a --- /dev/null +++ b/crates/storage/trie/dumb_nibbles.rs @@ -0,0 +1,81 @@ +pub struct DumbNibbles { + data: Vec, +} + +impl DumbNibbles { + pub fn from_hex(hex: Vec) -> Self { + Self { data: hex } + } + + pub fn from_bytes(bytes: Vec) -> Self { + Self { + data: bytes + .iter() + .map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) + .flatten() + .collect(), + } + } + + pub fn len(&self) -> usize { + self.data.len() + } + + pub fn is_empty(&self) -> bool { + self.data.is_empty() + } + + /// If `prefix` is a prefix of self, move the offset after + /// the prefix and return true, otherwise return false. + pub fn skip_prefix(&mut self, prefix: DumbNibbles) -> bool { + if self.len() >= prefix.len() && &self.data[..prefix.len()] == prefix.as_ref() { + self.data = self.data[prefix.len()..].to_vec(); + return true; + } else { + false + } + } +} + +impl AsRef<[u8]> for DumbNibbles { + fn as_ref(&self) -> &[u8] { + &self.data + } +} + +#[cfg(test)] +mod test { + use super::*; + + #[test] + fn skip_prefix_true() { + let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![1, 2, 3]); + assert!(a.skip_prefix(b)); + assert_eq!(a.as_ref(), &[4, 5]) + } + + #[test] + fn skip_prefix_true_same_length() { + let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + assert!(a.skip_prefix(b)); + assert!(a.is_empty()); + } + + #[test] + fn skip_prefix_longer_prefix() { + let mut a = DumbNibbles::from_hex(vec![1, 2, 3]); + let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + assert!(!a.skip_prefix(b)); + assert_eq!(a.as_ref(), &[1, 2, 3]) + } + + #[test] + fn skip_prefix_false() { + let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![1, 2, 4]); + assert!(!a.skip_prefix(b)); + assert_eq!(a.as_ref(), &[1, 2, 3, 4, 5]) + } +} From 7cad9b6dad488443b7fc6cfff3d22a3977c67640 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 14:58:34 -0300 Subject: [PATCH 109/345] Impl common_prefix --- crates/storage/trie/dumb_nibbles.rs | 35 ++++++++++++++++++++++++++--- crates/storage/trie/trie.rs | 1 + 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 4221e486a..9de800555 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -11,8 +11,7 @@ impl DumbNibbles { Self { data: bytes .iter() - .map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) - .flatten() + .flat_map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) .collect(), } } @@ -30,11 +29,20 @@ impl DumbNibbles { pub fn skip_prefix(&mut self, prefix: DumbNibbles) -> bool { if self.len() >= prefix.len() && &self.data[..prefix.len()] == prefix.as_ref() { self.data = self.data[prefix.len()..].to_vec(); - return true; + true } else { false } } + + /// Count how many nibbles two DumbNibbles have in common + pub fn common_prefix(&self, other: &DumbNibbles) -> usize { + self.as_ref() + .iter() + .zip(other.as_ref().iter()) + .take_while(|(a, b)| a == b) + .count() + } } impl AsRef<[u8]> for DumbNibbles { @@ -78,4 +86,25 @@ mod test { assert!(!a.skip_prefix(b)); assert_eq!(a.as_ref(), &[1, 2, 3, 4, 5]) } + + #[test] + fn common_prefix_all() { + let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + assert_eq!(a.common_prefix(&b), a.len()); + } + + #[test] + fn common_prefix_partial() { + let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![1, 2, 3]); + assert_eq!(a.common_prefix(&b), b.len()); + } + + #[test] + fn common_prefix_none() { + let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = DumbNibbles::from_hex(vec![2, 3, 4, 5, 6]); + assert_eq!(a.common_prefix(&b), 0); + } } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 45b098f03..d35928746 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -7,6 +7,7 @@ mod rlp; mod state; mod trie_iter; +mod dumb_nibbles; #[cfg(test)] mod test_utils; From 3b69cd7cbb40ef9cb7bc160ed179afd46b695494 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 15:00:01 -0300 Subject: [PATCH 110/345] Update name & doc --- crates/storage/trie/dumb_nibbles.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 9de800555..8b9fe4fcc 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -35,8 +35,8 @@ impl DumbNibbles { } } - /// Count how many nibbles two DumbNibbles have in common - pub fn common_prefix(&self, other: &DumbNibbles) -> usize { + /// Compares self to another and returns the shared nibble count (amount of nibbles that are equal, from the start) + pub fn count_prefix(&self, other: &DumbNibbles) -> usize { self.as_ref() .iter() .zip(other.as_ref().iter()) @@ -88,23 +88,23 @@ mod test { } #[test] - fn common_prefix_all() { + fn count_prefix_all() { let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - assert_eq!(a.common_prefix(&b), a.len()); + assert_eq!(a.count_prefix(&b), a.len()); } #[test] - fn common_prefix_partial() { + fn count_prefix_partial() { let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![1, 2, 3]); - assert_eq!(a.common_prefix(&b), b.len()); + assert_eq!(a.count_prefix(&b), b.len()); } #[test] - fn common_prefix_none() { + fn count_prefix_none() { let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![2, 3, 4, 5, 6]); - assert_eq!(a.common_prefix(&b), 0); + assert_eq!(a.count_prefix(&b), 0); } } From 16d14325652393cb2c4dd2743fbe2828ee1626b1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 15:02:25 -0300 Subject: [PATCH 111/345] Impl next --- crates/storage/trie/dumb_nibbles.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 8b9fe4fcc..b9c01cbbe 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -43,6 +43,11 @@ impl DumbNibbles { .take_while(|(a, b)| a == b) .count() } + + /// Removes and returns the first nibble + pub fn next(&mut self) -> Option { + (!self.is_empty()).then_some(self.data.remove(0)) + } } impl AsRef<[u8]> for DumbNibbles { From d7c036f866d6945f5ff7b5641174b373cd5dcc42 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 15:56:10 -0300 Subject: [PATCH 112/345] Use DumbNibbles: LeafNode get & insert --- crates/storage/trie/dumb_nibbles.rs | 17 +++++ crates/storage/trie/node/extension.rs | 5 +- crates/storage/trie/node/leaf.rs | 91 +++++++++++---------------- 3 files changed, 55 insertions(+), 58 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index b9c01cbbe..468c2ec50 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -1,3 +1,4 @@ +#[derive(Debug, Clone, Default, PartialEq)] pub struct DumbNibbles { data: Vec, } @@ -48,6 +49,22 @@ impl DumbNibbles { pub fn next(&mut self) -> Option { (!self.is_empty()).then_some(self.data.remove(0)) } + + pub fn offset(&self, offset: usize) -> DumbNibbles { + self.slice(offset, self.len()) + } + + pub fn slice(&self, start: usize, end: usize) -> DumbNibbles { + DumbNibbles::from_hex(self.data[start..end].to_vec()) + } + + pub fn extend(&mut self, other: &DumbNibbles) { + self.data.extend_from_slice(other.as_ref()); + } + + pub fn at(&self, i: usize) -> usize { + self.data[i] as usize + } } impl AsRef<[u8]> for DumbNibbles { diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 33903d014..9d5f8e600 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,3 +1,4 @@ +use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; use crate::nibble::NibbleSlice; use crate::nibble::NibbleVec; @@ -11,13 +12,13 @@ use super::{BranchNode, LeafNode, Node}; /// Contains the node's prefix and a its child node hash, doesn't store any value #[derive(Debug, Clone)] pub struct ExtensionNode { - pub prefix: NibbleVec, + pub prefix: DumbNibbles, pub child: NodeHash, } impl ExtensionNode { /// Creates a new extension node given its child hash and prefix - pub(crate) fn new(prefix: NibbleVec, child: NodeHash) -> Self { + pub(crate) fn new(prefix: DumbNibbles, child: NodeHash) -> Self { Self { prefix, child } } diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index c3c495c46..95b3161cf 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,4 +1,5 @@ use crate::{ + dumb_nibbles::DumbNibbles, error::TrieError, nibble::NibbleSlice, node::BranchNode, @@ -12,19 +13,19 @@ use super::{ExtensionNode, Node}; /// Contains the node's hash, value & path #[derive(Debug, Clone, Default)] pub struct LeafNode { - pub path: PathRLP, + pub partial: DumbNibbles, pub value: ValueRLP, } impl LeafNode { /// Creates a new leaf node and stores the given (path, value) pair - pub fn new(path: PathRLP, value: ValueRLP) -> Self { - Self { path, value } + pub fn new(partial: DumbNibbles, value: ValueRLP) -> Self { + Self { partial, value } } /// Returns the stored value if the given path matches the stored path - pub fn get(&self, path: NibbleSlice) -> Result, TrieError> { - if path.cmp_rest(&self.path) { + pub fn get(&self, path: DumbNibbles) -> Result, TrieError> { + if self.partial == path { Ok(Some(self.value.clone())) } else { Ok(None) @@ -35,72 +36,53 @@ impl LeafNode { pub fn insert( mut self, state: &mut TrieState, - path: NibbleSlice, + path: DumbNibbles, value: ValueRLP, ) -> Result { /* Possible flow paths: - Leaf { SelfPath, SelfValue } -> Leaf { SelfPath, Value } - Leaf { SelfPath, SelfValue } -> Extension { Branch { [Self,...] Path, Value } } - Leaf { SelfPath, SelfValue } -> Extension { Branch { [ Leaf { Path, Value } , ... ], SelfPath, SelfValue} } - Leaf { SelfPath, SelfValue } -> Branch { [ Leaf { Path, Value }, Self, ... ], None, None} + Leaf { SelfValue } -> Leaf { Value } + Leaf { SelfValue } -> Extension { Branch { [Self,...] Value } } + Leaf { SelfValue } -> Extension { Branch { [ Leaf { Value } , ... ], SelfValue} } + Leaf { SelfValue } -> Branch { [ Leaf { Value }, Self, ... ], None, None} */ // If the path matches the stored path, update the value and return self - if path.cmp_rest(&self.path) { + if self.partial == path { self.value = value; Ok(self.into()) } else { - let offset = path.count_prefix_slice(&{ - let mut value_path = NibbleSlice::new(&self.path); - value_path.offset_add(path.offset()); - value_path - }); - - let mut path_branch = path.clone(); - path_branch.offset_add(offset); - - let absolute_offset = path_branch.offset(); - // The offset that will be used when computing the hash of newly created leaf nodes - let leaf_offset = absolute_offset + 1; - let branch_node = if absolute_offset == 2 * path.as_ref().len() { + let match_index = path.count_prefix(&self.partial); + let branch_node = if self.partial.at(match_index) == 16 { + // Create a new leaf node and store the value in it + // Create a new branch node with the leaf as a child and store self's value + // Branch { [ Leaf { Value } , ... ], SelfValue} + let new_leaf = LeafNode::new(path.offset(match_index + 1), value); + let mut choices = BranchNode::EMPTY_CHOICES; + choices[path.at(match_index)] = new_leaf.insert_self(state)?; + BranchNode::new_with_value(Box::new(choices), self.value) + } else if path.at(match_index) == 16 { // Create a branch node with self as a child and store the value in the branch node - // Branch { [Self,...] Path, Value } + // Branch { [Self,...], Value } let mut choices = BranchNode::EMPTY_CHOICES; - choices[NibbleSlice::new(self.path.as_ref()) - .nth(absolute_offset) - .unwrap() as usize] = self.clone().insert_self(leaf_offset, state)?; - + choices[self.partial.at(match_index)] = self.clone().insert_self(state)?; BranchNode::new_with_value(Box::new(choices), value) - } else if absolute_offset == 2 * self.path.len() { - // Create a new leaf node and store the path and value in it - // Create a new branch node with the leaf as a child and store self's path and value - // Branch { [ Leaf { Path, Value } , ... ], SelfPath, SelfValue} - let new_leaf = LeafNode::new(path.data(), value); - let mut choices = BranchNode::EMPTY_CHOICES; - choices[path_branch.next().unwrap() as usize] = - new_leaf.insert_self(leaf_offset, state)?; - - BranchNode::new_with_value(Box::new(choices), self.value) } else { // Create a new leaf node and store the path and value in it // Create a new branch node with the leaf and self as children // Branch { [ Leaf { Path, Value }, Self, ... ], None, None} - let new_leaf = LeafNode::new(path.data(), value); - let child_hash = new_leaf.insert_self(leaf_offset, state)?; + let new_leaf = LeafNode::new(path.offset(match_index + 1), value); let mut choices = BranchNode::EMPTY_CHOICES; - choices[NibbleSlice::new(self.path.as_ref()) - .nth(absolute_offset) - .unwrap() as usize] = self.clone().insert_self(leaf_offset, state)?; - choices[path_branch.next().unwrap() as usize] = child_hash; + choices[path.at(match_index)] = new_leaf.insert_self(state)?; + choices[self.partial.at(match_index)] = self.clone().insert_self(state)?; BranchNode::new(Box::new(choices)) }; - let final_node = if offset != 0 { + let final_node = if match_index == 0 { + branch_node.into() + } else { // Create an extension node with the branch node as child // Extension { BranchNode } - let branch_hash = branch_node.insert_self(state)?; - ExtensionNode::new(path.split_to_vec(offset), branch_hash).into() - } else { - branch_node.into() + ExtensionNode::new(path.slice(0, match_index), branch_node.insert_self(state)?) + .into() }; Ok(final_node) @@ -144,12 +126,9 @@ impl LeafNode { /// Inserts the node into the state and returns its hash /// Receives the offset that needs to be traversed to reach the leaf node from the canonical root, used to compute the node hash - pub fn insert_self( - self, - path_offset: usize, - state: &mut TrieState, - ) -> Result { - let hash = self.compute_hash(path_offset); + pub fn insert_self(self, state: &mut TrieState) -> Result { + // TODO: Fix + let hash = self.compute_hash(0); state.insert_node(self.into(), hash.clone()); Ok(hash) } From a2afdfd10fe07502abcc6a5521a2c39c277c9d2d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 16:48:39 -0300 Subject: [PATCH 113/345] Use DumbNibbles: ExtensionNode get & insert --- crates/storage/trie/dumb_nibbles.rs | 10 ++-- crates/storage/trie/node.rs | 9 ++- crates/storage/trie/node/extension.rs | 83 ++++++++++----------------- crates/storage/trie/node/leaf.rs | 9 +-- 4 files changed, 45 insertions(+), 66 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 468c2ec50..053a705d0 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -27,7 +27,7 @@ impl DumbNibbles { /// If `prefix` is a prefix of self, move the offset after /// the prefix and return true, otherwise return false. - pub fn skip_prefix(&mut self, prefix: DumbNibbles) -> bool { + pub fn skip_prefix(&mut self, prefix: &DumbNibbles) -> bool { if self.len() >= prefix.len() && &self.data[..prefix.len()] == prefix.as_ref() { self.data = self.data[prefix.len()..].to_vec(); true @@ -81,7 +81,7 @@ mod test { fn skip_prefix_true() { let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![1, 2, 3]); - assert!(a.skip_prefix(b)); + assert!(a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[4, 5]) } @@ -89,7 +89,7 @@ mod test { fn skip_prefix_true_same_length() { let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - assert!(a.skip_prefix(b)); + assert!(a.skip_prefix(&b)); assert!(a.is_empty()); } @@ -97,7 +97,7 @@ mod test { fn skip_prefix_longer_prefix() { let mut a = DumbNibbles::from_hex(vec![1, 2, 3]); let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - assert!(!a.skip_prefix(b)); + assert!(!a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[1, 2, 3]) } @@ -105,7 +105,7 @@ mod test { fn skip_prefix_false() { let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); let b = DumbNibbles::from_hex(vec![1, 2, 4]); - assert!(!a.skip_prefix(b)); + assert!(!a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[1, 2, 3, 4, 5]) } diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index b19a0111b..3bd2673f4 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -6,7 +6,7 @@ pub use branch::BranchNode; pub use extension::ExtensionNode; pub use leaf::LeafNode; -use crate::error::TrieError; +use crate::{dumb_nibbles::DumbNibbles, error::TrieError}; use super::{nibble::NibbleSlice, node_hash::NodeHash, state::TrieState, ValueRLP}; @@ -38,7 +38,7 @@ impl From for Node { impl Node { /// Retrieves a value from the subtrie originating from this node given its path - pub fn get(&self, state: &TrieState, path: NibbleSlice) -> Result, TrieError> { + pub fn get(&self, state: &TrieState, path: DumbNibbles) -> Result, TrieError> { match self { Node::Branch(n) => n.get(state, path), Node::Extension(n) => n.get(state, path), @@ -50,7 +50,7 @@ impl Node { pub fn insert( self, state: &mut TrieState, - path: NibbleSlice, + path: DumbNibbles, value: ValueRLP, ) -> Result { match self { @@ -92,13 +92,12 @@ impl Node { pub fn insert_self( self, - path_offset: usize, state: &mut TrieState, ) -> Result { match self { Node::Branch(n) => n.insert_self(state), Node::Extension(n) => n.insert_self(state), - Node::Leaf(n) => n.insert_self(path_offset, state), + Node::Leaf(n) => n.insert_self(state), } } } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 9d5f8e600..b5ff31f1b 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,7 +1,6 @@ use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; use crate::nibble::NibbleSlice; -use crate::nibble::NibbleVec; use crate::node_hash::{NodeEncoder, NodeHash, PathKind}; use crate::state::TrieState; use crate::ValueRLP; @@ -26,7 +25,7 @@ impl ExtensionNode { pub fn get( &self, state: &TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, ) -> Result, TrieError> { // If the path is prefixed by this node's prefix, delegate to its child. // Otherwise, no value is present. @@ -42,12 +41,14 @@ impl ExtensionNode { } /// Inserts a value into the subtrie originating from this node and returns the new root of the subtrie + /// TODO: Code changed a lot, check and rewrite doc pub fn insert( mut self, state: &mut TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, value: ValueRLP, ) -> Result { + // OUTDATED /* Possible flow paths (there are duplicates between different prefix lengths): Extension { prefix, child } -> Extension { prefix , child' } (insert into child) Extension { prefixL+C+prefixR, child } -> Extension { prefixL, Branch { [ Extension { prefixR, child }, ..], Path, Value} } (if path fully traversed) @@ -57,61 +58,39 @@ impl ExtensionNode { Extension { None+C+prefixR } -> Branch { [ Extension { prefixR, child } , ..], Path, Value} (if path fully traversed) Extension { None+C+prefixR } -> Branch { [ Extension { prefixR, child } , Leaf { Path, Value } , ... ], None, None} (if path not fully traversed) */ - - if path.skip_prefix(&self.prefix) { - // Insert into child node - let child_node = state - .get_node(self.child)? - .expect("inconsistent internal tree structure"); - let child_node = child_node.insert(state, path.clone(), value.clone())?; - // Child node will never be a leaf, so the path_offset is not used - self.child = child_node.insert_self(0, state)?; - + let match_index = path.count_prefix(&self.prefix); + if match_index == self.prefix.len() { + // Insert into child node + let child_node = state + .get_node(self.child)? + .expect("inconsistent internal tree structure"); + let new_child_node = child_node.insert(state, path.offset(match_index), value.clone())?; + self.child = new_child_node.insert_self(state)?; Ok(self.into()) - } else { - let offset = path.clone().count_prefix_vec(&self.prefix); - path.offset_add(offset); - // Offset used when computing the hash of the new child - let child_offset = path.offset() + 1; - // Split prefix into left_prefix and right_prefix - let (left_prefix, choice, right_prefix) = self.prefix.split_extract_at(offset); - - let left_prefix = (!left_prefix.is_empty()).then_some(left_prefix); - let right_prefix = (!right_prefix.is_empty()).then_some(right_prefix); - - // Create right prefix node: - // If there is a right prefix the node will be Extension { prefixR, Self.child } - // If there is no right prefix the node will be Self.child - let right_prefix_node = if let Some(right_prefix) = right_prefix { - let extension_node = ExtensionNode::new(right_prefix, self.child); - extension_node.insert_self(state)? - } else { + } else if match_index == 0 { + let new_node = if self.prefix.len() == 1 { self.child + } else { + ExtensionNode::new(self.prefix.offset(1), self.child).insert_self(state)? }; - - // Create a branch node: - // If the path hasn't been traversed: Branch { [ RPrefixNode, Leaf { Path, Value }, ... ], None, None } - // If the path has been fully traversed: Branch { [ RPrefixNode, ... ], Path, Value } let mut choices = BranchNode::EMPTY_CHOICES; - choices[choice as usize] = right_prefix_node; - let branch_node = if let Some(c) = path.next() { - let new_leaf = LeafNode::new(path.data(), value); - choices[c as usize] = new_leaf.insert_self(child_offset, state)?; - BranchNode::new(Box::new(choices)) + let branch_node = if self.prefix.at(0) == 16 { + match state.get_node(new_node)? { + Node::Leaf(leaf) => BranchNode::new_with_value(choices, leaf.value), + _ => panic!("inconsistent internal tree structure") + } } else { - BranchNode::new_with_value(Box::new(choices), value) + choices[self.prefix.at(0)] = new_node; + BranchNode::new(choices) }; - - // Create a final node, then return it: - // If there is a left_prefix: Extension { left_prefix , Branch } - // If there is no left_prefix: Branch - match left_prefix { - Some(left_prefix) => { - let branch_hash = branch_node.insert_self(state)?; - Ok(ExtensionNode::new(left_prefix, branch_hash).into()) - } - None => Ok(branch_node.into()), - } + return branch_node.insert(state, path, value); + } else { + let new_extension = ExtensionNode::new(self.prefix.offset(match_index), self.child); + let new_node = new_extension.insert(state, path.offset(match_index), value)?; + self.prefix = self.prefix.slice(0, match_index); + self.child = new_node.insert_self(state)?; + Ok(self.into()) + } } } diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 95b3161cf..d44724578 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -5,7 +5,7 @@ use crate::{ node::BranchNode, node_hash::{NodeEncoder, NodeHash, PathKind}, state::TrieState, - PathRLP, ValueRLP, + ValueRLP, }; use super::{ExtensionNode, Node}; @@ -90,8 +90,8 @@ impl LeafNode { } /// Removes own value if the path matches own path and returns self and the value if it was removed - pub fn remove(self, path: NibbleSlice) -> Result<(Option, Option), TrieError> { - Ok(if path.cmp_rest(&self.path) { + pub fn remove(self, path: DumbNibbles) -> Result<(Option, Option), TrieError> { + Ok(if self.partial == path { (None, Some(self.value)) } else { (Some(self.into()), None) @@ -104,9 +104,10 @@ impl LeafNode { } /// Encodes the node given the offset in the path traversed before reaching this node + /// TODO: Fix pub fn encode_raw(&self, offset: usize) -> Vec { let encoded_value = &self.value; - let encoded_path = &self.path; + let encoded_path = &vec![]; let mut path = NibbleSlice::new(encoded_path); path.offset_add(offset); From 3a65df95a0b043e276e0c8f232210b2bb2558ed3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 17:38:45 -0300 Subject: [PATCH 114/345] Make code compile --- crates/storage/trie/dumb_nibbles.rs | 28 +++- crates/storage/trie/node.rs | 11 +- crates/storage/trie/node/branch.rs | 180 +++++++++++++------------- crates/storage/trie/node/extension.rs | 153 +++++++++++++--------- crates/storage/trie/node/leaf.rs | 68 ++++++---- crates/storage/trie/rlp.rs | 6 +- crates/storage/trie/test_utils.rs | 18 +-- crates/storage/trie/trie.rs | 24 ++-- crates/storage/trie/trie_iter.rs | 2 +- 9 files changed, 276 insertions(+), 214 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 053a705d0..554aeed0d 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -1,3 +1,10 @@ +use ethereum_rust_rlp::{ + decode::RLPDecode, + encode::RLPEncode, + error::RLPDecodeError, + structs::{Decoder, Encoder}, +}; + #[derive(Debug, Clone, Default, PartialEq)] pub struct DumbNibbles { data: Vec, @@ -8,7 +15,7 @@ impl DumbNibbles { Self { data: hex } } - pub fn from_bytes(bytes: Vec) -> Self { + pub fn from_bytes(bytes: &[u8]) -> Self { Self { data: bytes .iter() @@ -65,6 +72,11 @@ impl DumbNibbles { pub fn at(&self, i: usize) -> usize { self.data[i] as usize } + + /// Inserts a nibble at the start + pub fn prepend(&mut self, nibble: u8) { + self.data.insert(0, nibble); + } } impl AsRef<[u8]> for DumbNibbles { @@ -73,6 +85,20 @@ impl AsRef<[u8]> for DumbNibbles { } } +impl RLPEncode for DumbNibbles { + fn encode(&self, buf: &mut dyn bytes::BufMut) { + Encoder::new(buf).encode_field(&self.data); + } +} + +impl RLPDecode for DumbNibbles { + fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { + let decoder = Decoder::new(rlp)?; + let (data, decoder) = decoder.decode_field("data")?; + Ok((Self { data }, decoder.finish()?)) + } +} + #[cfg(test)] mod test { use super::*; diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 3bd2673f4..2a07729d1 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -8,7 +8,7 @@ pub use leaf::LeafNode; use crate::{dumb_nibbles::DumbNibbles, error::TrieError}; -use super::{nibble::NibbleSlice, node_hash::NodeHash, state::TrieState, ValueRLP}; +use super::{node_hash::NodeHash, state::TrieState, ValueRLP}; /// A Node in an Ethereum Compatible Patricia Merkle Trie #[derive(Debug, Clone)] @@ -65,7 +65,7 @@ impl Node { pub fn remove( self, state: &mut TrieState, - path: NibbleSlice, + path: DumbNibbles, ) -> Result<(Option, Option), TrieError> { match self { Node::Branch(n) => n.remove(state, path), @@ -80,7 +80,7 @@ impl Node { pub fn get_path( &self, state: &TrieState, - path: NibbleSlice, + path: DumbNibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { match self { @@ -90,10 +90,7 @@ impl Node { } } - pub fn insert_self( - self, - state: &mut TrieState, - ) -> Result { + pub fn insert_self(self, state: &mut TrieState) -> Result { match self { Node::Branch(n) => n.insert_self(state), Node::Extension(n) => n.insert_self(state), diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 3fb91b6ae..6990b9006 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -1,6 +1,6 @@ use crate::{ + dumb_nibbles::DumbNibbles, error::TrieError, - nibble::{Nibble, NibbleSlice, NibbleVec}, node_hash::{NodeEncoder, NodeHash}, state::TrieState, ValueRLP, @@ -60,7 +60,7 @@ impl BranchNode { pub fn get( &self, state: &TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, ) -> Result, TrieError> { // If path is at the end, return to its own value if present. // Otherwise, check the corresponding choice and delegate accordingly if present. @@ -85,7 +85,7 @@ impl BranchNode { pub fn insert( mut self, state: &mut TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, value: ValueRLP, ) -> Result { // If path is at the end, insert or replace its own value. @@ -94,8 +94,8 @@ impl BranchNode { Some(choice) => match &mut self.choices[choice as usize] { // Create new child (leaf node) choice_hash if !choice_hash.is_valid() => { - let new_leaf = LeafNode::new(path.data(), value); - let child_hash = new_leaf.insert_self(path.offset(), state)?; + let new_leaf = LeafNode::new(path.offset(1), value); + let child_hash = new_leaf.insert_self(state)?; *choice_hash = child_hash; } // Insert into existing child and then update it @@ -104,8 +104,8 @@ impl BranchNode { .get_node(choice_hash.clone())? .expect("inconsistent internal tree structure"); - let child_node = child_node.insert(state, path.clone(), value)?; - *choice_hash = child_node.insert_self(path.offset(), state)?; + let child_node = child_node.insert(state, path.offset(1), value)?; + *choice_hash = child_node.insert_self(state)?; } }, None => { @@ -122,7 +122,7 @@ impl BranchNode { pub fn remove( mut self, state: &mut TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, ) -> Result<(Option, Option), TrieError> { /* Possible flow paths: Step 1: Removal @@ -143,8 +143,6 @@ impl BranchNode { */ // Step 1: Remove value - - let path_offset = path.offset(); // Check if the value is located in a child subtrie let value = match path.next() { Some(choice_index) => { @@ -156,8 +154,7 @@ impl BranchNode { let (child_node, old_value) = child_node.remove(state, path.clone())?; if let Some(child_node) = child_node { // Update child node - self.choices[choice_index as usize] = - child_node.insert_self(path.offset(), state)?; + self.choices[choice_index as usize] = child_node.insert_self(state)?; } else { // Remove child hash if the child subtrie was removed in the process self.choices[choice_index as usize] = NodeHash::default(); @@ -202,7 +199,6 @@ impl BranchNode { let child_hash = match choice_count { Ok(Some((choice_index, child_hash))) => { - let choice_index = Nibble::try_from(choice_index as u8).unwrap(); let child_node = state .get_node(child_hash.clone())? .expect("inconsistent internal tree structure"); @@ -212,7 +208,7 @@ impl BranchNode { // The extension node will then replace self if self has no value Node::Branch(_) => { let extension_node = ExtensionNode::new( - NibbleVec::from_single(choice_index, path_offset % 2 != 0), + DumbNibbles::from_hex(vec![choice_index as u8]), child_hash.clone(), ); *child_hash = extension_node.insert_self(state)? @@ -220,7 +216,7 @@ impl BranchNode { // Replace self with the child extension node, updating its path in the process Node::Extension(mut extension_node) => { debug_assert!(self.value.is_empty()); // Sanity check - extension_node.prefix.prepend(choice_index); + extension_node.prefix.prepend(choice_index as u8); // Return node here so we don't have to update it in the state and then fetch it return Ok((Some(extension_node.into()), value)); } @@ -236,7 +232,7 @@ impl BranchNode { // If this node still has a child and value return the updated node (Some(_), true) => Some(self.into()), // If this node still has a value but no longer has children, convert it into a leaf node - (None, true) => Some(LeafNode::new(path.data(), self.value).into()), + (None, true) => Some(LeafNode::new(DumbNibbles::from_hex(vec![16]), self.value).into()), // If this node doesn't have a value, replace it with its child node (Some(x), false) => Some( state @@ -312,7 +308,7 @@ impl BranchNode { pub fn get_path( &self, state: &TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { // Add self to node_path (if not inlined in parent) @@ -382,17 +378,19 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x00])).unwrap(), + node.get(&trie.state, DumbNibbles::from_bytes(&[0x00])) + .unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x10])).unwrap(), + node.get(&trie.state, DumbNibbles::from_bytes(&[0x10])) + .unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } @@ -402,13 +400,14 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x20])).unwrap(), + node.get(&trie.state, DumbNibbles::from_bytes(&[0x20])) + .unwrap(), None, ); } @@ -418,11 +417,11 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; - let path = NibbleSlice::new(&[0x2]); + let path = DumbNibbles::from_bytes(&[0x2]); let value = vec![0x3]; let node = node @@ -438,12 +437,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; - let path = NibbleSlice::new(&[0x20]); + let path = DumbNibbles::from_bytes(&[0x20]); let value = vec![0x21]; let node = node @@ -459,14 +458,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; // The extension node is ignored since it's irrelevant in this test. - let mut path = NibbleSlice::new(&[0x00]); - path.offset_add(2); + let path = DumbNibbles::from_bytes(&[0x00]).offset(2); let value = vec![0x1]; let new_node = node @@ -488,13 +486,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - 1 => leaf { vec![0x10] => vec![0x10] }, + 0 => leaf { &[0x00] => vec![0x00] }, + 1 => leaf { &[0x10] => vec![0x10] }, } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x00])) + .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -506,14 +504,14 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - 1 => leaf { vec![0x10] => vec![0x10] }, - 2 => leaf { vec![0x10] => vec![0x10] }, + 0 => leaf { &[0x00] => vec![0x00] }, + 1 => leaf { &[0x10] => vec![0x10] }, + 2 => leaf { &[0x10] => vec![0x10] }, } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x00])) + .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Branch(_)))); @@ -525,12 +523,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - } with_leaf { vec![0x01] => vec![0xFF] } + 0 => leaf { &[0x00] => vec![0x00] }, + } with_leaf { &[0x01] => vec![0xFF] } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x00])) + .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -542,11 +540,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - } with_leaf { vec![0x1] => vec![0xFF] } + 0 => leaf { &[0x00] => vec![0x00] }, + } with_leaf { &[0x1] => vec![0xFF] } }; - let (node, value) = node.remove(&mut trie.state, NibbleSlice::new(&[])).unwrap(); + let (node, value) = node + .remove(&mut trie.state, DumbNibbles::from_bytes(&[])) + .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); assert_eq!(value, Some(vec![0xFF])); @@ -557,12 +557,14 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - 1 => leaf { vec![0x10] => vec![0x10] }, - } with_leaf { vec![0x1] => vec![0xFF] } + 0 => leaf { &[0x00] => vec![0x00] }, + 1 => leaf { &[0x10] => vec![0x10] }, + } with_leaf { &[0x1] => vec![0xFF] } }; - let (node, value) = node.remove(&mut trie.state, NibbleSlice::new(&[])).unwrap(); + let (node, value) = node + .remove(&mut trie.state, DumbNibbles::from_bytes(&[])) + .unwrap(); assert!(matches!(node, Some(Node::Branch(_)))); assert_eq!(value, Some(vec![0xFF])); @@ -573,8 +575,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 2 => leaf { vec![0x20] => vec![0x20] }, - 4 => leaf { vec![0x40] => vec![0x40] }, + 2 => leaf { &[0x20] => vec![0x20] }, + 4 => leaf { &[0x40] => vec![0x40] }, } }; @@ -592,22 +594,22 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0x0 => leaf { vec![0x00] => vec![0x00] }, - 0x1 => leaf { vec![0x10] => vec![0x10] }, - 0x2 => leaf { vec![0x20] => vec![0x20] }, - 0x3 => leaf { vec![0x30] => vec![0x30] }, - 0x4 => leaf { vec![0x40] => vec![0x40] }, - 0x5 => leaf { vec![0x50] => vec![0x50] }, - 0x6 => leaf { vec![0x60] => vec![0x60] }, - 0x7 => leaf { vec![0x70] => vec![0x70] }, - 0x8 => leaf { vec![0x80] => vec![0x80] }, - 0x9 => leaf { vec![0x90] => vec![0x90] }, - 0xA => leaf { vec![0xA0] => vec![0xA0] }, - 0xB => leaf { vec![0xB0] => vec![0xB0] }, - 0xC => leaf { vec![0xC0] => vec![0xC0] }, - 0xD => leaf { vec![0xD0] => vec![0xD0] }, - 0xE => leaf { vec![0xE0] => vec![0xE0] }, - 0xF => leaf { vec![0xF0] => vec![0xF0] }, + 0x0 => leaf { &[0x00] => vec![0x00] }, + 0x1 => leaf { &[0x10] => vec![0x10] }, + 0x2 => leaf { &[0x20] => vec![0x20] }, + 0x3 => leaf { &[0x30] => vec![0x30] }, + 0x4 => leaf { &[0x40] => vec![0x40] }, + 0x5 => leaf { &[0x50] => vec![0x50] }, + 0x6 => leaf { &[0x60] => vec![0x60] }, + 0x7 => leaf { &[0x70] => vec![0x70] }, + 0x8 => leaf { &[0x80] => vec![0x80] }, + 0x9 => leaf { &[0x90] => vec![0x90] }, + 0xA => leaf { &[0xA0] => vec![0xA0] }, + 0xB => leaf { &[0xB0] => vec![0xB0] }, + 0xC => leaf { &[0xC0] => vec![0xC0] }, + 0xD => leaf { &[0xD0] => vec![0xD0] }, + 0xE => leaf { &[0xE0] => vec![0xE0] }, + 0xF => leaf { &[0xF0] => vec![0xF0] }, } }; @@ -626,9 +628,9 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 2 => leaf { vec![0x20] => vec![0x20] }, - 4 => leaf { vec![0x40] => vec![0x40] }, - } with_leaf { vec![0x1] => vec![0x1] } + 2 => leaf { &[0x20] => vec![0x20] }, + 4 => leaf { &[0x40] => vec![0x40] }, + } with_leaf { &[0x1] => vec![0x1] } }; assert_eq!( @@ -645,23 +647,23 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0x0 => leaf { vec![0x00] => vec![0x00] }, - 0x1 => leaf { vec![0x10] => vec![0x10] }, - 0x2 => leaf { vec![0x20] => vec![0x20] }, - 0x3 => leaf { vec![0x30] => vec![0x30] }, - 0x4 => leaf { vec![0x40] => vec![0x40] }, - 0x5 => leaf { vec![0x50] => vec![0x50] }, - 0x6 => leaf { vec![0x60] => vec![0x60] }, - 0x7 => leaf { vec![0x70] => vec![0x70] }, - 0x8 => leaf { vec![0x80] => vec![0x80] }, - 0x9 => leaf { vec![0x90] => vec![0x90] }, - 0xA => leaf { vec![0xA0] => vec![0xA0] }, - 0xB => leaf { vec![0xB0] => vec![0xB0] }, - 0xC => leaf { vec![0xC0] => vec![0xC0] }, - 0xD => leaf { vec![0xD0] => vec![0xD0] }, - 0xE => leaf { vec![0xE0] => vec![0xE0] }, - 0xF => leaf { vec![0xF0] => vec![0xF0] }, - } with_leaf { vec![0x1] => vec![0x1] } + 0x0 => leaf { &[0x00] => vec![0x00] }, + 0x1 => leaf { &[0x10] => vec![0x10] }, + 0x2 => leaf { &[0x20] => vec![0x20] }, + 0x3 => leaf { &[0x30] => vec![0x30] }, + 0x4 => leaf { &[0x40] => vec![0x40] }, + 0x5 => leaf { &[0x50] => vec![0x50] }, + 0x6 => leaf { &[0x60] => vec![0x60] }, + 0x7 => leaf { &[0x70] => vec![0x70] }, + 0x8 => leaf { &[0x80] => vec![0x80] }, + 0x9 => leaf { &[0x90] => vec![0x90] }, + 0xA => leaf { &[0xA0] => vec![0xA0] }, + 0xB => leaf { &[0xB0] => vec![0xB0] }, + 0xC => leaf { &[0xC0] => vec![0xC0] }, + 0xD => leaf { &[0xD0] => vec![0xD0] }, + 0xE => leaf { &[0xE0] => vec![0xE0] }, + 0xF => leaf { &[0xF0] => vec![0xF0] }, + } with_leaf { &[0x1] => vec![0x1] } }; assert_eq!( diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index b5ff31f1b..51105542b 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,11 +1,11 @@ use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; -use crate::nibble::NibbleSlice; +use crate::nibble::NibbleVec; use crate::node_hash::{NodeEncoder, NodeHash, PathKind}; use crate::state::TrieState; use crate::ValueRLP; -use super::{BranchNode, LeafNode, Node}; +use super::{BranchNode, Node}; /// Extension Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's prefix and a its child node hash, doesn't store any value @@ -45,7 +45,7 @@ impl ExtensionNode { pub fn insert( mut self, state: &mut TrieState, - mut path: DumbNibbles, + path: DumbNibbles, value: ValueRLP, ) -> Result { // OUTDATED @@ -60,11 +60,12 @@ impl ExtensionNode { */ let match_index = path.count_prefix(&self.prefix); if match_index == self.prefix.len() { - // Insert into child node - let child_node = state - .get_node(self.child)? - .expect("inconsistent internal tree structure"); - let new_child_node = child_node.insert(state, path.offset(match_index), value.clone())?; + // Insert into child node + let child_node = state + .get_node(self.child)? + .expect("inconsistent internal tree structure"); + let new_child_node = + child_node.insert(state, path.offset(match_index), value.clone())?; self.child = new_child_node.insert_self(state)?; Ok(self.into()) } else if match_index == 0 { @@ -76,12 +77,14 @@ impl ExtensionNode { let mut choices = BranchNode::EMPTY_CHOICES; let branch_node = if self.prefix.at(0) == 16 { match state.get_node(new_node)? { - Node::Leaf(leaf) => BranchNode::new_with_value(choices, leaf.value), - _ => panic!("inconsistent internal tree structure") + Some(Node::Leaf(leaf)) => { + BranchNode::new_with_value(Box::new(choices), leaf.value) + } + _ => panic!("inconsistent internal tree structure"), } } else { choices[self.prefix.at(0)] = new_node; - BranchNode::new(choices) + BranchNode::new(Box::new(choices)) }; return branch_node.insert(state, path, value); } else { @@ -91,13 +94,12 @@ impl ExtensionNode { self.child = new_node.insert_self(state)?; Ok(self.into()) } - } } pub fn remove( mut self, state: &mut TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, ) -> Result<(Option, Option), TrieError> { /* Possible flow paths: Extension { prefix, child } -> Extension { prefix, child } (no removal) @@ -113,7 +115,8 @@ impl ExtensionNode { .get_node(self.child)? .expect("inconsistent internal tree structure"); // Remove value from child subtrie - let (child_node, old_value) = child_node.remove(state, path.clone())?; + let (child_node, old_value) = + child_node.remove(state, path.offset(self.prefix.len()))?; // Restructure node based on removal let node = match child_node { // If there is no subtrie remove the node @@ -157,7 +160,7 @@ impl ExtensionNode { let mut encoder = NodeEncoder::new(); encoder.write_list_header(prefix_len + child_len); - encoder.write_path_vec(&self.prefix, PathKind::Extension); + encoder.write_path_vec(&NibbleVec::default(), PathKind::Extension); match child_hash { NodeHash::Inline(x) => encoder.write_raw(x), NodeHash::Hashed(x) => encoder.write_bytes(&x.0), @@ -178,7 +181,7 @@ impl ExtensionNode { pub fn get_path( &self, state: &TrieState, - mut path: NibbleSlice, + mut path: DumbNibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { // Add self to node_path (if not inlined in parent) @@ -200,14 +203,11 @@ impl ExtensionNode { #[cfg(test)] mod test { use super::*; - use crate::{ - pmt_node, - {nibble::Nibble, Trie}, - }; + use crate::{node::LeafNode, pmt_node, Trie}; #[test] fn new() { - let node = ExtensionNode::new(NibbleVec::new(), Default::default()); + let node = ExtensionNode::new(DumbNibbles::default(), Default::default()); assert_eq!(node.prefix.len(), 0); assert_eq!(node.child, Default::default()); @@ -218,17 +218,19 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x00])).unwrap(), + node.get(&trie.state, DumbNibbles::from_hex(vec![0x00])) + .unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x01])).unwrap(), + node.get(&trie.state, DumbNibbles::from_hex(vec![0x01])) + .unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } @@ -238,13 +240,14 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x02])).unwrap(), + node.get(&trie.state, DumbNibbles::from_hex(vec![0x02])) + .unwrap(), None, ); } @@ -254,19 +257,19 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, NibbleSlice::new(&[0x02]), vec![]) + .insert(&mut trie.state, DumbNibbles::from_hex(vec![0x02]), vec![]) .unwrap(); let node = match node { Node::Extension(x) => x, _ => panic!("expected an extension node"), }; - assert!(node.prefix.iter().eq([Nibble::V0].into_iter())); + assert_eq!(node.prefix.as_ref(), &[0]); } #[test] @@ -274,20 +277,25 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, NibbleSlice::new(&[0x10]), vec![0x20]) + .insert( + &mut trie.state, + DumbNibbles::from_hex(vec![0x10]), + vec![0x20], + ) .unwrap(); let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x10])).unwrap(), + node.get(&trie.state, DumbNibbles::from_hex(vec![0x10])) + .unwrap(), Some(vec![0x20]) ); } @@ -297,20 +305,25 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { vec![0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, NibbleSlice::new(&[0x10]), vec![0x20]) + .insert( + &mut trie.state, + DumbNibbles::from_hex(vec![0x10]), + vec![0x20], + ) .unwrap(); let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, NibbleSlice::new(&[0x10])).unwrap(), + node.get(&trie.state, DumbNibbles::from_hex(vec![0x10])) + .unwrap(), Some(vec![0x20]) ); } @@ -320,12 +333,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { vec![0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; - let path = NibbleSlice::new(&[0x01]); + let path = DumbNibbles::from_hex(vec![0x01]); let value = vec![0x02]; let node = node @@ -341,12 +354,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { vec![0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { vec![0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; - let path = NibbleSlice::new(&[0x01]); + let path = DumbNibbles::from_hex(vec![0x01]); let value = vec![0x04]; let node = node @@ -362,13 +375,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - 1 => leaf { vec![0x01] => vec![0x01] }, + 0 => leaf { &[0x00] => vec![0x00] }, + 1 => leaf { &[0x01] => vec![0x01] }, } } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x02])) + .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x02])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -380,13 +393,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x00] }, - 1 => leaf { vec![0x01] => vec![0x01] }, + 0 => leaf { &[0x00] => vec![0x00] }, + 1 => leaf { &[0x01] => vec![0x01] }, } } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x01])) + .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x01])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -398,16 +411,16 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { vec![0x00] => vec![0x00] }, + 0 => leaf { &[0x00] => vec![0x00] }, 1 => extension { [0], branch { - 0 => leaf { vec![0x01, 0x00] => vec![0x01, 0x00] }, - 1 => leaf { vec![0x01, 0x01] => vec![0x01, 0x01] }, + 0 => leaf { &[0x01, 0x00] => vec![0x01, 0x00] }, + 1 => leaf { &[0x01, 0x01] => vec![0x01, 0x01] }, } }, } } }; let (node, value) = node - .remove(&mut trie.state, NibbleSlice::new(&[0x00])) + .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x00])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -425,14 +438,20 @@ mod test { } } */ - let leaf_node_a = LeafNode::new(vec![0x00, 0x00], vec![0x12, 0x34]); - let leaf_node_b = LeafNode::new(vec![0x00, 0x10], vec![0x56, 0x78]); + let leaf_node_a = LeafNode::new( + DumbNibbles::from_hex(vec![0x00, 0x00]).offset(3), + vec![0x12, 0x34], + ); + let leaf_node_b = LeafNode::new( + DumbNibbles::from_hex(vec![0x00, 0x10]).offset(3), + vec![0x56, 0x78], + ); let mut choices = BranchNode::EMPTY_CHOICES; - choices[0] = leaf_node_a.compute_hash(3); - choices[1] = leaf_node_b.compute_hash(3); + choices[0] = leaf_node_a.compute_hash(0); + choices[1] = leaf_node_b.compute_hash(0); let branch_node = BranchNode::new(Box::new(choices)); let node = ExtensionNode::new( - NibbleVec::from_nibbles([Nibble::V0, Nibble::V0].into_iter(), false), + DumbNibbles::from_hex(vec![0, 0]), branch_node.compute_hash(), ); @@ -457,14 +476,20 @@ mod test { } } */ - let leaf_node_a = LeafNode::new(vec![0x00, 0x00], vec![0x12, 0x34, 0x56, 0x78, 0x9A]); - let leaf_node_b = LeafNode::new(vec![0x00, 0x10], vec![0x34, 0x56, 0x78, 0x9A, 0xBC]); + let leaf_node_a = LeafNode::new( + DumbNibbles::from_bytes(&[0x00, 0x00]), + vec![0x12, 0x34, 0x56, 0x78, 0x9A], + ); + let leaf_node_b = LeafNode::new( + DumbNibbles::from_bytes(&[0x00, 0x10]), + vec![0x34, 0x56, 0x78, 0x9A, 0xBC], + ); let mut choices = BranchNode::EMPTY_CHOICES; choices[0] = leaf_node_a.compute_hash(3); choices[1] = leaf_node_b.compute_hash(3); let branch_node = BranchNode::new(Box::new(choices)); let node = ExtensionNode::new( - NibbleVec::from_nibbles([Nibble::V0, Nibble::V0].into_iter(), false), + DumbNibbles::from_hex(vec![0, 0]), branch_node.compute_hash(), ); diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index d44724578..78f722df3 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -107,7 +107,7 @@ impl LeafNode { /// TODO: Fix pub fn encode_raw(&self, offset: usize) -> Vec { let encoded_value = &self.value; - let encoded_path = &vec![]; + let encoded_path = &[]; let mut path = NibbleSlice::new(encoded_path); path.offset_add(offset); @@ -137,10 +137,10 @@ impl LeafNode { /// Encodes the node and appends it to `node_path` if the encoded node is 32 or more bytes long pub fn get_path( &self, - path: NibbleSlice, + path: DumbNibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { - let encoded = self.encode_raw(path.offset()); + let encoded = self.encode_raw(0); if encoded.len() >= 32 { node_path.push(encoded); } @@ -156,18 +156,17 @@ mod test { #[test] fn new() { let node = LeafNode::new(Default::default(), Default::default()); - assert_eq!(node.path, PathRLP::default()); - assert_eq!(node.value, PathRLP::default()); + assert_eq!(node.value, ValueRLP::default()); } #[test] fn get_some() { let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; assert_eq!( - node.get(NibbleSlice::new(&[0x12])).unwrap(), + node.get(DumbNibbles::from_bytes(&[0x12])).unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); } @@ -175,28 +174,34 @@ mod test { #[test] fn get_none() { let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - assert!(node.get(NibbleSlice::new(&[0x34])).unwrap().is_none()); + assert!(node + .get(DumbNibbles::from_bytes(&[0x34])) + .unwrap() + .is_none()); } #[test] fn insert_replace() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; let node = node - .insert(&mut trie.state, NibbleSlice::new(&[0x12]), vec![0x13]) + .insert( + &mut trie.state, + DumbNibbles::from_bytes(&[0x12]), + vec![0x13], + ) .unwrap(); let node = match node { Node::Leaf(x) => x, _ => panic!("expected a leaf node"), }; - assert_eq!(node.path, vec![0x12]); assert_eq!(node.value, vec![0x13]); } @@ -204,9 +209,9 @@ mod test { fn insert_branch() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = NibbleSlice::new(&[0x22]); + let path = DumbNibbles::from_bytes(&[0x22]); let value = vec![0x23]; let node = node .insert(&mut trie.state, path.clone(), value.clone()) @@ -222,10 +227,10 @@ mod test { fn insert_extension_branch() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = NibbleSlice::new(&[0x13]); + let path = DumbNibbles::from_bytes(&[0x13]); let value = vec![0x15]; let node = node @@ -240,10 +245,10 @@ mod test { fn insert_extension_branch_value_self() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { vec![0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = NibbleSlice::new(&[0x12, 0x34]); + let path = DumbNibbles::from_bytes(&[0x12, 0x34]); let value = vec![0x17]; let node = node @@ -258,10 +263,10 @@ mod test { fn insert_extension_branch_value_other() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { vec![0x12, 0x34] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { &[0x12, 0x34] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = NibbleSlice::new(&[0x12]); + let path = DumbNibbles::from_bytes(&[0x12]); let value = vec![0x17]; let node = node @@ -282,8 +287,13 @@ mod test { #[test] fn remove_self() { - let node = LeafNode::new(vec![0x12, 0x34], vec![0x12, 0x34, 0x56, 0x78]); - let (node, value) = node.remove(NibbleSlice::new(&[0x12, 0x34])).unwrap(); + let node = LeafNode::new( + DumbNibbles::from_bytes(&[0x12, 0x34]), + vec![0x12, 0x34, 0x56, 0x78], + ); + let (node, value) = node + .remove(DumbNibbles::from_bytes(&[0x12, 0x34])) + .unwrap(); assert!(node.is_none()); assert_eq!(value, Some(vec![0x12, 0x34, 0x56, 0x78])); @@ -291,9 +301,12 @@ mod test { #[test] fn remove_none() { - let node = LeafNode::new(vec![0x12, 0x34], vec![0x12, 0x34, 0x56, 0x78]); + let node = LeafNode::new( + DumbNibbles::from_bytes(&[0x12, 0x34]), + vec![0x12, 0x34, 0x56, 0x78], + ); - let (node, value) = node.remove(NibbleSlice::new(&[0x12])).unwrap(); + let (node, value) = node.remove(DumbNibbles::from_bytes(&[0x12])).unwrap(); assert!(node.is_some()); assert_eq!(value, None); @@ -301,7 +314,7 @@ mod test { #[test] fn compute_hash() { - let node = LeafNode::new(b"key".to_vec(), b"value".to_vec()); + let node = LeafNode::new(DumbNibbles::from_bytes(b"key".as_ref()), b"value".to_vec()); let node_hash_ref = node.compute_hash(0); assert_eq!( node_hash_ref.as_ref(), @@ -311,7 +324,10 @@ mod test { #[test] fn compute_hash_long() { - let node = LeafNode::new(b"key".to_vec(), b"a comparatively long value".to_vec()); + let node = LeafNode::new( + DumbNibbles::from_bytes(b"key".as_ref()), + b"a comparatively long value".to_vec(), + ); let node_hash_ref = node.compute_hash(0); assert_eq!( diff --git a/crates/storage/trie/rlp.rs b/crates/storage/trie/rlp.rs index da121526b..13eff2282 100644 --- a/crates/storage/trie/rlp.rs +++ b/crates/storage/trie/rlp.rs @@ -48,7 +48,7 @@ impl RLPEncode for ExtensionNode { impl RLPEncode for LeafNode { fn encode(&self, buf: &mut dyn bytes::BufMut) { Encoder::new(buf) - .encode_field(&self.path) + .encode_field(&self.partial) .encode_field(&self.value) .finish() } @@ -80,9 +80,9 @@ impl RLPDecode for ExtensionNode { impl RLPDecode for LeafNode { fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { let decoder = Decoder::new(rlp)?; - let (path, decoder) = decoder.decode_field("path")?; + let (partial, decoder) = decoder.decode_field("partial")?; let (value, decoder) = decoder.decode_field("value")?; - Ok((Self { path, value }, decoder.finish()?)) + Ok((Self { partial, value }, decoder.finish()?)) } } diff --git a/crates/storage/trie/test_utils.rs b/crates/storage/trie/test_utils.rs index 26459ebad..5fb6b2191 100644 --- a/crates/storage/trie/test_utils.rs +++ b/crates/storage/trie/test_utils.rs @@ -48,7 +48,7 @@ macro_rules! pmt_node { $child_type { $( $child_tokens )* } offset offset }.into(); - choices[$choice as usize] = child_node.insert_self(1, &mut $trie.state).unwrap(); + choices[$choice as usize] = child_node.insert_self(&mut $trie.state).unwrap(); )* Box::new(choices) }) @@ -68,7 +68,7 @@ macro_rules! pmt_node { pmt_node! { @($trie) $child_type { $( $child_tokens )* } offset offset - }).insert_self(1, &mut $trie.state).unwrap(); + }).insert_self(&mut $trie.state).unwrap(); )* Box::new(choices) }, $value) @@ -80,23 +80,15 @@ macro_rules! pmt_node { $( offset $offset:expr )? ) => {{ #[allow(unused_variables)] - let offset = false $( ^ $offset )?; - let prefix = $crate::nibble::NibbleVec::from_nibbles( - $prefix - .into_iter() - .map(|x: u8| $crate::nibble::Nibble::try_from(x).unwrap()), - offset - ); + let prefix = $crate::dumb_nibbles::DumbNibbles::from_hex($prefix.to_vec()); - let offset = offset ^ (prefix.len() % 2 != 0); $crate::node::ExtensionNode::new( prefix.clone(), { let child_node = $crate::node::Node::from(pmt_node! { @($trie) $child_type { $( $child_tokens )* } - offset offset }); - child_node.insert_self(1, &mut $trie.state).unwrap() + child_node.insert_self(&mut $trie.state).unwrap() } ) }}; @@ -107,7 +99,7 @@ macro_rules! pmt_node { $( offset $offset:expr )? ) => { { - $crate::node::LeafNode::new($path, $value) + $crate::node::LeafNode::new(DumbNibbles::from_bytes($path), $value) } }; } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index d35928746..6c4922e92 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -11,6 +11,7 @@ mod dumb_nibbles; #[cfg(test)] mod test_utils; +use dumb_nibbles::DumbNibbles; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; use node::Node; @@ -23,7 +24,7 @@ pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; pub use self::error::TrieError; -use self::{nibble::NibbleSlice, node::LeafNode, state::TrieState, trie_iter::TrieIterator}; +use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator}; use lazy_static::lazy_static; @@ -76,7 +77,7 @@ impl Trie { .state .get_node(root.clone())? .expect("inconsistent internal tree structure"); - root_node.get(&self.state, NibbleSlice::new(path)) + root_node.get(&self.state, DumbNibbles::from_bytes(path)) } else { Ok(None) } @@ -91,13 +92,16 @@ impl Trie { .flatten() { // If the trie is not empty, call the root node's insertion logic - let root_node = - root_node.insert(&mut self.state, NibbleSlice::new(&path), value.clone())?; - self.root = Some(root_node.insert_self(0, &mut self.state)?) + let root_node = root_node.insert( + &mut self.state, + DumbNibbles::from_bytes(&path), + value.clone(), + )?; + self.root = Some(root_node.insert_self(&mut self.state)?) } else { // If the trie is empty, just add a leaf. - let new_leaf = Node::from(LeafNode::new(path.clone(), value)); - self.root = Some(new_leaf.insert_self(0, &mut self.state)?) + let new_leaf = Node::from(LeafNode::new(DumbNibbles::from_bytes(&path), value)); + self.root = Some(new_leaf.insert_self(&mut self.state)?) } Ok(()) } @@ -112,9 +116,9 @@ impl Trie { .get_node(root)? .expect("inconsistent internal tree structure"); let (root_node, old_value) = - root_node.remove(&mut self.state, NibbleSlice::new(&path))?; + root_node.remove(&mut self.state, DumbNibbles::from_bytes(&path))?; self.root = root_node - .map(|root| root.insert_self(0, &mut self.state)) + .map(|root| root.insert_self(&mut self.state)) .transpose()?; Ok(old_value) } else { @@ -150,7 +154,7 @@ impl Trie { node_path.push(node.to_vec()); } if let Some(root_node) = self.state.get_node(root.clone())? { - root_node.get_path(&self.state, NibbleSlice::new(path), &mut node_path)?; + root_node.get_path(&self.state, DumbNibbles::from_bytes(path), &mut node_path)?; } Ok(node_path) } diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index f8d6c2ef0..488fbbdd7 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -53,7 +53,7 @@ impl TrieIterator { (!branch_node.value.is_empty()).then_some((vec![], branch_node.value)) } Node::Extension(_) => None, - Node::Leaf(leaf_node) => Some((leaf_node.path, leaf_node.value)), + Node::Leaf(leaf_node) => Some((vec![], leaf_node.value)), }) } } From 43ae6808e401e4c7a96b8ff87581de112d929b7b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 18:06:34 -0300 Subject: [PATCH 115/345] Fix node encoding for LeafNode --- crates/storage/trie/dumb_nibbles.rs | 48 +++++++++++++++++++++++---- crates/storage/trie/node.rs | 2 +- crates/storage/trie/node/extension.rs | 8 ++--- crates/storage/trie/node/leaf.rs | 45 +++++++++---------------- crates/storage/trie/node_hash.rs | 28 +++------------- 5 files changed, 68 insertions(+), 63 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 554aeed0d..7de329614 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -16,12 +16,13 @@ impl DumbNibbles { } pub fn from_bytes(bytes: &[u8]) -> Self { - Self { - data: bytes - .iter() - .flat_map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) - .collect(), - } + let mut data: Vec = bytes + .iter() + .flat_map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) + .collect(); + data.push(16); + + Self { data } } pub fn len(&self) -> usize { @@ -77,6 +78,41 @@ impl DumbNibbles { pub fn prepend(&mut self, nibble: u8) { self.data.insert(0, nibble); } + + /// Taken from https://github.com/citahub/cita_trie/blob/master/src/nibbles.rs#L56 + pub fn encode_compact(&self) -> Vec { + let mut compact = vec![]; + let is_leaf = self.is_leaf(); + let mut hex = if is_leaf { + &self.data[0..self.data.len() - 1] + } else { + &self.data[0..] + }; + // node type path length | prefix hexchar + // -------------------------------------------------- + // extension even | 0000 0x0 + // extension odd | 0001 0x1 + // leaf even | 0010 0x2 + // leaf odd | 0011 0x3 + let v = if hex.len() % 2 == 1 { + let v = 0x10 + hex[0]; + hex = &hex[1..]; + v + } else { + 0x00 + }; + + compact.push(v + if is_leaf { 0x20 } else { 0x00 }); + for i in 0..(hex.len() / 2) { + compact.push((hex[i * 2] * 16) + (hex[i * 2 + 1])); + } + + compact + } + + pub fn is_leaf(&self) -> bool { + self.data[self.data.len() - 1] == 16 + } } impl AsRef<[u8]> for DumbNibbles { diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 2a07729d1..5d21d76ef 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -86,7 +86,7 @@ impl Node { match self { Node::Branch(n) => n.get_path(state, path, node_path), Node::Extension(n) => n.get_path(state, path, node_path), - Node::Leaf(n) => n.get_path(path, node_path), + Node::Leaf(n) => n.get_path(node_path), } } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 51105542b..14ce36571 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -447,8 +447,8 @@ mod test { vec![0x56, 0x78], ); let mut choices = BranchNode::EMPTY_CHOICES; - choices[0] = leaf_node_a.compute_hash(0); - choices[1] = leaf_node_b.compute_hash(0); + choices[0] = leaf_node_a.compute_hash(); + choices[1] = leaf_node_b.compute_hash(); let branch_node = BranchNode::new(Box::new(choices)); let node = ExtensionNode::new( DumbNibbles::from_hex(vec![0, 0]), @@ -485,8 +485,8 @@ mod test { vec![0x34, 0x56, 0x78, 0x9A, 0xBC], ); let mut choices = BranchNode::EMPTY_CHOICES; - choices[0] = leaf_node_a.compute_hash(3); - choices[1] = leaf_node_b.compute_hash(3); + choices[0] = leaf_node_a.compute_hash(); + choices[1] = leaf_node_b.compute_hash(); let branch_node = BranchNode::new(Box::new(choices)); let node = ExtensionNode::new( DumbNibbles::from_hex(vec![0, 0]), diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 78f722df3..032fdafe9 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,9 +1,8 @@ use crate::{ dumb_nibbles::DumbNibbles, error::TrieError, - nibble::NibbleSlice, node::BranchNode, - node_hash::{NodeEncoder, NodeHash, PathKind}, + node_hash::{NodeEncoder, NodeHash}, state::TrieState, ValueRLP, }; @@ -99,29 +98,23 @@ impl LeafNode { } /// Computes the node's hash given the offset in the path traversed before reaching this node - pub fn compute_hash(&self, offset: usize) -> NodeHash { - NodeHash::from_encoded_raw(self.encode_raw(offset)) + pub fn compute_hash(&self) -> NodeHash { + NodeHash::from_encoded_raw(self.encode_raw()) } /// Encodes the node given the offset in the path traversed before reaching this node /// TODO: Fix - pub fn encode_raw(&self, offset: usize) -> Vec { - let encoded_value = &self.value; - let encoded_path = &[]; - - let mut path = NibbleSlice::new(encoded_path); - path.offset_add(offset); - - let path_len = NodeEncoder::path_len(path.len()); + pub fn encode_raw(&self) -> Vec { + let path_len = NodeEncoder::path_len(self.partial.len()); let value_len = NodeEncoder::bytes_len( - encoded_value.len(), - encoded_value.first().copied().unwrap_or_default(), + self.value.len(), + self.value.first().copied().unwrap_or_default(), ); let mut encoder = crate::node_hash::NodeEncoder::new(); encoder.write_list_header(path_len + value_len); - encoder.write_path_slice(&path, PathKind::Leaf); - encoder.write_bytes(encoded_value); + encoder.write_path_slice(&self.partial); + encoder.write_bytes(&self.value); encoder.finalize() } @@ -129,18 +122,14 @@ impl LeafNode { /// Receives the offset that needs to be traversed to reach the leaf node from the canonical root, used to compute the node hash pub fn insert_self(self, state: &mut TrieState) -> Result { // TODO: Fix - let hash = self.compute_hash(0); + let hash = self.compute_hash(); state.insert_node(self.into(), hash.clone()); Ok(hash) } /// Encodes the node and appends it to `node_path` if the encoded node is 32 or more bytes long - pub fn get_path( - &self, - path: DumbNibbles, - node_path: &mut Vec>, - ) -> Result<(), TrieError> { - let encoded = self.encode_raw(0); + pub fn get_path(&self, node_path: &mut Vec>) -> Result<(), TrieError> { + let encoded = self.encode_raw(); if encoded.len() >= 32 { node_path.push(encoded); } @@ -291,9 +280,7 @@ mod test { DumbNibbles::from_bytes(&[0x12, 0x34]), vec![0x12, 0x34, 0x56, 0x78], ); - let (node, value) = node - .remove(DumbNibbles::from_bytes(&[0x12, 0x34])) - .unwrap(); + let (node, value) = node.remove(DumbNibbles::from_bytes(&[0x12, 0x34])).unwrap(); assert!(node.is_none()); assert_eq!(value, Some(vec![0x12, 0x34, 0x56, 0x78])); @@ -313,9 +300,9 @@ mod test { } #[test] - fn compute_hash() { + fn compute_hash_x() { let node = LeafNode::new(DumbNibbles::from_bytes(b"key".as_ref()), b"value".to_vec()); - let node_hash_ref = node.compute_hash(0); + let node_hash_ref = node.compute_hash(); assert_eq!( node_hash_ref.as_ref(), &[0xCB, 0x84, 0x20, 0x6B, 0x65, 0x79, 0x85, 0x76, 0x61, 0x6C, 0x75, 0x65], @@ -329,7 +316,7 @@ mod test { b"a comparatively long value".to_vec(), ); - let node_hash_ref = node.compute_hash(0); + let node_hash_ref = node.compute_hash(); assert_eq!( node_hash_ref.as_ref(), &[ diff --git a/crates/storage/trie/node_hash.rs b/crates/storage/trie/node_hash.rs index 868b3479b..81b28b5be 100644 --- a/crates/storage/trie/node_hash.rs +++ b/crates/storage/trie/node_hash.rs @@ -4,7 +4,9 @@ use ethereum_types::H256; use libmdbx::orm::{Decodable, Encodable}; use sha3::{Digest, Keccak256}; -use super::nibble::{NibbleSlice, NibbleVec}; +use crate::dumb_nibbles::DumbNibbles; + +use super::nibble::NibbleVec; /// Struct representing a trie node hash /// If the encoded node is less than 32 bits, contains the encoded node itself @@ -198,28 +200,8 @@ impl NodeEncoder { self.encoded.extend_from_slice(value); } - pub fn write_path_slice(&mut self, value: &NibbleSlice, kind: PathKind) { - let mut flag = kind.into_flag(); - - // TODO: Do not use iterators. - let nibble_count = value.clone().count(); - let nibble_iter = if nibble_count & 0x01 != 0 { - let mut iter = value.clone(); - flag |= 0x10; - flag |= iter.next().unwrap() as u8; - iter - } else { - value.clone() - }; - - let i2 = nibble_iter.clone().skip(1).step_by(2); - if nibble_count > 1 { - self.write_len(0x80, 0xB7, (nibble_count >> 1) + 1); - } - self.write_raw(&[flag]); - for (a, b) in nibble_iter.step_by(2).zip(i2) { - self.write_raw(&[((a as u8) << 4) | (b as u8)]); - } + pub fn write_path_slice(&mut self, value: &DumbNibbles) { + self.write_bytes(&value.encode_compact()); } pub fn write_path_vec(&mut self, value: &NibbleVec, kind: PathKind) { From e69e95442bf99faee5d59f6e52fad287b0dae3c1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 5 Nov 2024 18:46:19 -0300 Subject: [PATCH 116/345] Fix leaf nibbles --- crates/storage/trie/dumb_nibbles.rs | 10 ++++++++-- crates/storage/trie/node/extension.rs | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 7de329614..6aa599f61 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -16,11 +16,17 @@ impl DumbNibbles { } pub fn from_bytes(bytes: &[u8]) -> Self { + Self::from_raw(bytes, true) + } + + pub fn from_raw(bytes: &[u8], is_leaf: bool) -> Self { let mut data: Vec = bytes .iter() .flat_map(|byte| [(byte >> 4 & 0x0F), byte & 0x0F]) .collect(); - data.push(16); + if is_leaf { + data.push(16); + } Self { data } } @@ -123,7 +129,7 @@ impl AsRef<[u8]> for DumbNibbles { impl RLPEncode for DumbNibbles { fn encode(&self, buf: &mut dyn bytes::BufMut) { - Encoder::new(buf).encode_field(&self.data); + Encoder::new(buf).encode_field(&self.data).finish(); } } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 14ce36571..05eecc637 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -439,11 +439,11 @@ mod test { } */ let leaf_node_a = LeafNode::new( - DumbNibbles::from_hex(vec![0x00, 0x00]).offset(3), + DumbNibbles::from_bytes(&[0x00, 0x00]).offset(3), vec![0x12, 0x34], ); let leaf_node_b = LeafNode::new( - DumbNibbles::from_hex(vec![0x00, 0x10]).offset(3), + DumbNibbles::from_bytes(&[0x00, 0x10]).offset(3), vec![0x56, 0x78], ); let mut choices = BranchNode::EMPTY_CHOICES; From 998ed51c1bd63cc1eecb09dfed2d2e6cc33b5d2d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 11:47:18 -0300 Subject: [PATCH 117/345] Fix leaf partial when inserting to leaf --- crates/storage/trie/node/leaf.rs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 032fdafe9..4d12d33b2 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -50,19 +50,22 @@ impl LeafNode { Ok(self.into()) } else { let match_index = path.count_prefix(&self.partial); - let branch_node = if self.partial.at(match_index) == 16 { + let self_choice_idx = self.partial.at(match_index); + let new_leaf_choice_idx = path.at(match_index); + self.partial = self.partial.offset(match_index + 1); + let branch_node = if self_choice_idx == 16 { // Create a new leaf node and store the value in it // Create a new branch node with the leaf as a child and store self's value // Branch { [ Leaf { Value } , ... ], SelfValue} let new_leaf = LeafNode::new(path.offset(match_index + 1), value); let mut choices = BranchNode::EMPTY_CHOICES; - choices[path.at(match_index)] = new_leaf.insert_self(state)?; + choices[new_leaf_choice_idx] = new_leaf.insert_self(state)?; BranchNode::new_with_value(Box::new(choices), self.value) - } else if path.at(match_index) == 16 { + } else if new_leaf_choice_idx == 16 { // Create a branch node with self as a child and store the value in the branch node // Branch { [Self,...], Value } let mut choices = BranchNode::EMPTY_CHOICES; - choices[self.partial.at(match_index)] = self.clone().insert_self(state)?; + choices[self_choice_idx] = self.clone().insert_self(state)?; BranchNode::new_with_value(Box::new(choices), value) } else { // Create a new leaf node and store the path and value in it @@ -70,8 +73,8 @@ impl LeafNode { // Branch { [ Leaf { Path, Value }, Self, ... ], None, None} let new_leaf = LeafNode::new(path.offset(match_index + 1), value); let mut choices = BranchNode::EMPTY_CHOICES; - choices[path.at(match_index)] = new_leaf.insert_self(state)?; - choices[self.partial.at(match_index)] = self.clone().insert_self(state)?; + choices[new_leaf_choice_idx] = new_leaf.insert_self(state)?; + choices[self_choice_idx] = self.clone().insert_self(state)?; BranchNode::new(Box::new(choices)) }; From 3cfd7c46bfce490f87dfd22291156ce10c908cc4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 12:36:12 -0300 Subject: [PATCH 118/345] Fixes --- crates/storage/trie/node/extension.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 05eecc637..823963716 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -160,10 +160,14 @@ impl ExtensionNode { let mut encoder = NodeEncoder::new(); encoder.write_list_header(prefix_len + child_len); - encoder.write_path_vec(&NibbleVec::default(), PathKind::Extension); + encoder.write_path_slice(&self.prefix); match child_hash { - NodeHash::Inline(x) => encoder.write_raw(x), - NodeHash::Hashed(x) => encoder.write_bytes(&x.0), + NodeHash::Inline(x) => { + encoder.write_raw(x); + } + NodeHash::Hashed(x) => { + encoder.write_bytes(&x.0); + } } encoder.finalize() } From ad40f231faca4bb8cd21f1cf09dd11bce0836810 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 12:39:28 -0300 Subject: [PATCH 119/345] Fix --- crates/storage/trie/node/branch.rs | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 6990b9006..01c9088b3 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -64,20 +64,23 @@ impl BranchNode { ) -> Result, TrieError> { // If path is at the end, return to its own value if present. // Otherwise, check the corresponding choice and delegate accordingly if present. - if let Some(choice) = path.next().map(usize::from) { - // Delegate to children if present - let child_hash = &self.choices[choice]; - if child_hash.is_valid() { - let child_node = state - .get_node(child_hash.clone())? - .expect("inconsistent internal tree structure"); - child_node.get(state, path) - } else { - Ok(None) + match path.next().map(usize::from) { + Some(choice) if choice < 16 => { + // Delegate to children if present + let child_hash = &self.choices[choice]; + if child_hash.is_valid() { + let child_node = state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + child_node.get(state, path) + } else { + Ok(None) + } + } + _ => { + // Return internal value if present. + Ok((!self.value.is_empty()).then_some(self.value.clone())) } - } else { - // Return internal value if present. - Ok((!self.value.is_empty()).then_some(self.value.clone())) } } From a41cf2bc321bc6da21fc9999b43acb38bb291070 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 12:54:45 -0300 Subject: [PATCH 120/345] Fix --- crates/storage/trie/node/branch.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 01c9088b3..c5f8aee45 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -94,10 +94,10 @@ impl BranchNode { // If path is at the end, insert or replace its own value. // Otherwise, check the corresponding choice and insert or delegate accordingly. match path.next() { - Some(choice) => match &mut self.choices[choice as usize] { + Some(choice) if choice < 16 => match &mut self.choices[choice as usize] { // Create new child (leaf node) choice_hash if !choice_hash.is_valid() => { - let new_leaf = LeafNode::new(path.offset(1), value); + let new_leaf = LeafNode::new(path, value); let child_hash = new_leaf.insert_self(state)?; *choice_hash = child_hash; } @@ -107,11 +107,11 @@ impl BranchNode { .get_node(choice_hash.clone())? .expect("inconsistent internal tree structure"); - let child_node = child_node.insert(state, path.offset(1), value)?; + let child_node = child_node.insert(state, path, value)?; *choice_hash = child_node.insert_self(state)?; } }, - None => { + _ => { // Insert into self self.update(value); } From 68d5f83fb507ab8486fd50a25d56134d531e7223 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 13:09:42 -0300 Subject: [PATCH 121/345] Fix --- crates/storage/trie/node/branch.rs | 4 ++-- crates/storage/trie/node/extension.rs | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index c5f8aee45..170afb19a 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -148,7 +148,7 @@ impl BranchNode { // Step 1: Remove value // Check if the value is located in a child subtrie let value = match path.next() { - Some(choice_index) => { + Some(choice_index) if choice_index < 16 => { if self.choices[choice_index as usize].is_valid() { let child_node = state .get_node(self.choices[choice_index as usize].clone())? @@ -167,7 +167,7 @@ impl BranchNode { None } } - None => { + _ => { // Remove own value (if it has one) and return it if !self.value.is_empty() { let value = self.value; diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 823963716..011717cb8 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -115,8 +115,7 @@ impl ExtensionNode { .get_node(self.child)? .expect("inconsistent internal tree structure"); // Remove value from child subtrie - let (child_node, old_value) = - child_node.remove(state, path.offset(self.prefix.len()))?; + let (child_node, old_value) = child_node.remove(state, path)?; // Restructure node based on removal let node = match child_node { // If there is no subtrie remove the node From 275659a126dad787543ff0c86eda5d824e85e1bb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 15:20:47 -0300 Subject: [PATCH 122/345] Fix --- crates/storage/trie/node/branch.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 170afb19a..40bd69eff 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -223,6 +223,10 @@ impl BranchNode { // Return node here so we don't have to update it in the state and then fetch it return Ok((Some(extension_node.into()), value)); } + Node::Leaf(mut leaf) if !self.value.is_empty() => { + leaf.partial.prepend(choice_index as u8); + *child_hash = leaf.insert_self(state)?; + } _ => {} } From 6e1b4b711d4ff920c3ebefa71be40046a4fa8abd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 16:13:36 -0300 Subject: [PATCH 123/345] [DEBUG] Add debug prints --- crates/storage/trie/trie.rs | 20 ++++++++++++-- crates/storage/trie/trie_iter.rs | 47 ++++++++++++++++++++++++++++++++ 2 files changed, 65 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 6c4922e92..82c9f31c4 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -17,6 +17,7 @@ use ethereum_types::H256; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; +use trie_iter::print_trie; #[cfg(feature = "libmdbx")] pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; @@ -72,6 +73,7 @@ impl Trie { /// Retrieve an RLP-encoded value from the trie given its RLP-encoded path. pub fn get(&self, path: &PathRLP) -> Result, TrieError> { + println!("[GET] {:?}", DumbNibbles::from_bytes(&path).as_ref()); if let Some(root) = &self.root { let root_node = self .state @@ -85,6 +87,7 @@ impl Trie { /// Insert an RLP-encoded value into the trie. pub fn insert(&mut self, path: PathRLP, value: ValueRLP) -> Result<(), TrieError> { + println!("[INSERT] {:?}", DumbNibbles::from_bytes(&path).as_ref()); let root = self.root.take(); if let Some(root_node) = root .map(|root| self.state.get_node(root)) @@ -103,14 +106,25 @@ impl Trie { let new_leaf = Node::from(LeafNode::new(DumbNibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } + if let Some(root_node) = self + .root + .clone() + .map(|root| self.state.get_node(root)) + .transpose()? + .flatten() + { + dbg!(&root_node); + } + print_trie(&self); Ok(()) } /// Remove a value from the trie given its RLP-encoded path. /// Returns the value if it was succesfully removed or None if it wasn't part of the trie pub fn remove(&mut self, path: PathRLP) -> Result, TrieError> { + println!("[REMOVE] {:?}", DumbNibbles::from_bytes(&path).as_ref()); let root = self.root.take(); - if let Some(root) = root { + let res = if let Some(root) = root { let root_node = self .state .get_node(root)? @@ -123,7 +137,9 @@ impl Trie { Ok(old_value) } else { Ok(None) - } + }; + print_trie(&self); + res } /// Return the hash of the trie's root node. diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 488fbbdd7..e7a5bc053 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -57,3 +57,50 @@ impl TrieIterator { }) } } + +pub fn print_trie(trie: &Trie) { + let stack = if let Some(root) = &trie.root { + vec![root.clone()] + } else { + vec![] + }; + print_trie_inner(stack, trie); +} + +pub fn print_trie_inner(mut stack: Vec, trie: &Trie) { + if stack.is_empty() { + return; + }; + // Fetch the last node in the stack + let next_node_hash = stack.pop().unwrap(); + let next_node = trie.state.get_node(next_node_hash).ok().unwrap().unwrap(); + match &next_node { + Node::Branch(branch_node) => { + // Add all children to the stack (in reverse order so we process first child frist) + print!("BranchNode {{ Children: ["); + for (i, child) in branch_node.choices.iter().enumerate().rev() { + print!("{i}: {:?}", child.as_ref()); + if child.is_valid() { + stack.push(child.clone()) + } + } + print!("] Value: {:?} }}\n", branch_node.value); + } + Node::Extension(extension_node) => { + // Add child to the stack + println!( + "ExtensionNode {{ Prefix: {:?} Child: {:?}}}", + extension_node.prefix, + extension_node.child.as_ref() + ); + stack.push(extension_node.child.clone()); + } + Node::Leaf(leaf) => { + println!( + "LeafNode {{ Partial: {:?} Value: {:?}}}", + leaf.partial.as_ref(), + leaf.value + ); + } + } +} From 167c59102f5f9d6c469dde641fec7678659dfa4c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 16:28:23 -0300 Subject: [PATCH 124/345] [DEBUG] Add debug prints --- crates/storage/trie/trie.rs | 38 ++++++++++++++++++------ crates/storage/trie/trie_iter.rs | 51 ++++++++++---------------------- 2 files changed, 44 insertions(+), 45 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 82c9f31c4..5ce6eefcb 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -106,15 +106,6 @@ impl Trie { let new_leaf = Node::from(LeafNode::new(DumbNibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } - if let Some(root_node) = self - .root - .clone() - .map(|root| self.state.get_node(root)) - .transpose()? - .flatten() - { - dbg!(&root_node); - } print_trie(&self); Ok(()) } @@ -1014,4 +1005,33 @@ mod test { let trie_proof = trie.get_proof(&a).unwrap(); assert_eq!(cita_proof, trie_proof); } + + #[test] + fn jijo() { + let mut trie = Trie::new_temp(); + let mut data = vec![(vec![0, 0, 0, 0, 0], false), (vec![0, 0, 0, 0, 1], true)]; + // Remove duplicate values with different expected status + data.sort_by_key(|(val, _)| val.clone()); + data.dedup_by_key(|(val, _)| val.clone()); + // Insertions + for (val, _) in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap(); + } + // Removals + for (val, should_remove) in data.iter() { + if *should_remove { + let removed = trie.remove(val.clone()).unwrap(); + assert_eq!(removed, Some(val.clone())); + } + } + // Check trie values + for (val, removed) in data.iter() { + let item = trie.get(val).unwrap(); + if !removed { + assert_eq!(item, Some(val.clone())); + } else { + assert!(item.is_none()); + } + } + } } diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index e7a5bc053..293c7e985 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -59,48 +59,27 @@ impl TrieIterator { } pub fn print_trie(trie: &Trie) { - let stack = if let Some(root) = &trie.root { - vec![root.clone()] - } else { - vec![] - }; - print_trie_inner(stack, trie); + let Some(root) = &trie.root else { return }; + print_node(trie, root.clone()); + print!("\n") } -pub fn print_trie_inner(mut stack: Vec, trie: &Trie) { - if stack.is_empty() { - return; - }; - // Fetch the last node in the stack - let next_node_hash = stack.pop().unwrap(); - let next_node = trie.state.get_node(next_node_hash).ok().unwrap().unwrap(); - match &next_node { - Node::Branch(branch_node) => { - // Add all children to the stack (in reverse order so we process first child frist) - print!("BranchNode {{ Children: ["); - for (i, child) in branch_node.choices.iter().enumerate().rev() { - print!("{i}: {:?}", child.as_ref()); +pub fn print_node(trie: &Trie, node_hash: NodeHash) { + match trie.state.get_node(node_hash).unwrap().unwrap() { + Node::Branch(n) => { + print!("Branch{:?} [", n.value); + for (i, child) in n.choices.iter().enumerate() { if child.is_valid() { - stack.push(child.clone()) + print!(" {i}: "); + print_node(trie, child.clone()); } } - print!("] Value: {:?} }}\n", branch_node.value); + print!(" ]") } - Node::Extension(extension_node) => { - // Add child to the stack - println!( - "ExtensionNode {{ Prefix: {:?} Child: {:?}}}", - extension_node.prefix, - extension_node.child.as_ref() - ); - stack.push(extension_node.child.clone()); - } - Node::Leaf(leaf) => { - println!( - "LeafNode {{ Partial: {:?} Value: {:?}}}", - leaf.partial.as_ref(), - leaf.value - ); + Node::Extension(n) => { + print!("Ext{:?} -> ", n.prefix.as_ref()); + print_node(trie, n.child); } + Node::Leaf(n) => print!("Leaf{:?}", n.value), } } From aad0bcf6980ff0d1bb3b08beeaa0eca2a5a6a12e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 16:58:47 -0300 Subject: [PATCH 125/345] Fix --- crates/storage/trie/node/branch.rs | 2 +- crates/storage/trie/node/extension.rs | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 40bd69eff..a7b7f37ea 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -223,7 +223,7 @@ impl BranchNode { // Return node here so we don't have to update it in the state and then fetch it return Ok((Some(extension_node.into()), value)); } - Node::Leaf(mut leaf) if !self.value.is_empty() => { + Node::Leaf(mut leaf) if self.value.is_empty() => { leaf.partial.prepend(choice_index as u8); *child_hash = leaf.insert_self(state)?; } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 011717cb8..a5b8db21b 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -133,7 +133,11 @@ impl ExtensionNode { extension_node.into() } // If it is a leaf node replace self with it - Node::Leaf(leaf_node) => leaf_node.into(), + Node::Leaf(mut leaf_node) => { + self.prefix.extend(&leaf_node.partial); + leaf_node.partial = self.prefix; + leaf_node.into() + } }), }; From 71357ddfd79dafcd5611bb5ad173d8faa8999b5c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 16:59:04 -0300 Subject: [PATCH 126/345] [DEBUG] Add debug prints --- crates/storage/trie/trie_iter.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 293c7e985..108a7100c 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -80,6 +80,6 @@ pub fn print_node(trie: &Trie, node_hash: NodeHash) { print!("Ext{:?} -> ", n.prefix.as_ref()); print_node(trie, n.child); } - Node::Leaf(n) => print!("Leaf{:?}", n.value), + Node::Leaf(n) => print!("Leaf{:?}{:?}", n.partial.as_ref(), n.value), } } From cd847b9f1ce1f1f34e2fdf6ef48b9c952ea51e35 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 6 Nov 2024 17:13:13 -0300 Subject: [PATCH 127/345] refactor: add next_choice method --- crates/storage/trie/dumb_nibbles.rs | 5 ++ crates/storage/trie/node/branch.rs | 94 +++++++++++++---------------- 2 files changed, 48 insertions(+), 51 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index 6aa599f61..c331e3593 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -64,6 +64,11 @@ impl DumbNibbles { (!self.is_empty()).then_some(self.data.remove(0)) } + /// Removes and returns the first nibble if it is a suitable choice index (aka < 16) + pub fn next_choice(&mut self) -> Option { + self.next().filter(|choice| *choice < 16).map(usize::from) + } + pub fn offset(&self, offset: usize) -> DumbNibbles { self.slice(offset, self.len()) } diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index a7b7f37ea..754e26256 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -64,23 +64,20 @@ impl BranchNode { ) -> Result, TrieError> { // If path is at the end, return to its own value if present. // Otherwise, check the corresponding choice and delegate accordingly if present. - match path.next().map(usize::from) { - Some(choice) if choice < 16 => { - // Delegate to children if present - let child_hash = &self.choices[choice]; - if child_hash.is_valid() { - let child_node = state - .get_node(child_hash.clone())? - .expect("inconsistent internal tree structure"); - child_node.get(state, path) - } else { - Ok(None) - } - } - _ => { - // Return internal value if present. - Ok((!self.value.is_empty()).then_some(self.value.clone())) + if let Some(choice) = path.next_choice() { + // Delegate to children if present + let child_hash = &self.choices[choice]; + if child_hash.is_valid() { + let child_node = state + .get_node(child_hash.clone())? + .expect("inconsistent internal tree structure"); + child_node.get(state, path) + } else { + Ok(None) } + } else { + // Return internal value if present. + Ok((!self.value.is_empty()).then_some(self.value.clone())) } } @@ -93,8 +90,8 @@ impl BranchNode { ) -> Result { // If path is at the end, insert or replace its own value. // Otherwise, check the corresponding choice and insert or delegate accordingly. - match path.next() { - Some(choice) if choice < 16 => match &mut self.choices[choice as usize] { + if let Some(choice) = path.next_choice() { + match &mut self.choices[choice as usize] { // Create new child (leaf node) choice_hash if !choice_hash.is_valid() => { let new_leaf = LeafNode::new(path, value); @@ -110,12 +107,11 @@ impl BranchNode { let child_node = child_node.insert(state, path, value)?; *choice_hash = child_node.insert_self(state)?; } - }, - _ => { - // Insert into self - self.update(value); } - }; + } else { + // Insert into self + self.update(value); + } Ok(self.into()) } @@ -147,36 +143,33 @@ impl BranchNode { // Step 1: Remove value // Check if the value is located in a child subtrie - let value = match path.next() { - Some(choice_index) if choice_index < 16 => { - if self.choices[choice_index as usize].is_valid() { - let child_node = state - .get_node(self.choices[choice_index as usize].clone())? - .expect("inconsistent internal tree structure"); - // Remove value from child node - let (child_node, old_value) = child_node.remove(state, path.clone())?; - if let Some(child_node) = child_node { - // Update child node - self.choices[choice_index as usize] = child_node.insert_self(state)?; - } else { - // Remove child hash if the child subtrie was removed in the process - self.choices[choice_index as usize] = NodeHash::default(); - } - old_value + let value = if let Some(choice_index) = path.next_choice() { + if self.choices[choice_index as usize].is_valid() { + let child_node = state + .get_node(self.choices[choice_index as usize].clone())? + .expect("inconsistent internal tree structure"); + // Remove value from child node + let (child_node, old_value) = child_node.remove(state, path.clone())?; + if let Some(child_node) = child_node { + // Update child node + self.choices[choice_index as usize] = child_node.insert_self(state)?; } else { - None + // Remove child hash if the child subtrie was removed in the process + self.choices[choice_index as usize] = NodeHash::default(); } + old_value + } else { + None } - _ => { - // Remove own value (if it has one) and return it - if !self.value.is_empty() { - let value = self.value; - self.value = Default::default(); + } else { + // Remove own value (if it has one) and return it + if !self.value.is_empty() { + let value = self.value; + self.value = Default::default(); - (!value.is_empty()).then_some(value) - } else { - None - } + (!value.is_empty()).then_some(value) + } else { + None } }; @@ -199,7 +192,6 @@ impl BranchNode { (Some((i, x)), false) => Some((i, x)), }) }); - let child_hash = match choice_count { Ok(Some((choice_index, child_hash))) => { let child_node = state @@ -324,7 +316,7 @@ impl BranchNode { node_path.push(encoded); }; // Check the corresponding choice and delegate accordingly if present. - if let Some(choice) = path.next().map(usize::from) { + if let Some(choice) = path.next_choice() { // Continue to child let child_hash = &self.choices[choice]; if child_hash.is_valid() { From 37c93d7e9f7e407f4ae7130ee028783489f8533a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 10:36:22 -0300 Subject: [PATCH 128/345] Simplify leaf node encoding --- crates/storage/trie/node/leaf.rs | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 4d12d33b2..2118d4992 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,3 +1,6 @@ +use bytes::Bytes; +use ethereum_rust_rlp::structs::Encoder; + use crate::{ dumb_nibbles::DumbNibbles, error::TrieError, @@ -106,19 +109,13 @@ impl LeafNode { } /// Encodes the node given the offset in the path traversed before reaching this node - /// TODO: Fix pub fn encode_raw(&self) -> Vec { - let path_len = NodeEncoder::path_len(self.partial.len()); - let value_len = NodeEncoder::bytes_len( - self.value.len(), - self.value.first().copied().unwrap_or_default(), - ); - - let mut encoder = crate::node_hash::NodeEncoder::new(); - encoder.write_list_header(path_len + value_len); - encoder.write_path_slice(&self.partial); - encoder.write_bytes(&self.value); - encoder.finalize() + let mut buf = vec![]; + Encoder::new(&mut buf) + .encode_field(&Bytes::copy_from_slice(&self.partial.encode_compact())) + .encode_field(&Bytes::copy_from_slice(&self.value)) + .finish(); + buf } /// Inserts the node into the state and returns its hash From 92313f0fb320bacb4b7c696d23edc25aafa7689f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 12:26:48 -0300 Subject: [PATCH 129/345] Simplify encoding of Leaf --- crates/common/rlp/structs.rs | 13 +++++++++++++ crates/storage/trie/node/extension.rs | 3 +++ crates/storage/trie/node/leaf.rs | 8 ++++---- 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index 01e228e51..bfb1016ea 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -207,6 +207,19 @@ impl<'a> Encoder<'a> { encode_length(self.temp_buf.len(), self.buf); self.buf.put_slice(&self.temp_buf); } + + /// Adds a raw value to the buffer without rlp-encoding it + pub fn encode_raw(mut self, value: &[u8]) -> Self { + self.temp_buf.put_slice(value); + self + } + + /// Stores a field to be encoded as bytes + /// This method is used to bypass the conflicting implementations between Vec and Vec + pub fn encode_bytes(mut self, value: &[u8]) -> Self { + <[u8] as RLPEncode>::encode(value, &mut self.temp_buf); + self + } } #[cfg(test)] diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index a5b8db21b..e50047a6d 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,3 +1,6 @@ +use bytes::Bytes; +use ethereum_rust_rlp::structs::Encoder; + use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; use crate::nibble::NibbleVec; diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 2118d4992..e564c2999 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -103,17 +103,17 @@ impl LeafNode { }) } - /// Computes the node's hash given the offset in the path traversed before reaching this node + /// Computes the node's hash pub fn compute_hash(&self) -> NodeHash { NodeHash::from_encoded_raw(self.encode_raw()) } - /// Encodes the node given the offset in the path traversed before reaching this node + /// Encodes the node pub fn encode_raw(&self) -> Vec { let mut buf = vec![]; Encoder::new(&mut buf) - .encode_field(&Bytes::copy_from_slice(&self.partial.encode_compact())) - .encode_field(&Bytes::copy_from_slice(&self.value)) + .encode_bytes(&self.partial.encode_compact()) + .encode_bytes(&self.value) .finish(); buf } From 3eb5ee98be779bbd823669b47e3c3f7d904b5cc0 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 12:29:28 -0300 Subject: [PATCH 130/345] Simplify encoding of Extension --- crates/storage/trie/node/extension.rs | 21 +++++++-------------- 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index e50047a6d..6505ba7f5 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -157,25 +157,18 @@ impl ExtensionNode { /// Encodes the node pub fn encode_raw(&self) -> Vec { - let child_hash = &self.child; - let prefix_len = NodeEncoder::path_len(self.prefix.len()); - let child_len = match child_hash { - NodeHash::Inline(ref x) => x.len(), - NodeHash::Hashed(x) => NodeEncoder::bytes_len(32, x[0]), - }; - - let mut encoder = NodeEncoder::new(); - encoder.write_list_header(prefix_len + child_len); - encoder.write_path_slice(&self.prefix); - match child_hash { + let mut buf = vec![]; + let mut encoder = Encoder::new(&mut buf).encode_bytes(&self.prefix.encode_compact()); + match &self.child { NodeHash::Inline(x) => { - encoder.write_raw(x); + encoder = encoder.encode_raw(x); } NodeHash::Hashed(x) => { - encoder.write_bytes(&x.0); + encoder = encoder.encode_bytes(&x.0); } } - encoder.finalize() + encoder.finish(); + buf } /// Inserts the node into the state and returns its hash From eb4fd0c93b8c2daaf4df52c593e0a8e450533da1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 12:40:50 -0300 Subject: [PATCH 131/345] Simplify encoding of Branch --- crates/storage/trie/node/branch.rs | 56 +++++++----------------------- 1 file changed, 13 insertions(+), 43 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 754e26256..c45c8bf1c 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -1,9 +1,7 @@ +use ethereum_rust_rlp::structs::Encoder; + use crate::{ - dumb_nibbles::DumbNibbles, - error::TrieError, - node_hash::{NodeEncoder, NodeHash}, - state::TrieState, - ValueRLP, + dumb_nibbles::DumbNibbles, error::TrieError, node_hash::NodeHash, state::TrieState, ValueRLP, }; use super::{ExtensionNode, LeafNode, Node}; @@ -252,46 +250,18 @@ impl BranchNode { /// Encodes the node pub fn encode_raw(&self) -> Vec { - let hash_choice = |node_hash: &NodeHash| -> (Vec, usize) { - if node_hash.is_valid() { - match node_hash { - NodeHash::Hashed(x) => (x.as_bytes().to_vec(), 32), - NodeHash::Inline(x) => (x.clone(), x.len()), - } - } else { - (Vec::new(), 0) + let mut buf = vec![]; + let mut encoder = Encoder::new(&mut buf); + for child in self.choices.iter() { + match child { + NodeHash::Hashed(hash) => encoder = encoder.encode_bytes(&hash.0), + NodeHash::Inline(raw) if !raw.is_empty() => encoder = encoder.encode_raw(raw), + _ => encoder = encoder.encode_bytes(&[]), } - }; - let children = self.choices.iter().map(hash_choice).collect::>(); - let encoded_value = (!self.value.is_empty()).then_some(&self.value[..]); - - let mut children_len: usize = children - .iter() - .map(|x| match x { - (_, 0) => 1, - (x, 32) => NodeEncoder::bytes_len(32, x[0]), - (_, y) => *y, - }) - .sum(); - if let Some(value) = encoded_value { - children_len += - NodeEncoder::bytes_len(value.len(), value.first().copied().unwrap_or_default()); - } else { - children_len += 1; - } - - let mut encoder = NodeEncoder::new(); - encoder.write_list_header(children_len); - children.iter().for_each(|(x, len)| match len { - 0 => encoder.write_bytes(&[]), - 32 => encoder.write_bytes(x), - _ => encoder.write_raw(x), - }); - match encoded_value { - Some(value) => encoder.write_bytes(value), - None => encoder.write_bytes(&[]), } - encoder.finalize() + encoder = encoder.encode_bytes(&self.value); + encoder.finish(); + buf } /// Inserts the node into the state and returns its hash From d50e0ff341a2f8643bc4374b0f88fc783ffc97d3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 12:44:05 -0300 Subject: [PATCH 132/345] Remove the NodeEncoder --- crates/storage/trie/node/extension.rs | 4 +- crates/storage/trie/node/leaf.rs | 9 +-- crates/storage/trie/node_hash.rs | 104 -------------------------- 3 files changed, 3 insertions(+), 114 deletions(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 6505ba7f5..d7571dfa5 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,10 +1,8 @@ -use bytes::Bytes; use ethereum_rust_rlp::structs::Encoder; use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; -use crate::nibble::NibbleVec; -use crate::node_hash::{NodeEncoder, NodeHash, PathKind}; +use crate::node_hash::NodeHash; use crate::state::TrieState; use crate::ValueRLP; diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index e564c2999..ec9473f47 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,13 +1,8 @@ -use bytes::Bytes; use ethereum_rust_rlp::structs::Encoder; use crate::{ - dumb_nibbles::DumbNibbles, - error::TrieError, - node::BranchNode, - node_hash::{NodeEncoder, NodeHash}, - state::TrieState, - ValueRLP, + dumb_nibbles::DumbNibbles, error::TrieError, node::BranchNode, node_hash::NodeHash, + state::TrieState, ValueRLP, }; use super::{ExtensionNode, Node}; diff --git a/crates/storage/trie/node_hash.rs b/crates/storage/trie/node_hash.rs index 81b28b5be..f477b5e79 100644 --- a/crates/storage/trie/node_hash.rs +++ b/crates/storage/trie/node_hash.rs @@ -4,10 +4,6 @@ use ethereum_types::H256; use libmdbx::orm::{Decodable, Encodable}; use sha3::{Digest, Keccak256}; -use crate::dumb_nibbles::DumbNibbles; - -use super::nibble::NibbleVec; - /// Struct representing a trie node hash /// If the encoded node is less than 32 bits, contains the encoded node itself // TODO: Check if we can omit the Inline variant, as nodes will always be bigger than 32 bits in our use case @@ -23,21 +19,6 @@ const fn compute_byte_usage(value: usize) -> usize { (bits_used.saturating_sub(1) >> 3) + 1 } -#[derive(Clone, Copy, Debug, Eq, Hash, PartialEq)] -pub enum PathKind { - Extension, - Leaf, -} - -impl PathKind { - const fn into_flag(self) -> u8 { - match self { - PathKind::Extension => 0x00, - PathKind::Leaf => 0x20, - } - } -} - impl AsRef<[u8]> for NodeHash { fn as_ref(&self) -> &[u8] { match self { @@ -156,88 +137,3 @@ impl RLPDecode for NodeHash { Ok((hash, rest)) } } - -#[derive(Default, Debug)] -pub struct NodeEncoder { - encoded: Vec, -} - -impl NodeEncoder { - pub fn new() -> Self { - Self { - ..Default::default() - } - } - - pub const fn path_len(value_len: usize) -> usize { - Self::bytes_len((value_len >> 1) + 1, 0) - } - - pub const fn bytes_len(value_len: usize, first_value: u8) -> usize { - match value_len { - 1 if first_value < 128 => 1, - l if l < 56 => l + 1, - l => l + compute_byte_usage(l) + 1, - } - } - - pub fn write_list_header(&mut self, children_len: usize) { - self.write_len(0xC0, 0xF7, children_len); - } - - fn write_len(&mut self, short_base: u8, long_base: u8, value: usize) { - match value { - l if l < 56 => self.write_raw(&[short_base + l as u8]), - l => { - let l_len = compute_byte_usage(l); - self.write_raw(&[long_base + l_len as u8]); - self.write_raw(&l.to_be_bytes()[core::mem::size_of::() - l_len..]); - } - } - } - - pub fn write_raw(&mut self, value: &[u8]) { - self.encoded.extend_from_slice(value); - } - - pub fn write_path_slice(&mut self, value: &DumbNibbles) { - self.write_bytes(&value.encode_compact()); - } - - pub fn write_path_vec(&mut self, value: &NibbleVec, kind: PathKind) { - let mut flag = kind.into_flag(); - - // TODO: Do not use iterators. - let nibble_count = value.len(); - let nibble_iter = if nibble_count & 0x01 != 0 { - let mut iter = value.iter(); - flag |= 0x10; - flag |= iter.next().unwrap() as u8; - iter - } else { - value.iter() - }; - - let i2 = nibble_iter.clone().skip(1).step_by(2); - if nibble_count > 1 { - self.write_len(0x80, 0xB7, (nibble_count >> 1) + 1); - } - self.write_raw(&[flag]); - for (a, b) in nibble_iter.step_by(2).zip(i2) { - self.write_raw(&[((a as u8) << 4) | (b as u8)]); - } - } - - pub fn write_bytes(&mut self, value: &[u8]) { - if value.len() == 1 && value[0] < 128 { - self.write_raw(&[value[0]]); - } else { - self.write_len(0x80, 0xB7, value.len()); - self.write_raw(value); - } - } - - pub fn finalize(self) -> Vec { - self.encoded - } -} From 856224dd8227216c5df4ef11d05473475d77ed10 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 12:45:13 -0300 Subject: [PATCH 133/345] Clippy --- crates/storage/trie/node/branch.rs | 10 +++++----- crates/storage/trie/node_hash.rs | 5 ----- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index c45c8bf1c..51cc2de8d 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -89,7 +89,7 @@ impl BranchNode { // If path is at the end, insert or replace its own value. // Otherwise, check the corresponding choice and insert or delegate accordingly. if let Some(choice) = path.next_choice() { - match &mut self.choices[choice as usize] { + match &mut self.choices[choice] { // Create new child (leaf node) choice_hash if !choice_hash.is_valid() => { let new_leaf = LeafNode::new(path, value); @@ -142,18 +142,18 @@ impl BranchNode { // Step 1: Remove value // Check if the value is located in a child subtrie let value = if let Some(choice_index) = path.next_choice() { - if self.choices[choice_index as usize].is_valid() { + if self.choices[choice_index].is_valid() { let child_node = state - .get_node(self.choices[choice_index as usize].clone())? + .get_node(self.choices[choice_index].clone())? .expect("inconsistent internal tree structure"); // Remove value from child node let (child_node, old_value) = child_node.remove(state, path.clone())?; if let Some(child_node) = child_node { // Update child node - self.choices[choice_index as usize] = child_node.insert_self(state)?; + self.choices[choice_index] = child_node.insert_self(state)?; } else { // Remove child hash if the child subtrie was removed in the process - self.choices[choice_index as usize] = NodeHash::default(); + self.choices[choice_index] = NodeHash::default(); } old_value } else { diff --git a/crates/storage/trie/node_hash.rs b/crates/storage/trie/node_hash.rs index f477b5e79..655e98647 100644 --- a/crates/storage/trie/node_hash.rs +++ b/crates/storage/trie/node_hash.rs @@ -14,11 +14,6 @@ pub enum NodeHash { Inline(Vec), } -const fn compute_byte_usage(value: usize) -> usize { - let bits_used = usize::BITS as usize - value.leading_zeros() as usize; - (bits_used.saturating_sub(1) >> 3) + 1 -} - impl AsRef<[u8]> for NodeHash { fn as_ref(&self) -> &[u8] { match self { From f7136571544c4404c437b9dd88a13ed5e1d3595a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 13:51:32 -0300 Subject: [PATCH 134/345] Update TrieIterator --- crates/storage/trie/dumb_nibbles.rs | 21 +++++++++++ crates/storage/trie/trie.rs | 2 +- crates/storage/trie/trie_iter.rs | 55 ++++++++++++++++++++++------- 3 files changed, 64 insertions(+), 14 deletions(-) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/dumb_nibbles.rs index c331e3593..602532bcf 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/dumb_nibbles.rs @@ -90,6 +90,11 @@ impl DumbNibbles { self.data.insert(0, nibble); } + /// Inserts a nibble at the end + pub fn append(&mut self, nibble: u8) { + self.data.push(nibble); + } + /// Taken from https://github.com/citahub/cita_trie/blob/master/src/nibbles.rs#L56 pub fn encode_compact(&self) -> Vec { let mut compact = vec![]; @@ -124,6 +129,22 @@ impl DumbNibbles { pub fn is_leaf(&self) -> bool { self.data[self.data.len() - 1] == 16 } + + pub fn to_bytes(&self) -> Vec { + // Trim leaf flag + let data = if !self.is_empty() && self.is_leaf() { + &self.data[..self.len() - 1] + } else { + &self.data[..] + }; + // Combine nibbles into bytes + data.chunks(2) + .map(|chunk| match chunk.len() { + 1 => chunk[0] << 4, + _ => chunk[0] << 4 | chunk[1], + }) + .collect::>() + } } impl AsRef<[u8]> for DumbNibbles { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 5ce6eefcb..2ba05e2ed 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -218,7 +218,7 @@ impl Trie { } impl IntoIterator for Trie { - type Item = Node; + type Item = (DumbNibbles, Node); type IntoIter = TrieIterator; diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 108a7100c..ffc3dec60 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -1,14 +1,15 @@ -use crate::{node::Node, node_hash::NodeHash, PathRLP, Trie, ValueRLP}; +use crate::{dumb_nibbles::DumbNibbles, node::Node, node_hash::NodeHash, PathRLP, Trie, ValueRLP}; pub struct TrieIterator { trie: Trie, - stack: Vec, + // The stack contains the current traversed path and the next node to be traversed + stack: Vec<(DumbNibbles, NodeHash)>, } impl TrieIterator { pub(crate) fn new(trie: Trie) -> Self { let stack = if let Some(root) = &trie.root { - vec![root.clone()] + vec![(DumbNibbles::default(), root.clone())] } else { vec![] }; @@ -17,43 +18,51 @@ impl TrieIterator { } impl Iterator for TrieIterator { - type Item = Node; + type Item = (DumbNibbles, Node); fn next(&mut self) -> Option { if self.stack.is_empty() { return None; }; // Fetch the last node in the stack - let next_node_hash = self.stack.pop()?; + let (current_path, next_node_hash) = self.stack.pop()?; let next_node = self.trie.state.get_node(next_node_hash).ok()??; + let mut next_path = current_path.clone(); match &next_node { Node::Branch(branch_node) => { // Add all children to the stack (in reverse order so we process first child frist) - for child in branch_node.choices.iter().rev() { + for (choice, child) in branch_node.choices.iter().enumerate().rev() { if child.is_valid() { - self.stack.push(child.clone()) + let mut child_path = current_path.clone(); + child_path.append(choice as u8); + self.stack.push((child_path, child.clone())) } } } Node::Extension(extension_node) => { + // Update path + next_path.extend(&extension_node.prefix); // Add child to the stack - self.stack.push(extension_node.child.clone()); + self.stack + .push((next_path.clone(), extension_node.child.clone())); + } + Node::Leaf(leaf) => { + next_path.extend(&leaf.partial); } - Node::Leaf(_) => {} } - Some(next_node) + Some((next_path, next_node)) } } impl TrieIterator { // TODO: construct path from nibbles pub fn content(self) -> impl Iterator { - self.filter_map(|n| match n { + self.filter_map(|(p, n)| match n { Node::Branch(branch_node) => { - (!branch_node.value.is_empty()).then_some((vec![], branch_node.value)) + (!branch_node.value.is_empty()).then_some((p.to_bytes(), branch_node.value)) } Node::Extension(_) => None, - Node::Leaf(leaf_node) => Some((vec![], leaf_node.value)), + Node::Leaf(leaf_node) => Some((p.to_bytes(), leaf_node.value)), }) } } @@ -83,3 +92,23 @@ pub fn print_node(trie: &Trie, node_hash: NodeHash) { Node::Leaf(n) => print!("Leaf{:?}{:?}", n.partial.as_ref(), n.value), } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn trie_iter_content() { + let expected_content = vec![ + (vec![0, 0], vec![0, 0]), + (vec![1, 1], vec![1, 1]), + (vec![2, 2], vec![2, 2]), + ]; + let mut trie = Trie::new_temp(); + for (path, value) in expected_content.clone() { + trie.insert(path, value).unwrap() + } + let content = trie.into_iter().content().collect::>(); + assert_eq!(content, expected_content); + } +} From 30ba82fe0cdc4c6c167c157e9ad934bb14f80ca1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 15:36:00 -0300 Subject: [PATCH 135/345] Add proptest --- crates/storage/trie/trie_iter.rs | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index ffc3dec60..b40f52b8a 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -95,14 +95,20 @@ pub fn print_node(trie: &Trie, node_hash: NodeHash) { #[cfg(test)] mod tests { + use super::*; + use proptest::{ + collection::{btree_map, vec}, + prelude::any, + proptest, + }; #[test] fn trie_iter_content() { let expected_content = vec![ - (vec![0, 0], vec![0, 0]), - (vec![1, 1], vec![1, 1]), - (vec![2, 2], vec![2, 2]), + (vec![0, 9], vec![3, 4]), + (vec![1, 2], vec![5, 6]), + (vec![2, 7], vec![7, 8]), ]; let mut trie = Trie::new_temp(); for (path, value) in expected_content.clone() { @@ -111,4 +117,17 @@ mod tests { let content = trie.into_iter().content().collect::>(); assert_eq!(content, expected_content); } + proptest! { + + #[test] + fn proptest_trie_iter_content(data in btree_map(vec(any::(), 5..100), vec(any::(), 5..100), 5..100)) { + let expected_content = data.clone().into_iter().collect::>(); + let mut trie = Trie::new_temp(); + for (path, value) in data.into_iter() { + trie.insert(path, value).unwrap() + } + let content = trie.into_iter().content().collect::>(); + assert_eq!(content, expected_content); + } + } } From 21537d41c794964794bcc51b19f6e259934be911 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 15:38:40 -0300 Subject: [PATCH 136/345] Remove old nibble representation --- crates/storage/trie/nibble.rs | 1085 --------------------------------- crates/storage/trie/trie.rs | 1 - 2 files changed, 1086 deletions(-) delete mode 100644 crates/storage/trie/nibble.rs diff --git a/crates/storage/trie/nibble.rs b/crates/storage/trie/nibble.rs deleted file mode 100644 index 537f9d29a..000000000 --- a/crates/storage/trie/nibble.rs +++ /dev/null @@ -1,1085 +0,0 @@ -use ethereum_rust_rlp::{ - decode::RLPDecode, - encode::RLPEncode, - error::RLPDecodeError, - structs::{Decoder, Encoder}, -}; -use smallvec::SmallVec; - -/// Struct representing a half-byte -#[derive(Clone, Copy, Debug, Eq, Hash, Ord, PartialEq, PartialOrd)] -pub enum Nibble { - V0 = 0, - V1 = 1, - V2 = 2, - V3 = 3, - V4 = 4, - V5 = 5, - V6 = 6, - V7 = 7, - V8 = 8, - V9 = 9, - V10 = 10, - V11 = 11, - V12 = 12, - V13 = 13, - V14 = 14, - V15 = 15, -} - -impl TryFrom for Nibble { - type Error = u8; - - fn try_from(value: u8) -> Result { - Ok(match value { - 0x00 => Self::V0, - 0x01 => Self::V1, - 0x02 => Self::V2, - 0x03 => Self::V3, - 0x04 => Self::V4, - 0x05 => Self::V5, - 0x06 => Self::V6, - 0x07 => Self::V7, - 0x08 => Self::V8, - 0x09 => Self::V9, - 0x0A => Self::V10, - 0x0B => Self::V11, - 0x0C => Self::V12, - 0x0D => Self::V13, - 0x0E => Self::V14, - 0x0F => Self::V15, - x => return Err(x), - }) - } -} - -impl From for u8 { - fn from(value: Nibble) -> Self { - value as u8 - } -} - -impl From for usize { - fn from(value: Nibble) -> Self { - value as usize - } -} - -/// A slice of half-bytes (represented as a slice of bytes) with an offset -/// When used as a path, the offset indicates how much of the path has been traversed -#[derive(Clone, Debug)] -pub struct NibbleSlice<'a> { - data: &'a [u8], - offset: usize, -} - -impl<'a> NibbleSlice<'a> { - /// Creates a new NibbleSlice form a slice of bytes - pub const fn new(inner: &'a [u8]) -> Self { - Self { - data: inner, - offset: 0, - } - } - - /// Returns the amount of half-bytes in the slice, taking into account the current offset - pub const fn len(&self) -> usize { - 2 * self.data.len() - self.offset - } - - pub const fn offset(&self) -> usize { - self.offset - } - - /// Returns the full original path, ignoring the current offset - pub fn data(&self) -> super::PathRLP { - self.data.to_vec() - } - - /// Splits the slice into a vec at the given offset - // TODO: Improve doc (help wanted) - pub fn split_to_vec(&self, offset: usize) -> NibbleVec { - NibbleVec { - data: SmallVec::from_slice( - &self.data[self.offset >> 1..(self.offset + offset + 1) >> 1], - ), - first_is_half: self.offset % 2 != 0, - last_is_half: (self.offset + offset) % 2 != 0, - } - } - - /// Advance the current offset by a given amount - pub fn offset_add(&mut self, delta: usize) { - self.offset += delta; - } - - /// If `prefix` is a prefix of self (with the correct nibble alignment), move the offset after - /// the prefix and return true, otherwise return false. - /// - /// Unaligned comparations are bugs (panic). - pub fn skip_prefix(&mut self, prefix: &NibbleVec) -> bool { - // Check alignment. - assert_eq!( - (self.offset % 2 != 0), - prefix.first_is_half, - "inconsistent internal tree structure", - ); - - // Prefix can only be a prefix if self.len() >= prefix.len() - if self.data.len() < prefix.data.len() { - return false; - } - - // Prepare slices. - let mut prfx_slice = prefix.data.as_slice(); - let mut self_slice = &self.data[self.offset >> 1..(self.offset >> 1) + prfx_slice.len()]; - - // If the prefix is empty, it's always a prefix. - if prfx_slice.is_empty() - || (prfx_slice.len() == 1 && prefix.first_is_half && prefix.last_is_half) - { - return true; - } - - // Check the first nibble when unaligned. - if prefix.first_is_half { - if (prfx_slice[0] & 0x0F) != (self_slice[0] & 0x0F) { - return false; - } - - self_slice = &self_slice[1..]; - prfx_slice = &prfx_slice[1..]; - } - - // Check the last nibble when unaligned. - if prefix.last_is_half { - let i = self_slice.len() - 1; - if (prfx_slice[i] & 0xF0) != (self_slice[i] & 0xF0) { - return false; - } - - self_slice = &self_slice[..i]; - prfx_slice = &prfx_slice[..i]; - } - - // Check the rest of the values. - if self_slice != prfx_slice { - return false; - } - - // Advance self. - self.offset = self.offset + (prefix.data.len() << 1) - - prefix.first_is_half as usize - - prefix.last_is_half as usize; - - true - } - - /// Compare the rest of the data in self with the data in `other` after the offset in self. - pub fn cmp_rest(&self, other: &[u8]) -> bool { - // Prepare slices. - let mut othr_slice = &other[self.offset >> 1..]; - let mut self_slice = &self.data[self.offset >> 1..]; - - if self.offset % 2 != 0 { - if (othr_slice[0] & 0x0F) != (self_slice[0] & 0x0F) { - return false; - } - - othr_slice = &othr_slice[1..]; - self_slice = &self_slice[1..]; - } - - self_slice == othr_slice - } - - /// Compares self to a NibbleVec and returns the shared nibble count (amount of nibbles that are equal, starting from self's offset) - pub fn count_prefix_vec(&mut self, other: &NibbleVec) -> usize { - if other.data.is_empty() { - return 0; - } - - // Check alignment and length. - assert_eq!(self.offset % 2 != 0, other.first_is_half); - - // Compare first nibble (if not byte-aligned). - let mut eq_count = 0; - if other.first_is_half { - if self.next().map(u8::from) == Some(other.data[0] & 0x0F) { - eq_count += 1; - } else { - return 0; - } - } - - // Compare middle bytes. - let mut byte_nibble_count = 0; - let mut check_last_half = true; - for (a, b) in self.data[(self.offset + 1) >> 1..].iter().zip( - &other.data - [other.first_is_half as usize..other.data.len() - (other.last_is_half as usize)], - ) { - if a == b { - byte_nibble_count += 2; - } else if (a & 0xF0) == (b & 0xF0) { - byte_nibble_count += 1; - check_last_half = false; - break; - } else { - check_last_half = false; - break; - } - } - eq_count += byte_nibble_count; - self.offset += byte_nibble_count; - - // Compare last nibble (if not byte-aligned). - if check_last_half - && other.last_is_half - && self.clone().next().map(u8::from) == other.data.last().map(|x| x >> 4) - { - eq_count += 1; - } - - eq_count - } - - /// Compares self to another NibbleSlice and returns the shared nibble count (amount of nibbles that are equal, starting the common offset) - /// Both slices should have the same offset - pub fn count_prefix_slice(&self, other: &NibbleSlice) -> usize { - // Check offset (and therefore alignment implicitly). - assert_eq!(self.offset, other.offset); - - // Check first nibble (if not byte-aligned). - let mut eq_count = 0; - if self.offset % 2 != 0 { - if (self.data[self.offset >> 1] & 0x0F) == (other.data[self.offset >> 1] & 0x0F) { - eq_count += 1; - } else { - return 0; - } - } - - // Compare the rest. - for (a, b) in self.data[(self.offset + 1) >> 1..] - .iter() - .zip(&other.data[(self.offset + 1) >> 1..]) - { - if a == b { - eq_count += 2; - } else if (a & 0xF0) == (b & 0xF0) { - eq_count += 1; - break; - } else { - break; - } - } - - eq_count - } -} - -impl<'a> AsRef<[u8]> for NibbleSlice<'a> { - fn as_ref(&self) -> &'a [u8] { - self.data - } -} - -impl<'a> Iterator for NibbleSlice<'a> { - type Item = Nibble; - - fn next(&mut self) -> Option { - self.data.get(self.offset >> 1).map(|byte| { - let byte = if self.offset % 2 != 0 { - byte & 0x0F - } else { - byte >> 4 - }; - - self.offset += 1; - Nibble::try_from(byte).unwrap_or_else(|_| unreachable!()) - }) - } -} - -/// A vector of nibbles, represented as a byte array, with flags indicating wether the first or last bytes represent only half a byte -#[derive(Clone, Debug, Default, Eq, PartialEq)] -pub struct NibbleVec { - data: SmallVec<[u8; 111]>, - - first_is_half: bool, - last_is_half: bool, -} - -impl NibbleVec { - #[cfg(test)] - /// Creates a new, empty NibbleVec - pub fn new() -> Self { - NibbleVec { - data: Default::default(), - first_is_half: false, - last_is_half: false, - } - } - - #[cfg(test)] - /// Create a NibbleVec from an iterator of nibbles - pub fn from_nibbles( - data_iter: impl Iterator, - starts_with_half_byte: bool, - ) -> Self { - let mut last_is_half = false; - let mut data = SmallVec::new(); - - let mut data_iter = data_iter.peekable(); - if starts_with_half_byte && data_iter.peek().is_some() { - data.push(0); - last_is_half = true; - } - - for nibble in data_iter { - if !last_is_half { - data.push((nibble as u8) << 4); - } else { - *data.last_mut().unwrap() |= nibble as u8; - } - - last_is_half = !last_is_half; - } - - Self { - data, - first_is_half: starts_with_half_byte, - last_is_half, - } - } - - /// Create a NibbleVec from a single nibble - pub fn from_single(nibble: Nibble, is_right_half: bool) -> Self { - Self { - data: SmallVec::from_elem( - if is_right_half { - nibble as u8 - } else { - (nibble as u8) << 4 - }, - 1, - ), - first_is_half: is_right_half, - last_is_half: !is_right_half, - } - } - - /// Returns true if the NibbleVec is empty - pub fn is_empty(&self) -> bool { - self.data.is_empty() - } - - /// Returns the amount of nibbles (half-bytes) in the NibbleVec - pub fn len(&self) -> usize { - 2 * self.data.len() - self.first_is_half as usize - self.last_is_half as usize - } - - /// Returns an iterator over the NibbleVec - pub const fn iter(&self) -> NibbleVecIter { - NibbleVecIter { - inner: self, - pos: self.first_is_half as usize, - } - } - - /// Splits the NibbleVec at a given offset, returning the nibble at the specified offset - /// and the NibbleVecs containing the nibbles before and after the speficic nibble - pub fn split_extract_at(self, index: usize) -> (NibbleVec, Nibble, NibbleVec) { - let offset = (index + 1 + self.first_is_half as usize) >> 1; - let mut left_vec = NibbleVec { - data: SmallVec::from_slice(&self.data[..offset]), - first_is_half: self.first_is_half, - last_is_half: (index + self.first_is_half as usize) % 2 != 0, - }; - left_vec.normalize(); - - let offset = index + self.first_is_half as usize; - // Check out of bounds for last half-byte. - assert!( - ((offset + self.last_is_half as usize) >> 1) < self.data.len(), - "out of bounds" - ); - let value = if offset % 2 != 0 { - self.data[offset >> 1] & 0x0F - } else { - self.data[offset >> 1] >> 4 - }; - let value = Nibble::try_from(value).unwrap_or_else(|_| unreachable!()); - - let offset = (index + 1 + self.first_is_half as usize) >> 1; - let mut right_vec = NibbleVec { - data: if offset >= self.data.len() { - SmallVec::new() - } else { - SmallVec::from_slice(&self.data[offset..]) - }, - first_is_half: (index + self.first_is_half as usize) % 2 == 0, - last_is_half: self.last_is_half, - }; - right_vec.normalize(); - - (left_vec, value, right_vec) - } - - /// Inserts a nibble at the start of the NibbleVec - pub fn prepend(&mut self, nibble: Nibble) { - if self.first_is_half { - self.data[0] = (self.data[0] & 0x0F) | ((nibble as u8) << 4); - } else { - self.data.insert(0, nibble as u8); - } - self.first_is_half = !self.first_is_half; - } - - /// Extends self with another NibbleVec - pub fn extend(&mut self, other: &Self) { - // Ensure alignment. - assert_eq!(self.last_is_half, other.first_is_half); - - // Copy half-byte (if misaligned). - if self.last_is_half { - let last = self.data.last_mut().unwrap(); - *last = (*last & 0xF0) | (other.data[0] & 0x0F); - } - self.last_is_half = other.last_is_half; - - // Extend. - self.data.extend(if other.first_is_half { - other.data[1..].iter().copied() - } else { - other.data.iter().copied() - }); - } - - /// Resets self to default (full clear) if there are no nibbles represented by self - pub(crate) fn normalize(&mut self) { - if self.data.is_empty() || (self.data.len() == 1 && self.first_is_half && self.last_is_half) - { - self.data.clear(); - self.first_is_half = false; - self.last_is_half = false; - } - } -} - -#[derive(Clone)] -/// An iterator of nibbles (half-bytes) -pub struct NibbleVecIter<'a> { - inner: &'a NibbleVec, - pos: usize, -} - -impl<'a> Iterator for NibbleVecIter<'a> { - type Item = Nibble; - - fn next(&mut self) -> Option { - self.inner.data.get(self.pos >> 1).and_then(|byte| { - if (self.pos >> 1) + 1 == self.inner.data.len() - && self.pos % 2 == 1 - && self.inner.last_is_half - { - return None; - } - - let byte = if self.pos % 2 != 0 { - byte & 0x0F - } else { - byte >> 4 - }; - - self.pos += 1; - Nibble::try_from(byte).map_or_else(|_| unreachable!(), Some) - }) - } -} - -impl RLPEncode for NibbleVec { - fn encode(&self, buf: &mut dyn bytes::BufMut) { - Encoder::new(buf) - .encode_field(&self.data.clone().into_vec()) - .encode_field(&self.first_is_half) - .encode_field(&self.last_is_half) - .finish() - } -} - -impl RLPDecode for NibbleVec { - fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - let decoder = Decoder::new(rlp)?; - let (data, decoder) = decoder.decode_field("data")?; - let (first_is_half, decoder) = decoder.decode_field("first_is_half")?; - let (last_is_half, decoder) = decoder.decode_field("last_is_half")?; - let data = SmallVec::<[u8; 111]>::from_vec(data); - Ok(( - Self { - data, - first_is_half, - last_is_half, - }, - decoder.finish()?, - )) - } -} - -#[cfg(test)] -mod test { - use super::*; - - #[test] - fn nibble_try_from_u8() { - assert_eq!(Nibble::try_from(0x00u8), Ok(Nibble::V0)); - assert_eq!(Nibble::try_from(0x01u8), Ok(Nibble::V1)); - assert_eq!(Nibble::try_from(0x02u8), Ok(Nibble::V2)); - assert_eq!(Nibble::try_from(0x03u8), Ok(Nibble::V3)); - assert_eq!(Nibble::try_from(0x04u8), Ok(Nibble::V4)); - assert_eq!(Nibble::try_from(0x05u8), Ok(Nibble::V5)); - assert_eq!(Nibble::try_from(0x06u8), Ok(Nibble::V6)); - assert_eq!(Nibble::try_from(0x07u8), Ok(Nibble::V7)); - assert_eq!(Nibble::try_from(0x08u8), Ok(Nibble::V8)); - assert_eq!(Nibble::try_from(0x09u8), Ok(Nibble::V9)); - assert_eq!(Nibble::try_from(0x0Au8), Ok(Nibble::V10)); - assert_eq!(Nibble::try_from(0x0Bu8), Ok(Nibble::V11)); - assert_eq!(Nibble::try_from(0x0Cu8), Ok(Nibble::V12)); - assert_eq!(Nibble::try_from(0x0Du8), Ok(Nibble::V13)); - assert_eq!(Nibble::try_from(0x0Eu8), Ok(Nibble::V14)); - assert_eq!(Nibble::try_from(0x0Fu8), Ok(Nibble::V15)); - } - - #[test] - fn nibble_into_u8() { - assert_eq!(u8::from(Nibble::V0), 0x00); - assert_eq!(u8::from(Nibble::V1), 0x01); - assert_eq!(u8::from(Nibble::V2), 0x02); - assert_eq!(u8::from(Nibble::V3), 0x03); - assert_eq!(u8::from(Nibble::V4), 0x04); - assert_eq!(u8::from(Nibble::V5), 0x05); - assert_eq!(u8::from(Nibble::V6), 0x06); - assert_eq!(u8::from(Nibble::V7), 0x07); - assert_eq!(u8::from(Nibble::V8), 0x08); - assert_eq!(u8::from(Nibble::V9), 0x09); - assert_eq!(u8::from(Nibble::V10), 0x0A); - assert_eq!(u8::from(Nibble::V11), 0x0B); - assert_eq!(u8::from(Nibble::V12), 0x0C); - assert_eq!(u8::from(Nibble::V13), 0x0D); - assert_eq!(u8::from(Nibble::V14), 0x0E); - assert_eq!(u8::from(Nibble::V15), 0x0F); - } - - #[test] - fn nibble_into_usize() { - assert_eq!(usize::from(Nibble::V0), 0x00); - assert_eq!(usize::from(Nibble::V1), 0x01); - assert_eq!(usize::from(Nibble::V2), 0x02); - assert_eq!(usize::from(Nibble::V3), 0x03); - assert_eq!(usize::from(Nibble::V4), 0x04); - assert_eq!(usize::from(Nibble::V5), 0x05); - assert_eq!(usize::from(Nibble::V6), 0x06); - assert_eq!(usize::from(Nibble::V7), 0x07); - assert_eq!(usize::from(Nibble::V8), 0x08); - assert_eq!(usize::from(Nibble::V9), 0x09); - assert_eq!(usize::from(Nibble::V10), 0x0A); - assert_eq!(usize::from(Nibble::V11), 0x0B); - assert_eq!(usize::from(Nibble::V12), 0x0C); - assert_eq!(usize::from(Nibble::V13), 0x0D); - assert_eq!(usize::from(Nibble::V14), 0x0E); - assert_eq!(usize::from(Nibble::V15), 0x0F); - } - - #[test] - fn nibble_slice_skip_prefix_success() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - })); - assert_eq!(slice.offset, 6); - } - - #[test] - fn nibble_slice_skip_prefix_success_first_half() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 1, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x02, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - })); - assert_eq!(slice.offset, 6); - } - - #[test] - fn nibble_slice_skip_prefix_success_last_half() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x50]), - first_is_half: false, - last_is_half: true, - })); - assert_eq!(slice.offset, 5); - } - - #[test] - fn nibble_slice_skip_prefix_success_first_last_half() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 1, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x02, 0x34, 0x50]), - first_is_half: true, - last_is_half: true, - })); - assert_eq!(slice.offset, 5); - } - - #[test] - fn nibble_slice_skip_prefix_success_empty() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::new(), - first_is_half: false, - last_is_half: false - }),); - assert_eq!(slice.offset, 0); - - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 1, - }; - assert!(slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x00]), - first_is_half: true, - last_is_half: true - }),); - assert_eq!(slice.offset, 1); - } - - #[test] - fn nibble_slice_skip_prefix_failure() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(!slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x21, 0x43, 0x65]), - first_is_half: false, - last_is_half: false, - })); - assert_eq!(slice.offset, 0); - } - - #[test] - #[should_panic] - fn nibble_slice_skip_prefix_failure_alignment() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - slice.skip_prefix(&NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - }); - } - - #[test] - fn nibble_slice_cmp_rest_success() { - let slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(slice.cmp_rest(&[0x12, 0x34, 0x56])); - } - - #[test] - fn nibble_slice_cmp_rest_success_offset() { - let slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 3, - }; - assert!(slice.cmp_rest(&[0xFF, 0xF4, 0x56])); - } - - #[test] - fn nibble_slice_cmp_rest_failure() { - let slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert!(!slice.cmp_rest(&[0x12, 0xF4, 0x56])); - } - - #[test] - fn nibble_slice_cmp_rest_failure_offset() { - let slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 3, - }; - assert!(!slice.cmp_rest(&[0xFF, 0xF4, 0xF6])); - } - - #[test] - fn nibble_slice_next() { - let mut slice = NibbleSlice { - data: &[0x12, 0x34, 0x56], - offset: 0, - }; - assert_eq!(slice.offset, 0); - assert_eq!(slice.next(), Some(Nibble::V1)); - assert_eq!(slice.offset, 1); - assert_eq!(slice.next(), Some(Nibble::V2)); - assert_eq!(slice.offset, 2); - assert_eq!(slice.next(), Some(Nibble::V3)); - assert_eq!(slice.offset, 3); - assert_eq!(slice.next(), Some(Nibble::V4)); - assert_eq!(slice.offset, 4); - assert_eq!(slice.next(), Some(Nibble::V5)); - assert_eq!(slice.offset, 5); - assert_eq!(slice.next(), Some(Nibble::V6)); - assert_eq!(slice.offset, 6); - assert_eq!(slice.next(), None); - assert_eq!(slice.offset, 6); - } - - #[test] - fn nibble_vec_split_extract_at_zero() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(0); - assert!(l.data.as_slice().is_empty()); - assert!(!l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V1); - assert_eq!(r.data.as_slice(), &[0x12, 0x34, 0x56]); - assert!(r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_zero_first_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(0); - assert!(l.data.as_slice().is_empty()); - assert!(!l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V2); - assert_eq!(r.data.as_slice(), &[0x34, 0x56]); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_zero_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(0); - assert!(l.data.as_slice().is_empty()); - assert!(!l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V1); - assert_eq!(r.data.as_slice(), &[0x12, 0x34, 0x56]); - assert!(r.first_is_half); - assert!(r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_zero_first_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(0); - assert!(l.data.as_slice().is_empty()); - assert!(!l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V2); - assert_eq!(r.data.as_slice(), &[0x34, 0x56]); - assert!(!r.first_is_half); - assert!(r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_middle() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(3); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(!l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V4); - assert_eq!(r.data.as_slice(), &[0x56]); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_middle_first_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(2); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V4); - assert_eq!(r.data.as_slice(), &[0x56]); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_middle_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(3); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(!l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V4); - assert_eq!(r.data.as_slice(), &[0x56]); - assert!(!r.first_is_half); - assert!(r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_middle_first_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(2); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V4); - assert_eq!(r.data.as_slice(), &[0x56]); - assert!(!r.first_is_half); - assert!(r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_end_minus_one() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(5); - assert_eq!(l.data.as_slice(), &[0x12, 0x34, 0x56]); - assert!(!l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V6); - assert!(r.data.as_slice().is_empty()); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_end_minus_one_first_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - }; - - let (l, c, r) = vec.split_extract_at(4); - assert_eq!(l.data.as_slice(), &[0x12, 0x34, 0x56]); - assert!(l.first_is_half); - assert!(l.last_is_half); - assert_eq!(c, Nibble::V6); - assert!(r.data.as_slice().is_empty()); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_end_minus_one_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(4); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(!l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V5); - assert!(r.data.as_slice().is_empty()); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - fn nibble_vec_split_extract_at_end_minus_one_first_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: true, - }; - - let (l, c, r) = vec.split_extract_at(3); - assert_eq!(l.data.as_slice(), &[0x12, 0x34]); - assert!(l.first_is_half); - assert!(!l.last_is_half); - assert_eq!(c, Nibble::V5); - assert!(r.data.as_slice().is_empty()); - assert!(!r.first_is_half); - assert!(!r.last_is_half); - } - - #[test] - #[should_panic] - fn nibble_vec_split_extract_at_end() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - }; - - vec.split_extract_at(6); - } - - #[test] - fn nibble_vec_iter_next() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: false, - }; - let mut vec_iter = vec.iter(); - - assert_eq!(vec_iter.pos, 0); - assert_eq!(vec_iter.next(), Some(Nibble::V1)); - assert_eq!(vec_iter.pos, 1); - assert_eq!(vec_iter.next(), Some(Nibble::V2)); - assert_eq!(vec_iter.pos, 2); - assert_eq!(vec_iter.next(), Some(Nibble::V3)); - assert_eq!(vec_iter.pos, 3); - assert_eq!(vec_iter.next(), Some(Nibble::V4)); - assert_eq!(vec_iter.pos, 4); - assert_eq!(vec_iter.next(), Some(Nibble::V5)); - assert_eq!(vec_iter.pos, 5); - assert_eq!(vec_iter.next(), Some(Nibble::V6)); - assert_eq!(vec_iter.pos, 6); - assert_eq!(vec_iter.next(), None); - assert_eq!(vec_iter.pos, 6); - } - - #[test] - fn nibble_vec_iter_next_first_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: false, - }; - let mut vec_iter = vec.iter(); - - assert_eq!(vec_iter.pos, 1); - assert_eq!(vec_iter.next(), Some(Nibble::V2)); - assert_eq!(vec_iter.pos, 2); - assert_eq!(vec_iter.next(), Some(Nibble::V3)); - assert_eq!(vec_iter.pos, 3); - assert_eq!(vec_iter.next(), Some(Nibble::V4)); - assert_eq!(vec_iter.pos, 4); - assert_eq!(vec_iter.next(), Some(Nibble::V5)); - assert_eq!(vec_iter.pos, 5); - assert_eq!(vec_iter.next(), Some(Nibble::V6)); - assert_eq!(vec_iter.pos, 6); - assert_eq!(vec_iter.next(), None); - assert_eq!(vec_iter.pos, 6); - } - - #[test] - fn nibble_vec_iter_next_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: false, - last_is_half: true, - }; - let mut vec_iter = vec.iter(); - - assert_eq!(vec_iter.pos, 0); - assert_eq!(vec_iter.next(), Some(Nibble::V1)); - assert_eq!(vec_iter.pos, 1); - assert_eq!(vec_iter.next(), Some(Nibble::V2)); - assert_eq!(vec_iter.pos, 2); - assert_eq!(vec_iter.next(), Some(Nibble::V3)); - assert_eq!(vec_iter.pos, 3); - assert_eq!(vec_iter.next(), Some(Nibble::V4)); - assert_eq!(vec_iter.pos, 4); - assert_eq!(vec_iter.next(), Some(Nibble::V5)); - assert_eq!(vec_iter.pos, 5); - assert_eq!(vec_iter.next(), None); - assert_eq!(vec_iter.pos, 5); - } - - #[test] - fn nibble_vec_iter_next_first_last_half() { - let vec = NibbleVec { - data: SmallVec::from_slice(&[0x12, 0x34, 0x56]), - first_is_half: true, - last_is_half: true, - }; - let mut vec_iter = vec.iter(); - - assert_eq!(vec_iter.pos, 1); - assert_eq!(vec_iter.next(), Some(Nibble::V2)); - assert_eq!(vec_iter.pos, 2); - assert_eq!(vec_iter.next(), Some(Nibble::V3)); - assert_eq!(vec_iter.pos, 3); - assert_eq!(vec_iter.next(), Some(Nibble::V4)); - assert_eq!(vec_iter.pos, 4); - assert_eq!(vec_iter.next(), Some(Nibble::V5)); - assert_eq!(vec_iter.pos, 5); - assert_eq!(vec_iter.next(), None); - assert_eq!(vec_iter.pos, 5); - } -} diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 2ba05e2ed..faf639e81 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -1,6 +1,5 @@ mod db; mod error; -mod nibble; mod node; mod node_hash; mod rlp; From 28bd344b000e0db14ad25b5520d4363c8b36e667 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 15:41:23 -0300 Subject: [PATCH 137/345] Rename DumbNibbles -> Nibbles --- .../trie/{dumb_nibbles.rs => nibbles.rs} | 50 ++++++------- crates/storage/trie/node.rs | 10 +-- crates/storage/trie/node/branch.rs | 45 +++++------- crates/storage/trie/node/extension.rs | 72 +++++++------------ crates/storage/trie/node/leaf.rs | 47 ++++++------ crates/storage/trie/test_utils.rs | 4 +- crates/storage/trie/trie.rs | 27 ++++--- crates/storage/trie/trie_iter.rs | 8 +-- 8 files changed, 113 insertions(+), 150 deletions(-) rename crates/storage/trie/{dumb_nibbles.rs => nibbles.rs} (79%) diff --git a/crates/storage/trie/dumb_nibbles.rs b/crates/storage/trie/nibbles.rs similarity index 79% rename from crates/storage/trie/dumb_nibbles.rs rename to crates/storage/trie/nibbles.rs index 602532bcf..5181f7247 100644 --- a/crates/storage/trie/dumb_nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -6,11 +6,11 @@ use ethereum_rust_rlp::{ }; #[derive(Debug, Clone, Default, PartialEq)] -pub struct DumbNibbles { +pub struct Nibbles { data: Vec, } -impl DumbNibbles { +impl Nibbles { pub fn from_hex(hex: Vec) -> Self { Self { data: hex } } @@ -41,7 +41,7 @@ impl DumbNibbles { /// If `prefix` is a prefix of self, move the offset after /// the prefix and return true, otherwise return false. - pub fn skip_prefix(&mut self, prefix: &DumbNibbles) -> bool { + pub fn skip_prefix(&mut self, prefix: &Nibbles) -> bool { if self.len() >= prefix.len() && &self.data[..prefix.len()] == prefix.as_ref() { self.data = self.data[prefix.len()..].to_vec(); true @@ -51,7 +51,7 @@ impl DumbNibbles { } /// Compares self to another and returns the shared nibble count (amount of nibbles that are equal, from the start) - pub fn count_prefix(&self, other: &DumbNibbles) -> usize { + pub fn count_prefix(&self, other: &Nibbles) -> usize { self.as_ref() .iter() .zip(other.as_ref().iter()) @@ -69,15 +69,15 @@ impl DumbNibbles { self.next().filter(|choice| *choice < 16).map(usize::from) } - pub fn offset(&self, offset: usize) -> DumbNibbles { + pub fn offset(&self, offset: usize) -> Nibbles { self.slice(offset, self.len()) } - pub fn slice(&self, start: usize, end: usize) -> DumbNibbles { - DumbNibbles::from_hex(self.data[start..end].to_vec()) + pub fn slice(&self, start: usize, end: usize) -> Nibbles { + Nibbles::from_hex(self.data[start..end].to_vec()) } - pub fn extend(&mut self, other: &DumbNibbles) { + pub fn extend(&mut self, other: &Nibbles) { self.data.extend_from_slice(other.as_ref()); } @@ -147,19 +147,19 @@ impl DumbNibbles { } } -impl AsRef<[u8]> for DumbNibbles { +impl AsRef<[u8]> for Nibbles { fn as_ref(&self) -> &[u8] { &self.data } } -impl RLPEncode for DumbNibbles { +impl RLPEncode for Nibbles { fn encode(&self, buf: &mut dyn bytes::BufMut) { Encoder::new(buf).encode_field(&self.data).finish(); } } -impl RLPDecode for DumbNibbles { +impl RLPDecode for Nibbles { fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { let decoder = Decoder::new(rlp)?; let (data, decoder) = decoder.decode_field("data")?; @@ -173,54 +173,54 @@ mod test { #[test] fn skip_prefix_true() { - let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![1, 2, 3]); + let mut a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![1, 2, 3]); assert!(a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[4, 5]) } #[test] fn skip_prefix_true_same_length() { - let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let mut a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); assert!(a.skip_prefix(&b)); assert!(a.is_empty()); } #[test] fn skip_prefix_longer_prefix() { - let mut a = DumbNibbles::from_hex(vec![1, 2, 3]); - let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let mut a = Nibbles::from_hex(vec![1, 2, 3]); + let b = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); assert!(!a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[1, 2, 3]) } #[test] fn skip_prefix_false() { - let mut a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![1, 2, 4]); + let mut a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![1, 2, 4]); assert!(!a.skip_prefix(&b)); assert_eq!(a.as_ref(), &[1, 2, 3, 4, 5]) } #[test] fn count_prefix_all() { - let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); + let a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); assert_eq!(a.count_prefix(&b), a.len()); } #[test] fn count_prefix_partial() { - let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![1, 2, 3]); + let a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![1, 2, 3]); assert_eq!(a.count_prefix(&b), b.len()); } #[test] fn count_prefix_none() { - let a = DumbNibbles::from_hex(vec![1, 2, 3, 4, 5]); - let b = DumbNibbles::from_hex(vec![2, 3, 4, 5, 6]); + let a = Nibbles::from_hex(vec![1, 2, 3, 4, 5]); + let b = Nibbles::from_hex(vec![2, 3, 4, 5, 6]); assert_eq!(a.count_prefix(&b), 0); } } diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 5d21d76ef..71c0a0892 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -6,7 +6,7 @@ pub use branch::BranchNode; pub use extension::ExtensionNode; pub use leaf::LeafNode; -use crate::{dumb_nibbles::DumbNibbles, error::TrieError}; +use crate::{error::TrieError, nibbles::Nibbles}; use super::{node_hash::NodeHash, state::TrieState, ValueRLP}; @@ -38,7 +38,7 @@ impl From for Node { impl Node { /// Retrieves a value from the subtrie originating from this node given its path - pub fn get(&self, state: &TrieState, path: DumbNibbles) -> Result, TrieError> { + pub fn get(&self, state: &TrieState, path: Nibbles) -> Result, TrieError> { match self { Node::Branch(n) => n.get(state, path), Node::Extension(n) => n.get(state, path), @@ -50,7 +50,7 @@ impl Node { pub fn insert( self, state: &mut TrieState, - path: DumbNibbles, + path: Nibbles, value: ValueRLP, ) -> Result { match self { @@ -65,7 +65,7 @@ impl Node { pub fn remove( self, state: &mut TrieState, - path: DumbNibbles, + path: Nibbles, ) -> Result<(Option, Option), TrieError> { match self { Node::Branch(n) => n.remove(state, path), @@ -80,7 +80,7 @@ impl Node { pub fn get_path( &self, state: &TrieState, - path: DumbNibbles, + path: Nibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { match self { diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 51cc2de8d..6d7e73770 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -1,8 +1,6 @@ use ethereum_rust_rlp::structs::Encoder; -use crate::{ - dumb_nibbles::DumbNibbles, error::TrieError, node_hash::NodeHash, state::TrieState, ValueRLP, -}; +use crate::{error::TrieError, nibbles::Nibbles, node_hash::NodeHash, state::TrieState, ValueRLP}; use super::{ExtensionNode, LeafNode, Node}; @@ -55,11 +53,7 @@ impl BranchNode { } /// Retrieves a value from the subtrie originating from this node given its path - pub fn get( - &self, - state: &TrieState, - mut path: DumbNibbles, - ) -> Result, TrieError> { + pub fn get(&self, state: &TrieState, mut path: Nibbles) -> Result, TrieError> { // If path is at the end, return to its own value if present. // Otherwise, check the corresponding choice and delegate accordingly if present. if let Some(choice) = path.next_choice() { @@ -83,7 +77,7 @@ impl BranchNode { pub fn insert( mut self, state: &mut TrieState, - mut path: DumbNibbles, + mut path: Nibbles, value: ValueRLP, ) -> Result { // If path is at the end, insert or replace its own value. @@ -119,7 +113,7 @@ impl BranchNode { pub fn remove( mut self, state: &mut TrieState, - mut path: DumbNibbles, + mut path: Nibbles, ) -> Result<(Option, Option), TrieError> { /* Possible flow paths: Step 1: Removal @@ -201,7 +195,7 @@ impl BranchNode { // The extension node will then replace self if self has no value Node::Branch(_) => { let extension_node = ExtensionNode::new( - DumbNibbles::from_hex(vec![choice_index as u8]), + Nibbles::from_hex(vec![choice_index as u8]), child_hash.clone(), ); *child_hash = extension_node.insert_self(state)? @@ -229,7 +223,7 @@ impl BranchNode { // If this node still has a child and value return the updated node (Some(_), true) => Some(self.into()), // If this node still has a value but no longer has children, convert it into a leaf node - (None, true) => Some(LeafNode::new(DumbNibbles::from_hex(vec![16]), self.value).into()), + (None, true) => Some(LeafNode::new(Nibbles::from_hex(vec![16]), self.value).into()), // If this node doesn't have a value, replace it with its child node (Some(x), false) => Some( state @@ -277,7 +271,7 @@ impl BranchNode { pub fn get_path( &self, state: &TrieState, - mut path: DumbNibbles, + mut path: Nibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { // Add self to node_path (if not inlined in parent) @@ -353,13 +347,11 @@ mod test { }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_bytes(&[0x00])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x00])).unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, DumbNibbles::from_bytes(&[0x10])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x10])).unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } @@ -375,8 +367,7 @@ mod test { }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_bytes(&[0x20])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x20])).unwrap(), None, ); } @@ -390,7 +381,7 @@ mod test { 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; - let path = DumbNibbles::from_bytes(&[0x2]); + let path = Nibbles::from_bytes(&[0x2]); let value = vec![0x3]; let node = node @@ -411,7 +402,7 @@ mod test { } }; - let path = DumbNibbles::from_bytes(&[0x20]); + let path = Nibbles::from_bytes(&[0x20]); let value = vec![0x21]; let node = node @@ -433,7 +424,7 @@ mod test { }; // The extension node is ignored since it's irrelevant in this test. - let path = DumbNibbles::from_bytes(&[0x00]).offset(2); + let path = Nibbles::from_bytes(&[0x00]).offset(2); let value = vec![0x1]; let new_node = node @@ -461,7 +452,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -480,7 +471,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Branch(_)))); @@ -497,7 +488,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_bytes(&[0x00])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -514,7 +505,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_bytes(&[])) + .remove(&mut trie.state, Nibbles::from_bytes(&[])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -532,7 +523,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_bytes(&[])) + .remove(&mut trie.state, Nibbles::from_bytes(&[])) .unwrap(); assert!(matches!(node, Some(Node::Branch(_)))); diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index d7571dfa5..ddb65fbf3 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,7 +1,7 @@ use ethereum_rust_rlp::structs::Encoder; -use crate::dumb_nibbles::DumbNibbles; use crate::error::TrieError; +use crate::nibbles::Nibbles; use crate::node_hash::NodeHash; use crate::state::TrieState; use crate::ValueRLP; @@ -12,22 +12,18 @@ use super::{BranchNode, Node}; /// Contains the node's prefix and a its child node hash, doesn't store any value #[derive(Debug, Clone)] pub struct ExtensionNode { - pub prefix: DumbNibbles, + pub prefix: Nibbles, pub child: NodeHash, } impl ExtensionNode { /// Creates a new extension node given its child hash and prefix - pub(crate) fn new(prefix: DumbNibbles, child: NodeHash) -> Self { + pub(crate) fn new(prefix: Nibbles, child: NodeHash) -> Self { Self { prefix, child } } /// Retrieves a value from the subtrie originating from this node given its path - pub fn get( - &self, - state: &TrieState, - mut path: DumbNibbles, - ) -> Result, TrieError> { + pub fn get(&self, state: &TrieState, mut path: Nibbles) -> Result, TrieError> { // If the path is prefixed by this node's prefix, delegate to its child. // Otherwise, no value is present. if path.skip_prefix(&self.prefix) { @@ -46,7 +42,7 @@ impl ExtensionNode { pub fn insert( mut self, state: &mut TrieState, - path: DumbNibbles, + path: Nibbles, value: ValueRLP, ) -> Result { // OUTDATED @@ -100,7 +96,7 @@ impl ExtensionNode { pub fn remove( mut self, state: &mut TrieState, - mut path: DumbNibbles, + mut path: Nibbles, ) -> Result<(Option, Option), TrieError> { /* Possible flow paths: Extension { prefix, child } -> Extension { prefix, child } (no removal) @@ -182,7 +178,7 @@ impl ExtensionNode { pub fn get_path( &self, state: &TrieState, - mut path: DumbNibbles, + mut path: Nibbles, node_path: &mut Vec>, ) -> Result<(), TrieError> { // Add self to node_path (if not inlined in parent) @@ -208,7 +204,7 @@ mod test { #[test] fn new() { - let node = ExtensionNode::new(DumbNibbles::default(), Default::default()); + let node = ExtensionNode::new(Nibbles::default(), Default::default()); assert_eq!(node.prefix.len(), 0); assert_eq!(node.child, Default::default()); @@ -225,12 +221,12 @@ mod test { }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_hex(vec![0x00])) + node.get(&trie.state, Nibbles::from_hex(vec![0x00])) .unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, DumbNibbles::from_hex(vec![0x01])) + node.get(&trie.state, Nibbles::from_hex(vec![0x01])) .unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); @@ -247,7 +243,7 @@ mod test { }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_hex(vec![0x02])) + node.get(&trie.state, Nibbles::from_hex(vec![0x02])) .unwrap(), None, ); @@ -264,7 +260,7 @@ mod test { }; let node = node - .insert(&mut trie.state, DumbNibbles::from_hex(vec![0x02]), vec![]) + .insert(&mut trie.state, Nibbles::from_hex(vec![0x02]), vec![]) .unwrap(); let node = match node { Node::Extension(x) => x, @@ -284,18 +280,14 @@ mod test { }; let node = node - .insert( - &mut trie.state, - DumbNibbles::from_hex(vec![0x10]), - vec![0x20], - ) + .insert(&mut trie.state, Nibbles::from_hex(vec![0x10]), vec![0x20]) .unwrap(); let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_hex(vec![0x10])) + node.get(&trie.state, Nibbles::from_hex(vec![0x10])) .unwrap(), Some(vec![0x20]) ); @@ -312,18 +304,14 @@ mod test { }; let node = node - .insert( - &mut trie.state, - DumbNibbles::from_hex(vec![0x10]), - vec![0x20], - ) + .insert(&mut trie.state, Nibbles::from_hex(vec![0x10]), vec![0x20]) .unwrap(); let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, DumbNibbles::from_hex(vec![0x10])) + node.get(&trie.state, Nibbles::from_hex(vec![0x10])) .unwrap(), Some(vec![0x20]) ); @@ -339,7 +327,7 @@ mod test { } } }; - let path = DumbNibbles::from_hex(vec![0x01]); + let path = Nibbles::from_hex(vec![0x01]); let value = vec![0x02]; let node = node @@ -360,7 +348,7 @@ mod test { } } }; - let path = DumbNibbles::from_hex(vec![0x01]); + let path = Nibbles::from_hex(vec![0x01]); let value = vec![0x04]; let node = node @@ -382,7 +370,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x02])) + .remove(&mut trie.state, Nibbles::from_hex(vec![0x02])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -400,7 +388,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x01])) + .remove(&mut trie.state, Nibbles::from_hex(vec![0x01])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -421,7 +409,7 @@ mod test { }; let (node, value) = node - .remove(&mut trie.state, DumbNibbles::from_hex(vec![0x00])) + .remove(&mut trie.state, Nibbles::from_hex(vec![0x00])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -440,21 +428,18 @@ mod test { } */ let leaf_node_a = LeafNode::new( - DumbNibbles::from_bytes(&[0x00, 0x00]).offset(3), + Nibbles::from_bytes(&[0x00, 0x00]).offset(3), vec![0x12, 0x34], ); let leaf_node_b = LeafNode::new( - DumbNibbles::from_bytes(&[0x00, 0x10]).offset(3), + Nibbles::from_bytes(&[0x00, 0x10]).offset(3), vec![0x56, 0x78], ); let mut choices = BranchNode::EMPTY_CHOICES; choices[0] = leaf_node_a.compute_hash(); choices[1] = leaf_node_b.compute_hash(); let branch_node = BranchNode::new(Box::new(choices)); - let node = ExtensionNode::new( - DumbNibbles::from_hex(vec![0, 0]), - branch_node.compute_hash(), - ); + let node = ExtensionNode::new(Nibbles::from_hex(vec![0, 0]), branch_node.compute_hash()); assert_eq!( node.compute_hash().as_ref(), @@ -478,21 +463,18 @@ mod test { } */ let leaf_node_a = LeafNode::new( - DumbNibbles::from_bytes(&[0x00, 0x00]), + Nibbles::from_bytes(&[0x00, 0x00]), vec![0x12, 0x34, 0x56, 0x78, 0x9A], ); let leaf_node_b = LeafNode::new( - DumbNibbles::from_bytes(&[0x00, 0x10]), + Nibbles::from_bytes(&[0x00, 0x10]), vec![0x34, 0x56, 0x78, 0x9A, 0xBC], ); let mut choices = BranchNode::EMPTY_CHOICES; choices[0] = leaf_node_a.compute_hash(); choices[1] = leaf_node_b.compute_hash(); let branch_node = BranchNode::new(Box::new(choices)); - let node = ExtensionNode::new( - DumbNibbles::from_hex(vec![0, 0]), - branch_node.compute_hash(), - ); + let node = ExtensionNode::new(Nibbles::from_hex(vec![0, 0]), branch_node.compute_hash()); assert_eq!( node.compute_hash().as_ref(), diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index ec9473f47..10f4f39ea 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,8 +1,8 @@ use ethereum_rust_rlp::structs::Encoder; use crate::{ - dumb_nibbles::DumbNibbles, error::TrieError, node::BranchNode, node_hash::NodeHash, - state::TrieState, ValueRLP, + error::TrieError, nibbles::Nibbles, node::BranchNode, node_hash::NodeHash, state::TrieState, + ValueRLP, }; use super::{ExtensionNode, Node}; @@ -10,18 +10,18 @@ use super::{ExtensionNode, Node}; /// Contains the node's hash, value & path #[derive(Debug, Clone, Default)] pub struct LeafNode { - pub partial: DumbNibbles, + pub partial: Nibbles, pub value: ValueRLP, } impl LeafNode { /// Creates a new leaf node and stores the given (path, value) pair - pub fn new(partial: DumbNibbles, value: ValueRLP) -> Self { + pub fn new(partial: Nibbles, value: ValueRLP) -> Self { Self { partial, value } } /// Returns the stored value if the given path matches the stored path - pub fn get(&self, path: DumbNibbles) -> Result, TrieError> { + pub fn get(&self, path: Nibbles) -> Result, TrieError> { if self.partial == path { Ok(Some(self.value.clone())) } else { @@ -33,7 +33,7 @@ impl LeafNode { pub fn insert( mut self, state: &mut TrieState, - path: DumbNibbles, + path: Nibbles, value: ValueRLP, ) -> Result { /* Possible flow paths: @@ -90,7 +90,7 @@ impl LeafNode { } /// Removes own value if the path matches own path and returns self and the value if it was removed - pub fn remove(self, path: DumbNibbles) -> Result<(Option, Option), TrieError> { + pub fn remove(self, path: Nibbles) -> Result<(Option, Option), TrieError> { Ok(if self.partial == path { (None, Some(self.value)) } else { @@ -150,7 +150,7 @@ mod test { }; assert_eq!( - node.get(DumbNibbles::from_bytes(&[0x12])).unwrap(), + node.get(Nibbles::from_bytes(&[0x12])).unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); } @@ -161,10 +161,7 @@ mod test { leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - assert!(node - .get(DumbNibbles::from_bytes(&[0x34])) - .unwrap() - .is_none()); + assert!(node.get(Nibbles::from_bytes(&[0x34])).unwrap().is_none()); } #[test] @@ -175,11 +172,7 @@ mod test { }; let node = node - .insert( - &mut trie.state, - DumbNibbles::from_bytes(&[0x12]), - vec![0x13], - ) + .insert(&mut trie.state, Nibbles::from_bytes(&[0x12]), vec![0x13]) .unwrap(); let node = match node { Node::Leaf(x) => x, @@ -195,7 +188,7 @@ mod test { let node = pmt_node! { @(trie) leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = DumbNibbles::from_bytes(&[0x22]); + let path = Nibbles::from_bytes(&[0x22]); let value = vec![0x23]; let node = node .insert(&mut trie.state, path.clone(), value.clone()) @@ -214,7 +207,7 @@ mod test { leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = DumbNibbles::from_bytes(&[0x13]); + let path = Nibbles::from_bytes(&[0x13]); let value = vec![0x15]; let node = node @@ -232,7 +225,7 @@ mod test { leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = DumbNibbles::from_bytes(&[0x12, 0x34]); + let path = Nibbles::from_bytes(&[0x12, 0x34]); let value = vec![0x17]; let node = node @@ -250,7 +243,7 @@ mod test { leaf { &[0x12, 0x34] => vec![0x12, 0x34, 0x56, 0x78] } }; - let path = DumbNibbles::from_bytes(&[0x12]); + let path = Nibbles::from_bytes(&[0x12]); let value = vec![0x17]; let node = node @@ -272,10 +265,10 @@ mod test { #[test] fn remove_self() { let node = LeafNode::new( - DumbNibbles::from_bytes(&[0x12, 0x34]), + Nibbles::from_bytes(&[0x12, 0x34]), vec![0x12, 0x34, 0x56, 0x78], ); - let (node, value) = node.remove(DumbNibbles::from_bytes(&[0x12, 0x34])).unwrap(); + let (node, value) = node.remove(Nibbles::from_bytes(&[0x12, 0x34])).unwrap(); assert!(node.is_none()); assert_eq!(value, Some(vec![0x12, 0x34, 0x56, 0x78])); @@ -284,11 +277,11 @@ mod test { #[test] fn remove_none() { let node = LeafNode::new( - DumbNibbles::from_bytes(&[0x12, 0x34]), + Nibbles::from_bytes(&[0x12, 0x34]), vec![0x12, 0x34, 0x56, 0x78], ); - let (node, value) = node.remove(DumbNibbles::from_bytes(&[0x12])).unwrap(); + let (node, value) = node.remove(Nibbles::from_bytes(&[0x12])).unwrap(); assert!(node.is_some()); assert_eq!(value, None); @@ -296,7 +289,7 @@ mod test { #[test] fn compute_hash_x() { - let node = LeafNode::new(DumbNibbles::from_bytes(b"key".as_ref()), b"value".to_vec()); + let node = LeafNode::new(Nibbles::from_bytes(b"key".as_ref()), b"value".to_vec()); let node_hash_ref = node.compute_hash(); assert_eq!( node_hash_ref.as_ref(), @@ -307,7 +300,7 @@ mod test { #[test] fn compute_hash_long() { let node = LeafNode::new( - DumbNibbles::from_bytes(b"key".as_ref()), + Nibbles::from_bytes(b"key".as_ref()), b"a comparatively long value".to_vec(), ); diff --git a/crates/storage/trie/test_utils.rs b/crates/storage/trie/test_utils.rs index 5fb6b2191..ceefe3cdb 100644 --- a/crates/storage/trie/test_utils.rs +++ b/crates/storage/trie/test_utils.rs @@ -80,7 +80,7 @@ macro_rules! pmt_node { $( offset $offset:expr )? ) => {{ #[allow(unused_variables)] - let prefix = $crate::dumb_nibbles::DumbNibbles::from_hex($prefix.to_vec()); + let prefix = $crate::nibbles::Nibbles::from_hex($prefix.to_vec()); $crate::node::ExtensionNode::new( prefix.clone(), @@ -99,7 +99,7 @@ macro_rules! pmt_node { $( offset $offset:expr )? ) => { { - $crate::node::LeafNode::new(DumbNibbles::from_bytes($path), $value) + $crate::node::LeafNode::new(Nibbles::from_bytes($path), $value) } }; } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index faf639e81..c415a8343 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -6,13 +6,13 @@ mod rlp; mod state; mod trie_iter; -mod dumb_nibbles; +mod nibbles; #[cfg(test)] mod test_utils; -use dumb_nibbles::DumbNibbles; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; +use nibbles::Nibbles; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; @@ -72,13 +72,13 @@ impl Trie { /// Retrieve an RLP-encoded value from the trie given its RLP-encoded path. pub fn get(&self, path: &PathRLP) -> Result, TrieError> { - println!("[GET] {:?}", DumbNibbles::from_bytes(&path).as_ref()); + println!("[GET] {:?}", Nibbles::from_bytes(&path).as_ref()); if let Some(root) = &self.root { let root_node = self .state .get_node(root.clone())? .expect("inconsistent internal tree structure"); - root_node.get(&self.state, DumbNibbles::from_bytes(path)) + root_node.get(&self.state, Nibbles::from_bytes(path)) } else { Ok(None) } @@ -86,7 +86,7 @@ impl Trie { /// Insert an RLP-encoded value into the trie. pub fn insert(&mut self, path: PathRLP, value: ValueRLP) -> Result<(), TrieError> { - println!("[INSERT] {:?}", DumbNibbles::from_bytes(&path).as_ref()); + println!("[INSERT] {:?}", Nibbles::from_bytes(&path).as_ref()); let root = self.root.take(); if let Some(root_node) = root .map(|root| self.state.get_node(root)) @@ -94,15 +94,12 @@ impl Trie { .flatten() { // If the trie is not empty, call the root node's insertion logic - let root_node = root_node.insert( - &mut self.state, - DumbNibbles::from_bytes(&path), - value.clone(), - )?; + let root_node = + root_node.insert(&mut self.state, Nibbles::from_bytes(&path), value.clone())?; self.root = Some(root_node.insert_self(&mut self.state)?) } else { // If the trie is empty, just add a leaf. - let new_leaf = Node::from(LeafNode::new(DumbNibbles::from_bytes(&path), value)); + let new_leaf = Node::from(LeafNode::new(Nibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } print_trie(&self); @@ -112,7 +109,7 @@ impl Trie { /// Remove a value from the trie given its RLP-encoded path. /// Returns the value if it was succesfully removed or None if it wasn't part of the trie pub fn remove(&mut self, path: PathRLP) -> Result, TrieError> { - println!("[REMOVE] {:?}", DumbNibbles::from_bytes(&path).as_ref()); + println!("[REMOVE] {:?}", Nibbles::from_bytes(&path).as_ref()); let root = self.root.take(); let res = if let Some(root) = root { let root_node = self @@ -120,7 +117,7 @@ impl Trie { .get_node(root)? .expect("inconsistent internal tree structure"); let (root_node, old_value) = - root_node.remove(&mut self.state, DumbNibbles::from_bytes(&path))?; + root_node.remove(&mut self.state, Nibbles::from_bytes(&path))?; self.root = root_node .map(|root| root.insert_self(&mut self.state)) .transpose()?; @@ -160,7 +157,7 @@ impl Trie { node_path.push(node.to_vec()); } if let Some(root_node) = self.state.get_node(root.clone())? { - root_node.get_path(&self.state, DumbNibbles::from_bytes(path), &mut node_path)?; + root_node.get_path(&self.state, Nibbles::from_bytes(path), &mut node_path)?; } Ok(node_path) } @@ -217,7 +214,7 @@ impl Trie { } impl IntoIterator for Trie { - type Item = (DumbNibbles, Node); + type Item = (Nibbles, Node); type IntoIter = TrieIterator; diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index b40f52b8a..ca4a640ac 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -1,15 +1,15 @@ -use crate::{dumb_nibbles::DumbNibbles, node::Node, node_hash::NodeHash, PathRLP, Trie, ValueRLP}; +use crate::{nibbles::Nibbles, node::Node, node_hash::NodeHash, PathRLP, Trie, ValueRLP}; pub struct TrieIterator { trie: Trie, // The stack contains the current traversed path and the next node to be traversed - stack: Vec<(DumbNibbles, NodeHash)>, + stack: Vec<(Nibbles, NodeHash)>, } impl TrieIterator { pub(crate) fn new(trie: Trie) -> Self { let stack = if let Some(root) = &trie.root { - vec![(DumbNibbles::default(), root.clone())] + vec![(Nibbles::default(), root.clone())] } else { vec![] }; @@ -18,7 +18,7 @@ impl TrieIterator { } impl Iterator for TrieIterator { - type Item = (DumbNibbles, Node); + type Item = (Nibbles, Node); fn next(&mut self) -> Option { if self.stack.is_empty() { From c56ad04f9a30534ac82e3a60bbb09a73576330cb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 15:49:05 -0300 Subject: [PATCH 138/345] Update some doc --- crates/storage/trie/node/branch.rs | 20 ++++++++++---------- crates/storage/trie/node/leaf.rs | 4 ++-- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 6d7e73770..e211664ce 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -5,7 +5,7 @@ use crate::{error::TrieError, nibbles::Nibbles, node_hash::NodeHash, state::Trie use super::{ExtensionNode, LeafNode, Node}; /// Branch Node of an an Ethereum Compatible Patricia Merkle Trie -/// Contains the node's hash, value, path, and the hash of its children nodes +/// Contains the node's value and the hash of its children nodes #[derive(Debug, Clone)] pub struct BranchNode { // TODO: check if switching to hashmap is a better solution @@ -42,7 +42,7 @@ impl BranchNode { } } - /// Creates a new branch node given its children and stores the given (path, value) pair + /// Creates a new branch node given its children and value pub fn new_with_value(choices: Box<[NodeHash; 16]>, value: ValueRLP) -> Self { Self { choices, value } } @@ -117,20 +117,20 @@ impl BranchNode { ) -> Result<(Option, Option), TrieError> { /* Possible flow paths: Step 1: Removal - Branch { [ ... ], Path, Value } -> Branch { [...], None, None } (remove from self) - Branch { [ childA, ... ], Path, Value } -> Branch { [childA', ... ], Path, Value } (remove from child) + Branch { [ ... ] Value } -> Branch { [...], None, None } (remove from self) + Branch { [ childA, ... ], Value } -> Branch { [childA', ... ], Value } (remove from child) Step 2: Restructure [0 children] - Branch { [], Path, Value } -> Leaf { Path, Value } (no children, with value) - Branch { [], None, None } -> Branch { [], None, None } (no children, no value) + Branch { [], Value } -> Leaf { Value } (no children, with value) + Branch { [], None } -> Branch { [], None } (no children, no value) [1 child] Branch { [ ExtensionChild], _ , _ } -> Extension { ChoiceIndex+ExtensionChildPrefx, ExtensionChildChild } - Branch { [ BranchChild ], None, None } -> Extension { ChoiceIndex, BranchChild } - Branch { [ LeafChild], None, None } -> LeafChild - Branch { [LeafChild], Path, Value } -> Branch { [ LeafChild ], Path, Value } + Branch { [ BranchChild ], None } -> Extension { ChoiceIndex, BranchChild } + Branch { [ LeafChild], None } -> LeafChild + Branch { [LeafChild], Value } -> Branch { [ LeafChild ], Value } [+1 children] - Branch { [childA, childB, ... ], None, None } -> Branch { [childA, childB, ... ], None, None } + Branch { [childA, childB, ... ], None } -> Branch { [childA, childB, ... ], None } */ // Step 1: Remove value diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 10f4f39ea..59ddbea48 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -40,7 +40,7 @@ impl LeafNode { Leaf { SelfValue } -> Leaf { Value } Leaf { SelfValue } -> Extension { Branch { [Self,...] Value } } Leaf { SelfValue } -> Extension { Branch { [ Leaf { Value } , ... ], SelfValue} } - Leaf { SelfValue } -> Branch { [ Leaf { Value }, Self, ... ], None, None} + Leaf { SelfValue } -> Branch { [ Leaf { Value }, Self, ... ], None} */ // If the path matches the stored path, update the value and return self if self.partial == path { @@ -51,6 +51,7 @@ impl LeafNode { let self_choice_idx = self.partial.at(match_index); let new_leaf_choice_idx = path.at(match_index); self.partial = self.partial.offset(match_index + 1); + let branch_node = if self_choice_idx == 16 { // Create a new leaf node and store the value in it // Create a new branch node with the leaf as a child and store self's value @@ -116,7 +117,6 @@ impl LeafNode { /// Inserts the node into the state and returns its hash /// Receives the offset that needs to be traversed to reach the leaf node from the canonical root, used to compute the node hash pub fn insert_self(self, state: &mut TrieState) -> Result { - // TODO: Fix let hash = self.compute_hash(); state.insert_node(self.into(), hash.clone()); Ok(hash) From bebce62198549572c88773dded16371a67735728 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 16:45:36 -0300 Subject: [PATCH 139/345] Simplify BranchNode::remove --- crates/storage/trie/node/branch.rs | 81 +++++++++--------------------- 1 file changed, 25 insertions(+), 56 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index e211664ce..e42d9ed66 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -166,74 +166,43 @@ impl BranchNode { }; // Step 2: Restructure self - - // Check if self only has one child left - - // An `Err(_)` means more than one choice. `Ok(Some(_))` and `Ok(None)` mean a single and no - // choices respectively. - // If there is only one child choice_count will contain the choice index and the hash of the child node - let choice_count = self + let children = self .choices - .iter_mut() + .iter() .enumerate() - .try_fold(None, |acc, (i, x)| { - Ok(match (acc, x.is_valid()) { - (None, true) => Some((i, x)), - (None, false) => None, - (Some(_), true) => return Err(()), - (Some((i, x)), false) => Some((i, x)), - }) - }); - let child_hash = match choice_count { - Ok(Some((choice_index, child_hash))) => { - let child_node = state + .filter(|(_, child)| child.is_valid()) + .collect::>(); + let new_node = match (children.len(), !self.value.is_empty()) { + // If this node still has a value but no longer has children, convert it into a leaf node + // TODO: I replaced vec![16] for vec![] look for hits in proptests + (0, true) => Some(LeafNode::new(Nibbles::from_hex(vec![]), self.value).into()), + // If this node doesn't have a value and has only one child, replace it with its child node + (1, false) => { + let (choice_index, child_hash) = children[0]; + let child = state .get_node(child_hash.clone())? .expect("inconsistent internal tree structure"); - - match child_node { - // Replace the child node with an extension node leading to it - // The extension node will then replace self if self has no value - Node::Branch(_) => { - let extension_node = ExtensionNode::new( - Nibbles::from_hex(vec![choice_index as u8]), - child_hash.clone(), - ); - *child_hash = extension_node.insert_self(state)? - } + Some(match child { + // Replace self with an extension node leading to the child + Node::Branch(_) => ExtensionNode::new( + Nibbles::from_hex(vec![choice_index as u8]), + child_hash.clone(), + ) + .into(), // Replace self with the child extension node, updating its path in the process Node::Extension(mut extension_node) => { - debug_assert!(self.value.is_empty()); // Sanity check extension_node.prefix.prepend(choice_index as u8); - // Return node here so we don't have to update it in the state and then fetch it - return Ok((Some(extension_node.into()), value)); + extension_node.into() } - Node::Leaf(mut leaf) if self.value.is_empty() => { + Node::Leaf(mut leaf) => { leaf.partial.prepend(choice_index as u8); - *child_hash = leaf.insert_self(state)?; + leaf.into() } - _ => {} - } - - Some(child_hash) + }) } - _ => None, + // Return the updated node + _ => Some(self.into()), }; - - let new_node = match (child_hash, !self.value.is_empty()) { - // If this node still has a child and value return the updated node - (Some(_), true) => Some(self.into()), - // If this node still has a value but no longer has children, convert it into a leaf node - (None, true) => Some(LeafNode::new(Nibbles::from_hex(vec![16]), self.value).into()), - // If this node doesn't have a value, replace it with its child node - (Some(x), false) => Some( - state - .get_node(x.clone())? - .expect("inconsistent internal tree structure"), - ), - // Return this node - (None, false) => Some(self.into()), - }; - Ok((new_node, value)) } From 08c666830cf65043898629853fd9be5911712084 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 16:46:56 -0300 Subject: [PATCH 140/345] Simplify --- crates/storage/trie/node/branch.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index e42d9ed66..52fcfd169 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -174,15 +174,14 @@ impl BranchNode { .collect::>(); let new_node = match (children.len(), !self.value.is_empty()) { // If this node still has a value but no longer has children, convert it into a leaf node - // TODO: I replaced vec![16] for vec![] look for hits in proptests - (0, true) => Some(LeafNode::new(Nibbles::from_hex(vec![]), self.value).into()), + (0, true) => LeafNode::new(Nibbles::from_hex(vec![]), self.value).into(), // If this node doesn't have a value and has only one child, replace it with its child node (1, false) => { let (choice_index, child_hash) = children[0]; let child = state .get_node(child_hash.clone())? .expect("inconsistent internal tree structure"); - Some(match child { + match child { // Replace self with an extension node leading to the child Node::Branch(_) => ExtensionNode::new( Nibbles::from_hex(vec![choice_index as u8]), @@ -198,12 +197,12 @@ impl BranchNode { leaf.partial.prepend(choice_index as u8); leaf.into() } - }) + } } // Return the updated node - _ => Some(self.into()), + _ => self.into(), }; - Ok((new_node, value)) + Ok((Some(new_node), value)) } /// Computes the node's hash From e1f032f34cf1b2067bd6b83f0b5ea609284c7163 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 17:08:58 -0300 Subject: [PATCH 141/345] Update doc --- crates/storage/trie/node/extension.rs | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index ddb65fbf3..269d6d85c 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -46,14 +46,15 @@ impl ExtensionNode { value: ValueRLP, ) -> Result { // OUTDATED - /* Possible flow paths (there are duplicates between different prefix lengths): + /* Possible flow paths: + * Prefix fully matches path Extension { prefix, child } -> Extension { prefix , child' } (insert into child) - Extension { prefixL+C+prefixR, child } -> Extension { prefixL, Branch { [ Extension { prefixR, child }, ..], Path, Value} } (if path fully traversed) - Extension { prefixL+C+prefixR, child } -> Extension { prefixL, Branch { [ Extension { prefixR, child }, Leaf { Path, Value }..] None, None} } (if path not fully traversed) - Extension { prefixL+C+None, child } -> Extension { prefixL, Branch { [child, ... ], Path, Value} } (if path fully traversed) - Extension { prefixL+C+None, child } -> Extension { prefixL, Branch { [child, ... ], Leaf { Path, Value }, ... }, None, None } (if path not fully traversed) - Extension { None+C+prefixR } -> Branch { [ Extension { prefixR, child } , ..], Path, Value} (if path fully traversed) - Extension { None+C+prefixR } -> Branch { [ Extension { prefixR, child } , Leaf { Path, Value } , ... ], None, None} (if path not fully traversed) + * No match between path and prefix + Extension { prefix, child } -> Branch { [ ] childValue } (insert into new branch node) + Extension { prefix, child } -> Branch { [ child ] None } (insert into new branch node) + Extension { prefix, child } -> Branch { [ Extension { prefix[1..], child } ] None } (insert into new branch node) + * Prefix partially matches path + Extension { prefix, child } -> Extension { prefix[..match], Extension { path[match..] child } } (insert into new extension node) */ let match_index = path.count_prefix(&self.prefix); if match_index == self.prefix.len() { From 9dc2752c6999660f47567abe02371dca1275673d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 17:20:06 -0300 Subject: [PATCH 142/345] Fix unit test --- crates/storage/trie/node/branch.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 52fcfd169..68fc0d56c 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -315,11 +315,13 @@ mod test { }; assert_eq!( - node.get(&trie.state, Nibbles::from_bytes(&[0x00])).unwrap(), + node.get(&trie.state, Nibbles::from_hex(vec![0, 0, 0, 16])) + .unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, Nibbles::from_bytes(&[0x10])).unwrap(), + node.get(&trie.state, Nibbles::from_hex(vec![1, 1, 0, 16])) + .unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } From bbe367d38eca891b94e19242544743d940729992 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 17:25:00 -0300 Subject: [PATCH 143/345] Fix test + code --- crates/storage/trie/nibbles.rs | 6 +++++- crates/storage/trie/node/branch.rs | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 5181f7247..fd3d84862 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -127,7 +127,11 @@ impl Nibbles { } pub fn is_leaf(&self) -> bool { - self.data[self.data.len() - 1] == 16 + if self.is_empty() { + false + } else { + self.data[self.data.len() - 1] == 16 + } } pub fn to_bytes(&self) -> Vec { diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 68fc0d56c..39d88368d 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -351,7 +351,7 @@ mod test { 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; - let path = Nibbles::from_bytes(&[0x2]); + let path = Nibbles::from_hex(vec![2]); let value = vec![0x3]; let node = node From 63f16451e00b4349b6cbee8ab6d3784fd5ad0546 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 17:36:57 -0300 Subject: [PATCH 144/345] Update test values --- crates/storage/trie/node/branch.rs | 116 ++++++++++++++--------------- crates/storage/trie/node/leaf.rs | 14 ++-- crates/storage/trie/test_utils.rs | 5 +- 3 files changed, 67 insertions(+), 68 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 39d88368d..6f047ebb1 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -309,19 +309,17 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![0,16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0,16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0, 0, 0, 16])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x00])).unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![1, 1, 0, 16])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x10])).unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } @@ -331,8 +329,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![0,16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0,16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; @@ -347,8 +345,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![0, 16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0, 16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; let path = Nibbles::from_hex(vec![2]); @@ -367,8 +365,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![0, 16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0, 16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; @@ -388,8 +386,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![0, 16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0, 16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; @@ -416,8 +414,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x00] }, - 1 => leaf { &[0x10] => vec![0x10] }, + 0 => leaf { vec![0, 16] => vec![0x00] }, + 1 => leaf { vec![0, 16] => vec![0x10] }, } }; @@ -434,9 +432,9 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x00] }, - 1 => leaf { &[0x10] => vec![0x10] }, - 2 => leaf { &[0x10] => vec![0x10] }, + 0 => leaf { vec![0, 16] => vec![0x00] }, + 1 => leaf { vec![0, 16] => vec![0x10] }, + 2 => leaf { vec![0, 16] => vec![0x10] }, } }; @@ -453,7 +451,7 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x00] }, + 0 => leaf { vec![0, 16] => vec![0x00] }, } with_leaf { &[0x01] => vec![0xFF] } }; @@ -470,7 +468,7 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x00] }, + 0 => leaf { vec![0, 16] => vec![0x00] }, } with_leaf { &[0x1] => vec![0xFF] } }; @@ -487,8 +485,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0 => leaf { &[0x00] => vec![0x00] }, - 1 => leaf { &[0x10] => vec![0x10] }, + 0 => leaf { vec![0, 16] => vec![0x00] }, + 1 => leaf { vec![0, 16] => vec![0x10] }, } with_leaf { &[0x1] => vec![0xFF] } }; @@ -505,8 +503,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 2 => leaf { &[0x20] => vec![0x20] }, - 4 => leaf { &[0x40] => vec![0x40] }, + 2 => leaf { vec![0, 16] => vec![0x20] }, + 4 => leaf { vec![0, 16] => vec![0x40] }, } }; @@ -524,22 +522,22 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0x0 => leaf { &[0x00] => vec![0x00] }, - 0x1 => leaf { &[0x10] => vec![0x10] }, - 0x2 => leaf { &[0x20] => vec![0x20] }, - 0x3 => leaf { &[0x30] => vec![0x30] }, - 0x4 => leaf { &[0x40] => vec![0x40] }, - 0x5 => leaf { &[0x50] => vec![0x50] }, - 0x6 => leaf { &[0x60] => vec![0x60] }, - 0x7 => leaf { &[0x70] => vec![0x70] }, - 0x8 => leaf { &[0x80] => vec![0x80] }, - 0x9 => leaf { &[0x90] => vec![0x90] }, - 0xA => leaf { &[0xA0] => vec![0xA0] }, - 0xB => leaf { &[0xB0] => vec![0xB0] }, - 0xC => leaf { &[0xC0] => vec![0xC0] }, - 0xD => leaf { &[0xD0] => vec![0xD0] }, - 0xE => leaf { &[0xE0] => vec![0xE0] }, - 0xF => leaf { &[0xF0] => vec![0xF0] }, + 0x0 => leaf { vec![0, 16] => vec![0x00] }, + 0x1 => leaf { vec![0, 16] => vec![0x10] }, + 0x2 => leaf { vec![0, 16] => vec![0x20] }, + 0x3 => leaf { vec![0, 16] => vec![0x30] }, + 0x4 => leaf { vec![0, 16] => vec![0x40] }, + 0x5 => leaf { vec![0, 16] => vec![0x50] }, + 0x6 => leaf { vec![0, 16] => vec![0x60] }, + 0x7 => leaf { vec![0, 16] => vec![0x70] }, + 0x8 => leaf { vec![0, 16] => vec![0x80] }, + 0x9 => leaf { vec![0, 16] => vec![0x90] }, + 0xA => leaf { vec![0, 16] => vec![0xA0] }, + 0xB => leaf { vec![0, 16] => vec![0xB0] }, + 0xC => leaf { vec![0, 16] => vec![0xC0] }, + 0xD => leaf { vec![0, 16] => vec![0xD0] }, + 0xE => leaf { vec![0, 16] => vec![0xE0] }, + 0xF => leaf { vec![0, 16] => vec![0xF0] }, } }; @@ -558,8 +556,8 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 2 => leaf { &[0x20] => vec![0x20] }, - 4 => leaf { &[0x40] => vec![0x40] }, + 2 => leaf { vec![0, 16] => vec![0x20] }, + 4 => leaf { vec![0, 16] => vec![0x40] }, } with_leaf { &[0x1] => vec![0x1] } }; @@ -577,22 +575,22 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) branch { - 0x0 => leaf { &[0x00] => vec![0x00] }, - 0x1 => leaf { &[0x10] => vec![0x10] }, - 0x2 => leaf { &[0x20] => vec![0x20] }, - 0x3 => leaf { &[0x30] => vec![0x30] }, - 0x4 => leaf { &[0x40] => vec![0x40] }, - 0x5 => leaf { &[0x50] => vec![0x50] }, - 0x6 => leaf { &[0x60] => vec![0x60] }, - 0x7 => leaf { &[0x70] => vec![0x70] }, - 0x8 => leaf { &[0x80] => vec![0x80] }, - 0x9 => leaf { &[0x90] => vec![0x90] }, - 0xA => leaf { &[0xA0] => vec![0xA0] }, - 0xB => leaf { &[0xB0] => vec![0xB0] }, - 0xC => leaf { &[0xC0] => vec![0xC0] }, - 0xD => leaf { &[0xD0] => vec![0xD0] }, - 0xE => leaf { &[0xE0] => vec![0xE0] }, - 0xF => leaf { &[0xF0] => vec![0xF0] }, + 0x0 => leaf { vec![0, 16] => vec![0x00] }, + 0x1 => leaf { vec![0, 16] => vec![0x10] }, + 0x2 => leaf { vec![0, 16] => vec![0x20] }, + 0x3 => leaf { vec![0, 16] => vec![0x30] }, + 0x4 => leaf { vec![0, 16] => vec![0x40] }, + 0x5 => leaf { vec![0, 16] => vec![0x50] }, + 0x6 => leaf { vec![0, 16] => vec![0x60] }, + 0x7 => leaf { vec![0, 16] => vec![0x70] }, + 0x8 => leaf { vec![0, 16] => vec![0x80] }, + 0x9 => leaf { vec![0, 16] => vec![0x90] }, + 0xA => leaf { vec![0, 16] => vec![0xA0] }, + 0xB => leaf { vec![0, 16] => vec![0xB0] }, + 0xC => leaf { vec![0, 16] => vec![0xC0] }, + 0xD => leaf { vec![0, 16] => vec![0xD0] }, + 0xE => leaf { vec![0, 16] => vec![0xE0] }, + 0xF => leaf { vec![0, 16] => vec![0xF0] }, } with_leaf { &[0x1] => vec![0x1] } }; diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 59ddbea48..cf6345097 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -146,7 +146,7 @@ mod test { #[test] fn get_some() { let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![0, 12, 16] => vec![0x12, 0x34, 0x56, 0x78] } }; assert_eq!( @@ -158,7 +158,7 @@ mod test { #[test] fn get_none() { let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1,2,16] => vec![0x12, 0x34, 0x56, 0x78] } }; assert!(node.get(Nibbles::from_bytes(&[0x34])).unwrap().is_none()); @@ -168,7 +168,7 @@ mod test { fn insert_replace() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1,2,16] => vec![0x12, 0x34, 0x56, 0x78] } }; let node = node @@ -186,7 +186,7 @@ mod test { fn insert_branch() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1,2,16] => vec![0x12, 0x34, 0x56, 0x78] } }; let path = Nibbles::from_bytes(&[0x22]); let value = vec![0x23]; @@ -204,7 +204,7 @@ mod test { fn insert_extension_branch() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1,2,16] => vec![0x12, 0x34, 0x56, 0x78] } }; let path = Nibbles::from_bytes(&[0x13]); @@ -222,7 +222,7 @@ mod test { fn insert_extension_branch_value_self() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { &[0x12] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1,2,16] => vec![0x12, 0x34, 0x56, 0x78] } }; let path = Nibbles::from_bytes(&[0x12, 0x34]); @@ -240,7 +240,7 @@ mod test { fn insert_extension_branch_value_other() { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) - leaf { &[0x12, 0x34] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1, 2, 3, 4, 16] => vec![0x12, 0x34, 0x56, 0x78] } }; let path = Nibbles::from_bytes(&[0x12]); diff --git a/crates/storage/trie/test_utils.rs b/crates/storage/trie/test_utils.rs index ceefe3cdb..b49ec2588 100644 --- a/crates/storage/trie/test_utils.rs +++ b/crates/storage/trie/test_utils.rs @@ -32,7 +32,8 @@ pub mod libmdbx { } #[macro_export] -/// Creates a trie node, doesn't guarantee that the correct offsets are used when computing hashes for extension nodes +/// Creates a trie node +/// All partial paths are expressed in nibbles and values in bytes macro_rules! pmt_node { ( @( $trie:expr ) @@ -99,7 +100,7 @@ macro_rules! pmt_node { $( offset $offset:expr )? ) => { { - $crate::node::LeafNode::new(Nibbles::from_bytes($path), $value) + $crate::node::LeafNode::new(Nibbles::from_hex($path), $value) } }; } From 237f2919d36bb53a8f04955f66a18a511a945c22 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 17:43:23 -0300 Subject: [PATCH 145/345] Fix potential panick --- crates/storage/trie/nibbles.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index fd3d84862..34c9757f7 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -61,7 +61,7 @@ impl Nibbles { /// Removes and returns the first nibble pub fn next(&mut self) -> Option { - (!self.is_empty()).then_some(self.data.remove(0)) + (!self.is_empty()).then(|| self.data.remove(0)) } /// Removes and returns the first nibble if it is a suitable choice index (aka < 16) From d5dfa30dd26e9b6015e694337228913c083ee654 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:19:04 -0300 Subject: [PATCH 146/345] Fix --- crates/storage/trie/node/branch.rs | 2 +- crates/storage/trie/node/extension.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 6f047ebb1..5e204c4b1 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -174,7 +174,7 @@ impl BranchNode { .collect::>(); let new_node = match (children.len(), !self.value.is_empty()) { // If this node still has a value but no longer has children, convert it into a leaf node - (0, true) => LeafNode::new(Nibbles::from_hex(vec![]), self.value).into(), + (0, true) => LeafNode::new(Nibbles::from_hex(vec![16]), self.value).into(), // If this node doesn't have a value and has only one child, replace it with its child node (1, false) => { let (choice_index, child_hash) = children[0]; diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 269d6d85c..14a2c0d02 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -45,7 +45,6 @@ impl ExtensionNode { path: Nibbles, value: ValueRLP, ) -> Result { - // OUTDATED /* Possible flow paths: * Prefix fully matches path Extension { prefix, child } -> Extension { prefix , child' } (insert into child) From 55b0c37e191f1000ed79f035fa9d832bf58515f0 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:23:24 -0300 Subject: [PATCH 147/345] Fix unit tests --- crates/storage/trie/node/extension.rs | 105 ++++++++++++-------------- crates/storage/trie/node/leaf.rs | 2 +- 2 files changed, 50 insertions(+), 57 deletions(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 14a2c0d02..d8c78af2e 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -215,19 +215,17 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0x00])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x00])).unwrap(), Some(vec![0x12, 0x34, 0x56, 0x78]), ); assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0x01])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x01])).unwrap(), Some(vec![0x34, 0x56, 0x78, 0x9A]), ); } @@ -237,14 +235,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0x02])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x02])).unwrap(), None, ); } @@ -254,13 +251,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, Nibbles::from_hex(vec![0x02]), vec![]) + .insert(&mut trie.state, Nibbles::from_bytes(&[0x02]), vec![]) .unwrap(); let node = match node { Node::Extension(x) => x, @@ -274,21 +271,24 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x01] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, Nibbles::from_hex(vec![0x10]), vec![0x20]) + .insert( + &mut trie.state, + dbg!(Nibbles::from_bytes(&[0x10])), + vec![0x20], + ) .unwrap(); - let node = match node { + let node = match dbg!(node) { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0x10])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x10])).unwrap(), Some(vec![0x20]) ); } @@ -298,21 +298,20 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16]=> vec![0x34, 0x56, 0x78, 0x9A] }, } } }; let node = node - .insert(&mut trie.state, Nibbles::from_hex(vec![0x10]), vec![0x20]) + .insert(&mut trie.state, Nibbles::from_bytes(&[0x10]), vec![0x20]) .unwrap(); let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; assert_eq!( - node.get(&trie.state, Nibbles::from_hex(vec![0x10])) - .unwrap(), + node.get(&trie.state, Nibbles::from_bytes(&[0x10])).unwrap(), Some(vec![0x20]) ); } @@ -322,12 +321,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; - let path = Nibbles::from_hex(vec![0x01]); + let path = Nibbles::from_bytes(&[0x01]); let value = vec![0x02]; let node = node @@ -343,12 +342,12 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0, 0], branch { - 0 => leaf { &[0x00, 0x00] => vec![0x12, 0x34, 0x56, 0x78] }, - 1 => leaf { &[0x00, 0x10] => vec![0x34, 0x56, 0x78, 0x9A] }, + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } }; - let path = Nibbles::from_hex(vec![0x01]); + let path = Nibbles::from_bytes(&[0x01]); let value = vec![0x04]; let node = node @@ -364,13 +363,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x00] }, - 1 => leaf { &[0x01] => vec![0x01] }, + 0 => leaf { vec![16] => vec![0x00] }, + 1 => leaf { vec![16] => vec![0x01] }, } } }; let (node, value) = node - .remove(&mut trie.state, Nibbles::from_hex(vec![0x02])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x02])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -382,13 +381,13 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x00] }, - 1 => leaf { &[0x01] => vec![0x01] }, + 0 => leaf { vec![16] => vec![0x00] }, + 1 => leaf { vec![16] => vec![0x01] }, } } }; let (node, value) = node - .remove(&mut trie.state, Nibbles::from_hex(vec![0x01])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x01])) .unwrap(); assert!(matches!(node, Some(Node::Leaf(_)))); @@ -400,16 +399,16 @@ mod test { let mut trie = Trie::new_temp(); let node = pmt_node! { @(trie) extension { [0], branch { - 0 => leaf { &[0x00] => vec![0x00] }, + 0 => leaf { vec![16] => vec![0x00] }, 1 => extension { [0], branch { - 0 => leaf { &[0x01, 0x00] => vec![0x01, 0x00] }, - 1 => leaf { &[0x01, 0x01] => vec![0x01, 0x01] }, + 0 => leaf { vec![16] => vec![0x01, 0x00] }, + 1 => leaf { vec![16] => vec![0x01, 0x01] }, } }, } } }; let (node, value) = node - .remove(&mut trie.state, Nibbles::from_hex(vec![0x00])) + .remove(&mut trie.state, Nibbles::from_bytes(&[0x00])) .unwrap(); assert!(matches!(node, Some(Node::Extension(_)))); @@ -420,21 +419,15 @@ mod test { fn compute_hash() { /* Extension { - [0x00, 0x00] + [0, 0] Branch { [ - Leaf { [0x00, 0x00], [0x12, 0x34] } - Leaf { [0x00, 0x10], [0x56, 0x78] } + 0: Leaf { [0, 16], [0x12, 0x34] } + 1: Leaf { [0, 16], [0x56, 0x78] } } } */ - let leaf_node_a = LeafNode::new( - Nibbles::from_bytes(&[0x00, 0x00]).offset(3), - vec![0x12, 0x34], - ); - let leaf_node_b = LeafNode::new( - Nibbles::from_bytes(&[0x00, 0x10]).offset(3), - vec![0x56, 0x78], - ); + let leaf_node_a = LeafNode::new(Nibbles::from_hex(vec![0, 16]), vec![0x12, 0x34]); + let leaf_node_b = LeafNode::new(Nibbles::from_hex(vec![0, 16]), vec![0x56, 0x78]); let mut choices = BranchNode::EMPTY_CHOICES; choices[0] = leaf_node_a.compute_hash(); choices[1] = leaf_node_b.compute_hash(); @@ -455,19 +448,19 @@ mod test { fn compute_hash_long() { /* Extension { - [0x00, 0x00] + [0, 0] Branch { [ - Leaf { [0x00, 0x00], [0x12, 0x34, 0x56, 0x78, 0x9A] } - Leaf { [0x00, 0x10], [0x34, 0x56, 0x78, 0x9A, 0xBC] } + 0: Leaf { [0, 16], [0x12, 0x34, 0x56, 0x78, 0x9A] } + 1: Leaf { [0, 16], [0x34, 0x56, 0x78, 0x9A, 0xBC] } } } */ let leaf_node_a = LeafNode::new( - Nibbles::from_bytes(&[0x00, 0x00]), + Nibbles::from_hex(vec![0, 16]), vec![0x12, 0x34, 0x56, 0x78, 0x9A], ); let leaf_node_b = LeafNode::new( - Nibbles::from_bytes(&[0x00, 0x10]), + Nibbles::from_hex(vec![0, 16]), vec![0x34, 0x56, 0x78, 0x9A, 0xBC], ); let mut choices = BranchNode::EMPTY_CHOICES; diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index cf6345097..72d01bc34 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -146,7 +146,7 @@ mod test { #[test] fn get_some() { let node = pmt_node! { @(trie) - leaf { vec![0, 12, 16] => vec![0x12, 0x34, 0x56, 0x78] } + leaf { vec![1, 2, 16] => vec![0x12, 0x34, 0x56, 0x78] } }; assert_eq!( From 01090b4921a2c162e8bd306c6ae5d90107836e48 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:24:04 -0300 Subject: [PATCH 148/345] Remove outdated comment --- crates/storage/trie/trie.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index c415a8343..f869f384d 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -44,7 +44,6 @@ pub type PathRLP = Vec; pub type ValueRLP = Vec; /// Libmdx-based Ethereum Compatible Merkle Patricia Trie -/// Adapted from https://github.com/lambdaclass/merkle_patricia_tree pub struct Trie { /// Hash of the current node root: Option, From e1be0c65de0811950973886c1bf94e3e876cf0d3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:26:12 -0300 Subject: [PATCH 149/345] [DEBUG] Remove debug prints --- crates/storage/trie/trie.rs | 11 ++--------- crates/storage/trie/trie_iter.rs | 26 -------------------------- 2 files changed, 2 insertions(+), 35 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index f869f384d..fdbde2da7 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -16,7 +16,6 @@ use nibbles::Nibbles; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; -use trie_iter::print_trie; #[cfg(feature = "libmdbx")] pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; @@ -71,7 +70,6 @@ impl Trie { /// Retrieve an RLP-encoded value from the trie given its RLP-encoded path. pub fn get(&self, path: &PathRLP) -> Result, TrieError> { - println!("[GET] {:?}", Nibbles::from_bytes(&path).as_ref()); if let Some(root) = &self.root { let root_node = self .state @@ -85,7 +83,6 @@ impl Trie { /// Insert an RLP-encoded value into the trie. pub fn insert(&mut self, path: PathRLP, value: ValueRLP) -> Result<(), TrieError> { - println!("[INSERT] {:?}", Nibbles::from_bytes(&path).as_ref()); let root = self.root.take(); if let Some(root_node) = root .map(|root| self.state.get_node(root)) @@ -101,16 +98,14 @@ impl Trie { let new_leaf = Node::from(LeafNode::new(Nibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } - print_trie(&self); Ok(()) } /// Remove a value from the trie given its RLP-encoded path. /// Returns the value if it was succesfully removed or None if it wasn't part of the trie pub fn remove(&mut self, path: PathRLP) -> Result, TrieError> { - println!("[REMOVE] {:?}", Nibbles::from_bytes(&path).as_ref()); let root = self.root.take(); - let res = if let Some(root) = root { + if let Some(root) = root { let root_node = self .state .get_node(root)? @@ -123,9 +118,7 @@ impl Trie { Ok(old_value) } else { Ok(None) - }; - print_trie(&self); - res + } } /// Return the hash of the trie's root node. diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index ca4a640ac..9c0f127b6 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -67,32 +67,6 @@ impl TrieIterator { } } -pub fn print_trie(trie: &Trie) { - let Some(root) = &trie.root else { return }; - print_node(trie, root.clone()); - print!("\n") -} - -pub fn print_node(trie: &Trie, node_hash: NodeHash) { - match trie.state.get_node(node_hash).unwrap().unwrap() { - Node::Branch(n) => { - print!("Branch{:?} [", n.value); - for (i, child) in n.choices.iter().enumerate() { - if child.is_valid() { - print!(" {i}: "); - print_node(trie, child.clone()); - } - } - print!(" ]") - } - Node::Extension(n) => { - print!("Ext{:?} -> ", n.prefix.as_ref()); - print_node(trie, n.child); - } - Node::Leaf(n) => print!("Leaf{:?}{:?}", n.partial.as_ref(), n.value), - } -} - #[cfg(test)] mod tests { From bed25f707eda0444bbfec022ca07a62fcca920ea Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:28:41 -0300 Subject: [PATCH 150/345] Remove funny name test --- crates/storage/trie/trie.rs | 29 ----------------------------- 1 file changed, 29 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index fdbde2da7..5d6812d5b 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -993,33 +993,4 @@ mod test { let trie_proof = trie.get_proof(&a).unwrap(); assert_eq!(cita_proof, trie_proof); } - - #[test] - fn jijo() { - let mut trie = Trie::new_temp(); - let mut data = vec![(vec![0, 0, 0, 0, 0], false), (vec![0, 0, 0, 0, 1], true)]; - // Remove duplicate values with different expected status - data.sort_by_key(|(val, _)| val.clone()); - data.dedup_by_key(|(val, _)| val.clone()); - // Insertions - for (val, _) in data.iter() { - trie.insert(val.clone(), val.clone()).unwrap(); - } - // Removals - for (val, should_remove) in data.iter() { - if *should_remove { - let removed = trie.remove(val.clone()).unwrap(); - assert_eq!(removed, Some(val.clone())); - } - } - // Check trie values - for (val, removed) in data.iter() { - let item = trie.get(val).unwrap(); - if !removed { - assert_eq!(item, Some(val.clone())); - } else { - assert!(item.is_none()); - } - } - } } From f2c2eefff09fd5aa518624c7c3aa17867a34334e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:38:42 -0300 Subject: [PATCH 151/345] doc nibbles module --- crates/storage/trie/nibbles.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 34c9757f7..8036b1090 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -5,20 +5,24 @@ use ethereum_rust_rlp::{ structs::{Decoder, Encoder}, }; +/// Struct representing a list of nibbles (half-bytes) #[derive(Debug, Clone, Default, PartialEq)] pub struct Nibbles { data: Vec, } impl Nibbles { + /// Create `Nibbles` from hex-encoded nibbles pub fn from_hex(hex: Vec) -> Self { Self { data: hex } } + /// Splits incoming bytes into nibbles and appends the leaf flag (a 16 nibble at the end) pub fn from_bytes(bytes: &[u8]) -> Self { Self::from_raw(bytes, true) } + /// Splits incoming bytes into nibbles and appends the leaf flag (a 16 nibble at the end) if is_leaf is true pub fn from_raw(bytes: &[u8], is_leaf: bool) -> Self { let mut data: Vec = bytes .iter() @@ -31,10 +35,12 @@ impl Nibbles { Self { data } } + /// Returns the amount of nibbles pub fn len(&self) -> usize { self.data.len() } + /// Returns true if there are no nibbles pub fn is_empty(&self) -> bool { self.data.is_empty() } @@ -69,18 +75,23 @@ impl Nibbles { self.next().filter(|choice| *choice < 16).map(usize::from) } + /// Returns the nibbles after the given offset pub fn offset(&self, offset: usize) -> Nibbles { self.slice(offset, self.len()) } + /// Returns the nibbles beween the start and end indexes pub fn slice(&self, start: usize, end: usize) -> Nibbles { Nibbles::from_hex(self.data[start..end].to_vec()) } + /// Extends the nibbles with another list of nibbles pub fn extend(&mut self, other: &Nibbles) { self.data.extend_from_slice(other.as_ref()); } + + /// Return the nibble at the given index, will panic if the index is out of range pub fn at(&self, i: usize) -> usize { self.data[i] as usize } @@ -96,6 +107,7 @@ impl Nibbles { } /// Taken from https://github.com/citahub/cita_trie/blob/master/src/nibbles.rs#L56 + /// Encodes the nibbles in compact form pub fn encode_compact(&self) -> Vec { let mut compact = vec![]; let is_leaf = self.is_leaf(); @@ -126,6 +138,7 @@ impl Nibbles { compact } + /// Returns true if the nibbles contain the leaf flag (16) at the end pub fn is_leaf(&self) -> bool { if self.is_empty() { false @@ -134,6 +147,7 @@ impl Nibbles { } } + /// Combines the nibbles into bytes, trimming the leaf flag if necessary pub fn to_bytes(&self) -> Vec { // Trim leaf flag let data = if !self.is_empty() && self.is_leaf() { From 9050b0cb44f4e5990cba49fc0c40d4087224646d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:49:24 -0300 Subject: [PATCH 152/345] Remove todo --- crates/storage/trie/node/extension.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index d8c78af2e..0c608ff51 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -38,7 +38,6 @@ impl ExtensionNode { } /// Inserts a value into the subtrie originating from this node and returns the new root of the subtrie - /// TODO: Code changed a lot, check and rewrite doc pub fn insert( mut self, state: &mut TrieState, From 345ae961deef1b91b21f3d79e575117c3338cd45 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:50:47 -0300 Subject: [PATCH 153/345] remove debug print --- crates/storage/trie/nibbles.rs | 1 - crates/storage/trie/node/extension.rs | 8 ++------ 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 8036b1090..b36d3b1a5 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -90,7 +90,6 @@ impl Nibbles { self.data.extend_from_slice(other.as_ref()); } - /// Return the nibble at the given index, will panic if the index is out of range pub fn at(&self, i: usize) -> usize { self.data[i] as usize diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 0c608ff51..626c78b5c 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -276,13 +276,9 @@ mod test { }; let node = node - .insert( - &mut trie.state, - dbg!(Nibbles::from_bytes(&[0x10])), - vec![0x20], - ) + .insert(&mut trie.state, Nibbles::from_bytes(&[0x10]), vec![0x20]) .unwrap(); - let node = match dbg!(node) { + let node = match node { Node::Branch(x) => x, _ => panic!("expected a branch node"), }; From 0fa490f6e8b66b2cf02d3ecee973266f118ea294 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 7 Nov 2024 18:57:27 -0300 Subject: [PATCH 154/345] avoid clone --- crates/storage/trie/trie_iter.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 9c0f127b6..5b90ddd13 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -25,15 +25,14 @@ impl Iterator for TrieIterator { return None; }; // Fetch the last node in the stack - let (current_path, next_node_hash) = self.stack.pop()?; + let (mut path, next_node_hash) = self.stack.pop()?; let next_node = self.trie.state.get_node(next_node_hash).ok()??; - let mut next_path = current_path.clone(); match &next_node { Node::Branch(branch_node) => { // Add all children to the stack (in reverse order so we process first child frist) for (choice, child) in branch_node.choices.iter().enumerate().rev() { if child.is_valid() { - let mut child_path = current_path.clone(); + let mut child_path = path.clone(); child_path.append(choice as u8); self.stack.push((child_path, child.clone())) } @@ -41,16 +40,16 @@ impl Iterator for TrieIterator { } Node::Extension(extension_node) => { // Update path - next_path.extend(&extension_node.prefix); + path.extend(&extension_node.prefix); // Add child to the stack self.stack - .push((next_path.clone(), extension_node.child.clone())); + .push((path.clone(), extension_node.child.clone())); } Node::Leaf(leaf) => { - next_path.extend(&leaf.partial); + path.extend(&leaf.partial); } } - Some((next_path, next_node)) + Some((path, next_node)) } } From df62fabdcf214bf34c272120381678fbcdff7139 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 16:16:16 -0300 Subject: [PATCH 155/345] Decode leaf & extension nodes --- crates/common/rlp/decode.rs | 2 +- crates/common/rlp/structs.rs | 13 +++++ crates/storage/trie/nibbles.rs | 32 +++++++++++ crates/storage/trie/node/extension.rs | 78 ++++++++++++++++++++++++++- crates/storage/trie/node/leaf.rs | 46 +++++++++++++++- 5 files changed, 166 insertions(+), 5 deletions(-) diff --git a/crates/common/rlp/decode.rs b/crates/common/rlp/decode.rs index b5061c3e6..0760b2ee0 100644 --- a/crates/common/rlp/decode.rs +++ b/crates/common/rlp/decode.rs @@ -472,7 +472,7 @@ pub fn get_rlp_bytes_item_payload(rlp: &[u8]) -> &[u8] { /// It returns a 2-element tuple with the following elements: /// - The payload of the item. /// - The remaining bytes after the item. -fn decode_bytes(data: &[u8]) -> Result<(&[u8], &[u8]), RLPDecodeError> { +pub fn decode_bytes(data: &[u8]) -> Result<(&[u8], &[u8]), RLPDecodeError> { let (is_list, payload, rest) = decode_rlp_item(data)?; if is_list { return Err(RLPDecodeError::UnexpectedList); diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index bfb1016ea..26abd9199 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -1,3 +1,5 @@ +use crate::decode::decode_bytes; + use super::{ decode::{decode_rlp_item, get_item_with_prefix, RLPDecode}, encode::{encode_length, RLPEncode}, @@ -108,6 +110,17 @@ impl<'a> Decoder<'a> { pub fn finish_unchecked(self) -> &'a [u8] { self.remaining } + + /// Decodes the next field as bytes + pub fn decode_bytes(self, name: &'a str) -> Result<(&[u8], Self), RLPDecodeError> { + let (field, rest) = + decode_bytes(&self.payload).map_err(|err| field_decode_error::<&[u8]>(name, err))?; + let updated_self = Self { + payload: rest, + ..self + }; + Ok((field, updated_self)) + } } fn field_decode_error(field_name: &str, err: RLPDecodeError) -> RLPDecodeError { diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index b36d3b1a5..8c7f4b33b 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -137,6 +137,11 @@ impl Nibbles { compact } + /// Encodes the nibbles in compact form + pub fn decode_compact(compact: &[u8]) -> Self { + Self::from_hex(compact_to_hex(compact)) + } + /// Returns true if the nibbles contain the leaf flag (16) at the end pub fn is_leaf(&self) -> bool { if self.is_empty() { @@ -184,6 +189,33 @@ impl RLPDecode for Nibbles { } } +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L82 +fn compact_to_hex(compact: &[u8]) -> Vec { + if compact.is_empty() { + return vec![]; + } + let mut base = keybytes_to_hex(compact); + // delete terminator flag + if base[0] < 2 { + base = base[..base.len() - 1].to_vec(); + } + // apply odd flag + let chop = 2 - (base[0] & 1) as usize; + base[chop..].to_vec() +} + +// Code taken from https://github.com/ethereum/go-ethereum/blob/a1093d98eb3260f2abf340903c2d968b2b891c11/trie/encoding.go#L96 +fn keybytes_to_hex(keybytes: &[u8]) -> Vec { + let l = keybytes.len() * 2 + 1; + let mut nibbles = vec![0; l]; + for (i, b) in keybytes.iter().enumerate() { + nibbles[i * 2] = b / 16; + nibbles[i * 2 + 1] = b % 16; + } + nibbles[l - 1] = 16; + nibbles +} + #[cfg(test)] mod test { use super::*; diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 626c78b5c..b88e6b3f2 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,4 +1,9 @@ -use ethereum_rust_rlp::structs::Encoder; +use ethereum_rust_rlp::{ + decode::decode_bytes, + error::RLPDecodeError, + structs::{Decoder, Encoder}, +}; +use ethereum_types::H256; use crate::error::TrieError; use crate::nibbles::Nibbles; @@ -10,7 +15,7 @@ use super::{BranchNode, Node}; /// Extension Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's prefix and a its child node hash, doesn't store any value -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct ExtensionNode { pub prefix: Nibbles, pub child: NodeHash, @@ -164,6 +169,22 @@ impl ExtensionNode { buf } + /// Decodes the node + pub fn decode_raw(rlp: &[u8]) -> Result { + let decoder = Decoder::new(rlp)?; + let (prefix, decoder) = decoder.decode_bytes("prefix")?; + let (child, decoder) = decoder.get_encoded_item()?; + let child = match decode_bytes(&child) { + Ok((hash, &[])) if hash.len() == 32 => NodeHash::Hashed(H256::from_slice(hash)), + _ => NodeHash::Inline(child), + }; + decoder.finish()?; + Ok(Self { + prefix: Nibbles::decode_compact(prefix), + child, + }) + } + /// Inserts the node into the state and returns its hash pub fn insert_self(self, state: &mut TrieState) -> Result { let hash = self.compute_hash(); @@ -473,4 +494,57 @@ mod test { ], ); } + + #[test] + fn symetric_encoding_a() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + extension { [0], branch { + 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, + } } + }; + assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_b() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + extension { [0], branch { + 0 => leaf { vec![16] => vec![0x00] }, + 1 => extension { [0], branch { + 0 => leaf { vec![16] => vec![0x01, 0x00] }, + 1 => leaf { vec![16] => vec![0x01, 0x01] }, + } }, + } } + }; + + assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_c() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + extension { [0], branch { + 0 => leaf { vec![16] => vec![0x00] }, + 1 => extension { [0], branch { + 0 => leaf { vec![16] => vec![0x01, 0x00] }, + 1 => leaf { vec![16] => vec![0x01, 0x01] }, + 2 => leaf { vec![16] => vec![0x01, 0x00] }, + 3 => leaf { vec![16] => vec![0x03, 0x01] }, + 4 => leaf { vec![16] => vec![0x04, 0x00] }, + 5 => leaf { vec![16] => vec![0x05, 0x01] }, + 6 => leaf { vec![16] => vec![0x06, 0x00] }, + 7 => leaf { vec![16] => vec![0x07, 0x01] }, + 8 => leaf { vec![16] => vec![0x08, 0x00] }, + 9 => leaf { vec![16] => vec![0x09, 0x01] }, + 10 => leaf { vec![16] => vec![0x10, 0x00] }, + 11 => leaf { vec![16] => vec![0x11, 0x01] }, + } }, + } } + }; + assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + } } diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index 72d01bc34..fcc841642 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,4 +1,7 @@ -use ethereum_rust_rlp::structs::Encoder; +use ethereum_rust_rlp::{ + error::RLPDecodeError, + structs::{Decoder, Encoder}, +}; use crate::{ error::TrieError, nibbles::Nibbles, node::BranchNode, node_hash::NodeHash, state::TrieState, @@ -8,7 +11,7 @@ use crate::{ use super::{ExtensionNode, Node}; /// Leaf Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's hash, value & path -#[derive(Debug, Clone, Default)] +#[derive(Debug, Clone, Default, PartialEq)] pub struct LeafNode { pub partial: Nibbles, pub value: ValueRLP, @@ -114,6 +117,18 @@ impl LeafNode { buf } + /// Decodes the node + pub fn decode_raw(rlp: &[u8]) -> Result { + let decoder = Decoder::new(rlp)?; + let (partial, decoder) = decoder.decode_bytes("partial")?; + let (value, decoder) = decoder.decode_bytes("value")?; + decoder.finish()?; + Ok(Self { + partial: Nibbles::decode_compact(partial), + value: value.to_vec(), + }) + } + /// Inserts the node into the state and returns its hash /// Receives the offset that needs to be traversed to reach the leaf node from the canonical root, used to compute the node hash pub fn insert_self(self, state: &mut TrieState) -> Result { @@ -314,4 +329,31 @@ mod test { ], ); } + + #[test] + fn symetric_encoding_a() { + let node = LeafNode::new( + Nibbles::from_bytes(b"key".as_ref()), + b"a comparatively long value".to_vec(), + ); + assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_b() { + let node = LeafNode::new( + Nibbles::from_bytes(&[0x12, 0x34]), + vec![0x12, 0x34, 0x56, 0x78], + ); + assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_c() { + let node = LeafNode::new( + Nibbles::from_bytes(&[]), + vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 20], + ); + assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + } } From de3695f52fe6e9ffc419ed38c442e1fa7d4a8784 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 16:27:31 -0300 Subject: [PATCH 156/345] Decode branch nodesc --- crates/storage/trie/node/branch.rs | 89 +++++++++++++++++++++++++++++- 1 file changed, 87 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 5e204c4b1..858d85d9d 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -1,4 +1,9 @@ -use ethereum_rust_rlp::structs::Encoder; +use ethereum_rust_rlp::{ + decode::decode_bytes, + error::RLPDecodeError, + structs::{Decoder, Encoder}, +}; +use ethereum_types::H256; use crate::{error::TrieError, nibbles::Nibbles, node_hash::NodeHash, state::TrieState, ValueRLP}; @@ -6,7 +11,7 @@ use super::{ExtensionNode, LeafNode, Node}; /// Branch Node of an an Ethereum Compatible Patricia Merkle Trie /// Contains the node's value and the hash of its children nodes -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct BranchNode { // TODO: check if switching to hashmap is a better solution pub choices: Box<[NodeHash; 16]>, @@ -226,6 +231,32 @@ impl BranchNode { buf } + /// Decodes the node + pub fn decode_raw(rlp: &[u8]) -> Result { + let mut decoder = Decoder::new(rlp)?; + let mut choices = BranchNode::EMPTY_CHOICES; + let mut child; + for i in 0..16 { + (child, decoder) = decoder.get_encoded_item()?; + match decode_bytes(&child) { + // hashed child + Ok((hash, &[])) if hash.len() == 32 => { + choices[i] = NodeHash::Hashed(H256::from_slice(hash)) + } + // no child + Ok((&[], &[])) => {} + // inlined child + _ => choices[i] = NodeHash::Inline(child), + } + } + let (value, decoder) = decoder.decode_bytes("value")?; + decoder.finish()?; + Ok(Self { + choices: Box::new(choices), + value: value.to_vec(), + }) + } + /// Inserts the node into the state and returns its hash pub fn insert_self(self, state: &mut TrieState) -> Result { let hash = self.compute_hash(); @@ -603,4 +634,58 @@ mod test { ], ); } + + #[test] + fn symetric_encoding_a() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + branch { + 0 => leaf { vec![0,16] => vec![0x12, 0x34, 0x56, 0x78] }, + 1 => leaf { vec![0,16] => vec![0x34, 0x56, 0x78, 0x9A] }, + } + }; + assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_b() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + branch { + 0 => leaf { vec![0, 16] => vec![0x00] }, + 1 => leaf { vec![0, 16] => vec![0x10] }, + 3 => extension { [0], branch { + 0 => leaf { vec![16] => vec![0x01, 0x00] }, + 1 => leaf { vec![16] => vec![0x01, 0x01] }, + } }, + } + }; + assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + + #[test] + fn symetric_encoding_c() { + let mut trie = Trie::new_temp(); + let node = pmt_node! { @(trie) + branch { + 0x0 => leaf { vec![0, 16] => vec![0x00] }, + 0x1 => leaf { vec![0, 16] => vec![0x10] }, + 0x2 => leaf { vec![0, 16] => vec![0x20] }, + 0x3 => leaf { vec![0, 16] => vec![0x30] }, + 0x4 => leaf { vec![0, 16] => vec![0x40] }, + 0x5 => leaf { vec![0, 16] => vec![0x50] }, + 0x6 => leaf { vec![0, 16] => vec![0x60] }, + 0x7 => leaf { vec![0, 16] => vec![0x70] }, + 0x8 => leaf { vec![0, 16] => vec![0x80] }, + 0x9 => leaf { vec![0, 16] => vec![0x90] }, + 0xA => leaf { vec![0, 16] => vec![0xA0] }, + 0xB => leaf { vec![0, 16] => vec![0xB0] }, + 0xC => leaf { vec![0, 16] => vec![0xC0] }, + 0xD => leaf { vec![0, 16] => vec![0xD0] }, + 0xE => leaf { vec![0, 16] => vec![0xE0] }, + 0xF => leaf { vec![0, 16] => vec![0xF0] }, + } with_leaf { &[0x1] => vec![0x1] } + }; + assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } } From cb92b60b53a0f420ab5cf4eae037db4a570d9ced Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 17:53:07 -0300 Subject: [PATCH 157/345] Impl decode_raw for Node directly --- crates/common/rlp/structs.rs | 5 ++++ crates/storage/trie/node.rs | 49 ++++++++++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index 26abd9199..2581561f9 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -105,6 +105,11 @@ impl<'a> Decoder<'a> { } } + /// Returns true if the decoder has finished decoding the given input + pub fn is_done(&self) -> bool { + self.payload.is_empty() + } + /// Same as [`finish`](Self::finish), but discards the item's remaining payload /// instead of failing. pub fn finish_unchecked(self) -> &'a [u8] { diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 71c0a0892..35ceab9a1 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -2,7 +2,11 @@ mod branch; mod extension; mod leaf; +use std::array; + pub use branch::BranchNode; +use ethereum_rust_rlp::{decode::decode_bytes, error::RLPDecodeError, structs::Decoder}; +use ethereum_types::H256; pub use extension::ExtensionNode; pub use leaf::LeafNode; @@ -97,4 +101,49 @@ impl Node { Node::Leaf(n) => n.insert_self(state), } } + + pub fn decode_raw(rlp: &[u8]) -> Result { + let mut rlp_items = vec![]; + let mut decoder = Decoder::new(rlp)?; + let mut item; + // Get encoded fields + loop { + (item, decoder) = decoder.get_encoded_item()?; + rlp_items.push(item); + // Check if we reached the end or if we decoded more items than the ones we need + if decoder.is_done() || rlp_items.len() > 17 { + break + } + } + // Deserialize into node depending on the available fields + Ok(match rlp_items.len() { + // Leaf or Extension Node + 2 => { + let (path, _) = decode_bytes(&rlp_items[0])?; + let path = Nibbles::decode_compact(&path); + if path.is_leaf() { + // Decode as Leaf + let (value, _) = decode_bytes(&rlp_items[1])?; + LeafNode { partial: path, value: value.to_vec()}.into() + } else { + // Decode as Extension + ExtensionNode { prefix: path, child: decode_child(&rlp_items[1])}.into() + } + } + // Branch Node + 17 => { + let choices = array::from_fn(|i| decode_child(&rlp_items[i])); + let (value, _) = decode_bytes(&rlp_items[16])?; + BranchNode { choices: Box::new(choices), value: value.to_vec()}.into() + } + n => return Err(RLPDecodeError::Custom(format!("Invalid arg count for Node, expected 2 or 17, got {n}"))) + }) + } +} + +fn decode_child(rlp: &[u8]) -> NodeHash { + match decode_bytes(&rlp) { + Ok((hash, &[])) if hash.len() == 32 => NodeHash::Hashed(H256::from_slice(hash)), + _ => NodeHash::Inline(rlp.to_vec()), + } } From 87994d3ff101ecda7d048062899083b219fc62b2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 18:04:03 -0300 Subject: [PATCH 158/345] Update test & remove impls for each node kind --- crates/storage/trie/node.rs | 37 +++++++++++++++--- crates/storage/trie/node/branch.rs | 54 +++++++-------------------- crates/storage/trie/node/extension.rs | 44 +++++++--------------- crates/storage/trie/node/leaf.rs | 38 +++++++------------ 4 files changed, 70 insertions(+), 103 deletions(-) diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 35ceab9a1..d450b90b0 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -15,7 +15,7 @@ use crate::{error::TrieError, nibbles::Nibbles}; use super::{node_hash::NodeHash, state::TrieState, ValueRLP}; /// A Node in an Ethereum Compatible Patricia Merkle Trie -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub enum Node { Branch(BranchNode), Extension(ExtensionNode), @@ -102,6 +102,14 @@ impl Node { } } + pub fn encode_raw(&self) -> Vec { + match self { + Node::Branch(n) => n.encode_raw(), + Node::Extension(n) => n.encode_raw(), + Node::Leaf(n) => n.encode_raw(), + } + } + pub fn decode_raw(rlp: &[u8]) -> Result { let mut rlp_items = vec![]; let mut decoder = Decoder::new(rlp)?; @@ -112,7 +120,7 @@ impl Node { rlp_items.push(item); // Check if we reached the end or if we decoded more items than the ones we need if decoder.is_done() || rlp_items.len() > 17 { - break + break; } } // Deserialize into node depending on the available fields @@ -124,19 +132,35 @@ impl Node { if path.is_leaf() { // Decode as Leaf let (value, _) = decode_bytes(&rlp_items[1])?; - LeafNode { partial: path, value: value.to_vec()}.into() + LeafNode { + partial: path, + value: value.to_vec(), + } + .into() } else { // Decode as Extension - ExtensionNode { prefix: path, child: decode_child(&rlp_items[1])}.into() + ExtensionNode { + prefix: path, + child: decode_child(&rlp_items[1]), + } + .into() } } // Branch Node 17 => { let choices = array::from_fn(|i| decode_child(&rlp_items[i])); let (value, _) = decode_bytes(&rlp_items[16])?; - BranchNode { choices: Box::new(choices), value: value.to_vec()}.into() + BranchNode { + choices: Box::new(choices), + value: value.to_vec(), + } + .into() + } + n => { + return Err(RLPDecodeError::Custom(format!( + "Invalid arg count for Node, expected 2 or 17, got {n}" + ))) } - n => return Err(RLPDecodeError::Custom(format!("Invalid arg count for Node, expected 2 or 17, got {n}"))) }) } } @@ -144,6 +168,7 @@ impl Node { fn decode_child(rlp: &[u8]) -> NodeHash { match decode_bytes(&rlp) { Ok((hash, &[])) if hash.len() == 32 => NodeHash::Hashed(H256::from_slice(hash)), + Ok((hash, &[])) if hash.is_empty() => NodeHash::Inline(vec![]), _ => NodeHash::Inline(rlp.to_vec()), } } diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 858d85d9d..cfa36f2ce 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -1,9 +1,4 @@ -use ethereum_rust_rlp::{ - decode::decode_bytes, - error::RLPDecodeError, - structs::{Decoder, Encoder}, -}; -use ethereum_types::H256; +use ethereum_rust_rlp::structs::Encoder; use crate::{error::TrieError, nibbles::Nibbles, node_hash::NodeHash, state::TrieState, ValueRLP}; @@ -231,32 +226,6 @@ impl BranchNode { buf } - /// Decodes the node - pub fn decode_raw(rlp: &[u8]) -> Result { - let mut decoder = Decoder::new(rlp)?; - let mut choices = BranchNode::EMPTY_CHOICES; - let mut child; - for i in 0..16 { - (child, decoder) = decoder.get_encoded_item()?; - match decode_bytes(&child) { - // hashed child - Ok((hash, &[])) if hash.len() == 32 => { - choices[i] = NodeHash::Hashed(H256::from_slice(hash)) - } - // no child - Ok((&[], &[])) => {} - // inlined child - _ => choices[i] = NodeHash::Inline(child), - } - } - let (value, decoder) = decoder.decode_bytes("value")?; - decoder.finish()?; - Ok(Self { - choices: Box::new(choices), - value: value.to_vec(), - }) - } - /// Inserts the node into the state and returns its hash pub fn insert_self(self, state: &mut TrieState) -> Result { let hash = self.compute_hash(); @@ -638,19 +607,20 @@ mod test { #[test] fn symetric_encoding_a() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) branch { 0 => leaf { vec![0,16] => vec![0x12, 0x34, 0x56, 0x78] }, 1 => leaf { vec![0,16] => vec![0x34, 0x56, 0x78, 0x9A] }, } - }; - assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_b() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) branch { 0 => leaf { vec![0, 16] => vec![0x00] }, 1 => leaf { vec![0, 16] => vec![0x10] }, @@ -659,14 +629,15 @@ mod test { 1 => leaf { vec![16] => vec![0x01, 0x01] }, } }, } - }; - assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_c() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) branch { 0x0 => leaf { vec![0, 16] => vec![0x00] }, 0x1 => leaf { vec![0, 16] => vec![0x10] }, @@ -685,7 +656,8 @@ mod test { 0xE => leaf { vec![0, 16] => vec![0xE0] }, 0xF => leaf { vec![0, 16] => vec![0xF0] }, } with_leaf { &[0x1] => vec![0x1] } - }; - assert_eq!(BranchNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } } diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index b88e6b3f2..89f49e626 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -1,9 +1,4 @@ -use ethereum_rust_rlp::{ - decode::decode_bytes, - error::RLPDecodeError, - structs::{Decoder, Encoder}, -}; -use ethereum_types::H256; +use ethereum_rust_rlp::structs::Encoder; use crate::error::TrieError; use crate::nibbles::Nibbles; @@ -169,22 +164,6 @@ impl ExtensionNode { buf } - /// Decodes the node - pub fn decode_raw(rlp: &[u8]) -> Result { - let decoder = Decoder::new(rlp)?; - let (prefix, decoder) = decoder.decode_bytes("prefix")?; - let (child, decoder) = decoder.get_encoded_item()?; - let child = match decode_bytes(&child) { - Ok((hash, &[])) if hash.len() == 32 => NodeHash::Hashed(H256::from_slice(hash)), - _ => NodeHash::Inline(child), - }; - decoder.finish()?; - Ok(Self { - prefix: Nibbles::decode_compact(prefix), - child, - }) - } - /// Inserts the node into the state and returns its hash pub fn insert_self(self, state: &mut TrieState) -> Result { let hash = self.compute_hash(); @@ -498,19 +477,20 @@ mod test { #[test] fn symetric_encoding_a() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) extension { [0], branch { 0 => leaf { vec![16] => vec![0x12, 0x34, 0x56, 0x78] }, 1 => leaf { vec![16] => vec![0x34, 0x56, 0x78, 0x9A] }, } } - }; - assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_b() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) extension { [0], branch { 0 => leaf { vec![16] => vec![0x00] }, 1 => extension { [0], branch { @@ -518,15 +498,16 @@ mod test { 1 => leaf { vec![16] => vec![0x01, 0x01] }, } }, } } - }; + } + .into(); - assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_c() { let mut trie = Trie::new_temp(); - let node = pmt_node! { @(trie) + let node: Node = pmt_node! { @(trie) extension { [0], branch { 0 => leaf { vec![16] => vec![0x00] }, 1 => extension { [0], branch { @@ -544,7 +525,8 @@ mod test { 11 => leaf { vec![16] => vec![0x11, 0x01] }, } }, } } - }; - assert_eq!(ExtensionNode::decode_raw(&node.encode_raw()).unwrap(), node) + } + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } } diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index fcc841642..e596f8572 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -1,7 +1,4 @@ -use ethereum_rust_rlp::{ - error::RLPDecodeError, - structs::{Decoder, Encoder}, -}; +use ethereum_rust_rlp::structs::Encoder; use crate::{ error::TrieError, nibbles::Nibbles, node::BranchNode, node_hash::NodeHash, state::TrieState, @@ -117,18 +114,6 @@ impl LeafNode { buf } - /// Decodes the node - pub fn decode_raw(rlp: &[u8]) -> Result { - let decoder = Decoder::new(rlp)?; - let (partial, decoder) = decoder.decode_bytes("partial")?; - let (value, decoder) = decoder.decode_bytes("value")?; - decoder.finish()?; - Ok(Self { - partial: Nibbles::decode_compact(partial), - value: value.to_vec(), - }) - } - /// Inserts the node into the state and returns its hash /// Receives the offset that needs to be traversed to reach the leaf node from the canonical root, used to compute the node hash pub fn insert_self(self, state: &mut TrieState) -> Result { @@ -332,28 +317,31 @@ mod test { #[test] fn symetric_encoding_a() { - let node = LeafNode::new( + let node: Node = LeafNode::new( Nibbles::from_bytes(b"key".as_ref()), b"a comparatively long value".to_vec(), - ); - assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + ) + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_b() { - let node = LeafNode::new( + let node: Node = LeafNode::new( Nibbles::from_bytes(&[0x12, 0x34]), vec![0x12, 0x34, 0x56, 0x78], - ); - assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + ) + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } #[test] fn symetric_encoding_c() { - let node = LeafNode::new( + let node: Node = LeafNode::new( Nibbles::from_bytes(&[]), vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 20], - ); - assert_eq!(LeafNode::decode_raw(&node.encode_raw()).unwrap(), node) + ) + .into(); + assert_eq!(Node::decode_raw(&node.encode_raw()).unwrap(), node) } } From ec31d42235a2f6d9457964c059261de8e40a0164 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 18:04:52 -0300 Subject: [PATCH 159/345] Clippy --- crates/common/rlp/structs.rs | 2 +- crates/storage/trie/node.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index 2581561f9..4dabd4c13 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -119,7 +119,7 @@ impl<'a> Decoder<'a> { /// Decodes the next field as bytes pub fn decode_bytes(self, name: &'a str) -> Result<(&[u8], Self), RLPDecodeError> { let (field, rest) = - decode_bytes(&self.payload).map_err(|err| field_decode_error::<&[u8]>(name, err))?; + decode_bytes(self.payload).map_err(|err| field_decode_error::<&[u8]>(name, err))?; let updated_self = Self { payload: rest, ..self diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index d450b90b0..6db29e0c5 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -128,7 +128,7 @@ impl Node { // Leaf or Extension Node 2 => { let (path, _) = decode_bytes(&rlp_items[0])?; - let path = Nibbles::decode_compact(&path); + let path = Nibbles::decode_compact(path); if path.is_leaf() { // Decode as Leaf let (value, _) = decode_bytes(&rlp_items[1])?; @@ -166,9 +166,9 @@ impl Node { } fn decode_child(rlp: &[u8]) -> NodeHash { - match decode_bytes(&rlp) { + match decode_bytes(rlp) { Ok((hash, &[])) if hash.len() == 32 => NodeHash::Hashed(H256::from_slice(hash)), - Ok((hash, &[])) if hash.is_empty() => NodeHash::Inline(vec![]), + Ok((&[], &[])) => NodeHash::Inline(vec![]), _ => NodeHash::Inline(rlp.to_vec()), } } From 439a01082b61b1c99994795553a3a9fa1ff6529e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 18:06:02 -0300 Subject: [PATCH 160/345] Remove unused fn --- crates/common/rlp/structs.rs | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/crates/common/rlp/structs.rs b/crates/common/rlp/structs.rs index 4dabd4c13..e497cb26b 100644 --- a/crates/common/rlp/structs.rs +++ b/crates/common/rlp/structs.rs @@ -1,5 +1,3 @@ -use crate::decode::decode_bytes; - use super::{ decode::{decode_rlp_item, get_item_with_prefix, RLPDecode}, encode::{encode_length, RLPEncode}, @@ -115,17 +113,6 @@ impl<'a> Decoder<'a> { pub fn finish_unchecked(self) -> &'a [u8] { self.remaining } - - /// Decodes the next field as bytes - pub fn decode_bytes(self, name: &'a str) -> Result<(&[u8], Self), RLPDecodeError> { - let (field, rest) = - decode_bytes(self.payload).map_err(|err| field_decode_error::<&[u8]>(name, err))?; - let updated_self = Self { - payload: rest, - ..self - }; - Ok((field, updated_self)) - } } fn field_decode_error(field_name: &str, err: RLPDecodeError) -> RLPDecodeError { From 58d7cb0761d250d3aa75c634208c3d37af521a96 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 18:11:29 -0300 Subject: [PATCH 161/345] Add doc comments --- crates/storage/trie/node.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 6db29e0c5..d24ccb49e 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -102,6 +102,7 @@ impl Node { } } + /// Encodes the node pub fn encode_raw(&self) -> Vec { match self { Node::Branch(n) => n.encode_raw(), @@ -110,6 +111,7 @@ impl Node { } } + /// Decodes the node pub fn decode_raw(rlp: &[u8]) -> Result { let mut rlp_items = vec![]; let mut decoder = Decoder::new(rlp)?; From 797d48675f2a5d63391725649ca11746eae80b3a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 8 Nov 2024 18:13:43 -0300 Subject: [PATCH 162/345] Fix typos --- crates/storage/trie/node/branch.rs | 6 +++--- crates/storage/trie/node/extension.rs | 6 +++--- crates/storage/trie/node/leaf.rs | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index cfa36f2ce..15d2d4114 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -605,7 +605,7 @@ mod test { } #[test] - fn symetric_encoding_a() { + fn symmetric_encoding_a() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) branch { @@ -618,7 +618,7 @@ mod test { } #[test] - fn symetric_encoding_b() { + fn symmetric_encoding_b() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) branch { @@ -635,7 +635,7 @@ mod test { } #[test] - fn symetric_encoding_c() { + fn symmetric_encoding_c() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) branch { diff --git a/crates/storage/trie/node/extension.rs b/crates/storage/trie/node/extension.rs index 89f49e626..ae1bd8c5c 100644 --- a/crates/storage/trie/node/extension.rs +++ b/crates/storage/trie/node/extension.rs @@ -475,7 +475,7 @@ mod test { } #[test] - fn symetric_encoding_a() { + fn symmetric_encoding_a() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) extension { [0], branch { @@ -488,7 +488,7 @@ mod test { } #[test] - fn symetric_encoding_b() { + fn symmetric_encoding_b() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) extension { [0], branch { @@ -505,7 +505,7 @@ mod test { } #[test] - fn symetric_encoding_c() { + fn symmetric_encoding_c() { let mut trie = Trie::new_temp(); let node: Node = pmt_node! { @(trie) extension { [0], branch { diff --git a/crates/storage/trie/node/leaf.rs b/crates/storage/trie/node/leaf.rs index e596f8572..969bd4bea 100644 --- a/crates/storage/trie/node/leaf.rs +++ b/crates/storage/trie/node/leaf.rs @@ -316,7 +316,7 @@ mod test { } #[test] - fn symetric_encoding_a() { + fn symmetric_encoding_a() { let node: Node = LeafNode::new( Nibbles::from_bytes(b"key".as_ref()), b"a comparatively long value".to_vec(), @@ -326,7 +326,7 @@ mod test { } #[test] - fn symetric_encoding_b() { + fn symmetric_encoding_b() { let node: Node = LeafNode::new( Nibbles::from_bytes(&[0x12, 0x34]), vec![0x12, 0x34, 0x56, 0x78], @@ -336,7 +336,7 @@ mod test { } #[test] - fn symetric_encoding_c() { + fn symmetric_encoding_c() { let node: Node = LeafNode::new( Nibbles::from_bytes(&[]), vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 20], From b0beea85083b1fe70667d6e2d59645f1829ebefd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 11 Nov 2024 11:51:54 -0300 Subject: [PATCH 163/345] Dont write nodes to DB if they are already inlined --- crates/storage/trie/node/branch.rs | 2 +- crates/storage/trie/state.rs | 9 ++++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs index 15d2d4114..0b42455af 100644 --- a/crates/storage/trie/node/branch.rs +++ b/crates/storage/trie/node/branch.rs @@ -349,7 +349,7 @@ mod test { 1 => leaf { vec![0, 16] => vec![0x34, 0x56, 0x78, 0x9A] }, } }; - let path = Nibbles::from_hex(vec![2]); + let path = Nibbles::from_bytes(&[2]); let value = vec![0x3]; let node = node diff --git a/crates/storage/trie/state.rs b/crates/storage/trie/state.rs index d99bbf472..213e3716d 100644 --- a/crates/storage/trie/state.rs +++ b/crates/storage/trie/state.rs @@ -25,6 +25,10 @@ impl TrieState { /// Retrieves a node based on its hash pub fn get_node(&self, hash: NodeHash) -> Result, TrieError> { + // Decode the node if it is inlined + if let NodeHash::Inline(encoded) = hash { + return Ok(Some(Node::decode_raw(&encoded)?)); + } if let Some(node) = self.cache.get(&hash) { return Ok(Some(node.clone())); }; @@ -36,7 +40,10 @@ impl TrieState { /// Inserts a node pub fn insert_node(&mut self, node: Node, hash: NodeHash) { - self.cache.insert(hash, node); + // Don't insert the node if it is already inlined on the parent + if matches!(hash, NodeHash::Hashed(_)) { + self.cache.insert(hash, node); + } } /// Commits cache changes to DB and clears it From 39b9d19fa5261324fa0642a7dc6cdee025e6f6dd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 11 Nov 2024 15:53:37 -0300 Subject: [PATCH 164/345] progress --- crates/networking/p2p/snap.rs | 11 ++- crates/storage/trie/error.rs | 2 +- crates/storage/trie/proof.rs | 136 +++++++++++++++++++++++++++++++--- crates/storage/trie/trie.rs | 2 +- 4 files changed, 137 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 2eebffda6..1ab359f7d 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -157,9 +157,16 @@ pub fn process_trie_nodes_request( // Response Processing -fn validate_account_range_response(request: &GetAccountRange, response: &AccountRange) -> Result<(), RLPxError> { +fn validate_account_range_response( + request: &GetAccountRange, + response: &AccountRange, +) -> Result<(), RLPxError> { // Reconstruct a partial trie from the response and verify it - let (keys, accounts)= response.accounts.iter().map(|unit| (unit.hash, unit.account)).unzip(); + let (keys, accounts) = response + .accounts + .iter() + .map(|unit| (unit.hash, unit.account)) + .unzip(); Ok(()) // verify_range(origin, keys, accounts, nodes) } diff --git a/crates/storage/trie/error.rs b/crates/storage/trie/error.rs index fdefd2c04..433da4ccd 100644 --- a/crates/storage/trie/error.rs +++ b/crates/storage/trie/error.rs @@ -8,5 +8,5 @@ pub enum TrieError { #[error(transparent)] RLPDecode(#[from] RLPDecodeError), #[error("Verification Error: {0}")] - Verify(String) + Verify(String), } diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index fed8cb538..03269b28d 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -1,21 +1,40 @@ +use std::collections::HashMap; + use ethereum_types::H256; +use sha3::{Digest, Keccak256}; -use crate::{Trie, TrieError, ValueRLP}; +use crate::{nibbles::Nibbles, node::Node, node_hash::NodeHash, Trie, TrieError, ValueRLP}; /// The boolead indicates if there is more state to be fetched -fn verify_range_proof(root: H256, first_key: H256, keys: Vec, values: Vec, proof: Vec>) -> Result { +fn verify_range_proof( + root: H256, + first_key: H256, + keys: Vec, + values: Vec, + proof: Vec>, +) -> Result { + // Store proof nodes by hash + let proof_nodes = ProofNodeStorage::from_proof(&proof); if keys.len() != values.len() { - return Err(TrieError::Verify(format!("inconsistent proof data, got {} keys and {} values", keys.len(), values.len()))); + return Err(TrieError::Verify(format!( + "inconsistent proof data, got {} keys and {} values", + keys.len(), + values.len() + ))); } // Check that the key range is monotonically increasing for keys in keys.windows(2) { if keys[0] >= keys[1] { - return Err(TrieError::Verify(String::from("key range is not monotonically increasing"))); + return Err(TrieError::Verify(String::from( + "key range is not monotonically increasing", + ))); } } // Check for empty values if values.iter().find(|value| value.is_empty()).is_some() { - return Err(TrieError::Verify(String::from("value range contains empty value"))); + return Err(TrieError::Verify(String::from( + "value range contains empty value", + ))); } // Verify ranges depending on the given proof @@ -29,16 +48,113 @@ fn verify_range_proof(root: H256, first_key: H256, keys: Vec, values: Vec< } let hash = trie.hash().unwrap_or_default(); if hash != root { - return Err(TrieError::Verify(format!("invalid proof, expected root hash {}, got {}", root, hash))); + return Err(TrieError::Verify(format!( + "invalid proof, expected root hash {}, got {}", + root, hash + ))); } - return Ok(false) + return Ok(false); } // Case B) One edge proof no range given, there are no more values in the trie if keys.is_empty() { - + let (has_right_element, value) = + has_right_element(root, first_key.as_bytes(), &proof_nodes)?; + if has_right_element || value.is_empty() { + return Err(TrieError::Verify(format!( + "no keys returned but more are available on the trie" + ))); + } } - Ok(true) -} \ No newline at end of file +} + +// Traverses the path till the last node is reached +// Check weather there are no more values in the trie + +// Indicates where there exist more elements to the right side of the given path +// Also returns the value (or an empty value if it is not present on the trie) +fn has_right_element( + root_hash: H256, + key: &[u8], + proof_nodes: &ProofNodeStorage, +) -> Result<(bool, Vec), TrieError> { + let path = Nibbles::from_bytes(key); + let node = proof_nodes.get_node(&root_hash.into())?; + has_right_element_inner(&node, path, proof_nodes) +} + +fn has_right_element_inner( + node: &Node, + mut path: Nibbles, + proof_nodes: &ProofNodeStorage, +) -> Result<(bool, Vec), TrieError> { + match node { + Node::Branch(ref n) => { + // Check if there are children to the right side + if let Some(choice) = path.next_choice() { + if n.choices[choice..].iter().any(|child| child.is_valid()) { + Ok((true, vec![])) + } else { + let node = proof_nodes.get_node(&n.choices[choice])?; + has_right_element_inner(&node, path, proof_nodes) + } + } else { + Ok((false, n.value.clone())) + } + } + Node::Extension(n) => { + if path.skip_prefix(&n.prefix) { + let node = proof_nodes.get_node(&n.child)?; + has_right_element_inner(&node, path, proof_nodes) + } else { + Ok((n.prefix.as_ref() > path.as_ref(), vec![])) + } + } + // We reached the end of the path + Node::Leaf(ref n) => { + let value = (path == n.partial) + .then_some(n.value.clone()) + .unwrap_or_default(); + Ok((false, value)) + } + } +} + +fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option<&'a NodeHash> { + match node { + Node::Branch(n) => path.next_choice().map(|i| &n.choices[i]), + Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(&n.child), + Node::Leaf(_) => None, + } +} + +struct ProofNodeStorage<'a> { + nodes: HashMap, &'a Vec>, +} + +impl<'a> ProofNodeStorage<'a> { + fn from_proof(proof: &'a Vec>) -> Self { + Self { + nodes: proof + .iter() + .map(|node| (Keccak256::new_with_prefix(node).finalize().to_vec(), node)) + .collect::>(), + } + } + + fn get_node(&self, hash: &NodeHash) -> Result { + let encoded = match hash { + NodeHash::Hashed(hash) => { + let Some(encoded) = self.nodes.get(hash.as_bytes()) else { + return Err(TrieError::Verify(format!("proof node missing: {hash}"))); + }; + *encoded + } + + NodeHash::Inline(ref encoded) => encoded, + }; + Ok(Node::decode_raw(encoded)?) + } +} diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index f38ef2323..0c1665b01 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -3,10 +3,10 @@ mod encoding; mod error; mod node; mod node_hash; +mod proof; mod rlp; mod state; mod trie_iter; -mod proof; mod nibbles; #[cfg(test)] From b2375ca5cad7638011eafb06863b3187be7f8559 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 12:43:16 -0300 Subject: [PATCH 165/345] Save current progress --- crates/storage/trie/proof.rs | 223 +++++++++++++++++++++++++++++-- crates/storage/trie/trie.rs | 3 + crates/storage/trie/trie_iter.rs | 30 +++++ 3 files changed, 246 insertions(+), 10 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 03269b28d..a3efecd01 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -3,9 +3,12 @@ use std::collections::HashMap; use ethereum_types::H256; use sha3::{Digest, Keccak256}; -use crate::{nibbles::Nibbles, node::Node, node_hash::NodeHash, Trie, TrieError, ValueRLP}; +use crate::{ + nibbles::Nibbles, node::Node, node_hash::NodeHash, state::TrieState, trie_iter::print_trie, + Trie, TrieError, ValueRLP, +}; -/// The boolead indicates if there is more state to be fetched +/// The boolean indicates if there is more state to be fetched fn verify_range_proof( root: H256, first_key: H256, @@ -39,7 +42,7 @@ fn verify_range_proof( // Verify ranges depending on the given proof - // Case A) No proofs given, the range is expected to be the full set of leaves + // Special Case A) No proofs given, the range is expected to be the full set of leaves if proof.is_empty() { let mut trie = Trie::stateless(); for (index, key) in keys.iter().enumerate() { @@ -56,7 +59,9 @@ fn verify_range_proof( return Ok(false); } - // Case B) One edge proof no range given, there are no more values in the trie + let last_key = *keys.last().unwrap(); + + // Special Case B) One edge proof no range given, there are no more values in the trie if keys.is_empty() { let (has_right_element, value) = has_right_element(root, first_key.as_bytes(), &proof_nodes)?; @@ -67,12 +72,49 @@ fn verify_range_proof( } } + // Special Case C) There is only one element and the two edge keys are the same + if keys.len() == 1 && first_key == last_key { + let (has_right_element, value) = + has_right_element(root, first_key.as_bytes(), &proof_nodes)?; + if first_key != keys[0] { + return Err(TrieError::Verify(format!("correct proof but invalid key"))); + } + if value != values[0] { + return Err(TrieError::Verify(format!("correct proof but invalid data"))); + } + return Ok(has_right_element); + } + + // Regular Case + // Here we will have two edge proofs + if first_key >= last_key { + return Err(TrieError::Verify(format!("invalid edge keys"))); + } + let mut trie = Trie::stateless(); + trie.root = Some(NodeHash::from(root)); + let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; + let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; + println!("FILL STATE"); + print_trie(&trie); + remove_internal_references(root, first_key, last_key, &mut trie.state); + println!("REMOVE INTERNAL REFERENCES"); + print_trie(&trie); + println!("KEY RANGE INSERT"); + for (i, key) in keys.iter().enumerate() { + trie.insert(key.0.to_vec(), values[i].clone())?; + } + // TODO: has_right_element + assert_eq!(trie.hash().unwrap(), root); + + // Use first proof to build node path + // use first proof root + second proof to complete it + // Remove internal references + // Add keys & values from range + // Check root + Ok(true) } -// Traverses the path till the last node is reached -// Check weather there are no more values in the trie - // Indicates where there exist more elements to the right side of the given path // Also returns the value (or an empty value if it is not present on the trie) fn has_right_element( @@ -122,14 +164,153 @@ fn has_right_element_inner( } } -fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option<&'a NodeHash> { +fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option { match node { - Node::Branch(n) => path.next_choice().map(|i| &n.choices[i]), - Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(&n.child), + Node::Branch(n) => path.next_choice().map(|i| n.choices[i].clone()), + Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(n.child.clone()), Node::Leaf(_) => None, } } +/// Fills up the TrieState with nodes from the proof traversing the path given by first_key +/// Also returns the value if it is part of the proof +fn fill_state( + trie_state: &mut TrieState, + root_hash: H256, + first_key: H256, + proof_nodes: &ProofNodeStorage, +) -> Result, TrieError> { + let mut path = Nibbles::from_bytes(&first_key.0); + fill_node( + &mut path, + &NodeHash::from(root_hash), + trie_state, + proof_nodes, + ) +} + +fn fill_node( + path: &mut Nibbles, + node_hash: &NodeHash, + trie_state: &mut TrieState, + proof_nodes: &ProofNodeStorage, +) -> Result, TrieError> { + let node = proof_nodes.get_node(node_hash)?; + let child_hash = get_child(path, &node); + if let Some(ref child_hash) = child_hash { + trie_state.insert_node(node, node_hash.clone()); + fill_node(path, child_hash, trie_state, proof_nodes) + } else { + let value = match &node { + Node::Branch(n) => n.value.clone(), + Node::Extension(_) => vec![], + Node::Leaf(n) => n.value.clone(), + }; + trie_state.insert_node(node, node_hash.clone()); + Ok(value) + } +} + +/// Removes references to internal nodes not contained in the state +/// These should be reconstructed when verifying the proof +fn remove_internal_references( + root_hash: H256, + left_key: H256, + right_key: H256, + trie_state: &mut TrieState, +) { + // First find the node at which the left and right path differ + let left_path = Nibbles::from_bytes(&left_key.0); + let right_path = Nibbles::from_bytes(&right_key.0); + + remove_internal_references_inner(NodeHash::from(root_hash), left_path, right_path, trie_state); +} + +fn remove_internal_references_inner( + node_hash: NodeHash, + mut left_path: Nibbles, + mut right_path: Nibbles, + trie_state: &mut TrieState, +) { + // We already looked up the nodes when filling the state so this shouldn't fail + let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); + match node { + Node::Branch(mut n) => { + let left_choice = left_path.next_choice().unwrap(); + let right_choice = right_path.next_choice().unwrap(); + if left_choice == right_choice && n.choices[left_choice].is_valid() { + // Keep going + return remove_internal_references_inner( + n.choices[left_choice].clone(), + left_path, + right_path, + trie_state, + ); + } + // We found our fork node, now we can remove the internal references + for choice in &mut n.choices[left_choice + 1..right_choice - 1] { + *choice = NodeHash::default() + } + // Remove nodes on the left and right choice's subtries + remove_nodes( + &node_hash, + n.choices[left_choice].clone(), + left_path, + false, + trie_state, + ); + remove_nodes( + &node_hash, + n.choices[right_choice].clone(), + right_path, + true, + trie_state, + ); + // Update node in the state + trie_state.insert_node(n.into(), node_hash); + } + Node::Extension(n) => todo!(), + Node::Leaf(_) => todo!(), + } +} + +fn remove_nodes( + parent_hash: &NodeHash, + node_hash: NodeHash, + mut path: Nibbles, + remove_left: bool, + trie_state: &mut TrieState, +) { + let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); + match node { + Node::Branch(mut n) => { + // Remove child nodes + let choice = path.next_choice().unwrap(); + if remove_left { + for child in &mut n.choices[..choice - 1] { + *child = NodeHash::default() + } + } else { + for child in &mut n.choices[choice + 1..] { + *child = NodeHash::default() + } + // Remove nodes to the left/right of the choice's subtrie + remove_nodes( + &node_hash, + n.choices[choice].clone(), + path, + remove_left, + trie_state, + ); + } + // Update node in the state + trie_state.insert_node(n.into(), node_hash); + } + Node::Extension(extension_node) => todo!(), + Node::Leaf(leaf_node) => todo!(), + } +} + struct ProofNodeStorage<'a> { nodes: HashMap, &'a Vec>, } @@ -158,3 +339,25 @@ impl<'a> ProofNodeStorage<'a> { Ok(Node::decode_raw(encoded)?) } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn verify_range_proof_regular_case() { + // The trie will have keys and values ranging from 25-100 + // We will prove the range from 50-75 + // Note values are written as hashes in the form i -> [i;32] + let mut trie = Trie::new_temp(); + for k in 25..100_u8 { + trie.insert([k; 32].to_vec(), [k; 32].to_vec()).unwrap() + } + let mut proof = trie.get_proof(&[50; 32].to_vec()).unwrap(); + proof.extend(trie.get_proof(&[75; 32].to_vec()).unwrap()); + let root = trie.hash().unwrap(); + let keys = (50_u8..=75).map(|i| H256([i; 32])).collect::>(); + let values = (50_u8..=75).map(|i| [i; 32].to_vec()).collect::>(); + verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + } +} diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 0c1665b01..1949ae507 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -17,6 +17,7 @@ use nibbles::Nibbles; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; +use trie_iter::print_trie; #[cfg(feature = "libmdbx")] pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; @@ -84,6 +85,7 @@ impl Trie { /// Insert an RLP-encoded value into the trie. pub fn insert(&mut self, path: PathRLP, value: ValueRLP) -> Result<(), TrieError> { + println!("INSERT: [{}]", path[0]); let root = self.root.take(); if let Some(root_node) = root .map(|root| self.state.get_node(root)) @@ -99,6 +101,7 @@ impl Trie { let new_leaf = Node::from(LeafNode::new(Nibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } + print_trie(&self); Ok(()) } diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 5b90ddd13..9a90acc2a 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -66,6 +66,36 @@ impl TrieIterator { } } +pub fn print_trie(trie: &Trie) { + let Some(root) = &trie.root else { return }; + print_node(trie, root.clone()); + print!("\n") +} + +pub fn print_node(trie: &Trie, node_hash: NodeHash) { + if trie.state.get_node(node_hash.clone()).unwrap().is_none() { + print!("[MISSING NODE]"); + return; + } + match trie.state.get_node(node_hash).unwrap().unwrap() { + Node::Branch(n) => { + print!("Branch{:?} [", n.value); + for (i, child) in n.choices.iter().enumerate() { + if child.is_valid() { + print!(" {i}: "); + print_node(trie, child.clone()); + } + } + print!(" ]") + } + Node::Extension(n) => { + print!("Ext{:?} -> ", n.prefix.as_ref()); + print_node(trie, n.child); + } + Node::Leaf(n) => print!("Leaf({:?})", n.value[0]), + } +} + #[cfg(test)] mod tests { From 04f94a3ee553b1fb4ab14e672386ea2fd790f797 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 12:47:34 -0300 Subject: [PATCH 166/345] Fix logic --- crates/storage/trie/proof.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index a3efecd01..40b6d7548 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -248,7 +248,7 @@ fn remove_internal_references_inner( ); } // We found our fork node, now we can remove the internal references - for choice in &mut n.choices[left_choice + 1..right_choice - 1] { + for choice in &mut n.choices[left_choice..right_choice] { *choice = NodeHash::default() } // Remove nodes on the left and right choice's subtries @@ -287,11 +287,11 @@ fn remove_nodes( // Remove child nodes let choice = path.next_choice().unwrap(); if remove_left { - for child in &mut n.choices[..choice - 1] { + for child in &mut n.choices[..choice] { *child = NodeHash::default() } } else { - for child in &mut n.choices[choice + 1..] { + for child in &mut n.choices[choice..] { *child = NodeHash::default() } // Remove nodes to the left/right of the choice's subtrie From fc3ecc8e1b3e0f2accf7d2fc9d78b8eedae2f17f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 12:49:28 -0300 Subject: [PATCH 167/345] Fix logic --- crates/storage/trie/proof.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 40b6d7548..0957b31d7 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -281,6 +281,9 @@ fn remove_nodes( remove_left: bool, trie_state: &mut TrieState, ) { + if !node_hash.is_valid() { + return; + } let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); match node { Node::Branch(mut n) => { From 774e1d1907ac791d6c3c3dbe637d71f4adf8bc04 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 14:32:37 -0300 Subject: [PATCH 168/345] Fix logic --- crates/storage/trie/proof.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 0957b31d7..b8e5cc637 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -248,7 +248,7 @@ fn remove_internal_references_inner( ); } // We found our fork node, now we can remove the internal references - for choice in &mut n.choices[left_choice..right_choice] { + for choice in &mut n.choices[left_choice + 1..right_choice] { *choice = NodeHash::default() } // Remove nodes on the left and right choice's subtries @@ -294,7 +294,7 @@ fn remove_nodes( *child = NodeHash::default() } } else { - for child in &mut n.choices[choice..] { + for child in &mut n.choices[choice + 1..] { *child = NodeHash::default() } // Remove nodes to the left/right of the choice's subtrie @@ -310,7 +310,7 @@ fn remove_nodes( trie_state.insert_node(n.into(), node_hash); } Node::Extension(extension_node) => todo!(), - Node::Leaf(leaf_node) => todo!(), + Node::Leaf(_) => return, } } From 827439e83cb5a5fe786f9707a07aaded0652103d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 16:30:57 -0300 Subject: [PATCH 169/345] Add extension node logic to remove internal nodes --- crates/storage/trie/proof.rs | 169 +++++++++++++++++++++++++++-------- 1 file changed, 130 insertions(+), 39 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index b8e5cc637..7ad539f5f 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -1,4 +1,7 @@ -use std::collections::HashMap; +use std::{ + cmp::{self, Ordering}, + collections::HashMap, +}; use ethereum_types::H256; use sha3::{Digest, Keccak256}; @@ -213,25 +216,27 @@ fn fill_node( /// Removes references to internal nodes not contained in the state /// These should be reconstructed when verifying the proof +/// Returns true if the trie is left empty (rootless) as a result of this process fn remove_internal_references( root_hash: H256, left_key: H256, right_key: H256, trie_state: &mut TrieState, -) { +) -> bool { // First find the node at which the left and right path differ let left_path = Nibbles::from_bytes(&left_key.0); let right_path = Nibbles::from_bytes(&right_key.0); - remove_internal_references_inner(NodeHash::from(root_hash), left_path, right_path, trie_state); + remove_internal_references_inner(NodeHash::from(root_hash), left_path, right_path, trie_state) } +// Return = true -> child should be removed fn remove_internal_references_inner( node_hash: NodeHash, mut left_path: Nibbles, mut right_path: Nibbles, trie_state: &mut TrieState, -) { +) -> bool { // We already looked up the nodes when filling the state so this shouldn't fail let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); match node { @@ -240,49 +245,98 @@ fn remove_internal_references_inner( let right_choice = right_path.next_choice().unwrap(); if left_choice == right_choice && n.choices[left_choice].is_valid() { // Keep going - return remove_internal_references_inner( + // Check if the child extension node should be removed as a result of this process + let should_remove = remove_internal_references_inner( n.choices[left_choice].clone(), left_path, right_path, trie_state, ); + if should_remove { + n.choices[left_choice] = NodeHash::default(); + trie_state.insert_node(n.into(), node_hash); + } + } else { + // We found our fork node, now we can remove the internal references + for choice in &mut n.choices[left_choice + 1..right_choice] { + *choice = NodeHash::default() + } + // Remove nodes on the left and right choice's subtries + let should_remove_left = + remove_nodes(n.choices[left_choice].clone(), left_path, false, trie_state); + let should_remove_right = remove_nodes( + n.choices[right_choice].clone(), + right_path, + true, + trie_state, + ); + if should_remove_left { + n.choices[left_choice] = NodeHash::default(); + } + if should_remove_right { + n.choices[right_choice] = NodeHash::default(); + } + // Update node in the state + trie_state.insert_node(n.into(), node_hash); + } + } + Node::Extension(n) => { + // Compare left and right paths against prefix + let compare_path = |path: &Nibbles, prefix: &Nibbles| -> Ordering { + if path.len() > prefix.len() { + path.as_ref()[..prefix.len()].cmp(prefix.as_ref()) + } else { + path.as_ref().cmp(prefix.as_ref()) + } + }; + + let left_fork = compare_path(&left_path, &n.prefix); + let right_fork = compare_path(&right_path, &n.prefix); + + if left_fork.is_eq() && right_fork.is_eq() { + // Keep going + return remove_internal_references_inner( + n.child, + left_path.offset(n.prefix.len()), + right_path.offset(n.prefix.len()), + trie_state, + ); } // We found our fork node, now we can remove the internal references - for choice in &mut n.choices[left_choice + 1..right_choice] { - *choice = NodeHash::default() + match (left_fork, right_fork) { + // If both paths are greater or lesser than the node's prefix then the range is empty + // TODO: return the error instead of panicking here + (Ordering::Greater, Ordering::Greater) | (Ordering::Less, Ordering::Less) => { + panic!("empty range") + } + // Node of the paths fit the prefix, remove the entire subtrie + (left, right) if left.is_ne() && right.is_ne() => { + // Return true so that the parent node knows they need to remove this node + return true; + } + // One path fits teh prefix, the other one doesn't + (left, right) => { + // If the child is a leaf node, tell parent to remove the node -> we will let the child handle this + let path = if left.is_eq() { left_path } else { right_path }; + // If the child node is removed then this node will be removed too so we will leave that to the parent + return remove_nodes(node_hash, path, right.is_eq(), trie_state); + } } - // Remove nodes on the left and right choice's subtries - remove_nodes( - &node_hash, - n.choices[left_choice].clone(), - left_path, - false, - trie_state, - ); - remove_nodes( - &node_hash, - n.choices[right_choice].clone(), - right_path, - true, - trie_state, - ); - // Update node in the state - trie_state.insert_node(n.into(), node_hash); } - Node::Extension(n) => todo!(), Node::Leaf(_) => todo!(), } + false } +// Return = true -> child should be removed fn remove_nodes( - parent_hash: &NodeHash, node_hash: NodeHash, mut path: Nibbles, remove_left: bool, trie_state: &mut TrieState, -) { +) -> bool { if !node_hash.is_valid() { - return; + return false; } let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); match node { @@ -297,21 +351,30 @@ fn remove_nodes( for child in &mut n.choices[choice + 1..] { *child = NodeHash::default() } - // Remove nodes to the left/right of the choice's subtrie - remove_nodes( - &node_hash, - n.choices[choice].clone(), - path, - remove_left, - trie_state, - ); + } + // Remove nodes to the left/right of the choice's subtrie + let should_remove = + remove_nodes(n.choices[choice].clone(), path, remove_left, trie_state); + if should_remove { + n.choices[choice] = NodeHash::default(); } // Update node in the state trie_state.insert_node(n.into(), node_hash); } - Node::Extension(extension_node) => todo!(), - Node::Leaf(_) => return, + Node::Extension(n) => { + if !path.skip_prefix(&n.prefix) { + if (remove_left && n.prefix.as_ref() < path.as_ref()) + || (!remove_left && n.prefix.as_ref() > path.as_ref()) + { + return true; + } + } else { + return remove_nodes(n.child, path, remove_left, trie_state); + } + } + Node::Leaf(_) => return true, } + false } struct ProofNodeStorage<'a> { @@ -345,10 +408,12 @@ impl<'a> ProofNodeStorage<'a> { #[cfg(test)] mod tests { + use ethereum_types::{BigEndianHash, U256}; + use super::*; #[test] - fn verify_range_proof_regular_case() { + fn verify_range_proof_regular_case_only_branch_nodes() { // The trie will have keys and values ranging from 25-100 // We will prove the range from 50-75 // Note values are written as hashes in the form i -> [i;32] @@ -363,4 +428,30 @@ mod tests { let values = (50_u8..=75).map(|i| [i; 32].to_vec()).collect::>(); verify_range_proof(root, keys[0], keys, values, proof).unwrap(); } + + #[test] + fn verify_range_proof_regular_case() { + // The trie will have keys and values with the same bytes values ranging from 1 to 200 + // We will prove the range from 75 to 150 + let mut trie = Trie::new_temp(); + for i in 0..200 { + let val = H256::from_uint(&U256::from(i)).0.to_vec(); + trie.insert(val.clone(), val).unwrap() + } + let mut proof = trie + .get_proof(&H256::from_uint(&U256::from(75)).0.to_vec()) + .unwrap(); + proof.extend( + trie.get_proof(&H256::from_uint(&U256::from(150)).0.to_vec()) + .unwrap(), + ); + let root = trie.hash().unwrap(); + let keys = (75..=150) + .map(|i| H256::from_uint(&U256::from(i))) + .collect::>(); + let values = (75..=150) + .map(|i| H256::from_uint(&U256::from(i)).0.to_vec()) + .collect::>(); + verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + } } From 377b794a7a091916bedcf45fde948918fde69db4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 16:50:37 -0300 Subject: [PATCH 170/345] Improve test --- crates/storage/trie/proof.rs | 69 +++++++++++++++++++++++++----------- 1 file changed, 48 insertions(+), 21 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 7ad539f5f..dbd8e63fe 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -408,8 +408,7 @@ impl<'a> ProofNodeStorage<'a> { #[cfg(test)] mod tests { - use ethereum_types::{BigEndianHash, U256}; - + use std::str::FromStr; use super::*; #[test] @@ -431,27 +430,55 @@ mod tests { #[test] fn verify_range_proof_regular_case() { - // The trie will have keys and values with the same bytes values ranging from 1 to 200 - // We will prove the range from 75 to 150 + let account_addresses: [&str; 26] = [ + "0xaa56789abcde80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6", + "0xaa56789abcdeda9ae19dd26a33bd10bbf825e28b3de84fc8fe1d15a21645067f", + "0xaa56789abc39a8284ef43790e3a511b2caa50803613c5096bc782e8de08fa4c5", + "0xaa5678931f4754834b0502de5b0342ceff21cde5bef386a83d2292f4445782c2", + "0xaa567896492bfe767f3d18be2aab96441c449cd945770ef7ef8555acc505b2e4", + "0xaa5f478d53bf78add6fa3708d9e061d59bfe14b21329b2a4cf1156d4f81b3d2d", + "0xaa67c643f67b47cac9efacf6fcf0e4f4e1b273a727ded155db60eb9907939eb6", + "0xaa04d8eaccf0b942c468074250cbcb625ec5c4688b6b5d17d2a9bdd8dd565d5a", + "0xaa63e52cda557221b0b66bd7285b043071df4c2ab146260f4e010970f3a0cccf", + "0xaad9aa4f67f8b24d70a0ffd757e82456d9184113106b7d9e8eb6c3e8a8df27ee", + "0xaa3df2c3b574026812b154a99b13b626220af85cd01bb1693b1d42591054bce6", + "0xaa79e46a5ed8a88504ac7d579b12eb346fbe4fd7e281bdd226b891f8abed4789", + "0xbbf68e241fff876598e8e01cd529bd76416b248caf11e0552047c5f1d516aab6", + "0xbbf68e241fff876598e8e01cd529c908cdf0d646049b5b83629a70b0117e2957", + "0xbbf68e241fff876598e8e0180b89744abb96f7af1171ed5f47026bdf01df1874", + "0xbbf68e241fff876598e8a4cd8e43f08be4715d903a0b1d96b3d9c4e811cbfb33", + "0xbbf68e241fff8765182a510994e2b54d14b731fac96b9c9ef434bc1924315371", + "0xbbf68e241fff87655379a3b66c2d8983ba0b2ca87abaf0ca44836b2a06a2b102", + "0xbbf68e241fffcbcec8301709a7449e2e7371910778df64c89f48507390f2d129", + "0xbbf68e241ffff228ed3aa7a29644b1915fde9ec22e0433808bf5467d914e7c7a", + "0xbbf68e24190b881949ec9991e48dec768ccd1980896aefd0d51fd56fd5689790", + "0xbbf68e2419de0a0cb0ff268c677aba17d39a3190fe15aec0ff7f54184955cba4", + "0xbbf68e24cc6cbd96c1400150417dd9b30d958c58f63c36230a90a02b076f78b5", + "0xbbf68e2490f33f1d1ba6d1521a00935630d2c81ab12fa03d4a0f4915033134f3", + "0xc017b10a7cc3732d729fe1f71ced25e5b7bc73dc62ca61309a8c7e5ac0af2f72", + "0xc098f06082dc467088ecedb143f9464ebb02f19dc10bd7491b03ba68d751ce45", + ]; + let mut account_addresses = account_addresses + .iter() + .map(|addr| H256::from_str(addr).unwrap()) + .collect::>(); + account_addresses.sort(); + let trie_values = account_addresses + .iter() + .map(|addr| addr.0.to_vec()) + .collect::>(); + let key_range = account_addresses[7..=17].to_vec(); + let value_range = account_addresses[7..=17] + .iter() + .map(|v| v.0.to_vec()) + .collect::>(); let mut trie = Trie::new_temp(); - for i in 0..200 { - let val = H256::from_uint(&U256::from(i)).0.to_vec(); - trie.insert(val.clone(), val).unwrap() + for val in trie_values.iter() { + trie.insert(val.clone(), val.clone()).unwrap() } - let mut proof = trie - .get_proof(&H256::from_uint(&U256::from(75)).0.to_vec()) - .unwrap(); - proof.extend( - trie.get_proof(&H256::from_uint(&U256::from(150)).0.to_vec()) - .unwrap(), - ); + let mut proof = trie.get_proof(&trie_values[7]).unwrap(); + proof.extend(trie.get_proof(&trie_values[17]).unwrap()); let root = trie.hash().unwrap(); - let keys = (75..=150) - .map(|i| H256::from_uint(&U256::from(i))) - .collect::>(); - let values = (75..=150) - .map(|i| H256::from_uint(&U256::from(i)).0.to_vec()) - .collect::>(); - verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } } From 185b8846471ba68ec94dae58e21089a6425f62c7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 16:51:54 -0300 Subject: [PATCH 171/345] Add comment --- crates/storage/trie/proof.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index dbd8e63fe..2a2639410 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -430,6 +430,8 @@ mod tests { #[test] fn verify_range_proof_regular_case() { + // The account ranges were taken form a hive test state, but artificially modified + // so that the resulting trie has a wide variety of different nodes (and not only branches) let account_addresses: [&str; 26] = [ "0xaa56789abcde80cde11add7d3447cd4ca93a5f2205d9874261484ae180718bd6", "0xaa56789abcdeda9ae19dd26a33bd10bbf825e28b3de84fc8fe1d15a21645067f", From 2c8bb52fa3a550f22b4f216b9920caa2f497f825 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 17:22:05 -0300 Subject: [PATCH 172/345] Add proptest --- crates/storage/trie/proof.rs | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 2a2639410..2c3b40743 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -408,8 +408,10 @@ impl<'a> ProofNodeStorage<'a> { #[cfg(test)] mod tests { - use std::str::FromStr; use super::*; + use proptest::prelude::any; + use proptest::proptest; + use std::{collections::btree_set, str::FromStr}; #[test] fn verify_range_proof_regular_case_only_branch_nodes() { @@ -483,4 +485,26 @@ mod tests { let root = trie.hash().unwrap(); verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } + + use proptest::collection::{btree_set, vec}; + + proptest! { + + #[test] + fn proptest_verify_range_regular_case(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>()[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Generate proofs + let mut proof = trie.get_proof(&values[0]).unwrap(); + proof.extend(trie.get_proof(&values.last().unwrap()).unwrap()); + verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + } + } } From 76d719afa96004ad902c932df5069624eb7615cf Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 17:56:26 -0300 Subject: [PATCH 173/345] Add proptest --- crates/storage/trie/proof.rs | 43 +++++++++++++++++++++++++++++++++--- crates/storage/trie/trie.rs | 2 -- 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 2c3b40743..b10440c89 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -409,9 +409,10 @@ impl<'a> ProofNodeStorage<'a> { #[cfg(test)] mod tests { use super::*; + use proptest::collection::{btree_set, vec}; use proptest::prelude::any; use proptest::proptest; - use std::{collections::btree_set, str::FromStr}; + use std::str::FromStr; #[test] fn verify_range_proof_regular_case_only_branch_nodes() { @@ -486,11 +487,10 @@ mod tests { verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } - use proptest::collection::{btree_set, vec}; - proptest! { #[test] + // Regular Case: Two Edge Proofs, both keys exist fn proptest_verify_range_regular_case(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { // Build trie let mut trie = Trie::new_temp(); @@ -504,7 +504,44 @@ mod tests { // Generate proofs let mut proof = trie.get_proof(&values[0]).unwrap(); proof.extend(trie.get_proof(&values.last().unwrap()).unwrap()); + // Verify the range proof verify_range_proof(root, keys[0], keys, values, proof).unwrap(); } + + #[test] + // Two Edge Proofs, first and last keys dont exist + fn proptest_verify_range_nonexistant_edge_keys(data in btree_set(vec(1..u8::MAX-1, 32), 200), start in 1_usize..=100_usize, end in 101..199_usize) { + let data = data.into_iter().collect::>(); + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Select the first and last keys + // As we will be using non-existant keys we will choose values that are `just` higer/lower than + // the first and last values in our key range + // Skip the test entirely in the unlucky case that the values just next to the edge keys are also part of the trie + let mut first_key = data[start].clone(); + first_key[31] -=1; + if first_key == data[start -1] { + // Skip test + return Ok(()); + } + let mut last_key = data[end].clone(); + last_key[31] +=1; + if last_key == data[end +1] { + // Skip test + return Ok(()); + } + // Generate proofs + let mut proof = trie.get_proof(&first_key).unwrap(); + proof.extend(trie.get_proof(&last_key).unwrap()); + // Verify the range proof + verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + } } } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 1949ae507..d409ecd5c 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -85,7 +85,6 @@ impl Trie { /// Insert an RLP-encoded value into the trie. pub fn insert(&mut self, path: PathRLP, value: ValueRLP) -> Result<(), TrieError> { - println!("INSERT: [{}]", path[0]); let root = self.root.take(); if let Some(root_node) = root .map(|root| self.state.get_node(root)) @@ -101,7 +100,6 @@ impl Trie { let new_leaf = Node::from(LeafNode::new(Nibbles::from_bytes(&path), value)); self.root = Some(new_leaf.insert_self(&mut self.state)?) } - print_trie(&self); Ok(()) } From 9254ce895ba4fe9c1ffad2200522ee16542b7b3a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 18:04:10 -0300 Subject: [PATCH 174/345] Add test case --- crates/storage/trie/proof.rs | 41 +++++++++++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index b10440c89..91c8a585b 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -411,7 +411,7 @@ mod tests { use super::*; use proptest::collection::{btree_set, vec}; use proptest::prelude::any; - use proptest::proptest; + use proptest::{bool, proptest}; use std::str::FromStr; #[test] @@ -543,5 +543,44 @@ mod tests { // Verify the range proof verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); } + + #[test] + // Two Edge Proofs, one key doesn't exist + fn proptest_verify_range_one_key_doesnt_exist(data in btree_set(vec(1..u8::MAX-1, 32), 200), start in 1_usize..=100_usize, end in 101..199_usize, first_key_exists in bool::ANY) { + let data = data.into_iter().collect::>(); + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Select the first and last keys + // As we will be using non-existant keys we will choose values that are `just` higer/lower than + // the first and last values in our key range + // Skip the test entirely in the unlucky case that the values just next to the edge keys are also part of the trie + let mut first_key = data[start].clone(); + let mut last_key = data[end].clone(); + if first_key_exists { + last_key[31] +=1; + if last_key == data[end +1] { + // Skip test + return Ok(()); + } + } else { + first_key[31] -=1; + if first_key == data[start -1] { + // Skip test + return Ok(()); + } + } + // Generate proofs + let mut proof = trie.get_proof(&first_key).unwrap(); + proof.extend(trie.get_proof(&last_key).unwrap()); + // Verify the range proof + verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + } } } From 0a27aab9194ab236691fe7a85acf41d267759e81 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 18:31:03 -0300 Subject: [PATCH 175/345] rename func --- crates/storage/trie/proof.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 91c8a585b..f9a47d426 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -263,8 +263,8 @@ fn remove_internal_references_inner( } // Remove nodes on the left and right choice's subtries let should_remove_left = - remove_nodes(n.choices[left_choice].clone(), left_path, false, trie_state); - let should_remove_right = remove_nodes( + remove_node(n.choices[left_choice].clone(), left_path, false, trie_state); + let should_remove_right = remove_node( n.choices[right_choice].clone(), right_path, true, @@ -309,17 +309,17 @@ fn remove_internal_references_inner( (Ordering::Greater, Ordering::Greater) | (Ordering::Less, Ordering::Less) => { panic!("empty range") } - // Node of the paths fit the prefix, remove the entire subtrie + // None of the paths fit the prefix, remove the entire subtrie (left, right) if left.is_ne() && right.is_ne() => { // Return true so that the parent node knows they need to remove this node return true; } - // One path fits teh prefix, the other one doesn't + // One path fits the prefix, the other one doesn't (left, right) => { // If the child is a leaf node, tell parent to remove the node -> we will let the child handle this let path = if left.is_eq() { left_path } else { right_path }; // If the child node is removed then this node will be removed too so we will leave that to the parent - return remove_nodes(node_hash, path, right.is_eq(), trie_state); + return remove_node(node_hash, path, right.is_eq(), trie_state); } } } @@ -329,7 +329,7 @@ fn remove_internal_references_inner( } // Return = true -> child should be removed -fn remove_nodes( +fn remove_node( node_hash: NodeHash, mut path: Nibbles, remove_left: bool, @@ -354,7 +354,7 @@ fn remove_nodes( } // Remove nodes to the left/right of the choice's subtrie let should_remove = - remove_nodes(n.choices[choice].clone(), path, remove_left, trie_state); + remove_node(n.choices[choice].clone(), path, remove_left, trie_state); if should_remove { n.choices[choice] = NodeHash::default(); } @@ -369,7 +369,7 @@ fn remove_nodes( return true; } } else { - return remove_nodes(n.child, path, remove_left, trie_state); + return remove_node(n.child, path, remove_left, trie_state); } } Node::Leaf(_) => return true, @@ -544,7 +544,7 @@ mod tests { verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); } - #[test] + #[test] // Two Edge Proofs, one key doesn't exist fn proptest_verify_range_one_key_doesnt_exist(data in btree_set(vec(1..u8::MAX-1, 32), 200), start in 1_usize..=100_usize, end in 101..199_usize, first_key_exists in bool::ANY) { let data = data.into_iter().collect::>(); From 68661c951749d6760d397f524c6b6ffe0ed6f1eb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 12 Nov 2024 18:53:54 -0300 Subject: [PATCH 176/345] Add doc --- crates/storage/trie/proof.rs | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index f9a47d426..68e878c99 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -328,17 +328,23 @@ fn remove_internal_references_inner( false } -// Return = true -> child should be removed +// Removes all nodes in the node's subtrie to the left or right of the path (given by the `remove_left` flag) +// If the whole subtrie is removed in the process this function will return true, in which case +// the caller must remove the reference to this node from it's parent node fn remove_node( node_hash: NodeHash, mut path: Nibbles, remove_left: bool, trie_state: &mut TrieState, ) -> bool { + // Node doesn't exist already, no need to remove it if !node_hash.is_valid() { return false; } - let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); + // We already checked the canonical proof path when filling the state so this case should be unreachable + let Ok(Some(node)) = trie_state.get_node(node_hash.clone()) else { + return false; + }; match node { Node::Branch(mut n) => { // Remove child nodes @@ -362,6 +368,8 @@ fn remove_node( trie_state.insert_node(n.into(), node_hash); } Node::Extension(n) => { + // If no child subtrie would result from this process remove the node entirely + // (Such as removing the left side of a trie with no right side) if !path.skip_prefix(&n.prefix) { if (remove_left && n.prefix.as_ref() < path.as_ref()) || (!remove_left && n.prefix.as_ref() > path.as_ref()) @@ -369,6 +377,7 @@ fn remove_node( return true; } } else { + // Remove left/right side of the child subtrie return remove_node(n.child, path, remove_left, trie_state); } } From fdcbc99f1b5c652d3f7944ac9ee6638a38c36d6a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 10:58:02 -0300 Subject: [PATCH 177/345] Add doc --- crates/storage/trie/proof.rs | 29 +++++++++++++++++++---------- crates/storage/trie/trie.rs | 2 +- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 68e878c99..4a8125d44 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -12,7 +12,7 @@ use crate::{ }; /// The boolean indicates if there is more state to be fetched -fn verify_range_proof( +pub fn verify_range_proof( root: H256, first_key: H256, keys: Vec, @@ -94,12 +94,14 @@ fn verify_range_proof( return Err(TrieError::Verify(format!("invalid edge keys"))); } let mut trie = Trie::stateless(); - trie.root = Some(NodeHash::from(root)); let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; println!("FILL STATE"); print_trie(&trie); - remove_internal_references(root, first_key, last_key, &mut trie.state); + let empty = remove_internal_references(root, first_key, last_key, &mut trie.state); + if !empty { + trie.root = Some(NodeHash::from(root)); + } println!("REMOVE INTERNAL REFERENCES"); print_trie(&trie); println!("KEY RANGE INSERT"); @@ -214,8 +216,8 @@ fn fill_node( } } -/// Removes references to internal nodes not contained in the state -/// These should be reconstructed when verifying the proof +/// Removes references to internal nodes between the left and right key +/// These nodes should be entirely reconstructed when inserting the elements between left and right key (the proven range) /// Returns true if the trie is left empty (rootless) as a result of this process fn remove_internal_references( root_hash: H256, @@ -230,7 +232,10 @@ fn remove_internal_references( remove_internal_references_inner(NodeHash::from(root_hash), left_path, right_path, trie_state) } -// Return = true -> child should be removed +/// Traverses the left and right path starting from the given node until the paths diverge +/// Once the paths diverge, removes the nodes between the left and right path +/// Returns true if the given node was completely removed as a result of this process +/// In which case the caller should remove the reference to this node from its parent node fn remove_internal_references_inner( node_hash: NodeHash, mut left_path: Nibbles, @@ -253,11 +258,14 @@ fn remove_internal_references_inner( trie_state, ); if should_remove { + // Remove child node n.choices[left_choice] = NodeHash::default(); + // Update node in the state trie_state.insert_node(n.into(), node_hash); } } else { // We found our fork node, now we can remove the internal references + // Remove all child nodes between the left and right child nodes for choice in &mut n.choices[left_choice + 1..right_choice] { *choice = NodeHash::default() } @@ -270,6 +278,7 @@ fn remove_internal_references_inner( true, trie_state, ); + // Remove left and right child nodes if their subtries where wiped in the process if should_remove_left { n.choices[left_choice] = NodeHash::default(); } @@ -294,7 +303,7 @@ fn remove_internal_references_inner( let right_fork = compare_path(&right_path, &n.prefix); if left_fork.is_eq() && right_fork.is_eq() { - // Keep going + // If both paths contain the same prefix as the extension node, keep going return remove_internal_references_inner( n.child, left_path.offset(n.prefix.len()), @@ -311,14 +320,14 @@ fn remove_internal_references_inner( } // None of the paths fit the prefix, remove the entire subtrie (left, right) if left.is_ne() && right.is_ne() => { - // Return true so that the parent node knows they need to remove this node + // Return true so that the parent node removes this node return true; } // One path fits the prefix, the other one doesn't (left, right) => { - // If the child is a leaf node, tell parent to remove the node -> we will let the child handle this + // Remove the nodes from the child's subtrie let path = if left.is_eq() { left_path } else { right_path }; - // If the child node is removed then this node will be removed too so we will leave that to the parent + // Propagate the response so that this node will be removed too if the child's subtrie is wiped return remove_node(node_hash, path, right.is_eq(), trie_state); } } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index d409ecd5c..00f89a61e 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -25,7 +25,7 @@ pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; pub use self::error::TrieError; -use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator}; +use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator, proof::verify_range_proof}; use lazy_static::lazy_static; From 6f1e304f400420e33428d20e8a934fb77e5caad9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 11:31:31 -0300 Subject: [PATCH 178/345] Add special case test --- crates/storage/trie/proof.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 4a8125d44..554cf81ea 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -600,5 +600,23 @@ mod tests { // Verify the range proof verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); } + + #[test] + // Special Case: Range contains all the leafs in the trie, no proofs + fn proptest_verify_range_full_leafset(data in btree_set(vec(any::(), 32), 100..200)) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // The keyset contains the entire trie so we don't need edge proofs + let proof = vec![]; + // Verify the range proof + verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + } } } From 925f536c59fe2a005fc8e2201e045a26c88a12cf Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 11:41:30 -0300 Subject: [PATCH 179/345] Fix peripherial code --- crates/networking/p2p/Cargo.toml | 1 + crates/networking/p2p/rlpx/connection.rs | 13 ------------ crates/networking/p2p/rlpx/error.rs | 3 +++ crates/networking/p2p/rlpx/snap.rs | 2 +- crates/networking/p2p/snap.rs | 25 +++++++++++++++++++++--- crates/storage/trie/proof.rs | 5 +---- crates/storage/trie/trie.rs | 3 ++- 7 files changed, 30 insertions(+), 22 deletions(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index 128cff7e7..2fbdae71a 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -9,6 +9,7 @@ edition = "2021" ethereum_rust-core.workspace = true ethereum_rust-rlp.workspace = true ethereum_rust-storage.workspace = true +ethereum_rust-trie.workspace = true tracing.workspace = true tokio.workspace = true diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 3d97ec383..eebc01507 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -199,19 +199,6 @@ impl RLPxConnection { let response = process_trie_nodes_request(req, self.storage.clone())?; self.send(Message::TrieNodes(response)).await? } - Message::GetStorageRanges(req) => { - let response = - process_storage_ranges_request(req, self.storage.clone())?; - self.send(Message::StorageRanges(response)).await - } - Message::GetByteCodes(req) => { - let response = process_byte_codes_request(req, self.storage.clone())?; - self.send(Message::ByteCodes(response)).await - } - Message::GetTrieNodes(req) => { - let response = process_trie_nodes_request(req, self.storage.clone())?; - self.send(Message::TrieNodes(response)).await - } // TODO: Add new message types and handlers as they are implemented _ => return Err(RLPxError::MessageNotHandled()), }; diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 1e65634d1..a3d638fe5 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,5 +1,6 @@ use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; use ethereum_rust_storage::error::StoreError; +use ethereum_rust_trie::TrieError; use thiserror::Error; // TODO improve errors @@ -31,6 +32,8 @@ pub(crate) enum RLPxError { StoreError(#[from] StoreError), #[error("Error in cryptographic library: {0}")] CryptographyError(String), + #[error(transparent)] + Trie(#[from] TrieError), } // Grouping all cryptographic related errors in a single CryptographicError variant diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index abec6c9f1..f83ada6b9 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -326,7 +326,7 @@ pub struct AccountRangeUnit { pub account: AccountStateSlim, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AccountStateSlim { pub nonce: u64, pub balance: U256, diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 1ab359f7d..5017becba 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,6 +1,8 @@ use bytes::Bytes; +use ethereum_rust_core::types::AccountState; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; +use ethereum_rust_trie::verify_range_proof; use crate::rlpx::{ error::RLPxError, @@ -157,7 +159,8 @@ pub fn process_trie_nodes_request( // Response Processing -fn validate_account_range_response( +#[allow(unused)] +pub fn validate_account_range_response( request: &GetAccountRange, response: &AccountRange, ) -> Result<(), RLPxError> { @@ -165,10 +168,26 @@ fn validate_account_range_response( let (keys, accounts) = response .accounts .iter() - .map(|unit| (unit.hash, unit.account)) + .map(|unit| { + ( + unit.hash, + AccountState::from(unit.account.clone()).encode_to_vec(), + ) + }) .unzip(); + let proof = response + .proof + .iter() + .map(|bytes| bytes.as_ref().to_vec()) + .collect(); + verify_range_proof( + request.root_hash, + request.starting_hash, + keys, + accounts, + proof, + )?; Ok(()) - // verify_range(origin, keys, accounts, nodes) } #[cfg(test)] diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 554cf81ea..8d3449e17 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -1,7 +1,4 @@ -use std::{ - cmp::{self, Ordering}, - collections::HashMap, -}; +use std::{cmp::Ordering, collections::HashMap}; use ethereum_types::H256; use sha3::{Digest, Keccak256}; diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 00f89a61e..72ea06817 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -23,9 +23,10 @@ use trie_iter::print_trie; pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; +pub use self::proof::verify_range_proof; pub use self::error::TrieError; -use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator, proof::verify_range_proof}; +use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator}; use lazy_static::lazy_static; From f7f0bdb4baac4cb8804fdf6039cdd9348daa2e7b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 11:55:36 -0300 Subject: [PATCH 180/345] Add proptest for special case: no keys --- crates/storage/trie/proof.rs | 34 +++++++++++++++++++++++++++++----- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 8d3449e17..5b42ccefd 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -42,7 +42,7 @@ pub fn verify_range_proof( // Verify ranges depending on the given proof - // Special Case A) No proofs given, the range is expected to be the full set of leaves + // Special Case: No proofs given, the range is expected to be the full set of leaves if proof.is_empty() { let mut trie = Trie::stateless(); for (index, key) in keys.iter().enumerate() { @@ -59,9 +59,7 @@ pub fn verify_range_proof( return Ok(false); } - let last_key = *keys.last().unwrap(); - - // Special Case B) One edge proof no range given, there are no more values in the trie + // Special Case: One edge proof, no range given, there are no more values in the trie if keys.is_empty() { let (has_right_element, value) = has_right_element(root, first_key.as_bytes(), &proof_nodes)?; @@ -72,7 +70,9 @@ pub fn verify_range_proof( } } - // Special Case C) There is only one element and the two edge keys are the same + let last_key = *keys.last().unwrap(); + + // Special Case: There is only one element and the two edge keys are the same if keys.len() == 1 && first_key == last_key { let (has_right_element, value) = has_right_element(root, first_key.as_bytes(), &proof_nodes)?; @@ -502,8 +502,11 @@ mod tests { verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } + // Proptests for verify_range_proof proptest! { + // Successful Cases + #[test] // Regular Case: Two Edge Proofs, both keys exist fn proptest_verify_range_regular_case(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { @@ -615,5 +618,26 @@ mod tests { // Verify the range proof verify_range_proof(root, keys[0], keys, values, proof).unwrap(); } + + #[test] + // Special Case: No values, one edge proof + fn proptest_verify_range_no_values(mut data in btree_set(vec(any::(), 32), 100..200)) { + // Remove the last element so we can use it as key for the proof of non-existance + let last_element = data.pop_last().unwrap(); + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Range is empty + let values = vec![]; + let keys = vec![]; + let first_key = H256::from_slice(&last_element); + // Generate proof (last element) + let proof = trie.get_proof(&last_element).unwrap(); + // Verify the range proof + verify_range_proof(root, first_key, keys, values, proof).unwrap(); + } } } From 0023b8a8bc956b454d91384f18dfc95b15292de4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 12:16:17 -0300 Subject: [PATCH 181/345] Fix --- crates/storage/trie/proof.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 5b42ccefd..957e5e023 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -141,8 +141,12 @@ fn has_right_element_inner( if n.choices[choice..].iter().any(|child| child.is_valid()) { Ok((true, vec![])) } else { - let node = proof_nodes.get_node(&n.choices[choice])?; - has_right_element_inner(&node, path, proof_nodes) + if n.choices[choice].is_valid() { + let node = proof_nodes.get_node(&n.choices[choice])?; + has_right_element_inner(&node, path, proof_nodes) + } else { + Ok((false, vec![])) + } } } else { Ok((false, n.value.clone())) From 8e8dba3f183a54db5a44bc1de82f563334583444 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 12:19:55 -0300 Subject: [PATCH 182/345] Fix logic --- crates/storage/trie/proof.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 957e5e023..cb19706bc 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -63,10 +63,12 @@ pub fn verify_range_proof( if keys.is_empty() { let (has_right_element, value) = has_right_element(root, first_key.as_bytes(), &proof_nodes)?; - if has_right_element || value.is_empty() { + if has_right_element || !value.is_empty() { return Err(TrieError::Verify(format!( "no keys returned but more are available on the trie" ))); + } else { + return Ok(false) } } From 1a534cc4aebdf93c2cc2a293b9826f88b4f162ad Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 12:26:20 -0300 Subject: [PATCH 183/345] Fix logic --- crates/storage/trie/proof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index cb19706bc..7ce1fc91c 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -140,7 +140,7 @@ fn has_right_element_inner( Node::Branch(ref n) => { // Check if there are children to the right side if let Some(choice) = path.next_choice() { - if n.choices[choice..].iter().any(|child| child.is_valid()) { + if n.choices[choice+1..].iter().any(|child| child.is_valid()) { Ok((true, vec![])) } else { if n.choices[choice].is_valid() { From 6a9db299b0dc6e42ee8fe9e4d0eaab681f0ac026 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 13:11:44 -0300 Subject: [PATCH 184/345] Refactor has_right_element --- crates/storage/trie/proof.rs | 112 ++++++++++++++++++++++++----------- 1 file changed, 79 insertions(+), 33 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 7ce1fc91c..0aaa1850d 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -41,10 +41,10 @@ pub fn verify_range_proof( } // Verify ranges depending on the given proof + let mut trie = Trie::stateless(); // Special Case: No proofs given, the range is expected to be the full set of leaves if proof.is_empty() { - let mut trie = Trie::stateless(); for (index, key) in keys.iter().enumerate() { // Ignore the error as we don't rely on a DB let _ = trie.insert(key.0.to_vec(), values[index].clone()); @@ -61,14 +61,15 @@ pub fn verify_range_proof( // Special Case: One edge proof, no range given, there are no more values in the trie if keys.is_empty() { - let (has_right_element, value) = - has_right_element(root, first_key.as_bytes(), &proof_nodes)?; + let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; + trie.root = Some(root.into()); + let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if has_right_element || !value.is_empty() { return Err(TrieError::Verify(format!( "no keys returned but more are available on the trie" ))); } else { - return Ok(false) + return Ok(false); } } @@ -76,8 +77,8 @@ pub fn verify_range_proof( // Special Case: There is only one element and the two edge keys are the same if keys.len() == 1 && first_key == last_key { - let (has_right_element, value) = - has_right_element(root, first_key.as_bytes(), &proof_nodes)?; + let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; + let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if first_key != keys[0] { return Err(TrieError::Verify(format!("correct proof but invalid key"))); } @@ -92,7 +93,6 @@ pub fn verify_range_proof( if first_key >= last_key { return Err(TrieError::Verify(format!("invalid edge keys"))); } - let mut trie = Trie::stateless(); let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; println!("FILL STATE"); @@ -120,56 +120,51 @@ pub fn verify_range_proof( } // Indicates where there exist more elements to the right side of the given path -// Also returns the value (or an empty value if it is not present on the trie) fn has_right_element( root_hash: H256, key: &[u8], - proof_nodes: &ProofNodeStorage, -) -> Result<(bool, Vec), TrieError> { + trie_state: &TrieState, +) -> Result { let path = Nibbles::from_bytes(key); - let node = proof_nodes.get_node(&root_hash.into())?; - has_right_element_inner(&node, path, proof_nodes) + has_right_element_inner(root_hash.into(), path, trie_state) } fn has_right_element_inner( - node: &Node, + node_hash: NodeHash, mut path: Nibbles, - proof_nodes: &ProofNodeStorage, -) -> Result<(bool, Vec), TrieError> { + trie_state: &TrieState, +) -> Result { + let Some(node) = trie_state.get_node(node_hash.clone())? else { + return Ok(false); + }; match node { Node::Branch(ref n) => { // Check if there are children to the right side if let Some(choice) = path.next_choice() { - if n.choices[choice+1..].iter().any(|child| child.is_valid()) { - Ok((true, vec![])) + if n.choices[choice + 1..].iter().any(|child| child.is_valid()) { + return Ok(true); } else { if n.choices[choice].is_valid() { - let node = proof_nodes.get_node(&n.choices[choice])?; - has_right_element_inner(&node, path, proof_nodes) - } else { - Ok((false, vec![])) + return has_right_element_inner( + n.choices[choice].clone(), + path, + trie_state, + ); } } - } else { - Ok((false, n.value.clone())) } } Node::Extension(n) => { if path.skip_prefix(&n.prefix) { - let node = proof_nodes.get_node(&n.child)?; - has_right_element_inner(&node, path, proof_nodes) + return has_right_element_inner(n.child, path, trie_state); } else { - Ok((n.prefix.as_ref() > path.as_ref(), vec![])) + return Ok(n.prefix.as_ref() > path.as_ref()); } } // We reached the end of the path - Node::Leaf(ref n) => { - let value = (path == n.partial) - .then_some(n.value.clone()) - .unwrap_or_default(); - Ok((false, value)) - } + Node::Leaf(_) => {} } + Ok(false) } fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option { @@ -212,7 +207,9 @@ fn fill_node( let value = match &node { Node::Branch(n) => n.value.clone(), Node::Extension(_) => vec![], - Node::Leaf(n) => n.value.clone(), + Node::Leaf(n) => (*path == n.partial) + .then_some(n.value.clone()) + .unwrap_or_default(), }; trie_state.insert_node(node, node_hash.clone()); Ok(value) @@ -508,6 +505,37 @@ mod tests { verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } + #[test] + // Special Case: One element range + fn jijo() { + let mut data = std::collections::BTreeSet::from([ + vec![ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, + ], + vec![ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, + ], + ]); + // Remove the last element so we can use it as key for the proof of non-existance + let last_element = data.pop_last().unwrap(); + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Range is empty + let values = vec![]; + let keys = vec![]; + let first_key = H256::from_slice(&last_element); + // Generate proof (last element) + let proof = trie.get_proof(&last_element).unwrap(); + // Verify the range proof + verify_range_proof(root, first_key, keys, values, proof).unwrap(); + } + // Proptests for verify_range_proof proptest! { @@ -645,5 +673,23 @@ mod tests { // Verify the range proof verify_range_proof(root, first_key, keys, values, proof).unwrap(); } + + #[test] + // Special Case: One element range + fn proptest_verify_range_one_element(data in btree_set(vec(any::(), 32), 200), start in 0_usize..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = vec![data.iter().collect::>()[start].clone()]; + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Generate proofs + let proof = trie.get_proof(&values[0]).unwrap(); + // Verify the range proof + verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + } } } From ff8878052a43f1d45c475dcc4878252a5d52cf43 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 13:20:25 -0300 Subject: [PATCH 185/345] Fix logic --- crates/storage/trie/proof.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 0aaa1850d..daa3b7f79 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -134,7 +134,7 @@ fn has_right_element_inner( mut path: Nibbles, trie_state: &TrieState, ) -> Result { - let Some(node) = trie_state.get_node(node_hash.clone())? else { + let Ok(Some(node)) = trie_state.get_node(node_hash.clone()) else { return Ok(false); }; match node { @@ -169,7 +169,14 @@ fn has_right_element_inner( fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option { match node { - Node::Branch(n) => path.next_choice().map(|i| n.choices[i].clone()), + Node::Branch(n) => { + if let Some(choice) = path.next_choice() { + if n.choices[choice].is_valid() { + return Some(n.choices[choice].clone()); + } + } + None + } Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(n.child.clone()), Node::Leaf(_) => None, } From 7583488f2a87a768c8b55950e4b7a632d3529cd2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 13:32:17 -0300 Subject: [PATCH 186/345] Unify criteria --- crates/storage/trie/proof.rs | 40 +++++------------------------------- 1 file changed, 5 insertions(+), 35 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index daa3b7f79..6e445d5a1 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -249,6 +249,9 @@ fn remove_internal_references_inner( mut right_path: Nibbles, trie_state: &mut TrieState, ) -> bool { + if !node_hash.is_valid() { + return true + } // We already looked up the nodes when filling the state so this shouldn't fail let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); match node { @@ -357,10 +360,8 @@ fn remove_node( if !node_hash.is_valid() { return false; } - // We already checked the canonical proof path when filling the state so this case should be unreachable - let Ok(Some(node)) = trie_state.get_node(node_hash.clone()) else { - return false; - }; + // We already looked up the nodes when filling the state so this shouldn't fail + let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); match node { Node::Branch(mut n) => { // Remove child nodes @@ -512,37 +513,6 @@ mod tests { verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); } - #[test] - // Special Case: One element range - fn jijo() { - let mut data = std::collections::BTreeSet::from([ - vec![ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, - ], - vec![ - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1, - ], - ]); - // Remove the last element so we can use it as key for the proof of non-existance - let last_element = data.pop_last().unwrap(); - // Build trie - let mut trie = Trie::new_temp(); - for val in data.iter() { - trie.insert(val.clone(), val.clone()).unwrap() - } - let root = trie.hash().unwrap(); - // Range is empty - let values = vec![]; - let keys = vec![]; - let first_key = H256::from_slice(&last_element); - // Generate proof (last element) - let proof = trie.get_proof(&last_element).unwrap(); - // Verify the range proof - verify_range_proof(root, first_key, keys, values, proof).unwrap(); - } - // Proptests for verify_range_proof proptest! { From 849a35417ef3e6508bd5f982ba702e2220704995 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:01:38 -0300 Subject: [PATCH 187/345] Refactor --- crates/storage/trie/nibbles.rs | 10 +++++++++ crates/storage/trie/proof.rs | 40 +++++++++++++++------------------- 2 files changed, 27 insertions(+), 23 deletions(-) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 8c7f4b33b..369e7c97a 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -1,3 +1,5 @@ +use std::cmp; + use ethereum_rust_rlp::{ decode::RLPDecode, encode::RLPEncode, @@ -56,6 +58,14 @@ impl Nibbles { } } + pub fn compare_prefix(&self, prefix: &Nibbles) -> cmp::Ordering { + if self.len() > prefix.len() { + self.data[..prefix.len()].cmp(&prefix.data) + } else { + self.data.cmp(&prefix.data) + } + } + /// Compares self to another and returns the shared nibble count (amount of nibbles that are equal, from the start) pub fn count_prefix(&self, other: &Nibbles) -> usize { self.as_ref() diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 6e445d5a1..38e81c6ed 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -250,7 +250,7 @@ fn remove_internal_references_inner( trie_state: &mut TrieState, ) -> bool { if !node_hash.is_valid() { - return true + return true; } // We already looked up the nodes when filling the state so this shouldn't fail let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); @@ -301,28 +301,20 @@ fn remove_internal_references_inner( } Node::Extension(n) => { // Compare left and right paths against prefix - let compare_path = |path: &Nibbles, prefix: &Nibbles| -> Ordering { - if path.len() > prefix.len() { - path.as_ref()[..prefix.len()].cmp(prefix.as_ref()) - } else { - path.as_ref().cmp(prefix.as_ref()) - } - }; - let left_fork = compare_path(&left_path, &n.prefix); - let right_fork = compare_path(&right_path, &n.prefix); + let left_fork = left_path.compare_prefix(&n.prefix); + let right_fork = right_path.compare_prefix(&n.prefix); - if left_fork.is_eq() && right_fork.is_eq() { - // If both paths contain the same prefix as the extension node, keep going - return remove_internal_references_inner( - n.child, - left_path.offset(n.prefix.len()), - right_path.offset(n.prefix.len()), - trie_state, - ); - } - // We found our fork node, now we can remove the internal references match (left_fork, right_fork) { + // If both paths contain the same prefix as the extension node, keep going + (Ordering::Equal, Ordering::Equal) => { + return remove_internal_references_inner( + n.child, + left_path.offset(n.prefix.len()), + right_path.offset(n.prefix.len()), + trie_state, + ); + } // If both paths are greater or lesser than the node's prefix then the range is empty // TODO: return the error instead of panicking here (Ordering::Greater, Ordering::Greater) | (Ordering::Less, Ordering::Less) => { @@ -342,7 +334,9 @@ fn remove_internal_references_inner( } } } - Node::Leaf(_) => todo!(), + // This case should be unreachable as we checked that left_path != right_path + // before calling this function + Node::Leaf(_) => {} } false } @@ -388,8 +382,8 @@ fn remove_node( // If no child subtrie would result from this process remove the node entirely // (Such as removing the left side of a trie with no right side) if !path.skip_prefix(&n.prefix) { - if (remove_left && n.prefix.as_ref() < path.as_ref()) - || (!remove_left && n.prefix.as_ref() > path.as_ref()) + if (remove_left && path.compare_prefix(&n.prefix).is_gt()) + || !remove_left && path.compare_prefix(&n.prefix).is_lt() { return true; } From 0908bd56f0fd011dafdf297e454f03def2f35dc9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:03:03 -0300 Subject: [PATCH 188/345] Clippy --- crates/storage/trie/proof.rs | 28 ++++++++++++---------------- crates/storage/trie/trie.rs | 1 - crates/storage/trie/trie_iter.rs | 2 +- 3 files changed, 13 insertions(+), 18 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 38e81c6ed..e6626f6e1 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -34,7 +34,7 @@ pub fn verify_range_proof( } } // Check for empty values - if values.iter().find(|value| value.is_empty()).is_some() { + if values.iter().any(|value| value.is_empty()) { return Err(TrieError::Verify(String::from( "value range contains empty value", ))); @@ -65,9 +65,7 @@ pub fn verify_range_proof( trie.root = Some(root.into()); let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if has_right_element || !value.is_empty() { - return Err(TrieError::Verify(format!( - "no keys returned but more are available on the trie" - ))); + return Err(TrieError::Verify("no keys returned but more are available on the trie".to_string())); } else { return Ok(false); } @@ -80,10 +78,10 @@ pub fn verify_range_proof( let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if first_key != keys[0] { - return Err(TrieError::Verify(format!("correct proof but invalid key"))); + return Err(TrieError::Verify("correct proof but invalid key".to_string())); } if value != values[0] { - return Err(TrieError::Verify(format!("correct proof but invalid data"))); + return Err(TrieError::Verify("correct proof but invalid data".to_string())); } return Ok(has_right_element); } @@ -91,7 +89,7 @@ pub fn verify_range_proof( // Regular Case // Here we will have two edge proofs if first_key >= last_key { - return Err(TrieError::Verify(format!("invalid edge keys"))); + return Err(TrieError::Verify("invalid edge keys".to_string())); } let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; @@ -143,14 +141,12 @@ fn has_right_element_inner( if let Some(choice) = path.next_choice() { if n.choices[choice + 1..].iter().any(|child| child.is_valid()) { return Ok(true); - } else { - if n.choices[choice].is_valid() { - return has_right_element_inner( - n.choices[choice].clone(), - path, - trie_state, - ); - } + } else if n.choices[choice].is_valid() { + return has_right_element_inner( + n.choices[choice].clone(), + path, + trie_state, + ); } } } @@ -526,7 +522,7 @@ mod tests { let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); // Generate proofs let mut proof = trie.get_proof(&values[0]).unwrap(); - proof.extend(trie.get_proof(&values.last().unwrap()).unwrap()); + proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); // Verify the range proof verify_range_proof(root, keys[0], keys, values, proof).unwrap(); } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 72ea06817..700237813 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -17,7 +17,6 @@ use nibbles::Nibbles; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; -use trie_iter::print_trie; #[cfg(feature = "libmdbx")] pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 9a90acc2a..832fe2629 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -69,7 +69,7 @@ impl TrieIterator { pub fn print_trie(trie: &Trie) { let Some(root) = &trie.root else { return }; print_node(trie, root.clone()); - print!("\n") + println!() } pub fn print_node(trie: &Trie, node_hash: NodeHash) { From c445edc18f6b18c6c5ba87169d1e4177503f8e05 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:17:48 -0300 Subject: [PATCH 189/345] Add doc --- crates/storage/trie/proof.rs | 66 +++++++++++++++++++----------------- 1 file changed, 34 insertions(+), 32 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index e6626f6e1..c6db3b097 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -4,11 +4,11 @@ use ethereum_types::H256; use sha3::{Digest, Keccak256}; use crate::{ - nibbles::Nibbles, node::Node, node_hash::NodeHash, state::TrieState, trie_iter::print_trie, - Trie, TrieError, ValueRLP, + nibbles::Nibbles, node::Node, node_hash::NodeHash, state::TrieState, Trie, TrieError, ValueRLP, }; -/// The boolean indicates if there is more state to be fetched +/// Verifies that the key value range belongs to the trie with the given root given the edge proofs for the range +/// Also returns true if there is more state to be fetched (aka if there are more keys to the right of the given range) pub fn verify_range_proof( root: H256, first_key: H256, @@ -18,6 +18,7 @@ pub fn verify_range_proof( ) -> Result { // Store proof nodes by hash let proof_nodes = ProofNodeStorage::from_proof(&proof); + // Validate range if keys.len() != values.len() { return Err(TrieError::Verify(format!( "inconsistent proof data, got {} keys and {} values", @@ -45,11 +46,11 @@ pub fn verify_range_proof( // Special Case: No proofs given, the range is expected to be the full set of leaves if proof.is_empty() { + // Check that the trie constructed from the given keys and values has the expected root for (index, key) in keys.iter().enumerate() { - // Ignore the error as we don't rely on a DB - let _ = trie.insert(key.0.to_vec(), values[index].clone()); + trie.insert(key.0.to_vec(), values[index].clone())?; } - let hash = trie.hash().unwrap_or_default(); + let hash = trie.hash()?; if hash != root { return Err(TrieError::Verify(format!( "invalid proof, expected root hash {}, got {}", @@ -61,11 +62,15 @@ pub fn verify_range_proof( // Special Case: One edge proof, no range given, there are no more values in the trie if keys.is_empty() { + // We need to check that the proof confirms the non-existance of the first key + // and that there are no more elements to the right of the first key let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; trie.root = Some(root.into()); let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if has_right_element || !value.is_empty() { - return Err(TrieError::Verify("no keys returned but more are available on the trie".to_string())); + return Err(TrieError::Verify( + "no keys returned but more are available on the trie".to_string(), + )); } else { return Ok(false); } @@ -75,46 +80,47 @@ pub fn verify_range_proof( // Special Case: There is only one element and the two edge keys are the same if keys.len() == 1 && first_key == last_key { + // We need to check that the proof confirms the existance of the first key let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; - let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if first_key != keys[0] { - return Err(TrieError::Verify("correct proof but invalid key".to_string())); + return Err(TrieError::Verify( + "correct proof but invalid key".to_string(), + )); } if value != values[0] { - return Err(TrieError::Verify("correct proof but invalid data".to_string())); + return Err(TrieError::Verify( + "correct proof but invalid data".to_string(), + )); } - return Ok(has_right_element); + return has_right_element(root, first_key.as_bytes(), &trie.state); } - // Regular Case - // Here we will have two edge proofs + // Regular Case: Two edge proofs if first_key >= last_key { return Err(TrieError::Verify("invalid edge keys".to_string())); } + // Fill up the state with the nodes from the proof let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; - println!("FILL STATE"); - print_trie(&trie); + // Remove all references to the internal nodes that belong to the range so they can be reconstructed let empty = remove_internal_references(root, first_key, last_key, &mut trie.state); if !empty { trie.root = Some(NodeHash::from(root)); } - println!("REMOVE INTERNAL REFERENCES"); - print_trie(&trie); - println!("KEY RANGE INSERT"); + // Reconstruct the internal nodes by inserting the elements on the range for (i, key) in keys.iter().enumerate() { trie.insert(key.0.to_vec(), values[i].clone())?; } - // TODO: has_right_element - assert_eq!(trie.hash().unwrap(), root); - - // Use first proof to build node path - // use first proof root + second proof to complete it - // Remove internal references - // Add keys & values from range - // Check root + // Check that the hash is the one we expected (aka the trie was properly reconstructed from the edge proofs and the range) + let hash = trie.hash()?; + if hash != root { + return Err(TrieError::Verify(format!( + "invalid proof, expected root hash {}, got {}", + root, hash + ))); + } - Ok(true) + return has_right_element(root, first_key.as_bytes(), &trie.state); } // Indicates where there exist more elements to the right side of the given path @@ -142,11 +148,7 @@ fn has_right_element_inner( if n.choices[choice + 1..].iter().any(|child| child.is_valid()) { return Ok(true); } else if n.choices[choice].is_valid() { - return has_right_element_inner( - n.choices[choice].clone(), - path, - trie_state, - ); + return has_right_element_inner(n.choices[choice].clone(), path, trie_state); } } } From ec13ce55a561c54c897e7ad4436e465b0b79931a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:25:31 -0300 Subject: [PATCH 190/345] Reorder code + add doc --- crates/storage/trie/proof.rs | 130 +++++++++++++++++++---------------- 1 file changed, 70 insertions(+), 60 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index c6db3b097..3b865dcfe 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -100,8 +100,8 @@ pub fn verify_range_proof( return Err(TrieError::Verify("invalid edge keys".to_string())); } // Fill up the state with the nodes from the proof - let _ = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; - let _ = fill_state(&mut trie.state, root, last_key, &proof_nodes)?; + fill_state(&mut trie.state, root, first_key, &proof_nodes)?; + fill_state(&mut trie.state, root, last_key, &proof_nodes)?; // Remove all references to the internal nodes that belong to the range so they can be reconstructed let empty = remove_internal_references(root, first_key, last_key, &mut trie.state); if !empty { @@ -123,64 +123,8 @@ pub fn verify_range_proof( return has_right_element(root, first_key.as_bytes(), &trie.state); } -// Indicates where there exist more elements to the right side of the given path -fn has_right_element( - root_hash: H256, - key: &[u8], - trie_state: &TrieState, -) -> Result { - let path = Nibbles::from_bytes(key); - has_right_element_inner(root_hash.into(), path, trie_state) -} - -fn has_right_element_inner( - node_hash: NodeHash, - mut path: Nibbles, - trie_state: &TrieState, -) -> Result { - let Ok(Some(node)) = trie_state.get_node(node_hash.clone()) else { - return Ok(false); - }; - match node { - Node::Branch(ref n) => { - // Check if there are children to the right side - if let Some(choice) = path.next_choice() { - if n.choices[choice + 1..].iter().any(|child| child.is_valid()) { - return Ok(true); - } else if n.choices[choice].is_valid() { - return has_right_element_inner(n.choices[choice].clone(), path, trie_state); - } - } - } - Node::Extension(n) => { - if path.skip_prefix(&n.prefix) { - return has_right_element_inner(n.child, path, trie_state); - } else { - return Ok(n.prefix.as_ref() > path.as_ref()); - } - } - // We reached the end of the path - Node::Leaf(_) => {} - } - Ok(false) -} - -fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option { - match node { - Node::Branch(n) => { - if let Some(choice) = path.next_choice() { - if n.choices[choice].is_valid() { - return Some(n.choices[choice].clone()); - } - } - None - } - Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(n.child.clone()), - Node::Leaf(_) => None, - } -} - /// Fills up the TrieState with nodes from the proof traversing the path given by first_key +/// Returns an error if there are gaps in the proof node path /// Also returns the value if it is part of the proof fn fill_state( trie_state: &mut TrieState, @@ -197,6 +141,9 @@ fn fill_state( ) } +/// Fills up the TrieState with nodes from the proof traversing the path given by first_key +/// Returns an error if there are gaps in the proof node path +/// Also returns the value if it is part of the proof fn fill_node( path: &mut Nibbles, node_hash: &NodeHash, @@ -221,6 +168,67 @@ fn fill_node( } } +/// Returns the node hash of the node's child (if any) following the given path +fn get_child<'a>(path: &'a mut Nibbles, node: &'a Node) -> Option { + match node { + Node::Branch(n) => { + if let Some(choice) = path.next_choice() { + if n.choices[choice].is_valid() { + return Some(n.choices[choice].clone()); + } + } + None + } + Node::Extension(n) => path.skip_prefix(&n.prefix).then_some(n.child.clone()), + Node::Leaf(_) => None, + } +} + +/// Returns true if the trie contains elements to the right of the given key +/// (Aka if the given key is not the edge key of the trie) +fn has_right_element( + root_hash: H256, + key: &[u8], + trie_state: &TrieState, +) -> Result { + let path = Nibbles::from_bytes(key); + has_right_element_inner(root_hash.into(), path, trie_state) +} + +/// Returns true if the node's subtrie contains elements to the right of the given key +/// (Aka if the given key is not the edge key of the subtrie) +fn has_right_element_inner( + node_hash: NodeHash, + mut path: Nibbles, + trie_state: &TrieState, +) -> Result { + let Ok(Some(node)) = trie_state.get_node(node_hash.clone()) else { + return Ok(false); + }; + match node { + Node::Branch(ref n) => { + // Check if there are children to the right side + if let Some(choice) = path.next_choice() { + if n.choices[choice + 1..].iter().any(|child| child.is_valid()) { + return Ok(true); + } else if n.choices[choice].is_valid() { + return has_right_element_inner(n.choices[choice].clone(), path, trie_state); + } + } + } + Node::Extension(n) => { + if path.skip_prefix(&n.prefix) { + return has_right_element_inner(n.child, path, trie_state); + } else { + return Ok(n.prefix.as_ref() > path.as_ref()); + } + } + // We reached the end of the path + Node::Leaf(_) => {} + } + Ok(false) +} + /// Removes references to internal nodes between the left and right key /// These nodes should be entirely reconstructed when inserting the elements between left and right key (the proven range) /// Returns true if the trie is left empty (rootless) as a result of this process @@ -395,11 +403,13 @@ fn remove_node( false } +/// An intermediate storage for proof nodes, containing encoded nodes indexed by hash struct ProofNodeStorage<'a> { nodes: HashMap, &'a Vec>, } impl<'a> ProofNodeStorage<'a> { + // Construct a ProofNodeStorage for a proof fn from_proof(proof: &'a Vec>) -> Self { Self { nodes: proof @@ -408,7 +418,7 @@ impl<'a> ProofNodeStorage<'a> { .collect::>(), } } - + // Fetch a node by its hash, return an error if the node is not present or badly encoded fn get_node(&self, hash: &NodeHash) -> Result { let encoded = match hash { NodeHash::Hashed(hash) => { From 2445f53cf262e4b712c743df409c67a120381904 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:49:30 -0300 Subject: [PATCH 191/345] Fix logic --- crates/storage/trie/proof.rs | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 3b865dcfe..e0f98fa53 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -111,6 +111,8 @@ pub fn verify_range_proof( for (i, key) in keys.iter().enumerate() { trie.insert(key.0.to_vec(), values[i].clone())?; } + // Check for elements to the right of the range before we wipe the sate + let has_right_element = has_right_element(root, last_key.as_bytes(), &trie.state)?; // Check that the hash is the one we expected (aka the trie was properly reconstructed from the edge proofs and the range) let hash = trie.hash()?; if hash != root { @@ -119,8 +121,7 @@ pub fn verify_range_proof( root, hash ))); } - - return has_right_element(root, first_key.as_bytes(), &trie.state); + Ok(has_right_element) } /// Fills up the TrieState with nodes from the proof traversing the path given by first_key @@ -456,7 +457,9 @@ mod tests { let root = trie.hash().unwrap(); let keys = (50_u8..=75).map(|i| H256([i; 32])).collect::>(); let values = (50_u8..=75).map(|i| [i; 32].to_vec()).collect::>(); - verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + // Our trie contains more elements to the right + assert!(fetch_more) } #[test] @@ -500,8 +503,8 @@ mod tests { .iter() .map(|addr| addr.0.to_vec()) .collect::>(); - let key_range = account_addresses[7..=17].to_vec(); - let value_range = account_addresses[7..=17] + let keys = account_addresses[7..=17].to_vec(); + let values = account_addresses[7..=17] .iter() .map(|v| v.0.to_vec()) .collect::>(); @@ -512,7 +515,9 @@ mod tests { let mut proof = trie.get_proof(&trie_values[7]).unwrap(); proof.extend(trie.get_proof(&trie_values[17]).unwrap()); let root = trie.hash().unwrap(); - verify_range_proof(root, key_range[0], key_range, value_range, proof).unwrap(); + let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + // Our trie contains more elements to the right + assert!(fetch_more) } // Proptests for verify_range_proof @@ -671,4 +676,6 @@ mod tests { verify_range_proof(root, keys[0], keys, values, proof).unwrap(); } } + + // Unsuccesful Cases } From 16cb1b62a2eb395dbf2c9dece5936fbb7d04793d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 14:59:39 -0300 Subject: [PATCH 192/345] Check result in tests --- crates/storage/trie/proof.rs | 42 +++++++++++++++++++++++++++--------- 1 file changed, 32 insertions(+), 10 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index e0f98fa53..3ecd0bbce 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -541,7 +541,14 @@ mod tests { let mut proof = trie.get_proof(&values[0]).unwrap(); proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); // Verify the range proof - verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + if end == 199 { + // The last key is at the edge of the trie + assert!(!fetch_more) + } else { + // Our trie contains more elements to the right + assert!(fetch_more) + } } #[test] @@ -577,7 +584,9 @@ mod tests { let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + // Our trie contains more elements to the right + assert!(fetch_more) } #[test] @@ -606,17 +615,19 @@ mod tests { return Ok(()); } } else { - first_key[31] -=1; - if first_key == data[start -1] { - // Skip test - return Ok(()); + first_key[31] -=1; + if first_key == data[start -1] { + // Skip test + return Ok(()); } } // Generate proofs let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + // Our trie contains more elements to the right + assert!(fetch_more) } #[test] @@ -634,7 +645,9 @@ mod tests { // The keyset contains the entire trie so we don't need edge proofs let proof = vec![]; // Verify the range proof - verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + // Our range is the full leafset, there shouldn't be more values left in the trie + assert!(!fetch_more) } #[test] @@ -655,7 +668,9 @@ mod tests { // Generate proof (last element) let proof = trie.get_proof(&last_element).unwrap(); // Verify the range proof - verify_range_proof(root, first_key, keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, first_key, keys, values, proof).unwrap(); + // There are no more elements to the right of the range + assert!(!fetch_more) } #[test] @@ -673,7 +688,14 @@ mod tests { // Generate proofs let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + if start == 199 { + // The last key is at the edge of the trie + assert!(!fetch_more) + } else { + // Our trie contains more elements to the right + assert!(fetch_more) + } } } From 8151cef0ca25dc49f8baceb6518187f76539aff3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 15:19:09 -0300 Subject: [PATCH 193/345] Add tests for unsuccesful cases --- crates/storage/trie/proof.rs | 41 +++++++++++++++++++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 3ecd0bbce..1119d3422 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -697,7 +697,46 @@ mod tests { assert!(fetch_more) } } - } // Unsuccesful Cases + + #[test] + // Regular Case: Only one edge proof, both keys exist + fn proptest_verify_range_regular_case_only_one_edge_proof(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>()[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Generate proofs (only prove first key) + let proof = trie.get_proof(&values[0]).unwrap(); + // Verify the range proof + assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + } + + #[test] + // Regular Case: Two Edge Proofs, both keys exist, but there is a missing node in the proof + fn proptest_verify_range_regular_case_gap_in_proof(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>()[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Generate proofs + let mut proof = trie.get_proof(&values[0]).unwrap(); + proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); + // Remove the last node of the second proof (to make sure we don't remove a node that is also part of the first proof) + proof.pop(); + // Verify the range proof + assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + } + } } From a7d5415474626efbf131cdd92641bff01f1d571a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 15:27:52 -0300 Subject: [PATCH 194/345] Add tests for unsuccesful cases --- crates/storage/trie/proof.rs | 61 +++++++++++++++++++++++++++++++++++- 1 file changed, 60 insertions(+), 1 deletion(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 1119d3422..9ebc1f38a 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -651,7 +651,7 @@ mod tests { } #[test] - // Special Case: No values, one edge proof + // Special Case: No values, one edge proof (of non-existance) fn proptest_verify_range_no_values(mut data in btree_set(vec(any::(), 32), 100..200)) { // Remove the last element so we can use it as key for the proof of non-existance let last_element = data.pop_last().unwrap(); @@ -738,5 +738,64 @@ mod tests { // Verify the range proof assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); } + + #[test] + // Regular Case: No proofs both keys exist + fn proptest_verify_range_regular_case_no_proofs(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>()[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Dont generate proof + let proof = vec![]; + // Verify the range proof + assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + } + + #[test] + // Special Case: No values, one edge proof (of existance) + fn proptest_verify_range_no_values_proof_of_existance(data in btree_set(vec(any::(), 32), 100..200)) { + // Fetch the last element so we can use it as key for the proof + let last_element = data.last().unwrap(); + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Range is empty + let values = vec![]; + let keys = vec![]; + let first_key = H256::from_slice(&last_element); + // Generate proof (last element) + let proof = trie.get_proof(&last_element).unwrap(); + // Verify the range proof + assert!(verify_range_proof(root, first_key, keys, values, proof).is_err()); + } + + #[test] + // Special Case: One element range (but the proof is of nonexistance) + fn proptest_verify_range_one_element_bad_proof(data in btree_set(vec(any::(), 32), 200), start in 0_usize..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = vec![data.iter().collect::>()[start].clone()]; + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Remove the value to generate a proof of non-existance + trie.remove(values[0].clone()).unwrap(); + // Generate proofs + let proof = trie.get_proof(&values[0]).unwrap(); + // Verify the range proof + assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + } } } From 6c9dd0133dfeea9ae8541a5d0f42da85a9949b01 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 15:57:32 -0300 Subject: [PATCH 195/345] Add error handling --- crates/storage/trie/proof.rs | 32 +++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/proof.rs index 9ebc1f38a..542c974a4 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/proof.rs @@ -103,7 +103,7 @@ pub fn verify_range_proof( fill_state(&mut trie.state, root, first_key, &proof_nodes)?; fill_state(&mut trie.state, root, last_key, &proof_nodes)?; // Remove all references to the internal nodes that belong to the range so they can be reconstructed - let empty = remove_internal_references(root, first_key, last_key, &mut trie.state); + let empty = remove_internal_references(root, first_key, last_key, &mut trie.state)?; if !empty { trie.root = Some(NodeHash::from(root)); } @@ -233,12 +233,13 @@ fn has_right_element_inner( /// Removes references to internal nodes between the left and right key /// These nodes should be entirely reconstructed when inserting the elements between left and right key (the proven range) /// Returns true if the trie is left empty (rootless) as a result of this process +/// Asumes that left_key & right_key are not equal and of same length fn remove_internal_references( root_hash: H256, left_key: H256, right_key: H256, trie_state: &mut TrieState, -) -> bool { +) -> Result { // First find the node at which the left and right path differ let left_path = Nibbles::from_bytes(&left_key.0); let right_path = Nibbles::from_bytes(&right_key.0); @@ -250,21 +251,27 @@ fn remove_internal_references( /// Once the paths diverge, removes the nodes between the left and right path /// Returns true if the given node was completely removed as a result of this process /// In which case the caller should remove the reference to this node from its parent node +/// Asumes that left_key & right_key are not equal and of same length fn remove_internal_references_inner( node_hash: NodeHash, mut left_path: Nibbles, mut right_path: Nibbles, trie_state: &mut TrieState, -) -> bool { +) -> Result { if !node_hash.is_valid() { - return true; + return Ok(true); } // We already looked up the nodes when filling the state so this shouldn't fail - let node = trie_state.get_node(node_hash.clone()).unwrap().unwrap(); + let node = trie_state.get_node(node_hash.clone())?.unwrap(); match node { Node::Branch(mut n) => { + // If none of the paths have next choice nibble then it means that this is the end of the path + // which would mean that both paths are equal, which we already checked before + // Only one path doesn't have a next choice then it would mean that the paths have different lengths, + // which we also checked before calling this function let left_choice = left_path.next_choice().unwrap(); let right_choice = right_path.next_choice().unwrap(); + if left_choice == right_choice && n.choices[left_choice].is_valid() { // Keep going // Check if the child extension node should be removed as a result of this process @@ -273,7 +280,7 @@ fn remove_internal_references_inner( left_path, right_path, trie_state, - ); + )?; if should_remove { // Remove child node n.choices[left_choice] = NodeHash::default(); @@ -325,19 +332,19 @@ fn remove_internal_references_inner( // If both paths are greater or lesser than the node's prefix then the range is empty // TODO: return the error instead of panicking here (Ordering::Greater, Ordering::Greater) | (Ordering::Less, Ordering::Less) => { - panic!("empty range") + return Err(TrieError::Verify("empty range".to_string())) } // None of the paths fit the prefix, remove the entire subtrie (left, right) if left.is_ne() && right.is_ne() => { // Return true so that the parent node removes this node - return true; + return Ok(true); } // One path fits the prefix, the other one doesn't (left, right) => { // Remove the nodes from the child's subtrie let path = if left.is_eq() { left_path } else { right_path }; // Propagate the response so that this node will be removed too if the child's subtrie is wiped - return remove_node(node_hash, path, right.is_eq(), trie_state); + return Ok(remove_node(node_hash, path, right.is_eq(), trie_state)); } } } @@ -345,7 +352,7 @@ fn remove_internal_references_inner( // before calling this function Node::Leaf(_) => {} } - false + Ok(false) } // Removes all nodes in the node's subtrie to the left or right of the path (given by the `remove_left` flag) @@ -366,7 +373,10 @@ fn remove_node( match node { Node::Branch(mut n) => { // Remove child nodes - let choice = path.next_choice().unwrap(); + let Some(choice) = path.next_choice() else { + // Path ends in the branch node + return true; + }; if remove_left { for child in &mut n.choices[..choice] { *child = NodeHash::default() From ad0f921cb9e330e79fe1d7479eff628dde5d1363 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:06:24 -0300 Subject: [PATCH 196/345] [Postpone] changes not related to the trie --- crates/networking/p2p/Cargo.toml | 1 - crates/networking/p2p/rlpx/error.rs | 2 -- crates/networking/p2p/rlpx/snap.rs | 2 +- crates/networking/p2p/snap.rs | 37 ----------------------------- 4 files changed, 1 insertion(+), 41 deletions(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index 2fbdae71a..128cff7e7 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -9,7 +9,6 @@ edition = "2021" ethereum_rust-core.workspace = true ethereum_rust-rlp.workspace = true ethereum_rust-storage.workspace = true -ethereum_rust-trie.workspace = true tracing.workspace = true tokio.workspace = true diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 479a594ad..71cee5182 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -32,8 +32,6 @@ pub(crate) enum RLPxError { StoreError(#[from] StoreError), #[error("Error in cryptographic library: {0}")] CryptographyError(String), - #[error(transparent)] - Trie(#[from] TrieError), #[error("Failed to broadcast msg: {0}")] BroadcastError(String), } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index f83ada6b9..abec6c9f1 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -326,7 +326,7 @@ pub struct AccountRangeUnit { pub account: AccountStateSlim, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct AccountStateSlim { pub nonce: u64, pub balance: U256, diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 5017becba..fefd7374b 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,8 +1,6 @@ use bytes::Bytes; -use ethereum_rust_core::types::AccountState; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use ethereum_rust_trie::verify_range_proof; use crate::rlpx::{ error::RLPxError, @@ -12,8 +10,6 @@ use crate::rlpx::{ }, }; -// Request Processing - pub fn process_account_range_request( request: GetAccountRange, store: Store, @@ -157,39 +153,6 @@ pub fn process_trie_nodes_request( }) } -// Response Processing - -#[allow(unused)] -pub fn validate_account_range_response( - request: &GetAccountRange, - response: &AccountRange, -) -> Result<(), RLPxError> { - // Reconstruct a partial trie from the response and verify it - let (keys, accounts) = response - .accounts - .iter() - .map(|unit| { - ( - unit.hash, - AccountState::from(unit.account.clone()).encode_to_vec(), - ) - }) - .unzip(); - let proof = response - .proof - .iter() - .map(|bytes| bytes.as_ref().to_vec()) - .collect(); - verify_range_proof( - request.root_hash, - request.starting_hash, - keys, - accounts, - proof, - )?; - Ok(()) -} - #[cfg(test)] mod tests { use std::str::FromStr; From 4fc52447386f69e3acd6a14bf6e54740e596d61d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:07:47 -0300 Subject: [PATCH 197/345] Fix --- crates/networking/p2p/rlpx/error.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 71cee5182..d87019432 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,6 +1,5 @@ use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; use ethereum_rust_storage::error::StoreError; -use ethereum_rust_trie::TrieError; use thiserror::Error; // TODO improve errors From dab6172c1dc5502126aa706e9e776ceff8272431 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:08:29 -0300 Subject: [PATCH 198/345] Remove debug code --- crates/storage/trie/trie_iter.rs | 30 ------------------------------ 1 file changed, 30 deletions(-) diff --git a/crates/storage/trie/trie_iter.rs b/crates/storage/trie/trie_iter.rs index 832fe2629..5b90ddd13 100644 --- a/crates/storage/trie/trie_iter.rs +++ b/crates/storage/trie/trie_iter.rs @@ -66,36 +66,6 @@ impl TrieIterator { } } -pub fn print_trie(trie: &Trie) { - let Some(root) = &trie.root else { return }; - print_node(trie, root.clone()); - println!() -} - -pub fn print_node(trie: &Trie, node_hash: NodeHash) { - if trie.state.get_node(node_hash.clone()).unwrap().is_none() { - print!("[MISSING NODE]"); - return; - } - match trie.state.get_node(node_hash).unwrap().unwrap() { - Node::Branch(n) => { - print!("Branch{:?} [", n.value); - for (i, child) in n.choices.iter().enumerate() { - if child.is_valid() { - print!(" {i}: "); - print_node(trie, child.clone()); - } - } - print!(" ]") - } - Node::Extension(n) => { - print!("Ext{:?} -> ", n.prefix.as_ref()); - print_node(trie, n.child); - } - Node::Leaf(n) => print!("Leaf({:?})", n.value[0]), - } -} - #[cfg(test)] mod tests { From d02c3743cf4385d29d286990e9adb0de0f1db0c5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:10:24 -0300 Subject: [PATCH 199/345] Rename fn and module --- crates/storage/trie/trie.rs | 4 +-- .../trie/{proof.rs => verify_range.rs} | 34 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) rename crates/storage/trie/{proof.rs => verify_range.rs} (96%) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index ae350ffc6..2384cf1be 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -2,7 +2,7 @@ mod db; mod error; mod node; mod node_hash; -mod proof; +mod verify_range; mod rlp; mod state; mod trie_iter; @@ -20,7 +20,7 @@ use sha3::{Digest, Keccak256}; pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; -pub use self::proof::verify_range_proof; +pub use self::verify_range::verify_range; pub use self::error::TrieError; use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator}; diff --git a/crates/storage/trie/proof.rs b/crates/storage/trie/verify_range.rs similarity index 96% rename from crates/storage/trie/proof.rs rename to crates/storage/trie/verify_range.rs index 1aec714f5..e6a1604c1 100644 --- a/crates/storage/trie/proof.rs +++ b/crates/storage/trie/verify_range.rs @@ -10,7 +10,7 @@ use crate::{ /// Verifies that the key value range belongs to the trie with the given root given the edge proofs for the range /// Also returns true if there is more state to be fetched (aka if there are more keys to the right of the given range) -pub fn verify_range_proof( +pub fn verify_range( root: H256, first_key: H256, keys: Vec, @@ -455,7 +455,7 @@ mod tests { use std::str::FromStr; #[test] - fn verify_range_proof_regular_case_only_branch_nodes() { + fn verify_range_regular_case_only_branch_nodes() { // The trie will have keys and values ranging from 25-100 // We will prove the range from 50-75 // Note values are written as hashes in the form i -> [i;32] @@ -468,13 +468,13 @@ mod tests { let root = trie.hash().unwrap(); let keys = (50_u8..=75).map(|i| H256([i; 32])).collect::>(); let values = (50_u8..=75).map(|i| [i; 32].to_vec()).collect::>(); - let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } #[test] - fn verify_range_proof_regular_case() { + fn verify_range_regular_case() { // The account ranges were taken form a hive test state, but artificially modified // so that the resulting trie has a wide variety of different nodes (and not only branches) let account_addresses: [&str; 26] = [ @@ -526,12 +526,12 @@ mod tests { let mut proof = trie.get_proof(&trie_values[7]).unwrap(); proof.extend(trie.get_proof(&trie_values[17]).unwrap()); let root = trie.hash().unwrap(); - let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } - // Proptests for verify_range_proof + // Proptests for verify_range proptest! { // Successful Cases @@ -552,7 +552,7 @@ mod tests { let mut proof = trie.get_proof(&values[0]).unwrap(); proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); // Verify the range proof - let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); if end == 199 { // The last key is at the edge of the trie assert!(!fetch_more) @@ -595,7 +595,7 @@ mod tests { let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - let fetch_more = verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -636,7 +636,7 @@ mod tests { let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - let fetch_more = verify_range_proof(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -656,7 +656,7 @@ mod tests { // The keyset contains the entire trie so we don't need edge proofs let proof = vec![]; // Verify the range proof - let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); // Our range is the full leafset, there shouldn't be more values left in the trie assert!(!fetch_more) } @@ -679,7 +679,7 @@ mod tests { // Generate proof (last element) let proof = trie.get_proof(&last_element).unwrap(); // Verify the range proof - let fetch_more = verify_range_proof(root, first_key, keys, values, proof).unwrap(); + let fetch_more = verify_range(root, first_key, keys, values, proof).unwrap(); // There are no more elements to the right of the range assert!(!fetch_more) } @@ -699,7 +699,7 @@ mod tests { // Generate proofs let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - let fetch_more = verify_range_proof(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); if start == 199 { // The last key is at the edge of the trie assert!(!fetch_more) @@ -726,7 +726,7 @@ mod tests { // Generate proofs (only prove first key) let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, keys[0], keys, values, proof).is_err()); } #[test] @@ -747,7 +747,7 @@ mod tests { // Remove the last node of the second proof (to make sure we don't remove a node that is also part of the first proof) proof.pop(); // Verify the range proof - assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, keys[0], keys, values, proof).is_err()); } #[test] @@ -765,7 +765,7 @@ mod tests { // Dont generate proof let proof = vec![]; // Verify the range proof - assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, keys[0], keys, values, proof).is_err()); } #[test] @@ -786,7 +786,7 @@ mod tests { // Generate proof (last element) let proof = trie.get_proof(&last_element).unwrap(); // Verify the range proof - assert!(verify_range_proof(root, first_key, keys, values, proof).is_err()); + assert!(verify_range(root, first_key, keys, values, proof).is_err()); } #[test] @@ -806,7 +806,7 @@ mod tests { // Generate proofs let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - assert!(verify_range_proof(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, keys[0], keys, values, proof).is_err()); } } } From 07e1f464f381b50f5b9f3f55cc56bca47f993e30 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:16:55 -0300 Subject: [PATCH 200/345] clippy --- crates/storage/trie/verify_range.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index e6a1604c1..9ad84244f 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -422,7 +422,7 @@ struct ProofNodeStorage<'a> { impl<'a> ProofNodeStorage<'a> { // Construct a ProofNodeStorage for a proof - fn from_proof(proof: &'a Vec>) -> Self { + fn from_proof(proof: &'a [Vec]) -> Self { Self { nodes: proof .iter() @@ -782,9 +782,9 @@ mod tests { // Range is empty let values = vec![]; let keys = vec![]; - let first_key = H256::from_slice(&last_element); + let first_key = H256::from_slice(last_element); // Generate proof (last element) - let proof = trie.get_proof(&last_element).unwrap(); + let proof = trie.get_proof(last_element).unwrap(); // Verify the range proof assert!(verify_range(root, first_key, keys, values, proof).is_err()); } From 2b36bbb60e76593c89428b5d9b67c8a8e3850418 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:19:09 -0300 Subject: [PATCH 201/345] Update code --- crates/storage/trie/trie.rs | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 2384cf1be..89e295b99 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -168,19 +168,8 @@ impl Trie { // 2. the trie contains the (key, value) pair to verify // We will only be using the trie's cache so we don't need a working DB - // struct NullTrieDB; - // impl TrieDB for NullTrieDB { - // fn get(&self, _key: Vec) -> Result>, TrieError> { - // Ok(None) - // } - - // fn put(&self, _key: Vec, _value: Vec) -> Result<(), TrieError> { - // Ok(()) - // } - // } - - // let mut trie = Trie::new(Box::new(NullTrieDB)); + // let mut trie = Trie::stateless(); // Insert root into trie // let mut proof = proof.into_iter(); From 2d711b6fe26f243c5ee175073724e33e26f85db6 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:27:32 -0300 Subject: [PATCH 202/345] Improve interface --- crates/storage/trie/trie.rs | 6 ++-- crates/storage/trie/verify_range.rs | 53 ++++++++++++++--------------- 2 files changed, 29 insertions(+), 30 deletions(-) diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 89e295b99..c063dcf26 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -1,14 +1,14 @@ mod db; mod error; +mod nibbles; mod node; mod node_hash; -mod verify_range; mod rlp; mod state; -mod trie_iter; -mod nibbles; #[cfg(test)] mod test_utils; +mod trie_iter; +mod verify_range; use ethereum_rust_rlp::constants::RLP_NULL; use ethereum_types::H256; use nibbles::Nibbles; diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 9ad84244f..2fe2de3ec 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -4,21 +4,20 @@ use ethereum_types::H256; use sha3::{Digest, Keccak256}; use crate::{ - nibbles::Nibbles, node::Node, node_hash::NodeHash, state::TrieState, - Trie, TrieError, ValueRLP, + nibbles::Nibbles, node::Node, node_hash::NodeHash, state::TrieState, Trie, TrieError, ValueRLP, }; /// Verifies that the key value range belongs to the trie with the given root given the edge proofs for the range /// Also returns true if there is more state to be fetched (aka if there are more keys to the right of the given range) pub fn verify_range( root: H256, - first_key: H256, - keys: Vec, - values: Vec, - proof: Vec>, + first_key: &H256, + keys: &[H256], + values: &[ValueRLP], + proof: &[Vec], ) -> Result { // Store proof nodes by hash - let proof_nodes = ProofNodeStorage::from_proof(&proof); + let proof_nodes = ProofNodeStorage::from_proof(proof); // Validate range if keys.len() != values.len() { return Err(TrieError::Verify(format!( @@ -48,8 +47,8 @@ pub fn verify_range( // Special Case: No proofs given, the range is expected to be the full set of leaves if proof.is_empty() { // Check that the trie constructed from the given keys and values has the expected root - for (index, key) in keys.iter().enumerate() { - trie.insert(key.0.to_vec(), values[index].clone())?; + for (key, value) in keys.iter().zip(values.iter()) { + trie.insert(key.0.to_vec(), value.clone())?; } let hash = trie.hash()?; if hash != root { @@ -77,13 +76,13 @@ pub fn verify_range( } } - let last_key = *keys.last().unwrap(); + let last_key = keys.last().unwrap(); // Special Case: There is only one element and the two edge keys are the same if keys.len() == 1 && first_key == last_key { // We need to check that the proof confirms the existance of the first key let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; - if first_key != keys[0] { + if first_key != &keys[0] { return Err(TrieError::Verify( "correct proof but invalid key".to_string(), )); @@ -131,7 +130,7 @@ pub fn verify_range( fn fill_state( trie_state: &mut TrieState, root_hash: H256, - first_key: H256, + first_key: &H256, proof_nodes: &ProofNodeStorage, ) -> Result, TrieError> { let mut path = Nibbles::from_bytes(&first_key.0); @@ -237,8 +236,8 @@ fn has_right_element_inner( /// Asumes that left_key & right_key are not equal and of same length fn remove_internal_references( root_hash: H256, - left_key: H256, - right_key: H256, + left_key: &H256, + right_key: &H256, trie_state: &mut TrieState, ) -> Result { // First find the node at which the left and right path differ @@ -468,7 +467,7 @@ mod tests { let root = trie.hash().unwrap(); let keys = (50_u8..=75).map(|i| H256([i; 32])).collect::>(); let values = (50_u8..=75).map(|i| [i; 32].to_vec()).collect::>(); - let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &keys[0], &keys, &values, &proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -526,7 +525,7 @@ mod tests { let mut proof = trie.get_proof(&trie_values[7]).unwrap(); proof.extend(trie.get_proof(&trie_values[17]).unwrap()); let root = trie.hash().unwrap(); - let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &keys[0], &keys, &values, &proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -552,7 +551,7 @@ mod tests { let mut proof = trie.get_proof(&values[0]).unwrap(); proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); // Verify the range proof - let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &keys[0], &keys, &values, &proof).unwrap(); if end == 199 { // The last key is at the edge of the trie assert!(!fetch_more) @@ -595,7 +594,7 @@ mod tests { let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - let fetch_more = verify_range(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &H256::from_slice(&first_key), &keys, &values, &proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -636,7 +635,7 @@ mod tests { let mut proof = trie.get_proof(&first_key).unwrap(); proof.extend(trie.get_proof(&last_key).unwrap()); // Verify the range proof - let fetch_more = verify_range(root, H256::from_slice(&first_key), keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &H256::from_slice(&first_key), &keys, &values, &proof).unwrap(); // Our trie contains more elements to the right assert!(fetch_more) } @@ -656,7 +655,7 @@ mod tests { // The keyset contains the entire trie so we don't need edge proofs let proof = vec![]; // Verify the range proof - let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &keys[0], &keys, &values, &proof).unwrap(); // Our range is the full leafset, there shouldn't be more values left in the trie assert!(!fetch_more) } @@ -679,7 +678,7 @@ mod tests { // Generate proof (last element) let proof = trie.get_proof(&last_element).unwrap(); // Verify the range proof - let fetch_more = verify_range(root, first_key, keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &first_key, &keys, &values, &proof).unwrap(); // There are no more elements to the right of the range assert!(!fetch_more) } @@ -699,7 +698,7 @@ mod tests { // Generate proofs let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - let fetch_more = verify_range(root, keys[0], keys, values, proof).unwrap(); + let fetch_more = verify_range(root, &keys[0], &keys, &values, &proof).unwrap(); if start == 199 { // The last key is at the edge of the trie assert!(!fetch_more) @@ -726,7 +725,7 @@ mod tests { // Generate proofs (only prove first key) let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - assert!(verify_range(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); } #[test] @@ -747,7 +746,7 @@ mod tests { // Remove the last node of the second proof (to make sure we don't remove a node that is also part of the first proof) proof.pop(); // Verify the range proof - assert!(verify_range(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); } #[test] @@ -765,7 +764,7 @@ mod tests { // Dont generate proof let proof = vec![]; // Verify the range proof - assert!(verify_range(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); } #[test] @@ -786,7 +785,7 @@ mod tests { // Generate proof (last element) let proof = trie.get_proof(last_element).unwrap(); // Verify the range proof - assert!(verify_range(root, first_key, keys, values, proof).is_err()); + assert!(verify_range(root, &first_key, &keys, &values, &proof).is_err()); } #[test] @@ -806,7 +805,7 @@ mod tests { // Generate proofs let proof = trie.get_proof(&values[0]).unwrap(); // Verify the range proof - assert!(verify_range(root, keys[0], keys, values, proof).is_err()); + assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); } } } From 56a4ee9e3b44103254ea7173639739d56f02c16e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:29:41 -0300 Subject: [PATCH 203/345] Remove code leftover from debug --- crates/storage/trie/verify_range.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 2fe2de3ec..35b0ae84d 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -65,7 +65,6 @@ pub fn verify_range( // We need to check that the proof confirms the non-existance of the first key // and that there are no more elements to the right of the first key let value = fill_state(&mut trie.state, root, first_key, &proof_nodes)?; - trie.root = Some(root.into()); let has_right_element = has_right_element(root, first_key.as_bytes(), &trie.state)?; if has_right_element || !value.is_empty() { return Err(TrieError::Verify( From c444a017798b9fe0eb197bacd71a2787961dd1bd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:32:03 -0300 Subject: [PATCH 204/345] Consistency --- crates/storage/trie/verify_range.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 35b0ae84d..6ea107548 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -107,8 +107,8 @@ pub fn verify_range( trie.root = Some(NodeHash::from(root)); } // Reconstruct the internal nodes by inserting the elements on the range - for (i, key) in keys.iter().enumerate() { - trie.insert(key.0.to_vec(), values[i].clone())?; + for (key, value) in keys.iter().zip(values.iter()) { + trie.insert(key.0.to_vec(), value.clone())?; } // Check for elements to the right of the range before we wipe the sate let has_right_element = has_right_element(root, last_key.as_bytes(), &trie.state)?; From d59b09a4c688785baf1e3767b8ac42290aa55a77 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:36:38 -0300 Subject: [PATCH 205/345] Fix --- crates/storage/trie/verify_range.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 6ea107548..522114f7f 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -264,10 +264,11 @@ fn remove_internal_references_inner( let node = trie_state.get_node(node_hash.clone())?.unwrap(); match node { Node::Branch(mut n) => { - // If none of the paths have next choice nibble then it means that this is the end of the path + // If none of the paths have a next choice nibble then it means that this is the end of the path // which would mean that both paths are equal, which we already checked before - // Only one path doesn't have a next choice then it would mean that the paths have different lengths, + // If only one path doesn't have a next choice then it would mean that the paths have different lengths, // which we also checked before calling this function + // Therefore we can safely unwrap here let left_choice = left_path.next_choice().unwrap(); let right_choice = right_path.next_choice().unwrap(); From 8dd7c6d6caa99ab8a5319940b2ec1c82633e1398 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:51:29 -0300 Subject: [PATCH 206/345] Add test --- crates/storage/trie/verify_range.rs | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 522114f7f..8cc9eb2d5 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -749,6 +749,32 @@ mod tests { assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); } + #[test] + // Regular Case: Two Edge Proofs, both keys exist, but there is a missing node in the proof + fn proptest_verify_range_regular_case_gap_in_middle_of_proof(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { + // Build trie + let mut trie = Trie::new_temp(); + for val in data.iter() { + trie.insert(val.clone(), val.clone()).unwrap() + } + let root = trie.hash().unwrap(); + // Select range to prove + let values = data.into_iter().collect::>()[start..=end].to_vec(); + let keys = values.iter().map(|a| H256::from_slice(a)).collect::>(); + // Generate proofs + let mut proof = trie.get_proof(&values[0]).unwrap(); + let mut second_proof = trie.get_proof(&values[0]).unwrap(); + proof.extend(trie.get_proof(values.last().unwrap()).unwrap()); + // Remove the middle node of the second proof + let gap_idx = second_proof.len() / 2; + let removed = second_proof.remove(gap_idx); + // Remove the node from the first proof if it is also there + proof.retain(|n| n != &removed); + proof.extend(second_proof); + // Verify the range proof + assert!(verify_range(root, &keys[0], &keys, &values, &proof).is_err()); + } + #[test] // Regular Case: No proofs both keys exist fn proptest_verify_range_regular_case_no_proofs(data in btree_set(vec(any::(), 32), 200), start in 1_usize..=100_usize, end in 101..200_usize) { From 7e4205ce8f19aa62435b62b4baa9c4a97797ed4d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 16:58:47 -0300 Subject: [PATCH 207/345] Remove TODO --- crates/storage/trie/verify_range.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/storage/trie/verify_range.rs b/crates/storage/trie/verify_range.rs index 8cc9eb2d5..90c397a52 100644 --- a/crates/storage/trie/verify_range.rs +++ b/crates/storage/trie/verify_range.rs @@ -330,7 +330,6 @@ fn remove_internal_references_inner( ); } // If both paths are greater or lesser than the node's prefix then the range is empty - // TODO: return the error instead of panicking here (Ordering::Greater, Ordering::Greater) | (Ordering::Less, Ordering::Less) => { return Err(TrieError::Verify("empty range".to_string())) } From c59b8be4dfa5e0c6c804024e17f25bf033398ec1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 17:18:05 -0300 Subject: [PATCH 208/345] Revert "[Postpone] changes not related to the trie" This reverts commit ad0f921cb9e330e79fe1d7479eff628dde5d1363. --- crates/networking/p2p/Cargo.toml | 1 + crates/networking/p2p/rlpx/error.rs | 2 ++ crates/networking/p2p/rlpx/snap.rs | 2 +- crates/networking/p2p/snap.rs | 37 +++++++++++++++++++++++++++++ 4 files changed, 41 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index 128cff7e7..2fbdae71a 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -9,6 +9,7 @@ edition = "2021" ethereum_rust-core.workspace = true ethereum_rust-rlp.workspace = true ethereum_rust-storage.workspace = true +ethereum_rust-trie.workspace = true tracing.workspace = true tokio.workspace = true diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index d87019432..f883a56db 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -31,6 +31,8 @@ pub(crate) enum RLPxError { StoreError(#[from] StoreError), #[error("Error in cryptographic library: {0}")] CryptographyError(String), + #[error(transparent)] + Trie(#[from] TrieError), #[error("Failed to broadcast msg: {0}")] BroadcastError(String), } diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index abec6c9f1..f83ada6b9 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -326,7 +326,7 @@ pub struct AccountRangeUnit { pub account: AccountStateSlim, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AccountStateSlim { pub nonce: u64, pub balance: U256, diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index fefd7374b..5017becba 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,6 +1,8 @@ use bytes::Bytes; +use ethereum_rust_core::types::AccountState; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; +use ethereum_rust_trie::verify_range_proof; use crate::rlpx::{ error::RLPxError, @@ -10,6 +12,8 @@ use crate::rlpx::{ }, }; +// Request Processing + pub fn process_account_range_request( request: GetAccountRange, store: Store, @@ -153,6 +157,39 @@ pub fn process_trie_nodes_request( }) } +// Response Processing + +#[allow(unused)] +pub fn validate_account_range_response( + request: &GetAccountRange, + response: &AccountRange, +) -> Result<(), RLPxError> { + // Reconstruct a partial trie from the response and verify it + let (keys, accounts) = response + .accounts + .iter() + .map(|unit| { + ( + unit.hash, + AccountState::from(unit.account.clone()).encode_to_vec(), + ) + }) + .unzip(); + let proof = response + .proof + .iter() + .map(|bytes| bytes.as_ref().to_vec()) + .collect(); + verify_range_proof( + request.root_hash, + request.starting_hash, + keys, + accounts, + proof, + )?; + Ok(()) +} + #[cfg(test)] mod tests { use std::str::FromStr; From 03b73c862bb59cc91fde299738845c9fae7ffab7 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 17:20:24 -0300 Subject: [PATCH 209/345] Fix --- crates/networking/p2p/rlpx/error.rs | 1 + crates/networking/p2p/snap.rs | 16 ++++++++-------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index f883a56db..479a594ad 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,5 +1,6 @@ use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; use ethereum_rust_storage::error::StoreError; +use ethereum_rust_trie::TrieError; use thiserror::Error; // TODO improve errors diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 5017becba..460c0834a 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -2,7 +2,7 @@ use bytes::Bytes; use ethereum_rust_core::types::AccountState; use ethereum_rust_rlp::encode::RLPEncode; use ethereum_rust_storage::{error::StoreError, Store}; -use ethereum_rust_trie::verify_range_proof; +use ethereum_rust_trie::verify_range; use crate::rlpx::{ error::RLPxError, @@ -165,7 +165,7 @@ pub fn validate_account_range_response( response: &AccountRange, ) -> Result<(), RLPxError> { // Reconstruct a partial trie from the response and verify it - let (keys, accounts) = response + let (keys, accounts): (Vec<_>, Vec<_>) = response .accounts .iter() .map(|unit| { @@ -179,13 +179,13 @@ pub fn validate_account_range_response( .proof .iter() .map(|bytes| bytes.as_ref().to_vec()) - .collect(); - verify_range_proof( + .collect::>(); + verify_range( request.root_hash, - request.starting_hash, - keys, - accounts, - proof, + &request.starting_hash, + &keys, + &accounts, + &proof, )?; Ok(()) } From e7d3dc43ea9a2dfbb915a5dd7ff7be700de741a8 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 13 Nov 2024 17:37:56 -0300 Subject: [PATCH 210/345] Add helper methods --- crates/networking/p2p/snap.rs | 42 ++++++++++++++++++----------------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 460c0834a..f88743b84 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -30,15 +30,11 @@ pub fn process_account_range_request( break; } } - let proof = store - .get_account_range_proof( - request.root_hash, - request.starting_hash, - accounts.last().map(|acc| acc.hash), - )? - .iter() - .map(|bytes| Bytes::copy_from_slice(bytes)) - .collect(); + let proof = proof_to_encodable(store.get_account_range_proof( + request.root_hash, + request.starting_hash, + accounts.last().map(|acc| acc.hash), + )?); Ok(AccountRange { id: request.id, accounts, @@ -76,7 +72,7 @@ pub fn process_storage_ranges_request( // Generate proofs only if the response doesn't contain the full storage range for the account // Aka if the starting hash is not zero or if the response was capped due to byte limit if !request.starting_hash.is_zero() || res_capped && !account_slots.is_empty() { - proof.extend( + proof.extend(proof_to_encodable( store .get_storage_range_proof( request.root_hash, @@ -84,10 +80,8 @@ pub fn process_storage_ranges_request( request.starting_hash, account_slots.last().map(|acc| acc.hash), )? - .unwrap_or_default() - .iter() - .map(|bytes| Bytes::copy_from_slice(bytes)), - ); + .unwrap_or_default(), + )); } if !account_slots.is_empty() { @@ -164,7 +158,7 @@ pub fn validate_account_range_response( request: &GetAccountRange, response: &AccountRange, ) -> Result<(), RLPxError> { - // Reconstruct a partial trie from the response and verify it + // Verify Range Proof let (keys, accounts): (Vec<_>, Vec<_>) = response .accounts .iter() @@ -175,11 +169,7 @@ pub fn validate_account_range_response( ) }) .unzip(); - let proof = response - .proof - .iter() - .map(|bytes| bytes.as_ref().to_vec()) - .collect::>(); + let proof = encodable_to_proof(&response.proof); verify_range( request.root_hash, &request.starting_hash, @@ -190,6 +180,18 @@ pub fn validate_account_range_response( Ok(()) } +// Helper method to convert proof to RLP-encodable format +#[inline] +fn proof_to_encodable(proof: Vec>) -> Vec { + proof.into_iter().map(|bytes| Bytes::from(bytes)).collect() +} + +// Helper method to obtain proof from RLP-encodable format +#[inline] +fn encodable_to_proof(proof: &Vec) -> Vec> { + proof.into_iter().map(|bytes| bytes.to_vec()).collect() +} + #[cfg(test)] mod tests { use std::str::FromStr; From 5d4ad34df63a3f57d9318a421b35d8d1d819f082 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 14 Nov 2024 18:23:11 -0300 Subject: [PATCH 211/345] Add syncmode argument --- cmd/ethereum_rust/cli.rs | 6 ++++++ cmd/ethereum_rust/ethereum_rust.rs | 16 ++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/cmd/ethereum_rust/cli.rs b/cmd/ethereum_rust/cli.rs index 727de4f38..373c928f4 100644 --- a/cmd/ethereum_rust/cli.rs +++ b/cmd/ethereum_rust/cli.rs @@ -104,6 +104,12 @@ pub fn cli() -> Command { .required(false) .value_name("CHAIN_RLP_PATH"), ) + .arg( + Arg::new("syncmode") + .long("syncmode") + .required(false) + .value_name("SYNC_MODE"), + ) .subcommand( Command::new("removedb").about("Remove the database").arg( Arg::new("datadir") diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index e5c8a86f2..c6cccc267 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -114,6 +114,9 @@ async fn main() { let data_dir = matches .get_one::("datadir") .unwrap_or(&default_datadir); + // TODO: Use snap as default + let is_snap_sync = is_snap_sync(&matches); + let store = Store::new(data_dir, EngineType::Libmdbx).expect("Failed to create Store"); let genesis = read_genesis_file(genesis_file_path); @@ -283,6 +286,19 @@ fn parse_socket_addr(addr: &str, port: &str) -> io::Result { )) } +fn is_snap_sync(matches: &clap::ArgMatches) -> bool { + let syncmode = matches.get_one::("syncmode"); + if let Some(syncmode) = syncmode { + match &**syncmode { + "full" => false, + "snap" => true, + other => panic!("Invalid syncmode {other} expected either snap or full"), + } + } else { + false + } +} + fn get_default_datadir() -> String { let project_dir = ProjectDirs::from("", "", "ethereum_rust").expect("Couldn't find home directory"); From a3820e47c24e8f33ba8c1289cc3916aa5546794b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 15 Nov 2024 13:14:34 -0300 Subject: [PATCH 212/345] crate --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index fc7f99494..1f7ac69d5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,7 +17,7 @@ members = [ "crates/l2/", "crates/l2/prover", "crates/l2/contracts", - "crates/l2/sdk", + "crates/l2/sdk", "crates/networking/snap_sync", ] resolver = "2" From c3d510646b03bc4fc20e9d561a39f90d4db99b0b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 15 Nov 2024 17:34:38 -0300 Subject: [PATCH 213/345] Add SnapSyncManager + send snap replies to them --- Cargo.toml | 2 +- cmd/ethereum_rust/ethereum_rust.rs | 1 + crates/networking/p2p/net.rs | 64 ++++++++++++++++++------ crates/networking/p2p/rlpx/connection.rs | 25 +++++++-- crates/networking/p2p/rlpx/error.rs | 4 ++ 5 files changed, 76 insertions(+), 20 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index c1cc7ffe4..446ea3fa7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,7 +18,7 @@ members = [ "crates/l2/", "crates/l2/prover", "crates/l2/contracts", - "crates/l2/sdk", "crates/networking/snap_sync", + "crates/l2/sdk", ] resolver = "2" diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index 3e377eec4..a76067455 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -227,6 +227,7 @@ async fn main() { bootnodes, signer, store, + is_snap_sync ) .into_future(); tracker.spawn(networking); diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index 9b027fa92..ca335881b 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -19,9 +19,10 @@ use k256::{ use kademlia::{bucket_number, KademliaTable, MAX_NODES_PER_BUCKET}; use rand::rngs::OsRng; use rlpx::{connection::RLPxConnection, error::RLPxError, message::Message as RLPxMessage}; +use snap_sync::SnapSyncManager; use tokio::{ net::{TcpSocket, TcpStream, UdpSocket}, - sync::{broadcast, Mutex}, + sync::{broadcast, mpsc, Mutex}, try_join, }; use tracing::{debug, error, info}; @@ -32,6 +33,7 @@ pub(crate) mod discv4; pub(crate) mod kademlia; pub mod rlpx; pub(crate) mod snap; +mod snap_sync; pub mod types; const MAX_DISC_PACKET_SIZE: usize = 1280; @@ -48,11 +50,19 @@ pub async fn start_network( bootnodes: Vec, signer: SigningKey, storage: Store, + start_snap_sync: bool, ) { info!("Starting discovery service at {udp_addr}"); info!("Listening for requests at {tcp_addr}"); let local_node_id = node_id_from_signing_key(&signer); let table = Arc::new(Mutex::new(KademliaTable::new(local_node_id))); + // Communication between the backend and the main listen loop + let (channel_backend_send, channel_backend_receive) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + if start_snap_sync { + let snap_sync_manager = SnapSyncManager::new(channel_backend_receive); + snap_sync_manager.start(table.clone()) // we need the table in order to send requests to a random peer + } let (channel_broadcast_send_end, _) = tokio::sync::broadcast::channel::<( tokio::task::Id, Arc, @@ -64,6 +74,7 @@ pub async fn start_network( table.clone(), bootnodes, channel_broadcast_send_end.clone(), + channel_backend_send.clone(), )); let server_handle = tokio::spawn(serve_requests( tcp_addr, @@ -71,6 +82,7 @@ pub async fn start_network( storage.clone(), table.clone(), channel_broadcast_send_end, + channel_backend_send, )); try_join!(discovery_handle, server_handle).unwrap(); @@ -83,6 +95,7 @@ async fn discover_peers( table: Arc>, bootnodes: Vec, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, + backend_send: mpsc::Sender, ) { let udp_socket = Arc::new(UdpSocket::bind(udp_addr).await.unwrap()); @@ -93,6 +106,7 @@ async fn discover_peers( table.clone(), signer.clone(), connection_broadcast, + backend_send, )); let revalidation_handler = tokio::spawn(peers_revalidation( udp_addr, @@ -132,6 +146,7 @@ async fn discover_peers_server( table: Arc>, signer: SigningKey, tx_broadcaster_send: broadcast::Sender<(tokio::task::Id, Arc)>, + backend_send: mpsc::Sender, ) { let mut buf = vec![0; MAX_DISC_PACKET_SIZE]; @@ -218,6 +233,7 @@ async fn discover_peers_server( let signer = signer.clone(); let storage = storage.clone(); let broadcaster = tx_broadcaster_send.clone(); + let backend_send = backend_send.clone(); tokio::spawn(async move { handle_peer_as_initiator( signer, @@ -226,6 +242,7 @@ async fn discover_peers_server( storage, table, broadcaster, + backend_send, ) .await; }); @@ -758,6 +775,7 @@ async fn serve_requests( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, + backend_send: mpsc::Sender, ) { let tcp_socket = TcpSocket::new_v4().unwrap(); tcp_socket.bind(tcp_addr).unwrap(); @@ -771,6 +789,7 @@ async fn serve_requests( storage.clone(), table.clone(), connection_broadcast.clone(), + backend_send.clone(), )); } } @@ -781,9 +800,10 @@ async fn handle_peer_as_receiver( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, + backend_send: mpsc::Sender, ) { let conn = RLPxConnection::receiver(signer, stream, storage, connection_broadcast); - handle_peer(conn, table).await; + handle_peer(conn, table, backend_send).await; } async fn handle_peer_as_initiator( @@ -793,6 +813,7 @@ async fn handle_peer_as_initiator( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, + backend_send: mpsc::Sender, ) { info!("Trying RLPx connection with {node:?}"); let stream = TcpSocket::new_v4() @@ -801,20 +822,24 @@ async fn handle_peer_as_initiator( .await .unwrap(); match RLPxConnection::initiator(signer, msg, stream, storage, connection_broadcast).await { - Ok(conn) => handle_peer(conn, table).await, + Ok(conn) => handle_peer(conn, table, backend_send).await, Err(e) => { error!("Error: {e}, Could not start connection with {node:?}"); } } } -async fn handle_peer(mut conn: RLPxConnection, table: Arc>) { +async fn handle_peer( + mut conn: RLPxConnection, + table: Arc>, + backend_send: mpsc::Sender, +) { // Perform handshake if let Err(e) = conn.handshake().await { peer_conn_failed("Handshake failed", e, conn, table).await; } else { // Handshake OK: handle connection - if let Err(e) = conn.handle_peer_conn().await { + if let Err(e) = conn.handle_peer_conn(backend_send).await { peer_conn_failed("Error during RLPx connection", e, conn, table).await; } } @@ -886,7 +911,11 @@ mod tests { pub udp_socket: Arc, } - async fn start_mock_discovery_server(udp_port: u16, should_start_server: bool) -> MockServer { + async fn start_mock_discovery_server( + udp_port: u16, + should_start_server: bool, + backend_send: mpsc::Sender, + ) -> MockServer { let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), udp_port); let signer = SigningKey::random(&mut OsRng); let udp_socket = Arc::new(UdpSocket::bind(addr).await.unwrap()); @@ -906,6 +935,7 @@ mod tests { table.clone(), signer.clone(), channel_broadcast_send_end, + backend_send, )); } @@ -952,8 +982,10 @@ mod tests { * To make this run faster, we'll change the revalidation time to be every 2secs */ async fn discovery_server_revalidation() { - let mut server_a = start_mock_discovery_server(7998, true).await; - let mut server_b = start_mock_discovery_server(7999, true).await; + let (backend_send, _) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let mut server_a = start_mock_discovery_server(7998, true, backend_send.clone()).await; + let mut server_b = start_mock_discovery_server(7999, true, backend_send).await; connect_servers(&mut server_a, &mut server_b).await; @@ -1015,8 +1047,10 @@ mod tests { * This test for only one lookup, and not recursively. */ async fn discovery_server_lookup() { - let mut server_a = start_mock_discovery_server(8000, true).await; - let mut server_b = start_mock_discovery_server(8001, true).await; + let (backend_send, _) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let mut server_a = start_mock_discovery_server(8000, true, backend_send.clone()).await; + let mut server_b = start_mock_discovery_server(8001, true, backend_send).await; fill_table_with_random_nodes(server_a.table.clone()).await; @@ -1073,10 +1107,12 @@ mod tests { * - We'll run a recursive lookup on server `a` and we expect to end with `b`, `c`, `d` and its mock nodes */ async fn discovery_server_recursive_lookup() { - let mut server_a = start_mock_discovery_server(8002, true).await; - let mut server_b = start_mock_discovery_server(8003, true).await; - let mut server_c = start_mock_discovery_server(8004, true).await; - let mut server_d = start_mock_discovery_server(8005, true).await; + let (backend_send, _) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let mut server_a = start_mock_discovery_server(8002, true, backend_send.clone()).await; + let mut server_b = start_mock_discovery_server(8003, true, backend_send.clone()).await; + let mut server_c = start_mock_discovery_server(8004, true, backend_send.clone()).await; + let mut server_d = start_mock_discovery_server(8005, true, backend_send).await; connect_servers(&mut server_a, &mut server_b).await; connect_servers(&mut server_b, &mut server_c).await; diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index b319f09c8..48afda15a 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -23,7 +23,7 @@ use super::{ error::RLPxError, frame, handshake::{decode_ack_message, decode_auth_message, encode_auth_message}, - message as rlpx, + message::{self as rlpx}, p2p::Capability, utils::{ecdh_xchng, pubkey2id}, }; @@ -38,7 +38,10 @@ use k256::{ use sha3::{Digest, Keccak256}; use tokio::{ io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}, - sync::broadcast::{self, error::RecvError}, + sync::{ + broadcast::{self, error::RecvError}, + mpsc, + }, task, time::{sleep, Instant}, }; @@ -191,7 +194,10 @@ impl RLPxConnection { } } - pub async fn handle_peer_conn(&mut self) -> Result<(), RLPxError> { + pub async fn handle_peer_conn( + &mut self, + backend_send: mpsc::Sender, + ) -> Result<(), RLPxError> { if let RLPxConnectionState::Established(_) = &self.state { self.init_peer_conn().await?; info!("Started peer main loop"); @@ -210,7 +216,7 @@ impl RLPxConnection { tokio::select! { // TODO check if this is cancel safe, and fix it if not. message = self.receive() => { - self.handle_message(message?).await?; + self.handle_message(message?, backend_send.clone()).await?; } // This is not ideal, but using the receiver without // this function call, causes the loop to take ownwership @@ -261,7 +267,11 @@ impl RLPxConnection { Ok(()) } - async fn handle_message(&mut self, message: Message) -> Result<(), RLPxError> { + async fn handle_message( + &mut self, + message: Message, + backend_send: mpsc::Sender, + ) -> Result<(), RLPxError> { let peer_supports_eth = self.capabilities.contains(&CAP_ETH); match message { Message::Disconnect(msg_data) => { @@ -318,6 +328,11 @@ impl RLPxConnection { let response = process_trie_nodes_request(req, self.storage.clone())?; self.send(Message::TrieNodes(response)).await? } + // Send snap request responses to the SyncManager + message @ Message::AccountRange(_) + | message @ Message::StorageRanges(_) + | message @ Message::ByteCodes(_) + | message @ Message::TrieNodes(_) => backend_send.send(message).await?, // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::MessageNotHandled(format!("{message}"))), }; diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index d0c2974e7..242938e0e 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -4,6 +4,8 @@ use ethereum_rust_trie::TrieError; use thiserror::Error; use tokio::sync::broadcast::error::RecvError; +use super::message::Message; + // TODO improve errors #[derive(Debug, Error)] pub(crate) enum RLPxError { @@ -41,6 +43,8 @@ pub(crate) enum RLPxError { BroadcastError(String), #[error(transparent)] RecvError(#[from] RecvError), + #[error(transparent)] + Send(#[from] tokio::sync::mpsc::error::SendError), } // Grouping all cryptographic related errors in a single CryptographicError variant From fa275ec4e04ed61c9d6309c6729a8a155c5d4e92 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 15 Nov 2024 18:17:13 -0300 Subject: [PATCH 214/345] Add channel between kademlia table and peer connection handler --- crates/networking/p2p/kademlia.rs | 15 +++++++++++++++ crates/networking/p2p/net.rs | 15 ++++++++++++++- crates/networking/p2p/rlpx/connection.rs | 4 ++++ 3 files changed, 33 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 26578a540..3d8fdb1dc 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -1,5 +1,6 @@ use crate::{ discv4::{time_now_unix, FindNodeRequest}, + rlpx::message::Message, types::Node, }; use ethereum_rust_core::{H256, H512, U256}; @@ -224,6 +225,17 @@ impl KademliaTable { self.replace_peer_inner(node_id, bucket_idx) } + pub fn set_sender(&mut self, node_id: H512, sender: tokio::sync::mpsc::Sender) { + let bucket_idx = bucket_number(self.local_node_id, node_id); + if let Some(peer) = self.buckets[bucket_idx] + .peers + .iter_mut() + .find(|peer| peer.node.node_id == node_id) + { + peer.sender = Some(sender) + } + } + #[cfg(test)] pub fn replace_peer_on_custom_bucket( &mut self, @@ -279,6 +291,8 @@ pub struct PeerData { pub liveness: u16, /// if a revalidation was sent to the peer, the bool marks if it has answered pub revalidation: Option, + /// sender end of the channel between the peer and its rlpx connection + pub sender: Option>, } impl PeerData { @@ -292,6 +306,7 @@ impl PeerData { last_ping_hash: None, find_node_request: None, revalidation: None, + sender: None, } } diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index ca335881b..f03e73f3a 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -839,7 +839,20 @@ async fn handle_peer( peer_conn_failed("Handshake failed", e, conn, table).await; } else { // Handshake OK: handle connection - if let Err(e) = conn.handle_peer_conn(backend_send).await { + // Create channel to communicate directly to the peer + let (sender, backend_receive) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let Ok(node_id) = conn.get_remote_node_id() else { + return peer_conn_failed( + "Error during RLPx connection", + RLPxError::InvalidState(), + conn, + table, + ) + .await; + }; + table.lock().await.set_sender(node_id, sender); + if let Err(e) = conn.handle_peer_conn(backend_send, backend_receive).await { peer_conn_failed("Error during RLPx connection", e, conn, table).await; } } diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 48afda15a..7e6c99dae 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -197,6 +197,7 @@ impl RLPxConnection { pub async fn handle_peer_conn( &mut self, backend_send: mpsc::Sender, + mut backend_receive: mpsc::Receiver, ) -> Result<(), RLPxError> { if let RLPxConnectionState::Established(_) = &self.state { self.init_peer_conn().await?; @@ -229,6 +230,9 @@ impl RLPxConnection { Some(broadcasted_msg) = Self::maybe_wait_for_broadcaster(&mut broadcaster_receive) => { self.handle_broadcast(broadcasted_msg?).await? } + Some(message) = backend_receive.recv() => { + self.send(message).await?; + } _ = sleep(PERIODIC_TASKS_CHECK_INTERVAL) => { // no progress on other tasks, yield control to check // periodic tasks From c88d11429eada4320e9cb6c9cc7ebc9208df01ff Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 21 Nov 2024 17:21:25 -0300 Subject: [PATCH 215/345] SyncManager first iteration --- cmd/ethereum_rust/ethereum_rust.rs | 14 +- crates/common/types/fork_id.rs | 2 +- crates/networking/p2p/kademlia.rs | 43 ++++-- crates/networking/p2p/net.rs | 30 ++-- crates/networking/p2p/rlpx/connection.rs | 6 +- crates/networking/p2p/rlpx/error.rs | 4 +- crates/networking/p2p/rlpx/eth/blocks.rs | 22 ++- crates/networking/p2p/rlpx/eth/status.rs | 2 +- .../networking/p2p/rlpx/eth/transactions.rs | 2 +- crates/networking/p2p/rlpx/message.rs | 4 +- crates/networking/p2p/rlpx/p2p.rs | 8 +- crates/networking/p2p/rlpx/snap.rs | 20 +-- crates/networking/p2p/sync.rs | 145 ++++++++++++++++++ crates/networking/rpc/eth/filter.rs | 4 + crates/networking/rpc/eth/gas_price.rs | 9 +- crates/networking/rpc/rpc.rs | 7 + crates/networking/rpc/utils.rs | 12 +- 17 files changed, 269 insertions(+), 65 deletions(-) create mode 100644 crates/networking/p2p/sync.rs diff --git a/cmd/ethereum_rust/ethereum_rust.rs b/cmd/ethereum_rust/ethereum_rust.rs index a76067455..18b953a38 100644 --- a/cmd/ethereum_rust/ethereum_rust.rs +++ b/cmd/ethereum_rust/ethereum_rust.rs @@ -5,8 +5,9 @@ use ethereum_rust_blockchain::fork_choice::apply_fork_choice; use ethereum_rust_core::types::{Block, Genesis}; use ethereum_rust_core::H256; use ethereum_rust_net::bootnode::BootNode; -use ethereum_rust_net::node_id_from_signing_key; +use ethereum_rust_net::sync::SyncManager; use ethereum_rust_net::types::Node; +use ethereum_rust_net::{node_id_from_signing_key, peer_table}; use ethereum_rust_storage::{EngineType, Store}; use k256::ecdsa::SigningKey; use local_ip_address::local_ip; @@ -185,6 +186,13 @@ async fn main() { tcp_port: tcp_socket_addr.port(), node_id: local_node_id, }; + // Create Kademlia Table here so we can access it from rpc server (for syncing) + let peer_table = peer_table(signer.clone()); + // Communication between the backend and the main listen loop + let (channel_backend_send, channel_p2p_receive) = + tokio::sync::mpsc::channel::(100); + // Create SyncManager + let syncer = SyncManager::new(channel_p2p_receive, peer_table.clone(), is_snap_sync); // TODO: Check every module starts properly. let tracker = TaskTracker::new(); @@ -194,6 +202,7 @@ async fn main() { store.clone(), jwt_secret, local_p2p_node, + syncer, ) .into_future(); @@ -226,8 +235,9 @@ async fn main() { tcp_socket_addr, bootnodes, signer, + peer_table, store, - is_snap_sync + channel_backend_send ) .into_future(); tracker.spawn(networking); diff --git a/crates/common/types/fork_id.rs b/crates/common/types/fork_id.rs index a44795b35..8bc7ef647 100644 --- a/crates/common/types/fork_id.rs +++ b/crates/common/types/fork_id.rs @@ -10,7 +10,7 @@ use ethereum_types::H32; use super::{BlockHash, BlockNumber, ChainConfig}; -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ForkId { fork_hash: H32, fork_next: BlockNumber, diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 3d8fdb1dc..9f9dfd998 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -1,6 +1,6 @@ use crate::{ discv4::{time_now_unix, FindNodeRequest}, - rlpx::message::Message, + rlpx::{error::RLPxError, message::Message}, types::Node, }; use ethereum_rust_core::{H256, H512, U256}; @@ -188,7 +188,7 @@ impl KademliaTable { /// ## Dev note: /// This function should be improved: /// We might keep the `peers` list sorted by last_ping as we would avoid unnecessary loops - pub fn get_least_recently_pinged_peers(&mut self, limit: usize) -> Vec { + pub fn get_least_recently_pinged_peers(&self, limit: usize) -> Vec { let mut peers = vec![]; for bucket in &self.buckets { @@ -225,17 +225,6 @@ impl KademliaTable { self.replace_peer_inner(node_id, bucket_idx) } - pub fn set_sender(&mut self, node_id: H512, sender: tokio::sync::mpsc::Sender) { - let bucket_idx = bucket_number(self.local_node_id, node_id); - if let Some(peer) = self.buckets[bucket_idx] - .peers - .iter_mut() - .find(|peer| peer.node.node_id == node_id) - { - peer.sender = Some(sender) - } - } - #[cfg(test)] pub fn replace_peer_on_custom_bucket( &mut self, @@ -266,6 +255,34 @@ impl KademliaTable { None } + + /// Set the sender end of the channel between the kademlia table and the peer's active connection + /// This function should be called each time a connection is established so the backend can send requests to the peers + pub fn set_sender(&mut self, node_id: H512, sender: tokio::sync::mpsc::Sender) { + let bucket_idx = bucket_number(self.local_node_id, node_id); + if let Some(peer) = self.buckets[bucket_idx] + .peers + .iter_mut() + .find(|peer| peer.node.node_id == node_id) + { + peer.sender = Some(sender) + } + } + + /// TODO: Use most recently pinged peer instead + fn get_peer(&self) -> Option { + self.get_least_recently_pinged_peers(1).pop() + } + + /// Send a message to a peer, returns an error if there are no active peers + pub async fn send_message_to_peer(&self, message: Message) -> Result<(), RLPxError> { + if let Some(sender) = self.get_peer().and_then(|peer| peer.sender) { + Ok(sender.send(message).await?) + } else { + // Let the caller decide when to retry + Err(RLPxError::NoPeers) + } + } } /// Computes the distance between two nodes according to the discv4 protocol diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index f03e73f3a..fa99de349 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -16,10 +16,11 @@ use k256::{ ecdsa::SigningKey, elliptic_curve::{sec1::ToEncodedPoint, PublicKey}, }; -use kademlia::{bucket_number, KademliaTable, MAX_NODES_PER_BUCKET}; +pub use kademlia::KademliaTable; +use kademlia::{bucket_number, MAX_NODES_PER_BUCKET}; use rand::rngs::OsRng; -use rlpx::{connection::RLPxConnection, error::RLPxError, message::Message as RLPxMessage}; -use snap_sync::SnapSyncManager; +pub use rlpx::message::Message as RLPxMessage; +use rlpx::{connection::RLPxConnection, error::RLPxError}; use tokio::{ net::{TcpSocket, TcpStream, UdpSocket}, sync::{broadcast, mpsc, Mutex}, @@ -33,7 +34,7 @@ pub(crate) mod discv4; pub(crate) mod kademlia; pub mod rlpx; pub(crate) mod snap; -mod snap_sync; +pub mod sync; pub mod types; const MAX_DISC_PACKET_SIZE: usize = 1280; @@ -44,25 +45,22 @@ const MAX_DISC_PACKET_SIZE: usize = 1280; // we should bump this limit. const MAX_MESSAGES_TO_BROADCAST: usize = 1000; +pub fn peer_table(signer: SigningKey) -> Arc> { + let local_node_id = node_id_from_signing_key(&signer); + Arc::new(Mutex::new(KademliaTable::new(local_node_id))) +} + pub async fn start_network( udp_addr: SocketAddr, tcp_addr: SocketAddr, bootnodes: Vec, signer: SigningKey, + peer_table: Arc>, storage: Store, - start_snap_sync: bool, + channel_backend_send: tokio::sync::mpsc::Sender, ) { info!("Starting discovery service at {udp_addr}"); info!("Listening for requests at {tcp_addr}"); - let local_node_id = node_id_from_signing_key(&signer); - let table = Arc::new(Mutex::new(KademliaTable::new(local_node_id))); - // Communication between the backend and the main listen loop - let (channel_backend_send, channel_backend_receive) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - if start_snap_sync { - let snap_sync_manager = SnapSyncManager::new(channel_backend_receive); - snap_sync_manager.start(table.clone()) // we need the table in order to send requests to a random peer - } let (channel_broadcast_send_end, _) = tokio::sync::broadcast::channel::<( tokio::task::Id, Arc, @@ -71,7 +69,7 @@ pub async fn start_network( udp_addr, signer.clone(), storage.clone(), - table.clone(), + peer_table.clone(), bootnodes, channel_broadcast_send_end.clone(), channel_backend_send.clone(), @@ -80,7 +78,7 @@ pub async fn start_network( tcp_addr, signer.clone(), storage.clone(), - table.clone(), + peer_table.clone(), channel_broadcast_send_end, channel_backend_send, )); diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 7e6c99dae..32b0edd97 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -332,11 +332,13 @@ impl RLPxConnection { let response = process_trie_nodes_request(req, self.storage.clone())?; self.send(Message::TrieNodes(response)).await? } - // Send snap request responses to the SyncManager + // Send response messages to the backend message @ Message::AccountRange(_) | message @ Message::StorageRanges(_) | message @ Message::ByteCodes(_) - | message @ Message::TrieNodes(_) => backend_send.send(message).await?, + | message @ Message::TrieNodes(_) + | message @ Message::BlockBodies(_) + | message @ Message::BlockHeaders(_) => backend_send.send(message).await?, // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::MessageNotHandled(format!("{message}"))), }; diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 242938e0e..79ac967c8 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -8,7 +8,7 @@ use super::message::Message; // TODO improve errors #[derive(Debug, Error)] -pub(crate) enum RLPxError { +pub enum RLPxError { #[error("{0}")] HandshakeError(String), #[error("{0}")] @@ -45,6 +45,8 @@ pub(crate) enum RLPxError { RecvError(#[from] RecvError), #[error(transparent)] Send(#[from] tokio::sync::mpsc::error::SendError), + #[error("No peers to interact with yet")] + NoPeers, } // Grouping all cryptographic related errors in a single CryptographicError variant diff --git a/crates/networking/p2p/rlpx/eth/blocks.rs b/crates/networking/p2p/rlpx/eth/blocks.rs index 1ec932c0c..57b41f7f6 100644 --- a/crates/networking/p2p/rlpx/eth/blocks.rs +++ b/crates/networking/p2p/rlpx/eth/blocks.rs @@ -37,6 +37,12 @@ impl RLPEncode for HashOrNumber { } } +impl From for HashOrNumber { + fn from(value: BlockHash) -> Self { + Self::Hash(value) + } +} + impl RLPDecode for HashOrNumber { fn decode_unfinished(buf: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { let first_byte = buf.first().ok_or(RLPDecodeError::InvalidLength)?; @@ -54,8 +60,8 @@ impl RLPDecode for HashOrNumber { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#getblockheaders-0x03 -#[derive(Debug)] -pub(crate) struct GetBlockHeaders { +#[derive(Debug, Clone)] +pub struct GetBlockHeaders { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, @@ -159,8 +165,8 @@ impl RLPxMessage for GetBlockHeaders { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#blockheaders-0x04 -#[derive(Debug)] -pub(crate) struct BlockHeaders { +#[derive(Debug, Clone)] +pub struct BlockHeaders { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, @@ -199,8 +205,8 @@ impl RLPxMessage for BlockHeaders { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#getblockbodies-0x05 -#[derive(Debug)] -pub(crate) struct GetBlockBodies { +#[derive(Debug, Clone)] +pub struct GetBlockBodies { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, @@ -264,8 +270,8 @@ impl RLPxMessage for GetBlockBodies { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#blockbodies-0x06 -#[derive(Debug)] -pub(crate) struct BlockBodies { +#[derive(Debug, Clone)] +pub struct BlockBodies { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages pub id: u64, diff --git a/crates/networking/p2p/rlpx/eth/status.rs b/crates/networking/p2p/rlpx/eth/status.rs index b0e21bbf8..0839ab046 100644 --- a/crates/networking/p2p/rlpx/eth/status.rs +++ b/crates/networking/p2p/rlpx/eth/status.rs @@ -12,7 +12,7 @@ use ethereum_rust_rlp::{ structs::{Decoder, Encoder}, }; -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct StatusMessage { pub(crate) eth_version: u32, pub(crate) network_id: u64, diff --git a/crates/networking/p2p/rlpx/eth/transactions.rs b/crates/networking/p2p/rlpx/eth/transactions.rs index d40e85dec..26ed7da76 100644 --- a/crates/networking/p2p/rlpx/eth/transactions.rs +++ b/crates/networking/p2p/rlpx/eth/transactions.rs @@ -12,7 +12,7 @@ use crate::rlpx::{ // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#transactions-0x02 // Broadcast message -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct Transactions { pub(crate) transactions: Vec, } diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 6415d74e4..bdcbfeb01 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -18,8 +18,8 @@ pub trait RLPxMessage: Sized { fn decode(msg_data: &[u8]) -> Result; } -#[derive(Debug)] -pub(crate) enum Message { +#[derive(Debug, Clone)] +pub enum Message { Hello(HelloMessage), Disconnect(DisconnectMessage), Ping(PingMessage), diff --git a/crates/networking/p2p/rlpx/p2p.rs b/crates/networking/p2p/rlpx/p2p.rs index 70573c021..5a6aac053 100644 --- a/crates/networking/p2p/rlpx/p2p.rs +++ b/crates/networking/p2p/rlpx/p2p.rs @@ -44,7 +44,7 @@ impl RLPDecode for Capability { } } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct HelloMessage { pub(crate) capabilities: Vec<(Capability, u8)>, pub(crate) node_id: PublicKey, @@ -100,7 +100,7 @@ impl RLPxMessage for HelloMessage { } } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct DisconnectMessage { pub(crate) reason: Option, } @@ -147,7 +147,7 @@ impl RLPxMessage for DisconnectMessage { } } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct PingMessage {} impl PingMessage { @@ -177,7 +177,7 @@ impl RLPxMessage for PingMessage { } } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct PongMessage {} impl PongMessage { diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index f83ada6b9..50ac9b864 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -16,7 +16,7 @@ use ethereum_rust_rlp::{ // Snap Capability Messages -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct GetAccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response pub id: u64, @@ -26,7 +26,7 @@ pub(crate) struct GetAccountRange { pub response_bytes: u64, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response pub id: u64, @@ -34,7 +34,7 @@ pub(crate) struct AccountRange { pub proof: Vec, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct GetStorageRanges { pub id: u64, pub root_hash: H256, @@ -44,27 +44,27 @@ pub(crate) struct GetStorageRanges { pub response_bytes: u64, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct StorageRanges { pub id: u64, pub slots: Vec>, pub proof: Vec, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct GetByteCodes { pub id: u64, pub hashes: Vec, pub bytes: u64, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct ByteCodes { pub id: u64, pub codes: Vec, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct GetTrieNodes { pub id: u64, pub root_hash: H256, @@ -74,7 +74,7 @@ pub(crate) struct GetTrieNodes { pub bytes: u64, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub(crate) struct TrieNodes { pub id: u64, pub nodes: Vec, @@ -320,7 +320,7 @@ impl RLPxMessage for TrieNodes { // Intermediate structures -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AccountRangeUnit { pub hash: H256, pub account: AccountStateSlim, @@ -334,7 +334,7 @@ pub struct AccountStateSlim { pub code_hash: Bytes, } -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct StorageSlot { pub hash: H256, pub data: U256, diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs new file mode 100644 index 000000000..edc9c44bc --- /dev/null +++ b/crates/networking/p2p/sync.rs @@ -0,0 +1,145 @@ +use std::{sync::Arc, time::Duration}; + +use ethereum_rust_core::H256; +use tokio::sync::Mutex; +use tracing::{debug, info}; + +use crate::{ + kademlia::KademliaTable, + rlpx::{ + eth::blocks::{BlockHeaders, GetBlockHeaders}, + message::Message, + }, +}; + +/// Manager in charge of the snap-sync(for now, will also handle full sync) process +/// TaskList: +/// A) Fetch latest block headers (should we ask what the latest block is first?) +/// B) Validate block headers +/// C) Fetch full Blocks and Receipts || Download Raw State (accounts, storages, bytecodes) +/// D) Healing +#[derive(Debug)] +pub struct SyncManager { + // true: syncmode = snap, false = syncmode = full + snap_mode: bool, + peers: Arc>, + // Receiver end of the channel between the manager and the main p2p listen loop + reply_receiver: tokio::sync::mpsc::Receiver, + active: bool, +} + +impl SyncManager { + pub fn new( + reply_receiver: tokio::sync::mpsc::Receiver, + peers: Arc>, + snap_mode: bool, + ) -> Self { + Self { + snap_mode, + peers, + reply_receiver, + active: false, + } + } + // TODO: only uses snap sync, should also process full sync once implemented + pub async fn start_sync(&mut self, current_head: H256, sync_head: H256) { + const BYTES_PER_REQUEST: u64 = 500; // TODO: Adjust + const REPLY_TIMEOUT: Duration = Duration::from_secs(30); + info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); + self.active = true; + // Request all block headers between the current head and the sync head + // We will begin from the current head so that we download the earliest state first + // This step is not parallelized + // Ask for block headers + let mut block_headers_request = GetBlockHeaders { + id: 17, // TODO: randomize + skip: 0, + startblock: current_head.into(), + limit: BYTES_PER_REQUEST, + reverse: false, + }; + loop { + // TODO: Randomize id + // Send a GetBlockHeaders request to a peer + if self + .peers + .lock() + .await + .send_message_to_peer(Message::GetBlockHeaders(block_headers_request.clone())) + .await + .is_err() + { + // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + continue; + }; + // Wait for the peer to reply + if let Ok(Some(message)) = tokio::time::timeout( + REPLY_TIMEOUT, + receive_block_headers(&mut self.reply_receiver, block_headers_request.id), + ) + .await + { + // We received the correct message, we can now + // A) Validate the batch of headers received and start downloading their state + // B) Check if we need to download another batch (aka we don't have the sync_head yet) + + // If the response is empty, lets ask another peer + if message.block_headers.is_empty() { + continue; + } + // Discard the first header as we already have it + let headers = &message.block_headers[1..]; + let block_hashes = headers + .iter() + .map(|header| header.compute_block_hash()) + .collect::>(); + debug!( + "Received header batch {}..{}", + block_hashes.first().unwrap(), + block_hashes.last().unwrap() + ); + // Process headers (validate + download state) + // TODO! + // Check if we already reached our sync head or if we need to fetch more blocks + if !block_hashes.contains(&sync_head) { + // Update the request to fetch the next batch + block_headers_request.startblock = (*block_hashes.last().unwrap()).into(); + } else { + // No more headers to request + break; + } + }; + // Reply timeouted/ peer shut down, lets try a different peer + } + + // Sync finished + self.active = false; + } + + /// Creates a dummy SyncManager for tests where syncing is not needed + /// This should only be used it tests as it won't be able to connect to the p2p network + pub fn dummy() -> Self { + let dummy_peer_table = Arc::new(Mutex::new(KademliaTable::new(Default::default()))); + Self { + snap_mode: false, + peers: dummy_peer_table, + reply_receiver: tokio::sync::mpsc::channel(0).1, + active: false, + } + } +} + +async fn receive_block_headers( + channel: &mut tokio::sync::mpsc::Receiver, + id: u64, +) -> Option { + loop { + match channel.recv().await { + Some(Message::BlockHeaders(response)) if response.id == id => return Some(response), + // Ignore replies that don't match the expected id (such as late responses) + Some(_other_response) => continue, + None => return None, + } + } +} diff --git a/crates/networking/rpc/eth/filter.rs b/crates/networking/rpc/eth/filter.rs index c0e6018fd..b9cb41240 100644 --- a/crates/networking/rpc/eth/filter.rs +++ b/crates/networking/rpc/eth/filter.rs @@ -273,6 +273,7 @@ mod tests { utils::{test_utils::example_p2p_node, RpcRequest}, }; use ethereum_rust_core::types::Genesis; + use ethereum_rust_net::sync::SyncManager; use ethereum_rust_storage::{EngineType, Store}; use serde_json::{json, Value}; @@ -442,6 +443,7 @@ mod tests { jwt_secret: Default::default(), local_p2p_node: example_p2p_node(), active_filters: filters_pointer.clone(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; let request: RpcRequest = serde_json::from_value(json_req).expect("Test json is incorrect"); let genesis_config: Genesis = @@ -494,6 +496,7 @@ mod tests { local_p2p_node: example_p2p_node(), jwt_secret: Default::default(), active_filters: active_filters.clone(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; map_http_requests(&uninstall_filter_req, context).unwrap(); @@ -513,6 +516,7 @@ mod tests { local_p2p_node: example_p2p_node(), active_filters: active_filters.clone(), jwt_secret: Default::default(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; let uninstall_filter_req: RpcRequest = serde_json::from_value(json!( { diff --git a/crates/networking/rpc/eth/gas_price.rs b/crates/networking/rpc/eth/gas_price.rs index c35c5f241..b8dbd4eca 100644 --- a/crates/networking/rpc/eth/gas_price.rs +++ b/crates/networking/rpc/eth/gas_price.rs @@ -114,11 +114,15 @@ mod tests { }, Address, Bloom, H256, U256, }; - use ethereum_rust_net::types::Node; + use ethereum_rust_net::{sync::SyncManager, types::Node}; use ethereum_rust_storage::{EngineType, Store}; use hex_literal::hex; use serde_json::json; - use std::{net::Ipv4Addr, str::FromStr}; + use std::{ + net::Ipv4Addr, + str::FromStr, + sync::{Arc, Mutex}, + }; // Base price for each test transaction. const BASE_PRICE_IN_WEI: u64 = 10_u64.pow(9); fn test_header(block_num: u64) -> BlockHeader { @@ -385,6 +389,7 @@ mod tests { node_id: Default::default(), }, active_filters: Default::default(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), } } } diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 645386c29..35a56ff01 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -32,6 +32,7 @@ use eth::{ GetTransactionByHashRequest, GetTransactionReceiptRequest, }, }; +use ethereum_rust_net::sync::SyncManager; use serde_json::Value; use std::{ collections::HashMap, @@ -65,6 +66,7 @@ pub struct RpcApiContext { jwt_secret: Bytes, local_p2p_node: Node, active_filters: ActiveFilters, + syncer: Arc>, } trait RpcHandler: Sized { @@ -92,6 +94,7 @@ pub async fn start_api( storage: Store, jwt_secret: Bytes, local_p2p_node: Node, + syncer: SyncManager, ) { // TODO: Refactor how filters are handled, // filters are used by the filters endpoints (eth_newFilter, eth_getFilterChanges, ...etc) @@ -101,6 +104,7 @@ pub async fn start_api( jwt_secret, local_p2p_node, active_filters: active_filters.clone(), + syncer: Arc::new(Mutex::new(syncer)), }; // Periodically clean up the active filters for the filters endpoints. @@ -325,6 +329,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let rpc_response = rpc_response(request.id, result); @@ -362,6 +367,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let response = rpc_response(request.id, result); @@ -391,6 +397,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), + syncer: Arc::new(Mutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let response = diff --git a/crates/networking/rpc/utils.rs b/crates/networking/rpc/utils.rs index 3fd23c2da..e41cef1a2 100644 --- a/crates/networking/rpc/utils.rs +++ b/crates/networking/rpc/utils.rs @@ -247,7 +247,7 @@ pub mod test_utils { use std::{net::SocketAddr, str::FromStr}; use ethereum_rust_core::H512; - use ethereum_rust_net::types::Node; + use ethereum_rust_net::{sync::SyncManager, types::Node}; use ethereum_rust_storage::{EngineType, Store}; use crate::start_api; @@ -285,6 +285,14 @@ pub mod test_utils { let jwt_secret = Default::default(); let local_p2p_node = example_p2p_node(); - start_api(http_addr, authrpc_addr, storage, jwt_secret, local_p2p_node).await; + start_api( + http_addr, + authrpc_addr, + storage, + jwt_secret, + local_p2p_node, + SyncManager::dummy(), + ) + .await; } } From 798fb058ccd7af44b53cd9e1f78b8dfe6c270e4e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 21 Nov 2024 18:47:05 -0300 Subject: [PATCH 216/345] Start syncer in fork_choice handler --- crates/networking/p2p/sync.rs | 5 ----- crates/networking/rpc/engine/fork_choice.rs | 17 +++++++++++++++++ crates/networking/rpc/eth/filter.rs | 7 ++++--- crates/networking/rpc/eth/gas_price.rs | 7 ++----- crates/networking/rpc/rpc.rs | 12 ++++++------ 5 files changed, 29 insertions(+), 19 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index edc9c44bc..f51de6722 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -25,7 +25,6 @@ pub struct SyncManager { peers: Arc>, // Receiver end of the channel between the manager and the main p2p listen loop reply_receiver: tokio::sync::mpsc::Receiver, - active: bool, } impl SyncManager { @@ -38,7 +37,6 @@ impl SyncManager { snap_mode, peers, reply_receiver, - active: false, } } // TODO: only uses snap sync, should also process full sync once implemented @@ -46,7 +44,6 @@ impl SyncManager { const BYTES_PER_REQUEST: u64 = 500; // TODO: Adjust const REPLY_TIMEOUT: Duration = Duration::from_secs(30); info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); - self.active = true; // Request all block headers between the current head and the sync head // We will begin from the current head so that we download the earliest state first // This step is not parallelized @@ -114,7 +111,6 @@ impl SyncManager { } // Sync finished - self.active = false; } /// Creates a dummy SyncManager for tests where syncing is not needed @@ -125,7 +121,6 @@ impl SyncManager { snap_mode: false, peers: dummy_peer_table, reply_receiver: tokio::sync::mpsc::channel(0).1, - active: false, } } } diff --git a/crates/networking/rpc/engine/fork_choice.rs b/crates/networking/rpc/engine/fork_choice.rs index 78b7f7e89..777849d1b 100644 --- a/crates/networking/rpc/engine/fork_choice.rs +++ b/crates/networking/rpc/engine/fork_choice.rs @@ -89,6 +89,23 @@ impl RpcHandler for ForkChoiceUpdatedV3 { self.fork_choice_state.finalized_block_hash, ) { Ok(head) => head, + // TODO: handle this more elegantly + Err(error @ InvalidForkChoice::Syncing) => { + // Start sync + let current_number = context.storage.get_latest_block_number()?.unwrap(); + let current_head = context + .storage + .get_canonical_block_hash(current_number)? + .unwrap(); + let sync_head = self.fork_choice_state.head_block_hash; + tokio::spawn(async move { + // If we can't get hold of the syncer, then it means that there is an active sync in process + if let Ok(mut syncer) = context.syncer.try_lock() { + syncer.start_sync(current_head, sync_head).await + } + }); + return fork_choice_error_to_response(error); + } Err(error) => return fork_choice_error_to_response(error), }; diff --git a/crates/networking/rpc/eth/filter.rs b/crates/networking/rpc/eth/filter.rs index b9cb41240..3e241003f 100644 --- a/crates/networking/rpc/eth/filter.rs +++ b/crates/networking/rpc/eth/filter.rs @@ -257,6 +257,7 @@ mod tests { sync::{Arc, Mutex}, time::{Duration, Instant}, }; + use tokio::sync::Mutex as TokioMutex; use super::ActiveFilters; use crate::{ @@ -443,7 +444,7 @@ mod tests { jwt_secret: Default::default(), local_p2p_node: example_p2p_node(), active_filters: filters_pointer.clone(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; let request: RpcRequest = serde_json::from_value(json_req).expect("Test json is incorrect"); let genesis_config: Genesis = @@ -496,7 +497,7 @@ mod tests { local_p2p_node: example_p2p_node(), jwt_secret: Default::default(), active_filters: active_filters.clone(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; map_http_requests(&uninstall_filter_req, context).unwrap(); @@ -516,7 +517,7 @@ mod tests { local_p2p_node: example_p2p_node(), active_filters: active_filters.clone(), jwt_secret: Default::default(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; let uninstall_filter_req: RpcRequest = serde_json::from_value(json!( { diff --git a/crates/networking/rpc/eth/gas_price.rs b/crates/networking/rpc/eth/gas_price.rs index b8dbd4eca..5f5680a85 100644 --- a/crates/networking/rpc/eth/gas_price.rs +++ b/crates/networking/rpc/eth/gas_price.rs @@ -118,11 +118,8 @@ mod tests { use ethereum_rust_storage::{EngineType, Store}; use hex_literal::hex; use serde_json::json; - use std::{ - net::Ipv4Addr, - str::FromStr, - sync::{Arc, Mutex}, - }; + use std::{net::Ipv4Addr, str::FromStr, sync::Arc}; + use tokio::sync::Mutex; // Base price for each test transaction. const BASE_PRICE_IN_WEI: u64 = 10_u64.pow(9); fn test_header(block_num: u64) -> BlockHeader { diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 35a56ff01..e6031d52e 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -41,7 +41,7 @@ use std::{ sync::{Arc, Mutex}, time::Duration, }; -use tokio::net::TcpListener; +use tokio::{net::TcpListener, sync::Mutex as TokioMutex}; use tracing::info; use types::transaction::SendRawTransactionRequest; use utils::{ @@ -66,7 +66,7 @@ pub struct RpcApiContext { jwt_secret: Bytes, local_p2p_node: Node, active_filters: ActiveFilters, - syncer: Arc>, + syncer: Arc>, } trait RpcHandler: Sized { @@ -104,7 +104,7 @@ pub async fn start_api( jwt_secret, local_p2p_node, active_filters: active_filters.clone(), - syncer: Arc::new(Mutex::new(syncer)), + syncer: Arc::new(TokioMutex::new(syncer)), }; // Periodically clean up the active filters for the filters endpoints. @@ -329,7 +329,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let rpc_response = rpc_response(request.id, result); @@ -367,7 +367,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let response = rpc_response(request.id, result); @@ -397,7 +397,7 @@ mod tests { storage, jwt_secret: Default::default(), active_filters: Default::default(), - syncer: Arc::new(Mutex::new(SyncManager::dummy())), + syncer: Arc::new(TokioMutex::new(SyncManager::dummy())), }; let result = map_http_requests(&request, context); let response = From a875680596c29c8ff67c31853592718179c9b11e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 22 Nov 2024 17:24:44 -0300 Subject: [PATCH 217/345] First iteration of state retrieval planning --- crates/networking/p2p/sync.rs | 46 ++++++++++++++++++++++++++++++++--- 1 file changed, 43 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index f51de6722..4961e97f4 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,6 +1,6 @@ use std::{sync::Arc, time::Duration}; -use ethereum_rust_core::H256; +use ethereum_rust_core::{types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, H256}; use tokio::sync::Mutex; use tracing::{debug, info}; @@ -55,8 +55,12 @@ impl SyncManager { limit: BYTES_PER_REQUEST, reverse: false, }; + let mut all_block_headers = vec![]; + let mut all_block_hashes = vec![]; loop { // TODO: Randomize id + block_headers_request.id +=1; + info!("[Sync] Sending request {block_headers_request:?}"); // Send a GetBlockHeaders request to a peer if self .peers @@ -66,6 +70,7 @@ impl SyncManager { .await .is_err() { + info!("[Sync] No peers available, retrying in 10 sec"); // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; continue; @@ -83,6 +88,12 @@ impl SyncManager { // If the response is empty, lets ask another peer if message.block_headers.is_empty() { + debug!("[Sync] Bad peer response"); + continue; + } + // Validate header batch + if validate_header_batch(&message.block_headers).is_err() { + debug!("[Sync] Invalid header in batch"); continue; } // Discard the first header as we already have it @@ -96,8 +107,12 @@ impl SyncManager { block_hashes.first().unwrap(), block_hashes.last().unwrap() ); - // Process headers (validate + download state) - // TODO! + + // First iteration will not process the batch, but will wait for all headers to be fetched and validated + // before processing the whole batch + all_block_headers.extend_from_slice(&headers); + all_block_hashes.extend_from_slice(&block_hashes); + // Check if we already reached our sync head or if we need to fetch more blocks if !block_hashes.contains(&sync_head) { // Update the request to fetch the next batch @@ -107,8 +122,19 @@ impl SyncManager { break; } }; + debug!("[Sync] Peer response timeout"); // Reply timeouted/ peer shut down, lets try a different peer } + info!("[Sync] All headers fetched and validated"); + // [First Iteration] We finished fetching all headers, now we can process them + // We will launch 3 tasks to: + // 1) Fetch each block's state via snap p2p requests + // 2) Fetch each blocks and its receipts via eth p2p requests + // 3) Receive replies from the receiver and send them to the two tasks + let (snap_state_receiver, snap_state_sender) = + tokio::sync::mpsc::channel::(10); + let (block_and_receipt_receiver, block_and_receipt_sender) = + tokio::sync::mpsc::channel::(10); // Sync finished } @@ -138,3 +164,17 @@ async fn receive_block_headers( } } } + +fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHeaderError> { + // The first header is a header we have already validated (either current last block or last block in previous batch) + for headers in headers.windows(2) { + validate_block_header(&headers[0], &headers[1])?; + } + Ok(()) +} + +// Process an already validated block header +// In parallel: Fetches the block and its receipt +fn process_header(header: &BlockHeader, hash: &BlockHash) { + +} From 356eaf03d87e4e6748682afe7ba52fdcaf7c9d56 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 22 Nov 2024 17:25:20 -0300 Subject: [PATCH 218/345] [REVERT ME] Make V2 endpoints work in order to run hive sync test --- crates/networking/rpc/engine/payload.rs | 22 ++++++++++------------ crates/networking/rpc/rpc.rs | 2 ++ crates/networking/rpc/types/payload.rs | 4 ++-- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index b85188b7d..28cef6579 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -1,7 +1,7 @@ use ethereum_rust_blockchain::add_block; use ethereum_rust_blockchain::error::ChainError; use ethereum_rust_blockchain::payload::build_payload; -use ethereum_rust_core::types::Fork; +//use ethereum_rust_core::types::Fork; use ethereum_rust_core::{H256, U256}; use serde_json::Value; use tracing::{error, info, warn}; @@ -43,16 +43,14 @@ impl RpcHandler for NewPayloadV3Request { let params = params .as_ref() .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - if params.len() != 3 { - return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - } + // if params.len() != 3 { + // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); + // } Ok(NewPayloadV3Request { payload: serde_json::from_value(params[0].clone()) .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: serde_json::from_value(params[1].clone()) - .map_err(|_| RpcErr::WrongParam("expected_blob_versioned_hashes".to_string()))?, - parent_beacon_block_root: serde_json::from_value(params[2].clone()) - .map_err(|_| RpcErr::WrongParam("parent_beacon_block_root".to_string()))?, + expected_blob_versioned_hashes: vec![], + parent_beacon_block_root: H256::zero(), }) } @@ -78,10 +76,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post Cancun fork let chain_config = storage.get_chain_config()?; - let current_fork = chain_config.get_fork(block.header.timestamp); - if current_fork < Fork::Cancun { - return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - } + // let current_fork = chain_config.get_fork(block.header.timestamp); + // if current_fork < Fork::Cancun { + // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); + // } // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index e6031d52e..951849199 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -254,7 +254,9 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), + "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), + "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index b1abb57cb..b1d1b5aeb 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,9 +35,9 @@ pub struct ExecutionPayloadV3 { pub block_hash: H256, transactions: Vec, withdrawals: Vec, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] excess_blob_gas: u64, } From a6d64f9176a7b95cc82b1900214550e3ae010d9b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 25 Nov 2024 10:55:01 -0300 Subject: [PATCH 219/345] Add parallel state fetching first draft --- crates/networking/p2p/rlpx/connection.rs | 3 +- crates/networking/p2p/rlpx/eth/receipts.rs | 1 + crates/networking/p2p/rlpx/message.rs | 8 + crates/networking/p2p/sync.rs | 166 ++++++++++++++++++--- 4 files changed, 155 insertions(+), 23 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 32b0edd97..40674c3dd 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -338,7 +338,8 @@ impl RLPxConnection { | message @ Message::ByteCodes(_) | message @ Message::TrieNodes(_) | message @ Message::BlockBodies(_) - | message @ Message::BlockHeaders(_) => backend_send.send(message).await?, + | message @ Message::BlockHeaders(_) + | message @ Message::Receipts(_) => backend_send.send(message).await?, // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::MessageNotHandled(format!("{message}"))), }; diff --git a/crates/networking/p2p/rlpx/eth/receipts.rs b/crates/networking/p2p/rlpx/eth/receipts.rs index 0eae0af0a..5247a306a 100644 --- a/crates/networking/p2p/rlpx/eth/receipts.rs +++ b/crates/networking/p2p/rlpx/eth/receipts.rs @@ -48,6 +48,7 @@ impl RLPxMessage for GetReceipts { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#receipts-0x10 +#[derive(Debug, Clone)] pub(crate) struct Receipts { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index bdcbfeb01..00fd07b4f 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -3,6 +3,7 @@ use ethereum_rust_rlp::error::{RLPDecodeError, RLPEncodeError}; use std::fmt::Display; use super::eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}; +use super::eth::receipts::Receipts; use super::eth::status::StatusMessage; use super::eth::transactions::Transactions; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; @@ -31,6 +32,7 @@ pub enum Message { Transactions(Transactions), GetBlockBodies(GetBlockBodies), BlockBodies(BlockBodies), + Receipts(Receipts), // snap capability GetAccountRange(GetAccountRange), AccountRange(AccountRange), @@ -64,6 +66,7 @@ impl Message { 0x13 => Ok(Message::GetBlockHeaders(GetBlockHeaders::decode(msg_data)?)), 0x14 => Ok(Message::BlockHeaders(BlockHeaders::decode(msg_data)?)), 0x15 => Ok(Message::GetBlockBodies(GetBlockBodies::decode(msg_data)?)), + 0x20 => Ok(Message::Receipts(Receipts::decode(msg_data)?)), 0x21 => Ok(Message::GetAccountRange(GetAccountRange::decode(msg_data)?)), 0x22 => Ok(Message::AccountRange(AccountRange::decode(msg_data)?)), 0x23 => Ok(Message::GetStorageRanges(GetStorageRanges::decode( @@ -120,6 +123,10 @@ impl Message { 0x16_u8.encode(buf); msg.encode(buf) } + Message::Receipts(msg) => { + 0x20_u8.encode(buf); + msg.encode(buf) + } Message::GetAccountRange(msg) => { 0x21_u8.encode(buf); msg.encode(buf) @@ -169,6 +176,7 @@ impl Display for Message { Message::BlockBodies(_) => "eth:BlockBodies".fmt(f), Message::Transactions(_) => "eth:TransactionsMessage".fmt(f), Message::GetBlockBodies(_) => "eth:GetBlockBodies".fmt(f), + Message::Receipts(_) => "eth:Receipts".fmt(f), Message::GetAccountRange(_) => "snap:GetAccountRange".fmt(f), Message::AccountRange(_) => "snap:AccountRange".fmt(f), Message::GetStorageRanges(_) => "snap:GetStorageRanges".fmt(f), diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 4961e97f4..f3d289b6b 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,16 +1,20 @@ use std::{sync::Arc, time::Duration}; -use ethereum_rust_core::{types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, H256}; +use ethereum_rust_core::{ + types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, + H256, +}; use tokio::sync::Mutex; use tracing::{debug, info}; use crate::{ kademlia::KademliaTable, rlpx::{ - eth::blocks::{BlockHeaders, GetBlockHeaders}, + eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}, message::Message, }, }; +const REPLY_TIMEOUT: Duration = Duration::from_secs(30); /// Manager in charge of the snap-sync(for now, will also handle full sync) process /// TaskList: @@ -24,7 +28,7 @@ pub struct SyncManager { snap_mode: bool, peers: Arc>, // Receiver end of the channel between the manager and the main p2p listen loop - reply_receiver: tokio::sync::mpsc::Receiver, + reply_receiver: Arc>>, } impl SyncManager { @@ -36,13 +40,12 @@ impl SyncManager { Self { snap_mode, peers, - reply_receiver, + reply_receiver: Arc::new(Mutex::new(reply_receiver)), } } // TODO: only uses snap sync, should also process full sync once implemented pub async fn start_sync(&mut self, current_head: H256, sync_head: H256) { const BYTES_PER_REQUEST: u64 = 500; // TODO: Adjust - const REPLY_TIMEOUT: Duration = Duration::from_secs(30); info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); // Request all block headers between the current head and the sync head // We will begin from the current head so that we download the earliest state first @@ -59,7 +62,7 @@ impl SyncManager { let mut all_block_hashes = vec![]; loop { // TODO: Randomize id - block_headers_request.id +=1; + block_headers_request.id += 1; info!("[Sync] Sending request {block_headers_request:?}"); // Send a GetBlockHeaders request to a peer if self @@ -78,7 +81,10 @@ impl SyncManager { // Wait for the peer to reply if let Ok(Some(message)) = tokio::time::timeout( REPLY_TIMEOUT, - receive_block_headers(&mut self.reply_receiver, block_headers_request.id), + receive_block_headers( + &mut *self.reply_receiver.lock().await, + block_headers_request.id, + ), ) .await { @@ -88,12 +94,12 @@ impl SyncManager { // If the response is empty, lets ask another peer if message.block_headers.is_empty() { - debug!("[Sync] Bad peer response"); + info!("[Sync] Bad peer response"); continue; } // Validate header batch if validate_header_batch(&message.block_headers).is_err() { - debug!("[Sync] Invalid header in batch"); + info!("[Sync] Invalid header in batch"); continue; } // Discard the first header as we already have it @@ -102,7 +108,7 @@ impl SyncManager { .iter() .map(|header| header.compute_block_hash()) .collect::>(); - debug!( + info!( "Received header batch {}..{}", block_hashes.first().unwrap(), block_hashes.last().unwrap() @@ -122,7 +128,7 @@ impl SyncManager { break; } }; - debug!("[Sync] Peer response timeout"); + info!("[Sync] Peer response timeout"); // Reply timeouted/ peer shut down, lets try a different peer } info!("[Sync] All headers fetched and validated"); @@ -131,11 +137,28 @@ impl SyncManager { // 1) Fetch each block's state via snap p2p requests // 2) Fetch each blocks and its receipts via eth p2p requests // 3) Receive replies from the receiver and send them to the two tasks - let (snap_state_receiver, snap_state_sender) = - tokio::sync::mpsc::channel::(10); - let (block_and_receipt_receiver, block_and_receipt_sender) = - tokio::sync::mpsc::channel::(10); - + let (block_and_receipt_sender, block_and_receipt_receiver) = + tokio::sync::mpsc::channel::(10); + let (snap_state_sender, snap_state_receiver) = tokio::sync::mpsc::channel::(10); + let router_handle = tokio::spawn(route_replies( + self.reply_receiver.clone(), + snap_state_sender, + block_and_receipt_sender, + )); + let fetch_blocks_and_receipts_handle = tokio::spawn(fetch_blocks_and_receipts( + all_block_hashes.clone(), + block_and_receipt_receiver, + self.peers.clone(), + )); + let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( + all_block_hashes.clone(), + snap_state_receiver, + self.peers.clone(), + )); + // Store headers + // TODO: Handle error + let err = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); + router_handle.abort(); // Sync finished } @@ -146,7 +169,7 @@ impl SyncManager { Self { snap_mode: false, peers: dummy_peer_table, - reply_receiver: tokio::sync::mpsc::channel(0).1, + reply_receiver: Arc::new(Mutex::new(tokio::sync::mpsc::channel(0).1)), } } } @@ -165,16 +188,115 @@ async fn receive_block_headers( } } -fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHeaderError> { +fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHeaderError> { // The first header is a header we have already validated (either current last block or last block in previous batch) for headers in headers.windows(2) { - validate_block_header(&headers[0], &headers[1])?; + //validate_block_header(&headers[0], &headers[1])?; } Ok(()) } -// Process an already validated block header -// In parallel: Fetches the block and its receipt -fn process_header(header: &BlockHeader, hash: &BlockHash) { +/// Routes replies from the universal receiver to the different active processes +async fn route_replies( + receiver: Arc>>, + snap_state_sender: tokio::sync::mpsc::Sender, + block_and_receipt_sender: tokio::sync::mpsc::Sender, +) -> Option { + let mut receiver = receiver.lock().await; + loop { + match receiver.recv().await { + Some(message @ Message::BlockBodies(_) | message @ Message::Receipts(_)) => { + // TODO: Kill process and restart + let _ = block_and_receipt_sender.send(message).await; + } + Some( + message @ Message::AccountRange(_) + | message @ Message::StorageRanges(_) + | message @ Message::ByteCodes(_), + ) => { + // TODO: Kill process and restart + let _ = snap_state_sender.send(message).await; + } + _ => continue, + } + } +} + +async fn fetch_blocks_and_receipts( + block_hashes: Vec, + mut reply_receiver: tokio::sync::mpsc::Receiver, + peers: Arc>, +) { + // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts + // Fetch Block Bodies + let mut block_bodies_request = GetBlockBodies { + id: 34, + block_hashes, + }; + loop { + // TODO: Randomize id + block_bodies_request.id += 1; + info!("[Sync] Sending request {block_bodies_request:?}"); + // Send a GetBlockHeaders request to a peer + if peers + .lock() + .await + .send_message_to_peer(Message::GetBlockBodies(block_bodies_request.clone())) + .await + .is_err() + { + info!("[Sync] No peers available, retrying in 10 sec"); + // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + continue; + }; + // Wait for the peer to reply + match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { + Ok(Some(Message::BlockBodies(message))) + if message.id == block_bodies_request.id && !message.block_bodies.is_empty() => + { + info!( + "[SYNC] Received {} Block Bodies", + message.block_bodies.len() + ); + // Track which bodies we have already fetched + block_bodies_request.block_hashes = block_bodies_request.block_hashes + [block_bodies_request + .block_hashes + .len() + .min(message.block_bodies.len())..] + .to_vec(); + // Store Block Bodies + // Check if we need to ask for another batch + if block_bodies_request.block_hashes.is_empty() { + break; + } + } + // Bad peer response, lets try a different peer + Ok(Some(_)) => info!("[Sync] Bad peer response"), + // Reply timeouted/peer shut down, lets try a different peer + _ => info!("[Sync] Peer response timeout"), + } + } +} + +async fn fetch_snap_state( + block_hashes: Vec, + reply_receiver: tokio::sync::mpsc::Receiver, + peers: Arc>, +) { +} +async fn receive_block_bodies( + channel: &mut tokio::sync::mpsc::Receiver, + id: u64, +) -> Option { + loop { + match channel.recv().await { + Some(Message::BlockBodies(response)) if response.id == id => return Some(response), + // Ignore replies that don't match the expected id (such as late responses) + Some(_other_response) => continue, + None => return None, + } + } } From fc9cc3ea9004e66d6a4c6d17acb2dad8d65fac12 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 15:24:08 -0300 Subject: [PATCH 220/345] Store block bodies and headers --- cmd/ethrex/ethrex.rs | 21 ++++-- crates/networking/p2p/net.rs | 2 +- crates/networking/p2p/rlpx/connection.rs | 32 +++++---- crates/networking/p2p/sync.rs | 32 ++++++--- crates/networking/rpc/engine/fork_choice.rs | 73 +++++++++++---------- crates/networking/rpc/eth/filter.rs | 2 +- crates/networking/rpc/utils.rs | 2 +- crates/storage/trie/trie.rs | 2 +- 8 files changed, 99 insertions(+), 67 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 9bb89403b..e6da7aa36 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -1,13 +1,26 @@ use bytes::Bytes; use directories::ProjectDirs; use ethrex_blockchain::{add_block, fork_choice::apply_fork_choice}; -use ethrex_core::{types::{Block, Genesis}, H256}; -use ethrex_net::{bootnode::BootNode, node_id_from_signing_key, peer_table, sync::SyncManager, types::Node}; -use ethrex_storage::{EngineType, Store}; +use ethrex_core::{ + types::{Block, Genesis}, + H256, +}; +use ethrex_net::{ + bootnode::BootNode, node_id_from_signing_key, peer_table, sync::SyncManager, types::Node, +}; use ethrex_rlp::decode::RLPDecode; +use ethrex_storage::{EngineType, Store}; use k256::ecdsa::SigningKey; use local_ip_address::local_ip; -use std::{fs::{self, File}, future::IntoFuture, io, net::{Ipv4Addr, SocketAddr, ToSocketAddrs}, path::Path, str::FromStr as _, time::Duration}; +use std::{ + fs::{self, File}, + future::IntoFuture, + io, + net::{Ipv4Addr, SocketAddr, ToSocketAddrs}, + path::Path, + str::FromStr as _, + time::Duration, +}; use tokio_util::task::TaskTracker; use tracing::{error, info, warn}; use tracing_subscriber::{filter::Directive, EnvFilter, FmtSubscriber}; diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index f9743c76a..88df97fcd 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -19,8 +19,8 @@ use k256::{ pub use kademlia::KademliaTable; use kademlia::{bucket_number, MAX_NODES_PER_BUCKET}; use rand::rngs::OsRng; -pub use rlpx::message::Message as RLPxMessage; use rlpx::connection::RLPxConnection; +pub use rlpx::message::Message as RLPxMessage; use tokio::{ net::{TcpSocket, TcpStream, UdpSocket}, sync::{broadcast, mpsc, Mutex}, diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index d6b8e6f89..0616c01b2 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -40,8 +40,7 @@ use tokio::{ io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt}, sync::{ broadcast::{self, error::RecvError}, - mpsc, - Mutex, + mpsc, Mutex, }, task, time::{sleep, Instant}, @@ -145,24 +144,29 @@ impl RLPxConnection { /// Starts a handshake and runs the peer connection. /// It runs in it's own task and blocks until the connection is dropped - pub async fn start_peer(&mut self, table: Arc>, backend_send: mpsc::Sender,) { + pub async fn start_peer( + &mut self, + table: Arc>, + backend_send: mpsc::Sender, + ) { // Perform handshake if let Err(e) = self.handshake().await { self.peer_conn_failed("Handshake failed", e, table).await; } else { // Handshake OK: handle connection // Create channel to communicate directly to the peer - let (sender, backend_receive) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - let Ok(node_id) = self.get_remote_node_id() else { - return self.peer_conn_failed( - "Error during RLPx connection", - RLPxError::InvalidState(), - table, - ) - .await; - }; - table.lock().await.set_sender(node_id, sender); + let (sender, backend_receive) = + tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let Ok(node_id) = self.get_remote_node_id() else { + return self + .peer_conn_failed( + "Error during RLPx connection", + RLPxError::InvalidState(), + table, + ) + .await; + }; + table.lock().await.set_sender(node_id, sender); if let Err(e) = self.handle_peer_conn(backend_send, backend_receive).await { self.peer_conn_failed("Error during RLPx connection", e, table) .await; diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index cfcddd617..dde124c77 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -45,7 +45,7 @@ impl SyncManager { } } // TODO: only uses snap sync, should also process full sync once implemented - pub async fn start_sync(&mut self, current_head: H256, sync_head: H256) { + pub async fn start_sync(&mut self, current_head: H256, sync_head: H256, store: Store) { const BYTES_PER_REQUEST: u64 = 500; // TODO: Adjust info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); // Request all block headers between the current head and the sync head @@ -150,6 +150,7 @@ impl SyncManager { all_block_hashes.clone(), block_and_receipt_receiver, self.peers.clone(), + store.clone(), )); let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( all_block_hashes.clone(), @@ -157,8 +158,12 @@ impl SyncManager { self.peers.clone(), )); // Store headers - for (hash, header) in all_block_headers.into_iter().zip(all_block_hashes.into_iter()) { - + for (header, hash) in all_block_headers + .into_iter() + .zip(all_block_hashes.into_iter()) + { + // TODO: Handle error + store.add_block_header(hash, header).unwrap(); } // TODO: Handle error let err = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); @@ -230,6 +235,7 @@ async fn fetch_blocks_and_receipts( block_hashes: Vec, mut reply_receiver: tokio::sync::mpsc::Receiver, peers: Arc>, + store: Store, ) { // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts // Fetch Block Bodies @@ -264,17 +270,23 @@ async fn fetch_blocks_and_receipts( message.block_bodies.len() ); // Track which bodies we have already fetched - block_bodies_request.block_hashes = block_bodies_request.block_hashes - [block_bodies_request - .block_hashes - .len() - .min(message.block_bodies.len())..] - .to_vec(); + let (fetched_hashes, remaining_hahses) = block_bodies_request + .block_hashes + .split_at(message.block_bodies.len()); // Store Block Bodies + for (hash, body) in fetched_hashes + .into_iter() + .zip(message.block_bodies.into_iter()) + { + // TODO: handle error + store.add_block_body(hash.clone(), body).unwrap() + } + // Check if we need to ask for another batch - if block_bodies_request.block_hashes.is_empty() { + if remaining_hahses.is_empty() { break; } + block_bodies_request.block_hashes = remaining_hahses.to_vec(); } // Bad peer response, lets try a different peer Ok(Some(_)) => info!("[Sync] Bad peer response"), diff --git a/crates/networking/rpc/engine/fork_choice.rs b/crates/networking/rpc/engine/fork_choice.rs index 75baf36e3..8ef1e5aa4 100644 --- a/crates/networking/rpc/engine/fork_choice.rs +++ b/crates/networking/rpc/engine/fork_choice.rs @@ -58,53 +58,56 @@ impl RpcHandler for ForkChoiceUpdatedV3 { } fn handle(&self, context: RpcApiContext) -> Result { - let storage = &context.storage; info!( "New fork choice request with head: {}, safe: {}, finalized: {}.", self.fork_choice_state.head_block_hash, self.fork_choice_state.safe_block_hash, self.fork_choice_state.finalized_block_hash ); - let fork_choice_error_to_response = |error| { - let response = match error { - InvalidForkChoice::NewHeadAlreadyCanonical => ForkChoiceResponse::from( - PayloadStatus::valid_with_hash(latest_canonical_block_hash(storage).unwrap()), - ), - InvalidForkChoice::Syncing => ForkChoiceResponse::from(PayloadStatus::syncing()), - reason => { - warn!("Invalid fork choice state. Reason: {:#?}", reason); - return Err(RpcErr::InvalidForkChoiceState(reason.to_string())); - } - }; - - serde_json::to_value(response).map_err(|error| RpcErr::Internal(error.to_string())) - }; let head_block = match apply_fork_choice( - storage, + &context.storage, self.fork_choice_state.head_block_hash, self.fork_choice_state.safe_block_hash, self.fork_choice_state.finalized_block_hash, ) { Ok(head) => head, - // TODO: handle this more elegantly - Err(error @ InvalidForkChoice::Syncing) => { - // Start sync - let current_number = context.storage.get_latest_block_number()?.unwrap(); - let current_head = context - .storage - .get_canonical_block_hash(current_number)? - .unwrap(); - let sync_head = self.fork_choice_state.head_block_hash; - tokio::spawn(async move { - // If we can't get hold of the syncer, then it means that there is an active sync in process - if let Ok(mut syncer) = context.syncer.try_lock() { - syncer.start_sync(current_head, sync_head).await + Err(error) => { + let fork_choice_response = match error { + InvalidForkChoice::NewHeadAlreadyCanonical => { + ForkChoiceResponse::from(PayloadStatus::valid_with_hash( + latest_canonical_block_hash(&context.storage).unwrap(), + )) } - }); - return fork_choice_error_to_response(error); + InvalidForkChoice::Syncing => { + // Start sync + let current_number = context.storage.get_latest_block_number()?.unwrap(); + let Some(current_head) = + context.storage.get_canonical_block_hash(current_number)? + else { + return Err(RpcErr::Internal( + "Missing latest canonical block".to_owned(), + )); + }; + let sync_head = self.fork_choice_state.head_block_hash; + tokio::spawn(async move { + // If we can't get hold of the syncer, then it means that there is an active sync in process + if let Ok(mut syncer) = context.syncer.try_lock() { + syncer + .start_sync(current_head, sync_head, context.storage.clone()) + .await + } + }); + ForkChoiceResponse::from(PayloadStatus::syncing()) + } + reason => { + warn!("Invalid fork choice state. Reason: {:#?}", reason); + return Err(RpcErr::InvalidForkChoiceState(reason.to_string())); + } + }; + return serde_json::to_value(fork_choice_response) + .map_err(|error| RpcErr::Internal(error.to_string())); } - Err(error) => return fork_choice_error_to_response(error), }; // Build block from received payload. This step is skipped if applying the fork choice state failed @@ -118,7 +121,7 @@ impl RpcHandler for ForkChoiceUpdatedV3 { Ok(None) => (), Ok(Some(attributes)) => { info!("Fork choice updated includes payload attributes. Creating a new payload."); - let chain_config = storage.get_chain_config()?; + let chain_config = context.storage.get_chain_config()?; if !chain_config.is_cancun_activated(attributes.timestamp) { return Err(RpcErr::UnsuportedFork( "forkChoiceV3 used to build pre-Cancun payload".to_string(), @@ -140,14 +143,14 @@ impl RpcHandler for ForkChoiceUpdatedV3 { }; let payload_id = args.id(); response.set_id(payload_id); - let payload = match create_payload(&args, storage) { + let payload = match create_payload(&args, &context.storage) { Ok(payload) => payload, Err(ChainError::EvmError(error)) => return Err(error.into()), // Parent block is guaranteed to be present at this point, // so the only errors that may be returned are internal storage errors Err(error) => return Err(RpcErr::Internal(error.to_string())), }; - storage.add_payload(payload_id, payload)?; + context.storage.add_payload(payload_id, payload)?; } } diff --git a/crates/networking/rpc/eth/filter.rs b/crates/networking/rpc/eth/filter.rs index 531feec68..197dbc02b 100644 --- a/crates/networking/rpc/eth/filter.rs +++ b/crates/networking/rpc/eth/filter.rs @@ -274,8 +274,8 @@ mod tests { utils::{test_utils::example_p2p_node, RpcRequest}, }; use ethrex_core::types::Genesis; - use ethrex_storage::{EngineType, Store}; use ethrex_net::sync::SyncManager; + use ethrex_storage::{EngineType, Store}; use serde_json::{json, Value}; use test_utils::TEST_GENESIS; diff --git a/crates/networking/rpc/utils.rs b/crates/networking/rpc/utils.rs index 42efcd5a8..23e489b7f 100644 --- a/crates/networking/rpc/utils.rs +++ b/crates/networking/rpc/utils.rs @@ -247,7 +247,7 @@ pub mod test_utils { use std::{net::SocketAddr, str::FromStr}; use ethrex_core::H512; - use ethrex_net::{types::Node, sync::SyncManager}; + use ethrex_net::{sync::SyncManager, types::Node}; use ethrex_storage::{EngineType, Store}; use crate::start_api; diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 021c61d37..2177e7399 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -9,13 +9,13 @@ mod state; mod test_utils; mod trie_iter; mod verify_range; -use std::collections::HashSet; use ethereum_types::H256; use ethrex_rlp::constants::RLP_NULL; use nibbles::Nibbles; use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; +use std::collections::HashSet; #[cfg(feature = "libmdbx")] pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; From 6b33d11bca496a33a521f76793977da93814c634 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 15:36:23 -0300 Subject: [PATCH 221/345] Set latest block --- crates/networking/p2p/sync.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index dde124c77..f3f41e289 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -163,6 +163,8 @@ impl SyncManager { .zip(all_block_hashes.into_iter()) { // TODO: Handle error + store.set_canonical_block(header.number, hash).unwrap(); + store.update_latest_block_number(header.number).unwrap(); store.add_block_header(hash, header).unwrap(); } // TODO: Handle error From e6c2a0838c4da338cb4c8393ae043c75baed806b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 16:09:39 -0300 Subject: [PATCH 222/345] Start snap fetching --- crates/networking/p2p/sync.rs | 61 ++++++++++++++++++++++++++++++++--- 1 file changed, 57 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index f3f41e289..14671d6e4 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,4 +1,4 @@ -use std::{sync::Arc, time::Duration}; +use std::{str::FromStr, sync::Arc, time::Duration}; use ethrex_core::{ types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, @@ -13,6 +13,7 @@ use crate::{ rlpx::{ eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}, message::Message, + snap::GetAccountRange, }, }; const REPLY_TIMEOUT: Duration = Duration::from_secs(30); @@ -152,10 +153,15 @@ impl SyncManager { self.peers.clone(), store.clone(), )); + let state_roots = all_block_headers + .iter() + .map(|header| header.state_root) + .collect::>(); let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( - all_block_hashes.clone(), + state_roots.clone(), snap_state_receiver, self.peers.clone(), + store.clone(), )); // Store headers for (header, hash) in all_block_headers @@ -202,6 +208,7 @@ async fn receive_block_headers( fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHeaderError> { // The first header is a header we have already validated (either current last block or last block in previous batch) for headers in headers.windows(2) { + // TODO: Validation commented to make this work with older blocks //validate_block_header(&headers[0], &headers[1])?; } Ok(()) @@ -295,14 +302,60 @@ async fn fetch_blocks_and_receipts( // Reply timeouted/peer shut down, lets try a different peer _ => info!("[Sync] Peer response timeout( Blocks & Receipts)"), } + // TODO: Fetch Receipts and store them } } async fn fetch_snap_state( - block_hashes: Vec, - reply_receiver: tokio::sync::mpsc::Receiver, + state_roots: Vec, + mut reply_receiver: tokio::sync::mpsc::Receiver, peers: Arc>, + store: Store, ) { + for root_hash in state_roots { + // Fetch Account Ranges + let mut account_ranges_request = GetAccountRange { + id: 7, + root_hash, + starting_hash: H256::zero(), + limit_hash: H256::from_str( + "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + ) + .unwrap(), + response_bytes: 500, + }; + loop { + // TODO: Randomize id + account_ranges_request.id += 1; + info!("[Sync] Sending Block headers request "); + // Send a GetBlockBodies request to a peer + if peers + .lock() + .await + .send_message_to_peer(Message::GetAccountRange(account_ranges_request.clone())) + .await + .is_err() + { + info!("[Sync] No peers available, retrying in 10 sec"); + // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + continue; + }; + // Wait for the peer to reply + match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { + Ok(Some(Message::AccountRange(message))) + if message.id == account_ranges_request.id && !message.accounts.is_empty() => + { + info!("[SYNC] Received {} Accounts", message.accounts.len()); + } + + // Bad peer response, lets try a different peer + Ok(Some(_)) => info!("[Sync] Bad peer response"), + // Reply timeouted/peer shut down, lets try a different peer + _ => info!("[Sync] Peer response timeout( Snap Account Range)"), + } + } + } } async fn receive_block_bodies( From 8f7cbfe123e175838a904f034e85dabe77362399 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 26 Nov 2024 16:16:17 -0300 Subject: [PATCH 223/345] chore: p2p msg boilerplate --- crates/networking/p2p/rlpx/message.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index d44f1b4ca..412e55a33 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -3,6 +3,7 @@ use ethrex_rlp::error::{RLPDecodeError, RLPEncodeError}; use std::fmt::Display; use super::eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}; +use super::eth::receipts::GetReceipts; use super::eth::status::StatusMessage; use super::eth::transactions::Transactions; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; @@ -31,6 +32,7 @@ pub(crate) enum Message { Transactions(Transactions), GetBlockBodies(GetBlockBodies), BlockBodies(BlockBodies), + GetReceipts(GetReceipts), // snap capability GetAccountRange(GetAccountRange), AccountRange(AccountRange), @@ -64,6 +66,7 @@ impl Message { 0x13 => Ok(Message::GetBlockHeaders(GetBlockHeaders::decode(msg_data)?)), 0x14 => Ok(Message::BlockHeaders(BlockHeaders::decode(msg_data)?)), 0x15 => Ok(Message::GetBlockBodies(GetBlockBodies::decode(msg_data)?)), + 0x1F => Ok(Message::GetReceipts(GetReceipts::decode(msg_data)?)), 0x21 => Ok(Message::GetAccountRange(GetAccountRange::decode(msg_data)?)), 0x22 => Ok(Message::AccountRange(AccountRange::decode(msg_data)?)), 0x23 => Ok(Message::GetStorageRanges(GetStorageRanges::decode( @@ -120,6 +123,10 @@ impl Message { 0x16_u8.encode(buf); msg.encode(buf) } + Message::GetReceipts(msg) => { + 0x1F_u8.encode(buf); + msg.encode(buf) + } Message::GetAccountRange(msg) => { 0x21_u8.encode(buf); msg.encode(buf) @@ -167,6 +174,7 @@ impl Display for Message { Message::GetBlockHeaders(_) => "eth:getBlockHeaders".fmt(f), Message::BlockHeaders(_) => "eth:BlockHeaders".fmt(f), Message::BlockBodies(_) => "eth:BlockBodies".fmt(f), + Message::GetReceipts(_) => "eth:GetReceipts".fmt(f), Message::Transactions(_) => "eth:TransactionsMessage".fmt(f), Message::GetBlockBodies(_) => "eth:GetBlockBodies".fmt(f), Message::GetAccountRange(_) => "snap:GetAccountRange".fmt(f), From da6cff86a058b01502f5495743e21f10ae6659be Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 18:19:34 -0300 Subject: [PATCH 224/345] Refactor comunication between backend and peer --- cmd/ethrex/ethrex.rs | 6 +- crates/networking/p2p/kademlia.rs | 55 +++- crates/networking/p2p/net.rs | 48 +--- crates/networking/p2p/rlpx/connection.rs | 26 +- crates/networking/p2p/sync.rs | 338 +++++++---------------- 5 files changed, 167 insertions(+), 306 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index e6da7aa36..f45d9fba8 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -176,11 +176,8 @@ async fn main() { }; // Create Kademlia Table here so we can access it from rpc server (for syncing) let peer_table = peer_table(signer.clone()); - // Communication between the backend and the main listen loop - let (channel_backend_send, channel_p2p_receive) = - tokio::sync::mpsc::channel::(100); // Create SyncManager - let syncer = SyncManager::new(channel_p2p_receive, peer_table.clone(), is_snap_sync); + let syncer = SyncManager::new(peer_table.clone(), is_snap_sync); // TODO: Check every module starts properly. let tracker = TaskTracker::new(); @@ -226,7 +223,6 @@ async fn main() { signer, peer_table, store, - channel_backend_send ) .into_future(); tracker.spawn(networking); diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index c14dd942f..771601222 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -1,11 +1,17 @@ +use std::sync::Arc; + use crate::{ discv4::{time_now_unix, FindNodeRequest}, rlpx::{error::RLPxError, message::Message}, types::Node, + MAX_MESSAGES_TO_BROADCAST, }; use ethrex_core::{H256, H512, U256}; use sha3::{Digest, Keccak256}; -use tokio::sync::mpsc::UnboundedSender; +use tokio::sync::{ + mpsc::{self, UnboundedSender}, + Mutex, +}; pub const MAX_NODES_PER_BUCKET: usize = 16; const NUMBER_OF_BUCKETS: usize = 256; @@ -258,30 +264,25 @@ impl KademliaTable { /// Set the sender end of the channel between the kademlia table and the peer's active connection /// This function should be called each time a connection is established so the backend can send requests to the peers - pub fn set_sender(&mut self, node_id: H512, sender: tokio::sync::mpsc::Sender) { + pub fn set_channels(&mut self, node_id: H512, channels: PeerChannels) { let bucket_idx = bucket_number(self.local_node_id, node_id); if let Some(peer) = self.buckets[bucket_idx] .peers .iter_mut() .find(|peer| peer.node.node_id == node_id) { - peer.sender = Some(sender) + peer.channels = Some(channels) } } /// TODO: Use most recently pinged peer instead - fn get_peer(&self) -> Option { + pub(crate) fn get_peer(&self) -> Option { self.get_least_recently_pinged_peers(1).pop() } - /// Send a message to a peer, returns an error if there are no active peers - pub async fn send_message_to_peer(&self, message: Message) -> Result<(), RLPxError> { - if let Some(sender) = self.get_peer().and_then(|peer| peer.sender) { - Ok(sender.send(message).await?) - } else { - // Let the caller decide when to retry - Err(RLPxError::NoPeers) - } + // Returns true if the table has at least one peer + pub fn has_peers(&self) -> bool { + self.get_peer().is_some() } } @@ -308,8 +309,32 @@ pub struct PeerData { pub liveness: u16, /// if a revalidation was sent to the peer, the bool marks if it has answered pub revalidation: Option, - /// sender end of the channel between the peer and its rlpx connection - pub sender: Option>, + /// communication channels bewteen the peer data and its active connection + pub channels: Option, +} + +#[derive(Debug, Clone)] +/// Holds the respective sender and receiver ends of the communication channels bewteen the peer data and its active connection +pub struct PeerChannels { + pub sender: mpsc::Sender, + pub receiver: Arc>>, +} + +impl PeerChannels { + /// Sets up the communication channels for the peer + /// Returns the channel endpoints to send to the active connection's listen loop + pub fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + let (connection_sender, receiver) = mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + ( + Self { + sender, + receiver: Arc::new(Mutex::new(receiver)), + }, + connection_sender, + connection_receiver, + ) + } } impl PeerData { @@ -323,7 +348,7 @@ impl PeerData { last_ping_hash: None, find_node_request: None, revalidation: None, - sender: None, + channels: None, } } diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index 88df97fcd..97106623e 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -23,7 +23,7 @@ use rlpx::connection::RLPxConnection; pub use rlpx::message::Message as RLPxMessage; use tokio::{ net::{TcpSocket, TcpStream, UdpSocket}, - sync::{broadcast, mpsc, Mutex}, + sync::{broadcast, Mutex}, try_join, }; use tracing::{debug, error, info}; @@ -32,6 +32,7 @@ use types::{Endpoint, Node}; pub mod bootnode; pub(crate) mod discv4; pub(crate) mod kademlia; +pub mod peer_request; pub mod rlpx; pub(crate) mod snap; pub mod sync; @@ -57,7 +58,6 @@ pub async fn start_network( signer: SigningKey, peer_table: Arc>, storage: Store, - channel_backend_send: tokio::sync::mpsc::Sender, ) { info!("Starting discovery service at {udp_addr}"); info!("Listening for requests at {tcp_addr}"); @@ -72,7 +72,6 @@ pub async fn start_network( peer_table.clone(), bootnodes, channel_broadcast_send_end.clone(), - channel_backend_send.clone(), )); let server_handle = tokio::spawn(serve_requests( tcp_addr, @@ -80,7 +79,6 @@ pub async fn start_network( storage.clone(), peer_table.clone(), channel_broadcast_send_end, - channel_backend_send, )); try_join!(discovery_handle, server_handle).unwrap(); @@ -93,7 +91,6 @@ async fn discover_peers( table: Arc>, bootnodes: Vec, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, - backend_send: mpsc::Sender, ) { let udp_socket = Arc::new(UdpSocket::bind(udp_addr).await.unwrap()); @@ -104,7 +101,6 @@ async fn discover_peers( table.clone(), signer.clone(), connection_broadcast, - backend_send, )); let revalidation_handler = tokio::spawn(peers_revalidation( udp_addr, @@ -144,7 +140,6 @@ async fn discover_peers_server( table: Arc>, signer: SigningKey, tx_broadcaster_send: broadcast::Sender<(tokio::task::Id, Arc)>, - backend_send: mpsc::Sender, ) { let mut buf = vec![0; MAX_DISC_PACKET_SIZE]; @@ -231,7 +226,6 @@ async fn discover_peers_server( let signer = signer.clone(); let storage = storage.clone(); let broadcaster = tx_broadcaster_send.clone(); - let backend_send = backend_send.clone(); tokio::spawn(async move { handle_peer_as_initiator( signer, @@ -240,7 +234,6 @@ async fn discover_peers_server( storage, table, broadcaster, - backend_send, ) .await; }); @@ -773,7 +766,6 @@ async fn serve_requests( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, - backend_send: mpsc::Sender, ) { let tcp_socket = TcpSocket::new_v4().unwrap(); tcp_socket.bind(tcp_addr).unwrap(); @@ -787,7 +779,6 @@ async fn serve_requests( storage.clone(), table.clone(), connection_broadcast.clone(), - backend_send.clone(), )); } } @@ -798,10 +789,9 @@ async fn handle_peer_as_receiver( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, - backend_send: mpsc::Sender, ) { let mut conn = RLPxConnection::receiver(signer, stream, storage, connection_broadcast); - conn.start_peer(table, backend_send).await; + conn.start_peer(table).await; } async fn handle_peer_as_initiator( @@ -811,7 +801,6 @@ async fn handle_peer_as_initiator( storage: Store, table: Arc>, connection_broadcast: broadcast::Sender<(tokio::task::Id, Arc)>, - backend_send: mpsc::Sender, ) { info!("Trying RLPx connection with {node:?}"); let stream = TcpSocket::new_v4() @@ -820,7 +809,7 @@ async fn handle_peer_as_initiator( .await .unwrap(); match RLPxConnection::initiator(signer, msg, stream, storage, connection_broadcast).await { - Ok(mut conn) => conn.start_peer(table, backend_send).await, + Ok(mut conn) => conn.start_peer(table).await, Err(e) => { error!("Error: {e}, Could not start connection with {node:?}"); } @@ -878,11 +867,7 @@ mod tests { pub udp_socket: Arc, } - async fn start_mock_discovery_server( - udp_port: u16, - should_start_server: bool, - backend_send: mpsc::Sender, - ) -> MockServer { + async fn start_mock_discovery_server(udp_port: u16, should_start_server: bool) -> MockServer { let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), udp_port); let signer = SigningKey::random(&mut OsRng); let udp_socket = Arc::new(UdpSocket::bind(addr).await.unwrap()); @@ -902,7 +887,6 @@ mod tests { table.clone(), signer.clone(), channel_broadcast_send_end, - backend_send, )); } @@ -949,10 +933,8 @@ mod tests { * To make this run faster, we'll change the revalidation time to be every 2secs */ async fn discovery_server_revalidation() { - let (backend_send, _) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - let mut server_a = start_mock_discovery_server(7998, true, backend_send.clone()).await; - let mut server_b = start_mock_discovery_server(7999, true, backend_send).await; + let mut server_a = start_mock_discovery_server(7998, true).await; + let mut server_b = start_mock_discovery_server(7999, true).await; connect_servers(&mut server_a, &mut server_b).await; @@ -1014,10 +996,8 @@ mod tests { * This test for only one lookup, and not recursively. */ async fn discovery_server_lookup() { - let (backend_send, _) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - let mut server_a = start_mock_discovery_server(8000, true, backend_send.clone()).await; - let mut server_b = start_mock_discovery_server(8001, true, backend_send).await; + let mut server_a = start_mock_discovery_server(8000, true).await; + let mut server_b = start_mock_discovery_server(8001, true).await; fill_table_with_random_nodes(server_a.table.clone()).await; @@ -1074,12 +1054,10 @@ mod tests { * - We'll run a recursive lookup on server `a` and we expect to end with `b`, `c`, `d` and its mock nodes */ async fn discovery_server_recursive_lookup() { - let (backend_send, _) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - let mut server_a = start_mock_discovery_server(8002, true, backend_send.clone()).await; - let mut server_b = start_mock_discovery_server(8003, true, backend_send.clone()).await; - let mut server_c = start_mock_discovery_server(8004, true, backend_send.clone()).await; - let mut server_d = start_mock_discovery_server(8005, true, backend_send).await; + let mut server_a = start_mock_discovery_server(8002, true).await; + let mut server_b = start_mock_discovery_server(8003, true).await; + let mut server_c = start_mock_discovery_server(8004, true).await; + let mut server_d = start_mock_discovery_server(8005, true).await; connect_servers(&mut server_a, &mut server_b).await; connect_servers(&mut server_b, &mut server_c).await; diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 0616c01b2..86ed574f8 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,6 +1,7 @@ use std::sync::Arc; use crate::{ + kademlia::PeerChannels, rlpx::{ eth::{ backend, @@ -16,7 +17,7 @@ use crate::{ process_account_range_request, process_byte_codes_request, process_storage_ranges_request, process_trie_nodes_request, }, - MAX_DISC_PACKET_SIZE, MAX_MESSAGES_TO_BROADCAST, + MAX_DISC_PACKET_SIZE, }; use super::{ @@ -144,19 +145,14 @@ impl RLPxConnection { /// Starts a handshake and runs the peer connection. /// It runs in it's own task and blocks until the connection is dropped - pub async fn start_peer( - &mut self, - table: Arc>, - backend_send: mpsc::Sender, - ) { + pub async fn start_peer(&mut self, table: Arc>) { // Perform handshake if let Err(e) = self.handshake().await { self.peer_conn_failed("Handshake failed", e, table).await; } else { // Handshake OK: handle connection - // Create channel to communicate directly to the peer - let (sender, backend_receive) = - tokio::sync::mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); + // Create channels to communicate directly to the peer + let (peer_channels, sender, receiver) = PeerChannels::create(); let Ok(node_id) = self.get_remote_node_id() else { return self .peer_conn_failed( @@ -166,8 +162,8 @@ impl RLPxConnection { ) .await; }; - table.lock().await.set_sender(node_id, sender); - if let Err(e) = self.handle_peer_conn(backend_send, backend_receive).await { + table.lock().await.set_channels(node_id, peer_channels); + if let Err(e) = self.handle_peer_conn(sender, receiver).await { self.peer_conn_failed("Error during RLPx connection", e, table) .await; } @@ -256,8 +252,8 @@ impl RLPxConnection { pub async fn handle_peer_conn( &mut self, - backend_send: mpsc::Sender, - mut backend_receive: mpsc::Receiver, + sender: mpsc::Sender, + mut receiver: mpsc::Receiver, ) -> Result<(), RLPxError> { if let RLPxConnectionState::Established(_) = &self.state { self.init_peer_conn().await?; @@ -277,7 +273,7 @@ impl RLPxConnection { tokio::select! { // TODO check if this is cancel safe, and fix it if not. message = self.receive() => { - self.handle_message(message?, backend_send.clone()).await?; + self.handle_message(message?, sender.clone()).await?; } // This is not ideal, but using the receiver without // this function call, causes the loop to take ownwership @@ -290,7 +286,7 @@ impl RLPxConnection { Some(broadcasted_msg) = Self::maybe_wait_for_broadcaster(&mut broadcaster_receive) => { self.handle_broadcast(broadcasted_msg?).await? } - Some(message) = backend_receive.recv() => { + Some(message) = receiver.recv() => { self.send(message).await?; } _ = sleep(PERIODIC_TASKS_CHECK_INTERVAL) => { diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 14671d6e4..86587f852 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,4 +1,4 @@ -use std::{str::FromStr, sync::Arc, time::Duration}; +use std::sync::Arc; use ethrex_core::{ types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, @@ -6,106 +6,66 @@ use ethrex_core::{ }; use ethrex_storage::Store; use tokio::sync::Mutex; -use tracing::{debug, info}; +use tracing::info; -use crate::{ - kademlia::KademliaTable, - rlpx::{ - eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}, - message::Message, - snap::GetAccountRange, - }, -}; -const REPLY_TIMEOUT: Duration = Duration::from_secs(30); +use crate::kademlia::KademliaTable; /// Manager in charge of the snap-sync(for now, will also handle full sync) process /// TaskList: -/// A) Fetch latest block headers (should we ask what the latest block is first?) -/// B) Validate block headers -/// C) Fetch full Blocks and Receipts || Download Raw State (accounts, storages, bytecodes) -/// D) Healing +/// - Fetch latest block headers (should we ask what the latest block is first?) +/// - Validate block headers +/// - Fetch full Blocks and Receipts || Download Raw State (accounts, storages, bytecodes) +/// - Healing #[derive(Debug)] pub struct SyncManager { // true: syncmode = snap, false = syncmode = full snap_mode: bool, peers: Arc>, - // Receiver end of the channel between the manager and the main p2p listen loop - reply_receiver: Arc>>, } impl SyncManager { - pub fn new( - reply_receiver: tokio::sync::mpsc::Receiver, - peers: Arc>, - snap_mode: bool, - ) -> Self { - Self { - snap_mode, - peers, - reply_receiver: Arc::new(Mutex::new(reply_receiver)), - } + pub fn new(peers: Arc>, snap_mode: bool) -> Self { + Self { snap_mode, peers } } // TODO: only uses snap sync, should also process full sync once implemented - pub async fn start_sync(&mut self, current_head: H256, sync_head: H256, store: Store) { - const BYTES_PER_REQUEST: u64 = 500; // TODO: Adjust + pub async fn start_sync(&mut self, mut current_head: H256, sync_head: H256, store: Store) { info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); // Request all block headers between the current head and the sync head // We will begin from the current head so that we download the earliest state first // This step is not parallelized // Ask for block headers - let mut block_headers_request = GetBlockHeaders { - id: 17, // TODO: randomize - skip: 0, - startblock: current_head.into(), - limit: BYTES_PER_REQUEST, - reverse: false, - }; let mut all_block_headers = vec![]; let mut all_block_hashes = vec![]; + // Make sure we have active peers before we start making requests loop { - // TODO: Randomize id - block_headers_request.id += 1; - info!("[Sync] Sending request {block_headers_request:?}"); - // Send a GetBlockHeaders request to a peer - if self + if self.peers.lock().await.has_peers() { + break; + } + info!("[Sync] No peers available, retrying in 10 sec"); + // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + } + loop { + info!("[Sync] Requesting Block Headers from {current_head}"); + // Request Block Headers from Peer + if let Some(block_headers) = self .peers .lock() .await - .send_message_to_peer(Message::GetBlockHeaders(block_headers_request.clone())) + .request_block_headers(current_head) .await - .is_err() - { - info!("[Sync] No peers available, retrying in 10 sec"); - // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again - tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - continue; - }; - // Wait for the peer to reply - if let Ok(Some(message)) = tokio::time::timeout( - REPLY_TIMEOUT, - receive_block_headers( - &mut *self.reply_receiver.lock().await, - block_headers_request.id, - ), - ) - .await { - // We received the correct message, we can now - // A) Validate the batch of headers received and start downloading their state - // B) Check if we need to download another batch (aka we don't have the sync_head yet) + // We received the correct message, we can now: + // - Validate the batch of headers received and start downloading their state (Future Iteration) + // - Check if we need to download another batch (aka we don't have the sync_head yet) - // If the response is empty, lets ask another peer - if message.block_headers.is_empty() { - info!("[Sync] Bad peer response"); - continue; - } // Validate header batch - if validate_header_batch(&message.block_headers).is_err() { + if validate_header_batch(&block_headers).is_err() { info!("[Sync] Invalid header in batch"); continue; } // Discard the first header as we already have it - let headers = &message.block_headers[1..]; + let headers = &block_headers[1..]; let block_hashes = headers .iter() .map(|header| header.compute_block_hash()) @@ -124,32 +84,21 @@ impl SyncManager { // Check if we already reached our sync head or if we need to fetch more blocks if !block_hashes.contains(&sync_head) { // Update the request to fetch the next batch - block_headers_request.startblock = (*block_hashes.last().unwrap()).into(); + current_head = (*block_hashes.last().unwrap()).into(); } else { // No more headers to request break; } - }; + } info!("[Sync] Peer response timeout (Headers)"); - // Reply timeouted/ peer shut down, lets try a different peer } info!("[Sync] All headers fetched and validated"); // [First Iteration] We finished fetching all headers, now we can process them - // We will launch 3 tasks to: - // 1) Fetch each block's state via snap p2p requests - // 2) Fetch each blocks and its receipts via eth p2p requests - // 3) Receive replies from the receiver and send them to the two tasks - let (block_and_receipt_sender, block_and_receipt_receiver) = - tokio::sync::mpsc::channel::(10); - let (snap_state_sender, snap_state_receiver) = tokio::sync::mpsc::channel::(10); - let router_handle = tokio::spawn(route_replies( - self.reply_receiver.clone(), - snap_state_sender, - block_and_receipt_sender, - )); + // We will launch 2 tasks to: + // - Fetch each block's state via snap p2p requests + // - Fetch each blocks and its receipts via eth p2p requests let fetch_blocks_and_receipts_handle = tokio::spawn(fetch_blocks_and_receipts( all_block_hashes.clone(), - block_and_receipt_receiver, self.peers.clone(), store.clone(), )); @@ -159,23 +108,26 @@ impl SyncManager { .collect::>(); let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( state_roots.clone(), - snap_state_receiver, self.peers.clone(), store.clone(), )); // Store headers + let mut latest_block_number = 0; for (header, hash) in all_block_headers .into_iter() .zip(all_block_hashes.into_iter()) { // TODO: Handle error + latest_block_number = header.number; store.set_canonical_block(header.number, hash).unwrap(); - store.update_latest_block_number(header.number).unwrap(); store.add_block_header(hash, header).unwrap(); } // TODO: Handle error let err = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); - router_handle.abort(); + // Set latest block number here to avoid reading state that is currently being synced + store + .update_latest_block_number(latest_block_number) + .unwrap(); // Sync finished } @@ -186,21 +138,6 @@ impl SyncManager { Self { snap_mode: false, peers: dummy_peer_table, - reply_receiver: Arc::new(Mutex::new(tokio::sync::mpsc::channel(0).1)), - } - } -} - -async fn receive_block_headers( - channel: &mut tokio::sync::mpsc::Receiver, - id: u64, -) -> Option { - loop { - match channel.recv().await { - Some(Message::BlockHeaders(response)) if response.id == id => return Some(response), - // Ignore replies that don't match the expected id (such as late responses) - Some(_other_response) => continue, - None => return None, } } } @@ -214,160 +151,89 @@ fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHead Ok(()) } -/// Routes replies from the universal receiver to the different active processes -async fn route_replies( - receiver: Arc>>, - snap_state_sender: tokio::sync::mpsc::Sender, - block_and_receipt_sender: tokio::sync::mpsc::Sender, -) -> Option { - let mut receiver = receiver.lock().await; - loop { - match receiver.recv().await { - Some(message @ Message::BlockBodies(_) | message @ Message::Receipts(_)) => { - // TODO: Kill process and restart - let _ = block_and_receipt_sender.send(message).await; - } - Some( - message @ Message::AccountRange(_) - | message @ Message::StorageRanges(_) - | message @ Message::ByteCodes(_), - ) => { - // TODO: Kill process and restart - let _ = snap_state_sender.send(message).await; - } - _ => continue, - } - } -} - async fn fetch_blocks_and_receipts( - block_hashes: Vec, - mut reply_receiver: tokio::sync::mpsc::Receiver, + mut block_hashes: Vec, peers: Arc>, store: Store, ) { // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts // Fetch Block Bodies - let mut block_bodies_request = GetBlockBodies { - id: 34, - block_hashes, - }; loop { - // TODO: Randomize id - block_bodies_request.id += 1; - info!("[Sync] Sending Block headers request "); - // Send a GetBlockBodies request to a peer - if peers + info!("[Sync] Requesting Block Headers "); + if let Some(block_bodies) = peers .lock() .await - .send_message_to_peer(Message::GetBlockBodies(block_bodies_request.clone())) + .request_block_bodies(block_hashes.clone()) .await - .is_err() { - info!("[Sync] No peers available, retrying in 10 sec"); - // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again - tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - continue; - }; - // Wait for the peer to reply - match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { - Ok(Some(Message::BlockBodies(message))) - if message.id == block_bodies_request.id && !message.block_bodies.is_empty() => - { - info!( - "[SYNC] Received {} Block Bodies", - message.block_bodies.len() - ); - // Track which bodies we have already fetched - let (fetched_hashes, remaining_hahses) = block_bodies_request - .block_hashes - .split_at(message.block_bodies.len()); - // Store Block Bodies - for (hash, body) in fetched_hashes - .into_iter() - .zip(message.block_bodies.into_iter()) - { - // TODO: handle error - store.add_block_body(hash.clone(), body).unwrap() - } + info!("[SYNC] Received {} Block Bodies", block_bodies.len()); + // Track which bodies we have already fetched + let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); + // Store Block Bodies + for (hash, body) in fetched_hashes.into_iter().zip(block_bodies.into_iter()) { + // TODO: handle error + store.add_block_body(hash.clone(), body).unwrap() + } - // Check if we need to ask for another batch - if remaining_hahses.is_empty() { - break; - } - block_bodies_request.block_hashes = remaining_hahses.to_vec(); + // Check if we need to ask for another batch + if remaining_hashes.is_empty() { + break; + } else { + block_hashes = remaining_hashes.to_vec(); } - // Bad peer response, lets try a different peer - Ok(Some(_)) => info!("[Sync] Bad peer response"), - // Reply timeouted/peer shut down, lets try a different peer - _ => info!("[Sync] Peer response timeout( Blocks & Receipts)"), } - // TODO: Fetch Receipts and store them + info!("[Sync] Peer response timeout( Blocks & Receipts)"); } + // TODO: Fetch Receipts and store them } async fn fetch_snap_state( state_roots: Vec, - mut reply_receiver: tokio::sync::mpsc::Receiver, peers: Arc>, store: Store, ) { - for root_hash in state_roots { - // Fetch Account Ranges - let mut account_ranges_request = GetAccountRange { - id: 7, - root_hash, - starting_hash: H256::zero(), - limit_hash: H256::from_str( - "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - ) - .unwrap(), - response_bytes: 500, - }; - loop { - // TODO: Randomize id - account_ranges_request.id += 1; - info!("[Sync] Sending Block headers request "); - // Send a GetBlockBodies request to a peer - if peers - .lock() - .await - .send_message_to_peer(Message::GetAccountRange(account_ranges_request.clone())) - .await - .is_err() - { - info!("[Sync] No peers available, retrying in 10 sec"); - // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again - tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - continue; - }; - // Wait for the peer to reply - match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { - Ok(Some(Message::AccountRange(message))) - if message.id == account_ranges_request.id && !message.accounts.is_empty() => - { - info!("[SYNC] Received {} Accounts", message.accounts.len()); - } - - // Bad peer response, lets try a different peer - Ok(Some(_)) => info!("[Sync] Bad peer response"), - // Reply timeouted/peer shut down, lets try a different peer - _ => info!("[Sync] Peer response timeout( Snap Account Range)"), - } - } - } -} - -async fn receive_block_bodies( - channel: &mut tokio::sync::mpsc::Receiver, - id: u64, -) -> Option { - loop { - match channel.recv().await { - Some(Message::BlockBodies(response)) if response.id == id => return Some(response), - // Ignore replies that don't match the expected id (such as late responses) - Some(_other_response) => continue, - None => return None, - } - } + // for root_hash in state_roots { + // // Fetch Account Ranges + // let mut account_ranges_request = GetAccountRange { + // id: 7, + // root_hash, + // starting_hash: H256::zero(), + // limit_hash: H256::from_str( + // "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", + // ) + // .unwrap(), + // response_bytes: 500, + // }; + // loop { + // // TODO: Randomize id + // account_ranges_request.id += 1; + // info!("[Sync] Sending Block headers request "); + // // Send a GetBlockBodies request to a peer + // if peers + // .lock() + // .await + // .send_message_to_peer(Message::GetAccountRange(account_ranges_request.clone())) + // .await + // .is_err() + // { + // info!("[Sync] No peers available, retrying in 10 sec"); + // // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + // tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + // continue; + // }; + // // Wait for the peer to reply + // match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { + // Ok(Some(Message::AccountRange(message))) + // if message.id == account_ranges_request.id && !message.accounts.is_empty() => + // { + // info!("[SYNC] Received {} Accounts", message.accounts.len()); + // } + + // // Bad peer response, lets try a different peer + // Ok(Some(_)) => info!("[Sync] Bad peer response"), + // // Reply timeouted/peer shut down, lets try a different peer + // _ => info!("[Sync] Peer response timeout( Snap Account Range)"), + // } + // } + // } } From 64835f30befe819e9b317e285f535b761e4fce25 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 18:52:50 -0300 Subject: [PATCH 225/345] Avoid holding the mutex for the kademlia table when sending requests to peers --- crates/networking/p2p/kademlia.rs | 58 ++++-------- crates/networking/p2p/net.rs | 5 +- crates/networking/p2p/peer_channels.rs | 108 +++++++++++++++++++++++ crates/networking/p2p/rlpx/connection.rs | 9 +- crates/networking/p2p/rlpx/message.rs | 2 +- crates/networking/p2p/sync.rs | 21 +---- 6 files changed, 137 insertions(+), 66 deletions(-) create mode 100644 crates/networking/p2p/peer_channels.rs diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 771601222..8d7bce72e 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -1,17 +1,10 @@ -use std::sync::Arc; - use crate::{ - discv4::{time_now_unix, FindNodeRequest}, - rlpx::{error::RLPxError, message::Message}, - types::Node, - MAX_MESSAGES_TO_BROADCAST, + discv4::{time_now_unix, FindNodeRequest}, peer_channels::PeerChannels, types::Node, }; use ethrex_core::{H256, H512, U256}; use sha3::{Digest, Keccak256}; -use tokio::sync::{ - mpsc::{self, UnboundedSender}, - Mutex, -}; +use tokio::sync::mpsc::UnboundedSender; +use tracing::info; pub const MAX_NODES_PER_BUCKET: usize = 16; const NUMBER_OF_BUCKETS: usize = 256; @@ -275,15 +268,26 @@ impl KademliaTable { } } - /// TODO: Use most recently pinged peer instead - pub(crate) fn get_peer(&self) -> Option { + /// TODO: Randomly select peer + pub fn get_peer(&self) -> Option { self.get_least_recently_pinged_peers(1).pop() } - // Returns true if the table has at least one peer - pub fn has_peers(&self) -> bool { - self.get_peer().is_some() + /// Returns the channel ends to an active peer connection + /// The peer is selected randomly (TODO), and doesn't guarantee that the selected peer is not currenlty busy + /// If no peer is found, this method will try again after 10 seconds + /// TODO: Filter peers by capabilities, set max amount of retries + pub async fn get_peer_channels(&self) -> PeerChannels { + loop { + if let Some(channels) = self.get_peer().and_then(|peer| peer.channels) { + return channels + } + info!("[Sync] No peers available, retrying in 10 sec"); + // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + } } + } /// Computes the distance between two nodes according to the discv4 protocol @@ -313,30 +317,6 @@ pub struct PeerData { pub channels: Option, } -#[derive(Debug, Clone)] -/// Holds the respective sender and receiver ends of the communication channels bewteen the peer data and its active connection -pub struct PeerChannels { - pub sender: mpsc::Sender, - pub receiver: Arc>>, -} - -impl PeerChannels { - /// Sets up the communication channels for the peer - /// Returns the channel endpoints to send to the active connection's listen loop - pub fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { - let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - let (connection_sender, receiver) = mpsc::channel::(MAX_MESSAGES_TO_BROADCAST); - ( - Self { - sender, - receiver: Arc::new(Mutex::new(receiver)), - }, - connection_sender, - connection_receiver, - ) - } -} - impl PeerData { pub fn new(record: Node, last_ping: u64, last_pong: u64, is_proven: bool) -> Self { Self { diff --git a/crates/networking/p2p/net.rs b/crates/networking/p2p/net.rs index 97106623e..87108a50d 100644 --- a/crates/networking/p2p/net.rs +++ b/crates/networking/p2p/net.rs @@ -19,8 +19,7 @@ use k256::{ pub use kademlia::KademliaTable; use kademlia::{bucket_number, MAX_NODES_PER_BUCKET}; use rand::rngs::OsRng; -use rlpx::connection::RLPxConnection; -pub use rlpx::message::Message as RLPxMessage; +use rlpx::{connection::RLPxConnection, message::Message as RLPxMessage}; use tokio::{ net::{TcpSocket, TcpStream, UdpSocket}, sync::{broadcast, Mutex}, @@ -32,7 +31,7 @@ use types::{Endpoint, Node}; pub mod bootnode; pub(crate) mod discv4; pub(crate) mod kademlia; -pub mod peer_request; +pub mod peer_channels; pub mod rlpx; pub(crate) mod snap; pub mod sync; diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs new file mode 100644 index 000000000..a1689f26c --- /dev/null +++ b/crates/networking/p2p/peer_channels.rs @@ -0,0 +1,108 @@ +use std::{sync::Arc, time::Duration}; + +use ethrex_core::{ + types::{BlockBody, BlockHeader}, + H256, +}; +use tokio::sync::{Mutex, mpsc}; + +use crate::{ + rlpx::eth::blocks::{ + BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, + }, RLPxMessage +}; + +pub const PEER_REPLY_TIMOUT: Duration = Duration::from_secs(45); +pub const MAX_MESSAGES_IN_PEER_CHANNEL: usize = 25; + +#[derive(Debug, Clone)] +/// Holds the respective sender and receiver ends of the communication channels bewteen the peer data and its active connection +pub struct PeerChannels { + sender: mpsc::Sender, + pub receiver: Arc>>, +} + + +impl PeerChannels { + /// Sets up the communication channels for the peer + /// Returns the channel endpoints to send to the active connection's listen loop + pub fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); + let (connection_sender, receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); + ( + Self { + sender, + receiver: Arc::new(Mutex::new(receiver)), + }, + connection_sender, + connection_receiver, + ) + } + + /// Requests block headers from the peer + /// Returns the response message or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_block_headers(&self, start: H256) -> Option> { + let request_id = rand::random(); + let request = RLPxMessage::GetBlockHeaders(GetBlockHeaders { + id: request_id, + startblock: start.into(), + limit: BLOCK_HEADER_LIMIT, + skip: 0, + reverse: false, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let block_headers = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::BlockHeaders(BlockHeaders { id, block_headers })) + if id == request_id => + { + return Some(block_headers) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + (!block_headers.is_empty()).then_some(block_headers) + } + + /// Requests block headers from the peer + /// Returns the response message or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_block_bodies(&self, block_hashes: Vec) -> Option> { + let request_id = rand::random(); + let request = RLPxMessage::GetBlockBodies(GetBlockBodies { + id: request_id, + block_hashes, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let block_bodies = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::BlockBodies(BlockBodies { id, block_bodies })) + if id == request_id => + { + return Some(block_bodies) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + (!block_bodies.is_empty()).then_some(block_bodies) + } +} diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 86ed574f8..ed240cdd3 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,8 +1,7 @@ use std::sync::Arc; use crate::{ - kademlia::PeerChannels, - rlpx::{ + peer_channels::PeerChannels, rlpx::{ eth::{ backend, blocks::{BlockBodies, BlockHeaders}, @@ -12,12 +11,10 @@ use crate::{ message::Message, p2p::{self, DisconnectMessage, PingMessage, PongMessage}, utils::id2pubkey, - }, - snap::{ + }, snap::{ process_account_range_request, process_byte_codes_request, process_storage_ranges_request, process_trie_nodes_request, - }, - MAX_DISC_PACKET_SIZE, + }, MAX_DISC_PACKET_SIZE }; use super::{ diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 8961b69e1..486eb89b2 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -20,7 +20,7 @@ pub trait RLPxMessage: Sized { fn decode(msg_data: &[u8]) -> Result; } #[derive(Debug, Clone)] -pub enum Message { +pub(crate) enum Message { Hello(HelloMessage), Disconnect(DisconnectMessage), Ping(PingMessage), diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 86587f852..a2b28f26b 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -36,22 +36,11 @@ impl SyncManager { // Ask for block headers let mut all_block_headers = vec![]; let mut all_block_hashes = vec![]; - // Make sure we have active peers before we start making requests - loop { - if self.peers.lock().await.has_peers() { - break; - } - info!("[Sync] No peers available, retrying in 10 sec"); - // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again - tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - } loop { + let peer = self.peers.lock().await.get_peer_channels().await; info!("[Sync] Requesting Block Headers from {current_head}"); // Request Block Headers from Peer - if let Some(block_headers) = self - .peers - .lock() - .await + if let Some(block_headers) = peer .request_block_headers(current_head) .await { @@ -159,11 +148,9 @@ async fn fetch_blocks_and_receipts( // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts // Fetch Block Bodies loop { + let peer = peers.lock().await.get_peer_channels().await; info!("[Sync] Requesting Block Headers "); - if let Some(block_bodies) = peers - .lock() - .await - .request_block_bodies(block_hashes.clone()) + if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()) .await { info!("[SYNC] Received {} Block Bodies", block_bodies.len()); From c798c350bf15de9f70a8bbcc11189d6992b8a6b5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:06:28 -0300 Subject: [PATCH 226/345] Migrate to full sync --- crates/networking/p2p/Cargo.toml | 1 + crates/networking/p2p/kademlia.rs | 7 +- crates/networking/p2p/peer_channels.rs | 16 +- crates/networking/p2p/rlpx/connection.rs | 9 +- crates/networking/p2p/sync.rs | 200 +++++------------------ 5 files changed, 64 insertions(+), 169 deletions(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index e2b3f71e0..10c57bd41 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" [dependencies] ethrex-core.workspace = true +ethrex-blockchain.workspace = true ethrex-rlp.workspace = true ethrex-storage.workspace = true ethrex-trie.workspace = true diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 8d7bce72e..2dcfc99b6 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -1,5 +1,7 @@ use crate::{ - discv4::{time_now_unix, FindNodeRequest}, peer_channels::PeerChannels, types::Node, + discv4::{time_now_unix, FindNodeRequest}, + peer_channels::PeerChannels, + types::Node, }; use ethrex_core::{H256, H512, U256}; use sha3::{Digest, Keccak256}; @@ -280,14 +282,13 @@ impl KademliaTable { pub async fn get_peer_channels(&self) -> PeerChannels { loop { if let Some(channels) = self.get_peer().and_then(|peer| peer.channels) { - return channels + return channels; } info!("[Sync] No peers available, retrying in 10 sec"); // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; } } - } /// Computes the distance between two nodes according to the discv4 protocol diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index a1689f26c..871a09997 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -4,12 +4,13 @@ use ethrex_core::{ types::{BlockBody, BlockHeader}, H256, }; -use tokio::sync::{Mutex, mpsc}; +use tokio::sync::{mpsc, Mutex}; use crate::{ rlpx::eth::blocks::{ BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, - }, RLPxMessage + }, + RLPxMessage, }; pub const PEER_REPLY_TIMOUT: Duration = Duration::from_secs(45); @@ -22,13 +23,14 @@ pub struct PeerChannels { pub receiver: Arc>>, } - impl PeerChannels { /// Sets up the communication channels for the peer /// Returns the channel endpoints to send to the active connection's listen loop pub fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { - let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); - let (connection_sender, receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); + let (sender, connection_receiver) = + mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); + let (connection_sender, receiver) = + mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); ( Self { sender, @@ -80,6 +82,7 @@ impl PeerChannels { /// - The response timed out /// - The response was empty or not valid pub async fn request_block_bodies(&self, block_hashes: Vec) -> Option> { + let block_hashes_len = block_hashes.len(); let request_id = rand::random(); let request = RLPxMessage::GetBlockBodies(GetBlockBodies { id: request_id, @@ -103,6 +106,7 @@ impl PeerChannels { }) .await .ok()??; - (!block_bodies.is_empty()).then_some(block_bodies) + // Check that the response is not empty and does not contain more bodies than the ones requested + (!block_bodies.is_empty() && block_bodies.len() <= block_hashes_len).then_some(block_bodies) } } diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index ed240cdd3..a5c5622da 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -1,7 +1,8 @@ use std::sync::Arc; use crate::{ - peer_channels::PeerChannels, rlpx::{ + peer_channels::PeerChannels, + rlpx::{ eth::{ backend, blocks::{BlockBodies, BlockHeaders}, @@ -11,10 +12,12 @@ use crate::{ message::Message, p2p::{self, DisconnectMessage, PingMessage, PongMessage}, utils::id2pubkey, - }, snap::{ + }, + snap::{ process_account_range_request, process_byte_codes_request, process_storage_ranges_request, process_trie_nodes_request, - }, MAX_DISC_PACKET_SIZE + }, + MAX_DISC_PACKET_SIZE, }; use super::{ diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index a2b28f26b..b882bfcec 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use ethrex_core::{ - types::{validate_block_header, BlockHash, BlockHeader, InvalidBlockHeaderError}, + types::{validate_block_header, Block, BlockHash, BlockHeader, InvalidBlockHeaderError}, H256, }; use ethrex_storage::Store; @@ -10,15 +10,12 @@ use tracing::info; use crate::kademlia::KademliaTable; -/// Manager in charge of the snap-sync(for now, will also handle full sync) process -/// TaskList: -/// - Fetch latest block headers (should we ask what the latest block is first?) -/// - Validate block headers -/// - Fetch full Blocks and Receipts || Download Raw State (accounts, storages, bytecodes) -/// - Healing +/// Manager in charge the sync process +/// Only performs full-sync but will also be in charge of snap-sync in the future #[derive(Debug)] pub struct SyncManager { // true: syncmode = snap, false = syncmode = full + #[allow(unused)] snap_mode: bool, peers: Arc>, } @@ -27,48 +24,27 @@ impl SyncManager { pub fn new(peers: Arc>, snap_mode: bool) -> Self { Self { snap_mode, peers } } - // TODO: only uses snap sync, should also process full sync once implemented + + /// Starts a sync cycle, updating the state with all blocks between the current head and the sync head + /// TODO: only uses full sync, should also process snap sync once implemented pub async fn start_sync(&mut self, mut current_head: H256, sync_head: H256, store: Store) { - info!("Starting snap-sync from current head {current_head} to sync_head {sync_head}"); + info!("Syncing from current head {current_head} to sync_head {sync_head}"); + let start_time = Instant::now(); // Request all block headers between the current head and the sync head // We will begin from the current head so that we download the earliest state first // This step is not parallelized - // Ask for block headers let mut all_block_headers = vec![]; let mut all_block_hashes = vec![]; loop { let peer = self.peers.lock().await.get_peer_channels().await; - info!("[Sync] Requesting Block Headers from {current_head}"); + info!("Requesting Block Headers from {current_head}"); // Request Block Headers from Peer - if let Some(block_headers) = peer - .request_block_headers(current_head) - .await - { - // We received the correct message, we can now: - // - Validate the batch of headers received and start downloading their state (Future Iteration) - // - Check if we need to download another batch (aka we don't have the sync_head yet) - - // Validate header batch - if validate_header_batch(&block_headers).is_err() { - info!("[Sync] Invalid header in batch"); - continue; - } + if let Some(block_headers) = peer.request_block_headers(current_head).await { + info!("Received block headers {}..{}", block_headers.first().unwrap(), block_headers.last().unwrap()); + // Keep headers so we can process them later // Discard the first header as we already have it - let headers = &block_headers[1..]; - let block_hashes = headers - .iter() - .map(|header| header.compute_block_hash()) - .collect::>(); - info!( - "Received header batch {}..{}", - block_hashes.first().unwrap(), - block_hashes.last().unwrap() - ); - - // First iteration will not process the batch, but will wait for all headers to be fetched and validated - // before processing the whole batch - all_block_headers.extend_from_slice(&headers); - all_block_hashes.extend_from_slice(&block_hashes); + all_block_headers.extend_from_slice(&block_headers[1..]); + all_block_hashes.extend_from_slice(&block_hashes[1..]); // Check if we already reached our sync head or if we need to fetch more blocks if !block_hashes.contains(&sync_head) { @@ -79,49 +55,24 @@ impl SyncManager { break; } } - info!("[Sync] Peer response timeout (Headers)"); + info!("Peer response timeout (Headers)"); } - info!("[Sync] All headers fetched and validated"); - // [First Iteration] We finished fetching all headers, now we can process them - // We will launch 2 tasks to: - // - Fetch each block's state via snap p2p requests - // - Fetch each blocks and its receipts via eth p2p requests - let fetch_blocks_and_receipts_handle = tokio::spawn(fetch_blocks_and_receipts( - all_block_hashes.clone(), + info!("All headers fetched"); + // We finished fetching all headers, now we can process them + // TODO: snap-sync: launch tasks to fetch blocks and state in parallel + // full-sync: Fetch all block bodies and execute them sequentially to build the state + tokio::spawn(download_and_run_blocks( + all_block_hashes, + all_block_headers, self.peers.clone(), store.clone(), - )); - let state_roots = all_block_headers - .iter() - .map(|header| header.state_root) - .collect::>(); - let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( - state_roots.clone(), - self.peers.clone(), - store.clone(), - )); - // Store headers - let mut latest_block_number = 0; - for (header, hash) in all_block_headers - .into_iter() - .zip(all_block_hashes.into_iter()) - { - // TODO: Handle error - latest_block_number = header.number; - store.set_canonical_block(header.number, hash).unwrap(); - store.add_block_header(hash, header).unwrap(); - } - // TODO: Handle error - let err = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); - // Set latest block number here to avoid reading state that is currently being synced - store - .update_latest_block_number(latest_block_number) - .unwrap(); - // Sync finished + )) + .await; + info!("Sync finished, time elapsed: {} ", start_time.elapsed()); } /// Creates a dummy SyncManager for tests where syncing is not needed - /// This should only be used it tests as it won't be able to connect to the p2p network + /// This should only be used in tests as it won't be able to connect to the p2p network pub fn dummy() -> Self { let dummy_peer_table = Arc::new(Mutex::new(KademliaTable::new(Default::default()))); Self { @@ -131,96 +82,31 @@ impl SyncManager { } } -fn validate_header_batch(headers: &[BlockHeader]) -> Result<(), InvalidBlockHeaderError> { - // The first header is a header we have already validated (either current last block or last block in previous batch) - for headers in headers.windows(2) { - // TODO: Validation commented to make this work with older blocks - //validate_block_header(&headers[0], &headers[1])?; - } - Ok(()) -} - -async fn fetch_blocks_and_receipts( +async fn download_and_run_blocks( mut block_hashes: Vec, + mut block_headers: Vec, peers: Arc>, store: Store, -) { - // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts - // Fetch Block Bodies +) -> Resu{ loop { let peer = peers.lock().await.get_peer_channels().await; - info!("[Sync] Requesting Block Headers "); - if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()) - .await - { - info!("[SYNC] Received {} Block Bodies", block_bodies.len()); - // Track which bodies we have already fetched - let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); - // Store Block Bodies - for (hash, body) in fetched_hashes.into_iter().zip(block_bodies.into_iter()) { - // TODO: handle error - store.add_block_body(hash.clone(), body).unwrap() + info!("Requesting Block Bodies "); + if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { + info!("Received {} Block Bodies", block_bodies.len()); + // Execute and store blocks + for body in block_bodies { + // We already validated that there are no more block bodies than the ones requested + let header = block_headers.remove(0); + let hash = block_hashes.remove(0); + let block = Block::new(header, body); + ethrex_blockchain::add_block(&block, &store); } - + info!("Executed & stored {} blocks", block_bodies.len()); // Check if we need to ask for another batch - if remaining_hashes.is_empty() { + if block_hashes.is_empty() { break; - } else { - block_hashes = remaining_hashes.to_vec(); } } - info!("[Sync] Peer response timeout( Blocks & Receipts)"); + info!("Peer response timeout(Blocks)"); } - // TODO: Fetch Receipts and store them -} - -async fn fetch_snap_state( - state_roots: Vec, - peers: Arc>, - store: Store, -) { - // for root_hash in state_roots { - // // Fetch Account Ranges - // let mut account_ranges_request = GetAccountRange { - // id: 7, - // root_hash, - // starting_hash: H256::zero(), - // limit_hash: H256::from_str( - // "0xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff", - // ) - // .unwrap(), - // response_bytes: 500, - // }; - // loop { - // // TODO: Randomize id - // account_ranges_request.id += 1; - // info!("[Sync] Sending Block headers request "); - // // Send a GetBlockBodies request to a peer - // if peers - // .lock() - // .await - // .send_message_to_peer(Message::GetAccountRange(account_ranges_request.clone())) - // .await - // .is_err() - // { - // info!("[Sync] No peers available, retrying in 10 sec"); - // // This is the unlikely case where we just started the node and don't have peers, wait a bit and try again - // tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - // continue; - // }; - // // Wait for the peer to reply - // match tokio::time::timeout(REPLY_TIMEOUT, reply_receiver.recv()).await { - // Ok(Some(Message::AccountRange(message))) - // if message.id == account_ranges_request.id && !message.accounts.is_empty() => - // { - // info!("[SYNC] Received {} Accounts", message.accounts.len()); - // } - - // // Bad peer response, lets try a different peer - // Ok(Some(_)) => info!("[Sync] Bad peer response"), - // // Reply timeouted/peer shut down, lets try a different peer - // _ => info!("[Sync] Peer response timeout( Snap Account Range)"), - // } - // } - // } } From 62084e8c691cf72f9193c160fffc2be6f69219ec Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:21:18 -0300 Subject: [PATCH 227/345] Error handling --- crates/networking/p2p/sync.rs | 35 ++++++++++++++++++++++++++++------- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index b882bfcec..6ad06a1cf 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -6,7 +6,7 @@ use ethrex_core::{ }; use ethrex_storage::Store; use tokio::sync::Mutex; -use tracing::info; +use tracing::{info, warn}; use crate::kademlia::KademliaTable; @@ -40,7 +40,12 @@ impl SyncManager { info!("Requesting Block Headers from {current_head}"); // Request Block Headers from Peer if let Some(block_headers) = peer.request_block_headers(current_head).await { - info!("Received block headers {}..{}", block_headers.first().unwrap(), block_headers.last().unwrap()); + // We already checked that the range is not empty + info!( + "Received block headers {}..{}", + block_headers.first().unwrap(), + block_headers.last().unwrap() + ); // Keep headers so we can process them later // Discard the first header as we already have it all_block_headers.extend_from_slice(&block_headers[1..]); @@ -61,14 +66,24 @@ impl SyncManager { // We finished fetching all headers, now we can process them // TODO: snap-sync: launch tasks to fetch blocks and state in parallel // full-sync: Fetch all block bodies and execute them sequentially to build the state - tokio::spawn(download_and_run_blocks( + match tokio::spawn(download_and_run_blocks( all_block_hashes, all_block_headers, self.peers.clone(), store.clone(), )) - .await; - info!("Sync finished, time elapsed: {} ", start_time.elapsed()); + .await + { + Ok(Ok(())) => info!("Sync finished, time elapsed: {} ", start_time.elapsed()), + Ok(Err(error)) => warn!( + "Sync failed due to {error}, time elapsed: {} ", + start_time.elapsed() + ), + _ => warn!( + "Sync failed due to internal error, time elapsed: {} ", + start_time.elapsed() + ), + } } /// Creates a dummy SyncManager for tests where syncing is not needed @@ -82,12 +97,14 @@ impl SyncManager { } } +/// Requests block bodies from peers via p2p, executes and stores them +/// Returns an error if there was a problem while executing or validating the blocks async fn download_and_run_blocks( mut block_hashes: Vec, mut block_headers: Vec, peers: Arc>, store: Store, -) -> Resu{ +) -> Result<(), ChainError> { loop { let peer = peers.lock().await.get_peer_channels().await; info!("Requesting Block Bodies "); @@ -99,7 +116,10 @@ async fn download_and_run_blocks( let header = block_headers.remove(0); let hash = block_hashes.remove(0); let block = Block::new(header, body); - ethrex_blockchain::add_block(&block, &store); + if let Err(error) = ethrex_blockchain::add_block(&block, &store) { + warn!("Failed to add block during FullSync: {error}"); + return Err(error); + } } info!("Executed & stored {} blocks", block_bodies.len()); // Check if we need to ask for another batch @@ -109,4 +129,5 @@ async fn download_and_run_blocks( } info!("Peer response timeout(Blocks)"); } + Ok(()) } From 02a89b2ab5fb8aa130777299c26d56c66b71471c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:41:22 -0300 Subject: [PATCH 228/345] Fixes --- crates/networking/p2p/sync.rs | 36 +++++++++++++++++++---------------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 6ad06a1cf..d407d38be 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,11 +1,12 @@ use std::sync::Arc; +use ethrex_blockchain::error::ChainError; use ethrex_core::{ types::{validate_block_header, Block, BlockHash, BlockHeader, InvalidBlockHeaderError}, H256, }; use ethrex_storage::Store; -use tokio::sync::Mutex; +use tokio::{sync::Mutex, time::Instant}; use tracing::{info, warn}; use crate::kademlia::KademliaTable; @@ -40,12 +41,11 @@ impl SyncManager { info!("Requesting Block Headers from {current_head}"); // Request Block Headers from Peer if let Some(block_headers) = peer.request_block_headers(current_head).await { - // We already checked that the range is not empty - info!( - "Received block headers {}..{}", - block_headers.first().unwrap(), - block_headers.last().unwrap() - ); + info!("Received {} block headers", block_headers.len()); + let block_hashes = block_headers + .iter() + .map(|header| header.compute_block_hash()) + .collect::>(); // Keep headers so we can process them later // Discard the first header as we already have it all_block_headers.extend_from_slice(&block_headers[1..]); @@ -74,14 +74,17 @@ impl SyncManager { )) .await { - Ok(Ok(())) => info!("Sync finished, time elapsed: {} ", start_time.elapsed()), + Ok(Ok(())) => info!( + "Sync finished, time elapsed: {} secs", + start_time.elapsed().as_secs() + ), Ok(Err(error)) => warn!( - "Sync failed due to {error}, time elapsed: {} ", - start_time.elapsed() + "Sync failed due to {error}, time elapsed: {} secs ", + start_time.elapsed().as_secs() ), _ => warn!( - "Sync failed due to internal error, time elapsed: {} ", - start_time.elapsed() + "Sync failed due to internal error, time elapsed: {} secs", + start_time.elapsed().as_secs() ), } } @@ -109,19 +112,20 @@ async fn download_and_run_blocks( let peer = peers.lock().await.get_peer_channels().await; info!("Requesting Block Bodies "); if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { - info!("Received {} Block Bodies", block_bodies.len()); + let block_bodies_len = block_bodies.len(); + info!("Received {} Block Bodies", block_bodies_len); // Execute and store blocks - for body in block_bodies { + for body in block_bodies.into_iter() { // We already validated that there are no more block bodies than the ones requested let header = block_headers.remove(0); - let hash = block_hashes.remove(0); + let _hash = block_hashes.remove(0); let block = Block::new(header, body); if let Err(error) = ethrex_blockchain::add_block(&block, &store) { warn!("Failed to add block during FullSync: {error}"); return Err(error); } } - info!("Executed & stored {} blocks", block_bodies.len()); + info!("Executed & stored {} blocks", block_bodies_len); // Check if we need to ask for another batch if block_hashes.is_empty() { break; From a60948deea3fc43916f7fe3305444ebc733bcd9f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:45:04 -0300 Subject: [PATCH 229/345] revert uneeded changes --- crates/networking/p2p/rlpx/eth/receipts.rs | 2 +- crates/networking/p2p/rlpx/message.rs | 2 +- crates/networking/p2p/rlpx/p2p.rs | 8 ++++---- crates/networking/p2p/rlpx/snap.rs | 22 +++++++++++----------- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/crates/networking/p2p/rlpx/eth/receipts.rs b/crates/networking/p2p/rlpx/eth/receipts.rs index 1ddaf65ca..6c02119d8 100644 --- a/crates/networking/p2p/rlpx/eth/receipts.rs +++ b/crates/networking/p2p/rlpx/eth/receipts.rs @@ -48,7 +48,7 @@ impl RLPxMessage for GetReceipts { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#receipts-0x10 -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct Receipts { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 486eb89b2..377d6966a 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -19,7 +19,7 @@ pub trait RLPxMessage: Sized { fn decode(msg_data: &[u8]) -> Result; } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) enum Message { Hello(HelloMessage), Disconnect(DisconnectMessage), diff --git a/crates/networking/p2p/rlpx/p2p.rs b/crates/networking/p2p/rlpx/p2p.rs index 2281fe1ae..fdc752a83 100644 --- a/crates/networking/p2p/rlpx/p2p.rs +++ b/crates/networking/p2p/rlpx/p2p.rs @@ -44,7 +44,7 @@ impl RLPDecode for Capability { } } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct HelloMessage { pub(crate) capabilities: Vec<(Capability, u8)>, pub(crate) node_id: PublicKey, @@ -100,7 +100,7 @@ impl RLPxMessage for HelloMessage { } } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct DisconnectMessage { pub(crate) reason: Option, } @@ -147,7 +147,7 @@ impl RLPxMessage for DisconnectMessage { } } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct PingMessage {} impl PingMessage { @@ -177,7 +177,7 @@ impl RLPxMessage for PingMessage { } } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct PongMessage {} impl PongMessage { diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index ea38f02b6..2c7643790 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -16,7 +16,7 @@ use ethrex_rlp::{ // Snap Capability Messages -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct GetAccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response pub id: u64, @@ -26,7 +26,7 @@ pub(crate) struct GetAccountRange { pub response_bytes: u64, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct AccountRange { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response pub id: u64, @@ -34,7 +34,7 @@ pub(crate) struct AccountRange { pub proof: Vec, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct GetStorageRanges { pub id: u64, pub root_hash: H256, @@ -44,27 +44,27 @@ pub(crate) struct GetStorageRanges { pub response_bytes: u64, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct StorageRanges { pub id: u64, pub slots: Vec>, pub proof: Vec, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct GetByteCodes { pub id: u64, pub hashes: Vec, pub bytes: u64, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct ByteCodes { pub id: u64, pub codes: Vec, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct GetTrieNodes { pub id: u64, pub root_hash: H256, @@ -74,7 +74,7 @@ pub(crate) struct GetTrieNodes { pub bytes: u64, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct TrieNodes { pub id: u64, pub nodes: Vec, @@ -320,13 +320,13 @@ impl RLPxMessage for TrieNodes { // Intermediate structures -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct AccountRangeUnit { pub hash: H256, pub account: AccountStateSlim, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct AccountStateSlim { pub nonce: u64, pub balance: U256, @@ -334,7 +334,7 @@ pub struct AccountStateSlim { pub code_hash: Bytes, } -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct StorageSlot { pub hash: H256, pub data: U256, From cc4452e3de7262a6a485524a37c67a325f43b4c6 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:45:53 -0300 Subject: [PATCH 230/345] revert uneeded changes --- crates/common/types/fork_id.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/common/types/fork_id.rs b/crates/common/types/fork_id.rs index b639460c0..ec3945612 100644 --- a/crates/common/types/fork_id.rs +++ b/crates/common/types/fork_id.rs @@ -10,7 +10,7 @@ use ethereum_types::H32; use super::{BlockHash, BlockNumber, ChainConfig}; -#[derive(Debug, Clone, PartialEq)] +#[derive(Debug, PartialEq)] pub struct ForkId { fork_hash: H32, fork_next: BlockNumber, From c86b553c7d347b0203df2d20716ef8fffd4c9e3b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:56:56 -0300 Subject: [PATCH 231/345] Updates & fixes --- crates/networking/p2p/rlpx/eth/status.rs | 2 +- crates/networking/p2p/snap.rs | 29 ------------------------ crates/networking/p2p/sync.rs | 5 +++- 3 files changed, 5 insertions(+), 31 deletions(-) diff --git a/crates/networking/p2p/rlpx/eth/status.rs b/crates/networking/p2p/rlpx/eth/status.rs index cefd08e32..f39c36dcf 100644 --- a/crates/networking/p2p/rlpx/eth/status.rs +++ b/crates/networking/p2p/rlpx/eth/status.rs @@ -12,7 +12,7 @@ use ethrex_rlp::{ structs::{Decoder, Encoder}, }; -#[derive(Debug, Clone)] +#[derive(Debug)] pub(crate) struct StatusMessage { pub(crate) eth_version: u32, pub(crate) network_id: u64, diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index ebb22e435..352e5f947 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -151,35 +151,6 @@ pub fn process_trie_nodes_request( }) } -// Response Processing - -#[allow(unused)] -pub fn validate_account_range_response( - request: &GetAccountRange, - response: &AccountRange, -) -> Result<(), RLPxError> { - // Verify Range Proof - let (keys, accounts): (Vec<_>, Vec<_>) = response - .accounts - .iter() - .map(|unit| { - ( - unit.hash, - AccountState::from(unit.account.clone()).encode_to_vec(), - ) - }) - .unzip(); - let proof = encodable_to_proof(&response.proof); - verify_range( - request.root_hash, - &request.starting_hash, - &keys, - &accounts, - &proof, - )?; - Ok(()) -} - // Helper method to convert proof to RLP-encodable format #[inline] fn proof_to_encodable(proof: Vec>) -> Vec { diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index d407d38be..a2e47bfbd 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -118,12 +118,15 @@ async fn download_and_run_blocks( for body in block_bodies.into_iter() { // We already validated that there are no more block bodies than the ones requested let header = block_headers.remove(0); - let _hash = block_hashes.remove(0); + let hash = block_hashes.remove(0); + let number = header.number; let block = Block::new(header, body); if let Err(error) = ethrex_blockchain::add_block(&block, &store) { warn!("Failed to add block during FullSync: {error}"); return Err(error); } + store.set_canonical_block(number, hash)?; + store.update_latest_block_number(number)?; } info!("Executed & stored {} blocks", block_bodies_len); // Check if we need to ask for another batch From 05848a2c981f4b26d18269a2f2ce378487ed29ad Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 13:58:23 -0300 Subject: [PATCH 232/345] Add ethereum/sync test suite to hive (1 test) --- .github/workflows/hive.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 9ef3a66fe..7062a244f 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -43,8 +43,8 @@ jobs: name: "Devp2p eth tests" run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="eth/Status|GetBlockHeaders|SimultaneousRequests|SameRequestID|ZeroRequestID|GetBlockBodies|MaliciousHandshake|MaliciousStatus|Transaction" - simulation: engine - name: "Engine Auth and EC tests" - run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities)/" + name: "Engine Auth, EC & Sync tests" + run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities|sync)/" - simulation: engine-cancun name: "Cancun Engine tests" run_command: make run-hive-on-latest SIMULATION=ethereum/engine HIVE_EXTRA_ARGS="--sim.parallelism 4" TEST_PATTERN="engine-cancun/Blob Transactions On Block 1|Blob Transaction Ordering, Single|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|Incorrect BlobGasUsed|Bad Hash|ParentHash equals BlockHash|RPC:|in ForkchoiceState|Unknown|Invalid PayloadAttributes|Unique|ForkchoiceUpdated Version on Payload Request|Re-Execute Payload|In-Order Consecutive Payload|Multiple New Payloads|Valid NewPayload->|NewPayload with|Payload Build after|Build Payload with|Invalid Missing Ancestor ReOrg, StateRoot|Re-Org Back to|Re-org to Previously|Safe Re-Org to Side Chain|Transaction Re-Org, Re-Org Back In|Re-Org Back into Canonical Chain, Depth=5|Suggested Fee Recipient Test|PrevRandao Opcode|Invalid NewPayload, [^R][^e]|Fork ID Genesis=0, Cancun=0|Fork ID Genesis=0, Cancun=1|Fork ID Genesis=1, Cancun=0|Fork ID Genesis=1, Cancun=2, Shanghai=2" From 9d608f9a57cba6615e44a7961b40f16d0cbb7957 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:25:32 -0300 Subject: [PATCH 233/345] Default to full-sync --- cmd/ethrex/ethrex.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index f45d9fba8..8607d1428 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -115,8 +115,7 @@ async fn main() { .get_one::("datadir") .map_or(set_datadir(DEFAULT_DATADIR), |datadir| set_datadir(datadir)); - // TODO: Use snap as default - let is_snap_sync = is_snap_sync(&matches); + let snap_sync = is_snap_sync(&matches); let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); @@ -294,10 +293,13 @@ fn is_snap_sync(matches: &clap::ArgMatches) -> bool { if let Some(syncmode) = syncmode { match &**syncmode { "full" => false, - "snap" => true, + "snap" => { + info!("snap-sync not available, defaulting to full-sync") + }, other => panic!("Invalid syncmode {other} expected either snap or full"), } } else { + info!("snap-sync not available, defaulting to full-sync"); false } } From 57f1e29be376cff24e02f243f9a72e0936649e5e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:28:05 -0300 Subject: [PATCH 234/345] revert uneeded changes --- crates/networking/p2p/rlpx/error.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 51c0c5914..120a8ae1e 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -45,8 +45,6 @@ pub enum RLPxError { RecvError(#[from] RecvError), #[error(transparent)] Send(#[from] tokio::sync::mpsc::error::SendError), - #[error("No peers to interact with yet")] - NoPeers, } // Grouping all cryptographic related errors in a single CryptographicError variant From a7350d8a0a1ac06241c81316e43402d2d9094e55 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:42:18 -0300 Subject: [PATCH 235/345] revert uneeded changes --- cmd/ethrex/ethrex.rs | 12 ++++++------ crates/networking/p2p/rlpx/connection.rs | 6 +++--- crates/networking/p2p/rlpx/error.rs | 4 +--- 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 8607d1428..062989d27 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -116,6 +116,9 @@ async fn main() { .map_or(set_datadir(DEFAULT_DATADIR), |datadir| set_datadir(datadir)); let snap_sync = is_snap_sync(&matches); + if !snap_sync { + info!("snap-sync not available, defaulting to full-sync"); + } let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); @@ -176,7 +179,7 @@ async fn main() { // Create Kademlia Table here so we can access it from rpc server (for syncing) let peer_table = peer_table(signer.clone()); // Create SyncManager - let syncer = SyncManager::new(peer_table.clone(), is_snap_sync); + let syncer = SyncManager::new(peer_table.clone(), snap_sync); // TODO: Check every module starts properly. let tracker = TaskTracker::new(); @@ -293,14 +296,11 @@ fn is_snap_sync(matches: &clap::ArgMatches) -> bool { if let Some(syncmode) = syncmode { match &**syncmode { "full" => false, - "snap" => { - info!("snap-sync not available, defaulting to full-sync") - }, + "snap" => true, other => panic!("Invalid syncmode {other} expected either snap or full"), } } else { - info!("snap-sync not available, defaulting to full-sync"); - false + true } } diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index a5c5622da..a3e8d63b4 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -250,7 +250,7 @@ impl RLPxConnection { } } - pub async fn handle_peer_conn( + async fn handle_peer_conn( &mut self, sender: mpsc::Sender, mut receiver: mpsc::Receiver, @@ -330,7 +330,7 @@ impl RLPxConnection { async fn handle_message( &mut self, message: Message, - backend_send: mpsc::Sender, + sender: mpsc::Sender, ) -> Result<(), RLPxError> { let peer_supports_eth = self.capabilities.contains(&CAP_ETH); match message { @@ -392,7 +392,7 @@ impl RLPxConnection { | message @ Message::TrieNodes(_) | message @ Message::BlockBodies(_) | message @ Message::BlockHeaders(_) - | message @ Message::Receipts(_) => backend_send.send(message).await?, + | message @ Message::Receipts(_) => sender.send(message).await?, // TODO: Add new message types and handlers as they are implemented message => return Err(RLPxError::MessageNotHandled(format!("{message}"))), }; diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 120a8ae1e..0457051a1 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -8,7 +8,7 @@ use super::message::Message; // TODO improve errors #[derive(Debug, Error)] -pub enum RLPxError { +pub(crate) enum RLPxError { #[error("{0}")] HandshakeError(String), #[error("{0}")] @@ -37,8 +37,6 @@ pub enum RLPxError { StoreError(#[from] StoreError), #[error("Error in cryptographic library: {0}")] CryptographyError(String), - #[error(transparent)] - Trie(#[from] TrieError), #[error("Failed to broadcast msg: {0}")] BroadcastError(String), #[error(transparent)] From eb3ff59d05e9d3481d7ebbe87f226a9833472826 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:46:17 -0300 Subject: [PATCH 236/345] Fixes --- crates/networking/p2p/peer_channels.rs | 4 ++-- crates/networking/p2p/rlpx/error.rs | 1 - crates/networking/p2p/snap.rs | 3 +-- crates/networking/p2p/sync.rs | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 871a09997..d5cd2a0c5 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -20,13 +20,13 @@ pub const MAX_MESSAGES_IN_PEER_CHANNEL: usize = 25; /// Holds the respective sender and receiver ends of the communication channels bewteen the peer data and its active connection pub struct PeerChannels { sender: mpsc::Sender, - pub receiver: Arc>>, + receiver: Arc>>, } impl PeerChannels { /// Sets up the communication channels for the peer /// Returns the channel endpoints to send to the active connection's listen loop - pub fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + pub (crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); let (connection_sender, receiver) = diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 0457051a1..99b784e0b 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -1,6 +1,5 @@ use ethrex_rlp::error::{RLPDecodeError, RLPEncodeError}; use ethrex_storage::error::StoreError; -use ethrex_trie::TrieError; use thiserror::Error; use tokio::sync::broadcast::error::RecvError; diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 352e5f947..9f0b88ae0 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -1,8 +1,6 @@ use bytes::Bytes; -use ethrex_core::types::AccountState; use ethrex_rlp::encode::RLPEncode; use ethrex_storage::{error::StoreError, Store}; -use ethrex_trie::verify_range; use crate::rlpx::{ error::RLPxError, @@ -159,6 +157,7 @@ fn proof_to_encodable(proof: Vec>) -> Vec { // Helper method to obtain proof from RLP-encodable format #[inline] +#[allow(unused)] fn encodable_to_proof(proof: &Vec) -> Vec> { proof.into_iter().map(|bytes| bytes.to_vec()).collect() } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index a2e47bfbd..bc251c19c 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use ethrex_blockchain::error::ChainError; use ethrex_core::{ - types::{validate_block_header, Block, BlockHash, BlockHeader, InvalidBlockHeaderError}, + types::{Block, BlockHash, BlockHeader}, H256, }; use ethrex_storage::Store; From 0a8d4e8f202c9ced50e37b75cc6859a63661fa98 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:56:37 -0300 Subject: [PATCH 237/345] Clippy --- crates/networking/p2p/snap.rs | 6 +++--- crates/networking/p2p/sync.rs | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 9f0b88ae0..bf9ca53e8 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -152,14 +152,14 @@ pub fn process_trie_nodes_request( // Helper method to convert proof to RLP-encodable format #[inline] fn proof_to_encodable(proof: Vec>) -> Vec { - proof.into_iter().map(|bytes| Bytes::from(bytes)).collect() + proof.into_iter().map(Bytes::from).collect() } // Helper method to obtain proof from RLP-encodable format #[inline] #[allow(unused)] -fn encodable_to_proof(proof: &Vec) -> Vec> { - proof.into_iter().map(|bytes| bytes.to_vec()).collect() +fn encodable_to_proof(proof: &[Bytes]) -> Vec> { + proof.iter().map(|bytes| bytes.to_vec()).collect() } #[cfg(test)] diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index bc251c19c..81cc6248f 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -54,7 +54,7 @@ impl SyncManager { // Check if we already reached our sync head or if we need to fetch more blocks if !block_hashes.contains(&sync_head) { // Update the request to fetch the next batch - current_head = (*block_hashes.last().unwrap()).into(); + current_head = *block_hashes.last().unwrap(); } else { // No more headers to request break; From 493fa911f286d63e7ed938281de29c933cfc85f1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:58:18 -0300 Subject: [PATCH 238/345] revert hive workflow change --- .github/workflows/hive.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 0172b6e56..8f0bd9730 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -35,8 +35,8 @@ jobs: name: "Devp2p eth tests" run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="eth/Status|GetBlockHeaders|SimultaneousRequests|SameRequestID|ZeroRequestID|GetBlockBodies|MaliciousHandshake|MaliciousStatus|Transaction" - simulation: engine - name: "Engine Auth, EC & Sync tests" - run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities|sync)/" + name: "Engine Auth and EC tests" + run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities)/" - simulation: engine-cancun name: "Cancun Engine tests" run_command: make run-hive-on-latest SIMULATION=ethereum/engine HIVE_EXTRA_ARGS="--sim.parallelism 4" TEST_PATTERN="engine-cancun/Blob Transactions On Block 1|Blob Transaction Ordering, Single|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|Incorrect BlobGasUsed|Bad Hash|ParentHash equals BlockHash|RPC:|in ForkchoiceState|Unknown|Invalid PayloadAttributes|Unique|ForkchoiceUpdated Version on Payload Request|Re-Execute Payload|In-Order Consecutive Payload|Multiple New Payloads|Valid NewPayload->|NewPayload with|Payload Build after|Build Payload with|Invalid Missing Ancestor ReOrg, StateRoot|Re-Org Back to|Re-org to Previously|Safe Re-Org to Side Chain|Transaction Re-Org, Re-Org Back In|Re-Org Back into Canonical Chain, Depth=5|Suggested Fee Recipient Test|PrevRandao Opcode|Invalid NewPayload, [^R][^e]|Fork ID Genesis=0, Cancun=0|Fork ID Genesis=0, Cancun=1|Fork ID Genesis=1, Cancun=0|Fork ID Genesis=1, Cancun=2, Shanghai=2" From c6d6767f341e76f8a7db9f1523d815f3c8b1f82e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 15:59:32 -0300 Subject: [PATCH 239/345] Revert "[REVERT ME] Make V2 endpoints work in order to run hive sync test" This reverts commit 356eaf03d87e4e6748682afe7ba52fdcaf7c9d56. --- crates/networking/p2p/peer_channels.rs | 2 +- crates/networking/rpc/engine/payload.rs | 20 +++++++++++--------- crates/networking/rpc/rpc.rs | 2 -- crates/networking/rpc/types/payload.rs | 4 ++-- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index d5cd2a0c5..47858de15 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -26,7 +26,7 @@ pub struct PeerChannels { impl PeerChannels { /// Sets up the communication channels for the peer /// Returns the channel endpoints to send to the active connection's listen loop - pub (crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + pub(crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); let (connection_sender, receiver) = diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index abd59941f..9d7eb0089 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -43,14 +43,16 @@ impl RpcHandler for NewPayloadV3Request { let params = params .as_ref() .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - // if params.len() != 3 { - // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - // } + if params.len() != 3 { + return Err(RpcErr::BadParams("Expected 3 params".to_owned())); + } Ok(NewPayloadV3Request { payload: serde_json::from_value(params[0].clone()) .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: vec![], - parent_beacon_block_root: H256::zero(), + expected_blob_versioned_hashes: serde_json::from_value(params[1].clone()) + .map_err(|_| RpcErr::WrongParam("expected_blob_versioned_hashes".to_string()))?, + parent_beacon_block_root: serde_json::from_value(params[2].clone()) + .map_err(|_| RpcErr::WrongParam("parent_beacon_block_root".to_string()))?, }) } @@ -77,10 +79,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post Cancun fork let chain_config = storage.get_chain_config()?; - // let current_fork = chain_config.get_fork(block.header.timestamp); - // if current_fork < Fork::Cancun { - // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - // } + let current_fork = chain_config.get_fork(block.header.timestamp); + if current_fork < Fork::Cancun { + return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); + } // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 8ec54a7ca..cc7ed583c 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -254,9 +254,7 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), - "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), - "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index ee9d873eb..b35825039 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,9 +35,9 @@ pub struct ExecutionPayloadV3 { pub block_hash: H256, transactions: Vec, withdrawals: Vec, - #[serde(with = "serde_utils::u64::hex_str", default)] + #[serde(with = "serde_utils::u64::hex_str")] blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str", default)] + #[serde(with = "serde_utils::u64::hex_str")] excess_blob_gas: u64, } From 95a9af21ac1472ba876358711aa63b52b25cb7a6 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 16:13:14 -0300 Subject: [PATCH 240/345] revert hive workflow change --- .github/workflows/hive.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hive.yaml b/.github/workflows/hive.yaml index 8f0bd9730..f7dfd6db7 100644 --- a/.github/workflows/hive.yaml +++ b/.github/workflows/hive.yaml @@ -36,7 +36,7 @@ jobs: run_command: make run-hive-on-latest SIMULATION=devp2p TEST_PATTERN="eth/Status|GetBlockHeaders|SimultaneousRequests|SameRequestID|ZeroRequestID|GetBlockBodies|MaliciousHandshake|MaliciousStatus|Transaction" - simulation: engine name: "Engine Auth and EC tests" - run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities)/" + run_command: make run-hive-on-latest SIMULATION=ethereum/engine TEST_PATTERN="engine-(auth|exchange-capabilities)/" - simulation: engine-cancun name: "Cancun Engine tests" run_command: make run-hive-on-latest SIMULATION=ethereum/engine HIVE_EXTRA_ARGS="--sim.parallelism 4" TEST_PATTERN="engine-cancun/Blob Transactions On Block 1|Blob Transaction Ordering, Single|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdatedV3 Modifies Payload ID on Different Beacon Root|NewPayloadV3 After Cancun|NewPayloadV3 Versioned Hashes|Incorrect BlobGasUsed|Bad Hash|ParentHash equals BlockHash|RPC:|in ForkchoiceState|Unknown|Invalid PayloadAttributes|Unique|ForkchoiceUpdated Version on Payload Request|Re-Execute Payload|In-Order Consecutive Payload|Multiple New Payloads|Valid NewPayload->|NewPayload with|Payload Build after|Build Payload with|Invalid Missing Ancestor ReOrg, StateRoot|Re-Org Back to|Re-org to Previously|Safe Re-Org to Side Chain|Transaction Re-Org, Re-Org Back In|Re-Org Back into Canonical Chain, Depth=5|Suggested Fee Recipient Test|PrevRandao Opcode|Invalid NewPayload, [^R][^e]|Fork ID Genesis=0, Cancun=0|Fork ID Genesis=0, Cancun=1|Fork ID Genesis=1, Cancun=0|Fork ID Genesis=1, Cancun=2, Shanghai=2" From fd90f18677d32fe7dad704ac2c1bebeda404fc8e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 16:23:26 -0300 Subject: [PATCH 241/345] reduce tracing output --- crates/networking/p2p/sync.rs | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 81cc6248f..5f2589b01 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -7,7 +7,7 @@ use ethrex_core::{ }; use ethrex_storage::Store; use tokio::{sync::Mutex, time::Instant}; -use tracing::{info, warn}; +use tracing::{info, debug, warn}; use crate::kademlia::KademliaTable; @@ -38,10 +38,10 @@ impl SyncManager { let mut all_block_hashes = vec![]; loop { let peer = self.peers.lock().await.get_peer_channels().await; - info!("Requesting Block Headers from {current_head}"); + debug!("Requesting Block Headers from {current_head}"); // Request Block Headers from Peer if let Some(block_headers) = peer.request_block_headers(current_head).await { - info!("Received {} block headers", block_headers.len()); + debug!("Received {} block headers", block_headers.len()); let block_hashes = block_headers .iter() .map(|header| header.compute_block_hash()) @@ -60,9 +60,7 @@ impl SyncManager { break; } } - info!("Peer response timeout (Headers)"); } - info!("All headers fetched"); // We finished fetching all headers, now we can process them // TODO: snap-sync: launch tasks to fetch blocks and state in parallel // full-sync: Fetch all block bodies and execute them sequentially to build the state @@ -110,10 +108,10 @@ async fn download_and_run_blocks( ) -> Result<(), ChainError> { loop { let peer = peers.lock().await.get_peer_channels().await; - info!("Requesting Block Bodies "); + debug!("Requesting Block Bodies "); if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { let block_bodies_len = block_bodies.len(); - info!("Received {} Block Bodies", block_bodies_len); + debug!("Received {} Block Bodies", block_bodies_len); // Execute and store blocks for body in block_bodies.into_iter() { // We already validated that there are no more block bodies than the ones requested @@ -128,13 +126,12 @@ async fn download_and_run_blocks( store.set_canonical_block(number, hash)?; store.update_latest_block_number(number)?; } - info!("Executed & stored {} blocks", block_bodies_len); + debug!("Executed & stored {} blocks", block_bodies_len); // Check if we need to ask for another batch if block_hashes.is_empty() { break; } } - info!("Peer response timeout(Blocks)"); } Ok(()) } From 3191dddc4c567ddac7f94987e00cafb5c2a84726 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 16:23:49 -0300 Subject: [PATCH 242/345] fmt --- crates/networking/p2p/sync.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 5f2589b01..551967237 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -7,7 +7,7 @@ use ethrex_core::{ }; use ethrex_storage::Store; use tokio::{sync::Mutex, time::Instant}; -use tracing::{info, debug, warn}; +use tracing::{debug, info, warn}; use crate::kademlia::KademliaTable; From 576ee33b7b31018b9dc0c6318a5ab14ef795e570 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 16:32:18 -0300 Subject: [PATCH 243/345] Fix --- cmd/ethrex/ethrex.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index e9c7d8929..926df4902 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -116,7 +116,7 @@ async fn main() { .map_or(set_datadir(DEFAULT_DATADIR), |datadir| set_datadir(datadir)); let snap_sync = is_snap_sync(&matches); - if !snap_sync { + if snap_sync { info!("snap-sync not available, defaulting to full-sync"); } From a069f8096c6e40b584bf1ad595951b8a51c20c17 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 16:34:51 -0300 Subject: [PATCH 244/345] Revert "Revert "[REVERT ME] Make V2 endpoints work in order to run hive sync test"" This reverts commit c6d6767f341e76f8a7db9f1523d815f3c8b1f82e. --- crates/networking/p2p/peer_channels.rs | 2 +- crates/networking/rpc/engine/payload.rs | 20 +++++++++----------- crates/networking/rpc/rpc.rs | 2 ++ crates/networking/rpc/types/payload.rs | 4 ++-- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 47858de15..d5cd2a0c5 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -26,7 +26,7 @@ pub struct PeerChannels { impl PeerChannels { /// Sets up the communication channels for the peer /// Returns the channel endpoints to send to the active connection's listen loop - pub(crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + pub (crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); let (connection_sender, receiver) = diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index 9d7eb0089..abd59941f 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -43,16 +43,14 @@ impl RpcHandler for NewPayloadV3Request { let params = params .as_ref() .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - if params.len() != 3 { - return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - } + // if params.len() != 3 { + // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); + // } Ok(NewPayloadV3Request { payload: serde_json::from_value(params[0].clone()) .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: serde_json::from_value(params[1].clone()) - .map_err(|_| RpcErr::WrongParam("expected_blob_versioned_hashes".to_string()))?, - parent_beacon_block_root: serde_json::from_value(params[2].clone()) - .map_err(|_| RpcErr::WrongParam("parent_beacon_block_root".to_string()))?, + expected_blob_versioned_hashes: vec![], + parent_beacon_block_root: H256::zero(), }) } @@ -79,10 +77,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post Cancun fork let chain_config = storage.get_chain_config()?; - let current_fork = chain_config.get_fork(block.header.timestamp); - if current_fork < Fork::Cancun { - return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - } + // let current_fork = chain_config.get_fork(block.header.timestamp); + // if current_fork < Fork::Cancun { + // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); + // } // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index cc7ed583c..8ec54a7ca 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -254,7 +254,9 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), + "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), + "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index b35825039..ee9d873eb 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,9 +35,9 @@ pub struct ExecutionPayloadV3 { pub block_hash: H256, transactions: Vec, withdrawals: Vec, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] excess_blob_gas: u64, } From 2ac1135a68b4a873a22d5806f1b86f501bd466b3 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Wed, 27 Nov 2024 17:05:35 -0300 Subject: [PATCH 245/345] feat: update trait with receipts function --- crates/networking/p2p/rlpx/connection.rs | 8 +++++++ crates/networking/p2p/rlpx/eth/receipts.rs | 4 ++-- crates/storage/store/engines/api.rs | 2 ++ crates/storage/store/engines/in_memory.rs | 5 ++++ crates/storage/store/engines/libmdbx.rs | 28 ++++++++++++++++++++++ crates/storage/store/storage.rs | 4 ++++ 6 files changed, 49 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index bb4815b62..3d0550d7d 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -21,6 +21,7 @@ use crate::{ use super::{ error::RLPxError, + eth::receipts::GetReceipts, frame, handshake::{decode_ack_message, decode_auth_message, encode_auth_message}, message as rlpx, @@ -349,6 +350,13 @@ impl RLPxConnection { }; self.send(Message::BlockBodies(response)).await?; } + Message::GetReceipts(GetReceipts { id, block_hashes }) if peer_supports_eth => { + // FIXME: Implement this + todo!() + // block_hashes + // .into_iter() + // .map(|hash| ) + } Message::GetStorageRanges(req) => { let response = process_storage_ranges_request(req, self.storage.clone())?; self.send(Message::StorageRanges(response)).await? diff --git a/crates/networking/p2p/rlpx/eth/receipts.rs b/crates/networking/p2p/rlpx/eth/receipts.rs index 0ec74af35..495c2eb9c 100644 --- a/crates/networking/p2p/rlpx/eth/receipts.rs +++ b/crates/networking/p2p/rlpx/eth/receipts.rs @@ -14,8 +14,8 @@ use ethrex_rlp::{ pub(crate) struct GetReceipts { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages - id: u64, - block_hashes: Vec, + pub id: u64, + pub block_hashes: Vec, } impl GetReceipts { diff --git a/crates/storage/store/engines/api.rs b/crates/storage/store/engines/api.rs index cca2e7333..1f1d0ee3f 100644 --- a/crates/storage/store/engines/api.rs +++ b/crates/storage/store/engines/api.rs @@ -217,4 +217,6 @@ pub trait StoreEngine: Debug + Send + Sync + RefUnwindSafe { fn add_payload(&self, payload_id: u64, block: Block) -> Result<(), StoreError>; fn get_payload(&self, payload_id: u64) -> Result, StoreError>; + + fn get_receipts_for_block(&self, block_hash: BlockHash) -> Result, StoreError>; } diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 4d4ed4dbb..2cb9f341f 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -348,6 +348,11 @@ impl StoreEngine for Store { fn get_payload(&self, payload_id: u64) -> Result, StoreError> { Ok(self.inner().payloads.get(&payload_id).cloned()) } + + // FIXME: Implement this + fn get_receipts_for_block(&self, block_hash: BlockHash) -> Result, StoreError> { + todo!() + } } impl Debug for Store { diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 52b28e803..76a0d3b58 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -420,6 +420,34 @@ impl StoreEngine for Store { .read::(block_hash.into())? .map(|b| b.to())) } + + // FIXME: Comment this + fn get_receipts_for_block( + &self, + block_hash: BlockHash, + // FIXME: Alias this type + ) -> std::result::Result, StoreError> { + let mut receipts = vec![]; + let mut receipt_index = 0; + let mut key: TupleRLP = (block_hash, 0).into(); + // FIXME: Remove unwrap + let txn = self.db.begin_read().unwrap(); + // FIXME: Remove unwrap + let mut cursor = txn.cursor::().unwrap(); + + // FIXME: Remove unwrap + while let Some((_, encoded_receipt)) = cursor.seek_exact(key).unwrap() { + receipt_index += 1; + //FIXME: Comment this idea a bit more + key = (block_hash, receipt_index).into(); + receipts.push(encoded_receipt); + } + + Ok(receipts + .into_iter() + .map(|receipt| receipt.to()) + .collect()) + } } impl Debug for Store { diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index c500093e4..0b71b5d6a 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -890,6 +890,10 @@ impl Store { pub fn new_state_trie_for_test(&self) -> Trie { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } + + pub fn get_receipts_for_block_range(&self) -> Result, StoreError> { + todo!() + } } pub fn hash_address(address: &Address) -> Vec { From 90f9ca57ec672cf0a86a73058787df9be562787e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 27 Nov 2024 12:01:54 -0300 Subject: [PATCH 246/345] Snap Sync: Account Range --- cmd/ethrex/ethrex.rs | 3 - crates/networking/p2p/peer_channels.rs | 82 ++++++++++-- crates/networking/p2p/snap.rs | 7 +- crates/networking/p2p/sync.rs | 168 ++++++++++++++++++++++--- crates/storage/store/storage.rs | 8 +- 5 files changed, 237 insertions(+), 31 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 926df4902..168b8a6cd 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -116,9 +116,6 @@ async fn main() { .map_or(set_datadir(DEFAULT_DATADIR), |datadir| set_datadir(datadir)); let snap_sync = is_snap_sync(&matches); - if snap_sync { - info!("snap-sync not available, defaulting to full-sync"); - } let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index d5cd2a0c5..091065cca 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -1,20 +1,28 @@ use std::{sync::Arc, time::Duration}; use ethrex_core::{ - types::{BlockBody, BlockHeader}, + types::{AccountState, BlockBody, BlockHeader}, H256, }; +use ethrex_rlp::encode::RLPEncode; +use ethrex_trie::verify_range; use tokio::sync::{mpsc, Mutex}; use crate::{ - rlpx::eth::blocks::{ - BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, + rlpx::{ + eth::blocks::{ + BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, + }, + snap::{AccountRange, GetAccountRange}, }, + snap::encodable_to_proof, RLPxMessage, }; pub const PEER_REPLY_TIMOUT: Duration = Duration::from_secs(45); pub const MAX_MESSAGES_IN_PEER_CHANNEL: usize = 25; +pub const MAX_RESPONSE_BYTES: u64 = 500; // TODO: Set +pub const HASH_MAX: H256 = H256([0xFF; 32]); #[derive(Debug, Clone)] /// Holds the respective sender and receiver ends of the communication channels bewteen the peer data and its active connection @@ -26,7 +34,7 @@ pub struct PeerChannels { impl PeerChannels { /// Sets up the communication channels for the peer /// Returns the channel endpoints to send to the active connection's listen loop - pub (crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { + pub(crate) fn create() -> (Self, mpsc::Sender, mpsc::Receiver) { let (sender, connection_receiver) = mpsc::channel::(MAX_MESSAGES_IN_PEER_CHANNEL); let (connection_sender, receiver) = @@ -41,8 +49,8 @@ impl PeerChannels { ) } - /// Requests block headers from the peer - /// Returns the response message or None if: + /// Requests block headers from the peer, starting from the `start` block hash towards newer blocks + /// Returns the block headers or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) /// - The response timed out /// - The response was empty or not valid @@ -76,8 +84,8 @@ impl PeerChannels { (!block_headers.is_empty()).then_some(block_headers) } - /// Requests block headers from the peer - /// Returns the response message or None if: + /// Requests block headers from the peer given their block hashes + /// Returns the block headers or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) /// - The response timed out /// - The response was empty or not valid @@ -109,4 +117,62 @@ impl PeerChannels { // Check that the response is not empty and does not contain more bodies than the ones requested (!block_bodies.is_empty() && block_bodies.len() <= block_hashes_len).then_some(block_bodies) } + + /// Requests an account range from the peer given the state trie's root and the starting hash (the limit hash will be the maximum value of H256) + /// Will also return a boolean indicating if there is more state to be fetched towards the right of the trie + /// Returns the response message or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_account_range( + &self, + state_root: H256, + start: H256, + ) -> Option<(Vec, Vec, bool)> { + let request_id = rand::random(); + let request = RLPxMessage::GetAccountRange(GetAccountRange { + id: request_id, + root_hash: state_root, + starting_hash: start, + limit_hash: HASH_MAX, + response_bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let (accounts, proof) = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::AccountRange(AccountRange { + id, + accounts, + proof, + })) if id == request_id => return Some((accounts, proof)), + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + // Unzip & validate response + let proof = encodable_to_proof(&proof); + let (account_hashes, accounts): (Vec<_>, Vec<_>) = accounts + .into_iter() + .map(|unit| (unit.hash, AccountState::from(unit.account))) + .unzip(); + let encoded_accounts = accounts + .iter() + .map(|acc| acc.encode_to_vec()) + .collect::>(); + let should_continue = verify_range( + state_root, + &start, + &account_hashes, + &encoded_accounts, + &proof, + ) + .ok()?; + Some((account_hashes, accounts, should_continue)) + } } diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index bf9ca53e8..1b98345f3 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -151,14 +151,13 @@ pub fn process_trie_nodes_request( // Helper method to convert proof to RLP-encodable format #[inline] -fn proof_to_encodable(proof: Vec>) -> Vec { +pub(crate) fn proof_to_encodable(proof: Vec>) -> Vec { proof.into_iter().map(Bytes::from).collect() } // Helper method to obtain proof from RLP-encodable format #[inline] -#[allow(unused)] -fn encodable_to_proof(proof: &[Bytes]) -> Vec> { +pub(crate) fn encodable_to_proof(proof: &[Bytes]) -> Vec> { proof.iter().map(|bytes| bytes.to_vec()).collect() } @@ -973,7 +972,7 @@ mod tests { // Create a store and load it up with the accounts let store = Store::new("null", EngineType::InMemory).unwrap(); - let mut state_trie = store.new_state_trie_for_test(); + let mut state_trie = store.new_state_trie(); for (address, account) in accounts { let hashed_address = H256::from_str(address).unwrap().as_bytes().to_vec(); let account = AccountState::from(AccountStateSlim::decode(&account).unwrap()); diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 551967237..4b3b1148b 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -5,7 +5,9 @@ use ethrex_core::{ types::{Block, BlockHash, BlockHeader}, H256, }; +use ethrex_rlp::encode::RLPEncode; use ethrex_storage::Store; +use ethrex_trie::EMPTY_TRIE_HASH; use tokio::{sync::Mutex, time::Instant}; use tracing::{debug, info, warn}; @@ -62,20 +64,68 @@ impl SyncManager { } } // We finished fetching all headers, now we can process them - // TODO: snap-sync: launch tasks to fetch blocks and state in parallel - // full-sync: Fetch all block bodies and execute them sequentially to build the state - match tokio::spawn(download_and_run_blocks( - all_block_hashes, - all_block_headers, - self.peers.clone(), - store.clone(), - )) - .await - { - Ok(Ok(())) => info!( - "Sync finished, time elapsed: {} secs", - start_time.elapsed().as_secs() - ), + let result = if self.snap_mode { + // snap-sync: launch tasks to fetch blocks and state in parallel + // - Fetch each block's state via snap p2p requests + // - Fetch each blocks and its receipts via eth p2p requests + let fetch_blocks_and_receipts_handle = tokio::spawn(fetch_blocks_and_receipts( + all_block_hashes.clone(), + self.peers.clone(), + store.clone(), + )); + let state_roots = all_block_headers + .iter() + .map(|header| header.state_root) + .collect::>(); + let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( + state_roots.clone(), + self.peers.clone(), + store.clone(), + )); + // Store headers + let mut latest_block_number = 0; + for (header, hash) in all_block_headers + .into_iter() + .zip(all_block_hashes.into_iter()) + { + // TODO: Handle error + latest_block_number = header.number; + store.set_canonical_block(header.number, hash).unwrap(); + store.add_block_header(hash, header).unwrap(); + } + // TODO: Handle error + let result = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); + // Set latest block number here to avoid reading state that is currently being synced + store + .update_latest_block_number(latest_block_number) + .unwrap(); + // Collapse into one error, if both processes failed then they are likely to have a common cause (such as storage errors) + match result { + (error @ Err(_), _) + | (_, error @ Err(_)) + | (error @ Ok(Err(_)), _) + | (_, error @ Ok(Err(_))) => error, + _ => Ok(Ok(())), + } + } else { + // full-sync: Fetch all block bodies and execute them sequentially to build the state + tokio::spawn(download_and_run_blocks( + all_block_hashes, + all_block_headers, + self.peers.clone(), + store.clone(), + )) + .await + }; + match result { + Ok(Ok(())) => { + info!( + "Sync finished, time elapsed: {} secs", + start_time.elapsed().as_secs() + ); + // Next sync will be full-sync + self.snap_mode = false; + } Ok(Err(error)) => warn!( "Sync failed due to {error}, time elapsed: {} secs ", start_time.elapsed().as_secs() @@ -135,3 +185,93 @@ async fn download_and_run_blocks( } Ok(()) } + +async fn fetch_blocks_and_receipts( + mut block_hashes: Vec, + peers: Arc>, + store: Store, +) -> Result<(), ChainError> { + // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts + // Fetch Block Bodies + loop { + let peer = peers.lock().await.get_peer_channels().await; + info!("[Sync] Requesting Block Headers "); + if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { + info!("[SYNC] Received {} Block Bodies", block_bodies.len()); + // Track which bodies we have already fetched + let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); + // Store Block Bodies + for (hash, body) in fetched_hashes.into_iter().zip(block_bodies.into_iter()) { + // TODO: handle error + store.add_block_body(hash.clone(), body).unwrap() + } + + // Check if we need to ask for another batch + if remaining_hashes.is_empty() { + break; + } else { + block_hashes = remaining_hashes.to_vec(); + } + } + info!("[Sync] Peer response timeout( Blocks & Receipts)"); + } + // TODO: Fetch Receipts and store them + Ok(()) +} + +async fn fetch_snap_state( + state_roots: Vec, + peers: Arc>, + store: Store, +) -> Result<(), ChainError> { + for state_root in state_roots { + fetch_snap_state_inner(state_root, peers.clone(), store.clone()).await? + } + Ok(()) +} + +/// Rebuilds a Block's account state by requesting state from peers +async fn fetch_snap_state_inner( + state_root: H256, + peers: Arc>, + store: Store, +) -> Result<(), ChainError> { + let mut start_account_hash = H256::zero(); + // Start from an empty state trie + // We cannot keep an open trie here so we will track the root between lookups + let mut current_state_root = *EMPTY_TRIE_HASH; + // Fetch Account Ranges + loop { + let peer = peers.lock().await.get_peer_channels().await; + info!("[Sync] Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); + if let Some((account_hashes, accounts, should_continue)) = peer + .request_account_range(state_root, start_account_hash) + .await + { + // Update starting hash for next batch + if should_continue { + start_account_hash = *account_hashes.last().unwrap(); + } + + // Update trie + let mut trie = store.open_state_trie(current_state_root); + for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { + // TODO: Handle + trie.insert(account_hash.0.to_vec(), account.encode_to_vec()) + .unwrap(); + } + // TODO: Handle + current_state_root = trie.hash().unwrap(); + + if !should_continue { + // All accounts fetched! + break; + } + } + } + if current_state_root != state_root { + info!("[Sync] State sync failed for hash {state_root}"); + } + info!("[Sync] Completed state sync for hash {state_root}"); + Ok(()) +} diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index c500093e4..d3903addd 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -886,10 +886,14 @@ impl Store { self.engine.get_payload(payload_id) } - /// Creates a new state trie with an empty state root, for testing purposes only - pub fn new_state_trie_for_test(&self) -> Trie { + /// Creates a new clean state trie (with an empty root) + pub fn new_state_trie(&self) -> Trie { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } + + pub fn open_state_trie(&self, state_root: H256) -> Trie { + self.engine.open_state_trie(state_root) + } } pub fn hash_address(address: &Address) -> Vec { From bfcff0db6d75d9c6a80b04e1e557f53f660836cb Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Wed, 27 Nov 2024 17:40:18 -0300 Subject: [PATCH 247/345] feat: implement receipts_for_block for inmem --- crates/storage/store/engines/api.rs | 2 +- crates/storage/store/engines/in_memory.rs | 18 ++++++++++++++++-- crates/storage/store/engines/libmdbx.rs | 11 ++++------- 3 files changed, 21 insertions(+), 10 deletions(-) diff --git a/crates/storage/store/engines/api.rs b/crates/storage/store/engines/api.rs index 1f1d0ee3f..d6d92023b 100644 --- a/crates/storage/store/engines/api.rs +++ b/crates/storage/store/engines/api.rs @@ -218,5 +218,5 @@ pub trait StoreEngine: Debug + Send + Sync + RefUnwindSafe { fn get_payload(&self, payload_id: u64) -> Result, StoreError>; - fn get_receipts_for_block(&self, block_hash: BlockHash) -> Result, StoreError>; + fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError>; } diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 2cb9f341f..edc06e03f 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -350,8 +350,22 @@ impl StoreEngine for Store { } // FIXME: Implement this - fn get_receipts_for_block(&self, block_hash: BlockHash) -> Result, StoreError> { - todo!() + fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError> { + // FIXME: Remove this unwrap + let store = self.inner(); + let mut receipts = store + .receipts + .get(&block_hash) + .unwrap() + .iter() + .collect::>(); + + receipts.sort_by_key(|(index, _receipt)| **index); + + Ok(receipts + .into_iter() + .map(|(_index, receipt)| receipt.clone()) + .collect()) } } diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 76a0d3b58..2e255fd85 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -424,12 +424,12 @@ impl StoreEngine for Store { // FIXME: Comment this fn get_receipts_for_block( &self, - block_hash: BlockHash, + block_hash: &BlockHash, // FIXME: Alias this type ) -> std::result::Result, StoreError> { let mut receipts = vec![]; let mut receipt_index = 0; - let mut key: TupleRLP = (block_hash, 0).into(); + let mut key: TupleRLP = (*block_hash, 0).into(); // FIXME: Remove unwrap let txn = self.db.begin_read().unwrap(); // FIXME: Remove unwrap @@ -439,14 +439,11 @@ impl StoreEngine for Store { while let Some((_, encoded_receipt)) = cursor.seek_exact(key).unwrap() { receipt_index += 1; //FIXME: Comment this idea a bit more - key = (block_hash, receipt_index).into(); + key = (*block_hash, receipt_index).into(); receipts.push(encoded_receipt); } - Ok(receipts - .into_iter() - .map(|receipt| receipt.to()) - .collect()) + Ok(receipts.into_iter().map(|receipt| receipt.to()).collect()) } } From e94287696c824d618386678ebc91691512e875ad Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Wed, 27 Nov 2024 18:03:55 -0300 Subject: [PATCH 248/345] feat: send response msg --- crates/networking/p2p/rlpx/connection.rs | 13 ++++++++----- crates/networking/p2p/rlpx/eth/receipts.rs | 5 +++-- crates/networking/p2p/rlpx/message.rs | 8 +++++++- crates/storage/store/storage.rs | 4 ++-- 4 files changed, 20 insertions(+), 10 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 3d0550d7d..28075447e 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -5,6 +5,7 @@ use crate::{ eth::{ backend, blocks::{BlockBodies, BlockHeaders}, + receipts::Receipts, transactions::Transactions, }, handshake::encode_ack_message, @@ -351,11 +352,13 @@ impl RLPxConnection { self.send(Message::BlockBodies(response)).await?; } Message::GetReceipts(GetReceipts { id, block_hashes }) if peer_supports_eth => { - // FIXME: Implement this - todo!() - // block_hashes - // .into_iter() - // .map(|hash| ) + // FIXME: Remove unwrap + let receipts = block_hashes + .iter() + .map(|hash| self.storage.get_receipts_for_block(hash).unwrap()) + .collect(); + let response = Receipts { id, receipts }; + self.send(Message::Receipts(response)).await?; } Message::GetStorageRanges(req) => { let response = process_storage_ranges_request(req, self.storage.clone())?; diff --git a/crates/networking/p2p/rlpx/eth/receipts.rs b/crates/networking/p2p/rlpx/eth/receipts.rs index 495c2eb9c..f3e3b1150 100644 --- a/crates/networking/p2p/rlpx/eth/receipts.rs +++ b/crates/networking/p2p/rlpx/eth/receipts.rs @@ -48,11 +48,12 @@ impl RLPxMessage for GetReceipts { } // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#receipts-0x10 +#[derive(Debug)] pub(crate) struct Receipts { // id is a u64 chosen by the requesting peer, the responding peer must mirror the value for the response // https://github.com/ethereum/devp2p/blob/master/caps/eth.md#protocol-messages - id: u64, - receipts: Vec>, + pub id: u64, + pub receipts: Vec>, } impl Receipts { diff --git a/crates/networking/p2p/rlpx/message.rs b/crates/networking/p2p/rlpx/message.rs index 412e55a33..94360e0b8 100644 --- a/crates/networking/p2p/rlpx/message.rs +++ b/crates/networking/p2p/rlpx/message.rs @@ -3,7 +3,7 @@ use ethrex_rlp::error::{RLPDecodeError, RLPEncodeError}; use std::fmt::Display; use super::eth::blocks::{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders}; -use super::eth::receipts::GetReceipts; +use super::eth::receipts::{GetReceipts, Receipts}; use super::eth::status::StatusMessage; use super::eth::transactions::Transactions; use super::p2p::{DisconnectMessage, HelloMessage, PingMessage, PongMessage}; @@ -33,6 +33,7 @@ pub(crate) enum Message { GetBlockBodies(GetBlockBodies), BlockBodies(BlockBodies), GetReceipts(GetReceipts), + Receipts(Receipts), // snap capability GetAccountRange(GetAccountRange), AccountRange(AccountRange), @@ -127,6 +128,10 @@ impl Message { 0x1F_u8.encode(buf); msg.encode(buf) } + Message::Receipts(msg) => { + 0x20_u8.encode(buf); + msg.encode(buf) + } Message::GetAccountRange(msg) => { 0x21_u8.encode(buf); msg.encode(buf) @@ -175,6 +180,7 @@ impl Display for Message { Message::BlockHeaders(_) => "eth:BlockHeaders".fmt(f), Message::BlockBodies(_) => "eth:BlockBodies".fmt(f), Message::GetReceipts(_) => "eth:GetReceipts".fmt(f), + Message::Receipts(_) => "eth:Receipts".fmt(f), Message::Transactions(_) => "eth:TransactionsMessage".fmt(f), Message::GetBlockBodies(_) => "eth:GetBlockBodies".fmt(f), Message::GetAccountRange(_) => "snap:GetAccountRange".fmt(f), diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 0b71b5d6a..4d7b5dfa4 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -891,8 +891,8 @@ impl Store { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } - pub fn get_receipts_for_block_range(&self) -> Result, StoreError> { - todo!() + pub fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError> { + self.engine.get_receipts_for_block(block_hash) } } From b00ac7001034b576716212e1d0099caec9312376 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 28 Nov 2024 12:27:04 -0300 Subject: [PATCH 249/345] Update tracing + set correct bytes response limit --- crates/networking/p2p/peer_channels.rs | 2 +- crates/networking/p2p/sync.rs | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 091065cca..b4f9736d3 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -21,7 +21,7 @@ use crate::{ pub const PEER_REPLY_TIMOUT: Duration = Duration::from_secs(45); pub const MAX_MESSAGES_IN_PEER_CHANNEL: usize = 25; -pub const MAX_RESPONSE_BYTES: u64 = 500; // TODO: Set +pub const MAX_RESPONSE_BYTES: u64 = 512 * 1024; pub const HASH_MAX: H256 = H256([0xFF; 32]); #[derive(Debug, Clone)] diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 4b3b1148b..c4b946341 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -195,9 +195,9 @@ async fn fetch_blocks_and_receipts( // Fetch Block Bodies loop { let peer = peers.lock().await.get_peer_channels().await; - info!("[Sync] Requesting Block Headers "); + debug!("Requesting Block Headers "); if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { - info!("[SYNC] Received {} Block Bodies", block_bodies.len()); + debug!(" Received {} Block Bodies", block_bodies.len()); // Track which bodies we have already fetched let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); // Store Block Bodies @@ -213,7 +213,6 @@ async fn fetch_blocks_and_receipts( block_hashes = remaining_hashes.to_vec(); } } - info!("[Sync] Peer response timeout( Blocks & Receipts)"); } // TODO: Fetch Receipts and store them Ok(()) @@ -224,6 +223,7 @@ async fn fetch_snap_state( peers: Arc>, store: Store, ) -> Result<(), ChainError> { + info!("Syncing state roots: {}", state_roots.len()); for state_root in state_roots { fetch_snap_state_inner(state_root, peers.clone(), store.clone()).await? } @@ -243,7 +243,7 @@ async fn fetch_snap_state_inner( // Fetch Account Ranges loop { let peer = peers.lock().await.get_peer_channels().await; - info!("[Sync] Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); + debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer .request_account_range(state_root, start_account_hash) .await @@ -270,8 +270,8 @@ async fn fetch_snap_state_inner( } } if current_state_root != state_root { - info!("[Sync] State sync failed for hash {state_root}"); + warn!("[Sync] State sync failed for hash {state_root}"); } - info!("[Sync] Completed state sync for hash {state_root}"); + debug!("[Sync] Completed state sync for hash {state_root}"); Ok(()) } From 024efc93cf6a3349f7224bf59100a32f804b440b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 28 Nov 2024 16:03:21 -0300 Subject: [PATCH 250/345] Fix decoding of GetStorageRange --- crates/networking/p2p/rlpx/snap.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 2c7643790..70eb36a37 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -169,8 +169,14 @@ impl RLPxMessage for GetStorageRanges { let (id, decoder) = decoder.decode_field("request-id")?; let (root_hash, decoder) = decoder.decode_field("rootHash")?; let (account_hashes, decoder) = decoder.decode_field("accountHashes")?; - let (starting_hash, decoder) = decoder.decode_field("startingHash")?; - let (limit_hash, decoder) = decoder.decode_field("limitHash")?; + let (starting_hash, decoder): (Bytes, _) = decoder.decode_field("startingHash")?; + let starting_hash = (!starting_hash.is_empty()) + .then(|| H256::from_slice(&starting_hash)) + .unwrap_or_default(); + let (limit_hash, decoder): (Bytes, _) = decoder.decode_field("limitHash")?; + let limit_hash = (!limit_hash.is_empty()) + .then(|| H256::from_slice(&limit_hash)) + .unwrap_or(H256([0xFF; 32])); let (response_bytes, decoder) = decoder.decode_field("responseBytes")?; decoder.finish()?; From 7821db9c49ceff192fb31a368537e910ca4cd049 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 29 Nov 2024 15:55:10 -0300 Subject: [PATCH 251/345] First steps of bytecode fetch queue + improve task spawning and joining using newer api --- crates/networking/p2p/peer_channels.rs | 95 +++++++++++- crates/networking/p2p/sync.rs | 196 ++++++++++++++++++++----- crates/storage/store/storage.rs | 6 +- 3 files changed, 256 insertions(+), 41 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index b4f9736d3..ef8bfb833 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -1,8 +1,9 @@ use std::{sync::Arc, time::Duration}; +use bytes::Bytes; use ethrex_core::{ types::{AccountState, BlockBody, BlockHeader}, - H256, + H256, U256, }; use ethrex_rlp::encode::RLPEncode; use ethrex_trie::verify_range; @@ -13,7 +14,9 @@ use crate::{ eth::blocks::{ BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, }, - snap::{AccountRange, GetAccountRange}, + snap::{ + AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges, + }, }, snap::encodable_to_proof, RLPxMessage, @@ -85,7 +88,7 @@ impl PeerChannels { } /// Requests block headers from the peer given their block hashes - /// Returns the block headers or None if: + /// Returns the block bodies or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) /// - The response timed out /// - The response was empty or not valid @@ -175,4 +178,90 @@ impl PeerChannels { .ok()?; Some((account_hashes, accounts, should_continue)) } + + // TODO: Inefficient method -> replace with request_storage_ranges + pub async fn request_storage_range( + &self, + storage_root: H256, + account_hash: H256, + start: H256, + ) -> Option<(Vec, Vec, bool)> { + let request_id = rand::random(); + let request = RLPxMessage::GetStorageRanges(GetStorageRanges { + id: request_id, + root_hash: storage_root, + account_hashes: vec![account_hash], + starting_hash: start, + limit_hash: HASH_MAX, + response_bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let (mut slots, proof) = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::StorageRanges(StorageRanges { id, slots, proof })) + if id == request_id => + { + return Some((slots, proof)) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + // We only requested 1 account so lets make sure we got it: + if slots.len() != 1 { + return None; + } + // Unzip & validate response + let proof = encodable_to_proof(&proof); + let (hahsed_keys, values): (Vec<_>, Vec<_>) = slots + .remove(0) + .into_iter() + .map(|slot| (slot.hash, slot.data)) + .unzip(); + let encoded_values = values + .iter() + .map(|val| val.encode_to_vec()) + .collect::>(); + let should_continue = + verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &proof).ok()?; + Some((hahsed_keys, values, should_continue)) + } + + /// Requests bytecodes for the given code hashes + /// Returns the bytecodes or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_bytecodes(&self, hashes: Vec) -> Option> { + let request_id = rand::random(); + let hashes_len = hashes.len(); + let request = RLPxMessage::GetByteCodes(GetByteCodes { + id: request_id, + hashes, + bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let codes = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::ByteCodes(ByteCodes { id, codes })) if id == request_id => { + return Some(codes) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + (!codes.is_empty() && codes.len() <= hashes_len).then_some(codes) + } } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index c4b946341..6020067f9 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -6,9 +6,15 @@ use ethrex_core::{ H256, }; use ethrex_rlp::encode::RLPEncode; -use ethrex_storage::Store; +use ethrex_storage::{error::StoreError, Store}; use ethrex_trie::EMPTY_TRIE_HASH; -use tokio::{sync::Mutex, time::Instant}; +use tokio::{ + sync::{ + mpsc::{self, Receiver, Sender}, + Mutex, + }, + time::Instant, +}; use tracing::{debug, info, warn}; use crate::kademlia::KademliaTable; @@ -68,7 +74,16 @@ impl SyncManager { // snap-sync: launch tasks to fetch blocks and state in parallel // - Fetch each block's state via snap p2p requests // - Fetch each blocks and its receipts via eth p2p requests - let fetch_blocks_and_receipts_handle = tokio::spawn(fetch_blocks_and_receipts( + // TODO: We are currently testing against our implementation that doesn't hold an independant snapchot and can provide all historic state + // We should fetch all available state and then resort to state healing to fetch the rest + let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); + let mut set = tokio::task::JoinSet::new(); + set.spawn(bytecode_fetcher( + bytecode_receiver, + self.peers.clone(), + store.clone(), + )); + set.spawn(fetch_blocks_and_receipts( all_block_hashes.clone(), self.peers.clone(), store.clone(), @@ -77,7 +92,7 @@ impl SyncManager { .iter() .map(|header| header.state_root) .collect::>(); - let fetch_snap_state_handle = tokio::spawn(fetch_snap_state( + set.spawn(fetch_snap_state( state_roots.clone(), self.peers.clone(), store.clone(), @@ -93,32 +108,29 @@ impl SyncManager { store.set_canonical_block(header.number, hash).unwrap(); store.add_block_header(hash, header).unwrap(); } - // TODO: Handle error - let result = tokio::join!(fetch_blocks_and_receipts_handle, fetch_snap_state_handle); + let result = set.join_all().await; // Set latest block number here to avoid reading state that is currently being synced store .update_latest_block_number(latest_block_number) .unwrap(); - // Collapse into one error, if both processes failed then they are likely to have a common cause (such as storage errors) - match result { - (error @ Err(_), _) - | (_, error @ Err(_)) - | (error @ Ok(Err(_)), _) - | (_, error @ Ok(Err(_))) => error, - _ => Ok(Ok(())), - } + // Collapse into one error, if all processes failed then they are likely to have a common cause (such as unaccessible storage) + result + .into_iter() + .find(|res| res.is_err()) + .unwrap_or(Ok(())) + .map_err(ChainError::StoreError) } else { // full-sync: Fetch all block bodies and execute them sequentially to build the state - tokio::spawn(download_and_run_blocks( + download_and_run_blocks( all_block_hashes, all_block_headers, self.peers.clone(), store.clone(), - )) + ) .await }; match result { - Ok(Ok(())) => { + Ok(()) => { info!( "Sync finished, time elapsed: {} secs", start_time.elapsed().as_secs() @@ -126,14 +138,10 @@ impl SyncManager { // Next sync will be full-sync self.snap_mode = false; } - Ok(Err(error)) => warn!( + Err(error) => warn!( "Sync failed due to {error}, time elapsed: {} secs ", start_time.elapsed().as_secs() ), - _ => warn!( - "Sync failed due to internal error, time elapsed: {} secs", - start_time.elapsed().as_secs() - ), } } @@ -190,7 +198,7 @@ async fn fetch_blocks_and_receipts( mut block_hashes: Vec, peers: Arc>, store: Store, -) -> Result<(), ChainError> { +) -> Result<(), StoreError> { // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts // Fetch Block Bodies loop { @@ -222,27 +230,28 @@ async fn fetch_snap_state( state_roots: Vec, peers: Arc>, store: Store, -) -> Result<(), ChainError> { +) -> Result<(), StoreError> { info!("Syncing state roots: {}", state_roots.len()); - for state_root in state_roots { - fetch_snap_state_inner(state_root, peers.clone(), store.clone()).await? + // Fetch newer state first: This will be useful to detect where to switch to healing + for state_root in state_roots.into_iter().rev() { + rebuild_state_trie(state_root, peers.clone(), store.clone()).await? } Ok(()) } -/// Rebuilds a Block's account state by requesting state from peers -async fn fetch_snap_state_inner( +/// Rebuilds a Block's state trie by requesting snap state from peers +async fn rebuild_state_trie( state_root: H256, peers: Arc>, store: Store, -) -> Result<(), ChainError> { +) -> Result<(), StoreError> { let mut start_account_hash = H256::zero(); // Start from an empty state trie // We cannot keep an open trie here so we will track the root between lookups let mut current_state_root = *EMPTY_TRIE_HASH; // Fetch Account Ranges loop { - let peer = peers.lock().await.get_peer_channels().await; + let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer .request_account_range(state_root, start_account_hash) @@ -252,16 +261,25 @@ async fn fetch_snap_state_inner( if should_continue { start_account_hash = *account_hashes.last().unwrap(); } - + // Fetch Account Storage & Bytecode + for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { + // TODO: spawn tasks to fetch storage & bytecodes: Should we create a list of tasks and then await them at after this loop? + // The first iteration could await them here as we are only testing the one peer case + rebuild_storage_trie( + *account_hash, + account.storage_root, + peers.clone(), + store.clone(), + ) + .await?; + } // Update trie let mut trie = store.open_state_trie(current_state_root); for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { - // TODO: Handle trie.insert(account_hash.0.to_vec(), account.encode_to_vec()) - .unwrap(); + .map_err(StoreError::Trie)?; } - // TODO: Handle - current_state_root = trie.hash().unwrap(); + current_state_root = trie.hash().map_err(StoreError::Trie)?; if !should_continue { // All accounts fetched! @@ -270,8 +288,112 @@ async fn fetch_snap_state_inner( } } if current_state_root != state_root { - warn!("[Sync] State sync failed for hash {state_root}"); + warn!("State sync failed for state root {state_root}"); + } + info!("Completed state sync for state root {state_root}"); + Ok(()) +} + +/// Rebuilds an Account's storage trie by requesting snap state from peers +/// First Iteration: Fetch one account storage at a time -> this is inefficient, we should queue them and fetch multiple accounts at once +async fn rebuild_storage_trie( + account_hash: H256, + storage_root: H256, + peers: Arc>, + store: Store, +) -> Result<(), StoreError> { + let mut start_key_hash = H256::zero(); + // Start from an empty state trie + // We cannot keep an open trie here so we will track the root between lookups + let mut current_storage_root = *EMPTY_TRIE_HASH; + // Fetch Storage Ranges + loop { + let peer = peers.lock().await.get_peer_channels().await; + debug!("Requesting Storage Range for storage root {storage_root}, starting hash: {start_key_hash}"); + if let Some((hashed_keys, values, should_continue)) = peer + .request_storage_range(storage_root, account_hash, start_key_hash) + .await + { + // Update starting hash for next batch + if should_continue { + start_key_hash = *hashed_keys.last().unwrap(); + } + + // Update trie + let mut trie = store.open_storage_trie(account_hash, current_storage_root); + for (key_hash, value) in hashed_keys.iter().zip(values.iter()) { + trie.insert(key_hash.0.to_vec(), value.encode_to_vec()) + .map_err(StoreError::Trie)?; + } + current_storage_root = trie.hash().map_err(StoreError::Trie)?; + + if !should_continue { + // All keys fetched! + break; + } + } + } + if current_storage_root != storage_root { + warn!("State sync failed for storage root {storage_root}"); } - debug!("[Sync] Completed state sync for hash {state_root}"); + info!("Completed state sync for storage root {storage_root}"); Ok(()) } + +/// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches +async fn bytecode_fetcher( + mut receiver: Receiver>, + peers: Arc>, + store: Store, +) -> Result<(), StoreError> { + // Pending list of bytecodes to fetch + const BATCH_SIZE: usize = 10; + let mut pending_bytecodes: Vec = vec![]; + loop { + match receiver.recv().await { + Some(code_hashes) if !code_hashes.is_empty() => { + // Add hashes to the queue + pending_bytecodes.extend(code_hashes); + // If we have enought pending bytecodes to fill a batch, spawn a fetch process + if pending_bytecodes.len() >= 5 { + let next_batch = pending_bytecodes.drain(..BATCH_SIZE).collect::>(); + let remaining = + fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; + // Add unfeched bytecodes back to the queue + pending_bytecodes.extend(remaining); + }; + } + // Disconnect / Empty message signaling no more bytecodes to sync + _ => break, + } + } + // We have no more incoming requests, process the remaining batches + while !pending_bytecodes.is_empty() { + let next_batch = pending_bytecodes + .drain(..BATCH_SIZE.min(pending_bytecodes.len())) + .collect::>(); + let remaining = fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; + // Add unfeched bytecodes back to the queue + pending_bytecodes.extend(remaining); + } + Ok(()) +} + +/// Receives a batch of code hahses, fetches their respective bytecodes via p2p and returns a list of the code hashes that couldn't be fetched in the request (if applicable) +async fn fetch_bytecode_batch( + mut batch: Vec, + peers: Arc>, + store: Store, +) -> Result, StoreError> { + loop { + let peer = peers.lock().await.get_peer_channels().await; + if let Some(bytecodes) = peer.request_bytecodes(batch.clone()).await { + // Store the bytecodes + for code in bytecodes.into_iter() { + store.add_account_code(batch.remove(0), code)?; + } + // Return remaining code hashes in the batch if we couldn't fetch all of them + return Ok(batch); + } + } +} diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index d3903addd..ff009320c 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -315,7 +315,7 @@ impl Store { Ok(txs_by_sender) } - fn add_account_code(&self, code_hash: H256, code: Bytes) -> Result<(), StoreError> { + pub fn add_account_code(&self, code_hash: H256, code: Bytes) -> Result<(), StoreError> { self.engine.add_account_code(code_hash, code) } @@ -894,6 +894,10 @@ impl Store { pub fn open_state_trie(&self, state_root: H256) -> Trie { self.engine.open_state_trie(state_root) } + + pub fn open_storage_trie(&self, account_hash: H256, storage_root: H256) -> Trie { + self.engine.open_storage_trie(account_hash, storage_root) + } } pub fn hash_address(address: &Address) -> Vec { From 85ce20230101405bad355718f57e880724d0f6f1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 29 Nov 2024 17:41:43 -0300 Subject: [PATCH 252/345] Connect state rebuilding to bytecode fetcher --- crates/networking/p2p/sync.rs | 35 ++++++++++++++++++++++++++++++----- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 6020067f9..f70e025ed 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use ethrex_blockchain::error::ChainError; use ethrex_core::{ - types::{Block, BlockHash, BlockHeader}, + types::{Block, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, H256, }; use ethrex_rlp::encode::RLPEncode; @@ -93,6 +93,7 @@ impl SyncManager { .map(|header| header.state_root) .collect::>(); set.spawn(fetch_snap_state( + bytecode_sender, state_roots.clone(), self.peers.clone(), store.clone(), @@ -227,6 +228,7 @@ async fn fetch_blocks_and_receipts( } async fn fetch_snap_state( + bytecode_sender: Sender>, state_roots: Vec, peers: Arc>, store: Store, @@ -234,13 +236,24 @@ async fn fetch_snap_state( info!("Syncing state roots: {}", state_roots.len()); // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { - rebuild_state_trie(state_root, peers.clone(), store.clone()).await? + // TODO: maybe spawn taks here instead of awaiting + rebuild_state_trie( + bytecode_sender.clone(), + state_root, + peers.clone(), + store.clone(), + ) + .await? } + // We finished syncing the available state, lets make the fetcher processes aware + // Send empty batches to signal that no more batches are incoming + bytecode_sender.send(vec![]).await.unwrap(); Ok(()) } /// Rebuilds a Block's state trie by requesting snap state from peers async fn rebuild_state_trie( + bytecode_sender: Sender>, state_root: H256, peers: Arc>, store: Store, @@ -262,8 +275,15 @@ async fn rebuild_state_trie( start_account_hash = *account_hashes.last().unwrap(); } // Fetch Account Storage & Bytecode + let mut code_hashes = vec![]; for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { - // TODO: spawn tasks to fetch storage & bytecodes: Should we create a list of tasks and then await them at after this loop? + // Build the batch of code hashes to send to the bytecode fetcher + // Ignoring accounts without code / code we already have stored + if account.code_hash != *EMPTY_KECCACK_HASH + && store.get_account_code(account.code_hash)?.is_none() + { + code_hashes.push(account.code_hash) + } // The first iteration could await them here as we are only testing the one peer case rebuild_storage_trie( *account_hash, @@ -273,6 +293,11 @@ async fn rebuild_state_trie( ) .await?; } + // Send code hash batch to the bytecode fetcher + if !code_hashes.is_empty() { + // TODO: Handle + bytecode_sender.send(code_hashes).await.unwrap() + } // Update trie let mut trie = store.open_state_trie(current_state_root); for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { @@ -355,13 +380,13 @@ async fn bytecode_fetcher( // Add hashes to the queue pending_bytecodes.extend(code_hashes); // If we have enought pending bytecodes to fill a batch, spawn a fetch process - if pending_bytecodes.len() >= 5 { + while pending_bytecodes.len() >= 5 { let next_batch = pending_bytecodes.drain(..BATCH_SIZE).collect::>(); let remaining = fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; // Add unfeched bytecodes back to the queue pending_bytecodes.extend(remaining); - }; + } } // Disconnect / Empty message signaling no more bytecodes to sync _ => break, From 973f2d932e5aa7671a267ace655243a6d3eae0eb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 29 Nov 2024 18:24:24 -0300 Subject: [PATCH 253/345] Fix --- crates/networking/p2p/sync.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index f70e025ed..2a53c3d29 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -372,7 +372,7 @@ async fn bytecode_fetcher( store: Store, ) -> Result<(), StoreError> { // Pending list of bytecodes to fetch - const BATCH_SIZE: usize = 10; + const BATCH_SIZE: usize = 100; let mut pending_bytecodes: Vec = vec![]; loop { match receiver.recv().await { @@ -380,7 +380,7 @@ async fn bytecode_fetcher( // Add hashes to the queue pending_bytecodes.extend(code_hashes); // If we have enought pending bytecodes to fill a batch, spawn a fetch process - while pending_bytecodes.len() >= 5 { + while pending_bytecodes.len() >= BATCH_SIZE { let next_batch = pending_bytecodes.drain(..BATCH_SIZE).collect::>(); let remaining = fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; @@ -413,6 +413,7 @@ async fn fetch_bytecode_batch( loop { let peer = peers.lock().await.get_peer_channels().await; if let Some(bytecodes) = peer.request_bytecodes(batch.clone()).await { + info!("Received {} bytecodes", bytecodes.len()); // Store the bytecodes for code in bytecodes.into_iter() { store.add_account_code(batch.remove(0), code)?; From cafa1973944652910f3fa589be0fc9b6540ad08e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 29 Nov 2024 18:54:12 -0300 Subject: [PATCH 254/345] Impl request storage ranges --- crates/networking/p2p/peer_channels.rs | 92 ++++++++++++++++++++++++++ 1 file changed, 92 insertions(+) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index ef8bfb833..a67cdeff2 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -264,4 +264,96 @@ impl PeerChannels { .ok()??; (!codes.is_empty() && codes.len() <= hashes_len).then_some(codes) } + + /// Requests storage ranges for accounts given hasheshed address, storage roots, and the root of their state trie + /// account_hashes & storage_roots must have the same length + /// Returns true if the last accoun't storage was not completely fetched by the request + /// Returns the list of hashed storage keys and values for each account's storage or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_storage_ranges( + &self, + state_root: H256, + mut storage_roots: Vec, + account_hashes: Vec, + start: H256, + ) -> Option<(Vec>, Vec>, bool)> { + let request_id = rand::random(); + let request = RLPxMessage::GetStorageRanges(GetStorageRanges { + id: request_id, + root_hash: state_root, + account_hashes, + starting_hash: start, + limit_hash: HASH_MAX, + response_bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let (mut slots, proof) = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::StorageRanges(StorageRanges { id, slots, proof })) + if id == request_id => + { + return Some((slots, proof)) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + // Check we got a reasonable amount of storages + if slots.len() > storage_roots.len() || slots.is_empty() { + return None; + } + // Unzip & validate response + let mut proof = encodable_to_proof(&proof); + let mut storage_keys = vec![]; + let mut storage_values = vec![]; + let mut should_continue = false; + // Validate each storage range + while !slots.is_empty() { + let (hahsed_keys, values): (Vec<_>, Vec<_>) = slots + .remove(0) + .into_iter() + .map(|slot| (slot.hash, slot.data)) + .unzip(); + let encoded_values = values + .iter() + .map(|val| val.encode_to_vec()) + .collect::>(); + let storage_root = storage_roots.remove(0); + // We have 3 cases: + // - The range is empty: We expect one edge proof + // - The range has only 1 element (with key matching the start): We expect one edge proof + // - The range has the full storage: We expect no proofs + // - The range is not the full storage (last range): We expect 2 edge proofs + if hahsed_keys.is_empty() || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) { + if proof.len() < 1 { + return None + }; + let first_proof = vec![proof.remove(0)]; + verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &first_proof).ok()?; + } + if slots.is_empty() { + // Last element + if proof.len() < 2 { + return None + }; + let last_proof = vec![proof.remove(0), proof.remove(0)]; + should_continue = verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &last_proof).ok()?; + } else { + // Not the last element = Full range + verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &vec![]).ok()?; + } + + storage_keys.push(hahsed_keys); + storage_values.push(values); + } + Some((storage_keys, storage_values, should_continue)) + } } From cbdeeb658fa2ca07a0f74338d9bb49220cbcf005 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 29 Nov 2024 19:06:05 -0300 Subject: [PATCH 255/345] Storage fetcher skeleton --- crates/networking/p2p/peer_channels.rs | 22 ++++++-- crates/networking/p2p/sync.rs | 69 ++++++++++++++++++++++---- 2 files changed, 78 insertions(+), 13 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index a67cdeff2..66a64eab1 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -334,18 +334,32 @@ impl PeerChannels { // - The range is not the full storage (last range): We expect 2 edge proofs if hahsed_keys.is_empty() || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) { if proof.len() < 1 { - return None + return None; }; let first_proof = vec![proof.remove(0)]; - verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &first_proof).ok()?; + verify_range( + storage_root, + &start, + &hahsed_keys, + &encoded_values, + &first_proof, + ) + .ok()?; } if slots.is_empty() { // Last element if proof.len() < 2 { - return None + return None; }; let last_proof = vec![proof.remove(0), proof.remove(0)]; - should_continue = verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &last_proof).ok()?; + should_continue = verify_range( + storage_root, + &start, + &hahsed_keys, + &encoded_values, + &last_proof, + ) + .ok()?; } else { // Not the last element = Full range verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &vec![]).ok()?; diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 2a53c3d29..0f2a56ddc 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -77,6 +77,7 @@ impl SyncManager { // TODO: We are currently testing against our implementation that doesn't hold an independant snapchot and can provide all historic state // We should fetch all available state and then resort to state healing to fetch the rest let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); + let (storage_sender, storage_receiver) = mpsc::channel::>(500); let mut set = tokio::task::JoinSet::new(); set.spawn(bytecode_fetcher( bytecode_receiver, @@ -94,6 +95,7 @@ impl SyncManager { .collect::>(); set.spawn(fetch_snap_state( bytecode_sender, + storage_sender, state_roots.clone(), self.peers.clone(), store.clone(), @@ -229,6 +231,7 @@ async fn fetch_blocks_and_receipts( async fn fetch_snap_state( bytecode_sender: Sender>, + storage_sender: Sender>, state_roots: Vec, peers: Arc>, store: Store, @@ -239,6 +242,7 @@ async fn fetch_snap_state( // TODO: maybe spawn taks here instead of awaiting rebuild_state_trie( bytecode_sender.clone(), + storage_sender.clone(), state_root, peers.clone(), store.clone(), @@ -254,6 +258,7 @@ async fn fetch_snap_state( /// Rebuilds a Block's state trie by requesting snap state from peers async fn rebuild_state_trie( bytecode_sender: Sender>, + storage_sender: Sender>, state_root: H256, peers: Arc>, store: Store, @@ -276,28 +281,34 @@ async fn rebuild_state_trie( } // Fetch Account Storage & Bytecode let mut code_hashes = vec![]; + let mut account_hashes_and_storage_roots = vec![]; for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { // Build the batch of code hashes to send to the bytecode fetcher - // Ignoring accounts without code / code we already have stored + // Ignore accounts without code / code we already have stored if account.code_hash != *EMPTY_KECCACK_HASH && store.get_account_code(account.code_hash)?.is_none() { code_hashes.push(account.code_hash) } - // The first iteration could await them here as we are only testing the one peer case - rebuild_storage_trie( - *account_hash, - account.storage_root, - peers.clone(), - store.clone(), - ) - .await?; + // Build the batch of hashes and roots to send to the storage fetcher + // Ignore accounts without storage + if account.storage_root != *EMPTY_TRIE_HASH { + account_hashes_and_storage_roots.push((*account_hash, account.storage_root)); + } } // Send code hash batch to the bytecode fetcher if !code_hashes.is_empty() { // TODO: Handle bytecode_sender.send(code_hashes).await.unwrap() } + // Send hash and root batch to the storage fetcher + if !account_hashes_and_storage_roots.is_empty() { + // TODO: Handle + storage_sender + .send(account_hashes_and_storage_roots) + .await + .unwrap() + } // Update trie let mut trie = store.open_state_trie(current_state_root); for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { @@ -423,3 +434,43 @@ async fn fetch_bytecode_batch( } } } + +/// Waits for incoming account hashes & storage roots from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches +async fn storage_fetcher( + mut receiver: Receiver>, + peers: Arc>, + store: Store, +) -> Result<(), StoreError> { + // Pending list of bytecodes to fetch + const BATCH_SIZE: usize = 50; + // TODO: Also add a queue for storages that were incompletely fecthed, + // but for the first iteration we will asume not fully fetched -> fetch again + let mut pending_storage: Vec<(H256, H256)> = vec![]; + loop { + match receiver.recv().await { + Some(account_and_root) if !account_and_root.is_empty() => { + // Add hashes to the queue + pending_storage.extend(account_and_root); + // If we have enought pending bytecodes to fill a batch, spawn a fetch process + while pending_storage.len() >= BATCH_SIZE { + let next_batch = pending_storage.drain(..BATCH_SIZE).collect::>(); + let remaining = todo!(); + // Add unfeched bytecodes back to the queue + pending_storage.extend(remaining); + } + } + // Disconnect / Empty message signaling no more bytecodes to sync + _ => break, + } + } + // We have no more incoming requests, process the remaining batches + while !pending_storage.is_empty() { + let next_batch = pending_storage + .drain(..BATCH_SIZE.min(pending_storage.len())) + .collect::>(); + let remaining = todo!(); + // Add unfeched bytecodes back to the queue + pending_storage.extend(remaining); + } + Ok(()) +} From c76cfeb1a8fb0217f1c37a22bcce623ebbd39f26 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 10:48:06 -0300 Subject: [PATCH 256/345] Add storage fetcher process --- crates/networking/p2p/sync.rs | 110 ++++++++++++++++++---------------- 1 file changed, 57 insertions(+), 53 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 0f2a56ddc..379b6aa15 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -77,7 +77,6 @@ impl SyncManager { // TODO: We are currently testing against our implementation that doesn't hold an independant snapchot and can provide all historic state // We should fetch all available state and then resort to state healing to fetch the rest let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); - let (storage_sender, storage_receiver) = mpsc::channel::>(500); let mut set = tokio::task::JoinSet::new(); set.spawn(bytecode_fetcher( bytecode_receiver, @@ -95,7 +94,6 @@ impl SyncManager { .collect::>(); set.spawn(fetch_snap_state( bytecode_sender, - storage_sender, state_roots.clone(), self.peers.clone(), store.clone(), @@ -231,7 +229,6 @@ async fn fetch_blocks_and_receipts( async fn fetch_snap_state( bytecode_sender: Sender>, - storage_sender: Sender>, state_roots: Vec, peers: Arc>, store: Store, @@ -242,7 +239,6 @@ async fn fetch_snap_state( // TODO: maybe spawn taks here instead of awaiting rebuild_state_trie( bytecode_sender.clone(), - storage_sender.clone(), state_root, peers.clone(), store.clone(), @@ -258,11 +254,18 @@ async fn fetch_snap_state( /// Rebuilds a Block's state trie by requesting snap state from peers async fn rebuild_state_trie( bytecode_sender: Sender>, - storage_sender: Sender>, state_root: H256, peers: Arc>, store: Store, ) -> Result<(), StoreError> { + // Spawn a storage fetcher for this blocks's storage + let (storage_sender, storage_receiver) = mpsc::channel::>(500); + let storage_fetcher_handler = tokio::spawn(storage_fetcher( + storage_receiver, + peers.clone(), + store.clone(), + state_root, + )); let mut start_account_hash = H256::zero(); // Start from an empty state trie // We cannot keep an open trie here so we will track the root between lookups @@ -326,56 +329,13 @@ async fn rebuild_state_trie( if current_state_root != state_root { warn!("State sync failed for state root {state_root}"); } + storage_fetcher_handler + .await + .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; info!("Completed state sync for state root {state_root}"); Ok(()) } -/// Rebuilds an Account's storage trie by requesting snap state from peers -/// First Iteration: Fetch one account storage at a time -> this is inefficient, we should queue them and fetch multiple accounts at once -async fn rebuild_storage_trie( - account_hash: H256, - storage_root: H256, - peers: Arc>, - store: Store, -) -> Result<(), StoreError> { - let mut start_key_hash = H256::zero(); - // Start from an empty state trie - // We cannot keep an open trie here so we will track the root between lookups - let mut current_storage_root = *EMPTY_TRIE_HASH; - // Fetch Storage Ranges - loop { - let peer = peers.lock().await.get_peer_channels().await; - debug!("Requesting Storage Range for storage root {storage_root}, starting hash: {start_key_hash}"); - if let Some((hashed_keys, values, should_continue)) = peer - .request_storage_range(storage_root, account_hash, start_key_hash) - .await - { - // Update starting hash for next batch - if should_continue { - start_key_hash = *hashed_keys.last().unwrap(); - } - - // Update trie - let mut trie = store.open_storage_trie(account_hash, current_storage_root); - for (key_hash, value) in hashed_keys.iter().zip(values.iter()) { - trie.insert(key_hash.0.to_vec(), value.encode_to_vec()) - .map_err(StoreError::Trie)?; - } - current_storage_root = trie.hash().map_err(StoreError::Trie)?; - - if !should_continue { - // All keys fetched! - break; - } - } - } - if current_storage_root != storage_root { - warn!("State sync failed for storage root {storage_root}"); - } - info!("Completed state sync for storage root {storage_root}"); - Ok(()) -} - /// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches async fn bytecode_fetcher( mut receiver: Receiver>, @@ -440,6 +400,7 @@ async fn storage_fetcher( mut receiver: Receiver>, peers: Arc>, store: Store, + state_root: H256, ) -> Result<(), StoreError> { // Pending list of bytecodes to fetch const BATCH_SIZE: usize = 50; @@ -454,7 +415,9 @@ async fn storage_fetcher( // If we have enought pending bytecodes to fill a batch, spawn a fetch process while pending_storage.len() >= BATCH_SIZE { let next_batch = pending_storage.drain(..BATCH_SIZE).collect::>(); - let remaining = todo!(); + let remaining = + fetch_storage_batch(next_batch, state_root, peers.clone(), store.clone()) + .await?; // Add unfeched bytecodes back to the queue pending_storage.extend(remaining); } @@ -468,9 +431,50 @@ async fn storage_fetcher( let next_batch = pending_storage .drain(..BATCH_SIZE.min(pending_storage.len())) .collect::>(); - let remaining = todo!(); + let remaining = + fetch_storage_batch(next_batch, state_root, peers.clone(), store.clone()).await?; // Add unfeched bytecodes back to the queue pending_storage.extend(remaining); } Ok(()) } + +/// Receives a batch of account hashes with their storage roots, fetches their respective storage ranges via p2p and returns a list of the code hashes that couldn't be fetched in the request (if applicable) +async fn fetch_storage_batch( + mut batch: Vec<(H256, H256)>, + state_root: H256, + peers: Arc>, + store: Store, +) -> Result, StoreError> { + loop { + let peer = peers.lock().await.get_peer_channels().await; + let (batch_hahses, batch_roots) = batch.clone().into_iter().unzip(); + if let Some((mut keys, mut values, incomplete)) = peer + .request_storage_ranges(state_root, batch_hahses, batch_roots, H256::zero()) + .await + { + info!("Received {} storage ranges", keys.len()); + let mut last_range; + // Hold on to the last batch (if incomplete) + if incomplete { + // An incomplete range cannot be empty + last_range = (keys.pop().unwrap(), values.pop().unwrap()); + } + // Store the storage ranges & rebuild the storage trie for each account + for (keys, values) in keys.into_iter().zip(values.into_iter()) { + let (account_hash, storage_root) = batch.remove(0); + let mut trie = store.open_storage_trie(account_hash, *EMPTY_TRIE_HASH); + for (key, value) in keys.into_iter().zip(values.into_iter()) { + trie.insert(key.0.to_vec(), value.encode_to_vec())?; + } + if trie.hash()? != storage_root { + warn!("State sync failed for storage root {storage_root}"); + } + } + // TODO: if the last range is incomplete add it to the incomplete batches queue + // For now we will fetch the full range again + // Return remaining code hashes in the batch if we couldn't fetch all of them + return Ok(batch); + } + } +} From 586ec79e2afdc25592de7637ef3fb9b2b0931b31 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 14:39:20 -0300 Subject: [PATCH 257/345] Signal end --- crates/networking/p2p/sync.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 379b6aa15..3a8823e8f 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -329,6 +329,8 @@ async fn rebuild_state_trie( if current_state_root != state_root { warn!("State sync failed for state root {state_root}"); } + // Send empty batch to signal that no more batches are incoming + storage_sender.send(vec![]).await.unwrap(); storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; From 3f5bb7719e031eb84ae8431ff3bead5acb230537 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 16:49:18 -0300 Subject: [PATCH 258/345] Fix --- crates/networking/p2p/peer_channels.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 66a64eab1..aea928a14 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -306,8 +306,8 @@ impl PeerChannels { }) .await .ok()??; - // Check we got a reasonable amount of storages - if slots.len() > storage_roots.len() || slots.is_empty() { + // Check we got a reasonable amount of storage ranges + if slots.len() > storage_roots.len() || (slots.is_empty() && proof.is_empty()) { return None; } // Unzip & validate response @@ -328,11 +328,13 @@ impl PeerChannels { .collect::>(); let storage_root = storage_roots.remove(0); // We have 3 cases: - // - The range is empty: We expect one edge proof + // - The range is empty (and start != 0): We expect one edge proof // - The range has only 1 element (with key matching the start): We expect one edge proof // - The range has the full storage: We expect no proofs // - The range is not the full storage (last range): We expect 2 edge proofs - if hahsed_keys.is_empty() || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) { + if hahsed_keys.is_empty() && !start.is_zero() + || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) + { if proof.len() < 1 { return None; }; From 98f3270d53bf338cb6ac8fda8f50246ec80293a3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 18:18:43 -0300 Subject: [PATCH 259/345] Fix --- cmd/ethrex/ethrex.rs | 8 ++++---- crates/networking/p2p/sync.rs | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 168b8a6cd..92f1bb841 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -314,10 +314,10 @@ fn import_blocks(store: &Store, blocks: &Vec) { let size = blocks.len(); for block in blocks { let hash = block.hash(); - info!( - "Adding block {} with hash {:#x}.", - block.header.number, hash - ); + // info!( + // "Adding block {} with hash {:#x}.", + // block.header.number, hash + // ); let result = add_block(block, store); if let Some(error) = result.err() { warn!( diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 3a8823e8f..78b3b6d91 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -452,7 +452,7 @@ async fn fetch_storage_batch( let peer = peers.lock().await.get_peer_channels().await; let (batch_hahses, batch_roots) = batch.clone().into_iter().unzip(); if let Some((mut keys, mut values, incomplete)) = peer - .request_storage_ranges(state_root, batch_hahses, batch_roots, H256::zero()) + .request_storage_ranges(state_root, batch_roots, batch_hahses, H256::zero()) .await { info!("Received {} storage ranges", keys.len()); From 70286632b00137adc33af99c105583b075ade0a1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 18:36:15 -0300 Subject: [PATCH 260/345] Fix --- crates/networking/p2p/peer_channels.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index aea928a14..e5ff76fca 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -330,7 +330,7 @@ impl PeerChannels { // We have 3 cases: // - The range is empty (and start != 0): We expect one edge proof // - The range has only 1 element (with key matching the start): We expect one edge proof - // - The range has the full storage: We expect no proofs + // - The range has the full storage (empty range but start = 0 fits here): We expect no proofs // - The range is not the full storage (last range): We expect 2 edge proofs if hahsed_keys.is_empty() && !start.is_zero() || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) @@ -348,7 +348,7 @@ impl PeerChannels { ) .ok()?; } - if slots.is_empty() { + if slots.is_empty() && !(slots.is_empty() && start.is_zero()) { // Last element if proof.len() < 2 { return None; From 711d9878773a48c4b9ecc4c1a288b48bc7fa7cca Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 2 Dec 2024 18:37:39 -0300 Subject: [PATCH 261/345] Mute tracing --- crates/networking/p2p/sync.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 78b3b6d91..aba08cdab 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -233,7 +233,7 @@ async fn fetch_snap_state( peers: Arc>, store: Store, ) -> Result<(), StoreError> { - info!("Syncing state roots: {}", state_roots.len()); + debug!("Syncing state roots: {}", state_roots.len()); // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { // TODO: maybe spawn taks here instead of awaiting @@ -334,7 +334,7 @@ async fn rebuild_state_trie( storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - info!("Completed state sync for state root {state_root}"); + debug!("Completed state sync for state root {state_root}"); Ok(()) } @@ -386,7 +386,7 @@ async fn fetch_bytecode_batch( loop { let peer = peers.lock().await.get_peer_channels().await; if let Some(bytecodes) = peer.request_bytecodes(batch.clone()).await { - info!("Received {} bytecodes", bytecodes.len()); + debug!("Received {} bytecodes", bytecodes.len()); // Store the bytecodes for code in bytecodes.into_iter() { store.add_account_code(batch.remove(0), code)?; @@ -455,7 +455,7 @@ async fn fetch_storage_batch( .request_storage_ranges(state_root, batch_roots, batch_hahses, H256::zero()) .await { - info!("Received {} storage ranges", keys.len()); + debug!("Received {} storage ranges", keys.len()); let mut last_range; // Hold on to the last batch (if incomplete) if incomplete { From d5b1ab5aeb40e810c984024b176a976c0d18cb64 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 11:23:07 -0300 Subject: [PATCH 262/345] Update --- crates/networking/p2p/sync.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index aba08cdab..245c80770 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -405,7 +405,7 @@ async fn storage_fetcher( state_root: H256, ) -> Result<(), StoreError> { // Pending list of bytecodes to fetch - const BATCH_SIZE: usize = 50; + const BATCH_SIZE: usize = 75; // TODO: Also add a queue for storages that were incompletely fecthed, // but for the first iteration we will asume not fully fetched -> fetch again let mut pending_storage: Vec<(H256, H256)> = vec![]; @@ -456,11 +456,11 @@ async fn fetch_storage_batch( .await { debug!("Received {} storage ranges", keys.len()); - let mut last_range; + let mut _last_range; // Hold on to the last batch (if incomplete) if incomplete { // An incomplete range cannot be empty - last_range = (keys.pop().unwrap(), values.pop().unwrap()); + _last_range = (keys.pop().unwrap(), values.pop().unwrap()); } // Store the storage ranges & rebuild the storage trie for each account for (keys, values) in keys.into_iter().zip(values.into_iter()) { From 2a59cdc123ed5d3b737a9a4be0cb4fb599439eb2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 11:24:19 -0300 Subject: [PATCH 263/345] Update batch size --- crates/networking/p2p/sync.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 245c80770..7c7a77e72 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -345,7 +345,7 @@ async fn bytecode_fetcher( store: Store, ) -> Result<(), StoreError> { // Pending list of bytecodes to fetch - const BATCH_SIZE: usize = 100; + const BATCH_SIZE: usize = 200; let mut pending_bytecodes: Vec = vec![]; loop { match receiver.recv().await { @@ -405,7 +405,7 @@ async fn storage_fetcher( state_root: H256, ) -> Result<(), StoreError> { // Pending list of bytecodes to fetch - const BATCH_SIZE: usize = 75; + const BATCH_SIZE: usize = 100; // TODO: Also add a queue for storages that were incompletely fecthed, // but for the first iteration we will asume not fully fetched -> fetch again let mut pending_storage: Vec<(H256, H256)> = vec![]; From e21a77abfc752e16ef957b92fd95c123c197ada4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 11:51:57 -0300 Subject: [PATCH 264/345] Add error handling --- crates/networking/p2p/sync.rs | 125 ++++++++++++++++++++-------------- 1 file changed, 72 insertions(+), 53 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 7c7a77e72..702f0bb38 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -10,7 +10,7 @@ use ethrex_storage::{error::StoreError, Store}; use ethrex_trie::EMPTY_TRIE_HASH; use tokio::{ sync::{ - mpsc::{self, Receiver, Sender}, + mpsc::{self, error::SendError, Receiver, Sender}, Mutex, }, time::Instant, @@ -34,11 +34,48 @@ impl SyncManager { Self { snap_mode, peers } } + /// Creates a dummy SyncManager for tests where syncing is not needed + /// This should only be used in tests as it won't be able to connect to the p2p network + pub fn dummy() -> Self { + let dummy_peer_table = Arc::new(Mutex::new(KademliaTable::new(Default::default()))); + Self { + snap_mode: false, + peers: dummy_peer_table, + } + } + /// Starts a sync cycle, updating the state with all blocks between the current head and the sync head - /// TODO: only uses full sync, should also process snap sync once implemented - pub async fn start_sync(&mut self, mut current_head: H256, sync_head: H256, store: Store) { + /// Will perforn either full or snap sync depending on the manager's `snap_mode` + /// In full mode, all blocks will be fetched via p2p eth requests and executed to rebuild the state + /// In snap mode, blocks and receipts will be fetched and stored in parallel while the state is fetched via p2p snap requests + /// After the sync cycle is complete, the sync mode will be set to full + /// If the sync fails, no error will be returned but a warning will be emitted + pub async fn start_sync(&mut self, current_head: H256, sync_head: H256, store: Store) { info!("Syncing from current head {current_head} to sync_head {sync_head}"); let start_time = Instant::now(); + match self.sync_cycle(current_head, sync_head, store).await { + Ok(()) => { + info!( + "Sync finished, time elapsed: {} secs", + start_time.elapsed().as_secs() + ); + // Next sync will be full-sync + self.snap_mode = false; + } + Err(error) => warn!( + "Sync failed due to {error}, time elapsed: {} secs ", + start_time.elapsed().as_secs() + ), + } + } + + /// Performs the sync cycle described in `start_sync`, returns an error if the sync fails at any given step and aborts all active processes + async fn sync_cycle( + &mut self, + mut current_head: H256, + sync_head: H256, + store: Store, + ) -> Result<(), SyncError> { // Request all block headers between the current head and the sync head // We will begin from the current head so that we download the earliest state first // This step is not parallelized @@ -70,11 +107,11 @@ impl SyncManager { } } // We finished fetching all headers, now we can process them - let result = if self.snap_mode { + if self.snap_mode { // snap-sync: launch tasks to fetch blocks and state in parallel // - Fetch each block's state via snap p2p requests // - Fetch each blocks and its receipts via eth p2p requests - // TODO: We are currently testing against our implementation that doesn't hold an independant snapchot and can provide all historic state + // TODO: We are currently testing against our implementation that doesn't hold an independant snapshot and can provide all historic state // We should fetch all available state and then resort to state healing to fetch the rest let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); let mut set = tokio::task::JoinSet::new(); @@ -106,20 +143,15 @@ impl SyncManager { { // TODO: Handle error latest_block_number = header.number; - store.set_canonical_block(header.number, hash).unwrap(); - store.add_block_header(hash, header).unwrap(); + store.set_canonical_block(header.number, hash)?; + store.add_block_header(hash, header)?; + } + // If all processes failed then they are likely to have a common cause (such as unaccessible storage), so return the first error + for result in set.join_all().await { + result?; } - let result = set.join_all().await; // Set latest block number here to avoid reading state that is currently being synced - store - .update_latest_block_number(latest_block_number) - .unwrap(); - // Collapse into one error, if all processes failed then they are likely to have a common cause (such as unaccessible storage) - result - .into_iter() - .find(|res| res.is_err()) - .unwrap_or(Ok(())) - .map_err(ChainError::StoreError) + store.update_latest_block_number(latest_block_number)?; } else { // full-sync: Fetch all block bodies and execute them sequentially to build the state download_and_run_blocks( @@ -128,32 +160,9 @@ impl SyncManager { self.peers.clone(), store.clone(), ) - .await + .await? }; - match result { - Ok(()) => { - info!( - "Sync finished, time elapsed: {} secs", - start_time.elapsed().as_secs() - ); - // Next sync will be full-sync - self.snap_mode = false; - } - Err(error) => warn!( - "Sync failed due to {error}, time elapsed: {} secs ", - start_time.elapsed().as_secs() - ), - } - } - - /// Creates a dummy SyncManager for tests where syncing is not needed - /// This should only be used in tests as it won't be able to connect to the p2p network - pub fn dummy() -> Self { - let dummy_peer_table = Arc::new(Mutex::new(KademliaTable::new(Default::default()))); - Self { - snap_mode: false, - peers: dummy_peer_table, - } + Ok(()) } } @@ -199,7 +208,7 @@ async fn fetch_blocks_and_receipts( mut block_hashes: Vec, peers: Arc>, store: Store, -) -> Result<(), StoreError> { +) -> Result<(), SyncError> { // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts // Fetch Block Bodies loop { @@ -211,8 +220,7 @@ async fn fetch_blocks_and_receipts( let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); // Store Block Bodies for (hash, body) in fetched_hashes.into_iter().zip(block_bodies.into_iter()) { - // TODO: handle error - store.add_block_body(hash.clone(), body).unwrap() + store.add_block_body(hash.clone(), body)? } // Check if we need to ask for another batch @@ -232,7 +240,7 @@ async fn fetch_snap_state( state_roots: Vec, peers: Arc>, store: Store, -) -> Result<(), StoreError> { +) -> Result<(), SyncError> { debug!("Syncing state roots: {}", state_roots.len()); // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { @@ -247,7 +255,7 @@ async fn fetch_snap_state( } // We finished syncing the available state, lets make the fetcher processes aware // Send empty batches to signal that no more batches are incoming - bytecode_sender.send(vec![]).await.unwrap(); + bytecode_sender.send(vec![]).await?; Ok(()) } @@ -257,7 +265,7 @@ async fn rebuild_state_trie( state_root: H256, peers: Arc>, store: Store, -) -> Result<(), StoreError> { +) -> Result<(), SyncError> { // Spawn a storage fetcher for this blocks's storage let (storage_sender, storage_receiver) = mpsc::channel::>(500); let storage_fetcher_handler = tokio::spawn(storage_fetcher( @@ -302,15 +310,14 @@ async fn rebuild_state_trie( // Send code hash batch to the bytecode fetcher if !code_hashes.is_empty() { // TODO: Handle - bytecode_sender.send(code_hashes).await.unwrap() + bytecode_sender.send(code_hashes).await?; } // Send hash and root batch to the storage fetcher if !account_hashes_and_storage_roots.is_empty() { // TODO: Handle storage_sender .send(account_hashes_and_storage_roots) - .await - .unwrap() + .await?; } // Update trie let mut trie = store.open_state_trie(current_state_root); @@ -330,7 +337,7 @@ async fn rebuild_state_trie( warn!("State sync failed for state root {state_root}"); } // Send empty batch to signal that no more batches are incoming - storage_sender.send(vec![]).await.unwrap(); + storage_sender.send(vec![]).await?; storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; @@ -343,7 +350,7 @@ async fn bytecode_fetcher( mut receiver: Receiver>, peers: Arc>, store: Store, -) -> Result<(), StoreError> { +) -> Result<(), SyncError> { // Pending list of bytecodes to fetch const BATCH_SIZE: usize = 200; let mut pending_bytecodes: Vec = vec![]; @@ -480,3 +487,15 @@ async fn fetch_storage_batch( } } } + +#[derive(thiserror::Error, Debug)] +enum SyncError { + #[error(transparent)] + Chain(#[from] ChainError), + #[error(transparent)] + Store(#[from] StoreError), + #[error(transparent)] + SendBytecode(#[from] SendError>), + #[error(transparent)] + SendStorage(#[from] SendError>), +} From aec4c4883a80d1b9d1c2cc9cae363b464bf5d3e2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 12:13:11 -0300 Subject: [PATCH 265/345] Add syncmode enum --- cmd/ethrex/ethrex.rs | 17 +++-- crates/networking/p2p/sync.rs | 127 ++++++++++++++++++---------------- 2 files changed, 77 insertions(+), 67 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 92f1bb841..27ce5084f 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -6,7 +6,10 @@ use ethrex_core::{ H256, }; use ethrex_net::{ - bootnode::BootNode, node_id_from_signing_key, peer_table, sync::SyncManager, types::Node, + bootnode::BootNode, + node_id_from_signing_key, peer_table, + sync::{SyncManager, SyncMode}, + types::Node, }; use ethrex_rlp::decode::RLPDecode; use ethrex_storage::{EngineType, Store}; @@ -115,7 +118,7 @@ async fn main() { .get_one::("datadir") .map_or(set_datadir(DEFAULT_DATADIR), |datadir| set_datadir(datadir)); - let snap_sync = is_snap_sync(&matches); + let sync_mode = sync_mode(&matches); let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); @@ -176,7 +179,7 @@ async fn main() { // Create Kademlia Table here so we can access it from rpc server (for syncing) let peer_table = peer_table(signer.clone()); // Create SyncManager - let syncer = SyncManager::new(peer_table.clone(), snap_sync); + let syncer = SyncManager::new(peer_table.clone(), sync_mode); // TODO: Check every module starts properly. let tracker = TaskTracker::new(); @@ -288,16 +291,16 @@ fn parse_socket_addr(addr: &str, port: &str) -> io::Result { )) } -fn is_snap_sync(matches: &clap::ArgMatches) -> bool { +fn sync_mode(matches: &clap::ArgMatches) -> SyncMode { let syncmode = matches.get_one::("syncmode"); if let Some(syncmode) = syncmode { match &**syncmode { - "full" => false, - "snap" => true, + "full" => SyncMode::Full, + "snap" => SyncMode::Snap, other => panic!("Invalid syncmode {other} expected either snap or full"), } } else { - true + SyncMode::Snap } } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 702f0bb38..7e1cbd835 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -19,19 +19,23 @@ use tracing::{debug, info, warn}; use crate::kademlia::KademliaTable; +#[derive(Debug)] +pub enum SyncMode { + Full, + Snap, +} + /// Manager in charge the sync process /// Only performs full-sync but will also be in charge of snap-sync in the future #[derive(Debug)] pub struct SyncManager { - // true: syncmode = snap, false = syncmode = full - #[allow(unused)] - snap_mode: bool, + sync_mode: SyncMode, peers: Arc>, } impl SyncManager { - pub fn new(peers: Arc>, snap_mode: bool) -> Self { - Self { snap_mode, peers } + pub fn new(peers: Arc>, sync_mode: SyncMode) -> Self { + Self { sync_mode, peers } } /// Creates a dummy SyncManager for tests where syncing is not needed @@ -39,7 +43,7 @@ impl SyncManager { pub fn dummy() -> Self { let dummy_peer_table = Arc::new(Mutex::new(KademliaTable::new(Default::default()))); Self { - snap_mode: false, + sync_mode: SyncMode::Full, peers: dummy_peer_table, } } @@ -60,7 +64,7 @@ impl SyncManager { start_time.elapsed().as_secs() ); // Next sync will be full-sync - self.snap_mode = false; + self.sync_mode = SyncMode::Full; } Err(error) => warn!( "Sync failed due to {error}, time elapsed: {} secs ", @@ -107,61 +111,64 @@ impl SyncManager { } } // We finished fetching all headers, now we can process them - if self.snap_mode { - // snap-sync: launch tasks to fetch blocks and state in parallel - // - Fetch each block's state via snap p2p requests - // - Fetch each blocks and its receipts via eth p2p requests - // TODO: We are currently testing against our implementation that doesn't hold an independant snapshot and can provide all historic state - // We should fetch all available state and then resort to state healing to fetch the rest - let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); - let mut set = tokio::task::JoinSet::new(); - set.spawn(bytecode_fetcher( - bytecode_receiver, - self.peers.clone(), - store.clone(), - )); - set.spawn(fetch_blocks_and_receipts( - all_block_hashes.clone(), - self.peers.clone(), - store.clone(), - )); - let state_roots = all_block_headers - .iter() - .map(|header| header.state_root) - .collect::>(); - set.spawn(fetch_snap_state( - bytecode_sender, - state_roots.clone(), - self.peers.clone(), - store.clone(), - )); - // Store headers - let mut latest_block_number = 0; - for (header, hash) in all_block_headers - .into_iter() - .zip(all_block_hashes.into_iter()) - { - // TODO: Handle error - latest_block_number = header.number; - store.set_canonical_block(header.number, hash)?; - store.add_block_header(hash, header)?; + match self.sync_mode { + SyncMode::Snap => { + // snap-sync: launch tasks to fetch blocks and state in parallel + // - Fetch each block's state via snap p2p requests + // - Fetch each blocks and its receipts via eth p2p requests + // TODO: We are currently testing against our implementation that doesn't hold an independant snapshot and can provide all historic state + // We should fetch all available state and then resort to state healing to fetch the rest + let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); + let mut set = tokio::task::JoinSet::new(); + set.spawn(bytecode_fetcher( + bytecode_receiver, + self.peers.clone(), + store.clone(), + )); + set.spawn(fetch_blocks_and_receipts( + all_block_hashes.clone(), + self.peers.clone(), + store.clone(), + )); + let state_roots = all_block_headers + .iter() + .map(|header| header.state_root) + .collect::>(); + set.spawn(fetch_snap_state( + bytecode_sender, + state_roots.clone(), + self.peers.clone(), + store.clone(), + )); + // Store headers + let mut latest_block_number = 0; + for (header, hash) in all_block_headers + .into_iter() + .zip(all_block_hashes.into_iter()) + { + // TODO: Handle error + latest_block_number = header.number; + store.set_canonical_block(header.number, hash)?; + store.add_block_header(hash, header)?; + } + // If all processes failed then they are likely to have a common cause (such as unaccessible storage), so return the first error + for result in set.join_all().await { + result?; + } + // Set latest block number here to avoid reading state that is currently being synced + store.update_latest_block_number(latest_block_number)?; } - // If all processes failed then they are likely to have a common cause (such as unaccessible storage), so return the first error - for result in set.join_all().await { - result?; + SyncMode::Full => { + // full-sync: Fetch all block bodies and execute them sequentially to build the state + download_and_run_blocks( + all_block_hashes, + all_block_headers, + self.peers.clone(), + store.clone(), + ) + .await? } - // Set latest block number here to avoid reading state that is currently being synced - store.update_latest_block_number(latest_block_number)?; - } else { - // full-sync: Fetch all block bodies and execute them sequentially to build the state - download_and_run_blocks( - all_block_hashes, - all_block_headers, - self.peers.clone(), - store.clone(), - ) - .await? - }; + } Ok(()) } } From 317ccb5301491650bff233a00884f182f2651a0f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 15:21:46 -0300 Subject: [PATCH 266/345] Restore store creation --- cmd/ethrex/ethrex.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 27ce5084f..9885cf1fb 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -120,7 +120,15 @@ async fn main() { let sync_mode = sync_mode(&matches); - let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); + cfg_if::cfg_if! { + if #[cfg(feature = "redb")] { + let store = Store::new(&data_dir, EngineType::RedB).expect("Failed to create Store"); + } else if #[cfg(feature = "libmdbx")] { + let store = Store::new(&data_dir, EngineType::Libmdbx).expect("Failed to create Store"); + } else { + let store = Store::new(&data_dir, EngineType::InMemory).expect("Failed to create Store"); + } + } let genesis = read_genesis_file(genesis_file_path); store From 146d7679190dcc8d1ead53e4c3483b5db797b458 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 15:23:24 -0300 Subject: [PATCH 267/345] Uncomment tracing --- cmd/ethrex/ethrex.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmd/ethrex/ethrex.rs b/cmd/ethrex/ethrex.rs index 9885cf1fb..3dfd651b3 100644 --- a/cmd/ethrex/ethrex.rs +++ b/cmd/ethrex/ethrex.rs @@ -325,10 +325,10 @@ fn import_blocks(store: &Store, blocks: &Vec) { let size = blocks.len(); for block in blocks { let hash = block.hash(); - // info!( - // "Adding block {} with hash {:#x}.", - // block.header.number, hash - // ); + info!( + "Adding block {} with hash {:#x}.", + block.header.number, hash + ); let result = add_block(block, store); if let Some(error) = result.err() { warn!( From f53646ced6c6b830f7eb00f833c0d6ec757a3e59 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 15:25:03 -0300 Subject: [PATCH 268/345] Remove unused fn --- crates/networking/p2p/peer_channels.rs | 54 -------------------------- 1 file changed, 54 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 0f52fc0f8..a13464b08 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -179,60 +179,6 @@ impl PeerChannels { Some((account_hashes, accounts, should_continue)) } - // TODO: Inefficient method -> replace with request_storage_ranges - pub async fn request_storage_range( - &self, - storage_root: H256, - account_hash: H256, - start: H256, - ) -> Option<(Vec, Vec, bool)> { - let request_id = rand::random(); - let request = RLPxMessage::GetStorageRanges(GetStorageRanges { - id: request_id, - root_hash: storage_root, - account_hashes: vec![account_hash], - starting_hash: start, - limit_hash: HASH_MAX, - response_bytes: MAX_RESPONSE_BYTES, - }); - self.sender.send(request).await.ok()?; - let mut receiver = self.receiver.lock().await; - let (mut slots, proof) = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { - loop { - match receiver.recv().await { - Some(RLPxMessage::StorageRanges(StorageRanges { id, slots, proof })) - if id == request_id => - { - return Some((slots, proof)) - } - // Ignore replies that don't match the expected id (such as late responses) - Some(_) => continue, - None => return None, - } - } - }) - .await - .ok()??; - // We only requested 1 account so lets make sure we got it: - if slots.len() != 1 { - return None; - } - // Unzip & validate response - let proof = encodable_to_proof(&proof); - let (hahsed_keys, values): (Vec<_>, Vec<_>) = slots - .remove(0) - .into_iter() - .map(|slot| (slot.hash, slot.data)) - .unzip(); - let encoded_values = values - .iter() - .map(|val| val.encode_to_vec()) - .collect::>(); - let should_continue = - verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &proof).ok()?; - Some((hahsed_keys, values, should_continue)) - } - /// Requests bytecodes for the given code hashes /// Returns the bytecodes or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) From ec6a9d483e6975b0805abccdc5ff08242bde5dc8 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 16:33:47 -0300 Subject: [PATCH 269/345] Fix storage ranges verify logic --- crates/networking/p2p/peer_channels.rs | 28 +++++++++++++------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index a13464b08..d817f6c94 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -211,8 +211,9 @@ impl PeerChannels { (!codes.is_empty() && codes.len() <= hashes_len).then_some(codes) } - /// Requests storage ranges for accounts given hasheshed address, storage roots, and the root of their state trie + /// Requests storage ranges for accounts given their hashed address and storage roots, and the root of their state trie /// account_hashes & storage_roots must have the same length + /// storage_roots must not contain empty trie hashes, we will treat empty ranges as invalid responses /// Returns true if the last accoun't storage was not completely fetched by the request /// Returns the list of hashed storage keys and values for each account's storage or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) @@ -253,7 +254,7 @@ impl PeerChannels { .await .ok()??; // Check we got a reasonable amount of storage ranges - if slots.len() > storage_roots.len() || (slots.is_empty() && proof.is_empty()) { + if slots.len() > storage_roots.len() || slots.is_empty() { return None; } // Unzip & validate response @@ -268,19 +269,21 @@ impl PeerChannels { .into_iter() .map(|slot| (slot.hash, slot.data)) .unzip(); + // We won't accept empty storage ranges + if hahsed_keys.is_empty() { + return None; + } let encoded_values = values .iter() .map(|val| val.encode_to_vec()) .collect::>(); let storage_root = storage_roots.remove(0); - // We have 3 cases: - // - The range is empty (and start != 0): We expect one edge proof + + // We have 3 cases (as we won't accept empty storage ranges): // - The range has only 1 element (with key matching the start): We expect one edge proof - // - The range has the full storage (empty range but start = 0 fits here): We expect no proofs + // - The range has the full storage: We expect no proofs // - The range is not the full storage (last range): We expect 2 edge proofs - if hahsed_keys.is_empty() && !start.is_zero() - || (hahsed_keys.len() == 1 && hahsed_keys[0] == start) - { + if hahsed_keys.len() == 1 && hahsed_keys[0] == start { if proof.len() < 1 { return None; }; @@ -294,11 +297,8 @@ impl PeerChannels { ) .ok()?; } - if slots.is_empty() && !(slots.is_empty() && start.is_zero()) { - // Last element - if proof.len() < 2 { - return None; - }; + // Last element with two edge proofs + if slots.is_empty() && proof.len() >= 2 { let last_proof = vec![proof.remove(0), proof.remove(0)]; should_continue = verify_range( storage_root, @@ -309,7 +309,7 @@ impl PeerChannels { ) .ok()?; } else { - // Not the last element = Full range + // Full range (no proofs) verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &vec![]).ok()?; } From 1c945f63947ffaaa45971d4271e0acc44e52f3a4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 16:38:46 -0300 Subject: [PATCH 270/345] Clippy --- crates/networking/p2p/peer_channels.rs | 4 ++-- crates/networking/p2p/sync.rs | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index d817f6c94..160b5b57c 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -284,7 +284,7 @@ impl PeerChannels { // - The range has the full storage: We expect no proofs // - The range is not the full storage (last range): We expect 2 edge proofs if hahsed_keys.len() == 1 && hahsed_keys[0] == start { - if proof.len() < 1 { + if proof.is_empty() { return None; }; let first_proof = vec![proof.remove(0)]; @@ -310,7 +310,7 @@ impl PeerChannels { .ok()?; } else { // Full range (no proofs) - verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &vec![]).ok()?; + verify_range(storage_root, &start, &hahsed_keys, &encoded_values, &[]).ok()?; } storage_keys.push(hahsed_keys); diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 7e1cbd835..09e634c10 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -226,8 +226,8 @@ async fn fetch_blocks_and_receipts( // Track which bodies we have already fetched let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); // Store Block Bodies - for (hash, body) in fetched_hashes.into_iter().zip(block_bodies.into_iter()) { - store.add_block_body(hash.clone(), body)? + for (hash, body) in fetched_hashes.iter().zip(block_bodies.into_iter()) { + store.add_block_body(*hash, body)? } // Check if we need to ask for another batch From 5b93fbdc2140076f137a9a4cf5596e071fd9fcf1 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 16:39:22 -0300 Subject: [PATCH 271/345] Revert "Revert "Revert "[REVERT ME] Make V2 endpoints work in order to run hive sync test""" This reverts commit a069f8096c6e40b584bf1ad595951b8a51c20c17. --- crates/networking/rpc/engine/payload.rs | 20 +++++++++++--------- crates/networking/rpc/rpc.rs | 2 -- crates/networking/rpc/types/payload.rs | 4 ++-- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index abd59941f..9d7eb0089 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -43,14 +43,16 @@ impl RpcHandler for NewPayloadV3Request { let params = params .as_ref() .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - // if params.len() != 3 { - // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - // } + if params.len() != 3 { + return Err(RpcErr::BadParams("Expected 3 params".to_owned())); + } Ok(NewPayloadV3Request { payload: serde_json::from_value(params[0].clone()) .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: vec![], - parent_beacon_block_root: H256::zero(), + expected_blob_versioned_hashes: serde_json::from_value(params[1].clone()) + .map_err(|_| RpcErr::WrongParam("expected_blob_versioned_hashes".to_string()))?, + parent_beacon_block_root: serde_json::from_value(params[2].clone()) + .map_err(|_| RpcErr::WrongParam("parent_beacon_block_root".to_string()))?, }) } @@ -77,10 +79,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post Cancun fork let chain_config = storage.get_chain_config()?; - // let current_fork = chain_config.get_fork(block.header.timestamp); - // if current_fork < Fork::Cancun { - // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - // } + let current_fork = chain_config.get_fork(block.header.timestamp); + if current_fork < Fork::Cancun { + return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); + } // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 930c48f51..80581d2ab 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -254,9 +254,7 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), - "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), - "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index ee9d873eb..b35825039 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,9 +35,9 @@ pub struct ExecutionPayloadV3 { pub block_hash: H256, transactions: Vec, withdrawals: Vec, - #[serde(with = "serde_utils::u64::hex_str", default)] + #[serde(with = "serde_utils::u64::hex_str")] blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str", default)] + #[serde(with = "serde_utils::u64::hex_str")] excess_blob_gas: u64, } From 776ddc8fac88658022c983dfc6ddd2503b3b695c Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 16:55:41 -0300 Subject: [PATCH 272/345] Clippy --- crates/blockchain/mempool.rs | 6 +++--- crates/blockchain/payload.rs | 2 -- crates/l2/prover/src/prover.rs | 4 ++-- crates/networking/p2p/rlpx/error.rs | 12 ++++++++++-- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/crates/blockchain/mempool.rs b/crates/blockchain/mempool.rs index 9680f37b2..bfde66b92 100644 --- a/crates/blockchain/mempool.rs +++ b/crates/blockchain/mempool.rs @@ -78,9 +78,9 @@ pub fn filter_transactions( // Filter by tip & base_fee if let Some(min_tip) = filter.min_tip { - if !tx + if tx .effective_gas_tip(filter.base_fee) - .is_some_and(|tip| tip >= min_tip) + .is_none_or(|tip| tip < min_tip) { return false; } @@ -93,7 +93,7 @@ pub fn filter_transactions( // Filter by blob gas fee if let (true, Some(blob_fee)) = (is_blob_tx, filter.blob_fee) { - if !tx.max_fee_per_blob_gas().is_some_and(|fee| fee >= blob_fee) { + if tx.max_fee_per_blob_gas().is_none_or(|fee| fee < blob_fee) { return false; } } diff --git a/crates/blockchain/payload.rs b/crates/blockchain/payload.rs index a9e841fb8..7eb0c6916 100644 --- a/crates/blockchain/payload.rs +++ b/crates/blockchain/payload.rs @@ -169,9 +169,7 @@ impl<'a> PayloadBuildContext<'a> { blobs_bundle: BlobsBundle::default(), } } -} -impl<'a> PayloadBuildContext<'a> { fn parent_hash(&self) -> BlockHash { self.payload.header.parent_hash } diff --git a/crates/l2/prover/src/prover.rs b/crates/l2/prover/src/prover.rs index f366a61b8..9407e689d 100644 --- a/crates/l2/prover/src/prover.rs +++ b/crates/l2/prover/src/prover.rs @@ -14,13 +14,13 @@ pub struct Prover<'a> { pub stdout: Vec, } -impl<'a> Default for Prover<'a> { +impl Default for Prover<'_> { fn default() -> Self { Self::new() } } -impl<'a> Prover<'a> { +impl Prover<'_> { pub fn new() -> Self { Self { elf: ZKVM_PROGRAM_ELF, diff --git a/crates/networking/p2p/rlpx/error.rs b/crates/networking/p2p/rlpx/error.rs index 66f6c927b..835c57fb9 100644 --- a/crates/networking/p2p/rlpx/error.rs +++ b/crates/networking/p2p/rlpx/error.rs @@ -41,12 +41,20 @@ pub(crate) enum RLPxError { BroadcastError(String), #[error(transparent)] RecvError(#[from] RecvError), - #[error(transparent)] - Send(#[from] tokio::sync::mpsc::error::SendError), + #[error("Failed to send msg: {0}")] + SendMessage(String), #[error("Error when inserting transaction in the mempool: {0}")] MempoolError(#[from] MempoolError), } +// tokio::sync::mpsc::error::SendError is too large to be part of the RLPxError enum directly +// so we will instead save the error's display message +impl From> for RLPxError { + fn from(value: tokio::sync::mpsc::error::SendError) -> Self { + Self::SendMessage(value.to_string()) + } +} + // Grouping all cryptographic related errors in a single CryptographicError variant // We can improve this to individual errors if required impl From for RLPxError { From be70833681818ea1f2d3a4bebb0e15cbf280f159 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 16:57:02 -0300 Subject: [PATCH 273/345] Fix clippy --- crates/common/types/blobs_bundle.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/common/types/blobs_bundle.rs b/crates/common/types/blobs_bundle.rs index 974892476..33d91f49c 100644 --- a/crates/common/types/blobs_bundle.rs +++ b/crates/common/types/blobs_bundle.rs @@ -222,7 +222,6 @@ pub enum BlobsBundleError { } #[cfg(test)] - mod tests { use super::*; use crate::{ From f501b028cd23c2f6923553ba3a4b8145c54eb632 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 17:11:44 -0300 Subject: [PATCH 274/345] Revert unstable lib feature --- crates/blockchain/mempool.rs | 6 +++--- crates/blockchain/payload.rs | 2 ++ crates/l2/prover/src/prover.rs | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/crates/blockchain/mempool.rs b/crates/blockchain/mempool.rs index bfde66b92..9680f37b2 100644 --- a/crates/blockchain/mempool.rs +++ b/crates/blockchain/mempool.rs @@ -78,9 +78,9 @@ pub fn filter_transactions( // Filter by tip & base_fee if let Some(min_tip) = filter.min_tip { - if tx + if !tx .effective_gas_tip(filter.base_fee) - .is_none_or(|tip| tip < min_tip) + .is_some_and(|tip| tip >= min_tip) { return false; } @@ -93,7 +93,7 @@ pub fn filter_transactions( // Filter by blob gas fee if let (true, Some(blob_fee)) = (is_blob_tx, filter.blob_fee) { - if tx.max_fee_per_blob_gas().is_none_or(|fee| fee < blob_fee) { + if !tx.max_fee_per_blob_gas().is_some_and(|fee| fee >= blob_fee) { return false; } } diff --git a/crates/blockchain/payload.rs b/crates/blockchain/payload.rs index 7eb0c6916..a9e841fb8 100644 --- a/crates/blockchain/payload.rs +++ b/crates/blockchain/payload.rs @@ -169,7 +169,9 @@ impl<'a> PayloadBuildContext<'a> { blobs_bundle: BlobsBundle::default(), } } +} +impl<'a> PayloadBuildContext<'a> { fn parent_hash(&self) -> BlockHash { self.payload.header.parent_hash } diff --git a/crates/l2/prover/src/prover.rs b/crates/l2/prover/src/prover.rs index 9407e689d..f366a61b8 100644 --- a/crates/l2/prover/src/prover.rs +++ b/crates/l2/prover/src/prover.rs @@ -14,13 +14,13 @@ pub struct Prover<'a> { pub stdout: Vec, } -impl Default for Prover<'_> { +impl<'a> Default for Prover<'a> { fn default() -> Self { Self::new() } } -impl Prover<'_> { +impl<'a> Prover<'a> { pub fn new() -> Self { Self { elf: ZKVM_PROGRAM_ELF, From ca622b6bd7e77d71dc1efc547b97453f1c60bfd3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:08:50 -0300 Subject: [PATCH 275/345] Improve doc --- .../{p2p/README.md => docs/Networking.md} | 0 crates/networking/docs/Syncing.md | 13 +++++++++++++ crates/networking/docs/diagrams/snap_sync.jpg | Bin 0 -> 34806 bytes crates/networking/p2p/peer_channels.rs | 2 +- crates/networking/p2p/sync.rs | 9 +++------ 5 files changed, 17 insertions(+), 7 deletions(-) rename crates/networking/{p2p/README.md => docs/Networking.md} (100%) create mode 100644 crates/networking/docs/Syncing.md create mode 100644 crates/networking/docs/diagrams/snap_sync.jpg diff --git a/crates/networking/p2p/README.md b/crates/networking/docs/Networking.md similarity index 100% rename from crates/networking/p2p/README.md rename to crates/networking/docs/Networking.md diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md new file mode 100644 index 000000000..ba1b8b302 --- /dev/null +++ b/crates/networking/docs/Syncing.md @@ -0,0 +1,13 @@ +# Syncing + +## Snap Sync + +A snap sync cycle begins by fetching all the block headers (via p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). +The next two steps are performed in parallel: +On one side, blocks and receipts for all fetched headers are fetched via p2p and stored. +On the other side, the state is reconstructed via p2p snap requests. Our current implementation of this works as follows: +We will spawn two processes, the `bytecode_fetcher` which will remain active and process bytecode requests in batches by requesting bytecode batches from peers and storing them, and the `fetch_snap_state` process, which will iterate over the fetched headers and rebuild the block's state via `rebuild_state_trie`. +`rebuild_state_trie` will span a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the accoun's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and commited. +(Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. +This diagram illustrates the process described avobe: +[!snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) diff --git a/crates/networking/docs/diagrams/snap_sync.jpg b/crates/networking/docs/diagrams/snap_sync.jpg new file mode 100644 index 0000000000000000000000000000000000000000..7f2c93ae14fdfaa9408cde3d3a48c4c335261b35 GIT binary patch literal 34806 zcmeFZ2Ut_xwl=!x2-2H?fC7R_Q=}*@fYL-nEcBw%iGV02NFWH(1q1{HK8hl}NpFEr z0*Lf3p(Y5@NvHura^v1-pYz}Cx6j$9Joo<3z5j%nz*;MFtvNH-SYy2JJ0|HnX&zv` zt*xgGP*4DXF8K!_p@BPE5N8JfFfafv0{}n|P=q-H)a1X&hX8!!3IJ4D6aW?Zo#NNF zEXu#_rJBp4`rF?WqQ4F#wE%4N6rxmfR20GhB^w158wCjo2$4TYL-CjL>!4qSf|81w zhL(>06aypqfNEBNl7fnglA4N!hMIh83V-r`fSQel{rpu8S`MQpbiy8-vajPl(u-)8 zw{qPd!irvd`XcBQ12@kZUcL)r;ukMnmXlXdynaLJ<}EF49bLWK4<0@;HZe6bx3#l( zaCCBZdFkop?E~?B75pY7G%P$KGCtvbVp8&ll+;hzpL25a@(T(pDyyn%YU}D7+S-vF zon2qMzYUL!j*U-zpG2WC3qKZ@elD-9;BzsVjHeh`|I>vuL!Mk}BouI(ih?|usMr7yK+yRVc@g;E zt(Fy#0A~aWydO6Ra+Zla;ts*tbToLUT`mkH zPx8par;V|wF5=y^1@V)Zfb%3^n6NbOH3L*r=;=w|wG-8zaTM~f%I10m`K*_rSe+m-7WPAJoHe`p( zByuXnk^rG>B45{jE(!S5y#OLy^b#V1Y(PWmLhApoU#tJ|eOPxN!HNWwBW%dy_ht_* zzexhB#R-4Rb<7>}?~#C$>xR~B>NK22|E(%S8vGxfdQ?M1E{OPT{Aa?S$?mu(chbPy zo?nWy)>$z8F5VSz8t<+cEwfc7+&sL#n|5L#U8O(eDt1xc_gn<`gz@>0jj1!Gw#TVNsks#&!3l(06Rk}D;MkxB({rtmcDXShM@e>YzpO^Y#x|I0!)V_ z_h{Z9)@sGej%415dRP3C&sA6sAnbi=Mh(Gi*LaQOSh(@~OpKki4Q_~HQ?kjN+_BX0 zu+8E}#zhZ$n*K;x7r%O5^E4R#MY3>?eUL(n^WLvYH}NOd!g}j>v%(1&Y5lh<1!?Gi z)`Cnm1F)|vr6k~c%0lLe)}GR#SaS`YY`sNt8%e-Z#~9ci&*<8ZRN!J+7F7TE3e+0w z9j1W5K;*2{`28+c)72&*8^otNs<0)CMa zYAcpg=)$r%HPxo5C$XUX(C0cQMTC}FMP>b{`?-rW(ywkxK6v*vOH#D_K@}fviw&ZP z%;2aV+=6O2?(0JJ9?Pq$!jUkg!2$U%3G}@W4!-C%T1r6cJLyHKK75K?fj&U9ut6># zjTqqM^S*Do7fX6wf6%L^XzpS>G=u8?Fd4XWK2gZ|eDd|jXIOz7g%`{2j;b*c^oE;- z%nA1qLvOA5;M=`Ef>2uZ1eLfBiK&RwxDJU9;Vy|z5%2q45u6lDawY`?Mf_D96{e}Q zG9RKa+fhtV`67SP|FLl8$N)~cl5)Nm719B>YrF(VaAeF#T*D}~&7CEPkCn*T2(0wj zs(o}x9*@82L9wmi#iKT-S7pWN z=)@$v)E)wa#vd{gfdS=nB%oER>KHZ&!>|O+<2YXp#K99(|MUUt;bQfH%Ok~@^^;UQq@POI+l>G*lBotmT;2<tGYal&Jgq9eIi^U~q69p~+&L;$?#l|w_D z0^<0racqnwn3fJ4eg_OsSADf6LVlp9~kN zc8l6FtCL@c1-q$OExfkP$FDN+cko$#7T2jewlBa@G+pA#*`8(eaEQ0AigA3s$p-zJ z{j@CZu;|}RA_dvi+R!5b;)*yQV&@5r1l;AAcw^E?9L?N*3FCpSyOS;LE?O0$1#lYu z_bQAu{6A+&%kJZ)=C@80H1KI3Au=5kuM%U@i~9Dxll97#5;2WbmG%|yHWh8X2YY!$ zCD?)>ROq|a3x4`#Y_l^O=($tQG3+G30x@0luAKjkQ3%bZ^qCj$G)vB}YGyeI7&qZ8 zw6M+5zVq&{_bZoCkJcTSZkpAj+HGT5+a+IJ51HOJ>`4H;Kur>%TFEP{p(pXG!6(uYMT)gQw$R*$>T zwMYa%K@0cHyZP$q)V;8_OlB3^Z0B$z5eq+l8^+ZP%g5S9b#MP@w4Zug-qO`NN^!(= ztVwXiJC{NqRAFZZ-oRx_oIp>p?@gM8D|xQ1#xgah<)u}bVHy>dZcScjq+ot{NmDTh zyjcQUFnkRK;lQm&A&79fMhPBVn?i7YESK}ENK<@(1vp>8wPg21ZR+@%!8<)lI8VZs zj@F@`nO#}S(M75~LlazXotgkXK-oqR?=aE&#c7Q$yechPvvYfUt3$s?*~*d;l5?%c zLt-OCVsD0FDn|6OLM_{|G1NH>Gxw(X($t!u=$G-uq2`OV1QDb53zc%O&f;92npCnu z2V6D;ij+%@{wTUCdFbA+bdhRbXFfK1S3pGLB!i$^3t|RULZ!0xXsZj>Qaj*k=Y}c9 z|M4J_{0>$pV@tzd|G`gd|1}39K57PM-1g)L0qFrh^_5wt=*8Wz^z^Lh`# z;&ikXYK!;qzyuf1-r5%wuC5)CJZItK#dNtNXIiIbvrjQIWywp7y#Rkm4&COCpM%X| z;qGwWLbb;I6Q@>~z-+o$SuP~fU*Wt`bps6S93G{@PHRj3GGL$6QYbO0Id>u#ud!e_ zSx)4#f+T2E4iQw*YZ3?sYnkw8Pe;9GjN}eC9V^B#G1{FY-G}rPz3y%SUMWPVF^{L9odx2ADvqbIB|z5Zk&;}pK^Wuex*zo8*cD>R1-(f-x5 z(N691$gZm2-MX5w?s{$?#m4#CI#c$!-k3#J%c)Jas|@o2w`L*9)}wL)ifsqA6$Uv@ z+m;57Zpde$`b`;rGVly@$CDvbPl|P$D*ojPs5=fmD=_{T$DA%*;w5SxQgx-3>8p>V z75g;3x}@P%3V#0|{JNz-(FcSyaise(KS6*5kRy6FlitMLx%RC?1;g;UeMLMuU^E~K zXR%2BBZ#HU5zEug$hg5u(7`ABiM^ip^X0pcB+ukmIIR;jp8CwNz3%IdnEQ-6y|g}O zj6oR_0fe1+?I|G>KF4?7_Z;G_VK&6$Y`bsKrQ>9t(`tU;)3JMh=z(SeDH@yzBV7nq zs6I}}f^RN-(Qx`!&3;nN^7+b&>R2_a>SD(24t6`V&VBu>l4jww6G)qg1L6$}DVDnx zvF${TA^m6>?dqqGYght@zi#XpzC1_OmtmNW1yGUU%{Jr*m5Il(DGBHvf!dMLf&nDx z$Rnff)sa31DO4%m!4erDi%%wp%w18t19*L}bRCmpae}Zpbk6bEzc*RXB!#?>(tQ%) zwh`6Hi&DBrhpaLgfnm!K4u(e;XJyD>i?SgJSc;6ro6J>;pK!YKkgbP69tqgdQWugn zvLQn+ZOtbspyg}`f&|cSfPO{hAhbBaM5;&t(TNQ2g!muJ!iKbQFd`cYMgrs>QtVvd+MOdvY)vK*BNd~_z(C9h2{`5m3Vsd49Kkv0JS#p? zt$H7&9P%jTd{aX_WL!(zi6VKw`kakxT?!3mesv>OT;;XUMy3oO(jo ze}xE^$*mcOD*9bhuT(wqWuJNye{nHli9vxqvs9nU*={Di$f>q8Jt)Z;{(5-Ly@+D)5UKe<`ed}$Z`nh*^)qm#r_Z?dj z)CA9B%VMNe^yZmlTIjTKUHd+745EWfdkU6&lS9!_(oCaQ54}rk>AWZ6-m%g@T825! z&>@MUcpq%i@ImXV(x%U5E{bKj_`KHS?WsAu1IA)tjs&=LV)eoG48^O%>$J2+OOLq# zp=3Fe73NyBOz2gp2u`MR!b--f9{b*WE9gh!lZXe8CZx+M^IV#Bzg5;`zEmj^2dNfj z)RjGJ#+J1r_@Iw*1Fy+`Clv&CbC7Tf$FDuSyY_oS8P(0j7=)_>JlI z9-1JoI{*cKQOjTASbIfHcEnBPnhes0(m9ibp!3r8g+s-cY6Atm$NHhV_H`o*k4#xk zhvR7<-utO?4LbIpfDTRE=MBRGB=E6?-8p#cLEW12lr8DpxW|$&%DrI)(7xx3lJ;fX&pe4It&wIw&#U-b6+ie~7kI#Jc*NC0wx6RO=W0!?vbu9}F`N2AATqeLzx5mHSXS~o7V(!j=~t}u59^Wp zfA(oawPHg@9C)P|uEs}3c-ZAjd^#@6_e>q@mZ&JBD6b`-LmzjXhE7JWGx z>GM^{!747uX9nlu<6Lv9nz^~>P^{B>^6!0KIQMz%KWbEvP@bkIOp8<$3Ex2VO&@W; z#!a=E`?N>%plc)H*D!u%c%3aT#yWQu5a!{~{U`b1ijqTWDSmaGkNHinuR@B$K~8B+ zvYU+2ENwgLHEOId2J^!MP4pnlNq*PozEI_-daG7G)yOkJQ=#60r{{!-19O~e+)!Cq zxDcztisDW4Fl{~v7#zX%qb3Pscq1#NmqV-k(Z-@yoKH~et20oUV`ZotPSTopG__NL zNd$So&8FogKlTm`@#1}>aG=P&WF&Xxb{Cu8W#QY;**?UjZ$*Oy;4(Pzw$!dLxcIyU z31G3rCnL8oeRnZy4~c^h{N!Z2Bo%Osrl+5~y{~Zh6&2vk3{tWJ?Pm^q7Wr8PbhS)m zVpFD9X8DJ_EG9;EzPrh7F&WrPO1L{oQEA96+^sP?+b3K)elggWf#QexL>q!f4;PHI zIR&NhRM8N7?cu`QBk)%eny{ z_mWvpQ%^Ae8b8YqksiBMaQ?0K{sG&a1a3AJ*Cxxta&F>Tg2flCPlpXFL3;u!c2s4) z`zol5U5|ZwWqVFJVOdM7Y?_wpC1UDI0~&^ zD3Q4{QnegCni{?LatQkhyHR_vU&QAt*L4GIrNT}igr4`HIjuF0q;AD8CmH|*Q`$pR3r+_ z7*DRDH*=gZ{>pc8+6lSV-NlmQUr_kmRiZP>&**8W*^{t)Nf*C;i?m?=_B>yyo&>mA z-QM$>O3S35o0=IY04pWMOYU6=I>T?K#Uy~{*7ik8=L~&JolLtRB=Hu2Cd?a#!nlYR zRt_5Sf_Bj`cUwvCb#p(kchb@$M}0rf^r2F5d+jHu&;LN3nu-Ard$~`REpVmpC)$4Q z$z25(I4STE5=wLOG{9imk0FsaHpSA<>+ycQW&C_tXzxj(4LLkp=nHF&7BU(3A_qc{ zcqZ-D*_92o?-9AP#SGp1uA1ML#y2m0I^WI08tU0|`I{nTQ)URrg^5ZdrXO?A!V_DF z!ozM4(VUbJu392dx853UWcV=#y9*%g?|a?1zd)h)j{P%L1pFi=K*3^pR-h?T!f~{E zt;u27N=d)*y0qNEV4CZcwQoiKYYwKS#qR~L>!Md0`{c=fBc#10b!ILE?q!DxC~>fL zj7|}_^HMv$Ls*Z^=|{}1=gZ5z#ha0#Mj#NaaJEq#A8=CEpAQh;VE)t-hG5 zRgF{mVr6(%N-DTepJtQ+cl+x%+morz>H5P3Ofs@QKoat8p~L1${fO&*Mhgj3=}m)~ zwpM3K&0IZrU1Ca2lJNEHXgySTA&|wQ@{huV|Do+vq|v{6X$7`*8JzLgKilbPPcZ17 zrs$^^(Drw^d=V&-x@O!wEYGNVw_j82OX7tnm3+I$?Xj9VB25lM>H;iLDuzXn5hSWC z%4?E3`KWKEV||f!bjK~n`SW)~Th8=5y4y(oh=mwk+qk3~9!(nO6qvPjphD)Mk-(-G zOL)M!Q5BjTg2I$L_~S~l?Ckp>t{DBIf)}5_DRg}jonbw(;`AwiUAc-p=RK1cSekF#pke$p91H5h-|eLP|v*ZKdRI@dyEvW&}I+#($lGJo*Vx>MaAJ^QR{o zeQlVDv^y^(q5e2Z(4if}3!#Usc-^ideuc9qeSvDP^vU&4MGIT&jE6$}Zj3jN}) zT?|p~#K^5*ArCAchC9MWUZf(=adZL58F3`hOb*Cei68%kJpVt<{%6LvUWOOyh;PU7 z+=;q{4_+woLmqi@NFlvZxp`iG40j5@R!F}5x_VZkB9!XfRo+(lrK-O$X@6tXD73g% z5X{YIRs&SQGsdf%Qgm%c7j?$GioVR*SL*bSQUo}cing-03fjF34&*X>PsP)cL4i|5 zx_UYZo~bnx)t&W=Sn>)TA4aL!)zznC-?g7vOb-_Ty3Yw2;aFnf*RWf+n(+QuYjmj~ zQLJ_xe&_3u<*M$qYIXb<7T3qGSUu!hmaOt`NklCmB2dpzY;!7&OGWs^jq7^4g@c zy2&PN$~pDM50(=7>stXFj2~~&ImaY_aTgZe%smE|@t5_DYb}`6cNEDxNOWK@^Y7v! zhxqQ~yXS@BS6FmF6T||2A#o!)^piUPHn-n3cZ}YYD+`^K^C+O-WAvBh&1$O9ds0(> ztg=(=arh!@P8j<3g@K(Dh!$Nx20~vRhtuVLhiuBwtt9gugn4c;XcfO-qVug zqfW3eSKt2LPBkOfYCPA=%qkZQ2qk1rjl-rBwuKgS_V+RmL?2_FtkKsBAQ_2jMDaC1wzCTMbLWD(Za!O?3lnuFlI?`1eP^3$yS?K)gEj zcqS4AK(^4Hd{zgMyKBZHd*zDX-91lk4?L`>13wM&)KB9Te*@PVRdSzh4ArLZKWG!; zaGqtxM}NYnJ)9_2xytk=UTM{Md<8N-{2~2`v(yh0vn*LIj?PE5ZC0A`>SZ-V0Z2PK zXS3L*-TFLsHg-W(xjo=Y9ej*%EpOGkf#&o5?=6z0sl%8VUg=@2ca4*%z@Gc-GR{FWHP!A>+n-`5^}Yp5*EKNtqNiO!@Dx76OJI~|NB`5B zBKJ5C`^6#n4L_ekryYx=H;PA|2{&D|2UpHA*gv}ef&%wwpewT+Zv~~o_K@kgH#!ER z7+o*Fk6!yFh7TP&cQe+txTjpFOR99kAv@3sinV@;4I4J%n37LS75^L)Bp9i@JLBGZ ztz`C(n&BvKoF%F^`CG>=$hFH?rfw)d3Q%82EQ`$lZ%WIK=-Oc{aQS&Fh8~P!XSmh; z|B83k{=z#g{ujKn#iu$!_f-DwB+ zC>i5BL=r?WGRR{RaJD2VEO$}|y>)6ir!ytny!?tqSc|ir_hDZBJBp3$@}4i#t#gULF=@iMj)Fkh#V}eWR=?$Dau8f&&*0C zn}yuE-E_s?QPSwRpUAc^Pz~Hd7hh5$D1;!h52MCA%s=y&(9o_7vfFJ)erY*#;^y%F z$QHOqTYzU14+Hs<)p#hypl>H@;(x4~=QX4$Q(O%pD?}O~YE9RoN;QtJo14mqrn5v6 z47{1|@0Ch=UQY~zRU9g%)$N`f2|XExzla3puyf&iGIXmA{RQZiA133wYa5qsJ$s%n zjof?FdS$LU!2hlKv(themL~gmv~@lVC>$!fxlY$?)?fdrdIC0&1Ug-r$NH`H&IHnl z3IZ(~-ygufJR<=J@g-1O4OwXQ02|gR#Erk!UX#IDUNKlLAluWsJ@NocOvt}OT{^6I zq}5k6aPVQ_;16b6fU>~`Z)wds+HAu?xQOjrac`1O8TIV$I{q|{b8S6pX-B~YBY}g= z8e$*cL61i+B(Jri*35OYcyC!_&%)Ps8l510aJ)FS{0TVhG5^7x@&w~lUk{t zHRI>zH^wc3g@cmi=d13~nsW$=m`Ag2~#CD*&9Gw0 zWsUwDH}xM4dTDGendME2+Ro%vB}V+HwG4!Cacz-3yHYhS{K!wdYfo?KUuCwc{XnRNW4bV3)>l0Zi&xLk-Oq5u0s;}@I< zN}C8QqQ(5eL9iN+^Ego$F)9r9*FiQ!rwU!=-|EOUzw_DR7VIZHctiS%dM(X6G~^i5 zS+c;-2xC~`zK3^gMR|tmt8yd1)S6wb9jNmb(=Up@?!0&b5ke`(>T3WPA9H0Au(8BL zPXEXOB7}UGK_e@*022%SN9U6`sbA0=b(sD~LCe3h{}QfCbXd~plFtjSN7{Pgfq;Ns zi466K_ zbjWMAXQ!g~Tq!34eV1uKs18Pj6OZsB0cX}~o5gVk1XONO{#SdlQ`yrkk|C?NP76?x zb+`af9Fd#4M>TI7F)mSgt+xtRKJYH|rNu!?PA6LQA5tr&SWv<Dy+9(~yicRf^@%=yJBQQhWD|^dIS+bs3dL<9h1dx?-?rY4!r(KqV7**%|pLykF%HsBR79nHShk3_LwTH z>^9bsxq>p|%Ae4$4Msa3Q#pK1Q4f_wY48j@D4RzV&9V`&o z`=IaXL{kwb9KN@M%assjdB5l%yAv!~W*DyUgvzIF+vsiE%XzMbaAWfX=ylbfk&@ zoQWk3w$<9Iodht*kc~4~O%6EWWqU`!&85s{8Qpt1MjC$2c#tS<1}9$;5`Gnaz0{ zV$)Wueqp^7y=V3K(pT{D8>$0e{-klKFL~4vXMI0YeLOa7N4!Qz|MBb>V53$`Ej9l7 zGg%&WC-Jp`Hkt+ZMIk?6b^5dt9uyAODZ@8fz;WV{^1ORi#wWeWJSMVo6<>cgUSl^r z%c2o@z1aL8raVx6Z>Dh;6c{7;z9bXWcjK6w@>Vj0aB4;S{3+7HLB6}`%=%7x5OR(|a`iA2AOUeBR%CAcS{LXjYSnAo0Pw#+ z29&m1VY@d8yd>bY^+gh}xFLjXCXmVPTcFlVJQpUGXg$$Kb`pkd{(dw0UjFT~&?&M? z<^MGL+qifqD|H1mpNBHyKyS%pe;7(JcrTU(65GDdX876CW!tv~m7Y;1xpLV>DP-0E z_~PyNkDlyM>RvC40k0s!=C+?@vLV*pu|C8!f|k0VnN{X%6)KE-IBI!N<*HkSZXEo8 zqj!y|B&Q16?eS9i-Cr>ksz6y?Ixu>oUBIxb#IJD1sv`?;qeXvITJ6ohUgLC#3Y6)S zM}81KBhnks`kXS=m}Y<)U*d-$H1#;~UbZ&*uF*WlMoNx#2>5TJvZ}LW<5*Q|v+tu+)JIFreR$CuYwyNS()TI~s zj9ibXe*(-BOD>xru+X8B``gU@8+bCDc=y^MZSrqZzY5lNjI}voB>}Xsd6P)cp`VQc zIk`pex6L}geYASVm7H4h|FI_T5&W?I5%Xm+nQU-=beenl@)1k;o*&~yMR>-e>XIMj zx6*z{Zd7xguH?SnExKcE@F#NlOl)PV63tA>0WvcjJ<=L**3;~X&UW6I`vMfsV}QJD738hKBXFAC?i;0MU17U7PT6euVxE?Gcm(l!;?KIi zxH9tf^wqcsYtSWJ=39aujtW`ZEQPa2H;-<8=kYHLFN|cr{wbRBvypb7sK=*5VW5yQ zkfux!zh;e(#(-lv@UOGkzZGy^QO~fd&Z&~9;0Hs(*xDI@@ZhYoKYnB={7zK=#&pBA zn;Pfo2r;Q!ClMKBuzl3#K^9BA)4$^n|IH_qAcr&f3tfavn|9D~G;AefzZoba!`$=9 zLfihZ(km=V*@f2K`~GTe+}Bw=Zcs}b z(ns>?vGnjMKXIZ8sj6x~G=z3`^3>JWdBt4JzQ4Kolc6wMH2yAdHo@kWOppo})=cZ5 z1D|Yzd&M+}zsq~O!(}GTtS&tCZOs1jFKJa}6#VA=h=>4gyct;GHTK-MqtE$6){vSd zkQnh^J2*FuTdzFuZR|C%C5?lO3l5>p-VTmWd-X(rB#fp48!BEn#ts;Q(q;9IEjC6MQoQUYK6gC^|^@1WQHKG4LTG2hNg!Rhfc`ZutX5U;qcy0RMoj zR38oyh1eKj7rU|UF+q$|juE0$=;%8foNu1Ry(H)&{U%!fCF8^T7|S?69*YY-=%hpP z&FqKe;Xyb3-5f?d%Zq0ExwprAZ*rB_S5YT;o~=!3dC)q*H$v=3)keXMygm|`tt^l= zsaAiGb6@(K@=$ok^@;%zowKg{QHSW0C7y3$_P%E-iXGi_>H5g}tgd}6eE%#vb|2xI zBZJBI+V~ z0xGW-%HBI;#;@O~`pkO0(Li^G(NM~IP&>!!O{~~pYw~G|Z5pl(8|DC7d~Fx(tYiR{ zO~D{kL+Ye6(=+*^xxRA=Rn^MkC7MMWsYHEkFNirNd=S>uUHJV!x!0OOZh{G z%N+GrE=W-gC^0FaEK+PECvU?mE84oxst5IJ{x}`?9g;UMmhM&NL>3_{mRRGnO~&ou zp{_#4BPKV;c7K*fIQTTY-gFE(=M<&X_j7|%ahAgsfKF)nOM~o%BKK#{;>HmrtS+PEBq^_?!DLHOp4dJrlZ8wLmQoL8XbA(%_WSQ2$T}vh)ug)HQsy(%4bhARNhpxQHZ~_YoXyphLeMl-lbmyQpN&;Hy+ae!@#s%voU~9ad?sumv0`v{ z>LOX#E5%tiA_q1WV|&SV#(=882p{E?;dk0IhmDSjFF&TR2U)I=b(hJ0ffgA*sjj$vj+Ka4(jX+Tk zI`Mf`FE*Dr!(O25WCd>X^M`M-756QvCmaM=o_SYMaV^&XfG%kHnY1AS5BQQdq+Vi0 zFqX9s1THgqZJOvIA}32P40|>f;f)y^KjcJS{|RCBjlZR?EgJaBPca%{4`QB#xCyaB z0t8_Ch*7u8S zB4}2P5_{Y+8?;Op}gUbJDb@}?v)WQirUVIB-%eK)*9fd^*EMT z+rSnsbR9W{m&izYTIt`5eKd|~a+W*exm+waRU~f4Ed<)&k)1nZ%%0(#K-Lq*YXWMJ zztZ`1Hn)M_G8C;(aKZbM_$h2c`+kOMOdE&?B4kt@xj$tyq|%e7P?JI(A)5a+K{Llp z&spSDLLhP?kobGjV%bYPKQ^-+!I_6koQP7<@IlXrx4t@^x`7;5da;^(?Xoq>jNbP3 zW(m)Q(qL{`Ea%&^xH(APzR9?FYztYpOS`p6uy@2!&ONBBiaBbPvzW3Qwv_5kHtK$< z!e$ZjxCewfZAVL8aQ`CJk5!4aGA3fj94}Pw6QmwJ(t!wvU-J9df`X2qwA@v5>zRI( zTZ?^^`et!v=UD`=w%7Y!${KC{Zc#iPE=a&n7uz%t>L#`4>hq{+NOB{`36o@?bJuBn zl~SOy^KEZnGAhr9dMA+O+^6Qu*nK-3ZT7quPT{rG;(p1CS?-t~eT9n`nv#*2<8Q1g zKJ4~J#cdz$nZ$_3st5&{4l*j;)rp{nvY=%+Av>=%OmF+bF0NX1-5B-d;b%pn&0d5vSbJ^y=`Go=ew z@=N;#WM#4N{l)BCa}7@$G0bm^)z$8QYEFSoUR&URfZCfysz$hf-pBETSA&>zvOf>O zoZ9_z3VGq*^|!RBzn=X2>gPe*C`mIfaf*yruj7ocF#6CStWeC$;%{G0U5L}HGo!t6&V(aB;2Y?S+}3(JgB2wf-}rE>%yVG9MJn}r zbb4a-PF3=$7qb>!T%9Y!rVFNQH&s$*%LAc9THZtu-ebWq1htBP*`?$$Qf42doL|$! zg}gS^mnrQe{`yC#(-bvb^yCZrehn(JEo&lPBBbd>_Q>wZVh_*)FB9N4WM$ao_v(<2d)m z##Ho}^{VM${6TWXh67`Kc2CHQxSW6-f;z6N9Wlv+q3hf_v)V9P17n9i!9;~dS7p0< z6qv0Xq<%ga6n)+;XH)NVj>!|5D7G|1w{`xxN*Veyh#IPli;>|a$Rn^18>|jucAEl@ zU~JYaaXmQ@^CN!0`Z4<9j;sMEqLHj~B4+~w%us2^=zRI46yN7^9}GHv zCZBSz)U1%y)+^7HqK*kS`=TgYcAv%`_HU$fAw<~-ZP^*@VFkPGB|VAQsqgK=sclc zeSXKr7eAtRlIgC-^DB|?jW!E|Gxxh0@QCTOj+bqIpJ@32F$~Q52Wk~tGpO}BiRk}1 zuA-_t@m&$fR=^OQk%>((AwQ7F;+rSgur`wdo7RF=6Z7=woPp3MQ3hZO@R|jGlAe1X z#>Q&utFX)>jqyWvnx_=l*f&*~?Br=@9@}nBjV0z`RITlOVg?MV;wqM1+@$Yi*8V3(A9? zSl~hO8?w(>SS}|8@Z((H*dE`VzOcV`2@^{1{$z$T-mt9`N6U=lq&sIa)GUC5wb=+j zujBol(O@f{JEOadEIb|A%YtHKQu2&Wdz1DfpJOm9dmNma{86?tli3+pDeA(6t~z9~ z^n?EPT^DhzKAk)lzl4IsFRFZv${iB*i*(a8>}^a;7I_<_6l{}wBpr=4B(lpzX$PZs z6VffTYy_e;7YvB~H6yS}skD2prxueh1dO}AiOx%Uce}h;6#4D6H7ky;-JNF{#=^Z1 zVsuGTG7NGU-q2c~6%2k*p31^%rIKr5rK{sAm-+NcqZIPp*ETc9ns;9vU4TOpu>b5N zji8S!Le2`LA+3dQ-0Aa%tnmFF45O5Sp6ip$pP`pdmww`jkmi(ZwdA@t9VD@a;0$O% zaF77|Iactj^w?gyj5QXX!GVLb&a_{tX>fp@c6mZ&e#eJf^8*6pR980juy($b?fD3gvw9icp8^fVMHAG8GOPY2-tvz-zJ^ys z8%7YMFqf*o!WD}ko62h!cV{FzP&wmKxrTZ0m;MD=4K*2K=3BkniQLXZkMtg&#j3`d zp1gzeV>C*f8l+0I=UI5W7p#-SixeOoi|JDk=3n&Ym(6)jp9X>TIME*!mN zu<*k$ER)}+HRJSXz*&4ko4HPm%2oeYsi|fzo!c@Mb6vb)13_TMG_zzV#Ei9Ghv7Ok zm35rZMjB4T%df=S;F?bSMzue`X$1+;Nf92&yjgS1)YJrbR?4=JPf^&`xsveW#TmVw zK{}I<#BM=4oF^*;(T3p6PHPbxu&k>eciZ(X$*sw3Uvrfjm9ne6-M4hphx486SN)%0 zVFmRZx^*Al62jS0q9h(FdFdsW>vODfyYY|C@W_wX zOL|yP#~n!loKfxZctg$lCPSvOH&ezQ0WvYk%zQJfQuDQux^@o}pwA29;-EA0b&pxJ zg7kS@19%?Su`^Ap+y7-1e$3*m=i7%N_=izEh!d!)Iq>ULk>U?Tq6^?_7nX zZ5T>?|LJ%$^6OEF+(uW+bH1tMuehi0w;L_Ht z-YrA2O!2p&n4>)~EN|jXr}1-EIENHWTF>&XS=B+xx^N>o_0_q!)`zu6zJ2CNcD3m| z<5G=c&o<#E?j|Ou7HPU$#3bH|5)az#6P;G^ZcuqZ+y;LtkxVf3k?%ledT5I1Rmh*w zvMhhhQmtZx%|s7`5IOP6MuTvsdHv%Pa4RaHK&7EJQl8&60DN3$+Bmh+U8otO|F|uR z!r1MK4(A#fQxPxyI1l20c&NY%T{0)H;M24RRn!MQT7B0*JbahsdU+e2q>R56`NTKj ziDr**_D5=h$UzfY^|`kb+RgP@wdM?jRih9 zJS??vHZ~vs==jmPIVZQ(L(;qU{>^=#y4RNBi%$BWB&+8giVDLX32!Hc8W0;d3-nd` z&_a}Cp_IabH{W7nuen##i0=l5(YUf9C8xeV_3?cLkYl*+`S9Jhi=;>Tz5h1Fhl2ZcvjLm zN-&DNSPM~*z6|-T>NqU7_I2DJbhGAi!*~(@i`I5?1z#BscKd1r?>0Tlc zr=VuGClQM68xIWd$*yH-pKD=EI({CqJ@$*d3uNWrEMxw+h<8PD=g*!kI(zn&b{Kjt z40g&Z%J3{C79$^8E|wtNUvy{l?|Bkf1z5(>9IUu`C3m4w!)qaZY;92U(GF5!;R zVNqL~C0<(`s|}$=_GlQ7jz8u&THJ70-g#q>|Kg|GN_~j*w6z{bacPO_*SOwjzKXIw zbs_pmJk^3=cG2pr&OefoT(Egnop+9`?Sr5JXIGGAws|T5TL*bY_)H_fX5J9~Aarsn&s95#uty z;QiVR3~}Us#(@cO#E_;rY>6q2$0EVR^^Vo$!#cLG%TqW6(Z>FHg|JB}khc&hC`92B)3KHqkRZ!g06hV|0 zl}!^65TzHDPK3}25Qu<)lz@VWz=nWGQ9zX5By>@c;l;8KjM%O5!x@~6|S=@z~+C0opii zgO=&ji)J%@i4vo|Do3+WjmUX8^!{Kj5^=46L)o&hM!uU))e0IT*Lr$@$H zuWdhad1M^7;sCXc-q@&Wgl-EfjPG)(fW7g8hJ0J`-v=E*Y~De&iyz63U6Ts!=>e}@ z(#B&@@$~z=P8f5;r`&ApkJa^IhC9Cx)>B`Oc@-edJa|` zy@}=;ol~SNDAuZBr}&@E>3WI|&6E>FyUtNYL}l)H^qp@!B;1CQ)_{a0Y3$xZDGrvD zl+>Gx)Zff40UAf-{RX8yz`CT5vwV)+M^<8M%3$7*=qGL7Ul=%M&!{0-z>!5nL?F6Z zmWz4<2sra0m?;|;B{H^q$L!@AuJmT%X}m(Mm7cWG7#()zIPuzQT^+M~h#|Ofyaq>u zpb=wov_g;u65C3=lp8RCJLX!o6*+e0P~L0z6qi#io`M=OO`dLIQa>78H0_UcxG1a( zh7cN<`+RUvq}Oh1!#N;b25guf7E~{RS9(~^^Lo1O934zSI6M&8T-FTl$vB;wIdSNl zO7%w}x(nDW4M54$vguG4wL|2NM^xK;0lcqC;PN%*a(><*#|*x?S>?Tecdf(gH&n-r zk!;2cq*T7G=58ko9_Z+`Q3i%~ub^&_p91NV`lhXcA(>;!_zLYJH>n2}n>s$->|e&r z#4(7I@t3EaasJ-No2L^__)%lQ?;74qmq4;w02`CgXe{SCyn5Y$Yiynnm-cJBG%C>xd`1Ga}H5G+V|M_|?Ckh2m|Om$_g^i%!CPW6>^ ze3@>LwyDAM9nT=R+qu+C!St-RmIwSW#oIaHrAz^;HY)+G!hkISulco6EfR)LFZ5Mk z`NHwjUs0;XXqT4S{!cS^!-#WgnjF7eq8;qPbVSk$U@i#ZZEN*mwkjnrg)ccu6g zY{}sk&VvGN8M)n~%fD^)TXWs>4d(XvJjcy<;hNG@&hzoJzvP$-hOLt7Gr^e41~x1< zxm1~O@17lrc52uV|FYLaTEk`|Pev>(5^zDwEfsTvR7Q z1rEB9$puzk!oYAqTCkP3&vL|a&hQJ%g|S@9GktGgmCxPcu36g4(Z`1b@-5Z^&ocD+ ztYF5jlmbHmlurxS`!|G*jly<_v(28D@iXB@_!+#@^>-s39tXlqn#^06ntEL4EjQDE zg#0-U00B0@`&0y94qJ>gxjE$)4{1x()s@z#Q5@>(t25rR&TY;M=veb`M_j!LZ*|Us zX3OAQ(fq1Yn2U`z!u{)RH?403BXlbHntLp;Cein6f{M3_1qWlD@*)cfM4 zq6Cf+qkmFc!G6+NEi|9ag8G9&>rY~DUEj(n3?u$7nAia(0=3JF#5UgWrNOVOd0^&e7^@t$^$;` zLF}-P(d$#H9vdTn2*|SzB-iF>e?DFX?CP@yP8k!Zdh(*j06UYx6`uuRFQa<}Bf~ax zs{s~|B4itSjS@azt{8@Ji<-pOOUejspOT;Fol2{ca>|1$2OGwm!ioRTX7unqS50VMfDm^0TVsP z;TD^z4*8ZA*3zT#`8=;o=Dbx1hR45GU5e@F(P69+JEwDR8v(9I7=U%Noy;6At#T>R zMIhsIt)2~Q*aTzfwS;R?x5H@J=*v$}TCT$#&#C#IuDt8iNLK*1NbEV}DsXjfPIyEJ zoPEt4zHu~Ub8^d~Jgn=B0Pb6yF57vlktL2;M|M_-!56jdJKJI4H{>9rm!!!FjUfVc zh@-A!Qc=1C^{Q$q!Q@Tn{)HA}SL1JjPIbH1ZHw{m6`FPpja+66sWGJUuCuYbx>Tlk zor;X>gj-bvcX?4}^WZ!*bS|)e*nm_AUEHk;az-r2F)UrboEx+8Qm$n!D+)+oivi#B zropiK#7usx?l}Lj`NG9(0aMW9aFN?P4sqY`D)8JUmkSqqyrpD49pqtBa$+ss{M$s8 zXC;X^7tz=dVIbL)q|epNmoz>n!u{^~kYmcP{q@RRJ>DK`pWN1$POB8$-;Nqj)N&e` ze!G-2>w1=9!GM_>K#q#)SX`BTu!5!Y$lxv&zS;I{MFHYpCqA&kQ#H3%C-456X*P28 zi=0+Im5WwH zyQzFCLE^p)k9Q-_n%1T!b`PS0qs;C>1a>4gUx80hI#35bK_TG$N&aE*!DUd~6*Jg_ zWZD92XpHEP0GhfADksyaa1cP(P4C3-LB6NZRmMSm*I%8g|K-Dr=u&x*HZueAh!uf8 z+EZ}CwMUj3fNL)=$YL8`#K7nQ=&5eHJlfs`BJk5p&O>M_g;{Sj=&>_Mst^wbq+1`scY{v!== zrCBt5Yn;y=rTq?wx+*AM_D@~n{+Iml2Vby#7CXxz+w)r}7nKco zt8z3O^=m?lBHnrI^nkmMC3DP}X(D{rF#YB2JL1os&c4`P5qcwMcUP?q;MmnhqHW0I zt-^wnV0JT?DNZ>(7U7Eser+)y=$;|{HhJ@@I3v<{vFz0wM;*IEdR}r*_8^xrQTgh- zMpS7+MP0G$Zo83l*$7bB;UIGgzcLYIhBD4>vU4aNZp66~Pu4_grCn6El1;DS`IZP# zV+AW1j#!MvSjwbjI_$_ij5x;g0sp{#-Q;~8ogn*0%U9~=^Dxd`9yg&*OBa5tgNMah zbMY%nE91JEM6(sUXU)BG=aU_dmSlqT6b}f0%ZPmsvFcih24%*+(BXtuP*hc+ zyxu_e&8&J2JZ}_)KUB~y^gKO)sfq1k51aj=`PSj~om*|4Sx^`yWf4PBZ^ZE~(T~Eb zENnNs2Qfj#ugD`h(y8G$D{5_AGJUnQNH z!uKd4uIPS6l~QI_Ri%wYU&-Sq-B)Tn=HBRpT|A@17`vK1TJ=lyF>v5WNw%4WwkCJPtC+iZx<1c@ z#1HDO`+JJNTwan&>O8=5U);koz#v83o?;$IXhA>D6s+rC2dQC5%RBwc6^+IDZN7FY zoHooSeyK9&(94d6Tw+MnDh?hePi=ll{H{mI$ZyZMCG2$5&A7#XCj`rz?8aczmJhEa zq!{s4R>gV41t}LJRGB|}+SE)ziDDBO zzpRWAR}V-@D}J5?=W^=IUbruuq2+}zFR&IO9Z-loSAj<`a-<1 zzf`_f;^6v-hG%NFZo;UxRYp^D1tdfpTrSxgPRtV@1b#^-VQy_1Jii?(bFVa0+rl*I zlUZv_gswdHgrS$^j1a|qOoE&eQ_oeTOIqRoL8xUagqS4a4XY9dNepqXopbMauuR)~ zNv@-fX+`C#mK|8AuP|4;OO{Ke%^6BaMS&*Kp$~sA$+Y(A(&-v}zm{XX*)aQieW`b1 z4efFV>SkfYk~SL%#UKWCe(d1asjsGbePU$|mXeqo3pXljbn8-8GA`*`o2)HPUg6m6 zJW>=#I;J4vQE-w}}JjX}z!qaqi(y3>p&qdS;Xal>htNK6%tRA6;gYsw}9l$MId?aq+V z?ZWDltb6svyni zk>mt#IyeJ|jjq}W6)v}hs?3jT;_BX=2wI{ln}+vY6`^c`_iu{s9>nw|P}QoZ#IGPI zd=k3bS(&nwaPNfLI=Z-Ir^IR_{UIXs!J zrlLsdIr2mOeHD2&-SZrHzbY&$Ui@W?9s1!;JuO!aW@A~AhPpcVz)f_;@|%s0ox4_c zLciJxB{NqM{t(=|yP~O_h&6?jmgJ&GytPiYoWL zO>~C^33k(26Ik~ktNjnv{9ZPI>v^OV0LmGzL0QlwPZFALVFXqoYA9OtFp*gt zb_~k1pcw@vFVUnFHBisctKKXMSRrlqAkUwnTVzn+0a;PtlUhv%qa*3F4BuTPe_#QJ zo*3BrF>(~0PXv@c%;buayRh8oA16srs*H*(_ygR4Y!hR4BpyEr9$gguu@t4bZ?e!dzJs9fID%K^k3W@UU(dq|+epzekf3P5ApO#p=tjP>Nd+)bn>>3c1 ztVS@`pWGEWKVQ46qRK|Lx^v^DTJ8GxNlWt4DCuAoj~`|RLw@*ITnzVL&3*ICW0~rWNeL{- zK{xgGTQ=%B^8%M}uMDIk>itd6??E2S@dbL_c%rLNt26VVxyD$|;o5<+nAO^|3}5%D zy?>-?g2n7Vf;#I!g zs`em-52l2i#3qSn8TRl$WoNc9Lhm18?O?a_Z;E#%!NCDy68$D(a)pN30m&A>Nt8VK zlNYm~EF~6XZ2S0QHST`n=GkRw|CSp%^!CA%;J_#2#)X#4h?70k^@kl2IY(A4_M590 zH_?o0_mIp1yl~i;29;KC%*{#*75l&cI(AekrR8Z_VV76|BH; zI((Hf6v^~%OR2OQlHrs9bENvnT3J@0uFZyJU=jp&`o?~M-ikUY^B_Mc;Hs)7(0N(4 zLw)Sw6`~~AdbZ4Dr{ph(Tma;V^cg_uOZdzW;tL5NH~up5)6_uKa8bz@N5>7?p+1Ohgx2?P|Z1ZjJxy0>0GZ)8p& zImaz+aG5U*Iovn(MDmxNn9hHmlk@Mb^ndd2P=8tmY&y$zeDf@F9CW$C+^j$KVSLkP z50V&Ki2jy6g{EmLD3SlZugbs3aQ>M;K?GSB#i<>fF-CrGv*^K~jLDYGe6@tfvhE73 z3kl`GbC}ZnvSymgC9UE>-m}y@}v&gxob@rb;0q#@E&NgMrY7-=&kt0qz5fIP&`j zK6D=&7)>zk2P-L{0IeMWg_plx1Va(Q-xZY@qpKody@XYeYu{7|UJ8jc3Qv+k25ys@jeenVVzxbL}#{c5J za$re`uLCRe&BcVyPcf{Z;G*18>@D1J93NsB=}YCiiSxZ^LU~`%Y{ed=G~^(1Hv@XR z4uBVa+=Ix}zNHyx`Kcyuj=?If9^NfHUEhf2Ze;u2QY#9#C6AsSk6Ce zuhf8*^53y43OqMvDgR_;+VRreg}!GZlAk5tlbu>+1*ytkMQ@WetNe2e&@MR_(4mZ>I|8C z)=l5TafdaU@Uwt^aT|r(wd&O17$|c{RcK4j3Amb~=PUPw_mZ^uQ5D@rxr0~L^d~=D zY*k|zoF$KA)}^&tb}FiwboH+fEje8sJsfXXbYRglocK`E$n0l6Q!mpfPl{WirM2EW z&A0qzaiQ@{!SGu@3*i0+G~;g)UH{JiSN)Smb_+d(r{7>I2Hk3+Fnf@V<6psIJKyIHzSz z^8MuTF1hlc)aC+ePjzMF9HaN!8RpAiYn8|AQ&5JY-d7*^;hC@wQ?tZX?x&Bui)6&( zv@b>;nK<90-VF&!a~9#IrOKwXrG=#WPi(Ja?yt3)eE*!TQxT;5KJSe=i%f-JWIb;; zx~2$|s3Fpm7Leh8?(UpF>^*gip_C#$ttMyyCPxlhQ_<2a57BvZ7AOk@RjgeEJ6HQS zjXeBx?YhxsGC!hQ_TMVHIokNpo+K>5g~mXFu@tRV*kOedHe^>zhzP!3+8(8p$(f># z)eY+xT!?nUQ{IGo@~yaQrL}&J`~3d$aTagv6Znukyk}?+VpCR$r`{n2;h0>{%d`!Y z^UpLy#9wIme)Fq@qn|2Ku;8KaDLV^O97M&qz(T(WC5tNqVso4GwA?r{b${3iJ;&-N z+kJ>TyW2BeuFlMPe)gftnHcY!_9da1JI|h6JANWQE5_|#L(HHk>P159<)tEb#0u(S zZe_NU3z}OXDz;wrp6@372j}eQbY<;Vu{4+*Kc@!gsmeiSAv~IN6^W;ZH?V;T!FIP%N!h3C4T}rVMu20kD?Lirvpkza#*LB4CX?Yb6{; zuGvp1r9ej}u+@P0>208K1209-u&qGjXmQVu z!oY_`=%J;0f0NZVPq_HIPl+w(luDHk9`{b0ksNM-fmUvF+30VnzhBNz6oJ1PJpOeE z@n;K|waXiD^}>+lW$RhUe+xfjZ_G zv)^7lUg&giI1#3N%(a;Nz{*F;GALassKoM@3&I{>xGOvCUIs7D+OWRKrYp%l4(DT0 zKCMjW>Dk}o)<}rB6mXAAQ;bPd=08lIR!T!>>(arJv3-w&!oCsZH=u{Kf}sn~bIfHb z_Q|Zf%-YI{mj8N3BBA`r$F0+Gn?8=l-w47s*omGv^s#!@{EykrJJ|h=7)8{~<_0(7 z;?cA(GxkwvD{bz}^F>gFZ+j3piV+OE1!hxx$pq6Rx=ne;PGItt}VbSYxvpXxAlA zwUeo8Rlpmf9)_fxaozeCU&gZ(>BS4LirGTV${!dFqy!dtW;|MY0I{GI;HC@aKvlq5 zq~N|4qfCPzSYmRO-9D#;2JZu=eXY1|Nk+jRcDM6^f688C9vm0lgG~GM;uiJDiu9K| z4oFOpe+O7k5|)<*=b4V~&oiaki2Z>}Pw>SEXkEU;n6!8<*7dnIcU>~OT#e?m6TeVk z_*DUJ6U1f%b`PS8rhwCNM_?7}Yx~MMVFWqr=(%%`0yCb0;HE47UI*XgaDvp?gQ>KV z{CGATuz-$$xEPt^h&usuvUH_!lGb0J2Gaks3|#?U;B7Feimw8Uz@xx6pe@xG*Zfev zplEz47)B2SiOR)vZZsVHh1Ljvc~9s|^iPjTi3=^gLhuUxl|cho|UY#S?lr z+5};79IOf=5JR$>0Ea)>xWh(}B9Xve$KmF+>ZYh$DiE>wICS}NB+07w*)^drY>lTl zX9x;Fx&|aLVabbVjB>=3i>Il|1C~vW1McQFzTtF^jEh~IrH5{jJgW;JzfByly=dA8 ztKg$13&ZGbPzLn8Y%bx;*L|?lfn5fWC4Zxe%-4i|_38u1yno&DW`VE4r_`W8-?>csja*rPi_mp{@pCid z@C(~lA)*B$7l3PIR$GceEL{vpZ>hbMel+*{BY9RgnIMdx;00Pvj#j|wX3VLv&FRq_ zms;G9i;L$dM~gpDe1u$^@NdFJ1tQ6BLnl6%b|fJq>RrSwhSFxgY-uO7Y$=*-ei}@e zZf4^cF7BQm>8P`RVJj7ijh9CxfH$nll^|Mf7WF!X3G2Vf;*%`}4qLcONtaCzGnpUY z(5T*a>67ydwW1}o8AKDEA~s-*E)6mRODN@HZM>~ZpY6@({)i6T$rlynZk}AKZTX3h zd5od;Lh<2EI5s*r)q$)C3b4B;(i5UY1PyqAvE}c&X`7qgNaC4TxCi%*z8=4Zdyh0O zM=IXErG5m_4E?MRpW34L9oy(<^){F#>hnRHcvmRmty?f$00ITAl|k~ zAgyT&Ll<8tnM6NUqHRShIB7f{omZ5z)}P5Ay}EkwSYc9x5c@z<(ZbHk;|dor0X*hv zs+5`UaVNL7Hgz^*2y#bSEV1<=gkJ^1>&oqQYm9D45~7h#pp;V<3Z#=}lSwO4DJd3l zwf>K&X7saFAF7>~@&^YS=i74{Y4*nD@ViSM|001hoY4@wwAq)>*G5A^&u$c{E*WrcrH6HoDaihtQ>!lhpA zG7Yx6k(Kx>Jk}hgZyL`-9y&Ah{P>@271}&-m^{3BhSJfRz>ZC55PJetBMllS37TKi z7nSs~kal|CG@7ecMlz}~V=9TpT~EAirRjre#{iDJkUkqFJnGI{WK^^NV9G2vUcQIP z<-OT3u|q&QtLpIEs{)|ETRczk@3i`V#lomKwGaI=- z@di%OY3R+`SSn-~(7a33afStXmb?WFfRck0vU7o-qeJ|5RPJ~*gh5T{<&m%fX0Gc-GvBR=8TxPRMe zP|;-DiG+t>vcrKS+y=Z=_=vLMpk5;(s$}Nt`mm?p*vWVJ6V)0wlE9JUHEdAVIDRV ztl~O;R83Me%<%`xM58Z0gZ4~lK|*uQov8r)z!FV!ZEM@0HY>_E=t~AUs6F#? zechm%EO0X9992J>s*pDirk{k$o%rN?At{c!OIE`HHxUE(Lq*JWJ6IePPrh=?S&Nqz zV6iv9mu5D~ekb|iiJh=RSL7e(I68k%)eS3BhiJQJQ94zBFl;b4+8(NhP82h=))HD) zSbHYBMGRO_H`Y)d5;dz*c@^Pz@Jowv&dtMe=HkXYK5{kH>~Uh(p-CZwSO}eiDhqhi z3MViE%k7q`?u46oUq3!wx01P4i!SjL^Sb0jzmcb%rQcpQ#!94|V7HSdmuxEFp5I$@ zSp7I!<%e4lxHtUkScl~#?8xkRziY7$Q!IAJxex4I%SWjJ2yzlhGnEq9(@c2IsZT1*P|}_2{$vpBJDC8r{Z>@D+^?HAW*Pj_ z-m7xuPfUwnI1>m`$qY-IevP(Tzz~>A^Q*U`+v@&{;w_W|I6*_ zdwlP=i7fATfx-q8O5 DE0QH{ literal 0 HcmV?d00001 diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 160b5b57c..4a722805e 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -126,7 +126,7 @@ impl PeerChannels { /// Returns the response message or None if: /// - There are no available peers (the node just started up or was rejected by all other nodes) /// - The response timed out - /// - The response was empty or not valid + /// - The response was not valid pub async fn request_account_range( &self, state_root: H256, diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 09e634c10..acea3d929 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -146,7 +146,6 @@ impl SyncManager { .into_iter() .zip(all_block_hashes.into_iter()) { - // TODO: Handle error latest_block_number = header.number; store.set_canonical_block(header.number, hash)?; store.add_block_header(hash, header)?; @@ -316,12 +315,10 @@ async fn rebuild_state_trie( } // Send code hash batch to the bytecode fetcher if !code_hashes.is_empty() { - // TODO: Handle bytecode_sender.send(code_hashes).await?; } // Send hash and root batch to the storage fetcher if !account_hashes_and_storage_roots.is_empty() { - // TODO: Handle storage_sender .send(account_hashes_and_storage_roots) .await?; @@ -358,8 +355,8 @@ async fn bytecode_fetcher( peers: Arc>, store: Store, ) -> Result<(), SyncError> { - // Pending list of bytecodes to fetch const BATCH_SIZE: usize = 200; + // Pending list of bytecodes to fetch let mut pending_bytecodes: Vec = vec![]; loop { match receiver.recv().await { @@ -418,11 +415,11 @@ async fn storage_fetcher( store: Store, state_root: H256, ) -> Result<(), StoreError> { - // Pending list of bytecodes to fetch const BATCH_SIZE: usize = 100; + // Pending list of bytecodes to fetch + let mut pending_storage: Vec<(H256, H256)> = vec![]; // TODO: Also add a queue for storages that were incompletely fecthed, // but for the first iteration we will asume not fully fetched -> fetch again - let mut pending_storage: Vec<(H256, H256)> = vec![]; loop { match receiver.recv().await { Some(account_and_root) if !account_and_root.is_empty() => { From 43c9309c2420409d53a80f362d5a3d1df017d732 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:11:35 -0300 Subject: [PATCH 276/345] Fix diagram linking --- crates/networking/docs/Syncing.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index ba1b8b302..5a97590c6 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -9,5 +9,5 @@ On the other side, the state is reconstructed via p2p snap requests. Our current We will spawn two processes, the `bytecode_fetcher` which will remain active and process bytecode requests in batches by requesting bytecode batches from peers and storing them, and the `fetch_snap_state` process, which will iterate over the fetched headers and rebuild the block's state via `rebuild_state_trie`. `rebuild_state_trie` will span a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the accoun's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and commited. (Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. -This diagram illustrates the process described avobe: -[!snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) +This diagram illustrates the process described above: +![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) From ffb9e68cb8e123abb6a122919b117172f8ac4737 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:12:17 -0300 Subject: [PATCH 277/345] Improve spacing --- crates/networking/docs/Syncing.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index 5a97590c6..cd3ef0f68 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -5,9 +5,12 @@ A snap sync cycle begins by fetching all the block headers (via p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). The next two steps are performed in parallel: On one side, blocks and receipts for all fetched headers are fetched via p2p and stored. + On the other side, the state is reconstructed via p2p snap requests. Our current implementation of this works as follows: We will spawn two processes, the `bytecode_fetcher` which will remain active and process bytecode requests in batches by requesting bytecode batches from peers and storing them, and the `fetch_snap_state` process, which will iterate over the fetched headers and rebuild the block's state via `rebuild_state_trie`. + `rebuild_state_trie` will span a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the accoun's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and commited. + (Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) From fa81f677952dd54aa77c3674794a7ee1e42a95f9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:12:42 -0300 Subject: [PATCH 278/345] Improve spacing --- crates/networking/docs/Syncing.md | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index cd3ef0f68..474b31b86 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -13,4 +13,5 @@ We will spawn two processes, the `bytecode_fetcher` which will remain active and (Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. This diagram illustrates the process described above: + ![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) From b7197a82d4ac8ea3da17f14cb9a3052f4ee1f05f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:18:06 -0300 Subject: [PATCH 279/345] Fix typos --- crates/networking/docs/Syncing.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index 474b31b86..da78603bb 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -9,7 +9,7 @@ On one side, blocks and receipts for all fetched headers are fetched via p2p and On the other side, the state is reconstructed via p2p snap requests. Our current implementation of this works as follows: We will spawn two processes, the `bytecode_fetcher` which will remain active and process bytecode requests in batches by requesting bytecode batches from peers and storing them, and the `fetch_snap_state` process, which will iterate over the fetched headers and rebuild the block's state via `rebuild_state_trie`. -`rebuild_state_trie` will span a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the accoun's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and commited. +`rebuild_state_trie` will spawn a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the account's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and committed. (Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. This diagram illustrates the process described above: From eccaf9eaa4689122f0b9b188db451e17d38e7a2a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:41:44 -0300 Subject: [PATCH 280/345] Add doc about bytecode fetcher --- crates/networking/docs/Syncing.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index da78603bb..312ed2c45 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -15,3 +15,8 @@ We will spawn two processes, the `bytecode_fetcher` which will remain active and This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) + +The `bytecode_fetcher` has its own channel where it receives code hashes from active `rebuild_state_trie` processes. Once a code hash is received, it is added to a pending queue. Once the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them, if a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so that it can request the last (incomplete) bytecode batch and end gracefully. +This diagram illustrates the process described above: + +![snap_sync](/crates/networking/docs/diagrams/bytecode_fetcher.jpg) From 559076cf2d4d5ffbb5197f1ff74325236596da6d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:50:25 -0300 Subject: [PATCH 281/345] Add doc about storage_fetcher + add improvement idea --- crates/networking/docs/Syncing.md | 2 ++ crates/networking/p2p/sync.rs | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Syncing.md index 312ed2c45..fe79096a3 100644 --- a/crates/networking/docs/Syncing.md +++ b/crates/networking/docs/Syncing.md @@ -20,3 +20,5 @@ The `bytecode_fetcher` has its own channel where it receives code hashes from ac This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/bytecode_fetcher.jpg) + +The `storage_fetcher` works almost alike, but one will be spawned for each `rebuild_state_trie` process as we can't fetch storages from different blocks in the same request. diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index acea3d929..1a87aaa3d 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -309,7 +309,9 @@ async fn rebuild_state_trie( } // Build the batch of hashes and roots to send to the storage fetcher // Ignore accounts without storage - if account.storage_root != *EMPTY_TRIE_HASH { + // TODO: We could also check if the account's storage root is already part of the trie + // Aka, if the account was not changed shouldn't fetch the state we already have + if account.storage_root != *EMPTY_TRIE_HASH{ account_hashes_and_storage_roots.push((*account_hash, account.storage_root)); } } From a07f35e9409594e173f831136a4c5c9b14614089 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:55:34 -0300 Subject: [PATCH 282/345] revert uneeded change + add doc --- crates/networking/p2p/snap.rs | 2 +- crates/storage/store/storage.rs | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs index 1b98345f3..499934754 100644 --- a/crates/networking/p2p/snap.rs +++ b/crates/networking/p2p/snap.rs @@ -972,7 +972,7 @@ mod tests { // Create a store and load it up with the accounts let store = Store::new("null", EngineType::InMemory).unwrap(); - let mut state_trie = store.new_state_trie(); + let mut state_trie = store.new_state_trie_for_test(); for (address, account) in accounts { let hashed_address = H256::from_str(address).unwrap().as_bytes().to_vec(); let account = AccountState::from(AccountStateSlim::decode(&account).unwrap()); diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 471155061..6654cf79a 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -896,15 +896,19 @@ impl Store { self.engine.get_payload(payload_id) } - /// Creates a new clean state trie (with an empty root) - pub fn new_state_trie(&self) -> Trie { + /// Creates a new state trie with an empty state root, for testing purposes only + pub fn new_state_trie_for_test(&self) -> Trie { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } + // Obtain a state trie from the given state root + // Doesn't check if the state root is valid pub fn open_state_trie(&self, state_root: H256) -> Trie { self.engine.open_state_trie(state_root) } + // Obtain a storage trie from the given address and storage_root + // Doesn't check if the account is stored pub fn open_storage_trie(&self, account_hash: H256, storage_root: H256) -> Trie { self.engine.open_storage_trie(account_hash, storage_root) } From 81ae7ed9781c8d4dc49791be67e387a1a1b7d8a0 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:56:19 -0300 Subject: [PATCH 283/345] Commit diagram --- .../docs/diagrams/bytecode_fetcher.jpg | Bin 0 -> 24337 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 crates/networking/docs/diagrams/bytecode_fetcher.jpg diff --git a/crates/networking/docs/diagrams/bytecode_fetcher.jpg b/crates/networking/docs/diagrams/bytecode_fetcher.jpg new file mode 100644 index 0000000000000000000000000000000000000000..5d6d7ccc64fd342983663d26c530a75b39598d48 GIT binary patch literal 24337 zcmeFZ2UJr}yEnR#E+D-NQM%HUs)$5InurvY4pHefA}v55O792=C`bnZDUsfZbQS3( z5~L(3(n$a@LP);&p6`73toMJ;z3+S0Id|Q4*BO|QJ$vt&J;Tg1&+~hJ&!&8#ECC$1 zbPaR?8X5r5r~UvGEO1)~=H?6lMn=GS005W(niw~Lj{1#y2_Qtx06<$n1JF|c()=ka zIPtG%X?+T4|Mfe7`g0-WJ;2EXplE$)X~cjNoHVqYG?YF7M7@)q=5OiGMSl{_30gXO z21X|4lPuH=8aaRyG_8MT9gi_A~be!~DXD?o7;5L28DCWbX^enZIN&H4_ zH?P?^UPAfNlW^vfeEb4}LXuL_GUv{#sH&-7x~!pnQ%6@%-{98Wd-u&PEUl~^9G#q9 zT;1G#{rm$0VL`zWkx|hxv2pQf=`S)evtGW+E-Ef5eN$Fm@wTqMp|PpCrM0c6x37O- z@WaPX6O&WZGw3g~7%Xo2+xL|pt843o-M#&v2fv8F5C8OwIx7FtE$ZjL+b>RPzfRE6 z(b6&g=@-q3z&{=5q@zE3k%8;FDdR&QZZV~2OguMI3v0WX#g)zQypNuYpX8HJ!AcVT zbnS0F`^P#K{-5gE|LNHO)vq}~8#wW|M0?@{Ej=wQEj8v^3PoM9T?)0g_%(ybSQ) zk&zV?zzv1*I7l4>yD7#W@&sdNGV71FB0O1gftwOj%59RWdDL_w*DFs;mxp&M% zv(HFUP1!P{06OUd2%^b4m=u;m^Z;RU$O+Pk6aZg1g&-cosi%QZ8VbPXaC8?j?nuZd z^Jt_{08lYmXz<_-1t|Ks3?|9=gUDbAc>EgZ+W$Q3KQXT3c;SOc;?Z)x;D$XP0ojM* zhDGRf;$xEFMhK=L(Y*2WmZY4P`YjhN$j(aRKI z3UUvrRd6C+=D$3%Q^x-LR!GhAK`jMHe?$C80UYvU5&O$mAOHLsHUr-ETNDIu`V!d5 zuU!h*^H+=F0U2Z!1#pA_uq%UI5`-^=%+ykka|JN7y?gZuHmKG~zp0M+?GcHPgan%e zE8Y{gmPK6!#jjSP$!g@!WVSZ^&zN|!a7|Xpx(QJO%4zpyRlVLrLvp_4#Yi&F^-d(1aqG9(HkyXU4*sKl&{3wu-Qq*->9x zZq^*M!Sv0S8|iYHcst{(gL6s)%1p0|vl5f?FIMh`ebVBtpKv6mzJ=@Z<+H3zY&NpU z`yPo|wC8yowO0ykfsCzEvXqRRo;dbx7m9x^-K7WM7CK>XH&A$E<_;?YW1lHeXnoHZ zzdBP*0v9*EDqa`*UT?=DY_-zb*^-oCWyRle_lH5p?w1r#Prc#Ay1}0OQ?PU#Bt=t| zAbfLD)eU|QJ8!*x#mmJc-)J)3q&8b1qsF_sM*fw9*=Odx8;1u4?8&Rhu?^!{pDqnGp1(WW|23K_X=+A_mxB~srd-ml_~{${oD6yyubcjByu_#6nXYbSIjpxW zv^u&PPXSEP-Cy(F7u7=Os~ z{z?fapCFG4yu^cO`dZ;x$5nqdYah04B{3%-q>>EzO5mPf!E{q@UCx< zu6aInrTx|>yZ_7nI9E%Mi1~4g7^0Aig+wo0Mw6_{Tro;8XrB@+$z`%8Y%I%y|6r>E z@>|qU?uvk+;G@r75c-%jS8$!hQt+NW&KXjF<#&z7s5hVt~pLsMrpZ)T|H`SIUNt<_0M6UVC{YV3;UFeqB{ZjMu z6z@|{M{g6KDcXL(#Z-s79~p19QvgXl6ADnPJqSKbSkL$|y$Y_|A!UeYll$3EQ-F2F zw-)&(G&(&0<(ZK({{Ib0_Y)Nf;`e8d6v}H4@hDgelWZVJWvC^EI#gB<*%)}JjE#K7 z*xKBx5~b_r@*5S+H-Mi%uB!GMjSu4?AmjGu)qA@5h(2ZR5x4ORF%RW3Eq2{6zO0dV zIx_0312x8+aC?)1kG)VD9)ZuG4*8D=g!@h`na#}`S3lWkcMHO03wyl)(NjswX(tL{ z$Wr%SiNh4IkJ2g&JspeBUas&$xf^-hB-kZ4FOAX@eS0zTO*ug&wyP#KUxy!7Sh-94 zPX+GU+8PFeU*5=&dhGBe&%CD}!muhN9jW4~YUqH?f$e~d zI#aCU9&~wplxj=cc9TpJHa4);8}Y!A#?iqSLRPH5k)AnWG#q{r7n6Tq z<9np~Of$rJQkB0ce8%8R^pz4Tu~d1SBK=?vvm@gJLB=0YdeR*N;@2@b>B-{#%;wgPM zrBZ!&OO`IpO5d^Rl|+5s0_r)ez zY_#-}2xrq;?-r$xUJ!8BAMR+{RpA;``q+9ZkXbAE_mgiDrJ~+Y=pQMd`9_RoBGM15DkB6fH$Wl9n)W-E$Ms z+@YutrP?5Bi*g9S@AJ(Z6y4l|_lU@w3^nO|ly81;hMl|U=~e#s`P}|HF(@9tlmu%z zzDW-uwUV}yi?*_j^|#Z4p4pj>e{y;E*k_^h3bt%Z8K)SXCothBj9_y;oc;aH09tPamgJ+d+SqQ;x5yvp&I6HT$1Ay1vl0tyw}hqA;nANx`F5DcF=qxg~Y= z=Iq>Tt9e7dO08m+>zDHkwoCGOPHv8sHaxE~16{m;)&FfyoeeYy&#Pg^9M`MY?HTao z!c*zcGMm-0+x};zUZ}jutxn{x2yPxt_mR%7`>J8E8(ttr|2weXUSe9)2^Z?nf4E$g zo!J`X6YBS&kh82cwe9C-g`OF|JEMkU2333ZQZi(bIMO3cOkj zLs0;xEieVhJ`em(i$8>td4FwEHAx-eehNU;bE5#HajL0v^60>kYNRCLsK??B6oBkn zPP=kTb`3?eSge;m=7I84A4sH(0_^GNkz{vf)5-Db2~?FNd6EMBE-GMuCG$Uz{GS_T z^}nu&L-F@|2QcEpx9fT5$2B3iTwl}l>jZ`@FE<}-j)KD$^X7}^WOm}{C%Cs&}o!2GERe`GN{m_Ehe&I%kWON zp+}`nz7Ww_a4C=PoP^oH;d4j72lq3co(X3fhVcf18dFHQ>I23{r%BorpiFR;wu!pP zXqJr+PZ6o`082g#1=t-)A)1J)P=JD+2r#}CPhDwp8{e{JXg8pLDW?Jvf4RHMpy;XA z|8J|^N0UVgkGv4$mnguefpiM+@dcu{0qli5lm`8_oIw0mLRAIJ&{C$TldE%%EC@ZI zim@V{KLUuu9YlJR_1B`Onqu+45JV>w21<_O|08~`mm>)AU~J|e4Czq9o+=l+8K@h{ zfDCPq3^I@k{@5H-CFyGFA1O#D6hi?7ivOqD|F%Iust~fC!u|0ZcN1b?L>>Fb%hnsT2L#0E8rdE^ocpLUc*8va>l|>E6cJm#b|zhLoCY zl)utruiDz!)|6V9$Be~~ln{n^VztGKflr^V)15fwF@pLlt~UnA6*J3JWU_Ny818L4**%^#PMBxY*u*38kFWP|9H?@|W54zX!=J>R4@ zO&CWZ{}7CL==PR83aO9T62!@i%G%lVW@e+FX)2blFI>vOVw)!Y5@yNMJl1}EX6sVI zGM2g{F0Y>6UUFCwhI^PECxe|445I{S|AHK08(9Mr#{NV`D{g^$>(-i^Clq6)@|fkN z{R117yXCI&8=UYO+Pchg4J7an0*tGyh?dQ5TD%B-nP|dD{||w-ji?6oUc~H?C8)u; zcahE2p0Q)PgALp2?;+2wjsGz2KP)jP!S&Hs>aBP(^MvY?6`5XsR{rv{T?PpUl0@Mp zb}ly*hvwb48$yG<6o6~Pp1BDh9N{TBU5p;Hov&Wp?s*Y)|Hbc+JM7Yz=xCSziZK5> zfc3F@$8k8qg`Jm(!hzy{Lbw)Tvb|=+Wpn&MD*v2cf^6u;O)z>`eeQcwS+muU#D(jx zBF|dINi2U3>%~H5AqQbZohldeU3iPn$6IEdE0i{@_c4{0Zlab2Lr00Er5T`~~SB|LcDi*`OsJ*1dSLgC5UAEZQ4P z@*tdKXZr$s-W`Z2txnpR15!7=JToC%ifWfe%*+!9&k2|?zyC%7+(*9*Vi&uOBQ+3x z`O(4KWw<%hP!AWMO1UO}YokHT86`fQCPahPhq{M!7ANPT@8G{!1HYkvLI3T3Z5dDi zX>~#%dEgj90q$_4Kd4ei{_ji27$d1*5AdJK7CuLo)a`Z>PSxmZNBh8sK6x#{+h7mS zZw+W-{=sD;bsxN=JjOV>|9>NT5r`Mmvk1yG32|J7__o%ig!u9U2Rgnq3IU_X{$4%W z_;L6TshqOwi?NHlhFuI)`!9q7ROq%t&RI;OI8ooS;H=cAz440zm?Z@~)pY+4QThYp zQ^YGhUHo`Y5HDGQC3+lMn#Lw<(2)ADW9H3MoHN%uMEz_}pIrUsqiuNki+8C--T4PE z3G%O_L5xJb9#POFI(BO`dVk9+g? za1Z)r!M=N)>4D|D{e0ZVv*}f0hz^7XNy(!p-_5?gFS*lJThC&6DO!A$Z1jyUv3Eh z)fbq=IyVeFes2w5+SoAOH?wldDOutkJ6GlD69y019PG%mkcNzd3Wi4iWwu|LdmH>0 zKOW~lxTw&TXJ0g@Lk0ZH1jLzVE|g{ej;7)#ic5@JK{#YI6*QiMjhaj=YJ#ywAk%Tb zmi-6^FJC8yGMPNjgaj*WyY?BmM{NcSkFN_~t76A2AH0BKEO41w%ay{yxmuIGhgXx6 zoS~)hAs>tD+$5C0s+jik=SZZrU%1J132=!2eTRyr>i*jOy(PKM@v zv1?4KWDFli-h{Zjia*SV9cu6!xKu*SKeB`e69=&fqBi*qG1Z?<$)KR6)v~Hu-l;Qh zmi6&akZQ-pp~tH+C!L%(!+xpmmGS@2IY!~*BHqy(mea0!oGuZurLX7KP{A| zCh>!F^5KtbEkRx@uG+#6RZ?}%S}6@WpGR<$KWVX{2NChQ@iy2Oaf@8d$eur>+0@1`?VQMFT^ z)s4c!NXh0KPEGCcu2#j5`a#~`B(}t*I^!n~YKZ*~OEZ&a+UpUVO%2&^-S1jj-p-7J zI71Ks-JUi6`Za#NBAtyH zGnITVcCX7da}N9AHHXAsG}}CBWqv&TG(hwE5fd@L$66(LFb-}=U(igRNimDHuJ4Q* zjxK|~vDQALq+qmXrz3aMfphzk*PD(fkYt3Af+oMul5nnhPpH(MVS-MJl3S)?pjN*3 zh6INpY-Zs545N;#q5f?r3pW7)2nRfbNR@Hrjjo#7xF4M)2pnuvq$!qya7uH|6 z@;iigqw6xEXUz~dY~n6`vng|8yyT?F+3aoH;kD1@LO0O-clE7?)JGGyPBOm5gpE-E z(S{>C(M4nQmb}xrV}0h8@`fXc_7?JJY`1Gxv)JNrN3uYiPBON{sDtC00I&nUdvta! zj5VV}8wU}voh2%;F&?>xEN-?qO%*v)H;B44^yY=d4k?SAAA;?GPTHJpkW28d11F+I zB}og9fq4wvg5M8u!yOG7h`nnG33X>@XZB7ruy!j}v)L+=72xP{q%(M`o-YT*C@&`% zxUwB2%58eq_bo}0 z1gT=&b>9l>H^|ezA6Z#Hwe1>;7FR_5JJ#*(XOJn{Wh-J^ykTG zaW~BPAA#B2(iHO0xKQ1#z;uP#N4pQ>Zb5xI>oyvhSS`jCQB)m}#Wlf|D!anRV*PAJ zIoEp@_1!RohK*h6M!H7mi&c}Oe&um|-%LQh9}0Z=H%AF0i%YjgkL%xj-H<(1=2B(V z&6BCSxD{{<8z}iRX*O!^$unQiN4Y#o(Z3_4vwI0SxP4KDX2SDnd8v-Z+LnnK;RIQa z9KFdJ=gc54=jLqwco@bY7ZqIdz3W_7FBh!&NcQM*+PvSY zG|X8$=tutI+kuJ5-2f$4D0`ILILd7iBYq8y)HHkpi|+SlX&<_JUDQtcPOiK{15u~3 zD$~@^m4FE{k%`i28_;~mEdVgWmX~+>*tv;}H;KNbq3O3LHXDRP`3|qUxuTm5c&Du#8}GZOi=?kE-i1Sf+i8=($>Y990&LcQf67 z{?#tdVspXsSy93-vlE{TU)ZV!doA{$BI?2>5cO0D|M*mo%`k1>7*Y1<@tt}b!!XRo z-0+i;p}>nq1Y0jrgIL`Q1 zU4mh2K}fX?1Y|T>OuKS`>Xm;wF_7i^tuJgu<$jVJBH+>P z;zvsp`1%}SZ~Nm1Ml^(jOhZ&2v}eXoEi(H-PW9q)jR$IB&c;L?RS~m;0U2bG)EH#2 za#)3MdspwQUhHoj@$>*e-O#)Ji$E6}k>wzYXjFtDT}&{;v-h)EqkD}{)rVA64>u0y zmCJe8ETxScuwMOM)%44~PLPwe{XSp|O7f-vuE-P%c2_U|YITF;Pc{j$*og)&MZ<_K ze)9>8!DIZaRj=^K*=(I;m%CKTiteYcXnHI-7Q{_rcPMe|5}0wQZyNKTgK~W-3>Sna z*^kl3V{g}9UOK;>m@d3!w>oD5h#spTnD;3F0Bc?jB%SXRbtv69u;;BE=i9T=xF;`R z5ooqJuk6kiC@~=+%cI|Xu}`g@NA7ZGIh>p5u?&v1MyBlN7$Tx9?s{ZvU{bi|A#yiH ze%(sLSyo#)yd0vdSInhLlcE5!w4c2V;m1qRI51b2s8Lle!|nhM-KQgLS~pcC?|+4* zEN*CE*V_KN3jMP4qht;x$EY)0to9d)DFlnb8SBhk+?J0&ip`Fe zx9%PpUsZo=SLV`tPkNIRDMewta7CgKj%u?qq&r}975l>Er~RsLESoiSP9gVGxI-%( zTh-E5Q$G>bngek3agia;nHF3MG)%HTSa$fkaYF+$WXaXuzP34S508WX4mIt~8MSe1 z8@=z>nT+RqPbzFzC)UatTnUB@f)lz#CJV+pRLiZbFl#=e&{CtTQMn7Hr5IbyuVT#uYigBvd-W%4OZWj$1 zHd@Cl-TB(TwWel>J>6bBcQf&fyiEWZbo|;M6(f$zj|!ujWjEgu7!y=Cgk_ESH}cLV z6-@eV*j=nHj;d0;8?xP!`a_BDZ0>yNvu-%rb7nRZRB5dUNID2U76p162-!X@`+nw1+_ls_P|W?7!C zm*K_g8#Qwj61V@$al^d@V72kv;#J#Nl zD9#_~Fjap#&A6hb_b(JQ;1A>RKjxP({`yTl%Sf5|=ime6TdEhhT38#6a!nHL?F%^c z8jfGnl4udPXfQlt`rb6QQx@&ERTA%?`Y?mfuIujW^0Nl-k+m`JcNpPec)h18jcEAU z%JsIv1Z1_Mrry%AD2ToL;Ty{}CCIRgFI^MH+CXK9+kmd_8t1J`Zjbc=Q0kP$x_1_y z_83A_FN!025 z`l%g3V#d)$7;W!@&^tnYgumCF9V5aiB=(ZuszRUGx-f{#4XvIe@*znrCpHgw`!Uk* zN)xFU0I`}vDy0vg_@;~!80tO7Rt=N!**Y-ZuTyR1aAM4u?m$avQl&j4vqeAkbJxAZ z(%CMZz@=K;cHqaicqFnbhB{qR0lsEjV0$k#8yiWXncDjJkLL{AnE{02t$V| zVGPPPhUj)NmAZTjit+`fO5Cj@Mh_9anj{&5YDzDg2KbbRhS#PR%d~1fXOKo_u6w$j z)kc3*$;Z?_mW6V2#07ej-Ts*qt}b$>&^OP#_{4 ziak>Ny)?}O?9WfSbe<>Q2nswIFL8=+$twQ+_zM#(+3mQPOb=Hhx?qRJ`$6pf<6umq z?6BJMoffo;o%gDIo1TPv;l5jmesPX(MYM^i8l(gJtbqNwE3ylX8z=xfMl355Z<2(M zn>5Yan5fubr1p)yuI}TPPIK*>R(tBC1?b`Dy|zw2@(0LC|3wiPdI{2N1T}$d8VkTY zu6IqV=6LXDSY!Qz6!KA@|ASD0;FeNo~AC83&EQ ze8`jc4mq+(Qu2c8qP@4Qhd#P)3oYwp>eB_Xs1FGPtK;t^rz$}iGJoSXAS=Vf5g!$Z zRQBInM_Vbqyqz9(C@KdHJaj^P&6#2L*U<%uRJvCU+Vx z1xAH&lCBcRm%xI=6dWq5vGa-{+HY2}DSXb>La_CwRb1CZb#>3R&r%4b`@$T{@=vd6 zxe`)(!SN_|6&o-o@#uZquNal;Yf;T@Q+{h(4>DeQpG2O2n<=|T^E&kdu4CvZc=RYO zOq=Af4LwC-C(bSnhuDJ~nkvd+#_#PV2%-s2NUb7U#kh`nGrmm2%cpVB^PG|&+Y+Cb zx6JQ74U0*@I_Flvl0PXXK$!NSCy4`d5CShMQbnerf6~i;RZhhtA>m5U_^r@7?y#l= z-d=CzELg(3O91FabDF)|jG9nhVVT{W)p~75ldmDO{*7B75hI5WQsHHex!S zEFz6lNLmuyuF||%26Gp=UzmsBvXva}vlq$6O`MamapqE?Z|Yy^=rWbzfi3ng<1Mi0 zUQleNMoRBSlJ-r=X=0F#9)IoXmP^G)?C+BH2Tf1bnYn%sK=z@RNx2I)9OGLB?4OVC zEw0MDXRo{pDpUR0VUdYj;foyh(%6o7?OpbJqA0U`zW2G269Q^Q0iGCtKqQ3?*bo(a z^HgT4;m~5DuJEu{Z?_gr_?!`Wz?grXF96-NS5F_k?qTD))pL_C-Dn*i-TRPX)bH+Mw@F<{70TjBcgty@CboK0NS*_^vpW;-`w4s-HVIo4NF#rPD; zcxZmBRwNv=o7sWvCTsSGX7n9*y$dgV5N}O#iu$A_?3vou_WEH)1Zy~(0fdTx@_)w< z_Q{aNh}yPe`yzrQ5iB$&@F4F@h%`TbG@woL^`71QwY0B8K={_PI8JCopI(c59VJW>E+R(Y7$ZCU->hd5W^G-^yK%9lZON+wHbc+dF3_c2B|lO& z6k;&3%TO0O+*!8R48lVX+=)59IQAG2=Yt%Rm|pU^&X`{Q8mXzJkkAukoS;J*2O^-R z0QtJ2haisPp&@P&ngjt{3V^Oo=ssC-lqUG0JAikuG?eDZJU72#uwPh}0^Bbnrr)iL4B)W(dbQ(B2-C93 z)k>=WLUtVjK$=s!7+_i5S%bqXsbo)o!x)o=z&Nk7mEP~Dvl zRX&oo?tV)$nIc^c4L$YhVI|C>-^X^|mf4z#ON4LxyJW@#1E+JR-xa5x_9nsK4nMml zYNr!|BAr*H`d{{KI?hrtqC9B|@LdFiFYf|TfN^hp4EY2sp8~w_qrvi6P`9m~A|isS zXSGv(56n0O*%aMC{!dME0KfPYw5#EPI*@|>pz^~*?{pa;7eL9NT68yZws){?r4+x1 zli;4WgJ};e5)2nwa_&Z7)>NzjU6hCf%>z#~e;U8=kx1$fWB(han6Z#_f#`$-Cup4$ z<-;$}%?VW3DAw#KI_*+{k7{wHuXeUW(X%|RN{_4hu=lOvD>g|Ec&eRrb7J2v-l?j3M9ZgMbtn0H$nNoV)G|eX%@;A}TNCp_VKTKj=>#Uu@ z8)C?ymdF(8vTwN#_Z>`#{o4+HT>)|s*JWJdhqyhS`NTZ-7Sj2tFaXX#Fi&n;>;k8oYjaP!*T=3$@?0z+wu8z!J4 zSf$c1r_cjhF{>R4%A>npeC~9&(5@*(ChoT?cr{XhQyvcX z=xUMy&6LBW9}A11MD$f58@Igmg>{UxWlb&jD+4`Rxoco?|HT-MA2DPQ%mQb$d9d>a z*0yZc)R3&^tD%)SRyA;6hKE79X1Hp$`-8pm3=wq@OhjUh1&Nj2YHhq|D)uPwsV#RV z$;&s$4RGXQ45GVbdLg(CsSUXWn>^^6Za_w37+&Git(x9VdBgQ^!xnezHEy1Iu3cd} zH{|rrUgvRYg{c(RIoopJ3g|_CRfqDCaoC6fA;pB46|yXB-lU~v=S>CP&^;8uq!)>{?D@4#*(6J%gb(vc) z-2yR)D2u)jR{lO+FN*ES4wN$0LPa>*|xHSWa$b zPQ%||h#0kpr^eWs@Y3(uMd3GzcFTx_V44Oy(0dPBI~|p(EQ=dvxAPqn#pR+GF0PDb z+ND0%HRy*Wg{6|jkKbsWB#Dnx?RK@6G5HRoqTlMfMC3h}Df#bBS$#)U{`$pDy2-Nn z^=p+UX(h}mQY{C9=*X~*nQZU;8h14z)x-7d(8kZ>Hg<0{4Si2uw*1a~#=>Nsmfnd~ zIh+RfRH>ZG50u@Y0GM?09Vl++IO&ke_k1mleE#ng7O*q@J)Q0!J^n}ImNupz5V;9m z8$E{08_<|Mv8-M;nl_=gZt6)7ywDP#6mHs^cSmH-Cu^WHEVj$Ksp|mJ>yMWmh`J~$hYM+1-)k}pf9K8iDJfByEZ zl7S`wY`aZ%U#;4;{yH!=%);aM_2ADzYFl^Kr+N<$CmCsF;M36u0F=RJtx#KGwJwyL z>u`1YO*)rxkcz4DpchYC$vKJ71+mkmsp71CCpO<+K6Cohap4S{vCI|4NS1URUP=P@ z=AV@QqqW_=P4Mb0h~Bx^*fHG@M@1NV^Bd|f2T9(|uWze*Aae<-c-+!;n(F6vdHR!Y z0?rSPk*+0}eIbV7UHO7}I|q{A-LVtPyLxLq^pP0ff({C%ClUSF$?`p8_X7WMUY8be z`9>o_FPtm{%Ylp|1CR5LjK`7Q)y}(L2gtVLH$TkTS>0joJ;8nk{XFBr9Mjon16Nj1 z*GOK=#?g5q$2nxWwFO0da&e7&vzE||^~~<1t(ThYWBB!ow=vCplR>4n-qOL};2r+R z!jAj+J*;9lnHEDV)5ZHjT-%^gn&2w8!_B8ZcB0scREFLaY$IU_>`dj1N}4Tt-t|Ax{d7ozI@RQvM}*}5v=LgJ>oMQ z3xe19?HZw1G{0aF>ZO+9dys@$y)fS`@k`p|nvfe0*Um3UFImc63q(i>hhJZ^v*>BN z3}t}}p}KBt}(>&qrNp=;y{POg^PQkJTzO`$Srs3l3nlV)*C0iFgYd}>RnP{lOO@IBFvw3 zZaLR_=7(X9NrHKvtd~+xe~rIPQ5t8Ek7O06f!x;VnwsstF5S%xTbMjG)QCM2Oywpd z<}u*6)3MdGW_1@~{<#eam7UVYov4Wpv{FlLXKQmB?u(#63+IP+%S^i}gL2KQN4Mb* z@iXaVt7^fyxhuJEW*6K(G8C4&9fljag!(m4bo(B(2ZfCbv_rI1b709ghD9fa^XaMX z{kUnc`(pCp@cedRC{7@0rSLd;;;UZSm;OsR*POl7aF@lOS<5Co-Vz!Xv|`m8ay}PT za~FpgG>)Yjc}wr=38nL8U-s9$6&^KB?r&UG`Z{m7b)+-Y>%@9aVt3PSw?*zTgHL}D z8xPGiqQ?t|Al_i_O`(#W`!)PQThO}SK7XRLV?OM`hr3)3QCZ$worFA(F^A)#F6D0A zRgeCr1WlHeWQsB zSBXu2Uq}yqT${2ly~J3p&G2PAy;gkk3C$(n_`;-r<@TpHdULJsXfByBW{m#}{gn47 zv=91QlFB{XH0h)o%Eflb7nOfuDqRy*HL-o4WzxXK#asK@e`%$x=&6$9gxD)nr#u(S z=qc9Olbw&1F7Y5s24%%R`Y1O%IG;9qp1ZNGIdo4$gY)9(1}0p>)=QJR3R^QnI7w&XkBguRM(tS$w&v|` z4zC0Dt8dBBipulNoa4Ih@`*!Ne~W0|PX${Iu!;h(%JAcVobh7C+5pWDCkYI5DB_QHE&X#?oul|s2Bx^mX$|CD^tLJ;w8*#^jtW-nR zJlptN7mYqWxtEwdL~t2|#D`rXFG9FI5bgyP zDz+TXgmfc-qgIZKRW51TU{%$@T>FOlUX7vWbiGyEZ7fMGjVzL648=gbpKdw@V1}#v$a+sme{9e{=8o_ zSgPEy=S7^>dTl8EUuyZ#myzI>;>qcvJU zEcM5Ponw8fpt$JW8>=M(2j5b1RL6-q2c^VmTm>)EKg94-R86JpnA@R!{zItP#^cZ& z(~PC2=47?J6~&Ad*_BgjbxRIE+GJ=~jWJfIDhMahI8D&itUmeky^!QP!#+M72`0AG z*B!{;kbOCN)Vh2LnYXcuiV4n&Axpv}wU}UorV+YRay8DK_^#>z+!ad&Z=TTNJM9_C zV$JO`NjzoW!hN#lPESzbN*s!bSp2?i3mvWLQ2NysQ=Q|LiCI^R*BRB0Uaz`rF35VB zjvC^fi!2R0VWSpt-CI)L2B3qhwpZz>KX>UMvunoe|sW4jXU}BfF zNB$Qr*2(Mx$Gh@;a^E{&^24eSMZFfXGL= zy4t`%D66~Et5tv5%PZ@$A|r3N&*b@(r1I2$T6AiOw_LbZK43ZJ?L2265^{8o z_!Snm(-&Z!nRR+#$aF?dLr% zV!gEcvn>*}#7I|i>{{^9xR_XyB7IHm!KJGE59^x^h0|uzb7CC~Spv*OZJ^ru8S$}- z2=62pGva+JpYrN(S*-j|0hN$>hIu)Xk-tl!YU%Q4^N?Hxfyt4!=*x)@US*y;QJAK7 z^8Qmz=u-YBp%U&*^dRut<1=zs%5Vtgp`rxYD?wuVCGlj#$QHMo5fVjxrD|=<(q~%+ zy_QpE`AzXDq%&>0^>1xpSInn7E~<6bpklo<Qa_2$jYYYc zi3viBNA-;ob-Q?pKd|`+Pz(aAlCPh$6&5gBk8md(`PCHfvg ztYNa($KYTe?C_2GeK5;M?1!Ir+?ICAma$E>T2(GorN@B$S%Nz>@BTe!$B$eJx5gM1 zL7QF#CE-yHYgmls7+kZBuT=DT`no!jt?`(2FciqDw#SsMy7$1(FC^lm$a_Av=c;h% zk+e}6>FPKMv9!pshE_6IIs)-PT+tSQn2$uz{B=;T*K7F8U-~8|> z*i{X};XfK@3RAagobs<|ILgVfL2xyH@dwq;_Bzi5mvCEs%F2X(4|--Px73mFLj;@t zqXOZ^uC`*6Le%5cT%^gT+q9`V|IJ~`;g{Ys42TqWO@sn$?j32sd~q>7MmB?Jdv1ba zlD~&Bhs_$ZLxFpf^@DikYI_IGq(Fzjju?}I6A^jFyjVm7q*p|GX>?$RpQJdpqo7y? zzY^?KoVmp-eH7m>!LrMj^s>`jEdThickTd7W_1a@d=mAGR^J z(OzI1Bgy4#VV9=euUC^+_MM*GIe8k*Oen)eE(&y=T_AqnZ)vcUY_6+otc%xsZG&-} zskNFDHRhOH+M12EO(0!`zOCO3jGDCbECjE&TK|!c8IP&N2zf-tR#qP|W z-+Qjhv}JIO{?Cm(X#XT3y-WA$(l=$nM-Xb@W*&;TfrtX!Os!CuLj>t3RSB{o$&k^L z6yOJvtg(o_CUnmndeBA10)U=%{~NA;v^ zg@Vb!R4!39nTMuShl;3rtSwjltqIV-sh;*${P4eNLhs*J4@?x*BmPYjrqt?x?)^*C z|I9fvYTK3^mIm+F1O7Z`(*NUgQX@*K^hXgWzT)AO=e7i=w$#d1h6!*fVD|@S_(w5> zRVt*h(_usa_1T}pH^*r5!kT8j2Kd9TFyW7G@I(M!mxGU89XuicDF4nz=sqmBTMYZzs8*U-$y)=f7kvO`o3 zBe^ka=!v~`AXc;6_PtawA8=Qo)~C4;m;x>6zYtbyj$NM9x|ja z$nW&RS!|-dq4J%~N#9>Ff=82(%sPzlG7SK0kaVy}PNKF|D}E=zud?tx%p_vQLLY@ z4H#&m&!P^fp1neD>Kmpk))q-4V~j(77E@Gt=9GWPsp&$E9u3Q-0H3#n|55_Ln{`4B z{_ghizdQb)>i(m#gP4~~wzcG2!fO|?veUprR3H^omDOQgBd4!QmZQP|m-72F6;w%x#* zdC3X(P51oj`V&FRCrp~*8WxS}TfBr@o3h04Q0YE2Ky#3bxHh3WRL4l`KT_)Eqa3E- z(M#p9Q<-y7RB3N~NCQ)*YDUmMvB-5PLx_85$@nv)mrP_=++}Dd;TD7x95`|Rf3kgFkcC z^{-A*`Ei~#8%=xd1_AUT0-_~+`X=%G(_seT%(-JmNJsg}374d}nyi}Q`1^4cw8?8P zE_YD@7@TKoditE>r_5xH8cBOSkgci&D82>-d0Mc{rxWp4b~QgI=lPCkb4&#g^03y} zS662y$AjvvP7<$lGV!rPq!WCwWBupcckkAL7;;DGIvtQbHxG2cS6(~xsHwA;IA=;1 zq0dMBnu{5>VS(Q_bZP~~Q2FXo!wWf`+XYBGU79t_EfUVmA3lMomBaS*T&Mc)Zl z7zn5w_jb^J$d$rI$iHmAF*P{rUyAyb}y6!l<&WEg>Kg@5bPCX_OA_7mIpkIf=}F1e<3H47Wk!wEAt(QfD-1#pVz( ziOqB6T*vLJj=o&ZcZ|8%H9pKbrrRRTkeH1#pXn}L9KPAu=Q7C->eM2Je5%xGK%|h zHgd=mB7d^_S=&SLAj6}%yUNGqP_atUoG!c7D|I<3dqYJ3JFgk-#o6TS z=mxrXNvE261IMfgY_1p`S$f?GyVEl~48$_p+dD)W>D^m_kUS{-71v4IX`?WX+FW@B zkw$q8U0Ey!{@nnqdy`#^4b}gG79(5cX(pZDxo@()!AtSb{EF55)QpcICVd+OOk=5p z=qAhysV=Jt-bJajW>>q(j&pT2*N=MFqQ$Me5Lq)m*{pLiGciRK7k*F(YYe@`TH7Ir z2oA(+f2pjwbf9)N_i zx)M)7%Hf!0+N@r!;Ub0ok)fheLzK>p@o+uEU|CC31It|`8V7{Mujpuh zqJJ@ucv(#@li^f(Ay4J*VWEZ*4e(sE2I}LN>TLZd4fuolyWG!&Tg>GZgZL+3@QVae z*|OvJxj{AXYZzt22jQFM6X+oSl-d=RUke~bHxI6}HUj7Mtn5a}=sGtv*`0(~@eZ)B zWYb~PJcehNKRw1smY;Tt^D>qAu(`XZEb~Ul!z-ryI`kod1guoSr$ej5ZiV@d%suDl z{VN~8)loD_myb%Nt%>wZAytjvjAYzGd1Tn@=kBnScHp{;{RB%FD8(M(PI3->wzZMT zkO;GH<>JDQZe1(zpggvJwDd#mK>O4s2N}AYT?mUeg^yjH-39Fa-2hfdif6#sT9M*e z?nX0ksnMBBP0yP@_?Z>F2qg!Dhv6OA2(3|9)e?sYXwgJcb6 zr|U69zp6jt?4qDaP|_lCmT2T!P@Dt1jjo+i8!M)E^ch>2%Vey6_D%ErAhQ{x$?msA z<4vfJgLTh(hfQ;xN-EHW&{U}ARPf>w3@}5d2zOg*r#<27h6vl*=)7-oRfOlRvK{91 zj-UF7AuvqzOTDy)mE>8mO?bx!%L2dPHGLGJ!+E#np7Gw^jyI<{XX4b9_Q^O^2IBpN z{Opn<65oK|i!EWgU=x#0?3ON9Sf{VtvSGU-X0FEI0BoNM?TC0GG)&0{uGfAZSM47v zq~va#=AT@=Xzk0M>r_^J^-g8wHc@n*UC3^-vG{1vZjOzs?x~L|C{=1m#a?DA+W0uA zOO~JXK78d!Vov?17*teU_l%QL;e4*GCb5&?syu^S(;|2VBoh4GgoSgh_JH^b>^}BF zDld+sI9s0*D;xZH(UOvkqw9_|wWYPaAW#KBL7M&2qY~xh;cZQBa;)}hDmfL=8VYV% zZl3iLKx0MQ-#_GHV6Pnwt+AE%Xt3M2j+y$P`i$S{=I*k9BGh&*%_P9=t

9ixnRtBl(`)aFKg6FI7p|*lJrw zuLsLt`4;>x;XT#5UU!KOKk7)8NL_5iRy)hXtF|l_<0ZJsY@-gcM67pJ*~rgEiZ{zw zQL}`fksnq}<3wkeJwv%l2@#rk*lX2Ec1LpL2_8ruK+X0fzdO7!yn*;K*&3jMpHmG= zwrc{X<*h;JCm@g4X*=VkyV1bDNSto8tl}EN6Tk7~C3U89ER9YcMYgmP_Ait=vF`8N zU(r3QpX$$)DfO<~y9gHpgx54VGz=UoqqYwl=>rvj+ecSkdwypSPd#X;&y}9 z`8tScg;7n<1^Y#I?}}!gGez6Q*v|zqKR;I*RA27nI9cknpMAtpCrj)ah<_n(-JL6p zL^7Cn>_{IBG)9}}TiX)rwQv$}s&`k!)%tL)tYK?xN`FDet)*M|boumwX139-c$rv$ z0B%IGpQGI*`slz&Ly>$LhpDcuO4l4!rp7$Mnsk==Op)I{>(5l^0DmgFUH|UWP}v`D z4hLOCyz<)*dIiSgBL>_=OXQ?v(-HoXQF`F&=`mrzuRa7}Hw%=rh$8K#@ht*%*gFq7 zTql(qX&EcWezZBm&DjPSAif#t)K-9t4@2i}k$1MMbEmn5+n_I@tvD(ZwZMN&GHwso zYE?TtR&Y%LdY4?}qPqOX@Dy`FNe3mKV6o@l^;Z9mQF?zN>4hgrAYAWK9nj2+{+zdE zxuH!;qzO+yEQR*f-dY_Xcu5BVB@+ws|C4|Jm40Jq1b{km z6*`75P1%K;4aU)u4H=4-?~1 zq`JL{v~jMRsZih0uha9tlo0S=O$PWK)71W1iOGL*?l)IUn?_r9xDE^Q^%XELj$1GX zu;bQSp?4U7_S!Nf^UeCRe8|0EOpN~fzNaqD=qbG9NB1rZC^!SeD=Q~$1DP^umju(q zucrY()(8~JY6yh=nK;=fNGA$BB5?xOGO2`v^#C@bkHK1yd|4Rp3BXwKN*s8W3+u~( z?iB#~`S&0|+Am6Yc8?x#C2`Rd)&!uNqX2l$61EM}2J9=6&-rbz#Uh~IkPh3Rc`Asz zBV!Bb6RHtgQMe%h5xyPF2J#=8a9wq{P|Bt@Zo?aRYQrqS2J}CCNdXyE(fa`Kt87pF E38FEl0RR91 literal 0 HcmV?d00001 From 563d51b10614193b2ebcfb835385ea99493378d2 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:57:36 -0300 Subject: [PATCH 284/345] fmt --- crates/networking/p2p/sync.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 1a87aaa3d..8efc7cb4e 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -311,7 +311,7 @@ async fn rebuild_state_trie( // Ignore accounts without storage // TODO: We could also check if the account's storage root is already part of the trie // Aka, if the account was not changed shouldn't fetch the state we already have - if account.storage_root != *EMPTY_TRIE_HASH{ + if account.storage_root != *EMPTY_TRIE_HASH { account_hashes_and_storage_roots.push((*account_hash, account.storage_root)); } } From f10e6fbebde8318037c39e3c2bb490e98cd42aca Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 18:59:03 -0300 Subject: [PATCH 285/345] Rename doc files --- crates/networking/docs/{Networking.md => Network.md} | 0 crates/networking/docs/{Syncing.md => Sync.md} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename crates/networking/docs/{Networking.md => Network.md} (100%) rename crates/networking/docs/{Syncing.md => Sync.md} (100%) diff --git a/crates/networking/docs/Networking.md b/crates/networking/docs/Network.md similarity index 100% rename from crates/networking/docs/Networking.md rename to crates/networking/docs/Network.md diff --git a/crates/networking/docs/Syncing.md b/crates/networking/docs/Sync.md similarity index 100% rename from crates/networking/docs/Syncing.md rename to crates/networking/docs/Sync.md From 8d70af30f03a4fe0e74c676d8707bcd791398dd5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 4 Dec 2024 19:03:45 -0300 Subject: [PATCH 286/345] Typo check --- crates/networking/docs/Sync.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/docs/Sync.md b/crates/networking/docs/Sync.md index fe79096a3..8beedf312 100644 --- a/crates/networking/docs/Sync.md +++ b/crates/networking/docs/Sync.md @@ -16,7 +16,7 @@ This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) -The `bytecode_fetcher` has its own channel where it receives code hashes from active `rebuild_state_trie` processes. Once a code hash is received, it is added to a pending queue. Once the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them, if a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so that it can request the last (incomplete) bytecode batch and end gracefully. +The `bytecode_fetcher` has its own channel where it receives code hashes from active `rebuild_state_trie` processes. Once a code hash is received, it is added to a pending queue. When the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them. If a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so it can request the last (incomplete) bytecode batch and end gracefully. This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/bytecode_fetcher.jpg) From af20a75d2e2cfc65f05c7f33c3ca83d0d7c9eb2b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 5 Dec 2024 12:47:32 -0300 Subject: [PATCH 287/345] Revert "Revert "Revert "Revert "[REVERT ME] Make V2 endpoints work in order to run hive sync test"""" This reverts commit 5b93fbdc2140076f137a9a4cf5596e071fd9fcf1. --- crates/networking/rpc/engine/payload.rs | 20 +++++++++----------- crates/networking/rpc/rpc.rs | 2 ++ crates/networking/rpc/types/payload.rs | 4 ++-- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index 9d7eb0089..abd59941f 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -43,16 +43,14 @@ impl RpcHandler for NewPayloadV3Request { let params = params .as_ref() .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - if params.len() != 3 { - return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - } + // if params.len() != 3 { + // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); + // } Ok(NewPayloadV3Request { payload: serde_json::from_value(params[0].clone()) .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: serde_json::from_value(params[1].clone()) - .map_err(|_| RpcErr::WrongParam("expected_blob_versioned_hashes".to_string()))?, - parent_beacon_block_root: serde_json::from_value(params[2].clone()) - .map_err(|_| RpcErr::WrongParam("parent_beacon_block_root".to_string()))?, + expected_blob_versioned_hashes: vec![], + parent_beacon_block_root: H256::zero(), }) } @@ -79,10 +77,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post Cancun fork let chain_config = storage.get_chain_config()?; - let current_fork = chain_config.get_fork(block.header.timestamp); - if current_fork < Fork::Cancun { - return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - } + // let current_fork = chain_config.get_fork(block.header.timestamp); + // if current_fork < Fork::Cancun { + // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); + // } // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 80581d2ab..930c48f51 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -254,7 +254,9 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), + "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), + "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index b35825039..ee9d873eb 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,9 +35,9 @@ pub struct ExecutionPayloadV3 { pub block_hash: H256, transactions: Vec, withdrawals: Vec, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str")] + #[serde(with = "serde_utils::u64::hex_str", default)] excess_blob_gas: u64, } From 2715bf544deaff3f3f40b6d2041e7b311668ec5b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 5 Dec 2024 17:00:16 -0300 Subject: [PATCH 288/345] Fix `StorageRanges` decoding --- crates/networking/p2p/rlpx/snap.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 70eb36a37..0557589ef 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -449,7 +449,8 @@ impl RLPDecode for StorageSlot { fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { let decoder = Decoder::new(rlp)?; let (hash, decoder) = decoder.decode_field("hash")?; - let (data, decoder) = decoder.decode_field("data")?; + let (data, decoder) = decoder.get_encoded_item()?; + let data = U256::decode(ðrex_rlp::decode::decode_bytes(&data)?.0)?; Ok((Self { hash, data }, decoder.finish()?)) } } From 951b2085d24621664245a07c0c9f42cfb70a2437 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 5 Dec 2024 17:04:08 -0300 Subject: [PATCH 289/345] Update encoding --- crates/networking/p2p/rlpx/snap.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 0557589ef..55b465f32 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -440,7 +440,7 @@ impl RLPEncode for StorageSlot { fn encode(&self, buf: &mut dyn BufMut) { Encoder::new(buf) .encode_field(&self.hash) - .encode_field(&self.data) + .encode_bytes(&self.data.encode_to_vec()) .finish(); } } From 0f552285c92b3830de97e1abdf80d67ec07ede1b Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 13:20:06 -0300 Subject: [PATCH 290/345] feat: fix decoding of receipts --- crates/common/types/receipt.rs | 41 ++++++++++++++++++++-------------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index e0e888244..a2b9038e0 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -1,7 +1,7 @@ use bytes::Bytes; use ethereum_types::{Address, Bloom, BloomInput, H256}; use ethrex_rlp::{ - decode::RLPDecode, + decode::{get_rlp_bytes_item_payload, RLPDecode}, encode::RLPEncode, error::RLPDecodeError, structs::{Decoder, Encoder}, @@ -48,40 +48,47 @@ impl RLPEncode for Receipt { fn encode(&self, buf: &mut dyn bytes::BufMut) { // tx_type || RLP(receipt) if tx_type != 0 // RLP(receipt) else - match self.tx_type { - TxType::Legacy => {} - _ => buf.put_u8(self.tx_type as u8), - } - Encoder::new(buf) + let mut tmp_buff = match self.tx_type { + TxType::Legacy => vec![], + _ => vec![self.tx_type as u8], + }; + Encoder::new(&mut tmp_buff) .encode_field(&self.succeeded) .encode_field(&self.cumulative_gas_used) .encode_field(&self.bloom) .encode_field(&self.logs) .finish(); + let bytes = Bytes::from(tmp_buff); + bytes.encode(buf); } } impl RLPDecode for Receipt { + /// Receipts can be encoded in the following formats: + /// A) Legacy receipts: rlp(LegacyTransaction) + /// B) Non legacy receipts: rlp(tx_type | rlp(receipt)). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - // Decode tx type - let (tx_type, rlp) = match rlp.first() { + let payload = get_rlp_bytes_item_payload(rlp); + let tx_type = match payload.first() { Some(tx_type) if *tx_type < 0x7f => match tx_type { - 0x0 => (TxType::Legacy, &rlp[1..]), - 0x1 => (TxType::EIP2930, &rlp[1..]), - 0x2 => (TxType::EIP1559, &rlp[1..]), - 0x3 => (TxType::EIP4844, &rlp[1..]), - 0x7e => (TxType::Privileged, &rlp[1..]), + 0x0 => TxType::Legacy, + 0x1 => TxType::EIP2930, + 0x2 => TxType::EIP1559, + 0x3 => TxType::EIP4844, + 0x7e => TxType::Privileged, ty => { return Err(RLPDecodeError::Custom(format!( "Invalid transaction type: {ty}" ))) } }, - // Legacy Tx - _ => (TxType::Legacy, rlp), + Some(_) => TxType::Legacy, + None => return Err(RLPDecodeError::InvalidLength), }; - // Decode the remaining fields - let decoder = Decoder::new(rlp)?; + let Some(receipt_encoding) = &payload.get(1..) else { + return Err(RLPDecodeError::InvalidLength); + }; + let decoder = Decoder::new(receipt_encoding)?; let (succeeded, decoder) = decoder.decode_field("succeeded")?; let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative_gas_used")?; let (bloom, decoder) = decoder.decode_field("bloom")?; From 3e534e490253a3c64da6de59f5aed4d81be9f69a Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 13:42:39 -0300 Subject: [PATCH 291/345] feat: cleanup libmdbxp impl --- crates/storage/store/engines/libmdbx.rs | 23 +++++++++++++---------- crates/storage/store/error.rs | 4 ++++ 2 files changed, 17 insertions(+), 10 deletions(-) diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index 2e255fd85..a9a7f0d1a 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -421,24 +421,27 @@ impl StoreEngine for Store { .map(|b| b.to())) } - // FIXME: Comment this fn get_receipts_for_block( &self, block_hash: &BlockHash, - // FIXME: Alias this type ) -> std::result::Result, StoreError> { let mut receipts = vec![]; let mut receipt_index = 0; let mut key: TupleRLP = (*block_hash, 0).into(); - // FIXME: Remove unwrap - let txn = self.db.begin_read().unwrap(); - // FIXME: Remove unwrap - let mut cursor = txn.cursor::().unwrap(); - - // FIXME: Remove unwrap - while let Some((_, encoded_receipt)) = cursor.seek_exact(key).unwrap() { + let txn = self.db.begin_read().map_err(|_| StoreError::ReadError)?; + let mut cursor = txn + .cursor::() + .map_err(|_| StoreError::CursorError("Receipts".to_owned()))?; + + // We're searching receipts for a block, the keys + // for the receipt table are of the kind: rlp((BlockHash, Index)). + // So we search for values in the db that match with this kind + // of key, until we reach an Index that returns None + // and we stop the search. + while let Some((_, encoded_receipt)) = + cursor.seek_exact(key).map_err(|_| StoreError::ReadError)? + { receipt_index += 1; - //FIXME: Comment this idea a bit more key = (*block_hash, receipt_index).into(); receipts.push(encoded_receipt); } diff --git a/crates/storage/store/error.rs b/crates/storage/store/error.rs index 61687f543..1e61cb594 100644 --- a/crates/storage/store/error.rs +++ b/crates/storage/store/error.rs @@ -18,4 +18,8 @@ pub enum StoreError { Trie(#[from] TrieError), #[error("missing store: is an execution DB being used instead?")] MissingStore, + #[error("Could not open DB for reading")] + ReadError, + #[error("Could not instantiate cursor for table {0}")] + CursorError(String), } From 6a094612ffc161dd961082275779eac1797d8a5a Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 13:57:56 -0300 Subject: [PATCH 292/345] feat: cleanup unwraps --- crates/networking/p2p/rlpx/connection.rs | 10 ++++++---- crates/storage/store/engines/in_memory.rs | 10 ++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/crates/networking/p2p/rlpx/connection.rs b/crates/networking/p2p/rlpx/connection.rs index 6bf7fe43a..6b896d3ed 100644 --- a/crates/networking/p2p/rlpx/connection.rs +++ b/crates/networking/p2p/rlpx/connection.rs @@ -356,12 +356,14 @@ impl RLPxConnection { self.send(Message::BlockBodies(response)).await?; } Message::GetReceipts(GetReceipts { id, block_hashes }) if peer_supports_eth => { - // FIXME: Remove unwrap - let receipts = block_hashes + let receipts: Result<_, _> = block_hashes .iter() - .map(|hash| self.storage.get_receipts_for_block(hash).unwrap()) + .map(|hash| self.storage.get_receipts_for_block(hash)) .collect(); - let response = Receipts { id, receipts }; + let response = Receipts { + id, + receipts: receipts?, + }; self.send(Message::Receipts(response)).await?; } Message::GetStorageRanges(req) => { diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index edc06e03f..60fa649c8 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -349,14 +349,12 @@ impl StoreEngine for Store { Ok(self.inner().payloads.get(&payload_id).cloned()) } - // FIXME: Implement this fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError> { - // FIXME: Remove this unwrap let store = self.inner(); - let mut receipts = store - .receipts - .get(&block_hash) - .unwrap() + let Some(receipts_for_block) = store.receipts.get(&block_hash) else { + return Ok(vec![]); + }; + let mut receipts = receipts_for_block .iter() .collect::>(); From 7b544dabe8c813565f14ed10253cc71bf461d8a0 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 14:30:06 -0300 Subject: [PATCH 293/345] tests: add GetBlockReceipts to devp2p in hive --- .github/workflows/ci_l1.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_l1.yaml b/.github/workflows/ci_l1.yaml index 0a0292453..dbdc3129b 100644 --- a/.github/workflows/ci_l1.yaml +++ b/.github/workflows/ci_l1.yaml @@ -146,7 +146,7 @@ jobs: test_pattern: /AccountRange|StorageRanges|ByteCodes|TrieNodes - name: "Devp2p eth tests" simulation: devp2p - test_pattern: eth/Status|GetBlockHeaders|SimultaneousRequests|SameRequestID|ZeroRequestID|GetBlockBodies|MaliciousHandshake|MaliciousStatus|Transaction|InvalidTxs + test_pattern: eth/Status|GetBlockHeaders|SimultaneousRequests|SameRequestID|ZeroRequestID|GetBlockBodies|MaliciousHandshake|MaliciousStatus|Transaction|InvalidTxs|GetBlockReceipts - name: "Engine Auth and EC tests" simulation: ethereum/engine test_pattern: engine-(auth|exchange-capabilities)/ From f4f8f97263d6043ec6bfa4ed08c2bbb10963ef66 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 16:26:40 -0300 Subject: [PATCH 294/345] feat: implemente receipts for redb --- crates/storage/store/engines/redb.rs | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/crates/storage/store/engines/redb.rs b/crates/storage/store/engines/redb.rs index 1a7ab7f53..cac02832d 100644 --- a/crates/storage/store/engines/redb.rs +++ b/crates/storage/store/engines/redb.rs @@ -541,6 +541,34 @@ impl StoreEngine for RedBStore { .read(PAYLOADS_TABLE, payload_id)? .map(|b| b.value().to())) } + fn get_receipts_for_block( + &self, + block_hash: &BlockHash, + ) -> std::result::Result, StoreError> { + let mut encoded_receipts = vec![]; + let mut receipt_index = 0; + let read_tx = self.db.begin_read()?; + let mut expected_key: TupleRLP = (*block_hash, 0).into(); + let table = read_tx.open_table(RECEIPTS_TABLE)?; + // We're searching receipts for a block, the keys + // for the receipt table are of the kind: rlp((BlockHash, Index)). + // So we search for values in the db that match with this kind + // of key, until we reach an Index that returns None + // and we stop the search. + // TODO: Make sure this if this is the proper way of + // doing a search for each key, libmdbx has cursors + // for this purpose, we should do the equal here, + // if this approach is not correct. + while let Some(access_guard) = table.get(&expected_key)? { + encoded_receipts.push(access_guard.value()); + receipt_index += 1; + expected_key = (*block_hash, receipt_index).into() + } + Ok(encoded_receipts + .into_iter() + .map(|receipt| receipt.to()) + .collect()) + } } impl redb::Value for ChainDataIndex { From ba609b3bfd041b5bd02612db45e7d2b3a668fa66 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 16:29:28 -0300 Subject: [PATCH 295/345] issue: update todo tag with issue number --- crates/storage/store/engines/redb.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/store/engines/redb.rs b/crates/storage/store/engines/redb.rs index cac02832d..0946cd576 100644 --- a/crates/storage/store/engines/redb.rs +++ b/crates/storage/store/engines/redb.rs @@ -555,7 +555,7 @@ impl StoreEngine for RedBStore { // So we search for values in the db that match with this kind // of key, until we reach an Index that returns None // and we stop the search. - // TODO: Make sure this if this is the proper way of + // TODO(#1436): Make sure this if this is the proper way of // doing a search for each key, libmdbx has cursors // for this purpose, we should do the equal here, // if this approach is not correct. From 9aeaf94a504c2b98aa8d7c7cbe57233b96912532 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 17:01:16 -0300 Subject: [PATCH 296/345] chore: cargo fmt --- crates/storage/store/storage.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 0f24d639b..d6b0f5486 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -919,7 +919,10 @@ impl Store { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } - pub fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError> { + pub fn get_receipts_for_block( + &self, + block_hash: &BlockHash, + ) -> Result, StoreError> { self.engine.get_receipts_for_block(block_hash) } } From 7e4fb45e698aef251c7eb7ac928ee2abea44abdf Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 17:04:34 -0300 Subject: [PATCH 297/345] chore: lint --- crates/storage/store/engines/in_memory.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 60fa649c8..a4efd2a3d 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -351,7 +351,7 @@ impl StoreEngine for Store { fn get_receipts_for_block(&self, block_hash: &BlockHash) -> Result, StoreError> { let store = self.inner(); - let Some(receipts_for_block) = store.receipts.get(&block_hash) else { + let Some(receipts_for_block) = store.receipts.get(block_hash) else { return Ok(vec![]); }; let mut receipts = receipts_for_block From fb1ae9b50623a578ece6d2bc2ec6872ca1321576 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 17:51:50 -0300 Subject: [PATCH 298/345] feat: back to old encoding --- crates/common/types/receipt.rs | 31 ++++++++++++++++++++----------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index a2b9038e0..e429650bc 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -48,18 +48,27 @@ impl RLPEncode for Receipt { fn encode(&self, buf: &mut dyn bytes::BufMut) { // tx_type || RLP(receipt) if tx_type != 0 // RLP(receipt) else - let mut tmp_buff = match self.tx_type { - TxType::Legacy => vec![], - _ => vec![self.tx_type as u8], + match self.tx_type { + TxType::Legacy => { + Encoder::new(buf) + .encode_field(&self.succeeded) + .encode_field(&self.cumulative_gas_used) + .encode_field(&self.bloom) + .encode_field(&self.logs) + .finish(); + } + _ => { + let mut tmp_buff = vec![self.tx_type as u8]; + Encoder::new(&mut tmp_buff) + .encode_field(&self.succeeded) + .encode_field(&self.cumulative_gas_used) + .encode_field(&self.bloom) + .encode_field(&self.logs) + .finish(); + let bytes = Bytes::from(tmp_buff); + bytes.encode(buf); + } }; - Encoder::new(&mut tmp_buff) - .encode_field(&self.succeeded) - .encode_field(&self.cumulative_gas_used) - .encode_field(&self.bloom) - .encode_field(&self.logs) - .finish(); - let bytes = Bytes::from(tmp_buff); - bytes.encode(buf); } } From 1baf3dc1c53c4cceb797194c2c1053a6f8e5b07a Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Fri, 6 Dec 2024 17:52:51 -0300 Subject: [PATCH 299/345] chore: comments --- crates/common/types/receipt.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index e429650bc..ede622310 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -45,6 +45,9 @@ fn bloom_from_logs(logs: &[Log]) -> Bloom { } impl RLPEncode for Receipt { + /// Receipts can be encoded in the following formats: + /// A) Legacy receipts: rlp(LegacyTransaction) + /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn encode(&self, buf: &mut dyn bytes::BufMut) { // tx_type || RLP(receipt) if tx_type != 0 // RLP(receipt) else @@ -75,7 +78,7 @@ impl RLPEncode for Receipt { impl RLPDecode for Receipt { /// Receipts can be encoded in the following formats: /// A) Legacy receipts: rlp(LegacyTransaction) - /// B) Non legacy receipts: rlp(tx_type | rlp(receipt)). + /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { let payload = get_rlp_bytes_item_payload(rlp); let tx_type = match payload.first() { From 35a81c545c53c17f4f92a9e17e9a75b3032a442d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 9 Dec 2024 15:26:46 -0300 Subject: [PATCH 300/345] Add max retries --- crates/networking/p2p/sync.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 8efc7cb4e..64b81ea42 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -19,6 +19,10 @@ use tracing::{debug, info, warn}; use crate::kademlia::KademliaTable; +/// Maximum amount of times we will ask a peer for an account range +/// If the max amount of retries is exceeded we will asume that the state we are requesting is old and no longer available +const MAX_ACCOUNT_RETRIES: usize = 10; + #[derive(Debug)] pub enum SyncMode { Full, @@ -154,6 +158,8 @@ impl SyncManager { for result in set.join_all().await { result?; } + // Start state healing + info!("Starting state healing"); // Set latest block number here to avoid reading state that is currently being synced store.update_latest_block_number(latest_block_number)?; } @@ -285,7 +291,9 @@ async fn rebuild_state_trie( // We cannot keep an open trie here so we will track the root between lookups let mut current_state_root = *EMPTY_TRIE_HASH; // Fetch Account Ranges - loop { + // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available + // In that case we will delegate the work to state healing + for _ in 0..MAX_ACCOUNT_RETRIES { let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer From 90d68f2368b5d687693b8e8cc7008baa85edf91a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 9 Dec 2024 15:27:14 -0300 Subject: [PATCH 301/345] Add methods to manipulate trie state during snap sync --- crates/storage/store/storage.rs | 26 ++++++++++++++++++++++++++ crates/storage/trie/trie.rs | 12 ++++++++++++ 2 files changed, 38 insertions(+) diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs index 6654cf79a..3bd29e5fb 100644 --- a/crates/storage/store/storage.rs +++ b/crates/storage/store/storage.rs @@ -901,6 +901,8 @@ impl Store { self.engine.open_state_trie(*EMPTY_TRIE_HASH) } + /// Methods exclusive for trie management during snap-syncing + // Obtain a state trie from the given state root // Doesn't check if the state root is valid pub fn open_state_trie(&self, state_root: H256) -> Trie { @@ -912,6 +914,30 @@ impl Store { pub fn open_storage_trie(&self, account_hash: H256, storage_root: H256) -> Trie { self.engine.open_storage_trie(account_hash, storage_root) } + + /// Returns true if the given node is part of the state trie's internal storage + pub fn contains_state_node(&self, node_hash: H256) -> Result { + // Root is irrelevant, we only care about the internal state + Ok(self + .open_state_trie(*EMPTY_TRIE_HASH) + .state() + .get_node(node_hash.into())? + .is_some()) + } + + /// Returns true if the given node is part of the given storage trie's internal storage + pub fn contains_storage_node( + &self, + hashed_address: H256, + node_hash: H256, + ) -> Result { + // Root is irrelevant, we only care about the internal state + Ok(self + .open_storage_trie(hashed_address, *EMPTY_TRIE_HASH) + .state() + .get_node(node_hash.into())? + .is_some()) + } } pub fn hash_address(address: &Address) -> Vec { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 56f945093..2ad869c77 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -316,6 +316,18 @@ impl Trie { } } + /// Returns a mutable reference to the trie's internal node state + /// [WARNING] This will allow directly manipulating the trie's state and + /// may lead to inconsistent trie structures if not used resposibly + pub fn state_mut(&mut self) -> &mut TrieState { + &mut self.state + } + + /// Returns a reference to the trie's internal node state + pub fn state(&mut self) -> &TrieState { + &self.state + } + #[cfg(all(test, feature = "libmdbx"))] /// Creates a new Trie based on a temporary Libmdbx DB fn new_temp() -> Self { From ae33bbea90e5189e29a6798c6bc2cd79cbe29c45 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Mon, 9 Dec 2024 18:22:13 -0300 Subject: [PATCH 302/345] feat: fix hive tests --- crates/common/types/receipt.rs | 107 ++++++++++++++--------- crates/networking/rpc/eth/block.rs | 2 +- crates/networking/rpc/eth/transaction.rs | 1 + crates/storage/store/engines/libmdbx.rs | 2 +- 4 files changed, 70 insertions(+), 42 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index ede622310..378438a69 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -1,14 +1,14 @@ use bytes::Bytes; use ethereum_types::{Address, Bloom, BloomInput, H256}; use ethrex_rlp::{ - decode::{get_rlp_bytes_item_payload, RLPDecode}, + decode::{get_rlp_bytes_item_payload, is_encoded_as_bytes, RLPDecode}, encode::RLPEncode, error::RLPDecodeError, structs::{Decoder, Encoder}, }; use serde::{Deserialize, Serialize}; -use super::TxType; +use super::{PrivilegedL2Transaction, TxType}; pub type Index = u64; /// Result of a transaction @@ -31,6 +31,23 @@ impl Receipt { logs, } } + pub fn inner_encode_receipt(&self) -> Vec { + let mut encode_buff = match self.tx_type { + TxType::Legacy => { + vec![] + } + _ => { + vec![self.tx_type as u8] + } + }; + Encoder::new(&mut encode_buff) + .encode_field(&self.succeeded) + .encode_field(&self.cumulative_gas_used) + .encode_field(&self.bloom) + .encode_field(&self.logs) + .finish(); + return encode_buff; + } } fn bloom_from_logs(logs: &[Log]) -> Bloom { @@ -53,22 +70,12 @@ impl RLPEncode for Receipt { // RLP(receipt) else match self.tx_type { TxType::Legacy => { - Encoder::new(buf) - .encode_field(&self.succeeded) - .encode_field(&self.cumulative_gas_used) - .encode_field(&self.bloom) - .encode_field(&self.logs) - .finish(); + let legacy_encoded = self.inner_encode_receipt(); + buf.put_slice(&legacy_encoded); } _ => { - let mut tmp_buff = vec![self.tx_type as u8]; - Encoder::new(&mut tmp_buff) - .encode_field(&self.succeeded) - .encode_field(&self.cumulative_gas_used) - .encode_field(&self.bloom) - .encode_field(&self.logs) - .finish(); - let bytes = Bytes::from(tmp_buff); + let typed_recepipt_encoded = self.inner_encode_receipt(); + let bytes = Bytes::from(typed_recepipt_encoded); bytes.encode(buf); } }; @@ -80,39 +87,59 @@ impl RLPDecode for Receipt { /// A) Legacy receipts: rlp(LegacyTransaction) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - let payload = get_rlp_bytes_item_payload(rlp); - let tx_type = match payload.first() { - Some(tx_type) if *tx_type < 0x7f => match tx_type { + // FIXME: Remove unwrap + if is_encoded_as_bytes(rlp).unwrap() { + let payload = get_rlp_bytes_item_payload(rlp); + // FIXME: Remove unwrap + let tx_type = payload.first().unwrap(); + let receipt_encoding = &payload[1..]; + let tx_type = match tx_type { 0x0 => TxType::Legacy, 0x1 => TxType::EIP2930, 0x2 => TxType::EIP1559, 0x3 => TxType::EIP4844, - 0x7e => TxType::Privileged, + // 0x7e => TxType::PrivilegedL2Transaction, ty => { return Err(RLPDecodeError::Custom(format!( "Invalid transaction type: {ty}" ))) } - }, - Some(_) => TxType::Legacy, - None => return Err(RLPDecodeError::InvalidLength), - }; - let Some(receipt_encoding) = &payload.get(1..) else { - return Err(RLPDecodeError::InvalidLength); - }; - let decoder = Decoder::new(receipt_encoding)?; - let (succeeded, decoder) = decoder.decode_field("succeeded")?; - let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative_gas_used")?; - let (bloom, decoder) = decoder.decode_field("bloom")?; - let (logs, decoder) = decoder.decode_field("logs")?; - let receipt = Receipt { - tx_type, - succeeded, - cumulative_gas_used, - bloom, - logs, - }; - Ok((receipt, decoder.finish()?)) + }; + // FIXME: Remove unwrap + let decoder = Decoder::new(receipt_encoding).unwrap(); + let (succeeded, decoder) = decoder.decode_field("succeded").unwrap(); + let (cumulative_gas_used, decoder) = + decoder.decode_field("cumulative gas used").unwrap(); + let (bloom, decoder) = decoder.decode_field("bloom").unwrap(); + let (logs, decoder) = decoder.decode_field("logs").unwrap(); + Ok(( + Receipt { + tx_type, + succeeded, + bloom, + logs, + cumulative_gas_used, + }, + decoder.finish().unwrap(), + )) + } else { + let decoder = Decoder::new(rlp).unwrap(); + let (succeeded, decoder) = decoder.decode_field("succeded").unwrap(); + let (cumulative_gas_used, decoder) = + decoder.decode_field("cumulative gas used").unwrap(); + let (bloom, decoder) = decoder.decode_field("bloom").unwrap(); + let (logs, decoder) = decoder.decode_field("logs").unwrap(); + Ok(( + Receipt { + tx_type: TxType::Legacy, + succeeded, + bloom, + logs, + cumulative_gas_used, + }, + decoder.finish().unwrap(), + )) + } } } diff --git a/crates/networking/rpc/eth/block.rs b/crates/networking/rpc/eth/block.rs index fe014cc79..ddab39a56 100644 --- a/crates/networking/rpc/eth/block.rs +++ b/crates/networking/rpc/eth/block.rs @@ -297,7 +297,7 @@ impl RpcHandler for GetRawReceipts { }; let receipts: Vec = get_all_block_receipts(block_number, header, body, storage)? .iter() - .map(|receipt| format!("0x{}", hex::encode(receipt.encode_to_vec()))) + .map(|receipt| format!("0x{}", hex::encode(receipt.inner_encode_receipt()))) .collect(); serde_json::to_value(receipts).map_err(|error| RpcErr::Internal(error.to_string())) } diff --git a/crates/networking/rpc/eth/transaction.rs b/crates/networking/rpc/eth/transaction.rs index 2e3d0b076..d19b4c023 100644 --- a/crates/networking/rpc/eth/transaction.rs +++ b/crates/networking/rpc/eth/transaction.rs @@ -277,6 +277,7 @@ impl RpcHandler for GetTransactionReceiptRequest { }; let receipts = block::get_all_block_rpc_receipts(block_number, block.header, block.body, storage)?; + serde_json::to_value(receipts.get(index as usize)) .map_err(|error| RpcErr::Internal(error.to_string())) } diff --git a/crates/storage/store/engines/libmdbx.rs b/crates/storage/store/engines/libmdbx.rs index de729fba6..7637a284c 100644 --- a/crates/storage/store/engines/libmdbx.rs +++ b/crates/storage/store/engines/libmdbx.rs @@ -442,9 +442,9 @@ impl StoreEngine for Store { while let Some((_, encoded_receipt)) = cursor.seek_exact(key).map_err(|_| StoreError::ReadError)? { + receipts.push(encoded_receipt); receipt_index += 1; key = (*block_hash, receipt_index).into(); - receipts.push(encoded_receipt); } Ok(receipts.into_iter().map(|receipt| receipt.to()).collect()) From 97ea51377aa1e5ea9c4babb6784e40b8aaf3635e Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Mon, 9 Dec 2024 18:33:02 -0300 Subject: [PATCH 303/345] feat: add some error handling --- crates/common/rlp/decode.rs | 7 ++++--- crates/common/types/receipt.rs | 12 ++++++------ crates/common/types/transaction.rs | 10 +++++++--- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/crates/common/rlp/decode.rs b/crates/common/rlp/decode.rs index 6418e693a..809100b89 100644 --- a/crates/common/rlp/decode.rs +++ b/crates/common/rlp/decode.rs @@ -492,10 +492,11 @@ pub fn is_encoded_as_bytes(rlp: &[u8]) -> Result { } /// Receives an RLP bytes item (prefix between 0xb8 and 0xbf) and returns its payload -pub fn get_rlp_bytes_item_payload(rlp: &[u8]) -> &[u8] { - let prefix = rlp.first().unwrap(); +pub fn get_rlp_bytes_item_payload(rlp: &[u8]) -> Result<&[u8], RLPDecodeError> { + let prefix = rlp.first().ok_or_else(|| RLPDecodeError::InvalidLength)?; let offset: usize = (prefix - 0xb8 + 1).into(); - &rlp[offset + 1..] + rlp.get(offset + 1..) + .ok_or_else(|| RLPDecodeError::InvalidLength) } /// Decodes the payload of an RLP item from a slice of bytes. diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 378438a69..ac471c589 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -8,7 +8,7 @@ use ethrex_rlp::{ }; use serde::{Deserialize, Serialize}; -use super::{PrivilegedL2Transaction, TxType}; +use crate::types::TxType; pub type Index = u64; /// Result of a transaction @@ -87,11 +87,11 @@ impl RLPDecode for Receipt { /// A) Legacy receipts: rlp(LegacyTransaction) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - // FIXME: Remove unwrap - if is_encoded_as_bytes(rlp).unwrap() { - let payload = get_rlp_bytes_item_payload(rlp); - // FIXME: Remove unwrap - let tx_type = payload.first().unwrap(); + if is_encoded_as_bytes(rlp)? { + let payload = get_rlp_bytes_item_payload(rlp)?; + let tx_type = payload + .first() + .ok_or_else(|| RLPDecodeError::InvalidLength)?; let receipt_encoding = &payload[1..]; let tx_type = match tx_type { 0x0 => TxType::Legacy, diff --git a/crates/common/types/transaction.rs b/crates/common/types/transaction.rs index 33b8872a4..2a160eee1 100644 --- a/crates/common/types/transaction.rs +++ b/crates/common/types/transaction.rs @@ -230,9 +230,13 @@ impl RLPDecode for Transaction { fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { if is_encoded_as_bytes(rlp)? { // Adjust the encoding to get the payload - let payload = get_rlp_bytes_item_payload(rlp); - let tx_type = payload.first().unwrap(); - let tx_encoding = &payload[1..]; + let payload = get_rlp_bytes_item_payload(rlp)?; + let tx_type = payload + .first() + .ok_or_else(|| RLPDecodeError::InvalidLength)?; + let tx_encoding = &payload + .get(1..) + .ok_or_else(|| RLPDecodeError::InvalidLength)?; // Look at the first byte to check if it corresponds to a TransactionType match *tx_type { // Legacy From 1c6ace81b55eb07ecfef75f7733c41976eabb339 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Mon, 9 Dec 2024 18:43:51 -0300 Subject: [PATCH 304/345] chore: clippy --- crates/common/rlp/decode.rs | 5 ++--- crates/common/types/receipt.rs | 4 ++-- crates/common/types/transaction.rs | 4 ++-- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/crates/common/rlp/decode.rs b/crates/common/rlp/decode.rs index 809100b89..348999ff4 100644 --- a/crates/common/rlp/decode.rs +++ b/crates/common/rlp/decode.rs @@ -493,10 +493,9 @@ pub fn is_encoded_as_bytes(rlp: &[u8]) -> Result { /// Receives an RLP bytes item (prefix between 0xb8 and 0xbf) and returns its payload pub fn get_rlp_bytes_item_payload(rlp: &[u8]) -> Result<&[u8], RLPDecodeError> { - let prefix = rlp.first().ok_or_else(|| RLPDecodeError::InvalidLength)?; + let prefix = rlp.first().ok_or(RLPDecodeError::InvalidLength)?; let offset: usize = (prefix - 0xb8 + 1).into(); - rlp.get(offset + 1..) - .ok_or_else(|| RLPDecodeError::InvalidLength) + rlp.get(offset + 1..).ok_or(RLPDecodeError::InvalidLength) } /// Decodes the payload of an RLP item from a slice of bytes. diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index ac471c589..4a0cb812c 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -46,7 +46,7 @@ impl Receipt { .encode_field(&self.bloom) .encode_field(&self.logs) .finish(); - return encode_buff; + encode_buff } } @@ -91,7 +91,7 @@ impl RLPDecode for Receipt { let payload = get_rlp_bytes_item_payload(rlp)?; let tx_type = payload .first() - .ok_or_else(|| RLPDecodeError::InvalidLength)?; + .ok_or(RLPDecodeError::InvalidLength)?; let receipt_encoding = &payload[1..]; let tx_type = match tx_type { 0x0 => TxType::Legacy, diff --git a/crates/common/types/transaction.rs b/crates/common/types/transaction.rs index 2a160eee1..1bc62f439 100644 --- a/crates/common/types/transaction.rs +++ b/crates/common/types/transaction.rs @@ -233,10 +233,10 @@ impl RLPDecode for Transaction { let payload = get_rlp_bytes_item_payload(rlp)?; let tx_type = payload .first() - .ok_or_else(|| RLPDecodeError::InvalidLength)?; + .ok_or(RLPDecodeError::InvalidLength)?; let tx_encoding = &payload .get(1..) - .ok_or_else(|| RLPDecodeError::InvalidLength)?; + .ok_or(RLPDecodeError::InvalidLength)?; // Look at the first byte to check if it corresponds to a TransactionType match *tx_type { // Legacy From 2f5961f15ba7d773fc66e85865da0f683f2167c7 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Mon, 9 Dec 2024 18:52:24 -0300 Subject: [PATCH 305/345] chore: fmt --- crates/common/types/receipt.rs | 4 +--- crates/common/types/transaction.rs | 8 ++------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 4a0cb812c..44c937b44 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -89,9 +89,7 @@ impl RLPDecode for Receipt { fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { if is_encoded_as_bytes(rlp)? { let payload = get_rlp_bytes_item_payload(rlp)?; - let tx_type = payload - .first() - .ok_or(RLPDecodeError::InvalidLength)?; + let tx_type = payload.first().ok_or(RLPDecodeError::InvalidLength)?; let receipt_encoding = &payload[1..]; let tx_type = match tx_type { 0x0 => TxType::Legacy, diff --git a/crates/common/types/transaction.rs b/crates/common/types/transaction.rs index 1bc62f439..8f8fdf33e 100644 --- a/crates/common/types/transaction.rs +++ b/crates/common/types/transaction.rs @@ -231,12 +231,8 @@ impl RLPDecode for Transaction { if is_encoded_as_bytes(rlp)? { // Adjust the encoding to get the payload let payload = get_rlp_bytes_item_payload(rlp)?; - let tx_type = payload - .first() - .ok_or(RLPDecodeError::InvalidLength)?; - let tx_encoding = &payload - .get(1..) - .ok_or(RLPDecodeError::InvalidLength)?; + let tx_type = payload.first().ok_or(RLPDecodeError::InvalidLength)?; + let tx_encoding = &payload.get(1..).ok_or(RLPDecodeError::InvalidLength)?; // Look at the first byte to check if it corresponds to a TransactionType match *tx_type { // Legacy From 581b3258187b84a0682fa4d5538007fdff6c683a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Mon, 9 Dec 2024 19:07:30 -0300 Subject: [PATCH 306/345] State Healing first progress --- crates/networking/p2p/peer_channels.rs | 57 ++++++++++++++++++++-- crates/networking/p2p/sync.rs | 67 +++++++++++++++++++++++++- crates/storage/trie/nibbles.rs | 12 +++++ crates/storage/trie/node.rs | 9 ++++ crates/storage/trie/trie.rs | 4 +- 5 files changed, 142 insertions(+), 7 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 4a722805e..12fa9d444 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -5,8 +5,9 @@ use ethrex_core::{ types::{AccountState, BlockBody, BlockHeader}, H256, U256, }; -use ethrex_rlp::encode::RLPEncode; -use ethrex_trie::verify_range; +use ethrex_rlp::{decode::RLPDecode, encode::RLPEncode}; +use ethrex_trie::Nibbles; +use ethrex_trie::{verify_range, Node}; use tokio::sync::{mpsc, Mutex}; use crate::{ @@ -15,7 +16,8 @@ use crate::{ BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, }, snap::{ - AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, StorageRanges, + AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, GetTrieNodes, + StorageRanges, TrieNodes, }, }, snap::encodable_to_proof, @@ -318,4 +320,53 @@ impl PeerChannels { } Some((storage_keys, storage_values, should_continue)) } + + /// Requests state trie nodes given the root of the trie where they are contained and their path (be them full or partial) + /// Returns the nodes or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_state_trienodes( + &self, + state_root: H256, + paths: Vec, + ) -> Option> { + let request_id = rand::random(); + let expected_nodes = paths.len(); + let request = RLPxMessage::GetTrieNodes(GetTrieNodes { + id: request_id, + root_hash: state_root, + // [acc_path, acc_path,...] -> [[acc_path], [acc_path]] + paths: paths + .into_iter() + .map(|vec| vec![Bytes::from(vec.encode_compact())]) + .collect(), + bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let nodes = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::TrieNodes(TrieNodes { id, nodes })) if id == request_id => { + return Some(nodes) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + (!nodes.is_empty() && nodes.len() <= expected_nodes) + .then(|| { + nodes + .iter() + .map(|node| Node::decode(node)) + .collect::, _>>() + .ok() + }) + .flatten() + } } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 64b81ea42..6de64b8d2 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,5 +1,6 @@ use std::sync::Arc; +use bytes::Bytes; use ethrex_blockchain::error::ChainError; use ethrex_core::{ types::{Block, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, @@ -7,7 +8,7 @@ use ethrex_core::{ }; use ethrex_rlp::encode::RLPEncode; use ethrex_storage::{error::StoreError, Store}; -use ethrex_trie::EMPTY_TRIE_HASH; +use ethrex_trie::{Nibbles, Node, TrieError, EMPTY_TRIE_HASH}; use tokio::{ sync::{ mpsc::{self, error::SendError, Receiver, Sender}, @@ -502,6 +503,66 @@ async fn fetch_storage_batch( } } +async fn state_healing( + block_headers: Vec, + store: Store, + peers: Arc>, +) -> Result<(), SyncError> { + for header in block_headers { + // If we don't have the root node stored then we must fetch it + if !store.contains_state_node(header.state_root)? {} + } + Ok(()) +} + +async fn heal_state_trie( + state_root: H256, + store: Store, + peers: Arc>, +) -> Result<(), SyncError> { + let mut trie = store.open_state_trie(*EMPTY_TRIE_HASH); + let mut trie_state = trie.state_mut(); + // Begin by requesting the root node + let mut paths = vec![Nibbles::default()]; + while !paths.is_empty() { + let peer = peers.lock().await.get_peer_channels().await; + if let Some(nodes) = peer + .request_state_trienodes(state_root, paths.clone()) + .await + { + // For each fetched node: + // - Add its children to the queue (if we don't have them already) + // - If it is a leaf, request its bytecode & storage + // - Add it to the trie's state + for node in nodes { + let path = paths.remove(0); + match &node { + Node::Branch(node) => { + // Add children to the queue + for (index, child) in node.choices.iter().enumerate() { + if trie_state.get_node(child.clone())?.is_none() { + paths.push(path.append_new(index as u8)); + } + } + } + Node::Extension(node) => { + // Add child to the queue + if trie_state.get_node(node.child.clone())?.is_none() { + paths.push(path.concat(node.prefix.clone())); + } + } + Node::Leaf(leaf_node) => { + // Fetch bytecode & storage + } + } + let hash = node.compute_hash(); + trie_state.insert_node(node, hash); + } + } + } + Ok(()) +} + #[derive(thiserror::Error, Debug)] enum SyncError { #[error(transparent)] @@ -509,7 +570,9 @@ enum SyncError { #[error(transparent)] Store(#[from] StoreError), #[error(transparent)] - SendBytecode(#[from] SendError>), + SendHashes(#[from] SendError>), #[error(transparent)] SendStorage(#[from] SendError>), + #[error(transparent)] + Trie(#[from] TrieError), } diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 6b7a91d3c..1477e944e 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -177,6 +177,18 @@ impl Nibbles { }) .collect::>() } + + pub fn concat(&self, other: Nibbles) -> Nibbles { + Nibbles { + data: [self.data.clone(), other.data].concat(), + } + } + + pub fn append_new(&self, nibble: u8) -> Nibbles { + Nibbles { + data: [self.data.clone(), vec![nibble]].concat(), + } + } } impl AsRef<[u8]> for Nibbles { diff --git a/crates/storage/trie/node.rs b/crates/storage/trie/node.rs index 92c634a46..d6bf0b4a5 100644 --- a/crates/storage/trie/node.rs +++ b/crates/storage/trie/node.rs @@ -165,6 +165,15 @@ impl Node { } }) } + + /// Computes the node's hash + pub fn compute_hash(&self) -> NodeHash { + match self { + Node::Branch(n) => n.compute_hash(), + Node::Extension(n) => n.compute_hash(), + Node::Leaf(n) => n.compute_hash(), + } + } } fn decode_child(rlp: &[u8]) -> NodeHash { diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 2ad869c77..7788f8974 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -11,8 +11,6 @@ mod trie_iter; mod verify_range; use ethereum_types::H256; use ethrex_rlp::constants::RLP_NULL; -use nibbles::Nibbles; -use node::Node; use node_hash::NodeHash; use sha3::{Digest, Keccak256}; use std::collections::HashSet; @@ -21,6 +19,8 @@ use std::collections::HashSet; pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB}; pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; +pub use self::nibbles::Nibbles; +pub use self::node::Node; pub use self::verify_range::verify_range; pub use self::error::TrieError; From 4ef7f1185ce17c710169e85c6adfb1bee4756a8f Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 13:36:54 -0300 Subject: [PATCH 307/345] Add storage healing --- crates/networking/p2p/peer_channels.rs | 58 ++++++- crates/networking/p2p/sync.rs | 214 +++++++++++++++++++++---- crates/storage/trie/nibbles.rs | 2 +- crates/storage/trie/state.rs | 9 ++ crates/storage/trie/trie.rs | 4 +- 5 files changed, 250 insertions(+), 37 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 12fa9d444..dc526135b 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -1,4 +1,4 @@ -use std::{sync::Arc, time::Duration}; +use std::{collections::BTreeMap, sync::Arc, time::Duration}; use bytes::Bytes; use ethrex_core::{ @@ -369,4 +369,60 @@ impl PeerChannels { }) .flatten() } + + /// Requests storage trie nodes given the root of the state trie where they are contained and + /// a hashmap mapping the path to the account in the state trie (aka hashed address) to the paths to the nodes in its storage trie (can be full or partial) + /// Returns the nodes or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_storage_trienodes( + &self, + state_root: H256, + paths: BTreeMap>, + ) -> Option> { + let request_id = rand::random(); + let expected_nodes = paths.iter().fold(0, |acc, item| acc + item.1.len()); + let request = RLPxMessage::GetTrieNodes(GetTrieNodes { + id: request_id, + root_hash: state_root, + // {acc_path: [path, path, ...]} -> [[acc_path, path, path, ...]] + paths: paths + .into_iter() + .map(|(acc_path, paths)| { + [vec![Nibbles::from_hex(acc_path.0.to_vec())], paths] + .concat() + .into_iter() + .map(|path| Bytes::from(path.encode_compact())) + .collect() + }) + .collect(), + bytes: MAX_RESPONSE_BYTES, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let nodes = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::TrieNodes(TrieNodes { id, nodes })) if id == request_id => { + return Some(nodes) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + (!nodes.is_empty() && nodes.len() <= expected_nodes) + .then(|| { + nodes + .iter() + .map(|node| Node::decode(node)) + .collect::, _>>() + .ok() + }) + .flatten() + } } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 6de64b8d2..80a23bd5e 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,14 +1,12 @@ -use std::sync::Arc; - -use bytes::Bytes; use ethrex_blockchain::error::ChainError; use ethrex_core::{ - types::{Block, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, + types::{AccountState, Block, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, H256, }; -use ethrex_rlp::encode::RLPEncode; +use ethrex_rlp::{decode::RLPDecode, encode::RLPEncode, error::RLPDecodeError}; use ethrex_storage::{error::StoreError, Store}; -use ethrex_trie::{Nibbles, Node, TrieError, EMPTY_TRIE_HASH}; +use ethrex_trie::{Nibbles, Node, TrieError, TrieState, EMPTY_TRIE_HASH}; +use std::{collections::BTreeMap, sync::Arc}; use tokio::{ sync::{ mpsc::{self, error::SendError, Receiver, Sender}, @@ -124,13 +122,17 @@ impl SyncManager { // TODO: We are currently testing against our implementation that doesn't hold an independant snapshot and can provide all historic state // We should fetch all available state and then resort to state healing to fetch the rest let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); - let mut set = tokio::task::JoinSet::new(); - set.spawn(bytecode_fetcher( + // First set of tasks will be in charge of fetching all available state (state not older than 128 blocks) + let mut sync_set = tokio::task::JoinSet::new(); + // Second state of tasks will be active during the healing phase where we fetch the state we weren't able to in the first phase + let mut healing_set = tokio::task::JoinSet::new(); + // We need the bytecode fetcher to be active during healing too + healing_set.spawn(bytecode_fetcher( bytecode_receiver, self.peers.clone(), store.clone(), )); - set.spawn(fetch_blocks_and_receipts( + sync_set.spawn(fetch_blocks_and_receipts( all_block_hashes.clone(), self.peers.clone(), store.clone(), @@ -139,7 +141,7 @@ impl SyncManager { .iter() .map(|header| header.state_root) .collect::>(); - set.spawn(fetch_snap_state( + sync_set.spawn(fetch_snap_state( bytecode_sender, state_roots.clone(), self.peers.clone(), @@ -156,11 +158,20 @@ impl SyncManager { store.add_block_header(hash, header)?; } // If all processes failed then they are likely to have a common cause (such as unaccessible storage), so return the first error - for result in set.join_all().await { + for result in sync_set.join_all().await { result?; } // Start state healing info!("Starting state healing"); + healing_set.spawn(state_healing( + state_roots.clone(), + store.clone(), + self.peers.clone(), + )); + for result in healing_set.join_all().await { + result?; + } + // Set latest block number here to avoid reading state that is currently being synced store.update_latest_block_number(latest_block_number)?; } @@ -504,13 +515,15 @@ async fn fetch_storage_batch( } async fn state_healing( - block_headers: Vec, + state_roots: Vec, store: Store, peers: Arc>, ) -> Result<(), SyncError> { - for header in block_headers { + for state_root in state_roots { // If we don't have the root node stored then we must fetch it - if !store.contains_state_node(header.state_root)? {} + if !store.contains_state_node(state_root)? { + heal_state_trie(state_root, store.clone(), peers.clone()).await?; + } } Ok(()) } @@ -520,8 +533,14 @@ async fn heal_state_trie( store: Store, peers: Arc>, ) -> Result<(), SyncError> { - let mut trie = store.open_state_trie(*EMPTY_TRIE_HASH); - let mut trie_state = trie.state_mut(); + // Spawn a storage healer for this blocks's storage + let (storage_sender, storage_receiver) = mpsc::channel::>(500); + let storage_healer_handler = tokio::spawn(storage_healer( + state_root, + storage_receiver, + peers.clone(), + store.clone(), + )); // Begin by requesting the root node let mut paths = vec![Nibbles::default()]; while !paths.is_empty() { @@ -530,39 +549,166 @@ async fn heal_state_trie( .request_state_trienodes(state_root, paths.clone()) .await { + let mut storage_roots = vec![]; + let mut code_hashes = vec![]; // For each fetched node: // - Add its children to the queue (if we don't have them already) // - If it is a leaf, request its bytecode & storage // - Add it to the trie's state for node in nodes { let path = paths.remove(0); - match &node { - Node::Branch(node) => { - // Add children to the queue - for (index, child) in node.choices.iter().enumerate() { - if trie_state.get_node(child.clone())?.is_none() { - paths.push(path.append_new(index as u8)); - } - } + // We cannot keep the trie state open + let mut trie = store.open_state_trie(*EMPTY_TRIE_HASH); + let mut trie_state = trie.state_mut(); + paths.extend(node_missing_children(&node, &path, &trie_state)?); + if let Node::Leaf(node) = &node { + // Fetch bytecode & storage + let account = AccountState::decode(&node.value)?; + storage_roots.push(account.storage_root); + code_hashes.push(account.code_hash); + } + let hash = node.compute_hash(); + trie_state.write_node(node, hash)?; + } + // Send storage & bytecode requests + storage_sender.send(storage_roots).await?; + //TODO: send bytecode request here + } + } + // Send empty batch to signal that no more batches are incoming + storage_sender.send(vec![]).await?; + storage_healer_handler + .await + .map_err(|_| StoreError::Custom(String::from("Failed to join storage_handler task")))??; + Ok(()) +} + +/// Waits for incoming hashed addresses from the receiver channel endpoint and queues the associated root nodes for state retrieval +/// Also retrieves their children nodes until we have the full storage trie stored +async fn storage_healer( + state_root: H256, + mut receiver: Receiver>, + peers: Arc>, + store: Store, +) -> Result<(), SyncError> { + const BATCH_SIZE: usize = 200; + // Pending list of bytecodes to fetch + let mut pending_storages: Vec<(H256, Nibbles)> = vec![]; + loop { + match receiver.recv().await { + Some(account_paths) if !account_paths.is_empty() => { + // Add the root paths of each account trie to the queue + pending_storages.extend( + account_paths + .into_iter() + .map(|acc_path| (acc_path, Nibbles::default())), + ); + // If we have enought pending storages to fill a batch, spawn a fetch process + while pending_storages.len() >= BATCH_SIZE { + let mut next_batch: BTreeMap> = BTreeMap::new(); + // Group pending storages by account path + // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries + for (account, path) in pending_storages.drain(..BATCH_SIZE) { + next_batch.entry(account).or_default().push(path); } - Node::Extension(node) => { - // Add child to the queue - if trie_state.get_node(node.child.clone())?.is_none() { - paths.push(path.concat(node.prefix.clone())); + let return_batch = + heal_storage_batch(state_root, next_batch, peers.clone(), store.clone()) + .await?; + for (acc_path, paths) in return_batch { + for path in paths { + pending_storages.push((acc_path, path)); } } - Node::Leaf(leaf_node) => { - // Fetch bytecode & storage - } } - let hash = node.compute_hash(); - trie_state.insert_node(node, hash); + } + // Disconnect / Empty message signaling no more bytecodes to sync + _ => break, + } + } + // We have no more incoming requests, process the remaining batches + while !pending_storages.is_empty() { + let mut next_batch: BTreeMap> = BTreeMap::new(); + // Group pending storages by account path + // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries + for (account, path) in pending_storages.drain(..BATCH_SIZE.min(pending_storages.len())) { + next_batch.entry(account).or_default().push(path); + } + let return_batch = + heal_storage_batch(state_root, next_batch, peers.clone(), store.clone()).await?; + for (acc_path, paths) in return_batch { + for path in paths { + pending_storages.push((acc_path, path)); } } } Ok(()) } +/// Receives a set of storage trie paths (grouped by their corresponding account's state trie path), +/// fetches their respective nodes, stores them, and returns their children paths and the paths that couldn't be fetched so they can be returned to the queue +async fn heal_storage_batch( + state_root: H256, + mut batch: BTreeMap>, + peers: Arc>, + store: Store, +) -> Result>, StoreError> { + loop { + let peer = peers.lock().await.get_peer_channels().await; + if let Some(mut nodes) = peer + .request_storage_trienodes(state_root, batch.clone()) + .await + { + debug!("Received {} nodes", nodes.len()); + // Process the nodes for each account path + for (acc_path, paths) in batch.iter_mut() { + let mut trie = store.open_storage_trie(*acc_path, *EMPTY_TRIE_HASH); + let mut trie_state = trie.state_mut(); + // Get the corresponding nodes + for node in nodes.drain(..paths.len().min(nodes.len())) { + let path = paths.remove(0); + // Add children to batch + let children = node_missing_children(&node, &path, trie_state)?; + paths.extend(children); + // Add node to the state + let hash = node.compute_hash(); + trie_state.write_node(node, hash)?; + } + // Cut the loop if we ran out of nodes + if nodes.is_empty() { + break; + } + } + // Return remaining and added paths to be added to the queue + return Ok(batch); + } + } +} + +/// Returns the partial paths to the node's children if they are not already part of the trie state +fn node_missing_children( + node: &Node, + parent_path: &Nibbles, + trie_state: &TrieState, +) -> Result, TrieError> { + let mut paths = Vec::new(); + match &node { + Node::Branch(node) => { + for (index, child) in node.choices.iter().enumerate() { + if trie_state.get_node(child.clone())?.is_none() { + paths.push(parent_path.append_new(index as u8)); + } + } + } + Node::Extension(node) => { + if trie_state.get_node(node.child.clone())?.is_none() { + paths.push(parent_path.concat(node.prefix.clone())); + } + } + _ => {} + } + Ok(paths) +} + #[derive(thiserror::Error, Debug)] enum SyncError { #[error(transparent)] @@ -575,4 +721,6 @@ enum SyncError { SendStorage(#[from] SendError>), #[error(transparent)] Trie(#[from] TrieError), + #[error(transparent)] + RLP(#[from] RLPDecodeError), } diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 1477e944e..300776713 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -8,7 +8,7 @@ use ethrex_rlp::{ }; /// Struct representing a list of nibbles (half-bytes) -#[derive(Debug, Clone, Default, PartialEq)] +#[derive(Debug, Clone, Default, PartialEq, Eq, PartialOrd, Ord)] pub struct Nibbles { data: Vec, } diff --git a/crates/storage/trie/state.rs b/crates/storage/trie/state.rs index be42925e9..9a0b68c2f 100644 --- a/crates/storage/trie/state.rs +++ b/crates/storage/trie/state.rs @@ -75,4 +75,13 @@ impl TrieState { // Commit self self.db.put(node_hash.into(), node.encode_to_vec()) } + + /// Writes a node directly to the DB bypassing the cache + pub fn write_node(&mut self, node: Node, hash: NodeHash) -> Result<(), TrieError> { + // Don't insert the node if it is already inlined on the parent + if matches!(hash, NodeHash::Hashed(_)) { + self.db.put(hash.into(), node.encode_to_vec())?; + } + Ok(()) + } } diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs index 7788f8974..d2d8021e0 100644 --- a/crates/storage/trie/trie.rs +++ b/crates/storage/trie/trie.rs @@ -20,11 +20,11 @@ pub use self::db::{libmdbx::LibmdbxTrieDB, libmdbx_dupsort::LibmdbxDupsortTrieDB pub use self::db::{in_memory::InMemoryTrieDB, TrieDB}; pub use self::nibbles::Nibbles; -pub use self::node::Node; pub use self::verify_range::verify_range; +pub use self::{node::Node, state::TrieState}; pub use self::error::TrieError; -use self::{node::LeafNode, state::TrieState, trie_iter::TrieIterator}; +use self::{node::LeafNode, trie_iter::TrieIterator}; use lazy_static::lazy_static; From 2238897a82e9587e89f874afb7276f3206fc824a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 13:40:05 -0300 Subject: [PATCH 308/345] Intergrate bytecode sender --- crates/networking/p2p/sync.rs | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 80a23bd5e..e7ce4171d 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -142,7 +142,7 @@ impl SyncManager { .map(|header| header.state_root) .collect::>(); sync_set.spawn(fetch_snap_state( - bytecode_sender, + bytecode_sender.clone(), state_roots.clone(), self.peers.clone(), store.clone(), @@ -164,6 +164,7 @@ impl SyncManager { // Start state healing info!("Starting state healing"); healing_set.spawn(state_healing( + bytecode_sender, state_roots.clone(), store.clone(), self.peers.clone(), @@ -277,9 +278,6 @@ async fn fetch_snap_state( ) .await? } - // We finished syncing the available state, lets make the fetcher processes aware - // Send empty batches to signal that no more batches are incoming - bytecode_sender.send(vec![]).await?; Ok(()) } @@ -515,6 +513,7 @@ async fn fetch_storage_batch( } async fn state_healing( + bytecode_sender: Sender>, state_roots: Vec, store: Store, peers: Arc>, @@ -522,13 +521,23 @@ async fn state_healing( for state_root in state_roots { // If we don't have the root node stored then we must fetch it if !store.contains_state_node(state_root)? { - heal_state_trie(state_root, store.clone(), peers.clone()).await?; + heal_state_trie( + bytecode_sender.clone(), + state_root, + store.clone(), + peers.clone(), + ) + .await?; } } + // We finished both sync & healing, lets make the bytecode fetcher process aware + // Send empty batches to signal that no more batches are incoming + bytecode_sender.send(vec![]).await?; Ok(()) } async fn heal_state_trie( + bytecode_sender: Sender>, state_root: H256, store: Store, peers: Arc>, @@ -572,7 +581,7 @@ async fn heal_state_trie( } // Send storage & bytecode requests storage_sender.send(storage_roots).await?; - //TODO: send bytecode request here + bytecode_sender.send(code_hashes).await?; } } // Send empty batch to signal that no more batches are incoming From a878cf13a954d610c3068644dea33b1a50a8650d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 13:52:06 -0300 Subject: [PATCH 309/345] Fix --- crates/networking/p2p/sync.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 9a587b7a3..2a3e17503 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -559,7 +559,7 @@ async fn heal_state_trie( .await { // TODO: THIS SHOULD BE HASHED ADDRESSES!!1 - let mut storage_roots = vec![]; + let mut hahsed_addresses = vec![]; let mut code_hashes = vec![]; // For each fetched node: // - Add its children to the queue (if we don't have them already) @@ -574,14 +574,20 @@ async fn heal_state_trie( if let Node::Leaf(node) = &node { // Fetch bytecode & storage let account = AccountState::decode(&node.value)?; - storage_roots.push(account.storage_root); + // By now we should have the full path = account hash + let path = &path.concat(node.partial.clone()).to_bytes(); + if path.len() != 32 { + // Something went wrong + return Err(SyncError::CorruptPath) + } + hahsed_addresses.push(H256::from_slice(&path)); code_hashes.push(account.code_hash); } let hash = node.compute_hash(); trie_state.write_node(node, hash)?; } // Send storage & bytecode requests - storage_sender.send(storage_roots).await?; + storage_sender.send(hahsed_addresses).await?; bytecode_sender.send(code_hashes).await?; } } @@ -733,4 +739,6 @@ enum SyncError { Trie(#[from] TrieError), #[error(transparent)] RLP(#[from] RLPDecodeError), + #[error("Corrupt path during state healing")] + CorruptPath } From 5e0c501e910ac7a04522975936c6a7139770bb14 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 10 Dec 2024 14:15:37 -0300 Subject: [PATCH 310/345] feat: use consensus encoding for receipts --- crates/common/types/block.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/common/types/block.rs b/crates/common/types/block.rs index 90bab35d9..20c9e8262 100644 --- a/crates/common/types/block.rs +++ b/crates/common/types/block.rs @@ -241,7 +241,7 @@ pub fn compute_receipts_root(receipts: &[Receipt]) -> H256 { let iter = receipts .iter() .enumerate() - .map(|(idx, receipt)| (idx.encode_to_vec(), receipt.encode_to_vec())); + .map(|(idx, receipt)| (idx.encode_to_vec(), receipt.inner_encode_receipt())); Trie::compute_hash_from_unsorted_iter(iter) } From cd755eda7aaf494fc93b06142c40e9bd8430b94c Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 10 Dec 2024 14:21:20 -0300 Subject: [PATCH 311/345] chore: cargo fmt --- crates/storage/store/engines/in_memory.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/storage/store/engines/in_memory.rs b/crates/storage/store/engines/in_memory.rs index 264d15704..767d6a62a 100644 --- a/crates/storage/store/engines/in_memory.rs +++ b/crates/storage/store/engines/in_memory.rs @@ -368,7 +368,7 @@ impl StoreEngine for Store { Ok(receipts .into_iter() - .map(|(_index, receipt)| receipt.clone()) + .map(|(_index, receipt)| receipt.clone()) .collect()) } From 161565c1de3e5cdfd53aaace2b33072b125fd856 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 10 Dec 2024 14:45:11 -0300 Subject: [PATCH 312/345] docs: add some comments about receipts --- crates/common/types/receipt.rs | 29 +++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 44c937b44..5df1570ba 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -31,6 +31,19 @@ impl Receipt { logs, } } + // By reading the typed transactions EIP, and some geth code: + // - https://eips.ethereum.org/EIPS/eip-2718 + // - https://github.com/ethereum/go-ethereum/blob/330190e476e2a2de4aac712551629a4134f802d5/core/types/receipt.go#L143 + // We've noticed the are some subtleties around encoding receipts and transactions. + // First, `inner_encode_receipt` will encode a receipt according + // to the RLP of its fields, if typed, the RLP of the fields + // is padded with the byte representing this type. + // For P2P messages, receipts are re-encoded as bytes + // (see the `encode` implementation for receipt). + // For debug and computing receipt roots, the expected + // RLP encodings are the ones returned by `inner_encode_receipt`. + // On some documentations, this is also called the `consensus-encoding` + // for a receipt. pub fn inner_encode_receipt(&self) -> Vec { let mut encode_buff = match self.tx_type { TxType::Legacy => { @@ -66,8 +79,6 @@ impl RLPEncode for Receipt { /// A) Legacy receipts: rlp(LegacyTransaction) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn encode(&self, buf: &mut dyn bytes::BufMut) { - // tx_type || RLP(receipt) if tx_type != 0 - // RLP(receipt) else match self.tx_type { TxType::Legacy => { let legacy_encoded = self.inner_encode_receipt(); @@ -103,13 +114,11 @@ impl RLPDecode for Receipt { ))) } }; - // FIXME: Remove unwrap - let decoder = Decoder::new(receipt_encoding).unwrap(); - let (succeeded, decoder) = decoder.decode_field("succeded").unwrap(); - let (cumulative_gas_used, decoder) = - decoder.decode_field("cumulative gas used").unwrap(); - let (bloom, decoder) = decoder.decode_field("bloom").unwrap(); - let (logs, decoder) = decoder.decode_field("logs").unwrap(); + let decoder = Decoder::new(receipt_encoding)?; + let (succeeded, decoder) = decoder.decode_field("succeded")?; + let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative gas used")?; + let (bloom, decoder) = decoder.decode_field("bloom")?; + let (logs, decoder) = decoder.decode_field("logs")?; Ok(( Receipt { tx_type, @@ -121,7 +130,7 @@ impl RLPDecode for Receipt { decoder.finish().unwrap(), )) } else { - let decoder = Decoder::new(rlp).unwrap(); + let decoder = Decoder::new(rlp)?; let (succeeded, decoder) = decoder.decode_field("succeded").unwrap(); let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative gas used").unwrap(); From bc4b9e0131f5331c243825b7a05be1bbb7dd5fe3 Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 10 Dec 2024 14:49:06 -0300 Subject: [PATCH 313/345] fix: remove unwraps --- crates/common/types/receipt.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 5df1570ba..2ea48bdbd 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -127,15 +127,14 @@ impl RLPDecode for Receipt { logs, cumulative_gas_used, }, - decoder.finish().unwrap(), + decoder.finish()?, )) } else { let decoder = Decoder::new(rlp)?; - let (succeeded, decoder) = decoder.decode_field("succeded").unwrap(); - let (cumulative_gas_used, decoder) = - decoder.decode_field("cumulative gas used").unwrap(); - let (bloom, decoder) = decoder.decode_field("bloom").unwrap(); - let (logs, decoder) = decoder.decode_field("logs").unwrap(); + let (succeeded, decoder) = decoder.decode_field("succeded")?; + let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative gas used")?; + let (bloom, decoder) = decoder.decode_field("bloom")?; + let (logs, decoder) = decoder.decode_field("logs")?; Ok(( Receipt { tx_type: TxType::Legacy, @@ -144,7 +143,7 @@ impl RLPDecode for Receipt { logs, cumulative_gas_used, }, - decoder.finish().unwrap(), + decoder.finish()?, )) } } From 8ff54d9eb7e2cbb2fd077ef5f2cfa3d8165c891e Mon Sep 17 00:00:00 2001 From: fkrause98 Date: Tue, 10 Dec 2024 14:49:51 -0300 Subject: [PATCH 314/345] fix: restore priviliged match --- crates/common/types/receipt.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 2ea48bdbd..b7219d7a0 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -107,7 +107,7 @@ impl RLPDecode for Receipt { 0x1 => TxType::EIP2930, 0x2 => TxType::EIP1559, 0x3 => TxType::EIP4844, - // 0x7e => TxType::PrivilegedL2Transaction, + 0x7e => TxType::Privileged, ty => { return Err(RLPDecodeError::Custom(format!( "Invalid transaction type: {ty}" From ef2caf2c3c809acd57658d5de2b7f9bddf83865e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 16:38:45 -0300 Subject: [PATCH 315/345] Fix --- crates/networking/p2p/peer_channels.rs | 6 +++--- crates/networking/rpc/engine/payload.rs | 24 ------------------------ crates/networking/rpc/rpc.rs | 4 ---- crates/networking/rpc/types/payload.rs | 7 ------- 4 files changed, 3 insertions(+), 38 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index dc526135b..43759e4a0 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -5,7 +5,7 @@ use ethrex_core::{ types::{AccountState, BlockBody, BlockHeader}, H256, U256, }; -use ethrex_rlp::{decode::RLPDecode, encode::RLPEncode}; +use ethrex_rlp::encode::RLPEncode; use ethrex_trie::Nibbles; use ethrex_trie::{verify_range, Node}; use tokio::sync::{mpsc, Mutex}; @@ -363,7 +363,7 @@ impl PeerChannels { .then(|| { nodes .iter() - .map(|node| Node::decode(node)) + .map(|node| Node::decode_raw(node)) .collect::, _>>() .ok() }) @@ -419,7 +419,7 @@ impl PeerChannels { .then(|| { nodes .iter() - .map(|node| Node::decode(node)) + .map(|node| Node::decode_raw(node)) .collect::, _>>() .ok() }) diff --git a/crates/networking/rpc/engine/payload.rs b/crates/networking/rpc/engine/payload.rs index 5cb7c9b40..161e6e4e6 100644 --- a/crates/networking/rpc/engine/payload.rs +++ b/crates/networking/rpc/engine/payload.rs @@ -49,22 +49,6 @@ impl NewPayloadRequest { } } -<<<<<<< HEAD -impl RpcHandler for NewPayloadV3Request { - fn parse(params: &Option>) -> Result { - let params = params - .as_ref() - .ok_or(RpcErr::BadParams("No params provided".to_owned()))?; - // if params.len() != 3 { - // return Err(RpcErr::BadParams("Expected 3 params".to_owned())); - // } - Ok(NewPayloadV3Request { - payload: serde_json::from_value(params[0].clone()) - .map_err(|_| RpcErr::WrongParam("payload".to_string()))?, - expected_blob_versioned_hashes: vec![], - parent_beacon_block_root: H256::zero(), - }) -======= fn parent_beacon_block_root(&self) -> Option { match self.version { NewPayloadRequestVersion::V1 => None, @@ -119,7 +103,6 @@ impl RpcHandler for NewPayloadV3Request { *expected_blob_versioned_hashes != blob_versioned_hashes } } ->>>>>>> 514885f1dc7b533e609d1dd9b73cb4b6c45cac92 } fn handle(&self, context: RpcApiContext) -> Result { @@ -146,17 +129,10 @@ impl RpcHandler for NewPayloadV3Request { // Check timestamp is post valid fork let chain_config = storage.get_chain_config()?; -<<<<<<< HEAD - // let current_fork = chain_config.get_fork(block.header.timestamp); - // if current_fork < Fork::Cancun { - // return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); - // } -======= let current_fork = chain_config.get_fork(block.header.timestamp); if current_fork < self.valid_fork() { return Err(RpcErr::UnsuportedFork(format!("{current_fork:?}"))); } ->>>>>>> 514885f1dc7b533e609d1dd9b73cb4b6c45cac92 // Check that block_hash is valid let actual_block_hash = block.hash(); diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 23b3a10c8..2e8b3c854 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -259,11 +259,7 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ForkChoiceUpdatedV3::call(req, context), "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), -<<<<<<< HEAD - "engine_newPayloadV2" => NewPayloadV3Request::call(req, context), -======= "engine_newPayloadV2" => NewPayloadV2Request::call(req, context), ->>>>>>> 514885f1dc7b533e609d1dd9b73cb4b6c45cac92 "engine_exchangeTransitionConfigurationV1" => { ExchangeTransitionConfigV1Req::call(req, context) } diff --git a/crates/networking/rpc/types/payload.rs b/crates/networking/rpc/types/payload.rs index 77c1d69b5..415e1d6c3 100644 --- a/crates/networking/rpc/types/payload.rs +++ b/crates/networking/rpc/types/payload.rs @@ -35,12 +35,6 @@ pub struct ExecutionPayload { pub block_hash: H256, transactions: Vec, withdrawals: Vec, -<<<<<<< HEAD - #[serde(with = "serde_utils::u64::hex_str", default)] - blob_gas_used: u64, - #[serde(with = "serde_utils::u64::hex_str", default)] - excess_blob_gas: u64, -======= // ExecutionPayloadV3 fields. Optional since we support V2 too #[serde( skip_serializing_if = "Option::is_none", @@ -54,7 +48,6 @@ pub struct ExecutionPayload { default )] pub excess_blob_gas: Option, ->>>>>>> 514885f1dc7b533e609d1dd9b73cb4b6c45cac92 } #[derive(Clone, Debug)] From f33584f26a60955c90f4f383e626123099226123 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 16:54:46 -0300 Subject: [PATCH 316/345] Fix --- crates/networking/p2p/sync.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 2a3e17503..27f8eb9d1 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -558,7 +558,6 @@ async fn heal_state_trie( .request_state_trienodes(state_root, paths.clone()) .await { - // TODO: THIS SHOULD BE HASHED ADDRESSES!!1 let mut hahsed_addresses = vec![]; let mut code_hashes = vec![]; // For each fetched node: @@ -710,13 +709,13 @@ fn node_missing_children( match &node { Node::Branch(node) => { for (index, child) in node.choices.iter().enumerate() { - if trie_state.get_node(child.clone())?.is_none() { + if child.is_valid() && trie_state.get_node(child.clone())?.is_none() { paths.push(parent_path.append_new(index as u8)); } } } Node::Extension(node) => { - if trie_state.get_node(node.child.clone())?.is_none() { + if child.is_valid() && trie_state.get_node(node.child.clone())?.is_none() { paths.push(parent_path.concat(node.prefix.clone())); } } @@ -740,5 +739,5 @@ enum SyncError { #[error(transparent)] RLP(#[from] RLPDecodeError), #[error("Corrupt path during state healing")] - CorruptPath + CorruptPath, } From 385e2645f12603beb7eda4c5e9f507fa8d7e7021 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 10 Dec 2024 16:55:11 -0300 Subject: [PATCH 317/345] Fix --- crates/networking/p2p/sync.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 27f8eb9d1..e8a1a152d 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -715,7 +715,7 @@ fn node_missing_children( } } Node::Extension(node) => { - if child.is_valid() && trie_state.get_node(node.child.clone())?.is_none() { + if node.child.is_valid() && trie_state.get_node(node.child.clone())?.is_none() { paths.push(parent_path.concat(node.prefix.clone())); } } From f002e916d55dbbc9f3bfbc481d9046aeb347cacd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 10:24:40 -0300 Subject: [PATCH 318/345] Fix --- crates/networking/p2p/sync.rs | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index e8a1a152d..f60b59519 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -357,15 +357,14 @@ async fn rebuild_state_trie( } } } - if current_state_root != state_root { - warn!("State sync failed for state root {state_root}"); + if current_state_root == state_root { + debug!("Completed state sync for state root {state_root}"); } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - debug!("Completed state sync for state root {state_root}"); Ok(()) } @@ -577,17 +576,24 @@ async fn heal_state_trie( let path = &path.concat(node.partial.clone()).to_bytes(); if path.len() != 32 { // Something went wrong - return Err(SyncError::CorruptPath) + return Err(SyncError::CorruptPath); } hahsed_addresses.push(H256::from_slice(&path)); - code_hashes.push(account.code_hash); + if account.code_hash != *EMPTY_KECCACK_HASH { + code_hashes.push(account.code_hash); + } } let hash = node.compute_hash(); trie_state.write_node(node, hash)?; + info!("Node stored"); } // Send storage & bytecode requests - storage_sender.send(hahsed_addresses).await?; - bytecode_sender.send(code_hashes).await?; + if !hahsed_addresses.is_empty() { + storage_sender.send(hahsed_addresses).await?; + } + if !code_hashes.is_empty() { + bytecode_sender.send(code_hashes).await?; + } } } // Send empty batch to signal that no more batches are incoming From 52831d010ad48d084129700fac182c5119e828b9 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 11:36:04 -0300 Subject: [PATCH 319/345] Fix --- crates/networking/p2p/peer_channels.rs | 5 ++--- crates/networking/p2p/sync.rs | 7 +++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 43759e4a0..152fd5a72 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -390,11 +390,10 @@ impl PeerChannels { paths: paths .into_iter() .map(|(acc_path, paths)| { - [vec![Nibbles::from_hex(acc_path.0.to_vec())], paths] - .concat() + [vec![Bytes::from(acc_path.0.to_vec())], paths .into_iter() .map(|path| Bytes::from(path.encode_compact())) - .collect() + .collect()].concat() }) .collect(), bytes: MAX_RESPONSE_BYTES, diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index f60b59519..00a2cfe18 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -303,6 +303,7 @@ async fn rebuild_state_trie( // Fetch Account Ranges // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available // In that case we will delegate the work to state healing + // TODO: Make it so that exceeding max replies will also stop requests for older blocks for _ in 0..MAX_ACCOUNT_RETRIES { let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); @@ -578,14 +579,16 @@ async fn heal_state_trie( // Something went wrong return Err(SyncError::CorruptPath); } - hahsed_addresses.push(H256::from_slice(&path)); + if account.storage_root != *EMPTY_TRIE_HASH { + hahsed_addresses.push(H256::from_slice(&path)); + } if account.code_hash != *EMPTY_KECCACK_HASH { code_hashes.push(account.code_hash); } } let hash = node.compute_hash(); trie_state.write_node(node, hash)?; - info!("Node stored"); + info!("State Node stored"); } // Send storage & bytecode requests if !hahsed_addresses.is_empty() { From 5d58c9a258f23a1aeeb1078b3a4f19ed19f64438 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 12:59:57 -0300 Subject: [PATCH 320/345] Fix --- crates/networking/p2p/sync.rs | 55 +++++++++++++++++++++++++---------- 1 file changed, 40 insertions(+), 15 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 00a2cfe18..6b4d17f93 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -14,13 +14,13 @@ use tokio::{ }, time::Instant, }; -use tracing::{debug, info, warn}; +use tracing::{info, debug, warn}; use crate::kademlia::KademliaTable; -/// Maximum amount of times we will ask a peer for an account range +/// Maximum amount of times we will ask a peer for an account/storage range /// If the max amount of retries is exceeded we will asume that the state we are requesting is old and no longer available -const MAX_ACCOUNT_RETRIES: usize = 10; +const MAX_RETRIES: usize = 10; #[derive(Debug)] pub enum SyncMode { @@ -119,8 +119,6 @@ impl SyncManager { // snap-sync: launch tasks to fetch blocks and state in parallel // - Fetch each block's state via snap p2p requests // - Fetch each blocks and its receipts via eth p2p requests - // TODO: We are currently testing against our implementation that doesn't hold an independant snapshot and can provide all historic state - // We should fetch all available state and then resort to state healing to fetch the rest let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); // First set of tasks will be in charge of fetching all available state (state not older than 128 blocks) let mut sync_set = tokio::task::JoinSet::new(); @@ -163,6 +161,7 @@ impl SyncManager { } // Start state healing info!("Starting state healing"); + let start_time = Instant::now(); healing_set.spawn(state_healing( bytecode_sender, state_roots.clone(), @@ -172,6 +171,7 @@ impl SyncManager { for result in healing_set.join_all().await { result?; } + info!("State healing finished in {} seconds", start_time.elapsed().as_secs()); // Set latest block number here to avoid reading state that is currently being synced store.update_latest_block_number(latest_block_number)?; @@ -266,17 +266,23 @@ async fn fetch_snap_state( peers: Arc>, store: Store, ) -> Result<(), SyncError> { - debug!("Syncing state roots: {}", state_roots.len()); + info!("Syncing state roots: {}", state_roots.len()); // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { // TODO: maybe spawn taks here instead of awaiting - rebuild_state_trie( + match rebuild_state_trie( bytecode_sender.clone(), state_root, peers.clone(), store.clone(), ) - .await? + .await { + Ok(_) => {}, + // If we reached the maximum number of retries then the state we are fetching is most probably old and no longer part of our peer's snapshots + // We should give up on fetching this and older block's state and instead begin state healing + Err(SyncError::MaxRetries) => {dbg!("break"); break}, + err => return err + } } Ok(()) } @@ -304,13 +310,17 @@ async fn rebuild_state_trie( // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available // In that case we will delegate the work to state healing // TODO: Make it so that exceeding max replies will also stop requests for older blocks - for _ in 0..MAX_ACCOUNT_RETRIES { + let mut retry_count = 0; + while retry_count < MAX_RETRIES { + dbg!(retry_count); let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer .request_account_range(state_root, start_account_hash) .await { + // Reset retry counter for the following batch (if needed) + retry_count = 0; // Update starting hash for next batch if should_continue { start_account_hash = *account_hashes.last().unwrap(); @@ -356,17 +366,25 @@ async fn rebuild_state_trie( // All accounts fetched! break; } + } else { + retry_count += 1; } } + dbg!(retry_count); if current_state_root == state_root { - debug!("Completed state sync for state root {state_root}"); + info!("Completed state sync for state root {state_root}"); } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - Ok(()) + if retry_count == MAX_RETRIES { + dbg!("Max Retries error"); + Err(SyncError::MaxRetries) + } else { + Ok(()) + } } /// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches @@ -445,7 +463,7 @@ async fn storage_fetcher( Some(account_and_root) if !account_and_root.is_empty() => { // Add hashes to the queue pending_storage.extend(account_and_root); - // If we have enought pending bytecodes to fill a batch, spawn a fetch process + // If we have enought pending storages to fill a batch, spawn a fetch process while pending_storage.len() >= BATCH_SIZE { let next_batch = pending_storage.drain(..BATCH_SIZE).collect::>(); let remaining = @@ -478,15 +496,15 @@ async fn fetch_storage_batch( state_root: H256, peers: Arc>, store: Store, -) -> Result, StoreError> { - loop { +) -> Result, SyncError> { + for _ in 0..MAX_RETRIES { let peer = peers.lock().await.get_peer_channels().await; let (batch_hahses, batch_roots) = batch.clone().into_iter().unzip(); if let Some((mut keys, mut values, incomplete)) = peer .request_storage_ranges(state_root, batch_roots, batch_hahses, H256::zero()) .await { - debug!("Received {} storage ranges", keys.len()); + info!("Received {} storage ranges", keys.len()); let mut _last_range; // Hold on to the last batch (if incomplete) if incomplete { @@ -510,6 +528,9 @@ async fn fetch_storage_batch( return Ok(batch); } } + // This is a corner case where we fetched an account range for a block but the chain has moved on and the block + // was dropped by the peer's snapshot. We will keep the fetcher alive to avoid errors and stop fetching as from the next block + Ok(vec![]) } async fn state_healing( @@ -749,4 +770,8 @@ enum SyncError { RLP(#[from] RLPDecodeError), #[error("Corrupt path during state healing")] CorruptPath, + #[error("Max Retries Reached")] + // This is more of an internal signal rather than an error, it should never be returned to the user + // It marks that we have reached the end of the available state (last 128 blocks) and we should begin state healing + MaxRetries } From e90bc1d675c6e710136325ddf9a7cf8f97bf3bf3 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 13:01:33 -0300 Subject: [PATCH 321/345] Remove debug prints --- crates/networking/p2p/peer_channels.rs | 12 +++++++---- crates/networking/p2p/sync.rs | 30 +++++++++++++------------- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 152fd5a72..2f6ae9717 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -390,10 +390,14 @@ impl PeerChannels { paths: paths .into_iter() .map(|(acc_path, paths)| { - [vec![Bytes::from(acc_path.0.to_vec())], paths - .into_iter() - .map(|path| Bytes::from(path.encode_compact())) - .collect()].concat() + [ + vec![Bytes::from(acc_path.0.to_vec())], + paths + .into_iter() + .map(|path| Bytes::from(path.encode_compact())) + .collect(), + ] + .concat() }) .collect(), bytes: MAX_RESPONSE_BYTES, diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 6b4d17f93..c5214e262 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -14,7 +14,7 @@ use tokio::{ }, time::Instant, }; -use tracing::{info, debug, warn}; +use tracing::{debug, info, warn}; use crate::kademlia::KademliaTable; @@ -171,7 +171,10 @@ impl SyncManager { for result in healing_set.join_all().await { result?; } - info!("State healing finished in {} seconds", start_time.elapsed().as_secs()); + info!( + "State healing finished in {} seconds", + start_time.elapsed().as_secs() + ); // Set latest block number here to avoid reading state that is currently being synced store.update_latest_block_number(latest_block_number)?; @@ -266,7 +269,7 @@ async fn fetch_snap_state( peers: Arc>, store: Store, ) -> Result<(), SyncError> { - info!("Syncing state roots: {}", state_roots.len()); + debug!("Syncing state roots: {}", state_roots.len()); // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { // TODO: maybe spawn taks here instead of awaiting @@ -276,12 +279,13 @@ async fn fetch_snap_state( peers.clone(), store.clone(), ) - .await { - Ok(_) => {}, + .await + { + Ok(_) => {} // If we reached the maximum number of retries then the state we are fetching is most probably old and no longer part of our peer's snapshots // We should give up on fetching this and older block's state and instead begin state healing - Err(SyncError::MaxRetries) => {dbg!("break"); break}, - err => return err + Err(SyncError::MaxRetries) => break, + err => return err, } } Ok(()) @@ -310,9 +314,8 @@ async fn rebuild_state_trie( // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available // In that case we will delegate the work to state healing // TODO: Make it so that exceeding max replies will also stop requests for older blocks - let mut retry_count = 0; + let mut retry_count = 0; while retry_count < MAX_RETRIES { - dbg!(retry_count); let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer @@ -370,7 +373,6 @@ async fn rebuild_state_trie( retry_count += 1; } } - dbg!(retry_count); if current_state_root == state_root { info!("Completed state sync for state root {state_root}"); } @@ -380,7 +382,6 @@ async fn rebuild_state_trie( .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; if retry_count == MAX_RETRIES { - dbg!("Max Retries error"); Err(SyncError::MaxRetries) } else { Ok(()) @@ -496,7 +497,7 @@ async fn fetch_storage_batch( state_root: H256, peers: Arc>, store: Store, -) -> Result, SyncError> { +) -> Result, StoreError> { for _ in 0..MAX_RETRIES { let peer = peers.lock().await.get_peer_channels().await; let (batch_hahses, batch_roots) = batch.clone().into_iter().unzip(); @@ -504,7 +505,7 @@ async fn fetch_storage_batch( .request_storage_ranges(state_root, batch_roots, batch_hahses, H256::zero()) .await { - info!("Received {} storage ranges", keys.len()); + debug!("Received {} storage ranges", keys.len()); let mut _last_range; // Hold on to the last batch (if incomplete) if incomplete { @@ -609,7 +610,6 @@ async fn heal_state_trie( } let hash = node.compute_hash(); trie_state.write_node(node, hash)?; - info!("State Node stored"); } // Send storage & bytecode requests if !hahsed_addresses.is_empty() { @@ -773,5 +773,5 @@ enum SyncError { #[error("Max Retries Reached")] // This is more of an internal signal rather than an error, it should never be returned to the user // It marks that we have reached the end of the available state (last 128 blocks) and we should begin state healing - MaxRetries + MaxRetries, } From 612deb42b2a1b70487aede6ccb67ba09c46c1079 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 14:32:22 -0300 Subject: [PATCH 322/345] Simplify storage healer + fetcher --- crates/networking/p2p/sync.rs | 132 ++++++++++++++-------------------- 1 file changed, 52 insertions(+), 80 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index c5214e262..6d55a55fd 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -395,34 +395,27 @@ async fn bytecode_fetcher( store: Store, ) -> Result<(), SyncError> { const BATCH_SIZE: usize = 200; - // Pending list of bytecodes to fetch let mut pending_bytecodes: Vec = vec![]; - loop { + let mut incoming = true; + while incoming { + // Fetch incoming requests match receiver.recv().await { Some(code_hashes) if !code_hashes.is_empty() => { - // Add hashes to the queue pending_bytecodes.extend(code_hashes); - // If we have enought pending bytecodes to fill a batch, spawn a fetch process - while pending_bytecodes.len() >= BATCH_SIZE { - let next_batch = pending_bytecodes.drain(..BATCH_SIZE).collect::>(); - let remaining = - fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; - // Add unfeched bytecodes back to the queue - pending_bytecodes.extend(remaining); - } } // Disconnect / Empty message signaling no more bytecodes to sync - _ => break, + _ => incoming = false, + } + // If we have enough pending bytecodes to fill a batch + // or if we have no more incoming batches, spawn a fetch process + while pending_bytecodes.len() >= BATCH_SIZE || !incoming && !pending_bytecodes.is_empty() { + let next_batch = pending_bytecodes + .drain(..BATCH_SIZE.min(pending_bytecodes.len())) + .collect::>(); + let remaining = fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; + // Add unfeched bytecodes back to the queue + pending_bytecodes.extend(remaining); } - } - // We have no more incoming requests, process the remaining batches - while !pending_bytecodes.is_empty() { - let next_batch = pending_bytecodes - .drain(..BATCH_SIZE.min(pending_bytecodes.len())) - .collect::>(); - let remaining = fetch_bytecode_batch(next_batch, peers.clone(), store.clone()).await?; - // Add unfeched bytecodes back to the queue - pending_bytecodes.extend(remaining); } Ok(()) } @@ -455,38 +448,31 @@ async fn storage_fetcher( state_root: H256, ) -> Result<(), StoreError> { const BATCH_SIZE: usize = 100; - // Pending list of bytecodes to fetch + // Pending list of storages to fetch let mut pending_storage: Vec<(H256, H256)> = vec![]; // TODO: Also add a queue for storages that were incompletely fecthed, // but for the first iteration we will asume not fully fetched -> fetch again - loop { + let mut incoming = true; + while incoming { + // Fetch incoming requests match receiver.recv().await { - Some(account_and_root) if !account_and_root.is_empty() => { - // Add hashes to the queue - pending_storage.extend(account_and_root); - // If we have enought pending storages to fill a batch, spawn a fetch process - while pending_storage.len() >= BATCH_SIZE { - let next_batch = pending_storage.drain(..BATCH_SIZE).collect::>(); - let remaining = - fetch_storage_batch(next_batch, state_root, peers.clone(), store.clone()) - .await?; - // Add unfeched bytecodes back to the queue - pending_storage.extend(remaining); - } + Some(account_hashes_and_roots) if !account_hashes_and_roots.is_empty() => { + pending_storage.extend(account_hashes_and_roots); } // Disconnect / Empty message signaling no more bytecodes to sync - _ => break, + _ => incoming = false, + } + // If we have enough pending bytecodes to fill a batch + // or if we have no more incoming batches, spawn a fetch process + while pending_storage.len() >= BATCH_SIZE || !incoming && !pending_storage.is_empty() { + let next_batch = pending_storage + .drain(..BATCH_SIZE.min(pending_storage.len())) + .collect::>(); + let remaining = + fetch_storage_batch(next_batch, state_root, peers.clone(), store.clone()).await?; + // Add unfeched bytecodes back to the queue + pending_storage.extend(remaining); } - } - // We have no more incoming requests, process the remaining batches - while !pending_storage.is_empty() { - let next_batch = pending_storage - .drain(..BATCH_SIZE.min(pending_storage.len())) - .collect::>(); - let remaining = - fetch_storage_batch(next_batch, state_root, peers.clone(), store.clone()).await?; - // Add unfeched bytecodes back to the queue - pending_storage.extend(remaining); } Ok(()) } @@ -639,7 +625,9 @@ async fn storage_healer( const BATCH_SIZE: usize = 200; // Pending list of bytecodes to fetch let mut pending_storages: Vec<(H256, Nibbles)> = vec![]; - loop { + let mut incoming = true; + while incoming { + // Fetch incoming requests match receiver.recv().await { Some(account_paths) if !account_paths.is_empty() => { // Add the root paths of each account trie to the queue @@ -648,41 +636,25 @@ async fn storage_healer( .into_iter() .map(|acc_path| (acc_path, Nibbles::default())), ); - // If we have enought pending storages to fill a batch, spawn a fetch process - while pending_storages.len() >= BATCH_SIZE { - let mut next_batch: BTreeMap> = BTreeMap::new(); - // Group pending storages by account path - // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries - for (account, path) in pending_storages.drain(..BATCH_SIZE) { - next_batch.entry(account).or_default().push(path); - } - let return_batch = - heal_storage_batch(state_root, next_batch, peers.clone(), store.clone()) - .await?; - for (acc_path, paths) in return_batch { - for path in paths { - pending_storages.push((acc_path, path)); - } - } - } } // Disconnect / Empty message signaling no more bytecodes to sync - _ => break, + _ => incoming = false, } - } - // We have no more incoming requests, process the remaining batches - while !pending_storages.is_empty() { - let mut next_batch: BTreeMap> = BTreeMap::new(); - // Group pending storages by account path - // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries - for (account, path) in pending_storages.drain(..BATCH_SIZE.min(pending_storages.len())) { - next_batch.entry(account).or_default().push(path); - } - let return_batch = - heal_storage_batch(state_root, next_batch, peers.clone(), store.clone()).await?; - for (acc_path, paths) in return_batch { - for path in paths { - pending_storages.push((acc_path, path)); + // If we have enough pending storages to fill a batch + // or if we have no more incoming batches, spawn a fetch process + while pending_storages.len() >= BATCH_SIZE || !incoming && !pending_storages.is_empty() { + let mut next_batch: BTreeMap> = BTreeMap::new(); + // Group pending storages by account path + // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries + for (account, path) in pending_storages.drain(..BATCH_SIZE) { + next_batch.entry(account).or_default().push(path); + } + let return_batch = + heal_storage_batch(state_root, next_batch, peers.clone(), store.clone()).await?; + for (acc_path, paths) in return_batch { + for path in paths { + pending_storages.push((acc_path, path)); + } } } } @@ -707,7 +679,7 @@ async fn heal_storage_batch( // Process the nodes for each account path for (acc_path, paths) in batch.iter_mut() { let mut trie = store.open_storage_trie(*acc_path, *EMPTY_TRIE_HASH); - let mut trie_state = trie.state_mut(); + let trie_state = trie.state_mut(); // Get the corresponding nodes for node in nodes.drain(..paths.len().min(nodes.len())) { let path = paths.remove(0); From 86203dd7c56976c402e86cabbc822e3336bf44ad Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 14:37:31 -0300 Subject: [PATCH 323/345] Avoid requesting storage we already have --- crates/networking/p2p/sync.rs | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 6d55a55fd..7a8176b88 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -313,7 +313,6 @@ async fn rebuild_state_trie( // Fetch Account Ranges // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available // In that case we will delegate the work to state healing - // TODO: Make it so that exceeding max replies will also stop requests for older blocks let mut retry_count = 0; while retry_count < MAX_RETRIES { let peer = peers.clone().lock().await.get_peer_channels().await; @@ -340,10 +339,10 @@ async fn rebuild_state_trie( code_hashes.push(account.code_hash) } // Build the batch of hashes and roots to send to the storage fetcher - // Ignore accounts without storage - // TODO: We could also check if the account's storage root is already part of the trie - // Aka, if the account was not changed shouldn't fetch the state we already have - if account.storage_root != *EMPTY_TRIE_HASH { + // Ignore accounts without storage and account's which storage hasn't changed from our current stored state + if account.storage_root != *EMPTY_TRIE_HASH + && !store.contains_storage_node(*account_hash, account.storage_root)? + { account_hashes_and_storage_roots.push((*account_hash, account.storage_root)); } } @@ -576,7 +575,7 @@ async fn heal_state_trie( let path = paths.remove(0); // We cannot keep the trie state open let mut trie = store.open_state_trie(*EMPTY_TRIE_HASH); - let mut trie_state = trie.state_mut(); + let trie_state = trie.state_mut(); paths.extend(node_missing_children(&node, &path, &trie_state)?); if let Node::Leaf(node) = &node { // Fetch bytecode & storage @@ -587,10 +586,15 @@ async fn heal_state_trie( // Something went wrong return Err(SyncError::CorruptPath); } - if account.storage_root != *EMPTY_TRIE_HASH { - hahsed_addresses.push(H256::from_slice(&path)); + let account_hash = H256::from_slice(&path); + if account.storage_root != *EMPTY_TRIE_HASH + && !store.contains_storage_node(account_hash, account.storage_root)? + { + hahsed_addresses.push(account_hash); } - if account.code_hash != *EMPTY_KECCACK_HASH { + if account.code_hash != *EMPTY_KECCACK_HASH + && store.get_account_code(account.code_hash)?.is_none() + { code_hashes.push(account.code_hash); } } From 8642f535e7c1a91930addd1ebf47945383969848 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 14:37:58 -0300 Subject: [PATCH 324/345] Fix --- crates/networking/rpc/rpc.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/networking/rpc/rpc.rs b/crates/networking/rpc/rpc.rs index 2e8b3c854..cb0207c14 100644 --- a/crates/networking/rpc/rpc.rs +++ b/crates/networking/rpc/rpc.rs @@ -257,7 +257,6 @@ pub fn map_engine_requests(req: &RpcRequest, context: RpcApiContext) -> Result ExchangeCapabilitiesRequest::call(req, context), "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV2::call(req, context), "engine_forkchoiceUpdatedV3" => ForkChoiceUpdatedV3::call(req, context), - "engine_forkchoiceUpdatedV2" => ForkChoiceUpdatedV3::call(req, context), "engine_newPayloadV3" => NewPayloadV3Request::call(req, context), "engine_newPayloadV2" => NewPayloadV2Request::call(req, context), "engine_exchangeTransitionConfigurationV1" => { From 74e0e5d9c512aa8f7aa45f244524a51171e5c459 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 14:41:44 -0300 Subject: [PATCH 325/345] Remove error used as signal --- crates/networking/p2p/sync.rs | 21 ++++++--------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 7a8176b88..99d38ae97 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -273,31 +273,30 @@ async fn fetch_snap_state( // Fetch newer state first: This will be useful to detect where to switch to healing for state_root in state_roots.into_iter().rev() { // TODO: maybe spawn taks here instead of awaiting - match rebuild_state_trie( + if !rebuild_state_trie( bytecode_sender.clone(), state_root, peers.clone(), store.clone(), ) - .await + .await? { - Ok(_) => {} // If we reached the maximum number of retries then the state we are fetching is most probably old and no longer part of our peer's snapshots // We should give up on fetching this and older block's state and instead begin state healing - Err(SyncError::MaxRetries) => break, - err => return err, + break; } } Ok(()) } /// Rebuilds a Block's state trie by requesting snap state from peers +/// Returns true if all state was fetched or false if the block is too old and the state is no longer available async fn rebuild_state_trie( bytecode_sender: Sender>, state_root: H256, peers: Arc>, store: Store, -) -> Result<(), SyncError> { +) -> Result { // Spawn a storage fetcher for this blocks's storage let (storage_sender, storage_receiver) = mpsc::channel::>(500); let storage_fetcher_handler = tokio::spawn(storage_fetcher( @@ -380,11 +379,7 @@ async fn rebuild_state_trie( storage_fetcher_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - if retry_count == MAX_RETRIES { - Err(SyncError::MaxRetries) - } else { - Ok(()) - } + Ok(retry_count == MAX_RETRIES) } /// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches @@ -746,8 +741,4 @@ enum SyncError { RLP(#[from] RLPDecodeError), #[error("Corrupt path during state healing")] CorruptPath, - #[error("Max Retries Reached")] - // This is more of an internal signal rather than an error, it should never be returned to the user - // It marks that we have reached the end of the available state (last 128 blocks) and we should begin state healing - MaxRetries, } From 6bbbada0f4e121c7486bc4d6b58dd5898c7d8742 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 11 Dec 2024 14:44:12 -0300 Subject: [PATCH 326/345] Fix --- crates/networking/p2p/sync.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 99d38ae97..1b5838019 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -372,7 +372,7 @@ async fn rebuild_state_trie( } } if current_state_root == state_root { - info!("Completed state sync for state root {state_root}"); + debug!("Completed state sync for state root {state_root}"); } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; @@ -645,7 +645,7 @@ async fn storage_healer( let mut next_batch: BTreeMap> = BTreeMap::new(); // Group pending storages by account path // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries - for (account, path) in pending_storages.drain(..BATCH_SIZE) { + for (account, path) in pending_storages.drain(..BATCH_SIZE.min(pending_storages.len())) { next_batch.entry(account).or_default().push(path); } let return_batch = From 1351a9d821d05313fb3513ac3e703af8a7862d28 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 12 Dec 2024 19:16:32 -0300 Subject: [PATCH 327/345] Snap only pivot block, full sync rest --- crates/networking/p2p/sync.rs | 227 ++++++++++++++-------------------- 1 file changed, 94 insertions(+), 133 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 1b5838019..91de8f8a0 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -1,6 +1,6 @@ use ethrex_blockchain::error::ChainError; use ethrex_core::{ - types::{AccountState, Block, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, + types::{AccountState, Block, BlockBody, BlockHash, BlockHeader, EMPTY_KECCACK_HASH}, H256, }; use ethrex_rlp::{decode::RLPDecode, encode::RLPEncode, error::RLPDecodeError}; @@ -21,6 +21,8 @@ use crate::kademlia::KademliaTable; /// Maximum amount of times we will ask a peer for an account/storage range /// If the max amount of retries is exceeded we will asume that the state we are requesting is old and no longer available const MAX_RETRIES: usize = 10; +/// The minimum amount of blocks from the head that we want to full sync during a snap sync +const MIN_FULL_BLOCKS: usize = 64; #[derive(Debug)] pub enum SyncMode { @@ -34,11 +36,19 @@ pub enum SyncMode { pub struct SyncManager { sync_mode: SyncMode, peers: Arc>, + /// The last block number used as a pivot for snap-sync + /// Syncing beyond this pivot should re-enable snap-sync (as we will not have that state stored) + /// TODO: Reorgs + last_snap_pivot: u64, } impl SyncManager { pub fn new(peers: Arc>, sync_mode: SyncMode) -> Self { - Self { sync_mode, peers } + Self { + sync_mode, + peers, + last_snap_pivot: 0, + } } /// Creates a dummy SyncManager for tests where syncing is not needed @@ -48,6 +58,7 @@ impl SyncManager { Self { sync_mode: SyncMode::Full, peers: dummy_peer_table, + last_snap_pivot: 0, } } @@ -117,67 +128,67 @@ impl SyncManager { match self.sync_mode { SyncMode::Snap => { // snap-sync: launch tasks to fetch blocks and state in parallel - // - Fetch each block's state via snap p2p requests - // - Fetch each blocks and its receipts via eth p2p requests - let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); - // First set of tasks will be in charge of fetching all available state (state not older than 128 blocks) - let mut sync_set = tokio::task::JoinSet::new(); - // Second state of tasks will be active during the healing phase where we fetch the state we weren't able to in the first phase - let mut healing_set = tokio::task::JoinSet::new(); - // We need the bytecode fetcher to be active during healing too - healing_set.spawn(bytecode_fetcher( - bytecode_receiver, - self.peers.clone(), - store.clone(), - )); - sync_set.spawn(fetch_blocks_and_receipts( + // - Fetch each block's body and its receipt via eth p2p requests + // - Fetch the pivot block's state via snap p2p requests + // - Execute blocks after the pivote (like in full-sync) + let fetch_bodies_handle = tokio::spawn(fetch_block_bodies( all_block_hashes.clone(), self.peers.clone(), - store.clone(), )); - let state_roots = all_block_headers - .iter() - .map(|header| header.state_root) - .collect::>(); - sync_set.spawn(fetch_snap_state( - bytecode_sender.clone(), - state_roots.clone(), + let mut pivot_idx = if all_block_headers.len() > MIN_FULL_BLOCKS { + all_block_headers.len() - MIN_FULL_BLOCKS + } else { + all_block_headers.len() - 1 + }; + let mut pivot_root = all_block_headers[pivot_idx].state_root; + let mut pivot_number = all_block_headers[pivot_idx].number; + + let mut stale_pivot = !tokio::spawn(rebuild_state_trie( + pivot_root, self.peers.clone(), store.clone(), - )); - // Store headers - let mut latest_block_number = 0; - for (header, hash) in all_block_headers - .into_iter() - .zip(all_block_hashes.into_iter()) - { - latest_block_number = header.number; - store.set_canonical_block(header.number, hash)?; - store.add_block_header(hash, header)?; + )) + .await + .unwrap()?; + // If the pivot became stale, set a further pivot and try again + if stale_pivot && pivot_idx != all_block_headers.len() - 1 { + warn!("Stale pivot, switching to newer head"); + pivot_idx = all_block_headers.len() - 1; + pivot_root = all_block_headers[pivot_idx].state_root; + pivot_number = all_block_headers[pivot_idx].number; + stale_pivot = !tokio::spawn(rebuild_state_trie( + pivot_root, + self.peers.clone(), + store.clone(), + )) + .await + .unwrap()?; } - // If all processes failed then they are likely to have a common cause (such as unaccessible storage), so return the first error - for result in sync_set.join_all().await { - result?; + if stale_pivot { + warn!("Stale pivot, aborting sync"); + return Ok(()); } - // Start state healing - info!("Starting state healing"); - let start_time = Instant::now(); - healing_set.spawn(state_healing( - bytecode_sender, - state_roots.clone(), - store.clone(), - self.peers.clone(), - )); - for result in healing_set.join_all().await { - result?; + // Wait for all bodies to be downloaded + let all_block_bodies = fetch_bodies_handle.await.unwrap()?; + // For all blocks before the pivot: Store the bodies and fetch the receipts + // For all blocks after the pivot: Process them fully + // let store_receipts_handle = tokio::spawn(store_receipts(all_block_hashes[pivot_index..])); + for (hash, (header, body)) in all_block_hashes.into_iter().zip( + all_block_headers + .into_iter() + .zip(all_block_bodies.into_iter()), + ) { + if header.number <= pivot_number { + store.set_canonical_block(header.number, hash)?; + store.add_block(Block::new(header, body))?; + } else { + store.set_canonical_block(header.number, hash)?; + store.update_latest_block_number(header.number)?; + ethrex_blockchain::add_block(&Block::new(header, body), &store)?; + } } - info!( - "State healing finished in {} seconds", - start_time.elapsed().as_secs() - ); - - // Set latest block number here to avoid reading state that is currently being synced - store.update_latest_block_number(latest_block_number)?; + // store_receipts.await.unwrap()?; + self.last_snap_pivot = pivot_number; } SyncMode::Full => { // full-sync: Fetch all block bodies and execute them sequentially to build the state @@ -232,74 +243,44 @@ async fn download_and_run_blocks( Ok(()) } -async fn fetch_blocks_and_receipts( +async fn fetch_block_bodies( mut block_hashes: Vec, peers: Arc>, - store: Store, -) -> Result<(), SyncError> { - // Snap state fetching will take much longer than this so we don't need to paralelize fetching blocks and receipts - // Fetch Block Bodies +) -> Result, SyncError> { + let mut all_block_bodies = Vec::new(); loop { let peer = peers.lock().await.get_peer_channels().await; debug!("Requesting Block Headers "); if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { debug!(" Received {} Block Bodies", block_bodies.len()); // Track which bodies we have already fetched - let (fetched_hashes, remaining_hashes) = block_hashes.split_at(block_bodies.len()); - // Store Block Bodies - for (hash, body) in fetched_hashes.iter().zip(block_bodies.into_iter()) { - store.add_block_body(*hash, body)? - } - + let block_hashes = block_hashes.split_off(block_bodies.len()); + all_block_bodies.extend(block_bodies); // Check if we need to ask for another batch - if remaining_hashes.is_empty() { + if block_hashes.is_empty() { break; - } else { - block_hashes = remaining_hashes.to_vec(); } } } - // TODO: Fetch Receipts and store them - Ok(()) -} - -async fn fetch_snap_state( - bytecode_sender: Sender>, - state_roots: Vec, - peers: Arc>, - store: Store, -) -> Result<(), SyncError> { - debug!("Syncing state roots: {}", state_roots.len()); - // Fetch newer state first: This will be useful to detect where to switch to healing - for state_root in state_roots.into_iter().rev() { - // TODO: maybe spawn taks here instead of awaiting - if !rebuild_state_trie( - bytecode_sender.clone(), - state_root, - peers.clone(), - store.clone(), - ) - .await? - { - // If we reached the maximum number of retries then the state we are fetching is most probably old and no longer part of our peer's snapshots - // We should give up on fetching this and older block's state and instead begin state healing - break; - } - } - Ok(()) + Ok(all_block_bodies) } /// Rebuilds a Block's state trie by requesting snap state from peers /// Returns true if all state was fetched or false if the block is too old and the state is no longer available async fn rebuild_state_trie( - bytecode_sender: Sender>, state_root: H256, peers: Arc>, store: Store, ) -> Result { - // Spawn a storage fetcher for this blocks's storage + // Spawn storage & bytecode fetchers + let (bytecode_sender, bytecode_receiver) = mpsc::channel::>(500); let (storage_sender, storage_receiver) = mpsc::channel::>(500); - let storage_fetcher_handler = tokio::spawn(storage_fetcher( + let bytecode_fetcher_handle = tokio::spawn(bytecode_fetcher( + bytecode_receiver, + peers.clone(), + store.clone(), + )); + let storage_fetcher_handle = tokio::spawn(storage_fetcher( storage_receiver, peers.clone(), store.clone(), @@ -312,16 +293,13 @@ async fn rebuild_state_trie( // Fetch Account Ranges // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available // In that case we will delegate the work to state healing - let mut retry_count = 0; - while retry_count < MAX_RETRIES { + for _ in 0..MAX_RETRIES { let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); if let Some((account_hashes, accounts, should_continue)) = peer .request_account_range(state_root, start_account_hash) .await { - // Reset retry counter for the following batch (if needed) - retry_count = 0; // Update starting hash for next batch if should_continue { start_account_hash = *account_hashes.last().unwrap(); @@ -367,8 +345,6 @@ async fn rebuild_state_trie( // All accounts fetched! break; } - } else { - retry_count += 1; } } if current_state_root == state_root { @@ -376,10 +352,17 @@ async fn rebuild_state_trie( } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; - storage_fetcher_handler + storage_fetcher_handle .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - Ok(retry_count == MAX_RETRIES) + // If we exceeded MAX_RETRIES while fetchig state, leave the rest to state healing + heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await?; + // Send empty batch to signal that no more batches are incoming + bytecode_sender.send(vec![]).await?; + bytecode_fetcher_handle + .await + .map_err(|_| StoreError::Custom(String::from("Failed to join bytecode_fetcher task")))??; + Ok(true) } /// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches @@ -514,36 +497,13 @@ async fn fetch_storage_batch( Ok(vec![]) } -async fn state_healing( - bytecode_sender: Sender>, - state_roots: Vec, - store: Store, - peers: Arc>, -) -> Result<(), SyncError> { - for state_root in state_roots { - // If we don't have the root node stored then we must fetch it - if !store.contains_state_node(state_root)? { - heal_state_trie( - bytecode_sender.clone(), - state_root, - store.clone(), - peers.clone(), - ) - .await?; - } - } - // We finished both sync & healing, lets make the bytecode fetcher process aware - // Send empty batches to signal that no more batches are incoming - bytecode_sender.send(vec![]).await?; - Ok(()) -} - async fn heal_state_trie( bytecode_sender: Sender>, state_root: H256, store: Store, peers: Arc>, ) -> Result<(), SyncError> { + // TODO: set max retries here // Spawn a storage healer for this blocks's storage let (storage_sender, storage_receiver) = mpsc::channel::>(500); let storage_healer_handler = tokio::spawn(storage_healer( @@ -645,7 +605,8 @@ async fn storage_healer( let mut next_batch: BTreeMap> = BTreeMap::new(); // Group pending storages by account path // We do this here instead of keeping them sorted so we don't prioritize further nodes from the first tries - for (account, path) in pending_storages.drain(..BATCH_SIZE.min(pending_storages.len())) { + for (account, path) in pending_storages.drain(..BATCH_SIZE.min(pending_storages.len())) + { next_batch.entry(account).or_default().push(path); } let return_batch = From b6b28db1d78b967468b437c1b6903d123922ff74 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 11:00:27 -0300 Subject: [PATCH 328/345] Fix --- crates/networking/p2p/sync.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 91de8f8a0..f6bdf9615 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -254,7 +254,7 @@ async fn fetch_block_bodies( if let Some(block_bodies) = peer.request_block_bodies(block_hashes.clone()).await { debug!(" Received {} Block Bodies", block_bodies.len()); // Track which bodies we have already fetched - let block_hashes = block_hashes.split_off(block_bodies.len()); + block_hashes.drain(0..block_bodies.len()); all_block_bodies.extend(block_bodies); // Check if we need to ask for another batch if block_hashes.is_empty() { @@ -347,16 +347,17 @@ async fn rebuild_state_trie( } } } - if current_state_root == state_root { - debug!("Completed state sync for state root {state_root}"); - } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; storage_fetcher_handle .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - // If we exceeded MAX_RETRIES while fetchig state, leave the rest to state healing - heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await?; + if current_state_root == state_root { + debug!("Completed state sync for state root {state_root}"); + } else { + // If we exceeded MAX_RETRIES while fetchig state, leave the rest to state healing + heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await?; + } // Send empty batch to signal that no more batches are incoming bytecode_sender.send(vec![]).await?; bytecode_fetcher_handle From ba1740989ce0fb299eeea4ff9bdc6870e2597b74 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 12:42:51 -0300 Subject: [PATCH 329/345] Add MAX_RETRIES Limit to healing --- crates/networking/p2p/sync.rs | 31 +++++++++++++++++++------------ 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index f6bdf9615..871b4f5a9 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -352,18 +352,19 @@ async fn rebuild_state_trie( storage_fetcher_handle .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; - if current_state_root == state_root { + let sync_complete = if current_state_root == state_root { debug!("Completed state sync for state root {state_root}"); + true } else { - // If we exceeded MAX_RETRIES while fetchig state, leave the rest to state healing - heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await?; - } + // If failed to fetch the full state leave the rest to state healing + heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await? + }; // Send empty batch to signal that no more batches are incoming bytecode_sender.send(vec![]).await?; bytecode_fetcher_handle .await .map_err(|_| StoreError::Custom(String::from("Failed to join bytecode_fetcher task")))??; - Ok(true) + Ok(sync_complete) } /// Waits for incoming code hashes from the receiver channel endpoint, queues them, and fetches and stores their bytecodes in batches @@ -494,7 +495,7 @@ async fn fetch_storage_batch( } } // This is a corner case where we fetched an account range for a block but the chain has moved on and the block - // was dropped by the peer's snapshot. We will keep the fetcher alive to avoid errors and stop fetching as from the next block + // was dropped by the peer's snapshot. We will keep the fetcher alive to avoid errors and stop fetching as from the next account Ok(vec![]) } @@ -503,8 +504,7 @@ async fn heal_state_trie( state_root: H256, store: Store, peers: Arc>, -) -> Result<(), SyncError> { - // TODO: set max retries here +) -> Result { // Spawn a storage healer for this blocks's storage let (storage_sender, storage_receiver) = mpsc::channel::>(500); let storage_healer_handler = tokio::spawn(storage_healer( @@ -515,12 +515,16 @@ async fn heal_state_trie( )); // Begin by requesting the root node let mut paths = vec![Nibbles::default()]; - while !paths.is_empty() { + // Count the number of request retries so we don't get stuck requesting old state + let mut retry_count = 0; + while !paths.is_empty() && retry_count < MAX_RETRIES { let peer = peers.lock().await.get_peer_channels().await; if let Some(nodes) = peer .request_state_trienodes(state_root, paths.clone()) .await { + // Reset retry counter for next request + retry_count = 0; let mut hahsed_addresses = vec![]; let mut code_hashes = vec![]; // For each fetched node: @@ -564,6 +568,8 @@ async fn heal_state_trie( if !code_hashes.is_empty() { bytecode_sender.send(code_hashes).await?; } + } else { + retry_count += 1; } } // Send empty batch to signal that no more batches are incoming @@ -571,7 +577,7 @@ async fn heal_state_trie( storage_healer_handler .await .map_err(|_| StoreError::Custom(String::from("Failed to join storage_handler task")))??; - Ok(()) + Ok(retry_count < MAX_RETRIES) } /// Waits for incoming hashed addresses from the receiver channel endpoint and queues the associated root nodes for state retrieval @@ -629,8 +635,8 @@ async fn heal_storage_batch( mut batch: BTreeMap>, peers: Arc>, store: Store, -) -> Result>, StoreError> { - loop { +) -> Result>, SyncError> { + for _ in 0..MAX_RETRIES { let peer = peers.lock().await.get_peer_channels().await; if let Some(mut nodes) = peer .request_storage_trienodes(state_root, batch.clone()) @@ -660,6 +666,7 @@ async fn heal_storage_batch( return Ok(batch); } } + Err(SyncError::MaxRetries) } /// Returns the partial paths to the node's children if they are not already part of the trie state From b5383aa3cd04ae6335288d6f87c67dfae3e745bb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 12:44:02 -0300 Subject: [PATCH 330/345] Fix --- crates/networking/p2p/sync.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 871b4f5a9..146dd5db7 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -666,7 +666,9 @@ async fn heal_storage_batch( return Ok(batch); } } - Err(SyncError::MaxRetries) + // This is a corner case where we fetched an account range for a block but the chain has moved on and the block + // was dropped by the peer's snapshot. We will keep the fetcher alive to avoid errors and stop fetching as from the next account + Ok(BTreeMap::new()) } /// Returns the partial paths to the node's children if they are not already part of the trie state From fd0518e3c0bf4022d95a616ba248caab79457d33 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 17:23:46 -0300 Subject: [PATCH 331/345] Doc new methods + add clippy lint --- crates/storage/trie/nibbles.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/storage/trie/nibbles.rs b/crates/storage/trie/nibbles.rs index 300776713..0eb2c58ab 100644 --- a/crates/storage/trie/nibbles.rs +++ b/crates/storage/trie/nibbles.rs @@ -76,6 +76,7 @@ impl Nibbles { } /// Removes and returns the first nibble + #[allow(clippy::should_implement_trait)] pub fn next(&mut self) -> Option { (!self.is_empty()).then(|| self.data.remove(0)) } @@ -178,12 +179,14 @@ impl Nibbles { .collect::>() } + /// Concatenates self and another Nibbles returning a new Nibbles pub fn concat(&self, other: Nibbles) -> Nibbles { Nibbles { data: [self.data.clone(), other.data].concat(), } } + /// Returns a copy of self with the nibble added at the and pub fn append_new(&self, nibble: u8) -> Nibbles { Nibbles { data: [self.data.clone(), vec![nibble]].concat(), From 78e56412c246f1f982fd3e78358fdd7753b3b93b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 17:28:57 -0300 Subject: [PATCH 332/345] Remove unecessary explicit mapping --- crates/networking/p2p/sync.rs | 43 ++++++++++++----------------------- 1 file changed, 14 insertions(+), 29 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 146dd5db7..2e672eaaf 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -143,33 +143,23 @@ impl SyncManager { let mut pivot_root = all_block_headers[pivot_idx].state_root; let mut pivot_number = all_block_headers[pivot_idx].number; - let mut stale_pivot = !tokio::spawn(rebuild_state_trie( - pivot_root, - self.peers.clone(), - store.clone(), - )) - .await - .unwrap()?; + let mut stale_pivot = + !rebuild_state_trie(pivot_root, self.peers.clone(), store.clone()).await?; // If the pivot became stale, set a further pivot and try again if stale_pivot && pivot_idx != all_block_headers.len() - 1 { warn!("Stale pivot, switching to newer head"); pivot_idx = all_block_headers.len() - 1; pivot_root = all_block_headers[pivot_idx].state_root; pivot_number = all_block_headers[pivot_idx].number; - stale_pivot = !tokio::spawn(rebuild_state_trie( - pivot_root, - self.peers.clone(), - store.clone(), - )) - .await - .unwrap()?; + stale_pivot = + !rebuild_state_trie(pivot_root, self.peers.clone(), store.clone()).await?; } if stale_pivot { warn!("Stale pivot, aborting sync"); return Ok(()); } // Wait for all bodies to be downloaded - let all_block_bodies = fetch_bodies_handle.await.unwrap()?; + let all_block_bodies = fetch_bodies_handle.await??; // For all blocks before the pivot: Store the bodies and fetch the receipts // For all blocks after the pivot: Process them fully // let store_receipts_handle = tokio::spawn(store_receipts(all_block_hashes[pivot_index..])); @@ -187,7 +177,7 @@ impl SyncManager { ethrex_blockchain::add_block(&Block::new(header, body), &store)?; } } - // store_receipts.await.unwrap()?; + // store_receipts.await??; self.last_snap_pivot = pivot_number; } SyncMode::Full => { @@ -265,7 +255,7 @@ async fn fetch_block_bodies( Ok(all_block_bodies) } -/// Rebuilds a Block's state trie by requesting snap state from peers +/// Rebuilds a Block's state trie by requesting snap state from peers, also performs state healing /// Returns true if all state was fetched or false if the block is too old and the state is no longer available async fn rebuild_state_trie( state_root: H256, @@ -336,10 +326,9 @@ async fn rebuild_state_trie( // Update trie let mut trie = store.open_state_trie(current_state_root); for (account_hash, account) in account_hashes.iter().zip(accounts.iter()) { - trie.insert(account_hash.0.to_vec(), account.encode_to_vec()) - .map_err(StoreError::Trie)?; + trie.insert(account_hash.0.to_vec(), account.encode_to_vec())?; } - current_state_root = trie.hash().map_err(StoreError::Trie)?; + current_state_root = trie.hash()?; if !should_continue { // All accounts fetched! @@ -349,9 +338,7 @@ async fn rebuild_state_trie( } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; - storage_fetcher_handle - .await - .map_err(|_| StoreError::Custom(String::from("Failed to join storage_fetcher task")))??; + storage_fetcher_handle.await??; let sync_complete = if current_state_root == state_root { debug!("Completed state sync for state root {state_root}"); true @@ -361,9 +348,7 @@ async fn rebuild_state_trie( }; // Send empty batch to signal that no more batches are incoming bytecode_sender.send(vec![]).await?; - bytecode_fetcher_handle - .await - .map_err(|_| StoreError::Custom(String::from("Failed to join bytecode_fetcher task")))??; + bytecode_fetcher_handle.await??; Ok(sync_complete) } @@ -574,9 +559,7 @@ async fn heal_state_trie( } // Send empty batch to signal that no more batches are incoming storage_sender.send(vec![]).await?; - storage_healer_handler - .await - .map_err(|_| StoreError::Custom(String::from("Failed to join storage_handler task")))??; + storage_healer_handler.await??; Ok(retry_count < MAX_RETRIES) } @@ -712,4 +695,6 @@ enum SyncError { RLP(#[from] RLPDecodeError), #[error("Corrupt path during state healing")] CorruptPath, + #[error(transparent)] + JoinHandle(#[from] tokio::task::JoinError), } From dda5e72fcfb3fc47124a6e8ddd1e2939c95f3998 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Fri, 13 Dec 2024 18:19:27 -0300 Subject: [PATCH 333/345] Add doc --- crates/networking/p2p/sync.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 2e672eaaf..b05e2dec2 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -233,6 +233,7 @@ async fn download_and_run_blocks( Ok(()) } +/// Fetches all block bodies for the given block hashes via p2p and returns them async fn fetch_block_bodies( mut block_hashes: Vec, peers: Arc>, @@ -484,6 +485,7 @@ async fn fetch_storage_batch( Ok(vec![]) } +/// Heals the trie given its state_root by fetching any missing nodes in it via p2p async fn heal_state_trie( bytecode_sender: Sender>, state_root: H256, From c272e1b2e6efd44efc8acdc13729e92aab7eb917 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 17 Dec 2024 14:39:47 -0300 Subject: [PATCH 334/345] Update doc --- crates/networking/docs/Sync.md | 26 ++++++++++-------- .../docs/diagrams/rebuild_state_trie.jpg | Bin 0 -> 55483 bytes crates/networking/docs/diagrams/snap_sync.jpg | Bin 34806 -> 42542 bytes 3 files changed, 15 insertions(+), 11 deletions(-) create mode 100644 crates/networking/docs/diagrams/rebuild_state_trie.jpg diff --git a/crates/networking/docs/Sync.md b/crates/networking/docs/Sync.md index 8beedf312..c18101221 100644 --- a/crates/networking/docs/Sync.md +++ b/crates/networking/docs/Sync.md @@ -2,23 +2,27 @@ ## Snap Sync -A snap sync cycle begins by fetching all the block headers (via p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). -The next two steps are performed in parallel: -On one side, blocks and receipts for all fetched headers are fetched via p2p and stored. +A snap sync cycle begins by fetching all the block headers (via eth p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). +We will then fetch the block bodies from each header and at the same time we will select a pivot block (sync head - 64) and start rebuilding its state via snap p2p requests, if the pivot were to become stale during this rebuild we will select a newer pivot (sync head) and restart it. +After we fully rebuilt the pivot state and fetched all the block bodies we will fetch and store the receipts for the range between the current head and the pivot (including it), and at the same time store all blocks in the same range and execute all blocks after the pivot (like in full sync). +This diagram illustrates the process described above: + +![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg). -On the other side, the state is reconstructed via p2p snap requests. Our current implementation of this works as follows: -We will spawn two processes, the `bytecode_fetcher` which will remain active and process bytecode requests in batches by requesting bytecode batches from peers and storing them, and the `fetch_snap_state` process, which will iterate over the fetched headers and rebuild the block's state via `rebuild_state_trie`. +### Snap State Rebuild -`rebuild_state_trie` will spawn a `storage_fetcher` process (which works similarly to the `bytecode_fetcher` and is kept alive for the duration of the rebuild process), it will open a new state trie and will fetch the block's accounts in batches and for each account it will: send the account's code hash to the `bytecode_fetcher` (if not empty), send the account's address and storage root to the `storage_fetcher` (if not empty), and add the account to the state trie. Once all accounts are processed, the final state root will be checked and committed. +During snap sync we need to fully rebuild the pivot block's state. We can divide this process into the initial sync and the healing phase. +For the first phase we will spawn two processes, the `bytecode_fetcher` and the `storage_fetcher` which will both remain active and listening for requests from the main rebuild process which they will then queue and process in fixed size batches (more on this later). It will then request the full extent of accounts from the pivot block's state trie via p2p snap requests. For each obtained range we will send the account's code hash and storage root to the `bytecode_fetcher` and `storage_fetcher` respectively for fetching. Once we fetch all accounts (or the account state is no longer available), we will signal the `storage_fetcher` to finish all pending requests and move on to the next phase, while keeping the `bytecode_fetcher` active. +In the healing phase we will spawn another queue-like process called `storage_healer`, and we will begin requesting state trie nodes. We will begin by requesting the pivot block's state's root node proceed by requesting the current node's children (if they are not already part of the state) until we have the full trie stored (aka all child nodes are known). For each fetched leaf node we will send its code hash to the `bytecode_fetcher` and account hash to the `storage_healer`. +The `storage_healer` will contain a list of pending account hashes and paths. And will add new entries by either adding the root node of an account's storage trie when receiving an account hash from the main process or by adding the unknown children of nodes returned by peers. -(Not implemented yet) When `fetch_snap_state` runs out of available state (aka, the state we need to fetch is older than 128 blocks and peers don't provide it), it will begin the `state_healing` process. This diagram illustrates the process described above: -![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg) +![rebuild_state](/crates/networking/docs/diagrams/rebuild_state_trie.jpg). -The `bytecode_fetcher` has its own channel where it receives code hashes from active `rebuild_state_trie` processes. Once a code hash is received, it is added to a pending queue. When the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them. If a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so it can request the last (incomplete) bytecode batch and end gracefully. +To exemplify how queue-like processes work we will explain how the `bytecode_fetcher` works: + +The `bytecode_fetcher` has its own channel where it receives code hashes from an active `rebuild_state_trie` process. Once a code hash is received, it is added to a pending queue. When the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them. If a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so it can request the last (incomplete) bytecode batch and end gracefully. This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/bytecode_fetcher.jpg) - -The `storage_fetcher` works almost alike, but one will be spawned for each `rebuild_state_trie` process as we can't fetch storages from different blocks in the same request. diff --git a/crates/networking/docs/diagrams/rebuild_state_trie.jpg b/crates/networking/docs/diagrams/rebuild_state_trie.jpg new file mode 100644 index 0000000000000000000000000000000000000000..e9255ec89ba0198df6c875ed8540e3ed65331e40 GIT binary patch literal 55483 zcmd?R2|Sc<+c$n8*$uLUn95e!N|r)~%9a?EeVGbbCRD=6WJFm*2%(TQSth$|lRYHa zYb;|%$UbA87~{RV@B7*A-~BxA{oK#BaPXR>03II%53;+}O$ng7E z7UMs?%Je9U>7Sl6s{TH4|0BT5%Bafph>77iz{tzM#LKYX20+1gGBf%akCX1-%*bPgUeyUB9=9>3D#_&2N) z7t5Lj%=^ibXK&pPVPh8*IxH+AB`tH}l>R}+tgn>zwH9A%Ad^w|Nc$8ctN`u z4;)}R!1CKJ21ftihVvd^K6d6HpN<*J&3lKAD?MiAzZn0fteH*XtT|cW*8P5VK}ltz z6y>*Rf3ocFGc4pEvh1G?`xm>$fJ*@5p9&KrBNHYlS z{;6>PE|7mN`(PssU>gSx9AE)|c{tcOc>dF6e*zp_=l1acCldoWn3#A07(mm{jyeJS z->led8ic~pN#MfVCPav=wr27z1PKfkP`sab=GF~hrORGVAt5{(mj|?ewsRaWGrxN! z^x0#Cx>czREv+whA6W7shtty=bhVEVXex2b#`}P7Vh4t5x&WgE#Zi$^d^$Z^=Gi_# zelvuj?xDe#fkzDc0GHG5HB7%VC6mst9=8ubbLb+STlxDycGonFcET4*haq77+E8sw zt2KRyuwkR=x(__%&e;b#f9?ZeWqXCVpXI2&eSlp|e;eSoq9BNk&kym#a>&h7&(;b!~5^yc~=Kf?yg|8QT&EcXFHt&=pp_-rjr*sb(!-LEFd#BIL(vJa%>Sd~$Cr>a9KIV4;Q zf)$n7b_g}v(!@{oyhma~8*JbE-ba!}y1P8T=%w&-FNx2}I<>&FKc6%M{|vxHDxr|Y zrc)#Gniq&M>T_1Z-h`f-%F3s6wW)m?PO+zL;@%$^F+9M>+AOfEMy?=2S_s@|PpS+> zsKrtQ5!#@m82u&1pSY~iF4!ckUx~Dl4*4Y&X5$h!YZYbT6Ek+@=|m_=oSll?RiQol zy7?OZagq&3Mp6&8WgM=2T`(-u=;>-SvMDtwct?RFYwg(r$7QH!r))gJd>k!AK5o1V|K-OQ zfZ)9CwsC3BFd9S6kV*(Yzc+hB4yP6Ht8N|P>H$3{*PxH3J)lbEd#Jf1OHqkLhdBJI zmviHX7)8$+_ai+TXFq5!msyh;-+ly4S*G*TiF<)Il|$YeF{-mWqXtjxCwmSk9=HAI z;KX;en>B``u5|X}xb_!=LbMt&lb70QTTPyOQdv}jv~2WzZ)t{D}-SPqNtGcp@ zr{|}F(VNq_CxkUD3j_T;T(Hb>$9 zoCGC{q{p1Oz6Zp0u*1=wN{eu+53y0d1;ewfb&e)A(BPRnfv}rw56oXwx7hnay}^9j z-)-u7Vtvo8(kY(P*3shf)W9jlNJQn_G80Jy8&Mc6d!E6dCQQ~2rlH!-=q3frgH@EhG)@hNWhb+bByHZ)0G zh@2VK;9a6a#4-OIt)pY_A8B_j80+%&IT~a1!8+U051&=`af%TxFrPxGfG>;Y+t_>9 zrnF!M@V2&kYrV>KVzhhhM{ZQq*I#6O7i=gw#cdog(}u4JrrAA$hO2}}tzN0Et&XC% z+>U#gqwGRTP-Q`u-qo3b=RKlOiaMH9hk}IZpYfi3kt%SBHg%~s+hDu-p$WOP;xe_g zOEX=UYUDG{&1*w5@K)hfdk^I=dP1+l$Ok1Orya`Cg4-1>+zsx#*u2sFzuO;N8 ztEAg&J=3#0RoOp-RMqLV$Z%&2g%j+{K^5WR(bG;q9Q zhDJ#G(Q|4VH-vG*Jf0M8tfuOa(v8(j>ggS^fqfzl)b5s%1%hS9X9vBU%Ly(p=XK#* zt@^V{s^o92r&*?B_|3GL|2ykL5~iz|x=Qbav0~=y+blM%THsU}Y|K931_@I|@M&?N zMlX&;8mH{|j=^1hT~Jv<)#qX@ZYi;UI8mH?deI)zf%`s9>RejgJ&&rqDu4HmaYc9iGsbVXxG$}W#yS(qo{d)6 zQ)OF%l)|m4DLI$2$31pfzBz|+IFGEzdpa3-H*_BAIZ#u*sQ&9jQSFkuYP>d-#|)7( zf+!otYKxA zg{D;>v=Y8FQ9@%xa4z_pFPwFEl#N<0fmJI9#&DP!$~jxl8Wxudf4Aa}6Aizy(3*45 zlwn<0V8@@*I@K54An&HBIHkyl;$6XRlrYFUr?jhwqFxa^wk)3waH&+m6)|`Pj#KHL z8IFew9g$Xs`#^}_7}bs>6jrPy>9JE!y^r8g4EJ@1>eLUkrIg#jm5uXzA9U9w$BwLo z&G-rY677~fp1|-6q5CLkAE>Bl-Gdl^E!`wUm`T@TZ7-F~<5fB`$nvxCD#Q?;ZKMV( zR_3X#OAY&!f>7_IHug3Jw{(fPDu{L%(NrXhw#P5<=uW5a1Dp=?t2ZCy=(CBw+b&Gs!}|Rf;rbE4cLAJ25FaG z9ZL3_dlKW&CwUTx-_fOT(AoOPV_Uv^8E73+j2CIN6UL)*La?Rdk=3=TugZaJE95W~ zeqrIeh{5#={n_r_afRb%uYe#CGsjiX@_7mqJ!Fyx^dkwaP+EBoU3iehvd7$T40KF! zVX$8=+Ty&rfjHXPf;Pww2kH{&)nPa>Yu#~FS9k<;LlAV61?h+Pfte%#r7N%kdcq8s8CVB~h z#(JtRJLNDM%xfb`3|&$o@oS*dy}3(gJG&KCe~x#p|cNv3hKiadNK&?Na>HGaF^pY72QBH1Q6Pe3-i_OK z9USeO@1`7|`n0Ve@Hxtm!B1?Z-RN~Ii`CxyAUThRozdt^o`SFQ7J?LUux_S|c- z_;IR(s&!TMuU85|8*f@yBxdmEut{&h57Knlb*84bHHlDa!j=0NsfgBa(q#|LOa2A; z3HjdT7Lyvry94%hZDYf+ZKh8?&Ag}{XgW_uP9q|l*cU}!Z$-P@EOAfUOq|D;CcSI% zl2cq33q3re>wLrFg_62bd3|LL`JIn$--K%5S(8 zxGQ?gE8Pf8UCB?=H;9yyIB-Jp)uF+n#=Qzn-Qq<@l}3tCaKo8*^Nw1zRYNMq(r(_# zT~!*!Gv3!+@ShXv4LVFEzfH&Fl%PGq*zEPAT~E}s6v#0kr zsnthD#Su)W+@%<{|#Z4z`~>uRgB`@ zA#!(Q1I{staeC!yUT5Y~pnz!%d~mA<(V?R)eoZ%!0Pd3vsK8ljI7RJBb$rlX#$*1v z&2O7+@xRmq18;3_%*Ii#&;hDb92XU8O7ZM|P)iZd8rdoigub88xEj5oR;uwxTl4sd zyo}3X{7Qz$*x%(yMNR=M=16?y>i1F7X74N8(uy~69PWu+T>&J~2xTLG_^RIMD%*>P ztf2`gAmh|$T%~_I$GMErW-HYU=IR?(u3<}W2J-L^izYs1uW0K-8&j#I1}oE{Dlx}3 z2YrjWQ3L1bwaI&xSG!ZcJl1A_G;Ph)wT6@NrkDZ&s3<(pA3oZHq6^zO~_2 z+bZI&_JgaV1Q8lWM=+t!hV8iQ)?~=w(yzUMuF5xNl~oJVm&!f^(=sI}DSn2X#NBh4 zX6@3ZgXlAql^hILY2LJ$BT~bYL%(t`Yw$t6L5|_G6m$6(rXAPMzIA~%K7SYUQSjcH zVjTS#wU31Th>6V5pgw-rTfCT75JNjAxqV0+PBUt_EMTE0z^l(1#rPHgn9)SX+RQbi zmc~9%*4F~v@uCO-=o*A0YHX{O0(*vNPm3vd+yGj|gCgOLCGM znlqQBb*kZ~B&7_6g|W|nW97olvX4YfhSsz#$#iZu32rd!eu1ZfOc=J$J}?VnFz z?%3OOd2iVma^3e7B-x?crCRJFZPj3S*K1to9hOz;#JCMJBFXSP2xlxQhXxfS9QSF z*-Ry^Twi#=^G0QjFQ0OFP|t(glFt#6!0DW^P3Ra)TOxJPfEt=jg|>=Fe{d~TjKG5K zpO;OQgFbhumk2UdLR(eq_{F)%8)|sZ3ioRD z!mu^uuCe5#X#4w^{5sUAQr_2jyHJ|^k4pqu@H2B z;Qs{D`BPJk2;-;rm^naOKpf!%==u6hc1#+j#y;Ycx(!PLd^W5a_kmVTYK6fs`te0| znc)7QHKa(0XxrvQ!T0F&8JcHjL5a+k_tG>m$d}6rL7C|GjW!7pWmlzvPrko?myatV*!XW-3h9b%=kvVjZlJ20D&JfA=~_ z7nwlZ2f`c9^=x~Umc)K|h`}qqQ)y$T$8hpGy=|VYMF^uUsVNjTr|6E!6GOh~D9o^q zkB`Jy3yTi_`L$`L_qxDCQx<;^@KhbU_=_%yT5oMSEW2Hj-Jsq(zYjR;=E;8sQGlgh zIqRPWnr2&s@3X3-NP-pc5mDAyQOu1bb;-qE)X{s zZQlo|`tJJxtsew(*>|^`LAW>uj4@~uAW}+qd&@L?MRp!XwIWo=?D2aGfu@NS>;vn% z`ZU?q?}_v%wP?`kjvd?wer0DtQcqw(=#+>E`kc98)_~Xtp7>?%1HA%JG8T6mzH?$y z5%lym;QPRARIrwFc2hZd8=Q0BSKDp3(ur{p>UxJ&T|U-_E3cTBss>k-Td$2(Mo{0@ zRq!zTLP6Kq6&eC#(K?DmaB2EX6NGFRZTTm!E@VL7yM0p%H;poWd0gu-QDB9B zY#&H;CPVgO=nDJ5ASV1RQ_DY%pMUgmfAGI!O#vjQOc@;Dp){^(XoPxY9W3+#bycde zI$>=E8U0qS#8dv3vt9;ic4j*4aI3*PM}4zorb5R0637;Y+FUx{zv$HJsw=&DIE#PJ_tLh<>L1a%>hVXq=b^( z*0yS|+*v&vVSkDD-SI=b_AE3>=mv<^K=*+~tHqJ02QzOH<3#OgJU9?O451(EoSM*_ zwEd0>nOwZw?<+*%gDAp>Ruyj7R3``Cx6U3f@03h8YpwFdLD2Weorcu7VpRTUr|tE1 zcq!@@j6eMX(%|b-Y1*h!wMZ2$!M<6pF6qf={*z3n_s1N#fFLT7M}X8l2CY2{KE zzEUJli12b5^?DJXxrwTJ?@bj*$7qtMiPPY2C*a=qX2DUWuBM+*ntrpwWDv(8s}eST zuakEJD_Q{(Pv>BZY5TybivJWk2+hKg;S+XKm{vtG5AZJ0&rw50TE?1BzjIgj-dSAi z)uTw7NveJlp12ufEp&?4@$#+3>c%lTduS0FLM zak>tH)3qs;IzbU7*0dsG#I>kCZ_xbY7dzfh(K_$uEz9jEQKEx2-%g+jLHEyqe_O^N~=6f%PtwQ2QD}VF;iAiuwQ}t|4*K1 zMZZ9Gy-c?1ghxzrk$Y=Es)D8MVdK*mqLc7b_>nd1Y~wXSKYY|do9H9X2?1FU>A%T1 z*AduIES*|fjq3i78!o&>h0F(S_z9Mem&aN1%6f{=J$3{UL3W6Fe-I{8T#mdQG92LM zg>Byh8CvJ>^GJV9@1GTtK|i?Nn3~B;n;SKYV+{Nt#XuDluoMc10GP>Zw&3gy(S3t0>!9vr0*?C zzXczbQ_xN?-@M^m9`WFF4ZQkrXML@}cW}Q;CC@%yN9$ALvVMNSJ?y)hw7EOD!2gY< z{L_l~@_E|5*~!x&%&;1vW)v`uu*lLN(wE6a?~L^7 zxg+tT^OkN|9JZdDEAhDtJA$eAq?$6Z`uu8Ni^NiRn@{zL;L1NmwT@dtxAZ`FMH5Gv z-3LSlvV+fJ%9TOKW(Hi7)TsS?@u1_=r$`IHbyU91I+!izR+iQB7J%F|`B#40iR{L~ zW+8DNa!oVlFime1Eu4?)G=EQF34c&zy=k*~A}fSTZ_p~k&tlM9F7lQCO*WP0-pc?-=11mRw%0vC>!lT_2S6*1$TxT>mhh@lO zY$R;uf=*I8wSFIP3IaX10=C5#*$Hqez<-^`MdP;A7WaXjyBWcn$D>YUG5)Rk$IuN0 zBGeI>AKx{Zz(z6AxD5hv;Xl|7{(5)#cm9X%57RJv#h4%SO-h(=b0p~AxjqDJ^eeO~ zj`muu1HOBN21Wt}!i(utU2@niX@(HV2jy`OI$TE1PuLmW+bZ><_ldo;AH(0&O{yCD zJZtold;ry9C~+h>Kn!#imG?4ePQ)Z~(rB3Xod(R!X>3(~oDZ?z-)F0^*k1o0YfZ-F z*LgSu0=!+rN$+_|4(BzKb9M*PZ3iu2|^)SQwElj2B8P+AZZ zlB*BX{)E;3Kouo@LUZ{{j|D~}>ov^Ygw@0-B+nBNcKOu?0|q<>R!gx*a`<}`lLuU? zgHgQ#i{k23SqcZS;K>G55Vd$U9lRn>!q&~oLprwDma#dHR5c7G18$}%kBOH22(mda z$MyX5*`Fz}^3{mq2Ld04H0_9saK5fL1rdX-iT3uLg-xka*fI0Vel>!gvUYT`wgP(K z-1qvKEXd2BSZ@k+i9qRV)shWu7Rq29Ua9JdQ5dwgGcpp5j(nOlF01s({9F%^UTxUf zb-Y|v*x>Lw4MHvSo-&4z&Z4|FH{iS;HR|%wc@59j2>2%xJlY?6&b}~Np>*d+s{%~j zCrS%!$gYu^vLszHHfCctlf&cKv`zM~{9JXygtG*n(IGt-c7}--K7DPtzy?bs&ZXe1 zrdSTD6Aoe~-n`f-uC|9#u_@y9@>@k0DvOw!J^AadUi!Q)W^wu1L@`D zE4v5sR_<006^h)n(+0NK|i4rER^|*z%(``nU)d75~JLqvsP9 zHX0~gZ6Z?j1>4!k?eJdlkfT2`0GTL&a!PjcHzKO|ZxGSfD&HpdI_q$%XB2lSa?^v# zIC2@Cm+C!3R%<=yosXywAO-%aIy%tGT-6p| z2gAngrFCMgFG0{_QsslmdpFffCKYIm->%u`)(aH{ySwJ#rb{AchSL$@Fs@4DGove$ zNcNq*x$2pQeL&~jRypDaONg1acshvyiopcXzRt>-{HT2TsuBFdjf%r}B=&()zxX?k zVN)k$KfnD;?D+32H~$=4Znl97qz#3z7aIine1rZGMV}V`8hprtB>((3o8G?w6~6;b z5LWx6n#!t>ru92MrGk9C6DAZ~B}u2&x`Ds3`2L2+{;*8{{P$y7*@@pPp>)?26jX0H z{7Jn9v3g=vPYtIVCw_!&sr_E*M^&`=eDhRB4G>U!ywo>8>>xzz?E9w56=C z4YqSHqY0A*8Bq*}Rb{q1cwZs=&b8U~N|;d8sZQ-$LaBT$yT$_#)&qPzeQx?ls*8;M zlD(VhR+DDLX%xa4b=^lTvPm6%j)D!N-6XFYQfu;2>^lYx0`ACn^IML`9JJQB*nHLM zVq8;}@zKQxXPKjHXw@$|MV?$|?)=gCP^iT_d=w`@m!{egOEd3Q`=CN8oD#EIrp!5GxtLw=%1yj%Oe`w+2hP(~D{(@}^FZG#9|Qw!Q`M_wr?BYP~ubqAN6 zs|70%-h%))XnC_&qg4rM-s`*de$H%g zqxYtAUR`C4!NK-xa-P!jxy*JCpXcy1_;1BgGQ(6LPhgx{N9GB9ooy4|E%>;|Qbnv& z)ZVQyG2>$K?@lSEVZxG}x02FBtF|HaRq5joa3BK4o=sL(hd^p_B8oZO;q$jo(^kXM>f}8ezZx#rgE>7BYErTBmY3p zJol}gryI|^-Y$#^zIUk3^o+0$_{#CpDK+;@OG0g2uQVuBTwQV@mgvyVN;mozqMG&r zd$JyuPMG&ZB$X_VmCEhkPa5+vM{0f)4EPUf`_w`EC+~^MoF;-D?DX^kA8$>Uj-ct>L`6r zJxnz+9MKYAn3RgPs~f6}6P;+Cb+`3;7H@#h9X*Z5m)q9*n~HxskNTt|ts5Xh@5)dW zX>92u!60K)OM+gsBe-hk?)`1>x6;=IKQ**e3#84)*l$t|T)Yg)u z){vszmj1Rj*0(TzMdqK&Fk1rti&yT5+02lbk=@ypuK91JtgBMz|9odcy8ytvU#PfX__(PcxG zla~D|6USS=bg#GV?m@*);L4_9!&4v=Ue2C(0 z|CM$oU@sPpz+&3E=XH}`oqcPfw>Xv~P)F%g;+9=6$GO8iAmFMks_8^wBKZ}P2T9OK zWiBh?Wc|Bu9(4+@iyg2LlbTF%|e$Nen*jlr_)goCh87IVs0EH%?r)D9&* z+xM~hGy4*G;+4Anm_f5u{bSwn;8$NZ-n1;g|IQllV{^EbNsceFa| z2qCLI3p%?ow$w{28iidbLX;yV&)cc zRqC~3JGYE;1Lj|li_VQz$sDzc@?Is*w{8;MyhAMQ?mMFxEb}JB+w#o%uaCk^zP+(&V*V^)bx4R zk1R8=e+Ax}ffJ8q?=(1oz)){^Hx%n(83#RE=h)Gg_nvF6w`9QfUeXZ6u|{bJf5Uzf<@HDvc~Yn`;e^y6 z8aB6}o}c=zyIf#xI@V@`itYdeo(?^?42B)W5RZZ$3Y5AJ@`1K>tj_CiV(pzChIQua zAve$d1K2Cou9Z+3z6^QuQ{;AJHrkA0l3C!s^>u8_&Ti4a|I&doM-8T9m0zAZeuDj7 z7MO2xD_AomYTKLY0mpmOqgH>jH<-7W-=cnSdf|3toBm~bF}1j1{lLTkaE|Q*KTE0q zJjuYkg2XKlrpm7WO%y>FptPW1P3gb1iu^y)>i@Q{lR?};t&Wm=*-fm_$V_I|jSPX!O;k#`*ubmXpvZx``}aWlQHn$}V#Lt=tNgZiL64f4~=+^gEd zB{WweXgGQ)&_z4h4(wqwxqY^brv0Px2rf2x7%J?-e*u6DenHK4b)@ z_wf<>v|i_T6!g?YajkdBb(^6(EqG{O^wuk_dJrO7FtpWP7D!*M9zhk5N_ObW4p?nl5 zej3y%7;9|pM7ov6+|zh6c0?!Tp5}7~iH^llEp}AOM{z+)Is&a1YA*6%%b!OFlhfV5 zpl&f8?73RTP%e4WT~9VZK;xrD9{ugYyMx^0M= zH&?zd_FVDI`;dJ4HfNK4O=7X%0c3F=BEJXY){Xb1mi4uZz=pS$1^GYAuGo zNxeuMW4pRVu|LCjQaXmWOVs*@)$qg4WdwQABJMIMm{wUmL>-ZZK&VBR>nL6YH}9xK z9&)TU@VsT`6nHFovVU4mbFpQv_fb8-?$)zwiau5`y6GL2=057BX*V71&G{W^`=Hvd z$LXeEobOHF0I9rAd}_g2)r@Gb7*0p?dPW*MKxPGl3j$tcvxmT?_JmVCBV()I_`vAM z>G!qu<2R}iO1==^qZf7S4ym5@yYXy2LufB8=p3r2JFJ^!a-1;=BP>$%nYgZ{am2nE`U!rjiLfpn|>h(P*pfb}NL>eQO^O&-*Z>ybEPVUF+Dn@G8Vu zRaWB(&hzGS(3sGe@o?3TKFoXD+_P-2b6yy|zW@Xwkpw_Xk?Qmj%x*cPLf|$+5*H&% z3f7;!uPoCE1e+U#-zgW3xRn`q%FyX@qJ-0tXFD=2gkUWQC9cITGg|x%MWH6&uCr7% zQXoH)I5R5cYUlhx>60zJug6~X>*?{>hS3_@?Y$BjFNmbnM96O_R#{WyI@I-IatZG9 z#0l<^!v|BhuAMb3(HC8?djDzU?bo(RvuD#jRW)j%S zy!^v*=h0?wQzp2JUsHJHbvreg6MC-2#o)bvZF8qZaZo2rc&W-$*08@sZZmACQWVN# zsBHW6h2=={_SofnefQ)R+0Xm2K7JKYLdW_B;-13pc(dbar^uVQ2nBBv1@~kl25A(N z^Htec&*6%vx!s3^nW{z|siq3ydfyMR${1xvi=Y3+}$68iE8nC`d_|`hDPJ<27kh$}q z3eS6lG^{+&DL5*4*3C!RidQe;i)3BAg^0d^lU(q9)@H0y3J7=fLLQd{NpulY^+E%W z9O_hj&GJfqC6A~1?$)F$UZN&hk2SFYd58~iiqCwrv=4w(eYmB#8r8GJ<{J`3&@T`o ze&`{~=Z|NXbPq5QGb6!Gvgs&Im~xbiY3n^!dBLqk zMinck^eH|C)qIbmspZNU_HR;mKYF^zd?^}iQlU&1J7TAZ{wANR$`(fV0WoiK z&MpGfAY|aEi1m0uP|rKR`Q z?~!={Lm2_1Jy_KSy~&%dP9SQFSp1Q9FPigSegvaG1YN%py1=4(UmRM-m#o+#&Q;Si zQS2ThK@r>gEPg%IrD`uDMxcIG_Wf$0VVFQPSLzL>lSjqx{r%lNK_3OB7jxJ@cjda} z!-Z;vy9t8lWS%;G6qYHtY5Z5O{~IgJ-hES>NIW<9)Qj*GaO=03wkW&N?kstttEGnN zT-Wr_@DJa3!OQ0@fzTy4h9NUufdB8TnElZ|z$AYYx}MtsWJjAA(7F{=BT7y)j<-s~ zuYPdj!WA~`F>Q`6H#?UWY1H_w+Z?M=Ya*bqCP)&k^H>w|h6eeXZ#SX#5#Jdh6>U}X zy5Y4sm+^A48qbT1QG!FK$J;W`GMg5%)KMLFP20&!G|h@3O&M`q4pnQ^VQ^D}t@N`Q zQdNEZOtR2%>C;P}PPJk_zZ-WFSaacL*eU>p*z)3}g;4~xo+z6sw73I__HJ;^8P_R{ zK77q|$dS!Ue)Txg{qsnPN~O7{eq@L6rQlz}XM%?98f1_K9^&F@i*2$*2TV(idfZm9 zL4LH&Q0~yc4{iA#xiTZochkMcwXccWT&w>9TC_+25g*MU0rJ66@@P9eKAJP5OK;c>D{ z)Bb*+93=1PhO_5FO$QrXNe7J= zBx5fle+?Mq#gv!po)R?VeQc$7>SG9!7zNJjKf-;CHSI7KHzoHGgkaDS?OWvV2p6~9 z%5mpMqSrW@9JKT_UvxNE#eYZ;XH(KYf#Ahnqgqi84mLlGSxF?>=?|l!q;tuAw(s@W z*FK8ghckHbEr1ke0MphDiwe=YYD`;S_$Y$A;bPtWiTt8mkJEWuiG?kBH;3e;Wx8K8 z%$VpHO2CxOi#iBOBEWk0zk`{|+S`mxc&-onjZY`2@13goEBa7gWpfebu6bmXPfSpw z8fWJ)*CPZMX&ac{ZRc0~`X956BO%8-qD44Q1p{0B3x7L#8EX&!rIINIDiG#~kUce2 zR6}gzJ=l*Cv_I$rWj=vzOps0izr6kyhxM0ZkbF86(^Zbv`z=Fk24#p8fdH3>nXRj! zTrq+G;-(tEans{ixqpZ_!@)~b8ok@oj=&g~tHhAReIQbv4h^9THP+M&HynD0*U3%~ zeXA?UhyCuQa1-h#Bz^vh{SoE*FxmBkbp)!-J|K)EvLM1R#amG%Ta#Lx6b*_V)!QUfK+ zw`1q&-DLY!W(o6^&&>gQYEzh{_34Q#>*in45T9Hds!W^X$-&!$o|{HR(Q!OuKhtzO z@{nu+{)bDY62_cfGBQ5%D<$XBH11JENt=YQ2XP1OsoS@}ZX* zthC;;O4k>s4z_OJvk`HPq#tt|jrS1IkKQzree28gF8BR}v1n3Er$7wCe?3O+OY+<&8l z%QqM!YOD>%qb0duwg}~+*%YGEcBGtRJ+I6nKS=1^nla0dk1D)n(*HPdpxUcR5`CkQ ztc>9F)_ha~C$e6A=MH&vcZ5B{|5UV(VoH@kcj-9OC*Ui~7!=I9VM1Rbi6N$bh#Z*| z#cF%v#Ny)EU>%kT=TrD{F0pifRX2np0Fin{ke%aD{^=X{0WR74fUQ% zoQlp2pFA?q;E&}fGFs?vg1zVY9nN}!!O$b_--18}f&@N&> zs%E*xQB?i1ywz8ZL+aVia8=b%nL(7A^En%fkM~cr5SpZ;r|#}y>8F}r-y=y(W2t&* ztETfeDnv{`SV*rF-CHh~j}8bJd+}w-H5R#6)KRw@Lzi5{k}wfk(oGcU;-6)rje711 z`D=qKSpi}Z7F{i|EH@oG3f@pB2U~0ro#!=?Z+lBo*lVK?G*Yi7J}SvBFMq%OEXDC& zfaR?G^UMb1^(OPxu{mQnh$!Q|MO zH5?Iii&kXb!PwR|K5 z+_fm?I_D%!S$7`&KWn~*JMPw`%vb7L5i<9wA1 z_tx!m$7{9uE|T;y6GgY}gL-icOF0y5DEdNfZxJ31NrDHA9 zUrFKMunfoJ;M1~daB2+2-;hi%BVJJKtQ*dcCJDP_npZ?^5sx+`->Y`bICIoS#H{V< zV})~fzOq=5qk12#K8A^)y`6z!dCy7*k@GA&pFDj7Yt&3K9uIue{^3(@X%hQ!PkEo2ZCv)3U`ztJk(fL1urK(^P6WR_+z%-LliVCHSA_@uGL(kY=^4s={MsM zQObvD^%IKic%=Ji^;#jvizEg4xr?GR32Y?&YvTtmt<_r@m!LiNvKkEYN+U^%QB5a? zm474%TjCe3nlc!qDvF=i#>}~JiM}g8;$qEsS{qEoDxflxA$CLP3)2Yh`G+eg1WDy(j`T*z?TG8-qL4iWoW9FvUV*|58y5FiPOox0tZ`2s`uO5kgN{pIOQ=KuqxXl3TC%827&!s8Xfu5O__eK~ z#mGpWE9>pA*Ixa)-y<2BS-0}}v|$$Ht1Rp(j(vmhOq@BCxRT-|6ih=;o*-8jTzESB zM%kao4xdBX>a&PZ@Kn6>Iez>KgIhye3<+;Kw{F=-*PH@rR3xov^e8B+yFKTS+3u)7 zF?mC3^i)otkb&0^mlo7NI8PMa=EJ31X!7b;6+FvNhI%AAO?u0KtPww;7Ui1zC~NR}#`w2!&(E_DHUoW*lsrD!k!%j|`8!}}95vW!LGzfoYg+GCjYsny< zCl8($W>3ul>0}5ihIo&Nq1#$zf|N3db8vp3=+i^QK@zxS8YCWQm0)(J4=HZ%faWnU zNwM=OsEl%Vd<079AvC4AcIXc4pVBkqIQD_HfjFv5UleSE2h7hD5*GqhOL+lc!rf08 zxhIn!^Ya*(o~fV-0*yUa{^z?|hrwU67*rwPp`_TFY{yG)-SOd3zV*?_^Z-F?*D$;J zjPnxlY*zZ_!ZwBncy!uPjr@QW4t{=uxQqe3b!jUxZkY>>j%zJa-Tukn{q}xdH(WRVyJN9`9kYR| zk;gPe|LN6W$Ma;9@Itq(x}gkda?_h5BBd1tZj+uRCCMK+*DmkGw3i#sG`1Z}I{bc; z`%EwT_ZQ6kPv7PzFc)0Z)r+NXh>UOma^%m$8fm(hsCC+1ZB)du2=-JC>nZ7$AL1=S zy>~Yl z-3$+u-h02uHG$ZI@VG3?N2DSb?b3S?sGs^__y8_sfDd~6^+xBxvfCnOYq9JDouZ75 z{;kKnjDaAK#s8n3SQ|QI!6I_&7K#!SNh-OflV~=od5xGJZRDxgCSp={w$;h0RH`jT zJl+3(f3Bwwd-C0^6oAhMq0z4k{LY%LlA*mKv=eqkK@eo`9NdxDadp=m)7AT1O;Lse z7m&dxAcys@htj`({x>oYia=4_A4M6af6@N@*8lutl>yAG{(HX7i3zSZ;)~#3*s`TR z^_L3E%kLFyuWFMsH@F=cDv`r|Nfyw--lN9nxWjnR%7i1JQbHir-6^zf$D!^?oXL>% zm!7v0P`~bt5L1R~;hIY=Fa2rT4%@Am2$8tfIKI*vq_=pEiw}SD!`RI8FxMeDnU5Lx z8|U#;F4pKzMCXfQ{K4D@qc$_6aFC5vwux`3#O7@!4@Y*2`h_oX{zt_<-tkCRd>Ct) zuq|UdahQ|6JEtd}ZO!AKvwi;2Y>lxRieg0ZhHM`P-Xq1-4gaJJ?&F+THwITe_T;`H zkA~Ruv&^^HgwHKHF7LvrDQzI64AoMinsb>iT;N;pz)yVMlGTW|agCxAosx(wPU;75 z6zgfru%!K}`}-t0|36dn{QU`YO!y!H)Y!%9N*vXRB0!+?VEO8lD^8_K4Pd_tgqlBQ zk?c&U=zxu0w9%r*BtD^xhWHY~gHAM(QzGJ-CsNJJFOm!tjwM}wKCW|=WN+->dM+h{_&zS@f$YwTS+|^(hsd&wNRyGy=DmHfXG&6Fmz2FiQAG!&Fuqbk4%Ek`UUmCHZD;6 zK=_vnjQ-P3ss8l=5K#bj5?zm{}{pNNh zTIeiP=IO;oqf2C~ShTTp6*6a0y_k1FT&TRNDrHyHq;|;hq)fPdEsEAw<7~;(bokk` z+f5j(&Rz=U$J+(y)`>q~U-O2YV0PIhcqpO5{|W6Lx#tRHW+aJ}DOz3dDDYqjCo0~YwOZ<70omsk+2TZt)Nc;23E z56}LF=u#+P#zkC}y(t&3>3-NovNn=+ouklK>Y`E^rXBPB3;anYOL)vCMDQCm+(umH z(Lwl<9k^F~AA2c(b+Gk>Th2Yr`<}uFIVhHc*-c$!{iVLCURNR{ydl8HiGf1h<2}9Hpdc_IdT1LtVDyedA+kyjAPs+LGjlnM<$ViJfg#pN;%q z+`VU5Q~SRC3xa_3CcRgsixh=GM5T#@$ud?u$e3)3@l!TfeWG0w$t+SES%QFLLRg*oWrQ0S!RT>x6Ag?UUB2uf%Je&lXiP65M&GR7xV21Lw`c^~_frp2+ zaWtuAL4YG-r>N)gao3}SUA}{s0V&PQlyXU_UCP9Yh8LLj4i&q_MJ#+*zw%sto5?VRpRcv-EwkPNvX8&vs1`ZwH{cJ3=+304MZ+`3_tyZ6Ns3AuG`W6f-eg9rJK?!9JOnYOJu>A6#Gwh_XE zRR~8b2SSi?JGXx$(w`^-Cfe)z_3nek?b$8`->NM8grc;%Q$HkjUYD7@#dAqLnBDnI zbTbud*_$Um*%Dl&Tk6urZp+}(&OjC>!G(m zQ-5+T`Db1D|2Ka1_d)3YIRpQ{kew{;6YSoSxNk5LR!&PH_=bGy6@U%7ik?VbK}2{F zu54`5CAFw9HpK3{cwEQ%;I>E&M^g0#W9iU!Z_6auz`aL!;@V~&v1#c9>0>7hJgJaz zN0};_%FUs2Hz$;DrBeLO`3snGsXLGsm0N4Zr559_H`gfd?D@IaxLfA_)RxQ*F}tVD zDchgkA1iR}w%!xCUx;$Z7l4ambS7pygO{yOa3ye6Ve1>@u&d#fWy^^c^Yv+8VILgB zZtQf zQF>=5Lq+^L)hakR{fyrR=dM3axM2hB{Cr?AipG%kq8If)>Ttee-gquJMO z#{2rIsYy?XRPC|Jp)fzpo{=>Znmk%wn%Ye&%(#A1Vy*gRuQqXn&--guVb>G9tujI; zVc=0!M!Ycw(7}hq?@Tyh2#L$>9QEM^ZclmuBREXcGtgx}Kep&Y^aC;!M3qzbL*i(? z@%|kjf*8)W&^3TZWKxwLcBLMeYdvR;lsw|4-9%k3hd5u{4j6{RvV~jc(Plq;#pPc> zOzIBUt0?^VNU&3J=dF`qCol~Zar_+9%|nj4m@yoANEaI-K90XFxx-Mt^gw8FCpW0H}G6Io5W=odFzC$%>(Nlf2f^YBltIXmc&e_U@;f z219M)$V2-_^2(ljG*3aLC0k z?T~RV5^k_bocW`vouX70wI`dRu_ze|y83n{U=xQ}rutSgKVogsFww4N=gzmgb?3S! zpXjPK4Zcd2^I1xCzDa+OEn&H;`q1&`Eg>RUa;1yKVw8N)FdZ~qzj4e@S5+fwZEROG z41MXra++2V<&RHC)WnxFY^$ENM}U;E*Nw7{BXFLj$#DenX*_M!hW}~(i|YD9wC9EW zKHW*@^!u(-jPA+X>y~wq*|)ZLM8-@nCwpQRAH-_}4O7B6F>Mv)*0`wM@xIJzjVu-$ zp1OFQs%#;h0?soj(Lky1frTAO4l-nq*))jo2?g!ub^6f+`f)tD?^cNrQBpT1TJJxQ!6)U@6$7);Sh@ZXSskeci2TCqJQi%3RTo5dc1S|7OX*dr z;H#Ba_Wo9p{)wHh8579HC*YlT&Sc>u;lEs5;(+Us;9M5_CrcL9D8L^C44m>Vw_aaM(T>0p4zNwOcQMN8_&mubU0Eq&`<$Np&`kqKh|(+B4*EspRd z?OI(Q-Rb42f@q7d8DBX4f~yj9xOrn^^MSNnKa=mirS?;_WCKOzbH)dsUqa%6mE#*A zc!z4}<`7N9>#Xe1bXgeL~icpx>TlI=k zDz3aPj^5#44%Y0bS0%=-KlkIlqn*{20t|r^II_(L#9dD#vZD}z?ronP!?Y5{dIWA{ z2$_72>0e(5STBn($1{B^Xtk~q$l#>|)Up|~IlXBaDMY~r#h`~I>1wfi#C zucQYXGta*b1q(Z#xoiNGz(Me(EtG1pd&%O0yS~a|T#K5*8?Ve~YgzJ%U-K#;rEh9N zNjOlo*D{`0?@Pq#Ee=y&AvwHbb^f+R5TlI;W2ToA>q|h2Js_<2~MQ?cL3#YtGun3y~R7$0_E+K*GQ&RrL=;on>XSbd=JhGX*aDkl;G zAKFLjbcSf>eJ?SfNO%JIe6B~P)Dec+I6a~aDUf3HixXUojtcyi-S1b7^RD>!y{L&oG z>+{r#kvSv{9c&@#vN-&t-j%Dk^6_1$30p_xZSgXiqcObNc?q5ytD}e63&-DC+M^#@ z9+G|l+&Ayy?>3lbYE4z&&JM{6@+rM{JS8qD{?msllm9ZwRnFnhwGYTY0%OsCIC{cy z#VZh;xh)gS8q2wDJF|$%aA!S5&cKG1`_KHkhF1e~j6#Oa0z9o&Zg7i7{7E9`U$?vb z-JAR$&YJ)8*%=}v67Ecw2)-;gq}U6msDSA}7igyNtSf3h#rFt)^^Y*eZ=NPHYHgM4KhzF#8CR%PJ?#sOabMNl*mFvLP-Xc;@IdH;C4=&< zPo9c5=J%sOrhOTwS6{yAh!y*3VL#ESfuUFY9fO zo4oFbexvvuem2|mr}lijulJ#XXeOwJG|ynZTXZK@F_q`Nr}No`HhiGFPs3;Oz}}IW zxnMUq1O6H=Ynf9ySifh1Y4Xt0a%KU=zE>_KS(u5Iz0;j6cEy!Bj9pEL{;Bw(vJS2R zBQkvqp8#?5#&2^PPPFmX$FShsPdKtSKgt{?j+DP-iP^lzMXwkqVF z=|g#Yd3o`!@ON$}3$o+#?QwaGQ!UDm1R7I!B|JphVzW_KCruQ!RvV1t^dr^L&@f|7X_R*M8|g@=b`N`huuzip4;!pcSV$x2i=6AhX!>o z&`wsiD55K55AICtU>aBgJ`8w*0*OP@u3q@CLhA60hw#>R^ zn<3!(+DZ8!$)}UP>l41SoW6k!Q`1Haf#qAE%x?~XPV_ak9(@Oc$4DH&O6JDpbo%qK zwT>MD`I{f~8@k?wUGIfo`872*UQ<`7@mAUCyemRkJ-CY97b8^{B3bPM4N=b4CfMCJ z;}>-ieMeHFcQs-6JF8tc5t$y8ix_9*l20x!%iVu3PeZd+A?^yv55!CW;a(CDKd_5V zf*-L?iVv5Kz#SKViYY~($MM|!jcCI5AyV~EqQhb6HO_~Odb1B-JCh#Zi~}Ym3O%cD z8ihVFc;3Oz{W0!ea;cfuQCSA;?^4~xky9s>lBYb;d`9%~e{LK58(Y{P&210=*7ugl zt?@OG)IhswHot@W=I)gXJHMD!LGAw#{RGl?gbBpH51bXg4FODdhs!E|1YDU00Qq<@ zrn*f8r@1^l3$JNk7MqfgL)Ha`1f(8sMT8ISpNO32oOsSP^~x%ZkY7@Nwl+NDOo(m= zV(`oSrSJoFjlcW(7LVrt-i!Vb*;VxqgR9*mI(EXKBA}Di(nW(YS-2tahoeMW<1OGO&&YOWllHE&SvNhR%f5oNnmLmIS`WS%tjrbmI6 zsA=3oA=rR`tPF7SO10L+%h>|tS<+w7EsA7-JX_!Rk0*e)B=f*tSwP@}kB9<_>s_e* zZ$ufj@Gf=yEkJ7p3@(@$43ISuckjHCPY0N@qT+GSeDRQx3 zi443^Q`MA;63?-`!Tvo;Wn7(#)m%7{toKpfp;zjXEp5nzNgL%fUJXlPfz5heF0UKt zl-x6gq>CwNsu~lTXzOlHeAAM)_F2tCG3LPltE!ddyh*$C&hC+?!s)QdNZqBw3XM*O zyoJpf)YUf`H%${q%EU?f`NG62v8+A%iGZk1n}r@`hTWa^vv$7&zpmfX)ZUa9q`0)( z;Qylb{=Kk!^xEv85%L*T0i5ocJQULTY`td|LZgP(vDCjbyIHkiqG)kcnFK-DM&CF0 ztgO2tDeY7bbvQ@kzol}U80=?kA0~$_lD6^SHAY&rfr#uu_b32iQ)M4N++v>ay5mB> zX`z=}?qzzPeO1<``nIz~q~zk0uNLjfEk-NJaXgCc6LnxV+_cG5W!+1Gx>kT*6_~xY zu%MoK#c`2;k?W~gv!J7K=p$nG3Kep6Rxg4|=j|NS9WB*btXr-I@q`|uBELv`Uo5iY zgnoB;!D41-!1V}`O4iefy~V4I7plpGpb#rgZ{E!I`JvBwx_XtOqYOqJQ{0ZnJ{EQ` zk6y#>fZd@@l0K zf@P=nb;olwp=d^XW$u_{T3wi~D>&8zbT?1BYPxc#?t7!6`&&(kSbr%U$n{HIv80hn zI_o#rZ&nW5;Bv6(J*wj4Fh8_@jM73WT9Om;&5tvFD^_Yp;dQcr(~B0XgMEf!6&I4N zyEs#fJb@DsLhtZKZ=bbD$zNY;tP@}Ms!?q|y|)-2wi9@)FD2VUUG($YPkc?>sU@2H z6XH3lFUc>N6VO=oqQDJqA@rEKlJxgh?zhpslOXBhFNVC^mdd4>=iPZ$wl2mowYvK^ zh3Ezkadp@>Y0d$x7&r#C53M$%DcdZAKatjL0_1WLZ-B`p{XF99==0~FipNx?@IE8* z)|S?zz!ETQ&~UmSj{Eke2&k4Y5W56Z_b1>8uIVNhhM3NikK4psSh{B#7wyPTW^ zXT?jy1A={)gGD>r4qLhQP3BuET#tUEk^GAF||U5lgo#7jot$1?-e z&K43#*%lAGEhj+qdmX};M1j}W4U?K+OApIn(<#ucO0Czg}YLQRD6 zkJlvvJgW7cf0>~YEU2MgsxITXCEvpf9M?)2nbokz?}2^RHLGURB#YM_@wK6?$-#Bk zF8#|~`n}ONfy_~M@3n6*6Q2n0ZLj=F9gEvj?{|UY_Fxz3;-~iKw$e8dp1F4iE`^DU zj2U)2Z-3dlk?QdC)ikF#?+W~R2p3)``5rDB1HLt07=LTp%U35>o;i=Od$p)0I4Lsh+iEt5z6pa=gG&w-c)k}IRLLL ze%U7gc{U6bB$= zU&JB0S*{hjDcLlHsP`n`em;+y-o_44(Zib9&CQ`V{2U+9eYz*UvzH8}JsM7EXFLKk z;@L)9#HAwHS+e~Tmv2@)YKlWK-VV(6jrWAhcYUv|mT$7_^SVV@xKa_+)y9XZER~8! z2g`10PVoIkWbPSAHTwmzr^#o+8RYqG-?q81XWNq=)~zAK_GsP?o0ODqyNDp+t4`r+0CMLd*kAlPudV* z&MGYID##%IjvCZ86k=}D9(^=#)OmWGgZ8BL@qFjXDzjjDuhl+pzB{$?#N>es$;sgH zfhq-#zu4LqXSo9J22L^sOIn#0)|qwh-7MU#h1~h#Tq{L+bQpv@x^^y(DVmyD-wVgh z{#y$AkJ(~N*58O~`x07=fQxvZJTBT5f|iu-YY+&AsQU={KYb7TaGo0v%G{q79fZg@ z`I5M^YWRf&^@1<$z(1_xVlf5BwoFh3*^=+okrTE4K9TW;&#!-Q*wKmj!ibGBoo5U+ z|0ffu|GpCX?{lqy)KrIqFR3`A6=NS`){=!3-W#tsw|=zeE>0{GgiV{u67>kg5PvLz1yX96V(g|PPTa}A-8wZSnP9%|;3MO&B zgiEQ@uN{6C69oa?6j~u8r@OBL3*6z%FeAW17a?moH*4h{FxCZ-EaT_u*~0x0Q@2YfK|K_hbZz3*T`x&k`%V5h!tN!9t80C^Z#BP=Kigkqff{v(T*OKhFYU2z>~C01B`eQb zqrNH(P>;TrO&$NCp~e-aM%FekN|1pmVr;i(FlAcPc-G~`nC3`C3`p18+wtA}LUB)o zEY0;nRpZ1tPLp+aBIhJ^i1;cSE!_`=S6IPihY&d|pvxf{U9($;z`KWER4F#4|X}D|0NDpU;bn#!L z_H1Lt^;Fgh2D=g;Q_Z zP+@_My%-3wCj+E2KdBr~fx0t&N2c0Q39vu3NZn46riglke8tmq76-9z#vJX%OKv-) zYmR$)Ztr@9o==#vx{G&k@(c-dTs4^EN6zdqz%)j|%a5Q+x*EY$Mp!E*T(*vQ4)En} zC^m%kt_?g66Sp?iANMS;-ENX~Em(co!}qvaX3It#dw^yRXs1-Y1_j^3MbaXL3583! zCi`DKb#&cdg?*b>gEM6I_d(tblcM{rblu9W-uJJ3_b1||fvKD$RK;KqMpuAvL8*m& z?&OzDlf;U>+jfwnwBF2gEO#utXfKiNy65bMd>TvO|D4~NF_L%BGejS)x5 zu|N~Hnc+ucG6uWujO}jtKrCmc#&Az27Wz@2B}5eC9L;OpklbC*kvNG*+0?CWjtm8o z&y4M!oPjcg1!AYfp4q!vQq0AP znK`AF^|_Ch^T`%AeFXaBBb>~+LPG*J7cZ#_{D9Np<*`OaqYlxXdD3}n(>m>p8rWf# zoS!}#w{JpU%G_WS+at>8Crt|IwV}tdb^)z^?mT2zztV5=trvFPIcMS5g!d~Y=lTR7 zvALY?7i7yH9t)W+E?mTwVhHdE+e@k!p$C?Awv^B^-APG#Q^um?e7CalxuN1s!+hm} zYwTbs?bvbkV8W_yuudRWu`2*TH*x!XT9Cs`B1269W4v?-Uyo{GY1Fh_pqHVJzgbC& z_FGF`*EUY*PIu^ior}~RZ%ME|UB9+2e%8pA78P9nV~Uj{cX5vqEHySkB7snr+R0DO zc-r?te#ms2CwygxV|;1nkcCGYL&(LHY%Ld>spl&cmR7!pY58*=vV6?k+cDG50a=zCbUW!3kWt3{wpu|Yji9$q zlZSO?5EA}*9_(5l`@;~0=Ra3&afS#_BVop971Qq1ICbs?WPr-|!3z`x=2AkkGKu7r z`BkXG&>KkY5mi-~!N}pa7H~ir9fQlYC(R~J%+D&~klxIS?tBr5y{@Xh2cJUSdjb8g zV7nrO3TDxmtXG?;%7QD{2)papmzAIoxA(1&K!WH!tZr8kYU3 zDc)<{u4}pyZt+bs4;>P!h@PrLPmYfeCqnpW-1J6rpQQ6K9E;c)eur1W21Lr!$VHEAZn*Xr@(}*fU z_%dB=J7b%GJ3yA^pYYoXW;}Q}Di=h~Sa&K9I0bL->wJ0az~MQ#L1E%`Z@jF>CPeAwHMiJQ?>lj9l@oJ4z@I@@99ldK=?e(ek%2b&s72?E?l2C z!^-QHrd#YVN7Q@%&X0zq_Y~f{r87+Vw;7vgeI(U4gRce#~Q3XT1G5d?3 zPHDni;4~lx^h-ZwcjjbcPdn#P*iY0C*s7CG6N?sMg%`o|*y$17Q0~2bYfgja(W7|r z9L_U#8>;xgm20}xqt{H6?s~{Jf8c)+s>TMIl-}6_?`r>~!y>;8$L3MtEa`x?5F>7{ zBZPSzWV)=Y5+ZEv&g9ObEx7{i(v8~R+CK4Mdr|w)thhQ!L3g7SAsZ#WrjW-{h2;$^ z#1&(Pm;Hg9Q0(b+xEzg!3gh(Mxcz=j5hOr;GCe-MU3PmcvbM&Ais2#P;mhAA0?WV~ z%6}tj68#SDP5$eaRFEw9ZFfk=!;H(_2^?ddwTS2tK5Hi9+E{~;AvErAOPBrTII^_2fKb) zp&nZh>}jpOEs)VARvr4wi4Os87@q3)!xLQ#7X}J-&^Hce>&G57CXMNITvFKZ76YYr zj-7W17I}Rx`W|UVMvSa%Va3dLvcT)XJ;}r_1fKe75zH5YOmb&7_XOOf;p2qwdaj3O z9t*e~6=N>o@v|~6LdK4+aUPm%jC00xnIHi46E#e_bn6rm-@?vPTwt|<)pzt1?tS&P zHcjF6Cv|5(3AKPu))VD#Z#4xUk*|Z4G-E5}l5tuFVk#v3Ww2G4!0V{DoW#~EUPXlVCVFbuy@DKM+J)vW#8R= zlyIx_bx7qFFAaecr!b2qkKEZAzdOR%*PHD)fu-)x_RC5-EOQV}mi+!nrb(qN?7SQ= zkJhIO`AU!*J4%i^b?(}cnnrseJPWv0hqnU7 zn)C06#OQge^Z5XnpCEsF`HdNr1x^IDU+!KYAF-Bij|kxhGGcG1uJ{?f-xzv7X*IZ)R^JWwrU>rQTAD1R5 z8=k_h%_dM#(;jsOf5GO_8(`l)a%Q@{9!22MLG#i=r8RIrG41*o)jHaSmH1Ziu6hn7 z0XN0WY-7YXFxq6?Qg{|(%D|s}E#P~;)1y_yQacLS6~cy@PO7EwwWR~5xh5?ml2Y>V zmWfi+xBXk(<1@6A(p>|f^)$lGJJOvF)l7@wX0RqCmG+Ov|n^W8xZuDUcJzeQL zST|NFnl>dqPDN!d#eX+VE6pgvl|fj;k6esfo9*!nvVono0$t5AyPui%CM7ZM)&P4xU50$&TPVaJn77-)aV-PR z*t-Yuqf(PC>aXtY5$YnsRyV(XTEx&nR#Me4C4WP>dE7T>tMO5N=Dudse9^=D(8x?D z@f5+t80z&^qYi=ghb`WfI9^?BPcIVCM-TvgG6;^Px`jyuZdaUXOTxy+)vIe;!^nj4 zY2-X(StTMEBN%m^aKl~oro0py(ZKKQAed)A8TtS1e6HvPd$;IMXgCbt`~bc3oPzpTPKQB0!@}DbVa^>+p)KOxXDW=*G_Ju ziKY?ns=)=e=q@8EheB#xFfJmxw`cn@pA59?)n&R!I?+>5rxZoJb(DJlMV%d6Jufw+LBCE}CUUG<`bOz?(HsXZtFNE1BZ*t44t<+5 zG8SIBeMteWg3D* zYfoe(ZXGVx>$dtU-r3~2fqPXxk5>un>foVo#X!2K2HUS%2@ai?1$iktof$Z)H)qQP zP^1VMIA5h?{IY0GK-p3eSa9OY$e7`17`JM=WARpYaA~BU<4AdJj(ZtdoE^cYAUmV` zbk3(+@a0_26Z=PsY~6Wg6F64KW!o48Z5ut#r`JNVq*OI=8O*xLA)MP5@A=MZVY40; z6D7x@EBUs~n>Z6v)AH#aHtJDd81YYUo)Y1P%w2KU1q#Yop-V4r|3tCZJpV|}z7^L{ zqOw~n^)oZbH7cu#cWY7$ERMtf3AwNx*T}?60R)YoQ_5975_Ef{s^GZl72H$( zP>aEz&my~r1DBQG#l4L-NxhZUk(1|9Tx_~n2v*g8r#Y=pI=*s)8Kr17CAuTSo@{nu3$k! z8Z70R9(HK;@QL-NlJ(Hb`0eXm-j{niZ&8+M3#EQ01P>DJuIbhi7~ z?f`pmYz@RP-B4axm;(b$Wh%2O9C4iU79v^L*k-@lS69kVI&k1Y71}3+7`}G`pMd~K zq-9nBi9}9#&6iJ?@+&YlKLXN4LM_(T4!WkR&GZ9Z{T^TMy(R4)M5X4<@HNC-z+dB7 zwF`}*?U`Hc>9;IbDG-{Sd}Mzo`u`?wYMt(%2qY+h$r zKTX)_T-@kb51=`h;DG72JtDgOaOENdhgd=M?6JA7t51W>Lr~hoc%6#XGm*UO)1JBQ zSzhfCcA-l8IOj!r7eov06`F?WlQ(i9-muMlukWo=aA;GD#x|(`fpZzrC0dT;?USWE z-yuaKTm;85>z^*Rp=Xg3y4y)ZZqO93hdD3& znTb-(p)}k0&~~lG^7dt$`%fIxVHJ2i_rcW!9nS7~E_&}h{esYI;13?Z5w(l1fjf(H zwBnY-he2a9#A;5Y%Nj)AeMySl4Q{bBxQwU?43UoY3D6s7gaRHj`WuC3JY4;-4IgQ` z1G<;aAndRp*kZaMRF*cGip(V2kSpxdhwn)d*jahrV;aQCcYGhtWS*Hg@jTXpzE?Qi zl`l0lAt{>eq~tfEM>D?>h3J2QM|zIlVncjWfG?&W@Gcv66{$n`G8Y01yi?dMUB5rt zC-=6Pct9rl``a0uhdZDD1YaXuZcrz(tBiI_U>8a?;{qt`;0&5Ks49suE$#1Vn&4tj zgFn8lc43FqqJpcfU8MCFT4rAo!TiIDXIDs=&d7$(_CxIPVy0sPX*bYmfv*Jm)g@D! zZr)&95vhBdPn|Vpmu3J;|1w7XINmX8f&9iW8@f`>jW5kSAz31w>!VfxW8im0Zw1hn z2qf13a4?F(b2SO-X@6-fDXT3{4 zwZo`n;iC|JmVfY#N zx;uV->0Bm&LpTKmlG+m-5|?`=p>x4fR8*AfU`YN@$CW}>N1UGMd-x|dlG4Y2c}u0c zCc|s9!*cpzisGI(1W6re(XC9&wsGWL`plK(k3ID@+v9Y&ZSSWZcfH-1Ch<7ld)^^_ zpn?1Hlr%NOfvgFDLXD|jfvEzVf_9q9cz;xBRGqua-l48qs8K-aXRXB7v@9+&7FS1t z2mUgCZw+R^)UPd~oueFaRF(Hq7j=~vaE6;EL~8H*^R*v+dBVc2X8gcm*^5Qwr*X09 z`fQGSymG9^o&Z$zIqIgMpV*JhC!aqDlJZcfT)DvUCo0*WDQACu4(W@T?$n`Cyi;$nhXys zU|;Q43j2kPeVq?EJTYaw9LJUgYDpk_LZ(f0Is@2DLa6AfVOP_@Owe9cVXS($=ypA1 z&gFRg3jgzsK&4pa9%c!W2G$j1yeUCs%%~%TbsUXBgzXeD_7u+4$PS8O8dz)Ug+4a+ z;cL@Z@24`}jdl#=CbDdAZ<~Zv%BvWz=?o4wahR99LTNjsel?%}mi9$^Kpl#u)+9R5*oF(pb_UH5*ywp^pu}EDS<@%Dk(k1&3w8m1#JLf z;h&?Ys^0oP{$ixlG}x66CBQqg-k~&S%M3HjPF#$G89?MeZv}L^@;6kC@mK-&EGD%%f|@09;fy!0QgQ&uYfMq&P2nHBo46B6M6WVZZ2 z{A+(EP5u7d+qLW)z&zH!;7H{e7FASX&Dq-~gA0PSiD5(<0+)B3CN4)EcLvlqCTJ<# zFc%d2B%21q{cUMW_qFE&)`3{FQc?pHwZ-A1(HYJ~%NX zRUIRKY>z=C)Y4Hd;8YKKSP1gC7Yj^b=*<&@amVr6tsEKwNG7{rea2dS-g&n^=pOgO zpQ-h$SAR~zODofZy^F<>nF9!{>A%*MFJzflma~PlA%- zj)wmJ*~Gtp;{UfVtK0(QfAe6>pfN5HGu~tI6?qdh{)oYiUS3%B&x%NmtOcMpWQ=W7n)8JT?p5!yJ)lfcx0*E;Co^17%DYhFEC>=`Y#2%BF)aLr2no^==&z1Qe(9tL5Sjs|MhTwq8}{) zypWd!SFK=64+`SNH}HGGz(oT)M+1m*`0;~EcOrJA5pp3e{On3`eW(vIntqkpBbpQ$-%4GFxl+q2fwxme)V0}%O&J1sj9g$>;n_0hB&-)kzQ2-Dri_e1>#+)uvc#z zFaMhBDPQ~@)54-Fc;B<|D&H(!_Ks+#s`a)dXOjDCz_+TnT%WN5iB-K`hdINmP5AEk zTMj1LYGk^uJywhr+D9oIBy2uvYIaFw*;<2Kv!lI@-&j|au|>@ddcMDOaa(Ty{F)+b z$S8V&GiA2s^+8@8qjK+-kcaV$Fp9qE%FQz+(hdT*wx zP-32fAnlvc7a9H3MSf&$dVgZ%|MxGr?N1oC2g1RMm@|TII$9AZVWs_5!W#3IWDqEY z1AhWj`Ym$JTDu?2spUgWb^I*PArBg{C^8_A>JTd!d=B0p(VpM7Z}Gpg=+Fvw<~x~j6aN9obJjeE-ZXio_XW` z%E0mV0L4>9qCh@gj)}cG8dk%Pqeq^qI?7w@4nQXKX_~skedE!I$~)p0C_rK2dq58Z z>xu!+ez}#(F#Quj2{V*e5g_ue5Lf!3VMJbcnf-^VfU%CpXmJGj0`8Ps(Lb`(7PXN+7uc_0y`T;n(Og9=aF5R3 z3%Dqzu#|Ey{FGZ7h)hzrb2tK)SY8hIQ@_2gk91M0lqs$w18$%pr-51{e*wpzhzD9j z6JD|o6y*sDeUb6U&Dav;dpsT1CK?Ulg1UF>_+c4818-qsA$%VBoZkdrR11oP{G;aco0o@)$;J_SaEpTCOqw1^tYc$H-q#dr^*}q07 zvPQ0IQ9pX1FuB&UM_{BoeGCj>ej_?6J1kv-^w7;%UPf=CN+ zu&5Q!AY(^1y!+k?stYBY2A6x!?U>tL)7p9cK>!hm+{oj?L6&nb(6QreY$KYb61k&e z4zKKaN>$czd(E0W4v=6;1*07D{HWwxH^Uicmmnww8S*v6e_F|Ky8IS;V8VM@o*iU_ zW$MT-p6_lG1=ETo9q8wAvkd|SY{m_aWv*R}R*#61xb%hC2?>OD~ctZWx> z;mco}(8DxycvD0tfj8LQsCmlwhbIzQUKcdraOdqS_3{&wVFiO+;9G3m6-q3t zkvz43j}OB1Jr5DV`DkIJ6Me70aRY1%xqeMc*zF3?g`cC%2x7_z72^w>JJ~L`f z-oEbPK@{kW7uS#NLyO@>fs;TGL9D)>)oyUd|2Lvdj={6LiaG7T9+PalU>^dH<|J6` zx8t>+#wqXX=%9}uSP)9tVHoa#okI~GB=U7I1tQ^{yfad@pO z`L=$k^=tvi#7DnQ()X-RitRD;>hm=M7v2mL5Lf%H#WWaA6|@+SntooZHT$3eOO~;` zqh9h$)Yx0`)PK;>)DX3el8p8BGcS4$_27L;vd{U)1EL5sl-a&Ql{0Hgx3TErsq}f> zpvhjWHwA^#aLCjQK;;w36y9_ei=(OzKb^_PtKkCQE#ft+M~@J6IQ_2dl8+&%L_#`p zqx4a*6pCGUHS#>S+1(o^YJ%8ep(V7rn7=xKJ?k0Bk^FTY=TQpj3Kv3={_Tm2367&*2TwixgYHx{4h*2P($Yp6N4fO!J7gj@OZ4y}~ON zwMooKYigoSxs|4M$I;nd9yFt=JBvOP(^n|SSLq^sd6{v;`dBh#zU%0K{jxjDgx}S* z<|xz+$*yv@fcyR23z`p;o;-Q1v)$jX9%)7_+ z3@k5xzFWR(Dp{vZv^2e`-!mNdj3p-@*Zm_;c5FGlmxmfNR9VPAT_OKYT3+1xiRGg3 z_VI1OK(QG{k+d7pIkaoK0|z{1-ECI@8pqik{N-sW`K_x`7leWX!As?fR3J%!9A}0}%2_&dtQnU$ zLejaiU}?3Y^{S>`QSof|E@j#eUIRJ{k?VSG-?5s&?P%W3^MoTLIj+y9Y04K#2_5d@ zo$;QR$=EU3(w#QBt8K;(YEff*C}DhU|KgzYa`RufX5#P1@J#4n_f7zPTRSMNtd1E+ zX}+}zOhu-hY;SleTqh4+Q{WY-zjk0R2&xETkIBLq{|U`m%}# zJB5vzqX@U_vUV;))|HprtX5n}_v_9yF(<(4uS=TAC4_xqn>INgw-dsXqboLicgh2# ze4gtI(hU@zZ>`%c$@*Y6uj4X!<=|F8;+)3d+SN(~u=YwOu$lvIV8!^uMMhT+#Wnm`lV~{_|WRzxwTEwG?Eg;xviv2PNmYkjpqo7YY)& zj%5jF7_E3*Yi49nn<22@Cd^qKGaz{N!r;kIL$&Y|#3sQly{nx=4jgMy`6g%ueA$(8 zYk?c<9BiC$NaIA1b84!SWZoC!OVT!YdA$%e6sF(xZvh0;@kLtu zG{{ULnoH(g0Gtgc|Eu2*JNTFnO*8Q=&^Um4zq(Q3=1KnFsB8eFyE zN+@%yq0YH7Og_Ng?A^MH#`q9 z9$1}Q)215|{OFkMh_(&4XkEw&S^*;5&9GY-Jb&rOraBXAjREaJ)xt-srP1jJ_dxxZ zmYwWv*#ESWp1EYad_84QA}F`{b?SIhZvmly94KYTB7hcGEFL}}gZz;T5B@`?^i&2j zan2uVrIWW4!Gu8g$`C=mSe?iTz^<1ObpeVUb2mVIYR6k%qBsaXl1Hb_!+uao{yaib-Z$@JUDrF?K2(>wNq;u zlYAAQZ@B~XZw&siG4lcW*WZZ#`lc@b(+^QErxTB$S(S z4?Do2)vA0JN3;!s0O}+=Ol!)tn@4fdaS~Y=DY+6T!*WINYkxuFg(B}tZHhQ%Mm|#4W_cn9U#sDnOVJe4gkWiLKx~C+vl4~;RwEC zvk0`4Q!V#(?;XMuIP{N28bnFDn`v5HcKdM1!x8&1Se}I$HRF-pC%F%7Rbp(DVL+RyiPB=Y zym{@%(mTv;=43a~kXB~w4VSgp_PCpuQ$L2=)SQ=LAZX+unhM&+j?2^k8B!IeT}zdv zvMVtTJHDHzGkeKyyB?bW^dF<+4yC#AlGxI&k~&1U$vu=;0hQUNNt*Cf?z6JVirqJo z%r6PTG_7>B@1KefyZo1f-hZ42SuO!9C3k`f)+BWaK{;EoR#Q2z?&wUS!zExD`%TDo z?OuPJc??kvak}l}!8190FqXFyLIM*XYauTK$4TNJqr2o^XAWI@E(Amdi)yr`zO6*Y z{B-nZ$Q!!FOKo5{rzTvxZ>f`ry{3Wcm==5TlApfz!!x!<78K0v8=->*tmp7)_!9ui~m)m_obUgq6 zD(}04nrhR&qo}9|2vU?5l&UmEiUKAoT|@+=HxVK9UL_Wbya4>_K?%|BYB#{D?AQ|DsaK$9w*PMf zJDOC}t|;%p+Mz8MuZ5dkhn74F?(!lXe9kqjpFvuJ8Q?ZdA5ho)ihE{x0gt!Nb(UiJ zN*m67tiB$WM1>vk>08-O2}?IU7~SK&uY+W*Akd5KW0>tkXMc;77THxykBf&J=fs%8 zOel~ZYQFGHZ+={Xnx}9x;5q(c{T1$ROQZlW9$L7_*M<+jR5eo5RqpiJ)R{y4U3JE% z^!pZ(2fHGUk2;A3hS&RTBVyq0?9cen&lFj_B8EwkNhK9|^a8XM-@)v;d|ZoIsF zBcSAl_YQXTtn|!0GO0!~CaBUJFUc6UV?`R0(VW40AMrxOt!0rCm+eWR%hCtRa7uAS zdT0;vPAksdfPx#6CZD15?o|tinC9R@v?IVAC6>j*U?%YKp8oTZp6J@!z37n%rBxtzPj{d@w$m5^z2jd6l{ottM62OP1P*N^Fz-CKb z^f|0m{Rzv=XAW;hT`%yNzYL7MAwximX1U!&Ki%_+&rLA_SW2<6PfL>ZkI(T;wel<- zvGRB)omxzoszo4Kdrzpx?rs-64OvAb%#*YtoqtnN;VdgF5`KF1kkAb-NOGGrC+<{k zf=;KY=^^=dSA@eq2@?ah1nkcECkkq$BRa~@(r!&3N^IqEuiK+RyiZhLyfjt3hQ0Mr z*h6=gI~7AaMda6RphmT@-jNdoocd2-Pg#g)qwS&PjI7okh~SCRd8>tpw0KBx#3Qo~ z!Y(I+85}>Qfhc{6R->L5C9%T{eXIhdG&UNR*Boex*;ednbw9JZ)~zcal0=`2Z+Q$| zTiew=RD7Pg&d?w{0i(CXZO}`G6@AP-HGfbvb8k)kE*2_HYLO5%wc(!40qV>tzlLI( z*M%(pM99Sf@L1-5hAI7T`oTadNhaQU zjswz@men52$2-D5_g{w&NVKpgpx*_?Q;hEcH)idk_sn7pVqcq$thJ|zr-C7W{bl-B zo?hW_cyU2o07a?Vjld=kfoU{O0HT?})&!W5F^7OycAoD%Fs?=`L8GrWobyMhu=k&C zeTr5uRu%m)j#FyxiiE3<6d*yteUjWXnjVvCxA)Sm=D(LE)Q!=T#SECtWFK4_oU#l? z*a1%ibf&QbY8GWS&Vk8`9_-!Q$|ol*HFi8Mjn}!ndwl-hgQ3L+%u3X>9f#F~3rWfq zb&#r-r7+~*S+nZGGcO~dJ6JPBTswh1`3bpxZd9%0-1*H^*Mu)9BMP)bgluq74@|0q zvGybV$V4L4v>H3a80WXJ>*7?Ge$izg;C5EmNU}9aRbHv>$|0vFHt0BfCThFw3!!-N z5iA$2)JSjS@A8t*7b}a*P)ye3`W`vpCT10+ePW+?F5rRr;{ZP8m85C(Ns@BBKOy!! zw530H=~3sfXHCe1db0t!vXzQ*+ct?8G3?_Hk~yp1g?%o6_xijI`~f}46i`8(2ej`I zhVrdER8};_qx@Wo?lfe6Q!>2aR&tABy<^zv1I^^ru0-7jjkj!lFpNQ%Fws8_+*ts> z7Cz-jHN}wJiSL&dU*Fu&DbK+Xg^3js33?f6g>Et8*ylV+Uo{*aapRodw>t7ZW z;nkx|#fWAbzwf8%$^58+kE@7{M6D0Z8f-joEL)4a6fykJx!{C4?ZaDnZjk6L!5D#Y z^wpHBY)%(#hXQFhH|X;UIgc%ut#7Z*WfsDD`>iK!q`G{zQ#T)A$J}GtQ@i66;~0|2 zdv!pZ&@20VKvjcco0gvul`x_mi=Nc?Hz`PVS(^t@`!z`o7bUH{qouE<59|;k2@btc zk9s>Wk+0>KV1jYY>1z?>u4VcWkAT^s3UOS2PJM$EbK3S}r_6NCd)`V(?+$;!@OX_{ zbr{mP>|%TUP_pAKXv*sl=x8c>3_cdTc~cOWKu=f(w zrWm56uJr(ozY?mNExw^2fQE&j8p(0C=V(ASu(NN0rBlM4lxTPm)_I-xo{mEXiP!th z#CaV8oW{-ZBU|fSU%y^(H4tmBEKED(M@U0{(CSnuzJI;)T>YiDN_@B?V|spVCW5?5 z2oFFXb5?F>D@^2#B2!QF8Hg7Ny*fajkSxDi!+8$1h~Qy(C(f~Fr$4>~>b)_~grER7 z^#WKZy-u3>Mh)A3-piJ!pyX?*>E&CYos4&=OnRl1DssHy*1?ZK(4!R#GhVH%h#5>P z^dwD}dO~S?FFVG|316QuWFxGLc+f3khPa=Tb-)QW|K1hy`z^pu-fZx1gIpi0RsDnf zhl77Tj{Toulm5H5gT92Z2*7mC*)Hq1y*b!TrG;_Sd#XkhkFNDucTTy|AOQP8(hFmwY@uYRp7G?604aUGYqlFKT6p}c`!%%| z`A40yw1lO7!s}zyAZ~lIM3fAw6(&i~5TU1T5xBIAg5h^#(X1{FJw#4*2YR`KU{fzi z$khCUlmhk3zyXX{sDu2-<6Z0Xq2 zAzJ5)lkNLd)!M_)Ehc)?gqYNFUO3@Ko6+CDn|-b-i67UGBZWrrP$X=mcKzfM?CeO5 zjghXRKF^45$BLoE25KlfJqA5azyqP)JwUEb_0A=RerC{DMcJ|4RDRpI%1z zZ-CyuR4}50z>X3GvSyoYRjDk%_^Rx zZu$5ZhG`)=>{s95!vVj2m<#L@JxfiG3N;$`TE)5cE%8M~ZjF9L^@bJ&qgPf-yDnZB z;H8>By0OOM@gK|jonMp!Yr!I)!m)2-%yMe(5kD~~&-@A`j2s*t&X za%s61ylw`3d71)oQWrTcZzhOzWLNLB8bE^4W>gPGi9_-CM~H2ymQaTP(RWkh%l%(F zU3}ac6da&?TwTI1w7$=}++KhP9OmB2x2xoBKQbh%z)v^@JnAsvn=B`fOR2`+a$X?* z&=)wH#iF^vwmSJBdhGLAQ7QRqNiSibmr^xFwoRez8x7*j*cup$6q*XL|GFcq^G26w zDBIR`vu_`SQ|hfxLX18Lm~ijhso!S~#;5~Lv-%h2yQ~t)gs3eKgi*~<`uu2*$S4?W zZx!-P_IOkCg)1EAFX$XL=!Ut5Z#x4?#lWq>HX-PULZ;^oVt{kub`?HEt_+T~;B1{( zi@$Fy@+tOn_2qW+W87@s2j&9Sz=_}$K#}tD(fG2&Psjx}ctHUE4E@D=b{m*55TRx= zn<=F0zshWh=jBmJ>0rw)cJ4ji!og7d1MWZgcVa!X7PMl-8}+X8C}SUR5Ig4ep_zZt#0~Y1F;8El=OIJN zgKZi5xBLeb3W+@YEYYicPP`Xyeq39Kb-r&t^IEs+I})S{SzivnF2t{C5JX;l+Y5Ma^EDs3|=h{EyogQC3`>OD{7Tdfb7=*)OXlEF#ysb1+n|=EG zBP}eIStE?eJjJ5g!|Zm$`^KlN+S=9o)^*SI8`eAutdRCI57~^g9FIE#``boDf6Spp zlmZ1F=TFz|eYQf=3n|b;^^|m;NDeVmR_cr9I9zqC@XMd;Bh6<% z3C}r3^*noVJ>?2>ITZo^2oQErpzSHzhN>~OLsYqk-oNu$?c5&~QGf6r`@e%>@ee-6 znZUT&)^cM|fy@RP3QV#px#=MJ>2J%ut1p*N6FqxAi3`V`>e?rS>;%jfgmF)=kIS`Y zodM(ukn-;rGZh_a{DK9|Y9#->lB$Ep#+=jE*D-$f{LT28fS*#b0%e$jrl-^0sygVNh z$jy2GGWTwS7YShv7&uT8I>8!Ss*M*7F zF1`}^dZ*L7Jm5d`YqqN^YF$6LggP*QlR^YJf=uv_q_D!wP_j?pl*7=h*p!jWs0PGs zA>9;ol#`ZE%)T$7WI!@&>C+`=pINkHD+Ku|r|SAw6zI0=hRQtetellkb~>K3h;!~G zMc|IPZ=}U?o#^6hbVLQbc?^3IzWo$1DOi4lg9!opAw_^&J5NIaCOXH%;S13-kC=~r zud`awlbp|C(d(XkJ^oNZjMt?M5swnMOZE|0OQ%v0MxlYmuxdx$X>hmv#l_NH<-l_C zpnqR&TVK_~Oi$arxeD&HacS_tyTyu|GF%6KC^hXruw6#;BE_ipfU_Wtu8}M5H68|0sI%#tkg-FA8*^G|A`a)i@B&-$`y206tDBL?w6kWFV8)c4vtr;1~ z^jFpOzv@u{wI7_RF*v7MeLxxBz+rnKvca@LCRl z?w5AM=-L3CL5uzK9ATIhZ`|v;Y$w=lBTwIlzK&y91g~g+u@gUewlL!y2hNMo(zz(F#5{r_?(-WV>o7u8SJ|8jrP* zJ^7bTfb8zXSd`?F(0O3JnC6P_!}qi9#7Iyf1F~N$Jeu``J$v%%O{UH;cRk<;Vhc=o z)dNo5K2%??I_}HY!i4P-6MiQr^B*E863wm1`a6sEzq}_6jgt7bzN`lq)iR zhPCP8gc6M|v3j(#xXTYgKCf9xxkS0`qaXOt{xsP$^Yx(<-dCSrSg4Mrrh{hoz2I=6 z<`t%`_t^JI=0TilsX57o6z`yW`A*ZNPc^eYy9+8?Tg);TdqWbx-s$_NQ$G*A|mc(FaSNEj> z?N-z|3UA8~Anq**=$H{tbu6*4!bBQ{U6`1Px}@!cOn;JQc#Ay8^GtF#y4`0pp70Tx zpq$0@9>{RaZR&_zoqeCZdS0e7*~(2k3Ixc+EciF`G*4PB6hC zk@VRl9_$*~org%GPRaXsp5OH3QVeNual%; zUqhESUFgHjdNMV<;x;mRQ_@NU4ZT)WapTdrZePO=ea-Ca{*HM>>*o!YNnVVPD5M&*+?N9d(q7er1LT9Vr#kKHh&P`VI zxnfn~%ew>Xd`4UHCp#Rx=Upm}n4Whuyp(iqqh7*S>I!7Tbq5{$6VeEHP%NSAfD^-O zf4Nnq1}q8c`Qc-|+c9`*Lj5#9npKxxOMbtakB?uIo{d#(GI6uPvP(WS_AWcY!e99k zUCP=M@)Oc(qdnspQFeo}JaZ;gU;9H+Po7vt$5A&0tCF?+XEo|%$RLWHdgZ7UP~-gCxY`Hu9=)iN{DhpQNdsw= zs*NvQGg|4={3cjlQ(x_CO~lI@g0sFUU>a0%MTAX|=go^B_`dz$l+`~dsQz?@z#n|p z-(9)??D630F70T6hck_|G=M|1&x~St!7o?kwgebIjR*Qe+73#4UuzGp`5}hEu!inVGX9`Ucx>Vk%*4(lmM_0SFi=$UaUe%w)W1`i?o9|r4LWZjMt!OHV zL=3S4=yrg)L`Y0@V}mW|U3?o9~Y2?VB5vt*mpp8uAwfHQzL8 zL%cBw+!o3iLcxUFz!OB?;$AH-ORARY(NywUlI!dU%>Yq0zT1d!KU|0PPxV~Uxmo;B ztfNEWF7sp-8&&Qp!G)dKj;7{z&6G5CtD163ZCie^a!Egc{8c-Cpw7LkryhB`9d)-R{NB6zNZp6=jvJmlI{Bth zHTN4UEq^gf7->X??8+fo!Ks59@@y{pn;u#jY)o?4jJLCaO)u*D40mtzRufN)&cMLU z;dWNpUW2BWYK9_V6?%7Xw^_o!t_PBiwaaTh!aj#x1P5|(>g}HpL7YwDvKd~bX<$GK z!&;kJ*|(-xTo&ZuV$J4Cg#xu}km7VjfP!y&hjgiPB3^tN<;zu~~)2H=hI+l6=_M zLp$5sfrFD{@n)?T-Cj$-$jG<2$KoT{ZR^UOG3P`&27t`Vz44jURb$GgNxz4`S5%-1 zuF{3C=*ikDymo+VUz6~xCT&HO7j|!hJeM@_-L7K)cK0vp&VvF>f}aeLEn69C|M5 zT5#`&DJV(|kZ2~!J(I4nV+S%U23PKbb-q-KQHWAqYE{dL&~JklkG%lYQMj z!O6u8NiY%y4(5lb10dM@%KyLu(JTjKn;6nS=sO1uriesQ2p~@K9rP5)13-4nbr35? zliA5=hDjf@CglKw2i-hDSxs#(WeS`As^wHgkWzcyj)Hm{uI_DBAtIy{lbfT z^8^P_7HTj$we3&OWE|1{#UY(qf{EK5`@W;X9O=It)7CIyyQDla@=jNEJbKdfNyW}9 zW4@P9Vobl*pd`@0JEi3Ys`l#Q{vBZIR%IW~1>?NO$S8DJdEF2dfx8xP7|(luBQMp! z@wOaPQov{6u^MOW78G!W$HKSR!N`)63=m?Uvw>2GXfE`3C>U9CmIflppBo^XOHzI4 z-(Q*EpTKvf-eSy+zjI+g2#!^os?B)905OJUC^$f*mFBR+FoEjg>#Jx$6EsXuopRX3_b%*OzFa_LROzSv7#-cJxIB$mh0n-6`vsoC0|ZH9Qfrz~c8!z@AwF3t({ZC@4^`7AFa#WKUb@)4A3WbwbJhD~Df-1rn! zY%|KV{R+S8s`-sAjRYx~_6U7{v9>8=-B34t{rY{VFQzFtZ+i|)J^byFM|=E?jaLir zc0*HExyKBY=d0^T@{Q8=q~BH$$%mg3Emv4s_+4xgki!AHniwHQza7&og%FA3b|6bN zIebhndQ#J)w0tMr(mX&Aes`d@>DC%lX=_Z)S0P3+iqec5-NgD|^rorKA_T#xy&7tJ8dO#;eohOpfca|2XOlknA$f=lK2~O< zbkN$PE}5mEVc2&4Ca*!T?3W_844*^EEy(Mm5A_O+bba%aC7FI3K|zAGThO%=;3KI5 zGpibmyyh|kUkUlqPA9cMQXRldrsx#mU8qEt#pTVH{E7GpMu*VJA+^xy>;tZR2C*=J{AUWPkG!Gp+HC>imz z_6klhAIKVM-3cTOzsGKN$<&;(iVHQ1{Y+AQc-}{RVIx22enbMV3uH58UlN$Zi8I9O zMz;9l>ix%8J-f8^>a|MHc`mEM8F#S}(nrGm3D~XEHx?bp9qCIR%<11@T*2)IYleACUobXDh>C7JCD1b$Eh~F zELCk1WCQOa4fB9dJ-u=kOFpMk7y1*&!?_&bM9tLPOYH?Bp2Ld&o=)K0<10Pv+p0k3 zTcFgE5?K1y86`mU&kR5+9vzbFK3}}l`+0n%t@#o zFd;AkR0T`S2#oBt?wQ5bydLmSQ#>6!{AJUasNbn)mHFMAKe0O6-qD{$s$vCm5R5Ii zS;BKuf};IbP1#w;zXm(aT2sY-LZY3VrTxbohS&Xndt-Ql9kzIa<83Ra#fgYd&Clgx zgY{-!^*+P%1-FptnHgtj#-Vw(9jKJwvii58-UpDL9D6X2I<2xYp5@m;WXRL&YO-Ug z74cMq552JXV!(A=dze?3Xw*KlY+Oey7=qW5gKE;RI3|b8$+z}@DtqXA`XS^%XRzw_ z1?q6YYyjAYJrC$Pbz{)jd{Cj-vxunSQi?C)3c6d7DjchsR8ad$OzXut_Df7BJ3O4g zVJ-V}qh-sC>q82oFRk4$xr@9b83E;1`hA62s&T)a)4>%Umzg6^DkbIla4KE0X(O!0 zPp_I8V^ce_*(HW*FeltpJS)j={2mS~r68ff$(*#?5!7O>-Q@W;ooZ2zdHs@AQwxFqoin8?Pzf)hLVl_w}ZPN^A@(ak$)28GMUXxy@D>oW+~3dC3j?Kiyl?owYEL7F3V$J}q^T`HHBYtvxubyZ1|9 z;6gz#m|fwt3e4h`Kd6lUO%Cua`bEfBHwcGQRKX5xE&xiIGXNR+k=aMk(}7_B^?8Yr z?VQ6t&%zKW0rd@igVVhil!hscb0yLOk)do94BD&=j98@^)Awki@6A}om zOO@`0{fD0HP!H0b9*YM9Ca0=p8$q3*oJoHz9MD7Bk$bDq4-Nd3df&P1O{LA#-;k!x r?BDblZ@l6@3yvf1ne2kt%V7}2PI07k(Brr5_PmJTwZr`B^*r`p0y!HT4k(e&;1UHTCmv5IbIg_VV93c`AjQv zx4MJl(KudI;hAqF-4#wQZXRASafur@Zz(D%tK3ml)4ivsZ(wM2|M3%Z3rj0&8z*NM zS2uSL&li6F0fF$Kmr<{xV_wI`#iypdd!L^1Av5bsenDYTaY^adn%X*KeM4hYb7xm~ zPj6rUz>kT^sp*;7xp_1OySlc%vH9!w7GeM3&*9NA@$bn$a*?0P|05Rp^WT+=g)G-4 zDk@5<%m2tlaVhX0!C9!NuglS}>X==2^kEa0kDz70oASB3gHA-@5uW3j@Awr?QALax z;UCfdCE0&Yu*m;UlKn5i{zJDm%;@az(7esRwhao01!Yjx);Q6 zz?<%R2f&bO^7Ixs0g`Q$wX`b{`Q)^J%l!3}00bvEmyf+1LAs2U)UpcJ zyfujxk0|LXG2y8HVsX>2FYKnnbJ39)|9+t=p{Ib*2jdRDhMyUOW&H{1-X*f|2^TGnnqGNj}S2P)ngZp&F*x&YK@9|VD# zj&4HBwtU8dXMZy`?3A6Vw7=1y!+UP`poA0p@>p7dg0wG)A;^VRQ({tpUXAcaC!eLwvqeox zq5&vtN&BlY&g&?#;`)vlBKK09+G(vs+7;(4Me|;&A|# zlkSgK)`hrY9auEJp!wdJJ~q}zN>YV7`!uoAJ*4TmC2z7R#!hikN-#J|A{yamd)t{> zJ@VngvaIk6Kxgh1{EkgPWmY>0l;3ndtX&c4oC3^L%C82{rQMTwo6YUZNW&j|mn1)x*+~@SgY5+#ntJAY;dFW3L z!>V_OFtmXcfhm(v#2EJA)b%qVY5&JQIIj-MDLz76dSykbKVXj(F>6u=Fl$O~e7!-5 zW2QTw_4qVLbn1Fp^@__JzmbEXAEnt=Hh$>@z5io>7u2QrDAov>uHrK1BXcElAtb_QAUJ1{)qq+{sIsq+?XUiNxyji2wX~)Y8(s~e)VJdR_x?ctKPeKNY9N= zm*W3>GV{ea=`B&o$>}S?0k51{y3iNs7FhZF?Jz^XXuMTHbE4ff?yF=~*5MY3a`1M~ zqY7n$H;2V*vEjpFMOvq2NqS#Ao(}}(i@kd(zYIz6!olRSOx{U)UBBc)qvqrAFjg*->;=*SoZ-j8cPqV(7z+xlqLP>w^)|CSv_^2HRbI@-_o@G zNbQvrZT`_jX+L$)9G5|TbHjVo_)(RHP{cYo)7-ekoa&x1ZGrMbJn_zVL4#jh8G=ke zw+jFbvFkgEcLKR+*@HeRbNssKsb*gJ?U&cY>`t?v&!XOq@cU^R1-+csbl(x;lIzkS z`&p$AUKPkh{f!f8W#yP@iOh8?;UIfg!zfl38*72PT!0n@0VAa;xFoi?dpU-%)Pq0n z$*PJ;GXvOGW(O;M%JvS}V^Q0cOp6TSx$}wnt5W-XXO|hl|MQezOESag*LnEnqbbF~ zo7m?1iAoI>>M4Sa-fdP&-T`)>fu{Ly3xPBp#oNo5LaBZ!=iaiGZPk%pj`pi!9@?3l z@1i>s$q%R$GW=Xv9PXy{s*?Y)%K@aI; zD~f|($qhNj@rz`d;t)|yaIg5yJq<9nLPyrAeFoG{P!@8LMgA@a5J zoc&e<7G~j3(*M4ALTn>A*c!`gy7j&s-v2c=40RTtk`X;sRAKqxnj}3EAeF+#7ZEVkm-M5i@ za|ak)#YP4u=d$gTlIF`ntMR%;YiUg;b%I*hP&I#iA#m4rY|$F)-#g?;WyywlXB)p@hnJH3eRp@kn4q7JX?rHo3<^4Ikz^|(rU#eNZdk2a21ga= z>%G#;xGnb}Tym88d)}+7L^Z)TvdnQZ8j>Yea|5t1dM_Yn$ZI`E`GHTb6`$Tr0=b;> z$=_+v6>09-4>>rX-s{PWw@I&+jYTqZL%4VgW&;}b{Em+q73}6k`XH*lr>*_023C2h ziPn*C={QQXR7su!H<}1F@MAlZtM=doRV4LPrNtJ=G4i^H?rXV73c8QvR8Kfu>npJj zPv4O%NW4}+(yeC8#<3ZzA+r2#>$#Nl?qL~i7X39{<*!BVL`&Z-bv?M7Zx|md`ZsqQ zG^wG97alZisf{sq>$RC|YaQjSn>h(cVM%=BrEe}%DHC7MEIxZVVw6h5XVr}TE%Q~+ zOR8Q4lXjuAGBefbPCjDQeBT^{G7H_$fBV`dc4u|BOe?9M21}}MH&s5JeC96gl&bI7 z5M2=T#^@qfO!6zgT(ZvNTi?-|nGLh5^+6vR#oL2f7)NHScq_>aP z9XF%B|@v4@D0T>l6?naOP(4cJq!#?a3<6rkO`>NYbBb z`VN{XBsA`PPI@BjFO5|G66dtEo^E)qp0T>wXZ;84z&J|a>rVCH1iGGTW_upTparX& zjiBq@E^{bQZ=wKT&iwvLMEpr{AM=}&g+ZC>)wY1Y#W5R;L+JNI*`HePC1FLHMt9c@ z$L+MXsZI3(pO|Y%;8QUKH~u*8)A-{a-n{ugcD|?gX4Y)&Vw5dZRP@)Bkoh6eDhnL% z#lSH!5(jbq1{~i>kBEBsWz1d&jB{!JH&?A^u|0l#0(n!39@H2eQ(CV}0yij6{!@-L&Mo)MJuyiQmjUZS>y{La0Z%?-cX96KQ7_#0Tu+_9))HP$Mznmu~; z+6xN;XCg-ZeQnmvjUA+>e(zFnv5yqfNPULFhX>Y=PPzZaYmasx=dpwed2%(^7lvV> z53zgp={$#Y&aIib`;dTThO}z0AL9)}(fd#KZ+l7*j*|+efXD)ZK*8)XTEgmJc&*m~ zC$G!5czPUCEF`F|elW5HINpmyapNiBf~(Za-66O3EbH;O$}A@}SlYEML-f1nY>eIb z5h07k*D$GFhcrQDgBD`nQ2-HwaiD9QrC!|4SP3H<9?}<0&MZw@PdS1Ud6#C;*7i5w z^Zs0BlXHH1kJGel7$JfYj1#^$@Mb2?Dr{Eo7PRq;V#Z%~(8 zR!bqmnppUG`zrCZ0U@-?t#kq8uMsHcFj7!e|>p5=IaUsV_L zo5Ob|912>M`KFDOX0B8Uc8CiAN>q*t{-QT;kIF zel-_lIApPRtM)36!RwLy_jix(uJD^){y8IfN$cto@x}M7pp5okUG<9@uMsve~BiCC5>kEGYd`Jo5nn4lE+lX_>2|3g1hg@a~Q zx1gXHsrf5B<1u*tkh09^PV0yJCvl6}y&G{2&n3PmN9ShIVUGvyRS*yc%!5xkw^W_h96j3SV9~E4=z#2!g8^h8{_v)C$L& zAkc+waR-GU?F+zd+tt9~*{qIsVje-_$y9}Z4yflPa^|y>f-U$bC`5gjVUOKZqq@oW z2acUZe~4DhNl!G|6rj~Q-tHg3GJpT$ioBgwFf8uqv+sLqb7%NIUq1(obd)AK2^IYX@u)MQb9C}=Py#;uCJ zbLn`H(>PEWD$7IcWR$mKy%un;sTKwtst1?elRYYm-dyB{77s(o=sP*xIadSEWZ)CN^IzU znS|tae%{dV!j_SsFRb7FW?8=X66Mql^c}Lb`9*ZX>te(c3F=+B*Jk|AAdi~A)lYgO zv*$n8w~cKiSdY7+y5&B+>mFxqcXLZM)9QUL@||PkmtbvGr!0%di9t8t<4Q0bXf^)1 zWqw_&vlrjaMiOHM6Tv-bR0_MrsB4@@n8ua#QOSp6 zB?bU}mw>k=tHGSTSYtp)1=eJK5K|o4DuBqZafAVZ-SREu{MNMp4dm59-wc(>AGNRHE{;Nh*Sm~h^f~i` zdVB(AhTc&G=ALsw!bk>zmr)v^T7F##&n;ppyB{C5Y;^n6%@Em!o7`?aFN3 zxsDnAUpR{W*@Gx`4Q$k<-WxH?DNr8}O1%KcZDs*ZRe`R|Q7T#@vlZ0Gv43!q-%no0 zDpuH48kgBF4Sw<9LZ>uUo8077l6JN*6qPY|PCjmeWqka0mV@HlI=MEboZtOzTb^i# z0DH*u1rO>#{6p(!W)k_Ii>3~x7`Ch#m>$q1Iz6})YCo$bM*ni|zAbO`z@l zBEhd+X}W1(WHJo%fq_l+MY z;R959vnlJ?#6!jvl-cI=Y^UoxlL104VH}@t z+quNxQb?_qCk0`!of?wA4s0Y`|3w@vk@2KB$; z=8B8or~Qgdy)8 zw4$5#{e9Xh&MBbl=$jL_*gwlx7Jt#o9>S1+Y1FrGLL(-}jNmnk2I@NW;F~o5>G$L= zsYDM4)1Dz)eJ=nk8X>D-qRinn^`6;*(%&a~8()q&vy9j4D_~!3`V&OGQfcqeFx=8> z67_z9DEbmQu>_bDfzcCT-;a-SnWh)O88&r!Rz`cfVl&tn@l&Pcxs<11$4lH%y4S-# z7+NUQn&w)y zYQw^teZ^(=!8`NRG}&$RIV+O4}aGF{yy zSz^x0-X+gal_(qh@NNZWl;^jC2hGUBV~~1_yUoeWkusNw$Md9tBXj*B$^dMr{X2?L zB4}l{JDHytu-RkJHDR&$us%38N@=)`)kL~L7z@WaW<5#YF#O_@GC>t!F7%dy<5%Ev zO?!`0`$W0ojdK4NrN7XjH4}X$^LkWtB?JcfVX1B2AGrPA=mx{E>6nfx6Qa}UV}t_H z*ehLtg&154UeF8j*V>rD@osI=FIeFUV)lP|fJ76R<^>j=_RSu=9x}QO>cTjnmyO2i zVl^~_FbM~a;u-l?OG>%LFoyl%PYJO&=fur#IZj5`8=bS{lR$`Hzr*V&R=$xg5s~5< z{^g)gXQ}j(akwH-OEv`jD%c2-kfD6=BQ40(Xx~_QR7T|OPNNc}r=5ZX#k42dw100? z-#J-oE3`9#q0m^t+GcR?q><$Pi69^0d*%Bv$@{B@$N{sN$o(n>tdUn0Q?20a6< zN^%nj52iwz(!!7?1TL$mwj=N;xV;uDFQ5-MimneYIe+@oiZZY6Ec0Ax+0!!1xa#%z zuGA7*iQKTOVOcZ^;>v?vdl%z#ZF`VwB+hH1mPxxjA5V#$!{0^d@-pFA!S~t?z-6wD z!KxCcw;RDaH=EpFEw{L_ytlx_R(%R(KcBIuBI=gul)xWjrjE*rJT16ppaKe7m4($QYe_~NMUvjqX)kKt-K=Axe{fc~(1l)4uf|*em_Lxao2?2L0NV90KNrh; z0z4VRIJ!SXa2&0P=(E#wK)cknFW>Y(y^Zj#CD7zyf;5!cA5Hb_rSV&)zqk>i#^w{q zUSpsjs-SaqQB=OBKmY4yd}`Qy0~T7qYHjPGU{ozp}|&k8AfpBUO34mmnZD9NJ3I@2A{0q0bByO|6Ra;jkDjbv|~PA8yGyVYz@p1RvX zh3BbD8UwB7)CWE)M#a!!lF^grEXz#ru&zgNQ@f4=@l4qB-ebomWKG?SS0zi>b3}2j zv}3&%_=kIx<1snIaDtvHklb6Ri^g$vpukkoI{&;=6BEwIXKe)_^M+QHOK8ldyNgvS z&$XQV=bzpSS!S;whl^viIEgQh%^<iY`mi!~ zua~_dH1wUCup>Ztm1HjsjU$UC+RU}28E}Wf&oW3xb3fy)4sPA^jJ37vr`d!9FH^ch z!FgC;r~rYcwgCREG8EdeTY1~4T0>oufVuY?yeJ(8b8km^Vn#j zl;-MKx9LiBywcwq!8FU}huxHZxAVSwfozx`zph@exE_(}%x7428!ypW#ZNNIIuWF` z>hW#%i1QZ9-Jcm*6bQJ<5E?R7eb4!Q1k0yqBRXg%$zNggq0=^k;eRxBziYAJL47NU z-obv;ehY06ej1m|iIvYB=qdHhPl=8k1Q7<=-BS!xQrMsaE1@(9*cggQt3`2%FT<>P zN}?urWPfRw9x{==$3%LbRCuLtm9MX(U_nPc`y=02b6IMtkf z7Hmt68Thq73%j*GGhV>O%+HEQ?N529kNLp+y3>IMB?{l{YS6lcN2NBr^soJSkEas) zDK!Xsor|px!Z1DJ;V^+59Cq7WK03jr zJt2+|$8lUo6<$&?(G-ccfKW{Tdi_iC|IEV^r3qOif=}+>zu1~g5-gvcVO-aEXy%- zZb&j=suOg}LXSgD^9`lP8a+Np9&nrgWi|)m^3Fc5wqJ!Pz|H?8(?if+yf1&hG+#3_ zb{@COxoZU7m4V8LbkF+!H0&Q^ski&I&hd+K7{PWbjBqEwVx|5-4L}K9>Bnk0y>Dl2 za;A|rg}ASK=|ZVLl0|NGbkg(-y@_OWRN0v)$ecsmTV;z-`5QJ?5B8*HEc%bt!ryg4 z-8X*E;MT7;ML!d_zMIYhkR0Zh|t*763+YvXK{)xI`^m6D7>KcBCAV#i7d;i(i1R8z=iG1xmnjQPCPvR*i)8#xofor#%!dV_Pn||$hY&@{>kK7`pv|wrY0-< zutm|L>wKzLxKpXU5%+a>` z!(*9wOtRA=a9NhP($Zfju5sue9E}x1_cPd|UU^?^OZ=U5T}w%&qxdkid+)kcsyM5i z#=T1$@4Lz6%MjQz<2ezGdG1 zyAT$bs$E6Xj5mod-$4LNMvD!v6HHU;$S!^N)njBWd!i8{=*Re*lAk~BZqlHNr^7rn zCA(i{pb?DuMk`h9#F&FFGLpcmecE{O4W+i#%se(mT`~3QL=S&Wb&%ma5VdMa})`X;#*Q8_n#`MXf3eJzQ>mzs>qqo7@9>N5(>6 zWRpX=L*5W4O_IiC3v#&&B6rb8K&`1yxtysp*5=}`hfL(fuTMVznJ?V7iZ$68_yT7( ziJ$0tOn8XI^0kCWzwrlLYnKf zMeue)aCU*a>`qbrzE`HjFLKiD&c2Z0p!y&pe{_T^Yn$)Ln6R^?K7qJdJjyb0C|6KA zBNML!;m?YB&s6ceoYmCOXr8EZn-v0QQ3Ph{uonNoV2 z>fjIqLoi?GAXJes3cF(eu%<&qcRgo9pO9hKuTv><&eO6gcyvO1amu@lV=QL0GC3=B zJfeIy$>*rDKyJc6;OR&Z@ZP)4-n%Q=Fosei-6jTq{XI(tj)e-0I<74l2fxrN{;C%e z3PPJeIFJ=n0j7Mx@FARt%xqBR<6uNWEJF2E8}1S+=#h7&)gJ?{FzHHfHaPO{_%@aL zW8Wx$x>Cu?Ey+mQ?TPs*=q{n=j^ja$#TCHi8$dT$?16M*G%7=bw>xcX+`cPxrk=AN zRG@fMVIa`wdG*(_2cOYqtdXXYs@{@%rz#}R^kwTt2ZqM;&h!oXSTuItHAo6;y-Qsu zeIs@IN8F5o^0pA|k8a#klUfK6*_)01g*2w`IcG|BV1}e@1f4-XL*4Lt7OdUqvMlN`mj#WQG3mw3qX9}Y1q!WW%dQY zOJn!~@MjW+w*;1*j|lSm=}n>*F~vdt-~>KHp@ydVrlx~KrDf;T4yr0C2am@e9>jln z7?2ia9}aYHr%#|JT*pWEk))pB+eyMs=I4`w^A`Y@^HDVuG^*zarh;>D)UpVjUf(_2 z7$PCJwPen|A~;s6VtwIh82c#k&tZYU7}sgpN?7Y4s9zCm6$oAcpJk~^ze%l}7nSeV zdC*r!F8n8Kht;_CFbe9A$#e%ydh)iu%<1WNJ?ZfBs!*J-SW#j1bnsNp763=^{M@eB zW=p@rV{l|gBqN<@NiRRtyxJPY(jnCR^6D1Os=XawJ)Gb;2%DxB%{Yy?`~f_8y=+P` z3>}6^j`3?#>0II?28BWMe{;Tk$C;r3;cQT=WAjw8XfRe}8!l36eL&q5p5g@_e2a5= z3fdk$^|Z<3u_sPUcCz&Z29^zcg+h&;JC{8K=eY>p!FP@P{;s5zLyx=bDE2QWg$lx3 zFsKTqgjGHkHy?lRi~t<}*rJXP?D}=_NbKTTjMbo|%&*0mbrDr6OCI}4LOXT{$mu^U zyxTdl+h;QN=Wr62dV+(=@CQwd?5p=Mw%)Q{_YyVj4F?#?STFRH0KUZE_^)Kji%F7c zO-<`mmC$6}FVW_f=9FhK-0xab4cBgcxfN;s;Vpnt6fZe`xECqNt0DQq%>|nbfJYfL zPJt@Q1GCH*TSlB7?Fw6qJX$vcU2FPIl}l|(1O>Ylk<9U~`6Z**Dhex+#URCrZ-%Rh zGET06u0pp;s8SyS)@IS1Kt3(rtvq&Q)oW!N>D-K!Gf21P392xx)H4hgvlXlky{m2+ zUBJ>oWYYI48zPa>PMAEdB0i{GlZc)45AG&te z-(AG)1KQI1ziKt(fd57biIVmb}%oz|0$#DQ)S550_-iXL2 zNf7#?2Z?w4dQWDfcO+&i>e32A;&!2jrL-V}aAac(r=Eae1z(AU`GNU2ioc8bztDZL zND!2*>queiW#Ve=NVZGCRJ)w?hwlQww53Cp=Q<@_ zfc((g9gYz*7c;udK*MxLX|J6@DR`7Qbj&82bd2JflnovJy6{*3i3#%}U)jy4!Q|W` zOR)hIyXHROmuaE$6cmx>U}_E5#kh5AI&?Wu`GXdWSwq6~rXH#5l0pp~DvQ8G zg@Q64{HbI$zP=Q{-RZ#6%2lrGvr-16LJvEmjfauLA9=SI>Jy(gguzm2SaGI`5xFy6-_N4|(d0^wWG)8}|`qO09{)jej-rn-H^< z+!Mx<9WhI0pV0`-umnb@{4lSDrQv+p*G@^G3IiO@1Eup+hV!S#61*Nd9wqmRhOv(F zKnnyeslc|Ra~VoViM%bPU6*T=^t;nPqUKsQi*~4c9ox$M6`_njkHG)fHqPkcWsXOf zGk!EnFrN~s$=Xnrve5=KyBEB^lTuSrC#k${s?qS>rmv%dd3Cv~Enr0Yt}M-OG%C%f zjPYrg`c6E~eg#Wqe*3RBqJR^4d-n!qA9DE$ytwn2^C(w*yvi?bch&lH(*6=CFkRxb zGC{c1MKg9~CZM>;(?;g=ivkE|nw5if|F{WE1Ik`)_?jUpi9Ef~L3K&;&oP!c>73)& z%kxBRd+f`;`BspV50hxL8GVnvD8BpU8~bZhyNc7(IOpn?Q;kHP+!@=Tw@tRJ$#3NUIKK6z z=mo0y1Sz*O}n3nEmphLsU9ZLeqX_)-o`S2a~OwD!w{=_XjGz zWZVj`HaC6v96;{3xqfMrJm*@Dya4=qO(fTmF14Cn00z-OObAi-JT@!}h?ndUB(;5F zR{B46q1)b!v5%zcrE}D_BSp7>o`DA1;=j(Jc`710k{Y3$Rl!Klh5tP z|L@BGlBcu{6d);lflICycf)9it_wRawW6?^_MJZ%_axp!Awg1R;(zJYp?Z{3JxTIL z8nbEtX}fj#V)}pEjDNEo|Ix8>^{Dy+kXA$-xBxii#fJS^)qH;VGjs;H@3+hY+<8G@ zKL6mDeEJpL};G7j&dNi+!up zMfu);yN^jKJl;7ar2m-gu=F7sJZb}8Eq9($G^~&oZ_L`IEr2gL`+R;jXQJRv_4i?F z$m}C+4e$74dWZSZRhrK$QL7=6U6$xq&poq}(gGc!ZvN6|@0y)CXW0h)uXH0yA{T`i zhI2Zi004;}Bvni@E3wZqs%wd5!uxRe(YCN8h+DL&F^y8f*-TRX>5K257}O+x0b{~y z4Z4%#(C?QLpGp7GyxV6xP6BGyS8Sq(IW~|-c|xtvxLod?O8(8Hh_Upnf2f5Z4+NY{ z_Bqr@#39(26NTyt?J=zeB?jGSvItno>T^{@E!oU zLPwW5NH?*0Jj7?$F70*Guh2JDCn|#JogPEwoltLB7Pdi7L63f+`{luvS7gNhcS8qXM#OWD{G+su9|L z0Vt;1?vy0M4?Ru(shMlzWpC)9h?zkEE44fJOo7C2|&o`NG; zyrf`Sc8SyR2PxVD)Os()Hajl>oHYjM}#8| zUxu-7Bf#w1#{EDoc50e3z0mtdyMKn)z&7CbmMQ5=H;ckDr>K)8ykESzqdtvfp8h5BVFMyCGInk<%Z|D}_oo*G&U$(VrU6P;NQIb^PYqG*jS(St2ZWP=R z*|@^Z)ID1R?>l-ht8rW7gEXM5v`ug2V&(Ew z@W=ZRRX~zE#m*CN}<$vhrEx;-$flZ@eT_ zIZ9PJE&vG@OOjFF0|n@2g$YVS_I@^Cj|RyQQ7eg;3*4j`+e?AeePoTWQGXNebY^2Dv8XRZ+4HNiwmb9^VK!>#56+-x ziGQjNWcN%#`pxXj3$TGdcdtF?`zMC^BJV|gCDX_w1pm%vZCn0#w)KBHt*5-0`Tx(! zY;OZ2^8}oUns^25!vxIm!==%lT%j2a73q&$HhyZaeCOv4O6iLh{HbPsnH9fI@5)&g z*Y-kW>H$Ja&qoF8$RW;CyiB%i60(iH>s)G#CC~pz-Kr$eldY z7UNr=QU6f-7+g1!9bovG#$-c@JVfVT5IPykg|Dy1PjpUV1B%OS+<|Nr$hH?|>GWlR ztrHbqieR}wDGzS>zVk%CI6j(>y$ZtKVcFk-ZeiRA#W5`r`1xs%S~stGQANQr*u%jq z?Fo6h#wBb2^3~<~q#}~`Q_H6*&_&@dO;;X<9IwdkxUICuqo|4IWj~2-Jz3?;R|Wnm zU2}Rc?ROdiGyp^ILUcr|e_9Plbv%1>e@n(Z`)%u6$&Cu&kqH4+iEcgTQX4Se^;kl}eTDI{EOTQ~{@{Ks0zoAcv3+ z=Ec}VX;#D(l<|l1Pgd?Ag&)BSJ_J2tcVXCXDTvqY?=)^t<~Ul)I2By{-Ofk?uLAiX z^^9e|n(ZUPI1`GOOQ)m-?WVjO!#!Cv7_J(5`0i49_o~cqIH~i=TAU@40>#^_tlW>A zT{DFtYP*@AJ$RGy+n-Ed0KRRcsz zD!odNol36&Cjs^G=)@p~T!&;{v~ei><(@kNDLYM|e;{36x^eqp{Q$6#q{FM$fRn_B z*%R{%5e5@lVlok5h-`f?k7FwpKSN3ia>Z&;sXU+H7U}_sg_$hnYoL72{#aGCmF4E^ z#4xvH4&vqS76Nx>4=Z?FQFW7t%sc+gK3@%-zV?bCo8&Z%_+z&$1Jk4ud4vp0hvC-% zp&x8WP@<1UE;ZpdF;o1tOVK}V7+H;L5H~nRY|_-P=7LY111usgsO!HHjxVtSqy^>FpcwqX~6?6&Ljv zAd0X^9)52l5q~k!#Hkpgt2JtGe;5jlT~_!HlkJyP9fqwU{se`4-v|w_4h8vs2iDT^ z-Jd9i1TVyiVl=+Q?hG-j(=0Bk$eNw5uaZZi&$}-GwHJV$hH^vz{&H6i)5gAg?kFhv z@zw1|Mc+ooXstKXca`16EIeNM-d^CtJ-h8r`4@jy9=7%yB_B4oi4{D%Gio1&KRt>= z5iL)Z23JO@w$kg*Z?+I1TWnxEkARcN9E~|qkZ|@w(C5b+rM5R&UnU|qmyyY&9F<;^ zQ+|>zISS(0-aG-E*%4JwO%Y0yT5Rz9#|~3}q*c0?u4Z&+sO2EfikfHDqI;{PYVzmM z9xBSswq+W=^d78xPZLd@vAVQ+v;8rJxA5*Kq1Ekfk$wmk6BR3a3T0=H z72WM}Z5l3|!B-OH3`$N1%N%i_cuwCIxnzVJm-*6`^n6F=RgL|1c_T?z!!_#OUTxy@ zEv*m4OhPthB!6X;M+aLKXA{^10VlwdBW*&2;UuRbTZ2IQ?%9+4c`)CdMW7(NV-xJC()#`!G+;yryUm7E+I9{v0rLg2#1Cpv~@_+6WGlMp(10V*T zsiJwTDE~0q5>N4I4#X_f%?T}8Jyj8Dx0r>(6?Iky4th(>panSu!HyL-ONPgmuBy{N(fh0w$z(+W5$u? z+EE4*&HSJ;HVmKdRgHsg;J5Y!4Y5j+TQfl>nLNa4@_k0FZ6xaDBv@g|$RmWG^V##S zAUa1^SK5@+?@y~1s{{xtJBb8|E*eP>f4eKErVZ0ligxxR3#K$sdef-}TH&dzdxrTu z29~KRsdwhB?WP-r*v_YcS4q;V?Hoj16a3jxW~nvY@@~UKVbi$d)Xc2p3!mrXx-zvs znz+txadge_`USwpRQM+FGxMoI=x}prn_~!cHBk$!&MXLj(Ov!U8G&QBSS!PNL!Q?rQNFSSL^F13_ED0EfIdd?XQj)`RS8f=KFD=}6 zNg@=bg5etik6R)8oeM*8>wAn@>tXJ&sOG>9yWq|x!O26#=>;qNZeH1_Kh?GmM5>`k zw|Pd-a3DIx{O#=}`M?+70n@~2vf-hFfS=~JnAh*4^m^VUO&0O! zgoG!V-UfCWhZtaC=z}gTWkMj14ztVx%9n^j*fbZMP){dCb7lsYfM_Z>*59abDJxem z8aCX{56biEm(sm&UIY1|MHxCmp43ak+w5UAP}GFt?rU}$`hKu#;v+1##mvG^+?~dc zS~-Y^cJXUq$O@EzB0d- zw?D8i_+ozwR}S@4967`Ty_h38LxCoI9_ZfWpaOVW58CycP)#j?>ZCFQ3gzhb2=r5u zEc({aSbfl0V1tmViT-2z4$5NxAul!ffo9JP!C0=+^4r<@ef) z-&hcPl+oC2@d`zwz%o@4&NC;M&Hy%mrX>nH=8#?tF>=Pm~Y; zXNmD*mK?mJ5{S_&Ve>7Rs^nG>e$$Q^g`Gc?y|NY9{6ENh@1Umsc3%{v3y4ys2B|7d zx>8j-h!mwa0g)1lNDB}M(mMhI3eu!YlNy0Qq<85cp(YfiCrF79;#t4F&)xT)_xJ8| z-+SlYIWuSf#js`#WHR6NtmpZ($7DKl`Pyd}o?!Mo8V(`YReomC$Oe?2@+RO*3hUKk|<2L+USNXI6n+6ikp*X92PyuvMO ziuL<2P~TbHe?8w0p~-HX>Gal0bFFwz|2pq}lGAyUfi-@{$F z%yV;3Dh_WXCU{3X4s$2fsYx0$#}jn_vAs&d1)s#?@_T`P4XTQVefl)-0=jbaPG(rd z{9XO9sLQqe5@Gh#u1~KgnC$~at-is)px+8!iTNk@@G*cnTY8*o?C8H!7$GsMNR%|% zGYJjYtIQHRI9Kp7*7#?024yy#7|`yyPG0bDiSaS_Oh#R33l~G$1}*8dGYec|FSyjL zYoU)Qh>e7+GBpNs2k1}9275g4X#DwXPO$wQq8aqUKTwQ=2y58=qH&(n?cNPl(;x}y ztqGk4*d(Yj1{{kZ^#&y&yzL9-cPtyW(bYCbB9)5ya?2x@l!;3rNzfVB50p33Np;~L zB?!#&=2?$s9MT`*D0A=5Ks72{O;K9$o{gh23|DPNgE!nvPbT=6IW5ZkwUW(gQ!=7a zg!zwQDoC8yo@y!h4+$yTYDa#7n^D!cYu%lW*(-%5-}TEfq^!40UoHG@er1y(?$*FSx|3Vi0GlXuNfr?-Z;4rkX7!aHL<{VU*67A?sW4EV&?ZZZ1pBAr-M)H=J zfXdSt!AHg*8H00U*x)F^251A3b^s_Bd5mTz1sLk99c*qLRMBI$^=&adJWMn5(K*!? zlyY+v_r;hhNwC%h{3GFp+(BUjCJQ4X6GR@uVFpfH@K9kY2i8gzN90u7@yzZ>jvdujN+fZ_d?_Wa9pX6j7uxhU@I_N0~#-siT3 z<3NDz<|tu|9Lf-=wcrR2dD1hbwZ6@f=q@S1T9G#EuRmU(;@f@UL!qMkk|V2Em1BE_ z?#M3EEUO1^_n@P7+M_zG|4bkCsqG$7tnhIsL2(5HnaiEH&)vW0_=iMTAD2$-SG@`p z`@W#x#E!6kc;xI<$N!=>slKb+x~P%0akepFatXS$rH}zn5=A(B@Y@9LFrUaxJm8{$ zmY>p3P)VfDc`hF#kWB&MN@9u&&^uU`<&#)N3b_3X;{1hDp4{(q zr4JwWmNw~DrO_!p?X+5W#cI5osc(wWI8^T=q)jhr(4aX-LT++@?>Jd=!JXEQeymaZSD4U#Dbqp$>uZJxN-1;^+#q=o_Q0Za&<~cn-U+y zA88mpStpm@>;>Q(!CK-1kqf6El@*OJLc^{>A~epgG!(l@yDbZ|p-O=6aM1OY>ST7W z@vZ6_cMQb8cZ1xBJ@G>ejv)@fomE(%PaBb~JZvr4#SR|qY-%Co#mf1ET6Tb;wUYv8 zv0|&D^)%ng)5N_TD&`U%errfo-HS8bWC+*8`lCjH@hkrI@oa})cv}QYtoDsP&D6#c z64tzYyhISaC=w67?$SX$QGFyh&D3QpJl(XDbSfk>EhY;V(W9+aeyjNiEWo;7`UuW= zF_yLYyELBUTn>gx=;Dz<;(S%v2-ke7e=4lRiXk*g&9_jWzRP8wq{>#AYFdbSk~jB_ zYmm{$uBum6UC{{A>PDih)bJTNru%apH)VKurhT~sdQ}8-o@VbwRZZ;R$OO|YtJ_-M zzo6Mr{ZP5FZ)%N>`C;-OabBdyKnj%(a8~7U0mQxw7*NTX<|577h|~FpJ}@@et{0%G zJ3=c%{uOfi-|v9|savqW9rd|!&_9{R|1%0z#<|jnlqiQ zp7+%=76s4LY#sDM3%g6ebS7RJ4A! zSIdhFNK!IeTXHkF1{pT_GIPxSuq@~0#!w$Z)I~wSfDnx1teXy<*F`rVI9)AvHq1fAlpbf{nT(r$yA(4efK-(upGS;OwTyj=0WL*q$~KzOIIc-5n;*cc)k55OsJ zlz_0q`OL7NH_!|OaU}^rAfWqe`zF~q44tI{Fwo;_T_C(Q1bE$KzJ+y*LjeUrK}@77 z_6ru^n8ZZJ$_p4C0I4z7c%Y$rz(5T{d!u1QOW-cL7MvC_oChd`U26d8Lujm3`52~5 zMP&EEq+F0I-2j@gzw*d_yJ``ne)NLQs3B?qshvXmhvaY)arVFdlNO!4neiPBIEld* zRsYO-v(4dOh{^TDpf;B^GTG6YXU@sm>kmyPrlU=bACO=5wv_;)@=Q;om5eYTik;?A zVQgukT-K-WDtK|;SIIHg`T5JAX*=DcqanYlUBv;^Wyd1C>e|5pLD0(;Z4f&)lKs#F zK6zY5Jjr{^IJ=c%Wvp(@!+W#$nW}=_*6EqaabMOaD;?)RSAGo{CVnkZ;~S`fpI=#= z_{p!maVrU7vtI+gJ%U9Y9NXVFoLN6xy=E?R!1_t1p|4!7pR$tU(y8exNu^YT_hqco zx=AbvbW6kE_AZt!j>rQh)5OVsP8f?`9@#d-x!WT^WzyU-%mi;05rD&#d8z20&)iYxi z!~#{RpKe@I>3z_FYs)f>Wz_xuL=t` zKTgTJ))L~_W$D{mS`pOi5Y95pMc2Uil7;1gv;Pf~2QPRpxsc_P)LkwTz>?G!xf5v6 zUJc&#grbMr1H>_e3cwOFWU1V?Xgb5m7;zgWt>)z&C4C)?Y49E9ga?NqmSJlU@y>FTd)g1E!Uj_w4=sl zWhtE+g@2cCl5I1f*j6PH2|l<}j9n5!tQzw0R0Bs9*OdFP4~Ex=%7Svi0l}iggfqo(3RMEw0xvklQ831ayB^Oo|rq7Zn@O!`yxZZZCC2jNw!h1M^M*>zv%7=;tvy z_>|HsEk4GY%x+M&Q5t7Cu=CgVBhpNJA#iyDe+#ygypFW~)?G zjVb5GPS;o}OU>_F-tUo;>SC3Eftk*j)@>RsF2<$jhB&Ia;40U+g9kiKY?_G9B zFCq9sb4V}PHR5;bTk{R47ruhibmhXA+7nh|_KZSG_10`e)}oOW;0Xv)xA8L|hjjOV zT8Sl?FN-be&T4;G@pvz?&jop^A zwe8(hcZj^_%5s=2e^6c2j7(Rq8!4shue5Jux=7901LUNu;cjbXig;8&viSPN${azjIrgcdpVRba>^eXw&&9ZNF|dbxL>ZL9Cc3Z`#Tec&p9j zN$`7KooZp12)2YF*oQ{-jW_vkcir+iRJeAKjbqU&Vmm11{UWro6o_6epNEt zo05#a@wB~XlV87u4YW;>##d+d`%k>ndmbeO_>faQ)f3zOXr<9A zDh9=UN!+9TL3cc`Wt-x0&iym!0vYKCqlF+%B7iKnYFJUm96|Xa6~;AdDuf@UuV1k) z=^-cOy!&4F9!bSOm6*s$4AvxnFQqJW+{enz8}e#)e>oueW}!lOm8qV4FTb>?6Fjl@ zK#RK*(;lF6QB7pW8xME4O7ASo;ghpNo6j0sQ{sx?dI^BVr=ez-I9B#mq6RA*oDn!* z!=fE`Co0;7Y@POCer`=A2TNp#JI!OyP*V2sqCgD@y{^$HnGU}xZaey1UdK!1P2C8N zda-`b@ryGe;Y1x@S8TnafxFU)kY%Y!sowVf0uIJ_%2-JU3MOuO-IJU8jA9pB;G-N& zTw~e2Mx3^zZ@jl3DvpP`&4CzYz36`0a($VsabLCJSZa(YD~8{CCC;J4=;F>SK)K|v zl(;aUK_9GtQPyzckJsR2Sm`-E)bu-)?io5+En^spsSct&!QGcd=xt}q5U-6@r_71 z|Et9Kb?|(LGJdTypGif~j-Y|{OPbAG7ScYZH=RVA4M&)N@a$l%s@Jxi-&_*VIFX9} zZg!a#GD^bo&qj>_mb1WfU4jHQCPLN4E3QKrKULNAWg4MhWoQ1AL#JNq$ zuYS7cKqb3thfOB%7PCz@jt-wxmu&>^WC;cLb8BU9F#jQ`OqT40Y>)&=Bol8#hB4Qh z8s@}}SG1=ZPPlIm2{ul-NBA9zInZ*{fr1{=rOYg)$o_cx|1NTar+?KTzHW;{5)WIu zP{QYbJ!O=M?it)z{H`xVX5$JcmxSlMZh1MOOTt)WuiL%05@iet(2eQmxiU=^uXHVh zH=f@3CZR=a)BcO)N@u1VoR{pd?)jTbe?QjHv2kr@sKzZ03!i#NCp~sKSN~borz<8yUyjSxa!@hLd@r(g*>E4;H|WBAbEM3cVhK8SHgO1ZuDvs1W?g`?EsP@CZ$>NMma zAUMOGqNAJE)&eUyV|@+iw>Yhy%+@9dW~6oaSKOx6kzS20h_keyRx~4x-6ojeR5d0V zIgv4{l+~M)(0l$Lm9>n%Tn6MNjS-s9ng_KGdT$;Yh1zqEdt@>3Se+_G7_y4~z!KJt~naJg=z_sYxbm(=Jg z4pdl)6QN3YwW9f-dCc(75Ygj- z1E8_HA&#q{Yxu4OOl`7R7k~#LTDhw%BeSi48TbWYXr)8Iv=(tJg_8G^=kEDSvV3J^ zlpeAG4z{Jq#X1LDHychH>;QK{A%0#F4; z9bV8QNCJ^&q5|K5kw~z|ir8*(DEG%5>r-_%pIB-T>z>CL+%DDEGSswpbf3JNB$zeg zpjKxXYl%?nR;|l>kD)!FhCm^|U~y1=^a)P1RW>t2aOOx(xBC1Y^BBjHtzwZoT+`S{ zFVmKJ)y=NCEJ;Z#qTCpl z*2p`iF}&Ma0X03HYfLih>9{@t)p9O@gfh@`at_px0~tik_>EtDxx(=q=9Elo0?)67 z$z{@!c0*0EqN?;j_#%-8J}of4HpO$NVbobAUy@NhZ7+LvmBv0p-aVD`uHW4k){?)5 zj$rYBNSJ^ZNFtA13vMj#MmTCaI9O3VdSO&j&CA}4yXsn{fk1=G_m76Q7Tn+08yJ0T zz5hMrC^Ri}+>Q(3R08~9Q4+?uR6||S33jYITdWy1eV&DIgFPCel}?11v!eCfX{|+IJ4Z%`RkpKmUW4Xg@W*$nFsM$x*!-RkO{*IMpPk;3`&A?Fj{=-#Ldv%!^a? zIJY09PP{fXB?+hImmyt!Y6gdOh2hFBsGtDa6gBo2hU2V7xXHi)8xH{d3D)?(up(bl zjLd({8%+juh#;K}Yt+DmZqAx$Qqy!|CCl9b0UvXj z|Gr`&fQKCKgE@#jOGLq<+!5;yDc=yS?B9-(-Us<`Hm5ibsTdgW7-r!A}Fxf$LY9hBb%8+~ zfZ3)wGEtkGLz^PQ_iSlnxt}L%f3Ga67^?rgel?KXx3-W8V;9jO!aGI0*Dcum<*TTz z*c65#$f|%i-ArNnJLH9N7JsD@{@OUzx3^}+_*1l9yb7q_wPN`JdN}9DK;w$a%$3QUGli?j zVn?Jql|+rTOOlRx0cfyNgyaGFwd}$_B$vth@{hQ&&O|nqzX0D{5=auu-U_xr$rXTX zS14g<9{{qo04zoS1#g_o!k2X`tb`*BD)I6?vn!6XGMvK+wBO~6^Grsol9($xqy`pj zw1?`RB!+9rkVgCuLdhW2@K6>A5<{8s3Eax$n%U}^BwO7yGh12Z+A2e%yghtQ`F(DVUwhb$GYCiUJ7MTqPhqJcW%vm7 zwx*n6|0Dv?i#_+Ksj3ct`88rdh7>Te|3B%b{~4e4PiU*!{|T2>y+V&K>dCt@%Zp=( zY_h6md)jm_e*2p<{JrWDWWYX;@XP$EA=S7lwi&^>ma<52#wNaP)h$iSX^6K9 z)#c1=t6Q&c_sb;a6K=X|Y+mv(jlY({!u%4o@&gvrcFgeFCi0U- z_eX-+lvpU=*2{u;=}Iq-48*f%v>}iA;M&X~154|Xj_+bI)eJHDmkF>q6;?>?Im46m z(m2d}zGjDhyx8$m8fS@Fw1+Z@4C$}EfWL8r2l)$nCmho%^mxVYd=3O@u^t%hq1n|d zTS;>G7^V3U$JZjkuT6Y3`P}J!xVHZfFki}R@yMe_T4l3&TvbZRxRY+Yl_ga>Q^XOI~E@y1akileVkc>ZXC zcS&hJ_MZQb=SDG^j~rwidxQ9)+E}|TCplFzOz>VvsD+YM`9sDn&O+i)lgIFjj`S9A zdiSje!JC&j4I)ug-3#V^2Ps)kwq6Y-$ks-?$0vF2nLAmoetAAsX#eeZMyY)S*OfBJ z8o>|eWsA$qWr%7#2gB$gnnMOsbLvxSS}w_y6EY*c#Vud@DFf$M>N6hQ{3X)X*8XrC zyz@J6a}y2F5UB}>wXhSZ(afb;|qmQm;ZB`ib!B`ottB*&8+1EC@R zaQPGA5e^=OFmqkKVCb|I+>iQ*SJVo*y{xQem}1s+ja zr!4JGN+Ui~RbhKyGuu^8tNN4p4+sBGQS4tyo|#e+M7^R`j@15QiC=dBOwz_4WQLh^ z7m2OwV1rH20pvC1Uxag3Bdn_r4;)H$X@I5X{L2|VmjrN3!vGh`i4M59Iz|m!^@>;h z9jx`g(_d`rhMf0V=J(ifLsBqT;LLa2RD`E%W%b_e&15oXpS;M^NV3Tdr+mdMElI{M z@XI>)Ka?N;)<~LFLz(dns+|%%mIU?BfD8r`qWAcr`^VXj*}+!eq#@G{hiI1oz3b#c zhbDSnwCzS&gDwROQw)G|$5ho-f&&jT!uC}bLo9svOqZ>!Vl{I_%W=9T?7G(0x{1`v z?ApqmEW<7ewh_VSR>V2W%r9pYML&lz3B!s9j9ac%uCr|XJtr^YP&0d8ua>P-u8WX; zUJkmX9||@5Oq}SD1zfK%*IE>V0rvREycod=(f&)gs5>;rxT-{U{X_c zkAwn^bKbI`!F345(TaLN z|KTLbi=^?_eCFL$xdy6dR+M4NwbmDjh#Q4aYrsMS+uPYW_hQ8(t;d~Pk{pxO>!(cn zv<1~~E(qQ7Uw+lRTJ?qG*^hQ?%EWjmfx$3dMJbky!3`QT>A*3>Q^Mmjhb4qH?@X&n0l_^giK%>|Wn{N0^UCNV$9b z{^qTHkBwkhBQ)-2QmS1P?+>b^DVR%bVylvZJBM8y&x~*MMBa42muEE-1U9DNO~2u( zqJ?tWHhAB5nuEPVx?{%!RmHyQUuEd3Wh8(;PmAp}v+4z zHmJXTzjnP#Cb>vhbma+tdu=uUH9@)XJLK?3C)oo8-m7RtN!bZqkasJ~9El|u(2t!| zDZC6x2hVZ7Vy96A{~E)@^n1mrUUzRy66rh5^eDxL>a{eUE7HpiKWc6c{zGze z@ooIjb+gmz%hJz5IimMo2!Ft*Rs#-GGu+Jv{JX5HJ9(hS#tF;i?eCTb9+VlEEFX9Y zJm9}Qp}|qO-1bdsy;+Cke9H3TtrGj19joh@{ijhU)~sOkcZBgXcoRInU#%V6+wpl0 zr{P8CQWNI|6>z5Go|SuL^Bqnjv5Uj1h}~-BFUP+|Sg}Ps>3^lai4uQ*LUTj!cq;7h zQND|uC)*z9Nl4N9p>}OfIDGC?6fW?=Kj5M-G zTHRxGPyu|IM7X%U3L}nKSPa#e5cAWK$-n#n{e90j>7iomENXk+!=tc08ROC~Bq#Z{ zQ$VMGW8+6ZC=QNfmO&;SZl?+-HU_%yiyc$%?m2p?^xTYgCe*V83zGTdR80b+;P6t(>@$NdI0?UlGm>nk$2VPD1f) zRQ;m{XtrpfX<7P!ZZQm!!Oa+-DdM$jl`!nZu-zts0oJdKK10B-#3>G-ACA<%eTHNk zJy#=`<)cwhiFD&;1t|-9=~0f5chiqQ2(eun*uDg6%CWymzY*95A>+KyvVu=pJKfW% z`YFdFJGUTNWq-Q8A zlnZOnNB`u+24^xdtK$<~tQCx@#P7XTpM7I(;3Jm)R%m(XbTm?fzm1Hs>M$3{2MO4_ zfi2PmwwcmPJi;EA7CD!>ojtdV1ee=<-@c!fY5u;tiT>_I_WmV@p`rLl#(@EL8l^@M zx5}*rd`{Q$9euEW-Rdy)QM(kdm24@kPY^P>{A?oC`+Blw0#lsywZpN2ONj=tFeZcz zPR*!(TFFH0v%dJkuD@xsoIffgqXeG)E6~u&Qj1x#BF)UdYvX-_q(J(8kb3CkiCK2m zo&nCvV5%T(d;3RU(5|-{V^*40h~~3T@dcR<5E8m{GpcSRIaJ`%ueImZpCEDfQo1am zoiF0rU~WsLXgAe%FV>y(A>{lwj&`q;Y&_CyH9rN!(r6HA=?Doh2K-VjJQX#Ayg8;{ z()8vdaaNC5G;*6UL#$T?eK8gmk2|r0f^h=tc}Ju_^OW%=K#b9g$te?vyRV1z%c-LT z>fYvDhwAAEm(t|rD1Tcw)?a(iy^#Iu)Bi+%(PltKc!&$=&NbDy>N4A$so}q~`1NPH z<*UfTwxnPb4Y)}xMVwE8&OAbDs>uvz9onI<2o2rdGOnAk7yS`UI zm-&FoJ>uXR-)+JVY)Zc^7v5_PY`}`o2iD5oYib}b&8a{nxH%2n-miNjfFkAOlHwu^ zdD~TVi2fl7uY$;90=h$v#Th+n~R_NmgUx+4>LM6t@)x@o4~Wnp*-w%3Pz_LS}P+AtmVaV_(Nc5!sFx|4GM zr7x8!sE9Q#)M#`yWud*(l`L17J-8J0^UR|oou#Hc&t?}kKe13GpFJ9`eS~j<|e&7%;De6Tt%`d4VkVexi zKTzZuw8VMx5KmS)iV^cco!1ssJS?;NNdBJZ(*&*VRrlXL+77)D-(Z zXH$&trO3L-3h)UvPF|Pb{cRlP`6iJK{?`4a-}^%per`O@=bk}*6>drX{^_@?bSt5V zch#{F?~F@1Ngq$Qm_<-|3Dr`zR9L%BAzB4%D(Ql%izWP&_gW1+q^uSn(8|;b+_wIZX24xEcK9n%rGAQaXsRdsF%pJdGlc8?7L_gXxKdFoTe4BR;PG_9T8lR> zb(xn@UD5nnHeng-oYZj}0(y$TcLIMx<$<4AUv3C2@I}^%ys?|)y}a37>wYmrALO(_ zi&rpAl!~2u^k5H++dliHKMm?(#Tpdn-bLkvh&+a06HoQ`s^T|^pEnzB&P~wbk-JUE zfg&@*l%JefP9koG@)7LXaF<+8GEods$5w56xprDHmi3>rl{LS~yfYuA`?+B>PsuMp zd3R0rsJR8(fS&6~y29<)d9utsv(N)$_|eQk%r7cUcz(94a(l|-d4|j%lG`_2#hTiu zy_M=d2FC7ckw^Tl#qMBEd+ee+gn&S=)r2ohPjD2C&1?xiW%Jv27CU`z>*8da)}^~T zi<1}@2l&Y*+a$OMF#JVul^aST@z}z5VL((6yEOZu)vqQy(|=-F%SQ|VO~@rtUs~as zK3u@!O}V;00EJL2BMz>R5qvP*W;$5FM89>CM~>q{fntScSZCHd!WRst{b)RWzZYo} zDlTvNd|VisOzn`9IL(BsP=$W8qv}wE`1C5+bzAHoFO~?z_K3q)#fRW#&X8lA&gZU zyXZ%FICH$DQvFaZiQ)dpi21MkJX3|lF%UJv2Aj|ms)VgNTiR@L=7Ib1WW!Oe9g`kM zqoZCYdkQAeoBQ`tbuu@nXuP5qlj@?ZE;0zT*s=b68iEEI$+(p+*M`aG!t;0SRP;5c z={Px7jru#C%cI0LAjPULJPtTTumQ*p_>x7p6Z|9#0G?mamEL&79!C*C*TXVP@B~)9u1J4>R zqp9}r6Ct}X>74Cq@B7#@XIS%D+V6;0$l)-C(#Em*UnkD39oR#iN=Q9sxSx3Y3pP1I zlyA@pp8eHQ!y6V1$~>R=fly;9c2DLbZ|se&R#rW+yU6nm;h_a_gY)>)d+tEg3~_$z z+4c=!$^^aX8uz?e1}UkH8jY4)w>;_?uUCa-*kveDoNGgE@UCS`TOCX@Vny+r7ft4< zACK!1X=BEI)N`4}*4_M;i!3C8Z}a?3iQA( zIF51^YwgM}8#(L_Izk=9i((no`AshrKPIttE(!6(wHz7m*W41l(scqD+a){R4Wy2o z?TYbIIUV96Wk&gpdGWF>$(4MNDCIEY| zQl|j^v9N%F=41N;@!G+B7BS`kU^&qIf2oSqv%~)Kae4jU2FV0CRq-IyZG=!l!nx-^M&b2(@&I+?}B1lWuY7&3If1p=8Y za4SHB5}<;g3W%-THsEs0yM&eRwj;Ip4hHrq>>dAjeblz)yQlRv6ID|b;HTWin$g5M z<^21{Gm;lMw*&9)SmZaJMkAe7X`$RWV*a@tBomMXW>XP_Y#F7S*3IK-kPQTl^R~bN ztz3nq(d%Ey7=!{CGWJy64wB^ZKvPP0PA$dul9L=+b6mU-(uH=`8m^c2k|i1!zYpHK ztibvvbtfBGVXl3V1ZBYFlR?FBr4co5nQLp}B_%GFGV+FWuBSww?#kgniMMg0G#i^E znkE~|0axuqP_6yDF<9eUFtpGV;mNB!(0FyzLc61WjP-a^yDiv*XGtg;Y#~$eA{^Az z_S;GL;kbwBqav1H8d?1eC_7UuvATB4GK0{{8kK}wo|jr?X+A7e2FeH#9Q`1-<@NH^ zh|s`Z1X%6YE`=%V1*~Ru?df&iAw0kb2G-EhYxU&uOziFX$s`)kSvHj{!Y_YI<@<&k zb|-(L`6;>V?a~{S{ML&!6*_QAA)Z0^JT%7}y<~+Uot4$8t$LPr{uqlGNbTX-de$6m zL91RVS}$runXz~W#kM9UebS8oRqTc;ZE&oyjBC=ITI>}{FC7NEpXti@GdOt&Hau`S z85pUPZPk1#*{?JsB7w?TONu&l^9ExFeY0ySst&Rq*S&3;ZH|&<4#8fXozsN{yTK;D zul^1s38RI9fMwo3z&kb(DqBRjJ3VI^(Q65nH<}kJE8CHmlQ*{3bDw>+?CN>aaL=VD zb`XAxFIwZ7Mu_*hHGv76nT55}oeO1mZ56G9Frs}|m~36UsO=uj@I`+c^5D~XDu9Zt z_J7gBCRH)=yvp5Jt~L{)w+dvD^sj|B2`36+L9u?WN-|7(cDpc!QTn{M%>|E}Q@b5K zKU)Pl7~M!R(Uw3YHb2T!xF}L}weji-Dot6lFkQHh^M%WuMf-Bn-fhy`vEWO?SgZq! z98bqpGc8~BLEs_Bzc^n~d7aDHK$RytxFIn{IQ3>8mN~Cuu%3r>PX`}%9LE#s5Kp83 zEqe%SErys3Ox2#O5+_=&t&^u8+18%RLX~jP*8E7qg8~#ptJktWSqAJo4OM+5-Uw;T zSzYT3r_SsoB=_gx@>1IkCjKfUcPlRn%~9Q zu5rP&?9s0$#%bt1?LQ>i<~=XGG8`9vF)-BO%K$y)%em*{N-SlAl}n=~E3W-|*v$wq zvz}wQc5RSGNmTtunQzR4cki#i{uM@26#`p>^^eiwb$_hJ#6S1oQ{d2T7;Ij~<&(f0}8-X^>+zN#ItLD`-xojcJR( zkePG7zF~t=bMJV(yMKrb$VHe9*wF)TY>9XzO=ISS5<|s;<$~DZ_REl>$3VW%w~MQq z;?)9e(Pqs#4pCC-Cb49Bl%m^6t`2GFgYccs$!(BFOpj5=2Y1n&)ZYbv16p z?XS%R%MP*F#n`!1m>{q~5zAo^wak8zM6ItE{Sr5e7AL>9ucLN*kK)n?ponu?T;!^d;T5|a*W}}lC9c^(*eoNu0VX8KFC9hi zu)Jn%wdoxo-7{Wqm84>vW3Tp-M=8)cdpne$I2w9s3i)D`Y>nJUX*TFePWe^^edIMu zw>fw{B*&zwdAiB`xtgeHT{oAJ zGWo1s!})mD(`mYf5mIwwaTA;n^F_y+&Ghy}{hodSPt>_CaSX|z!h(OIi__}1x=^#7 z+?S>LO#CjN9YV~3lH0l|(CRmQH|VBpXjS7SyYX$7lAjdsP6T(jowR8n87H z8VS1iKO{Ic4A7*TC3)>R!NJHq+SjgS7Ur^h#BTOBM6kI7I43-@d*p2cbi^mq0aoGl zKaw3`s{cP`ca{SWS%`3NC@-F-$Ym0$I1OZYI)Ip$m-^)+y!Jhmw9rc$dwY)3i=Q6I zP_h@Pg99?;qLc4CJ=2exvL1t!U19HvQK%d<~h)otYRH zhxAh?zP$U%VYTl=QFgRMR=fzhHg+rq%B_;_RI-yDQWoXsG+x!11FocucOcq+%+BaJ zTlDGSf47+P((#%6^`1O)LXOl>P#aJC7|xGyJ~^m#SnRFGub@t?A@HW)!y7737qWv3 zGC+vw346hEQ7GORBD2lIIng9RVerBsF{G?VKI<|n^VN8g~k6_D9L^@@?-5vr`kvoEc*m|v>-RE#R=72iy|Kl_n9@P~gt zzuVkuwRZhbyb*VX4&8kLx>~hkcAzXa!5m@K3vQvJxayj^)yi5%H-#7P5*WLStnl$a z^m6vz6sI>wy~5=*Uglu;&&9?pQwDBn2 zp$0UiCe-zJOa#JczKY(_K6^dt*P)905!)e%Z!p4@#XITvaT`AnUAEj!@t-lspdyQ6 z2dE>1Fo2M&_FIX_rxEuI<=kQ;(&M;PgzebzA9o(>luqq=yvT5PUuz)eb?FlJ?sKWn zhlfKYU{6&lf-gX&i;181oiW5h9rEsBS7LnOH1@Y42HikJ!|soW5yeM7D@rd*4_R2* zQs?t@5-13tXAyxIvEVLXpN|uv;^I9Tvj4H+sZDkBzK@UaJFS%q^uU1N3a$Z5un3Bi9o)H0{rU5GSHSejqJlFr zn#o6tbxxga(W>GTv#a~4Lz!n`4{J9CeM2R1rgdv6%-h5}bL+rKSlSpp?tIXsv{PZ} zDkq^~yx&7QCHu{gYVMy14r~wpHfEv!N8au~30ENf#{l5k{GFc+e~)h^VL7Y7-q{M0 zCeSfj!CZby0APn)Q84+dcl5oxsbC|UKI=`ZGIA1;! zMd0nx;w||sw%Pb0x;J|Msl{kexsFo#;NzUEmz`p77-m3Wtba&IR7400*#4N52vFBD zM?Oa5)6)Fnw$~@I3zUn(+h%X3n@t$2sAPW7yTVhC<(5(3IS`eSSjbu~A5oQ5rwsq8 zzYEn-6kKKt;hW^#Db|08S{@2krM7bO(Gu&Td11XMW2B_>gXL9J3g*~6xUE81-uTg; z1e2T=nRn=#|+{oia?gZG*2G3;|}7gZXozijcu%!5%oO` zr>TNTDGir@S9l8Ff2cjguFJ(JRhEW)bW(Y^6I6u~M8*hwaV$kG>icosB~|}b<;uTK zmu4!!m7eIG_#rGb1F&2{sj%`o7x{!h;piq0aKD1;>w+&+&-Lv^kNoH>wq3%vM?H94 zu>o0^Hdr-oTwVoCc9EgX(A)SzNAy?>>LkDH=c$|uGd3%!rimr_u5-xk!OaxAgpoD> z9gECWA+0-HufT^6K)v>!P*Ow#Tc!0haWAs4wOR&Ip(5i5t7=G@P0AxRxr9$@;vZ*c zUoMxArp%iv3O2zXCU|es4-3EZj>C-^&&sxwVQkyVw*XMTvvT3tLm`CGM%-X;Ca^VQz{^iq2>Oo{lHw&0%#82;~64Lh|;- zwmD-p%XZTYAcwC`p*3@ zAkqc^;`NvLYBPV59UYw9&3S}l=$4=(DB;8tO1IBg8*q-mp0ZRMi*Auoy37F20VgMk zh>bliY~uD^f2ty@gC1U?h1B*aMAi1te(}wKC@GiB_j;xlOG8J? z_$<6y*hn*Vek|H*@<#jy85jT3;!4%xzpWRxO+y-XkDVLqLU#PB-w5xAPN7)dI#H#E zN>qry#un-0Y}%KISjn-u$^RcS+%wD(o`=;PnyB4C4n5a%d~Z(-$|Z- zAf0S!CAo+hhh9SmFYGMIugZ7RGxgWI*h+oIM#IJWGPA=q3b$tcXZOnGo?hhoaW=JQ zgBL{#o{~;oLKvK+U}wwLR3njB8doAiK{f`R9UNX9vQ5V&AMEE0CYwa!e3Xa6EHVcK z4k;b1RP4z=)yRs*Ld?fPq)YEb^LHC=0%ec(4~_nu_*fQ^d!uh6Jcj z_)d<46$Fdna{;REFEhzJ=43lDf0B8nJ+$GQysA1>2 zP32UX5F$7#%eGvZ?@`B%nE{sTs8^_OU+#pY?56Tv|GF6AxNrE{V*xJ&WDcUIm)Y@t zMcvpXhA?2M7;97O!nS-+$?l0kDC^OsQy+HJ=B;X454iRzNe(3tA+GBkaFbtDw?cpp zOSAL(04P_1&9fb}_#z_5lTIS%l0U655JV`|?v`8CA zt2k!r>5{af;^Sc?-P6$9)Aa^Qo#ge>vD3Fb1@?@_>5h1&NzWOI&G8W$*qY{Scz2R? zH;5U?Ts!Y&sBn4datEb|ve=gU*k@Gw+i@`UBb#8eX29;piT*a(b+f8^ za&}n+d>vRYx)y(j#e`qOC`FfKwesu&VCk%3gMgpl56PD~Vx9!}_odpMQzDJ`9}=Nk z*fHXo-5CT}-|_!h%JY$Mf{Hv|x?kn??0;+TJfoUQ(>NYQ0qF!mT0kJmARrFKI1-8@ z0|t-)N-voKM{0<$lu#rT0gHg$0+ALe3PXnk7>JZaF-F?#bv-+0 zcAT@r*|YOy&wjf1-1~fa-*evQZO{MrkMg7E$4!ehk2IZ&3@k&r0?X;D_oEI$y2FTS z1GG(;zC`Wi&uAc?{V|3*MHC{S1W98x?%wnQ&>*l6rXmk_rC{yrn@V}W(P!(-Vccs@ zjG=8&3&sdg(2Qlx@vVBYB0D1hMu4I8hdJtbQixuhTqL1IqAu5<;eDX{{Qo0u>roG%YQ-O__zpRHbdNdlhfkKQg2Hx<@%26-|sGNVSjldx#WU!8jc=e z(XCQH9{+&XGr?2^IMMxG?-huCS4JY><~*ZL0*W6Q0L>!IHv0vn1qcFuA1SQ$2{X8| zhB~eNVpo~_sskq$QePNv!!|4qFL$K|R6Kb!TBG|~!*^vxOy_nX{-$)dYR>%9UTng6 z>US#sh0TjH8XW^rn=`Y|<+%hWHtANhAV8*YW*=PbD!d-Z$p=$^c5{0+eDV0Rx3ynQ zu2{Xaes~5XB+&Y46ZYQN|HVkct52}c6*FAN8NrsN#1Bjyo)65O-toDn>kLgKT`RrG zjLqkAOIlA+g6WV`+!xC16}T9wgH-OSq_y#UAs}-vHZ?Q6K*Z{w#hSqLkCr_pO%iPF z;r2Yqk!{IJtdSWzJ*9GbdGkj0CV}~RxP!BQ*tIq&vebRdXtMd9cBA%#_!v_^|YmV1bWMVWgD=}@ZS&_DNtTc<@8jkbE zuR(Vfx=NwM1_XrSb%w?Z0)cS12|$$KAF{hvsy}8I7}s$vpG?gFG}3m%!K7qcF2Q~@N|=6V+L8RK zce%1~P%jv{aUY!CdAaO5=e2l)MW}1$tD~sfgqIYl?Q_+v1X|(Tx|B1boU7AOmc7}T z>VKfoI!p3p73TYs#CvxWGWay*7Qc`e6ICl}mY^Fw%{7}_EA=f|G;lE)Q92S~Mevtf z8Rev2gTO#`gRuw?f4D8Lsf_&lsgFWz5Tdn+lE`%`aoUCtWKcx@Le#`-zZHYcv zzSLJR&ueSSPa0E{+LmFNP~a~%6mLcE8-fe7)}D7LfRoO>n5}Y=(bKsOKZFA{+Y;fn z#%~ZteeFLLv|5VWd2quRT41bSYYwslmzZM~SL=cj!}h!nJ`-%)8zP*S<^MI7d~e#Q z&I)oN7FIqPZcghf#>W`x?#f0X&cc%3BKKh@%C4z&C%Tn1QwKd??D`?(2BubZ4gF19UF%cd-dF7=Lz ztF~?A`J`_aZ?^LNY?Z}AGETS51+ys$vzjxcrFH@ zFqF1pZW`)vJ@S;;Q+4+vA1{xNUwyQzSY79LJ@mbuYn{fD%Y_W`{^jEr@Jw9=N{j`|M6A9Nsab965?a&S} zo;P7_ErPnMFtb{27W>ZPV3f+yHyT8-n{N$*N!!&`rUxpZ>|+el_tMW$+Q!Q8j1G(p zHlGelMw4nU8U!z8^ApL$lS8WMBQz?s4dWvxg~oeKBzxT%%N?7-XDlz&j@)5a{%pq0 z+}&R@3KwLklfaW$s!LPNmfJ`&#s^BLO=&%-aoovk)0|K8E6r6G3-dxpJ={7!){oD(u{dtWQvo=EzK`ab*@6DjGYG5rBb&Ud`BH%LOQPfkme?|M)eaDlY@ACItUbm zFgO?;A3f%rYHuhww0)#xdE~V7RXk%|@vDHNG;XU!6EE?X@U?r*%>r zxfboRN0PQxSuI_4h9XZ`bnkc5g1KuU22#8hW0foCRe?@oMi`QcdW9)9o(4C55IY|Nb~PE)&3?y!EYVJ6 zpPKTi;BTI)9RLeQ_hOA^Zy2g&xo33k^2Yh)A7;91StQi3`hpPXz<|t7F9@v|ItV$9 z(X&}auYTk8prPgp!kUYkdv~LDM|K9725$tRB(ao<^W1O`P9Qd7rekQrCum7rH{HJy zbB1nd!As3 z*00G$eEpUo?@$m0q>#+CO57+=fioc0v>2nf*qw?w(*}uDATO)te4V0^O8>l;2OLUr z6+LJ3k#z&E7=|JRF7(RZ(LE4LMMW{kw@et1(t7!}sOut0OamX~Ja1_-Sohml+sn7je>fl?YQQ|PXV0VGu3P?F J?>xVZ{T*J(oeKZ} literal 34806 zcmeFZ2Ut_xwl=!x2-2H?fC7R_Q=}*@fYL-nEcBw%iGV02NFWH(1q1{HK8hl}NpFEr z0*Lf3p(Y5@NvHura^v1-pYz}Cx6j$9Joo<3z5j%nz*;MFtvNH-SYy2JJ0|HnX&zv` zt*xgGP*4DXF8K!_p@BPE5N8JfFfafv0{}n|P=q-H)a1X&hX8!!3IJ4D6aW?Zo#NNF zEXu#_rJBp4`rF?WqQ4F#wE%4N6rxmfR20GhB^w158wCjo2$4TYL-CjL>!4qSf|81w zhL(>06aypqfNEBNl7fnglA4N!hMIh83V-r`fSQel{rpu8S`MQpbiy8-vajPl(u-)8 zw{qPd!irvd`XcBQ12@kZUcL)r;ukMnmXlXdynaLJ<}EF49bLWK4<0@;HZe6bx3#l( zaCCBZdFkop?E~?B75pY7G%P$KGCtvbVp8&ll+;hzpL25a@(T(pDyyn%YU}D7+S-vF zon2qMzYUL!j*U-zpG2WC3qKZ@elD-9;BzsVjHeh`|I>vuL!Mk}BouI(ih?|usMr7yK+yRVc@g;E zt(Fy#0A~aWydO6Ra+Zla;ts*tbToLUT`mkH zPx8par;V|wF5=y^1@V)Zfb%3^n6NbOH3L*r=;=w|wG-8zaTM~f%I10m`K*_rSe+m-7WPAJoHe`p( zByuXnk^rG>B45{jE(!S5y#OLy^b#V1Y(PWmLhApoU#tJ|eOPxN!HNWwBW%dy_ht_* zzexhB#R-4Rb<7>}?~#C$>xR~B>NK22|E(%S8vGxfdQ?M1E{OPT{Aa?S$?mu(chbPy zo?nWy)>$z8F5VSz8t<+cEwfc7+&sL#n|5L#U8O(eDt1xc_gn<`gz@>0jj1!Gw#TVNsks#&!3l(06Rk}D;MkxB({rtmcDXShM@e>YzpO^Y#x|I0!)V_ z_h{Z9)@sGej%415dRP3C&sA6sAnbi=Mh(Gi*LaQOSh(@~OpKki4Q_~HQ?kjN+_BX0 zu+8E}#zhZ$n*K;x7r%O5^E4R#MY3>?eUL(n^WLvYH}NOd!g}j>v%(1&Y5lh<1!?Gi z)`Cnm1F)|vr6k~c%0lLe)}GR#SaS`YY`sNt8%e-Z#~9ci&*<8ZRN!J+7F7TE3e+0w z9j1W5K;*2{`28+c)72&*8^otNs<0)CMa zYAcpg=)$r%HPxo5C$XUX(C0cQMTC}FMP>b{`?-rW(ywkxK6v*vOH#D_K@}fviw&ZP z%;2aV+=6O2?(0JJ9?Pq$!jUkg!2$U%3G}@W4!-C%T1r6cJLyHKK75K?fj&U9ut6># zjTqqM^S*Do7fX6wf6%L^XzpS>G=u8?Fd4XWK2gZ|eDd|jXIOz7g%`{2j;b*c^oE;- z%nA1qLvOA5;M=`Ef>2uZ1eLfBiK&RwxDJU9;Vy|z5%2q45u6lDawY`?Mf_D96{e}Q zG9RKa+fhtV`67SP|FLl8$N)~cl5)Nm719B>YrF(VaAeF#T*D}~&7CEPkCn*T2(0wj zs(o}x9*@82L9wmi#iKT-S7pWN z=)@$v)E)wa#vd{gfdS=nB%oER>KHZ&!>|O+<2YXp#K99(|MUUt;bQfH%Ok~@^^;UQq@POI+l>G*lBotmT;2<tGYal&Jgq9eIi^U~q69p~+&L;$?#l|w_D z0^<0racqnwn3fJ4eg_OsSADf6LVlp9~kN zc8l6FtCL@c1-q$OExfkP$FDN+cko$#7T2jewlBa@G+pA#*`8(eaEQ0AigA3s$p-zJ z{j@CZu;|}RA_dvi+R!5b;)*yQV&@5r1l;AAcw^E?9L?N*3FCpSyOS;LE?O0$1#lYu z_bQAu{6A+&%kJZ)=C@80H1KI3Au=5kuM%U@i~9Dxll97#5;2WbmG%|yHWh8X2YY!$ zCD?)>ROq|a3x4`#Y_l^O=($tQG3+G30x@0luAKjkQ3%bZ^qCj$G)vB}YGyeI7&qZ8 zw6M+5zVq&{_bZoCkJcTSZkpAj+HGT5+a+IJ51HOJ>`4H;Kur>%TFEP{p(pXG!6(uYMT)gQw$R*$>T zwMYa%K@0cHyZP$q)V;8_OlB3^Z0B$z5eq+l8^+ZP%g5S9b#MP@w4Zug-qO`NN^!(= ztVwXiJC{NqRAFZZ-oRx_oIp>p?@gM8D|xQ1#xgah<)u}bVHy>dZcScjq+ot{NmDTh zyjcQUFnkRK;lQm&A&79fMhPBVn?i7YESK}ENK<@(1vp>8wPg21ZR+@%!8<)lI8VZs zj@F@`nO#}S(M75~LlazXotgkXK-oqR?=aE&#c7Q$yechPvvYfUt3$s?*~*d;l5?%c zLt-OCVsD0FDn|6OLM_{|G1NH>Gxw(X($t!u=$G-uq2`OV1QDb53zc%O&f;92npCnu z2V6D;ij+%@{wTUCdFbA+bdhRbXFfK1S3pGLB!i$^3t|RULZ!0xXsZj>Qaj*k=Y}c9 z|M4J_{0>$pV@tzd|G`gd|1}39K57PM-1g)L0qFrh^_5wt=*8Wz^z^Lh`# z;&ikXYK!;qzyuf1-r5%wuC5)CJZItK#dNtNXIiIbvrjQIWywp7y#Rkm4&COCpM%X| z;qGwWLbb;I6Q@>~z-+o$SuP~fU*Wt`bps6S93G{@PHRj3GGL$6QYbO0Id>u#ud!e_ zSx)4#f+T2E4iQw*YZ3?sYnkw8Pe;9GjN}eC9V^B#G1{FY-G}rPz3y%SUMWPVF^{L9odx2ADvqbIB|z5Zk&;}pK^Wuex*zo8*cD>R1-(f-x5 z(N691$gZm2-MX5w?s{$?#m4#CI#c$!-k3#J%c)Jas|@o2w`L*9)}wL)ifsqA6$Uv@ z+m;57Zpde$`b`;rGVly@$CDvbPl|P$D*ojPs5=fmD=_{T$DA%*;w5SxQgx-3>8p>V z75g;3x}@P%3V#0|{JNz-(FcSyaise(KS6*5kRy6FlitMLx%RC?1;g;UeMLMuU^E~K zXR%2BBZ#HU5zEug$hg5u(7`ABiM^ip^X0pcB+ukmIIR;jp8CwNz3%IdnEQ-6y|g}O zj6oR_0fe1+?I|G>KF4?7_Z;G_VK&6$Y`bsKrQ>9t(`tU;)3JMh=z(SeDH@yzBV7nq zs6I}}f^RN-(Qx`!&3;nN^7+b&>R2_a>SD(24t6`V&VBu>l4jww6G)qg1L6$}DVDnx zvF${TA^m6>?dqqGYght@zi#XpzC1_OmtmNW1yGUU%{Jr*m5Il(DGBHvf!dMLf&nDx z$Rnff)sa31DO4%m!4erDi%%wp%w18t19*L}bRCmpae}Zpbk6bEzc*RXB!#?>(tQ%) zwh`6Hi&DBrhpaLgfnm!K4u(e;XJyD>i?SgJSc;6ro6J>;pK!YKkgbP69tqgdQWugn zvLQn+ZOtbspyg}`f&|cSfPO{hAhbBaM5;&t(TNQ2g!muJ!iKbQFd`cYMgrs>QtVvd+MOdvY)vK*BNd~_z(C9h2{`5m3Vsd49Kkv0JS#p? zt$H7&9P%jTd{aX_WL!(zi6VKw`kakxT?!3mesv>OT;;XUMy3oO(jo ze}xE^$*mcOD*9bhuT(wqWuJNye{nHli9vxqvs9nU*={Di$f>q8Jt)Z;{(5-Ly@+D)5UKe<`ed}$Z`nh*^)qm#r_Z?dj z)CA9B%VMNe^yZmlTIjTKUHd+745EWfdkU6&lS9!_(oCaQ54}rk>AWZ6-m%g@T825! z&>@MUcpq%i@ImXV(x%U5E{bKj_`KHS?WsAu1IA)tjs&=LV)eoG48^O%>$J2+OOLq# zp=3Fe73NyBOz2gp2u`MR!b--f9{b*WE9gh!lZXe8CZx+M^IV#Bzg5;`zEmj^2dNfj z)RjGJ#+J1r_@Iw*1Fy+`Clv&CbC7Tf$FDuSyY_oS8P(0j7=)_>JlI z9-1JoI{*cKQOjTASbIfHcEnBPnhes0(m9ibp!3r8g+s-cY6Atm$NHhV_H`o*k4#xk zhvR7<-utO?4LbIpfDTRE=MBRGB=E6?-8p#cLEW12lr8DpxW|$&%DrI)(7xx3lJ;fX&pe4It&wIw&#U-b6+ie~7kI#Jc*NC0wx6RO=W0!?vbu9}F`N2AATqeLzx5mHSXS~o7V(!j=~t}u59^Wp zfA(oawPHg@9C)P|uEs}3c-ZAjd^#@6_e>q@mZ&JBD6b`-LmzjXhE7JWGx z>GM^{!747uX9nlu<6Lv9nz^~>P^{B>^6!0KIQMz%KWbEvP@bkIOp8<$3Ex2VO&@W; z#!a=E`?N>%plc)H*D!u%c%3aT#yWQu5a!{~{U`b1ijqTWDSmaGkNHinuR@B$K~8B+ zvYU+2ENwgLHEOId2J^!MP4pnlNq*PozEI_-daG7G)yOkJQ=#60r{{!-19O~e+)!Cq zxDcztisDW4Fl{~v7#zX%qb3Pscq1#NmqV-k(Z-@yoKH~et20oUV`ZotPSTopG__NL zNd$So&8FogKlTm`@#1}>aG=P&WF&Xxb{Cu8W#QY;**?UjZ$*Oy;4(Pzw$!dLxcIyU z31G3rCnL8oeRnZy4~c^h{N!Z2Bo%Osrl+5~y{~Zh6&2vk3{tWJ?Pm^q7Wr8PbhS)m zVpFD9X8DJ_EG9;EzPrh7F&WrPO1L{oQEA96+^sP?+b3K)elggWf#QexL>q!f4;PHI zIR&NhRM8N7?cu`QBk)%eny{ z_mWvpQ%^Ae8b8YqksiBMaQ?0K{sG&a1a3AJ*Cxxta&F>Tg2flCPlpXFL3;u!c2s4) z`zol5U5|ZwWqVFJVOdM7Y?_wpC1UDI0~&^ zD3Q4{QnegCni{?LatQkhyHR_vU&QAt*L4GIrNT}igr4`HIjuF0q;AD8CmH|*Q`$pR3r+_ z7*DRDH*=gZ{>pc8+6lSV-NlmQUr_kmRiZP>&**8W*^{t)Nf*C;i?m?=_B>yyo&>mA z-QM$>O3S35o0=IY04pWMOYU6=I>T?K#Uy~{*7ik8=L~&JolLtRB=Hu2Cd?a#!nlYR zRt_5Sf_Bj`cUwvCb#p(kchb@$M}0rf^r2F5d+jHu&;LN3nu-Ard$~`REpVmpC)$4Q z$z25(I4STE5=wLOG{9imk0FsaHpSA<>+ycQW&C_tXzxj(4LLkp=nHF&7BU(3A_qc{ zcqZ-D*_92o?-9AP#SGp1uA1ML#y2m0I^WI08tU0|`I{nTQ)URrg^5ZdrXO?A!V_DF z!ozM4(VUbJu392dx853UWcV=#y9*%g?|a?1zd)h)j{P%L1pFi=K*3^pR-h?T!f~{E zt;u27N=d)*y0qNEV4CZcwQoiKYYwKS#qR~L>!Md0`{c=fBc#10b!ILE?q!DxC~>fL zj7|}_^HMv$Ls*Z^=|{}1=gZ5z#ha0#Mj#NaaJEq#A8=CEpAQh;VE)t-hG5 zRgF{mVr6(%N-DTepJtQ+cl+x%+morz>H5P3Ofs@QKoat8p~L1${fO&*Mhgj3=}m)~ zwpM3K&0IZrU1Ca2lJNEHXgySTA&|wQ@{huV|Do+vq|v{6X$7`*8JzLgKilbPPcZ17 zrs$^^(Drw^d=V&-x@O!wEYGNVw_j82OX7tnm3+I$?Xj9VB25lM>H;iLDuzXn5hSWC z%4?E3`KWKEV||f!bjK~n`SW)~Th8=5y4y(oh=mwk+qk3~9!(nO6qvPjphD)Mk-(-G zOL)M!Q5BjTg2I$L_~S~l?Ckp>t{DBIf)}5_DRg}jonbw(;`AwiUAc-p=RK1cSekF#pke$p91H5h-|eLP|v*ZKdRI@dyEvW&}I+#($lGJo*Vx>MaAJ^QR{o zeQlVDv^y^(q5e2Z(4if}3!#Usc-^ideuc9qeSvDP^vU&4MGIT&jE6$}Zj3jN}) zT?|p~#K^5*ArCAchC9MWUZf(=adZL58F3`hOb*Cei68%kJpVt<{%6LvUWOOyh;PU7 z+=;q{4_+woLmqi@NFlvZxp`iG40j5@R!F}5x_VZkB9!XfRo+(lrK-O$X@6tXD73g% z5X{YIRs&SQGsdf%Qgm%c7j?$GioVR*SL*bSQUo}cing-03fjF34&*X>PsP)cL4i|5 zx_UYZo~bnx)t&W=Sn>)TA4aL!)zznC-?g7vOb-_Ty3Yw2;aFnf*RWf+n(+QuYjmj~ zQLJ_xe&_3u<*M$qYIXb<7T3qGSUu!hmaOt`NklCmB2dpzY;!7&OGWs^jq7^4g@c zy2&PN$~pDM50(=7>stXFj2~~&ImaY_aTgZe%smE|@t5_DYb}`6cNEDxNOWK@^Y7v! zhxqQ~yXS@BS6FmF6T||2A#o!)^piUPHn-n3cZ}YYD+`^K^C+O-WAvBh&1$O9ds0(> ztg=(=arh!@P8j<3g@K(Dh!$Nx20~vRhtuVLhiuBwtt9gugn4c;XcfO-qVug zqfW3eSKt2LPBkOfYCPA=%qkZQ2qk1rjl-rBwuKgS_V+RmL?2_FtkKsBAQ_2jMDaC1wzCTMbLWD(Za!O?3lnuFlI?`1eP^3$yS?K)gEj zcqS4AK(^4Hd{zgMyKBZHd*zDX-91lk4?L`>13wM&)KB9Te*@PVRdSzh4ArLZKWG!; zaGqtxM}NYnJ)9_2xytk=UTM{Md<8N-{2~2`v(yh0vn*LIj?PE5ZC0A`>SZ-V0Z2PK zXS3L*-TFLsHg-W(xjo=Y9ej*%EpOGkf#&o5?=6z0sl%8VUg=@2ca4*%z@Gc-GR{FWHP!A>+n-`5^}Yp5*EKNtqNiO!@Dx76OJI~|NB`5B zBKJ5C`^6#n4L_ekryYx=H;PA|2{&D|2UpHA*gv}ef&%wwpewT+Zv~~o_K@kgH#!ER z7+o*Fk6!yFh7TP&cQe+txTjpFOR99kAv@3sinV@;4I4J%n37LS75^L)Bp9i@JLBGZ ztz`C(n&BvKoF%F^`CG>=$hFH?rfw)d3Q%82EQ`$lZ%WIK=-Oc{aQS&Fh8~P!XSmh; z|B83k{=z#g{ujKn#iu$!_f-DwB+ zC>i5BL=r?WGRR{RaJD2VEO$}|y>)6ir!ytny!?tqSc|ir_hDZBJBp3$@}4i#t#gULF=@iMj)Fkh#V}eWR=?$Dau8f&&*0C zn}yuE-E_s?QPSwRpUAc^Pz~Hd7hh5$D1;!h52MCA%s=y&(9o_7vfFJ)erY*#;^y%F z$QHOqTYzU14+Hs<)p#hypl>H@;(x4~=QX4$Q(O%pD?}O~YE9RoN;QtJo14mqrn5v6 z47{1|@0Ch=UQY~zRU9g%)$N`f2|XExzla3puyf&iGIXmA{RQZiA133wYa5qsJ$s%n zjof?FdS$LU!2hlKv(themL~gmv~@lVC>$!fxlY$?)?fdrdIC0&1Ug-r$NH`H&IHnl z3IZ(~-ygufJR<=J@g-1O4OwXQ02|gR#Erk!UX#IDUNKlLAluWsJ@NocOvt}OT{^6I zq}5k6aPVQ_;16b6fU>~`Z)wds+HAu?xQOjrac`1O8TIV$I{q|{b8S6pX-B~YBY}g= z8e$*cL61i+B(Jri*35OYcyC!_&%)Ps8l510aJ)FS{0TVhG5^7x@&w~lUk{t zHRI>zH^wc3g@cmi=d13~nsW$=m`Ag2~#CD*&9Gw0 zWsUwDH}xM4dTDGendME2+Ro%vB}V+HwG4!Cacz-3yHYhS{K!wdYfo?KUuCwc{XnRNW4bV3)>l0Zi&xLk-Oq5u0s;}@I< zN}C8QqQ(5eL9iN+^Ego$F)9r9*FiQ!rwU!=-|EOUzw_DR7VIZHctiS%dM(X6G~^i5 zS+c;-2xC~`zK3^gMR|tmt8yd1)S6wb9jNmb(=Up@?!0&b5ke`(>T3WPA9H0Au(8BL zPXEXOB7}UGK_e@*022%SN9U6`sbA0=b(sD~LCe3h{}QfCbXd~plFtjSN7{Pgfq;Ns zi466K_ zbjWMAXQ!g~Tq!34eV1uKs18Pj6OZsB0cX}~o5gVk1XONO{#SdlQ`yrkk|C?NP76?x zb+`af9Fd#4M>TI7F)mSgt+xtRKJYH|rNu!?PA6LQA5tr&SWv<Dy+9(~yicRf^@%=yJBQQhWD|^dIS+bs3dL<9h1dx?-?rY4!r(KqV7**%|pLykF%HsBR79nHShk3_LwTH z>^9bsxq>p|%Ae4$4Msa3Q#pK1Q4f_wY48j@D4RzV&9V`&o z`=IaXL{kwb9KN@M%assjdB5l%yAv!~W*DyUgvzIF+vsiE%XzMbaAWfX=ylbfk&@ zoQWk3w$<9Iodht*kc~4~O%6EWWqU`!&85s{8Qpt1MjC$2c#tS<1}9$;5`Gnaz0{ zV$)Wueqp^7y=V3K(pT{D8>$0e{-klKFL~4vXMI0YeLOa7N4!Qz|MBb>V53$`Ej9l7 zGg%&WC-Jp`Hkt+ZMIk?6b^5dt9uyAODZ@8fz;WV{^1ORi#wWeWJSMVo6<>cgUSl^r z%c2o@z1aL8raVx6Z>Dh;6c{7;z9bXWcjK6w@>Vj0aB4;S{3+7HLB6}`%=%7x5OR(|a`iA2AOUeBR%CAcS{LXjYSnAo0Pw#+ z29&m1VY@d8yd>bY^+gh}xFLjXCXmVPTcFlVJQpUGXg$$Kb`pkd{(dw0UjFT~&?&M? z<^MGL+qifqD|H1mpNBHyKyS%pe;7(JcrTU(65GDdX876CW!tv~m7Y;1xpLV>DP-0E z_~PyNkDlyM>RvC40k0s!=C+?@vLV*pu|C8!f|k0VnN{X%6)KE-IBI!N<*HkSZXEo8 zqj!y|B&Q16?eS9i-Cr>ksz6y?Ixu>oUBIxb#IJD1sv`?;qeXvITJ6ohUgLC#3Y6)S zM}81KBhnks`kXS=m}Y<)U*d-$H1#;~UbZ&*uF*WlMoNx#2>5TJvZ}LW<5*Q|v+tu+)JIFreR$CuYwyNS()TI~s zj9ibXe*(-BOD>xru+X8B``gU@8+bCDc=y^MZSrqZzY5lNjI}voB>}Xsd6P)cp`VQc zIk`pex6L}geYASVm7H4h|FI_T5&W?I5%Xm+nQU-=beenl@)1k;o*&~yMR>-e>XIMj zx6*z{Zd7xguH?SnExKcE@F#NlOl)PV63tA>0WvcjJ<=L**3;~X&UW6I`vMfsV}QJD738hKBXFAC?i;0MU17U7PT6euVxE?Gcm(l!;?KIi zxH9tf^wqcsYtSWJ=39aujtW`ZEQPa2H;-<8=kYHLFN|cr{wbRBvypb7sK=*5VW5yQ zkfux!zh;e(#(-lv@UOGkzZGy^QO~fd&Z&~9;0Hs(*xDI@@ZhYoKYnB={7zK=#&pBA zn;Pfo2r;Q!ClMKBuzl3#K^9BA)4$^n|IH_qAcr&f3tfavn|9D~G;AefzZoba!`$=9 zLfihZ(km=V*@f2K`~GTe+}Bw=Zcs}b z(ns>?vGnjMKXIZ8sj6x~G=z3`^3>JWdBt4JzQ4Kolc6wMH2yAdHo@kWOppo})=cZ5 z1D|Yzd&M+}zsq~O!(}GTtS&tCZOs1jFKJa}6#VA=h=>4gyct;GHTK-MqtE$6){vSd zkQnh^J2*FuTdzFuZR|C%C5?lO3l5>p-VTmWd-X(rB#fp48!BEn#ts;Q(q;9IEjC6MQoQUYK6gC^|^@1WQHKG4LTG2hNg!Rhfc`ZutX5U;qcy0RMoj zR38oyh1eKj7rU|UF+q$|juE0$=;%8foNu1Ry(H)&{U%!fCF8^T7|S?69*YY-=%hpP z&FqKe;Xyb3-5f?d%Zq0ExwprAZ*rB_S5YT;o~=!3dC)q*H$v=3)keXMygm|`tt^l= zsaAiGb6@(K@=$ok^@;%zowKg{QHSW0C7y3$_P%E-iXGi_>H5g}tgd}6eE%#vb|2xI zBZJBI+V~ z0xGW-%HBI;#;@O~`pkO0(Li^G(NM~IP&>!!O{~~pYw~G|Z5pl(8|DC7d~Fx(tYiR{ zO~D{kL+Ye6(=+*^xxRA=Rn^MkC7MMWsYHEkFNirNd=S>uUHJV!x!0OOZh{G z%N+GrE=W-gC^0FaEK+PECvU?mE84oxst5IJ{x}`?9g;UMmhM&NL>3_{mRRGnO~&ou zp{_#4BPKV;c7K*fIQTTY-gFE(=M<&X_j7|%ahAgsfKF)nOM~o%BKK#{;>HmrtS+PEBq^_?!DLHOp4dJrlZ8wLmQoL8XbA(%_WSQ2$T}vh)ug)HQsy(%4bhARNhpxQHZ~_YoXyphLeMl-lbmyQpN&;Hy+ae!@#s%voU~9ad?sumv0`v{ z>LOX#E5%tiA_q1WV|&SV#(=882p{E?;dk0IhmDSjFF&TR2U)I=b(hJ0ffgA*sjj$vj+Ka4(jX+Tk zI`Mf`FE*Dr!(O25WCd>X^M`M-756QvCmaM=o_SYMaV^&XfG%kHnY1AS5BQQdq+Vi0 zFqX9s1THgqZJOvIA}32P40|>f;f)y^KjcJS{|RCBjlZR?EgJaBPca%{4`QB#xCyaB z0t8_Ch*7u8S zB4}2P5_{Y+8?;Op}gUbJDb@}?v)WQirUVIB-%eK)*9fd^*EMT z+rSnsbR9W{m&izYTIt`5eKd|~a+W*exm+waRU~f4Ed<)&k)1nZ%%0(#K-Lq*YXWMJ zztZ`1Hn)M_G8C;(aKZbM_$h2c`+kOMOdE&?B4kt@xj$tyq|%e7P?JI(A)5a+K{Llp z&spSDLLhP?kobGjV%bYPKQ^-+!I_6koQP7<@IlXrx4t@^x`7;5da;^(?Xoq>jNbP3 zW(m)Q(qL{`Ea%&^xH(APzR9?FYztYpOS`p6uy@2!&ONBBiaBbPvzW3Qwv_5kHtK$< z!e$ZjxCewfZAVL8aQ`CJk5!4aGA3fj94}Pw6QmwJ(t!wvU-J9df`X2qwA@v5>zRI( zTZ?^^`et!v=UD`=w%7Y!${KC{Zc#iPE=a&n7uz%t>L#`4>hq{+NOB{`36o@?bJuBn zl~SOy^KEZnGAhr9dMA+O+^6Qu*nK-3ZT7quPT{rG;(p1CS?-t~eT9n`nv#*2<8Q1g zKJ4~J#cdz$nZ$_3st5&{4l*j;)rp{nvY=%+Av>=%OmF+bF0NX1-5B-d;b%pn&0d5vSbJ^y=`Go=ew z@=N;#WM#4N{l)BCa}7@$G0bm^)z$8QYEFSoUR&URfZCfysz$hf-pBETSA&>zvOf>O zoZ9_z3VGq*^|!RBzn=X2>gPe*C`mIfaf*yruj7ocF#6CStWeC$;%{G0U5L}HGo!t6&V(aB;2Y?S+}3(JgB2wf-}rE>%yVG9MJn}r zbb4a-PF3=$7qb>!T%9Y!rVFNQH&s$*%LAc9THZtu-ebWq1htBP*`?$$Qf42doL|$! zg}gS^mnrQe{`yC#(-bvb^yCZrehn(JEo&lPBBbd>_Q>wZVh_*)FB9N4WM$ao_v(<2d)m z##Ho}^{VM${6TWXh67`Kc2CHQxSW6-f;z6N9Wlv+q3hf_v)V9P17n9i!9;~dS7p0< z6qv0Xq<%ga6n)+;XH)NVj>!|5D7G|1w{`xxN*Veyh#IPli;>|a$Rn^18>|jucAEl@ zU~JYaaXmQ@^CN!0`Z4<9j;sMEqLHj~B4+~w%us2^=zRI46yN7^9}GHv zCZBSz)U1%y)+^7HqK*kS`=TgYcAv%`_HU$fAw<~-ZP^*@VFkPGB|VAQsqgK=sclc zeSXKr7eAtRlIgC-^DB|?jW!E|Gxxh0@QCTOj+bqIpJ@32F$~Q52Wk~tGpO}BiRk}1 zuA-_t@m&$fR=^OQk%>((AwQ7F;+rSgur`wdo7RF=6Z7=woPp3MQ3hZO@R|jGlAe1X z#>Q&utFX)>jqyWvnx_=l*f&*~?Br=@9@}nBjV0z`RITlOVg?MV;wqM1+@$Yi*8V3(A9? zSl~hO8?w(>SS}|8@Z((H*dE`VzOcV`2@^{1{$z$T-mt9`N6U=lq&sIa)GUC5wb=+j zujBol(O@f{JEOadEIb|A%YtHKQu2&Wdz1DfpJOm9dmNma{86?tli3+pDeA(6t~z9~ z^n?EPT^DhzKAk)lzl4IsFRFZv${iB*i*(a8>}^a;7I_<_6l{}wBpr=4B(lpzX$PZs z6VffTYy_e;7YvB~H6yS}skD2prxueh1dO}AiOx%Uce}h;6#4D6H7ky;-JNF{#=^Z1 zVsuGTG7NGU-q2c~6%2k*p31^%rIKr5rK{sAm-+NcqZIPp*ETc9ns;9vU4TOpu>b5N zji8S!Le2`LA+3dQ-0Aa%tnmFF45O5Sp6ip$pP`pdmww`jkmi(ZwdA@t9VD@a;0$O% zaF77|Iactj^w?gyj5QXX!GVLb&a_{tX>fp@c6mZ&e#eJf^8*6pR980juy($b?fD3gvw9icp8^fVMHAG8GOPY2-tvz-zJ^ys z8%7YMFqf*o!WD}ko62h!cV{FzP&wmKxrTZ0m;MD=4K*2K=3BkniQLXZkMtg&#j3`d zp1gzeV>C*f8l+0I=UI5W7p#-SixeOoi|JDk=3n&Ym(6)jp9X>TIME*!mN zu<*k$ER)}+HRJSXz*&4ko4HPm%2oeYsi|fzo!c@Mb6vb)13_TMG_zzV#Ei9Ghv7Ok zm35rZMjB4T%df=S;F?bSMzue`X$1+;Nf92&yjgS1)YJrbR?4=JPf^&`xsveW#TmVw zK{}I<#BM=4oF^*;(T3p6PHPbxu&k>eciZ(X$*sw3Uvrfjm9ne6-M4hphx486SN)%0 zVFmRZx^*Al62jS0q9h(FdFdsW>vODfyYY|C@W_wX zOL|yP#~n!loKfxZctg$lCPSvOH&ezQ0WvYk%zQJfQuDQux^@o}pwA29;-EA0b&pxJ zg7kS@19%?Su`^Ap+y7-1e$3*m=i7%N_=izEh!d!)Iq>ULk>U?Tq6^?_7nX zZ5T>?|LJ%$^6OEF+(uW+bH1tMuehi0w;L_Ht z-YrA2O!2p&n4>)~EN|jXr}1-EIENHWTF>&XS=B+xx^N>o_0_q!)`zu6zJ2CNcD3m| z<5G=c&o<#E?j|Ou7HPU$#3bH|5)az#6P;G^ZcuqZ+y;LtkxVf3k?%ledT5I1Rmh*w zvMhhhQmtZx%|s7`5IOP6MuTvsdHv%Pa4RaHK&7EJQl8&60DN3$+Bmh+U8otO|F|uR z!r1MK4(A#fQxPxyI1l20c&NY%T{0)H;M24RRn!MQT7B0*JbahsdU+e2q>R56`NTKj ziDr**_D5=h$UzfY^|`kb+RgP@wdM?jRih9 zJS??vHZ~vs==jmPIVZQ(L(;qU{>^=#y4RNBi%$BWB&+8giVDLX32!Hc8W0;d3-nd` z&_a}Cp_IabH{W7nuen##i0=l5(YUf9C8xeV_3?cLkYl*+`S9Jhi=;>Tz5h1Fhl2ZcvjLm zN-&DNSPM~*z6|-T>NqU7_I2DJbhGAi!*~(@i`I5?1z#BscKd1r?>0Tlc zr=VuGClQM68xIWd$*yH-pKD=EI({CqJ@$*d3uNWrEMxw+h<8PD=g*!kI(zn&b{Kjt z40g&Z%J3{C79$^8E|wtNUvy{l?|Bkf1z5(>9IUu`C3m4w!)qaZY;92U(GF5!;R zVNqL~C0<(`s|}$=_GlQ7jz8u&THJ70-g#q>|Kg|GN_~j*w6z{bacPO_*SOwjzKXIw zbs_pmJk^3=cG2pr&OefoT(Egnop+9`?Sr5JXIGGAws|T5TL*bY_)H_fX5J9~Aarsn&s95#uty z;QiVR3~}Us#(@cO#E_;rY>6q2$0EVR^^Vo$!#cLG%TqW6(Z>FHg|JB}khc&hC`92B)3KHqkRZ!g06hV|0 zl}!^65TzHDPK3}25Qu<)lz@VWz=nWGQ9zX5By>@c;l;8KjM%O5!x@~6|S=@z~+C0opii zgO=&ji)J%@i4vo|Do3+WjmUX8^!{Kj5^=46L)o&hM!uU))e0IT*Lr$@$H zuWdhad1M^7;sCXc-q@&Wgl-EfjPG)(fW7g8hJ0J`-v=E*Y~De&iyz63U6Ts!=>e}@ z(#B&@@$~z=P8f5;r`&ApkJa^IhC9Cx)>B`Oc@-edJa|` zy@}=;ol~SNDAuZBr}&@E>3WI|&6E>FyUtNYL}l)H^qp@!B;1CQ)_{a0Y3$xZDGrvD zl+>Gx)Zff40UAf-{RX8yz`CT5vwV)+M^<8M%3$7*=qGL7Ul=%M&!{0-z>!5nL?F6Z zmWz4<2sra0m?;|;B{H^q$L!@AuJmT%X}m(Mm7cWG7#()zIPuzQT^+M~h#|Ofyaq>u zpb=wov_g;u65C3=lp8RCJLX!o6*+e0P~L0z6qi#io`M=OO`dLIQa>78H0_UcxG1a( zh7cN<`+RUvq}Oh1!#N;b25guf7E~{RS9(~^^Lo1O934zSI6M&8T-FTl$vB;wIdSNl zO7%w}x(nDW4M54$vguG4wL|2NM^xK;0lcqC;PN%*a(><*#|*x?S>?Tecdf(gH&n-r zk!;2cq*T7G=58ko9_Z+`Q3i%~ub^&_p91NV`lhXcA(>;!_zLYJH>n2}n>s$->|e&r z#4(7I@t3EaasJ-No2L^__)%lQ?;74qmq4;w02`CgXe{SCyn5Y$Yiynnm-cJBG%C>xd`1Ga}H5G+V|M_|?Ckh2m|Om$_g^i%!CPW6>^ ze3@>LwyDAM9nT=R+qu+C!St-RmIwSW#oIaHrAz^;HY)+G!hkISulco6EfR)LFZ5Mk z`NHwjUs0;XXqT4S{!cS^!-#WgnjF7eq8;qPbVSk$U@i#ZZEN*mwkjnrg)ccu6g zY{}sk&VvGN8M)n~%fD^)TXWs>4d(XvJjcy<;hNG@&hzoJzvP$-hOLt7Gr^e41~x1< zxm1~O@17lrc52uV|FYLaTEk`|Pev>(5^zDwEfsTvR7Q z1rEB9$puzk!oYAqTCkP3&vL|a&hQJ%g|S@9GktGgmCxPcu36g4(Z`1b@-5Z^&ocD+ ztYF5jlmbHmlurxS`!|G*jly<_v(28D@iXB@_!+#@^>-s39tXlqn#^06ntEL4EjQDE zg#0-U00B0@`&0y94qJ>gxjE$)4{1x()s@z#Q5@>(t25rR&TY;M=veb`M_j!LZ*|Us zX3OAQ(fq1Yn2U`z!u{)RH?403BXlbHntLp;Cein6f{M3_1qWlD@*)cfM4 zq6Cf+qkmFc!G6+NEi|9ag8G9&>rY~DUEj(n3?u$7nAia(0=3JF#5UgWrNOVOd0^&e7^@t$^$;` zLF}-P(d$#H9vdTn2*|SzB-iF>e?DFX?CP@yP8k!Zdh(*j06UYx6`uuRFQa<}Bf~ax zs{s~|B4itSjS@azt{8@Ji<-pOOUejspOT;Fol2{ca>|1$2OGwm!ioRTX7unqS50VMfDm^0TVsP z;TD^z4*8ZA*3zT#`8=;o=Dbx1hR45GU5e@F(P69+JEwDR8v(9I7=U%Noy;6At#T>R zMIhsIt)2~Q*aTzfwS;R?x5H@J=*v$}TCT$#&#C#IuDt8iNLK*1NbEV}DsXjfPIyEJ zoPEt4zHu~Ub8^d~Jgn=B0Pb6yF57vlktL2;M|M_-!56jdJKJI4H{>9rm!!!FjUfVc zh@-A!Qc=1C^{Q$q!Q@Tn{)HA}SL1JjPIbH1ZHw{m6`FPpja+66sWGJUuCuYbx>Tlk zor;X>gj-bvcX?4}^WZ!*bS|)e*nm_AUEHk;az-r2F)UrboEx+8Qm$n!D+)+oivi#B zropiK#7usx?l}Lj`NG9(0aMW9aFN?P4sqY`D)8JUmkSqqyrpD49pqtBa$+ss{M$s8 zXC;X^7tz=dVIbL)q|epNmoz>n!u{^~kYmcP{q@RRJ>DK`pWN1$POB8$-;Nqj)N&e` ze!G-2>w1=9!GM_>K#q#)SX`BTu!5!Y$lxv&zS;I{MFHYpCqA&kQ#H3%C-456X*P28 zi=0+Im5WwH zyQzFCLE^p)k9Q-_n%1T!b`PS0qs;C>1a>4gUx80hI#35bK_TG$N&aE*!DUd~6*Jg_ zWZD92XpHEP0GhfADksyaa1cP(P4C3-LB6NZRmMSm*I%8g|K-Dr=u&x*HZueAh!uf8 z+EZ}CwMUj3fNL)=$YL8`#K7nQ=&5eHJlfs`BJk5p&O>M_g;{Sj=&>_Mst^wbq+1`scY{v!== zrCBt5Yn;y=rTq?wx+*AM_D@~n{+Iml2Vby#7CXxz+w)r}7nKco zt8z3O^=m?lBHnrI^nkmMC3DP}X(D{rF#YB2JL1os&c4`P5qcwMcUP?q;MmnhqHW0I zt-^wnV0JT?DNZ>(7U7Eser+)y=$;|{HhJ@@I3v<{vFz0wM;*IEdR}r*_8^xrQTgh- zMpS7+MP0G$Zo83l*$7bB;UIGgzcLYIhBD4>vU4aNZp66~Pu4_grCn6El1;DS`IZP# zV+AW1j#!MvSjwbjI_$_ij5x;g0sp{#-Q;~8ogn*0%U9~=^Dxd`9yg&*OBa5tgNMah zbMY%nE91JEM6(sUXU)BG=aU_dmSlqT6b}f0%ZPmsvFcih24%*+(BXtuP*hc+ zyxu_e&8&J2JZ}_)KUB~y^gKO)sfq1k51aj=`PSj~om*|4Sx^`yWf4PBZ^ZE~(T~Eb zENnNs2Qfj#ugD`h(y8G$D{5_AGJUnQNH z!uKd4uIPS6l~QI_Ri%wYU&-Sq-B)Tn=HBRpT|A@17`vK1TJ=lyF>v5WNw%4WwkCJPtC+iZx<1c@ z#1HDO`+JJNTwan&>O8=5U);koz#v83o?;$IXhA>D6s+rC2dQC5%RBwc6^+IDZN7FY zoHooSeyK9&(94d6Tw+MnDh?hePi=ll{H{mI$ZyZMCG2$5&A7#XCj`rz?8aczmJhEa zq!{s4R>gV41t}LJRGB|}+SE)ziDDBO zzpRWAR}V-@D}J5?=W^=IUbruuq2+}zFR&IO9Z-loSAj<`a-<1 zzf`_f;^6v-hG%NFZo;UxRYp^D1tdfpTrSxgPRtV@1b#^-VQy_1Jii?(bFVa0+rl*I zlUZv_gswdHgrS$^j1a|qOoE&eQ_oeTOIqRoL8xUagqS4a4XY9dNepqXopbMauuR)~ zNv@-fX+`C#mK|8AuP|4;OO{Ke%^6BaMS&*Kp$~sA$+Y(A(&-v}zm{XX*)aQieW`b1 z4efFV>SkfYk~SL%#UKWCe(d1asjsGbePU$|mXeqo3pXljbn8-8GA`*`o2)HPUg6m6 zJW>=#I;J4vQE-w}}JjX}z!qaqi(y3>p&qdS;Xal>htNK6%tRA6;gYsw}9l$MId?aq+V z?ZWDltb6svyni zk>mt#IyeJ|jjq}W6)v}hs?3jT;_BX=2wI{ln}+vY6`^c`_iu{s9>nw|P}QoZ#IGPI zd=k3bS(&nwaPNfLI=Z-Ir^IR_{UIXs!J zrlLsdIr2mOeHD2&-SZrHzbY&$Ui@W?9s1!;JuO!aW@A~AhPpcVz)f_;@|%s0ox4_c zLciJxB{NqM{t(=|yP~O_h&6?jmgJ&GytPiYoWL zO>~C^33k(26Ik~ktNjnv{9ZPI>v^OV0LmGzL0QlwPZFALVFXqoYA9OtFp*gt zb_~k1pcw@vFVUnFHBisctKKXMSRrlqAkUwnTVzn+0a;PtlUhv%qa*3F4BuTPe_#QJ zo*3BrF>(~0PXv@c%;buayRh8oA16srs*H*(_ygR4Y!hR4BpyEr9$gguu@t4bZ?e!dzJs9fID%K^k3W@UU(dq|+epzekf3P5ApO#p=tjP>Nd+)bn>>3c1 ztVS@`pWGEWKVQ46qRK|Lx^v^DTJ8GxNlWt4DCuAoj~`|RLw@*ITnzVL&3*ICW0~rWNeL{- zK{xgGTQ=%B^8%M}uMDIk>itd6??E2S@dbL_c%rLNt26VVxyD$|;o5<+nAO^|3}5%D zy?>-?g2n7Vf;#I!g zs`em-52l2i#3qSn8TRl$WoNc9Lhm18?O?a_Z;E#%!NCDy68$D(a)pN30m&A>Nt8VK zlNYm~EF~6XZ2S0QHST`n=GkRw|CSp%^!CA%;J_#2#)X#4h?70k^@kl2IY(A4_M590 zH_?o0_mIp1yl~i;29;KC%*{#*75l&cI(AekrR8Z_VV76|BH; zI((Hf6v^~%OR2OQlHrs9bENvnT3J@0uFZyJU=jp&`o?~M-ikUY^B_Mc;Hs)7(0N(4 zLw)Sw6`~~AdbZ4Dr{ph(Tma;V^cg_uOZdzW;tL5NH~up5)6_uKa8bz@N5>7?p+1Ohgx2?P|Z1ZjJxy0>0GZ)8p& zImaz+aG5U*Iovn(MDmxNn9hHmlk@Mb^ndd2P=8tmY&y$zeDf@F9CW$C+^j$KVSLkP z50V&Ki2jy6g{EmLD3SlZugbs3aQ>M;K?GSB#i<>fF-CrGv*^K~jLDYGe6@tfvhE73 z3kl`GbC}ZnvSymgC9UE>-m}y@}v&gxob@rb;0q#@E&NgMrY7-=&kt0qz5fIP&`j zK6D=&7)>zk2P-L{0IeMWg_plx1Va(Q-xZY@qpKody@XYeYu{7|UJ8jc3Qv+k25ys@jeenVVzxbL}#{c5J za$re`uLCRe&BcVyPcf{Z;G*18>@D1J93NsB=}YCiiSxZ^LU~`%Y{ed=G~^(1Hv@XR z4uBVa+=Ix}zNHyx`Kcyuj=?If9^NfHUEhf2Ze;u2QY#9#C6AsSk6Ce zuhf8*^53y43OqMvDgR_;+VRreg}!GZlAk5tlbu>+1*ytkMQ@WetNe2e&@MR_(4mZ>I|8C z)=l5TafdaU@Uwt^aT|r(wd&O17$|c{RcK4j3Amb~=PUPw_mZ^uQ5D@rxr0~L^d~=D zY*k|zoF$KA)}^&tb}FiwboH+fEje8sJsfXXbYRglocK`E$n0l6Q!mpfPl{WirM2EW z&A0qzaiQ@{!SGu@3*i0+G~;g)UH{JiSN)Smb_+d(r{7>I2Hk3+Fnf@V<6psIJKyIHzSz z^8MuTF1hlc)aC+ePjzMF9HaN!8RpAiYn8|AQ&5JY-d7*^;hC@wQ?tZX?x&Bui)6&( zv@b>;nK<90-VF&!a~9#IrOKwXrG=#WPi(Ja?yt3)eE*!TQxT;5KJSe=i%f-JWIb;; zx~2$|s3Fpm7Leh8?(UpF>^*gip_C#$ttMyyCPxlhQ_<2a57BvZ7AOk@RjgeEJ6HQS zjXeBx?YhxsGC!hQ_TMVHIokNpo+K>5g~mXFu@tRV*kOedHe^>zhzP!3+8(8p$(f># z)eY+xT!?nUQ{IGo@~yaQrL}&J`~3d$aTagv6Znukyk}?+VpCR$r`{n2;h0>{%d`!Y z^UpLy#9wIme)Fq@qn|2Ku;8KaDLV^O97M&qz(T(WC5tNqVso4GwA?r{b${3iJ;&-N z+kJ>TyW2BeuFlMPe)gftnHcY!_9da1JI|h6JANWQE5_|#L(HHk>P159<)tEb#0u(S zZe_NU3z}OXDz;wrp6@372j}eQbY<;Vu{4+*Kc@!gsmeiSAv~IN6^W;ZH?V;T!FIP%N!h3C4T}rVMu20kD?Lirvpkza#*LB4CX?Yb6{; zuGvp1r9ej}u+@P0>208K1209-u&qGjXmQVu z!oY_`=%J;0f0NZVPq_HIPl+w(luDHk9`{b0ksNM-fmUvF+30VnzhBNz6oJ1PJpOeE z@n;K|waXiD^}>+lW$RhUe+xfjZ_G zv)^7lUg&giI1#3N%(a;Nz{*F;GALassKoM@3&I{>xGOvCUIs7D+OWRKrYp%l4(DT0 zKCMjW>Dk}o)<}rB6mXAAQ;bPd=08lIR!T!>>(arJv3-w&!oCsZH=u{Kf}sn~bIfHb z_Q|Zf%-YI{mj8N3BBA`r$F0+Gn?8=l-w47s*omGv^s#!@{EykrJJ|h=7)8{~<_0(7 z;?cA(GxkwvD{bz}^F>gFZ+j3piV+OE1!hxx$pq6Rx=ne;PGItt}VbSYxvpXxAlA zwUeo8Rlpmf9)_fxaozeCU&gZ(>BS4LirGTV${!dFqy!dtW;|MY0I{GI;HC@aKvlq5 zq~N|4qfCPzSYmRO-9D#;2JZu=eXY1|Nk+jRcDM6^f688C9vm0lgG~GM;uiJDiu9K| z4oFOpe+O7k5|)<*=b4V~&oiaki2Z>}Pw>SEXkEU;n6!8<*7dnIcU>~OT#e?m6TeVk z_*DUJ6U1f%b`PS8rhwCNM_?7}Yx~MMVFWqr=(%%`0yCb0;HE47UI*XgaDvp?gQ>KV z{CGATuz-$$xEPt^h&usuvUH_!lGb0J2Gaks3|#?U;B7Feimw8Uz@xx6pe@xG*Zfev zplEz47)B2SiOR)vZZsVHh1Ljvc~9s|^iPjTi3=^gLhuUxl|cho|UY#S?lr z+5};79IOf=5JR$>0Ea)>xWh(}B9Xve$KmF+>ZYh$DiE>wICS}NB+07w*)^drY>lTl zX9x;Fx&|aLVabbVjB>=3i>Il|1C~vW1McQFzTtF^jEh~IrH5{jJgW;JzfByly=dA8 ztKg$13&ZGbPzLn8Y%bx;*L|?lfn5fWC4Zxe%-4i|_38u1yno&DW`VE4r_`W8-?>csja*rPi_mp{@pCid z@C(~lA)*B$7l3PIR$GceEL{vpZ>hbMel+*{BY9RgnIMdx;00Pvj#j|wX3VLv&FRq_ zms;G9i;L$dM~gpDe1u$^@NdFJ1tQ6BLnl6%b|fJq>RrSwhSFxgY-uO7Y$=*-ei}@e zZf4^cF7BQm>8P`RVJj7ijh9CxfH$nll^|Mf7WF!X3G2Vf;*%`}4qLcONtaCzGnpUY z(5T*a>67ydwW1}o8AKDEA~s-*E)6mRODN@HZM>~ZpY6@({)i6T$rlynZk}AKZTX3h zd5od;Lh<2EI5s*r)q$)C3b4B;(i5UY1PyqAvE}c&X`7qgNaC4TxCi%*z8=4Zdyh0O zM=IXErG5m_4E?MRpW34L9oy(<^){F#>hnRHcvmRmty?f$00ITAl|k~ zAgyT&Ll<8tnM6NUqHRShIB7f{omZ5z)}P5Ay}EkwSYc9x5c@z<(ZbHk;|dor0X*hv zs+5`UaVNL7Hgz^*2y#bSEV1<=gkJ^1>&oqQYm9D45~7h#pp;V<3Z#=}lSwO4DJd3l zwf>K&X7saFAF7>~@&^YS=i74{Y4*nD@ViSM|001hoY4@wwAq)>*G5A^&u$c{E*WrcrH6HoDaihtQ>!lhpA zG7Yx6k(Kx>Jk}hgZyL`-9y&Ah{P>@271}&-m^{3BhSJfRz>ZC55PJetBMllS37TKi z7nSs~kal|CG@7ecMlz}~V=9TpT~EAirRjre#{iDJkUkqFJnGI{WK^^NV9G2vUcQIP z<-OT3u|q&QtLpIEs{)|ETRczk@3i`V#lomKwGaI=- z@di%OY3R+`SSn-~(7a33afStXmb?WFfRck0vU7o-qeJ|5RPJ~*gh5T{<&m%fX0Gc-GvBR=8TxPRMe zP|;-DiG+t>vcrKS+y=Z=_=vLMpk5;(s$}Nt`mm?p*vWVJ6V)0wlE9JUHEdAVIDRV ztl~O;R83Me%<%`xM58Z0gZ4~lK|*uQov8r)z!FV!ZEM@0HY>_E=t~AUs6F#? zechm%EO0X9992J>s*pDirk{k$o%rN?At{c!OIE`HHxUE(Lq*JWJ6IePPrh=?S&Nqz zV6iv9mu5D~ekb|iiJh=RSL7e(I68k%)eS3BhiJQJQ94zBFl;b4+8(NhP82h=))HD) zSbHYBMGRO_H`Y)d5;dz*c@^Pz@Jowv&dtMe=HkXYK5{kH>~Uh(p-CZwSO}eiDhqhi z3MViE%k7q`?u46oUq3!wx01P4i!SjL^Sb0jzmcb%rQcpQ#!94|V7HSdmuxEFp5I$@ zSp7I!<%e4lxHtUkScl~#?8xkRziY7$Q!IAJxex4I%SWjJ2yzlhGnEq9(@c2IsZT1*P|}_2{$vpBJDC8r{Z>@D+^?HAW*Pj_ z-m7xuPfUwnI1>m`$qY-IevP(Tzz~>A^Q*U`+v@&{;w_W|I6*_ zdwlP=i7fATfx-q8O5 DE0QH{ From af2fb82002c08f3c5a4e3947f416199fdff5373b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 17 Dec 2024 16:30:45 -0300 Subject: [PATCH 335/345] Clippy --- crates/networking/p2p/rlpx/snap.rs | 2 +- crates/networking/p2p/sync.rs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/networking/p2p/rlpx/snap.rs b/crates/networking/p2p/rlpx/snap.rs index 55b465f32..8d56438a9 100644 --- a/crates/networking/p2p/rlpx/snap.rs +++ b/crates/networking/p2p/rlpx/snap.rs @@ -450,7 +450,7 @@ impl RLPDecode for StorageSlot { let decoder = Decoder::new(rlp)?; let (hash, decoder) = decoder.decode_field("hash")?; let (data, decoder) = decoder.get_encoded_item()?; - let data = U256::decode(ðrex_rlp::decode::decode_bytes(&data)?.0)?; + let data = U256::decode(ethrex_rlp::decode::decode_bytes(&data)?.0)?; Ok((Self { hash, data }, decoder.finish()?)) } } diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index b05e2dec2..5e1399b1a 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -523,7 +523,7 @@ async fn heal_state_trie( // We cannot keep the trie state open let mut trie = store.open_state_trie(*EMPTY_TRIE_HASH); let trie_state = trie.state_mut(); - paths.extend(node_missing_children(&node, &path, &trie_state)?); + paths.extend(node_missing_children(&node, &path, trie_state)?); if let Node::Leaf(node) = &node { // Fetch bytecode & storage let account = AccountState::decode(&node.value)?; @@ -533,7 +533,7 @@ async fn heal_state_trie( // Something went wrong return Err(SyncError::CorruptPath); } - let account_hash = H256::from_slice(&path); + let account_hash = H256::from_slice(path); if account.storage_root != *EMPTY_TRIE_HASH && !store.contains_storage_node(account_hash, account.storage_root)? { @@ -694,7 +694,7 @@ enum SyncError { #[error(transparent)] Trie(#[from] TrieError), #[error(transparent)] - RLP(#[from] RLPDecodeError), + Rlp(#[from] RLPDecodeError), #[error("Corrupt path during state healing")] CorruptPath, #[error(transparent)] From 215ae806d2c8739f12acf9c7c9bae0c5c179034a Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 17 Dec 2024 17:02:45 -0300 Subject: [PATCH 336/345] Fix md format --- crates/networking/docs/Sync.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/crates/networking/docs/Sync.md b/crates/networking/docs/Sync.md index c18101221..d0ee3dc01 100644 --- a/crates/networking/docs/Sync.md +++ b/crates/networking/docs/Sync.md @@ -3,8 +3,11 @@ ## Snap Sync A snap sync cycle begins by fetching all the block headers (via eth p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). + We will then fetch the block bodies from each header and at the same time we will select a pivot block (sync head - 64) and start rebuilding its state via snap p2p requests, if the pivot were to become stale during this rebuild we will select a newer pivot (sync head) and restart it. + After we fully rebuilt the pivot state and fetched all the block bodies we will fetch and store the receipts for the range between the current head and the pivot (including it), and at the same time store all blocks in the same range and execute all blocks after the pivot (like in full sync). + This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/snap_sync.jpg). @@ -13,7 +16,9 @@ This diagram illustrates the process described above: During snap sync we need to fully rebuild the pivot block's state. We can divide this process into the initial sync and the healing phase. For the first phase we will spawn two processes, the `bytecode_fetcher` and the `storage_fetcher` which will both remain active and listening for requests from the main rebuild process which they will then queue and process in fixed size batches (more on this later). It will then request the full extent of accounts from the pivot block's state trie via p2p snap requests. For each obtained range we will send the account's code hash and storage root to the `bytecode_fetcher` and `storage_fetcher` respectively for fetching. Once we fetch all accounts (or the account state is no longer available), we will signal the `storage_fetcher` to finish all pending requests and move on to the next phase, while keeping the `bytecode_fetcher` active. + In the healing phase we will spawn another queue-like process called `storage_healer`, and we will begin requesting state trie nodes. We will begin by requesting the pivot block's state's root node proceed by requesting the current node's children (if they are not already part of the state) until we have the full trie stored (aka all child nodes are known). For each fetched leaf node we will send its code hash to the `bytecode_fetcher` and account hash to the `storage_healer`. + The `storage_healer` will contain a list of pending account hashes and paths. And will add new entries by either adding the root node of an account's storage trie when receiving an account hash from the main process or by adding the unknown children of nodes returned by peers. This diagram illustrates the process described above: @@ -23,6 +28,7 @@ This diagram illustrates the process described above: To exemplify how queue-like processes work we will explain how the `bytecode_fetcher` works: The `bytecode_fetcher` has its own channel where it receives code hashes from an active `rebuild_state_trie` process. Once a code hash is received, it is added to a pending queue. When the queue has enough messages for a full batch it will request a batch of bytecodes via snap p2p and store them. If a bytecode could not be fetched by the request (aka, we reached the response limit) it is added back to the pending queue. After the whole state is synced `fetch_snap_state` will send an empty list to the `bytecode_fetcher` to signal the end of the requests so it can request the last (incomplete) bytecode batch and end gracefully. + This diagram illustrates the process described above: ![snap_sync](/crates/networking/docs/diagrams/bytecode_fetcher.jpg) From d52f95d0806b657481051ffa7f970dd7820e43fd Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 17 Dec 2024 17:04:02 -0300 Subject: [PATCH 337/345] Improve doc --- crates/networking/docs/Sync.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/docs/Sync.md b/crates/networking/docs/Sync.md index d0ee3dc01..836b804e1 100644 --- a/crates/networking/docs/Sync.md +++ b/crates/networking/docs/Sync.md @@ -4,7 +4,7 @@ A snap sync cycle begins by fetching all the block headers (via eth p2p) between the current head (latest canonical block) and the sync head (block hash sent by a forkChoiceUpdate). -We will then fetch the block bodies from each header and at the same time we will select a pivot block (sync head - 64) and start rebuilding its state via snap p2p requests, if the pivot were to become stale during this rebuild we will select a newer pivot (sync head) and restart it. +We will then fetch the block bodies from each header and at the same time select a pivot block (sync head - 64) and start rebuilding its state via snap p2p requests, if the pivot were to become stale during this rebuild we will select a newer pivot (sync head) and restart it. After we fully rebuilt the pivot state and fetched all the block bodies we will fetch and store the receipts for the range between the current head and the pivot (including it), and at the same time store all blocks in the same range and execute all blocks after the pivot (like in full sync). From ff1fd94679eae26554bf566d34a27bad1f6ad206 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 17 Dec 2024 17:05:14 -0300 Subject: [PATCH 338/345] Add hive workflow --- .github/workflows/ci_l1.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/ci_l1.yaml b/.github/workflows/ci_l1.yaml index 264f40824..19b558faf 100644 --- a/.github/workflows/ci_l1.yaml +++ b/.github/workflows/ci_l1.yaml @@ -164,6 +164,9 @@ jobs: - name: "Cancun Engine tests" simulation: ethereum/engine test_pattern: "engine-cancun/Blob Transactions On Block 1|Blob Transaction Ordering, Single|Blob Transaction Ordering, Multiple Accounts|Replace Blob Transactions|Parallel Blob Transactions|ForkchoiceUpdated|GetPayload|NewPayloadV3 After Cancun|NewPayloadV3 Before Cancun|NewPayloadV3 Versioned Hashes|Incorrect BlobGasUsed|Bad Hash|ParentHash equals BlockHash|RPC:|in ForkchoiceState|Unknown|Invalid PayloadAttributes|Unique|Re-Execute Payload|In-Order Consecutive Payload|Multiple New Payloads|Valid NewPayload->|NewPayload with|Payload Build after|Build Payload with|Invalid Missing Ancestor ReOrg, StateRoot|Re-Org Back to|Re-org to Previously|Safe Re-Org to Side Chain|Transaction Re-Org, Re-Org Back In|Re-Org Back into Canonical Chain, Depth=5|Suggested Fee Recipient Test|PrevRandao Opcode|Invalid NewPayload, [^R][^e]|Fork ID: Genesis=0, Cancun=0|Fork ID: Genesis=0, Cancun=1|Fork ID: Genesis=0, Cancun=2 |Fork ID: Genesis=0, Cancun=2, BlocksBeforePeering=1|Fork ID: Genesis=0, Cancun=2, Shanghai=[^2]|Fork ID Genesis=1, Cancun=2, Shanghai=2" + - name: "Sync" + simulation: ethereum/sync + test_pattern: "" steps: - name: Download artifacts uses: actions/download-artifact@v4 From 9e41b62872aed20036c637b80d2b5f592bce7cb4 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 18 Dec 2024 13:07:38 -0300 Subject: [PATCH 339/345] Rename inner_encode_receipt -> encode_inner --- crates/common/types/block.rs | 2 +- crates/common/types/receipt.rs | 10 +++++----- crates/networking/rpc/eth/block.rs | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/crates/common/types/block.rs b/crates/common/types/block.rs index b6a039858..fb4b3eaf9 100644 --- a/crates/common/types/block.rs +++ b/crates/common/types/block.rs @@ -241,7 +241,7 @@ pub fn compute_receipts_root(receipts: &[Receipt]) -> H256 { let iter = receipts .iter() .enumerate() - .map(|(idx, receipt)| (idx.encode_to_vec(), receipt.inner_encode_receipt())); + .map(|(idx, receipt)| (idx.encode_to_vec(), receipt.encode_inner())); Trie::compute_hash_from_unsorted_iter(iter) } diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index b7219d7a0..997614997 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -35,16 +35,16 @@ impl Receipt { // - https://eips.ethereum.org/EIPS/eip-2718 // - https://github.com/ethereum/go-ethereum/blob/330190e476e2a2de4aac712551629a4134f802d5/core/types/receipt.go#L143 // We've noticed the are some subtleties around encoding receipts and transactions. - // First, `inner_encode_receipt` will encode a receipt according + // First, `encode_inner` will encode a receipt according // to the RLP of its fields, if typed, the RLP of the fields // is padded with the byte representing this type. // For P2P messages, receipts are re-encoded as bytes // (see the `encode` implementation for receipt). // For debug and computing receipt roots, the expected - // RLP encodings are the ones returned by `inner_encode_receipt`. + // RLP encodings are the ones returned by `encode_inner`. // On some documentations, this is also called the `consensus-encoding` // for a receipt. - pub fn inner_encode_receipt(&self) -> Vec { + pub fn encode_inner(&self) -> Vec { let mut encode_buff = match self.tx_type { TxType::Legacy => { vec![] @@ -81,11 +81,11 @@ impl RLPEncode for Receipt { fn encode(&self, buf: &mut dyn bytes::BufMut) { match self.tx_type { TxType::Legacy => { - let legacy_encoded = self.inner_encode_receipt(); + let legacy_encoded = self.encode_inner(); buf.put_slice(&legacy_encoded); } _ => { - let typed_recepipt_encoded = self.inner_encode_receipt(); + let typed_recepipt_encoded = self.encode_inner(); let bytes = Bytes::from(typed_recepipt_encoded); bytes.encode(buf); } diff --git a/crates/networking/rpc/eth/block.rs b/crates/networking/rpc/eth/block.rs index ddab39a56..60f4675bd 100644 --- a/crates/networking/rpc/eth/block.rs +++ b/crates/networking/rpc/eth/block.rs @@ -297,7 +297,7 @@ impl RpcHandler for GetRawReceipts { }; let receipts: Vec = get_all_block_receipts(block_number, header, body, storage)? .iter() - .map(|receipt| format!("0x{}", hex::encode(receipt.inner_encode_receipt()))) + .map(|receipt| format!("0x{}", hex::encode(receipt.encode_inner()))) .collect(); serde_json::to_value(receipts).map_err(|error| RpcErr::Internal(error.to_string())) } From 7c12d7f927e058c055607334a22c5e28a84d69bb Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 18 Dec 2024 14:38:42 -0300 Subject: [PATCH 340/345] Add decode_inner for Receipt + tests --- crates/common/types/receipt.rs | 120 ++++++++++++++++++++++++++++++++- 1 file changed, 118 insertions(+), 2 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index 997614997..c467b86ed 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -44,6 +44,10 @@ impl Receipt { // RLP encodings are the ones returned by `encode_inner`. // On some documentations, this is also called the `consensus-encoding` // for a receipt. + + /// Encodes Receipts in the following formats: + /// A) Legacy receipts: rlp(receipt) + /// B) Non legacy receipts: tx_type | rlp(receipt). pub fn encode_inner(&self) -> Vec { let mut encode_buff = match self.tx_type { TxType::Legacy => { @@ -61,6 +65,45 @@ impl Receipt { .finish(); encode_buff } + + /// Decodes Receipts in the following formats: + /// A) Legacy receipts: rlp(receipt) + /// B) Non legacy receipts: tx_type | rlp(receipt). + pub fn decode_inner(rlp: &[u8]) -> Result { + // Obtain TxType + let (tx_type, rlp) = match rlp.first() { + Some(tx_type) if *tx_type < 0x7f => { + let tx_type = match tx_type { + 0x0 => TxType::Legacy, + 0x1 => TxType::EIP2930, + 0x2 => TxType::EIP1559, + 0x3 => TxType::EIP4844, + 0x7e => TxType::Privileged, + ty => { + return Err(RLPDecodeError::Custom(format!( + "Invalid transaction type: {ty}" + ))) + } + }; + (tx_type, &rlp[1..]) + } + _ => (TxType::Legacy, rlp), + }; + let decoder = Decoder::new(rlp)?; + let (succeeded, decoder) = decoder.decode_field("succeeded")?; + let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative_gas_used")?; + let (bloom, decoder) = decoder.decode_field("bloom")?; + let (logs, decoder) = decoder.decode_field("logs")?; + decoder.finish()?; + + Ok(Receipt { + tx_type, + succeeded, + cumulative_gas_used, + bloom, + logs, + }) + } } fn bloom_from_logs(logs: &[Log]) -> Bloom { @@ -76,7 +119,7 @@ fn bloom_from_logs(logs: &[Log]) -> Bloom { impl RLPEncode for Receipt { /// Receipts can be encoded in the following formats: - /// A) Legacy receipts: rlp(LegacyTransaction) + /// A) Legacy receipts: rlp(receipt) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn encode(&self, buf: &mut dyn bytes::BufMut) { match self.tx_type { @@ -95,7 +138,7 @@ impl RLPEncode for Receipt { impl RLPDecode for Receipt { /// Receipts can be encoded in the following formats: - /// A) Legacy receipts: rlp(LegacyTransaction) + /// A) Legacy receipts: rlp(receipt) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { if is_encoded_as_bytes(rlp)? { @@ -181,3 +224,76 @@ impl RLPDecode for Log { Ok((log, decoder.finish()?)) } } + +#[cfg(test)] +mod test { + use super::*; + + #[test] + fn test_encode_decode_receipt_legacy() { + let receipt = Receipt { + tx_type: TxType::Legacy, + succeeded: true, + cumulative_gas_used: 1200, + bloom: Bloom::random(), + logs: vec![Log { + address: Address::random(), + topics: vec![], + data: Bytes::from_static(b"foo"), + }], + }; + let encoded_receipt = receipt.encode_to_vec(); + assert_eq!(receipt, Receipt::decode(&encoded_receipt).unwrap()) + } + + #[test] + fn test_encode_decode_receipt_non_legacy() { + let receipt = Receipt { + tx_type: TxType::EIP4844, + succeeded: true, + cumulative_gas_used: 1500, + bloom: Bloom::random(), + logs: vec![Log { + address: Address::random(), + topics: vec![], + data: Bytes::from_static(b"bar"), + }], + }; + let encoded_receipt = receipt.encode_to_vec(); + assert_eq!(receipt, Receipt::decode(&encoded_receipt).unwrap()) + } + + #[test] + fn test_encode_decode_inner_receipt_legacy() { + let receipt = Receipt { + tx_type: TxType::Legacy, + succeeded: true, + cumulative_gas_used: 1200, + bloom: Bloom::random(), + logs: vec![Log { + address: Address::random(), + topics: vec![], + data: Bytes::from_static(b"foo"), + }], + }; + let encoded_receipt = receipt.encode_inner(); + assert_eq!(receipt, Receipt::decode_inner(&encoded_receipt).unwrap()) + } + + #[test] + fn test_encode_decode_receipt_inner_non_legacy() { + let receipt = Receipt { + tx_type: TxType::EIP4844, + succeeded: true, + cumulative_gas_used: 1500, + bloom: Bloom::random(), + logs: vec![Log { + address: Address::random(), + topics: vec![], + data: Bytes::from_static(b"bar"), + }], + }; + let encoded_receipt = receipt.encode_inner(); + assert_eq!(receipt, Receipt::decode_inner(&encoded_receipt).unwrap()) + } +} From 12b2c777bbffba3361a49d8f7dfea49a8804b49e Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 18 Dec 2024 14:49:25 -0300 Subject: [PATCH 341/345] Simplify logic of RLPDecode impl --- crates/common/types/receipt.rs | 57 +++++++++++++--------------------- 1 file changed, 22 insertions(+), 35 deletions(-) diff --git a/crates/common/types/receipt.rs b/crates/common/types/receipt.rs index c467b86ed..31ae45652 100644 --- a/crates/common/types/receipt.rs +++ b/crates/common/types/receipt.rs @@ -141,11 +141,9 @@ impl RLPDecode for Receipt { /// A) Legacy receipts: rlp(receipt) /// B) Non legacy receipts: rlp(Bytes(tx_type | rlp(receipt))). fn decode_unfinished(rlp: &[u8]) -> Result<(Self, &[u8]), RLPDecodeError> { - if is_encoded_as_bytes(rlp)? { + let (tx_type, rlp) = if is_encoded_as_bytes(rlp)? { let payload = get_rlp_bytes_item_payload(rlp)?; - let tx_type = payload.first().ok_or(RLPDecodeError::InvalidLength)?; - let receipt_encoding = &payload[1..]; - let tx_type = match tx_type { + let tx_type = match payload.first().ok_or(RLPDecodeError::InvalidLength)? { 0x0 => TxType::Legacy, 0x1 => TxType::EIP2930, 0x2 => TxType::EIP1559, @@ -157,38 +155,27 @@ impl RLPDecode for Receipt { ))) } }; - let decoder = Decoder::new(receipt_encoding)?; - let (succeeded, decoder) = decoder.decode_field("succeded")?; - let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative gas used")?; - let (bloom, decoder) = decoder.decode_field("bloom")?; - let (logs, decoder) = decoder.decode_field("logs")?; - Ok(( - Receipt { - tx_type, - succeeded, - bloom, - logs, - cumulative_gas_used, - }, - decoder.finish()?, - )) + (tx_type, &payload[1..]) } else { - let decoder = Decoder::new(rlp)?; - let (succeeded, decoder) = decoder.decode_field("succeded")?; - let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative gas used")?; - let (bloom, decoder) = decoder.decode_field("bloom")?; - let (logs, decoder) = decoder.decode_field("logs")?; - Ok(( - Receipt { - tx_type: TxType::Legacy, - succeeded, - bloom, - logs, - cumulative_gas_used, - }, - decoder.finish()?, - )) - } + (TxType::Legacy, rlp) + }; + + let decoder = Decoder::new(rlp)?; + let (succeeded, decoder) = decoder.decode_field("succeeded")?; + let (cumulative_gas_used, decoder) = decoder.decode_field("cumulative_gas_used")?; + let (bloom, decoder) = decoder.decode_field("bloom")?; + let (logs, decoder) = decoder.decode_field("logs")?; + + Ok(( + Receipt { + tx_type, + succeeded, + cumulative_gas_used, + bloom, + logs, + }, + decoder.finish()?, + )) } } From a6a52c730a3f7ba2b9b52c50237bc99b1ff847d6 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 18 Dec 2024 15:56:53 -0300 Subject: [PATCH 342/345] Fix typos in diagram --- .../docs/diagrams/rebuild_state_trie.jpg | Bin 55483 -> 55487 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/crates/networking/docs/diagrams/rebuild_state_trie.jpg b/crates/networking/docs/diagrams/rebuild_state_trie.jpg index e9255ec89ba0198df6c875ed8540e3ed65331e40..4ac255e822e5351f00071630a8245d6a975a3ff5 100644 GIT binary patch delta 2329 zcmV+!3Fh{@u>-%c1F(uRf9VYh4L-xe9yYo7A%Efj00sO*kHNZmytT6N1-;gZqUtZ= zBpzO$soq@8Z*460P_jfH$5Mm%%b@bHSNLB%rta0_)3k`D)O9NzTg_K7{hm9C ztYJyR6l8_Q@0Nf+A-}UxGd2i+Kl0cA03C1epbaW3H~U{t{{Wu9`0Ia#AKLnV{Pq6; z$6Nd;14@d|T~@`;5<vtT^R0wgO>O}@R-G@V~U(=H^p)Ai`$hWZs_vf?-;S>spsSrJ%!imQ`&oh%GN z5o1M-1{FX6dkS@vQA`#mSFR`gn$M2^0PJ({hyI(qMP5f!;y-(ry{F^7lZ~Ghk9sdBP{{RtJmQVp+ z@B31J$<6#-{{XSSU;hBpg!SQ*;Y$;LKpqiA6i@+06i@+Qr`DR@x8ltXAAxq-jfQ;}Z6MK2!PFv>30Lg57ch^s3zHnBE)E)9!xj01U2ps>{{W2N z@Pn_zKloTW;P8^6h-o7F;8^)h);}Z7+2x+QF}FP8fIj@!e`T-P*GhsP1*1cMb*yNy zcs}Y|%SYE`y}fO6=EW{n_%)L0R`#<;bpHTlcrJH>k*RlPV7&gD@IUr>@$RL0sc1e2 zxza5B6XGuv>9c9J`lZ#y$B1q8O(Nq`wS>I8j8_uLZL!P&0Y*Dlcv5~HxcH6ndhf$J zM}sbWd#y{L_fPMk$8YhP3@OG7d;k)_9 zo99b&M0JR@qz0<9xGF!-6 z*&&uY=_R)EqYSGCU7T*mtN26VC&C|!J`cC>N$^F_!|x7hUOU%?)ui{gvzCzwp+7j8Uw#+3zQk&e+5V zkIIFlD7yh6Fg{RD++^0qx$%4Br-^U$%~Qlb68uA{UTBiXJ%ra9mDI0mG$5>U!p$); zx`sG)!*Y4#&<9idPxuqzkA+*oJ|g&Kn$2Iv?<6U>g{{Rwo*YQ8Z9}{1AlU8(Cjoza)ms5FZ_fR~J#46Nxs(6Sy@2=V zL9*~4!G8f=c$deX4DijCrQo}t3HZjwWz)4iO$GDmuVCqIc@Cu-URhnBnMKG(F|c4^ z*g>z)IesgD{BEB@xA46G01@^1G;2#yWet;RH%8t&Wn^WAV)J%{1~kHffw6J-Mk{!u z;#bBm8(f>;7<@q1^{LIImIdzbt|lhZZKhZb`A%bys+V4lKr9L7fIjZie`nv>;ru_T z{ARz>bp10+ywZFpaciM!m)hg$)^{56UHzfF{Vv`R^H6-frrU8R$QnWu8{!5WNuzk~dJt!TH<*(RKmT3=j4BH2cAmf@06L`Y6m2g;|A0i@IZBYa)b zd_SysPfPLcwP~XGhFNFStm3`6NG~OvZJFncNESwvaUoz!jvF)q)qEneT|3}U#-AJ6 z05wZnZ5Kw4LH@EEU0&_Y)2Ea2$uRsf6;C z#N69hEz?Ci$0MP5V!2Jtxl&Nue?T9fcb^!3I_e%TwDJA-#I1gR64*f2*7}p`7bfl~ z8}CGu8F#8SPUQy;lbqz%9=YSsiry#B{3M<_)^%Knl>Msu;$lP+yzyszr#@V=$s%X& zyR!@uXan>L&*5*vy*J|rfnfN1rP#&bKOZ-WJT0luVR@)c9mE#)sQA6JV%1|yn@ig> zBzF>kS}!ppc(#-|{MEDn0FO97>ze-nD)q=dH+)&S*Jtrpi@Z~)c&}ScCih9x?)7V_ z?%z&LLNBx2O9z=FVcG_f0pkqfPrjhnltUkr3~^^W6fS*mF+dWNQNS&;c}GH#-ejvJ delta 2249 zcmV;)2sZb>u>-rY1F(uRJ=?rIei!IIFHLL0TGpB1{{R|Wcs<4Ehx|X{y9jKp@8s3N zcyz6MReRf87W**!MClBYxK?=MIcLBJVNY#EI z>6h2qi|X29ThFGxhh+Eb=S*W0r=8|C&8n))At0D>kUaCko2he8vD3h18H%fIlFA;UND2rXTqK0RI45 zx(KsmGbjj?-8vVO*}Ubhlc7l~1%Ln7{C|^v)hDxLJjwzu zD!2AVpZ@?nzwze(02v(BA9M=*Z7CW}TTV3bN7f%|*(&PKZ6fe+|>mnWv z)`B9&iy90nfB^Os>XWe-7ALPU9sdB>=iv|iH+YJ?j=ga|G>u96ti>wH<5W{{V?C{vU$=7rnQ))w~m`+Cie5gQ!I$60g~AE@2;N z7bZDMTpTpJhAZ<1y5IO&{{R`k;Rj!afAFw%!QmxC5Yk2Uz_Ie1tbRwEv&%hpV{UoH z0Dbwc{>xvpu9XBn3r2?PSkYqeebl#>kFLvqdwSaC&5B&D@M|T~t?g!x>Hf;_T<--V zQtr&bdHpxwf9&()-AeOP(0mVbq*?eU#9k@WX47r;ORI~I5ZmaQMaHFT33+!It|gM& zW0(R0jCQW@r2IQ^@f+gx--dLL23+{}T9-la{ISocShelJwzHNKn4Spy&9Ss+lqv>) z!}oDn8fS_AA!yf;Y8pq2^-C+LbpxuTu-#1?uA)Ixb_&P{oPt4@@8FS~PzS&M)Q{oI zuK@f5)HF{G%i!%Q{{X{x^NlyomgmS>HM{NY6qf|b(aFq_`HV(YS0=JX4Snd>gjdE8G&e%T!c-zFk z5I!X8QR^QN{8_1Zn))SVj%`ZjYq=vvV&ze$k&3BePSJsr(zb5CBz$1K@HAfxZ#;di zPvHncs%Z~>aMlWlcPfi>FPF7Q%C6i4$N+aK`Gp96eD;vHY$C7rQ|5g(O*3rJCR0zzPXpq;qMt&MZy_s35X-|3pCh<+vbhf=)J zC60Rut~D#EUe;(qSmlM9Vqz;ZAL-@Yb`;Bft~{wltmapEI?}QT^@~g(W?~VQ>>#yQ}h(0F1@g}V3u^YWcX)dPn z)9#>o9f(!Q!j1=FUB;)O_}T};-2IVP5Es~?lr?-vJ}-Q9T_VrJ>+vtdBTUnO?bg=X)M{5k7+{T& z(lk#uYO1toQ~}1t$K70WdOwSPBzSuMEmK1Aj;p8L!+zSLO9kBVy5G8#YoN;@K64`# zX+H5%K{zB(2k2B^vv0t?F5kr;6ZQQ&K#N67Zw>zdWZE5D#1~o&%dGveWW9!K3purA z4)$g^BDg>jLgGgRf8DEQleM6nfzsGOf8=wrR!)w< z{T1;m;T5;V&-f_4{{VpeP%dmV4-oi{2rh18Mp9uhBNUDHun#Ugv^!hoVe+>_@%Mu@ z{YSytUy3vs{JY&R#oB9W*S7Mh3zU^EE$2cT<&p{FM^oGeL-M;)imUi8CKfsSH7MBeSd zS<+y!@h#S|Z5{84E&M$c8aTCBE@iybbojLy86~>%?x3?*V!}P1UiMMk-*^+@&%-SX z;x~i59iUAQAU#o4BBFy%J1i-l*9-lpHorbCXzl=Z`)sc%MJ;l6dP` z)p8L%V;6w@eBLPV zwx>OX=Akrq5L?=#Wa9SERgErfFKo<_+)4pxyu^*-+EC~7R?q%C;Qs)wYy7L%Ao$(! zX6Id>#a=G)PNCwxZ8V$RCr`W8uBW?wIW-8r&v7gsWQ~Vt8bk+-Gl@R>gI-Y#eoQgN Xoa|7!^}xjdNs|EuD**?yd`Cl(MGcpY From d7e9eee10804adbdda8d51a9164268b36719d950 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Wed, 18 Dec 2024 17:45:35 -0300 Subject: [PATCH 343/345] Add receipt fetching to snap-sync --- crates/networking/p2p/peer_channels.rs | 41 ++++++++++++++++++++++++-- crates/networking/p2p/sync.rs | 36 ++++++++++++++++++++-- 2 files changed, 72 insertions(+), 5 deletions(-) diff --git a/crates/networking/p2p/peer_channels.rs b/crates/networking/p2p/peer_channels.rs index 2f6ae9717..ba01aa1fa 100644 --- a/crates/networking/p2p/peer_channels.rs +++ b/crates/networking/p2p/peer_channels.rs @@ -2,7 +2,7 @@ use std::{collections::BTreeMap, sync::Arc, time::Duration}; use bytes::Bytes; use ethrex_core::{ - types::{AccountState, BlockBody, BlockHeader}, + types::{AccountState, BlockBody, BlockHeader, Receipt}, H256, U256, }; use ethrex_rlp::encode::RLPEncode; @@ -12,8 +12,11 @@ use tokio::sync::{mpsc, Mutex}; use crate::{ rlpx::{ - eth::blocks::{ - BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, + eth::{ + blocks::{ + BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders, BLOCK_HEADER_LIMIT, + }, + receipts::{GetReceipts, Receipts}, }, snap::{ AccountRange, ByteCodes, GetAccountRange, GetByteCodes, GetStorageRanges, GetTrieNodes, @@ -123,6 +126,38 @@ impl PeerChannels { (!block_bodies.is_empty() && block_bodies.len() <= block_hashes_len).then_some(block_bodies) } + /// Requests all receipts in a set of blocks from the peer given their block hashes + /// Returns the lists of receipts or None if: + /// - There are no available peers (the node just started up or was rejected by all other nodes) + /// - The response timed out + /// - The response was empty or not valid + pub async fn request_receipts(&self, block_hashes: Vec) -> Option>> { + let block_hashes_len = block_hashes.len(); + let request_id = rand::random(); + let request = RLPxMessage::GetReceipts(GetReceipts { + id: request_id, + block_hashes, + }); + self.sender.send(request).await.ok()?; + let mut receiver = self.receiver.lock().await; + let receipts = tokio::time::timeout(PEER_REPLY_TIMOUT, async move { + loop { + match receiver.recv().await { + Some(RLPxMessage::Receipts(Receipts { id, receipts })) if id == request_id => { + return Some(receipts) + } + // Ignore replies that don't match the expected id (such as late responses) + Some(_) => continue, + None => return None, + } + } + }) + .await + .ok()??; + // Check that the response is not empty and does not contain more bodies than the ones requested + (!receipts.is_empty() && receipts.len() <= block_hashes_len).then_some(receipts) + } + /// Requests an account range from the peer given the state trie's root and the starting hash (the limit hash will be the maximum value of H256) /// Will also return a boolean indicating if there is more state to be fetched towards the right of the trie /// Returns the response message or None if: diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 5e1399b1a..31e911a22 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -162,7 +162,11 @@ impl SyncManager { let all_block_bodies = fetch_bodies_handle.await??; // For all blocks before the pivot: Store the bodies and fetch the receipts // For all blocks after the pivot: Process them fully - // let store_receipts_handle = tokio::spawn(store_receipts(all_block_hashes[pivot_index..])); + let store_receipts_handle = tokio::spawn(store_receipts( + all_block_hashes[pivot_idx..].to_vec(), + self.peers.clone(), + store.clone(), + )); for (hash, (header, body)) in all_block_hashes.into_iter().zip( all_block_headers .into_iter() @@ -177,7 +181,7 @@ impl SyncManager { ethrex_blockchain::add_block(&Block::new(header, body), &store)?; } } - // store_receipts.await??; + store_receipts_handle.await??; self.last_snap_pivot = pivot_number; } SyncMode::Full => { @@ -256,6 +260,34 @@ async fn fetch_block_bodies( Ok(all_block_bodies) } +/// Fetches all receipts for the given block hashes via p2p and stores them +async fn store_receipts( + mut block_hashes: Vec, + peers: Arc>, + store: Store, +) -> Result<(), SyncError> { + loop { + let peer = peers.lock().await.get_peer_channels().await; + debug!("Requesting Block Headers "); + if let Some(receipts) = peer.request_receipts(block_hashes.clone()).await { + debug!(" Received {} Receipts", receipts.len()); + // Track which blocks we have already fetched receipts for + for (block_hash, receipts) in block_hashes + .drain(0..receipts.len()) + .into_iter() + .zip(receipts) + { + store.add_receipts(block_hash, receipts)?; + } + // Check if we need to ask for another batch + if block_hashes.is_empty() { + break; + } + } + } + Ok(()) +} + /// Rebuilds a Block's state trie by requesting snap state from peers, also performs state healing /// Returns true if all state was fetched or false if the block is too old and the state is no longer available async fn rebuild_state_trie( From 32be829b7fb739571d32931b37bbf44c4484d88d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 19 Dec 2024 10:49:31 -0300 Subject: [PATCH 344/345] Clippy --- crates/networking/p2p/sync.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 31e911a22..60d618623 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -272,11 +272,7 @@ async fn store_receipts( if let Some(receipts) = peer.request_receipts(block_hashes.clone()).await { debug!(" Received {} Receipts", receipts.len()); // Track which blocks we have already fetched receipts for - for (block_hash, receipts) in block_hashes - .drain(0..receipts.len()) - .into_iter() - .zip(receipts) - { + for (block_hash, receipts) in block_hashes.drain(0..receipts.len()).zip(receipts) { store.add_receipts(block_hash, receipts)?; } // Check if we need to ask for another batch From 72b2156772ad6ba3d3afc8a5e02ad0694147bc1b Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 19 Dec 2024 14:22:41 -0300 Subject: [PATCH 345/345] Update comments --- crates/networking/p2p/sync.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/networking/p2p/sync.rs b/crates/networking/p2p/sync.rs index 5e1399b1a..df2998fcf 100644 --- a/crates/networking/p2p/sync.rs +++ b/crates/networking/p2p/sync.rs @@ -283,7 +283,6 @@ async fn rebuild_state_trie( let mut current_state_root = *EMPTY_TRIE_HASH; // Fetch Account Ranges // If we reached the maximum amount of retries then it means the state we are requesting is probably old and no longer available - // In that case we will delegate the work to state healing for _ in 0..MAX_RETRIES { let peer = peers.clone().lock().await.get_peer_channels().await; debug!("Requesting Account Range for state root {state_root}, starting hash: {start_account_hash}"); @@ -344,7 +343,8 @@ async fn rebuild_state_trie( debug!("Completed state sync for state root {state_root}"); true } else { - // If failed to fetch the full state leave the rest to state healing + // Perform state healing to fix any potential inconsistency in the rebuilt tries + // As we are not fetching different chunks of the same trie this step is not necessary heal_state_trie(bytecode_sender.clone(), state_root, store, peers).await? }; // Send empty batch to signal that no more batches are incoming