From 02fe7440d04f9dbae5b41e01a4dcdcc329a843e5 Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 22 Oct 2024 13:11:34 -0300 Subject: [PATCH 001/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] :( --- 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] [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/246] [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/246] 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/246] [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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] [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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] [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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] 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/246] [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/246] 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/246] 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/246] 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/246] 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 da6cff86a058b01502f5495743e21f10ae6659be Mon Sep 17 00:00:00 2001 From: fmoletta Date: Tue, 26 Nov 2024 18:19:34 -0300 Subject: [PATCH 223/246] 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 224/246] 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 225/246] 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 226/246] 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 227/246] 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 228/246] 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 229/246] 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 230/246] 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 231/246] 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 232/246] 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 233/246] 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 234/246] 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 235/246] 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 236/246] 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 237/246] 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 238/246] 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 239/246] 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 240/246] 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 241/246] 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 242/246] 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 e4ef02db443929ebe0059106ee55aa30e144190d Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 28 Nov 2024 12:36:22 -0300 Subject: [PATCH 243/246] Add syncmode to the CLI doc --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 688efdf0b..cef592c32 100644 --- a/README.md +++ b/README.md @@ -261,6 +261,7 @@ ethrex supports the following command line arguments: - `--discovery.port `: UDP port for P2P discovery. Default value: 30303. - `--bootnodes `: Comma separated enode URLs for P2P discovery bootstrap. - `--log.level `: The verbosity level used for logs. Default value: info. possible values: info, debug, trace, warn, error +- `--syncmode `: The way in which the node will sync its state. Can be either "full" or "snap" with "snap" as default value. # ethrex L2 From 0c21b5e177455730d612e6518e2407f64dcaf9ab Mon Sep 17 00:00:00 2001 From: fmoletta Date: Thu, 28 Nov 2024 12:45:23 -0300 Subject: [PATCH 244/246] remove potential panic when indexing kademlia table --- crates/networking/p2p/kademlia.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 2dcfc99b6..003371ad4 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -261,11 +261,12 @@ impl KademliaTable { /// This function should be called each time a connection is established so the backend can send requests to the peers 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) - { + if let Some(peer) = self.buckets.get_mut(bucket_idx).and_then(|bucket| { + bucket + .peers + .iter_mut() + .find(|peer| peer.node.node_id == node_id) + }) { peer.channels = Some(channels) } } From dd934f58785365abb56ea684309a7e83c43364d7 Mon Sep 17 00:00:00 2001 From: fmoletta <99273364+fmoletta@users.noreply.github.com> Date: Thu, 28 Nov 2024 12:46:19 -0300 Subject: [PATCH 245/246] Fix typo Co-authored-by: Francisco Krause Arnim <56402156+fkrause98@users.noreply.github.com> --- crates/networking/p2p/kademlia.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/networking/p2p/kademlia.rs b/crates/networking/p2p/kademlia.rs index 003371ad4..c87e47733 100644 --- a/crates/networking/p2p/kademlia.rs +++ b/crates/networking/p2p/kademlia.rs @@ -315,7 +315,7 @@ pub struct PeerData { pub liveness: u16, /// if a revalidation was sent to the peer, the bool marks if it has answered pub revalidation: Option, - /// communication channels bewteen the peer data and its active connection + /// communication channels between the peer data and its active connection pub channels: Option, } From 75be45253b15cf7271a33a01605f54a1ef1613fe Mon Sep 17 00:00:00 2001 From: fmoletta <99273364+fmoletta@users.noreply.github.com> Date: Thu, 28 Nov 2024 15:52:35 -0300 Subject: [PATCH 246/246] Fix merge --- crates/networking/p2p/Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/networking/p2p/Cargo.toml b/crates/networking/p2p/Cargo.toml index 010c2ce9c..10c57bd41 100644 --- a/crates/networking/p2p/Cargo.toml +++ b/crates/networking/p2p/Cargo.toml @@ -11,7 +11,6 @@ ethrex-blockchain.workspace = true ethrex-rlp.workspace = true ethrex-storage.workspace = true ethrex-trie.workspace = true -ethrex-blockchain.workspace = true tracing.workspace = true tokio.workspace = true