From 02fe7440d04f9dbae5b41e01a4dcdcc329a843e5 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 13:11:34 -0300
Subject: [PATCH 001/155] 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 e8d27b29c0..2a93559ede 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<impl Iterator<Item = (H256, AccountState)>, 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 b43559a160..46113c4e5d 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 0000000000..07a4475a02
--- /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<NodeHash>,
+}
+
+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<Self::Item> {
+        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<Item = (PathRLP, ValueRLP)> {
+        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 <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 13:26:15 -0300
Subject: [PATCH 002/155] 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 2919999f41..4e998dedb4 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 <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 13:55:40 -0300
Subject: [PATCH 003/155] 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 4e998dedb4..8bae953745 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 <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 17:10:44 -0300
Subject: [PATCH 004/155] 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 0000000000..92812556ed
--- /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<AccountRange, StoreError> {
+    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 <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 17:11:00 -0300
Subject: [PATCH 005/155] 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 b99b6eb582..a78f1806d0 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 <fedemoletta@hotmail.com>
Date: Tue, 22 Oct 2024 18:52:28 -0300
Subject: [PATCH 006/155] 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 847a1c6283..5c5ba23f2e 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<aes::Aes256>;
 

From 41dcaff2713bf83b7782f5ae5db573af04ccb489 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 12:27:03 -0300
Subject: [PATCH 007/155] 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 cdfafd1b69..98082e0550 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<u8>,
-}
-
-impl AccountRange {
-    pub fn new(id: u64, accounts: Vec<(H256, AccountState)>, proof: Vec<u8>) -> Self {
-        Self {
-            id,
-            accounts,
-            proof,
-        }
-    }
+    pub id: u64,
+    pub accounts: Vec<(H256, AccountState)>,
+    pub proof: Vec<Vec<u8>>,
 }
 
 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<u8>, _) = 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 92812556ed..aa47d10d49 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<AccountRange, StoreError> {
+pub fn process_account_range_request(
+    request: GetAccountRange,
+    store: Store,
+) -> Result<AccountRange, StoreError> {
     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 2a93559ede..9618d5cf32 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<impl Iterator<Item = (H256, AccountState)>, StoreError> {
-        Ok(self
-            .engine
+    pub fn iter_accounts(&self, state_root: H256) -> impl Iterator<Item = (H256, AccountState)> {
+        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<Vec<Vec<u8>>, 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 <esteban.dimitroff@lambdaclass.com>
Date: Wed, 23 Oct 2024 15:44:13 -0300
Subject: [PATCH 008/155] 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 b99b6eb582..1b368799e8 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<TcpStream>) {
     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 847a1c6283..5be4136e20 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<S> {
     signer: SigningKey,
     state: RLPxConnectionState,
     stream: S,
+    storage: Store,
     capabilities: Vec<(String, u8)>,
 }
 
 impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
             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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
 
         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 e74d4339a9..4177ea10fe 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 e94b2ccb80..480e180bf8 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 16:07:45 -0300
Subject: [PATCH 009/155] 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 6b93886d19..2f25755d7c 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<T: RLPEncodeSlim> RLPEncodeSlim for Vec<T> {
-    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<S: RLPEncodeSlim, T: RLPEncodeSlim> 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 207545e47f..01e228e51d 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<T: RLPEncodeSlim>(mut self, value: &T) -> Self {
-        <T as RLPEncodeSlim>::encode(value, &mut self.temp_buf);
-        self
-    }
-
     /// If `Some`, stores a field to be encoded, else does nothing.
     pub fn encode_optional_field<T: RLPEncode>(mut self, opt_value: &Option<T>) -> Self {
         if let Some(value) = opt_value {
diff --git a/crates/common/types/account.rs b/crates/common/types/account.rs
index 8bae953745..cbc0a0cda6 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 98082e0550..9ed737ed24 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<AccountState> 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<AccountStateSlim> 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<Vec<u8>>,
 }
 
@@ -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 aa47d10d49..44b5cc49f1 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 17:10:00 -0300
Subject: [PATCH 010/155] 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 44b5cc49f1..307d18480e 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 07a4475a02..e60d548dde 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<Item = (PathRLP, ValueRLP)> {
         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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 17:10:37 -0300
Subject: [PATCH 011/155] 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 307d18480e..4de78ba4a0 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 17:25:46 -0300
Subject: [PATCH 012/155] 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 4de78ba4a0..a3bd9a5a88 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 17:29:34 -0300
Subject: [PATCH 013/155] 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 a3bd9a5a88..8b48663e02 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<u8>)> = 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 9618d5cf32..82a88a3c95 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<Option<Block>, 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<u8> {

From d87b1cae6a76af260d29dff0748518857e12596e Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 17:36:39 -0300
Subject: [PATCH 014/155] 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 8b48663e02..ce3c934294 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:00:43 -0300
Subject: [PATCH 015/155] 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 53e6f0dbae..128cff7e76 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 ce3c934294..92af492223 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:04:04 -0300
Subject: [PATCH 016/155] 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 92af492223..86badd36d4 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:05:45 -0300
Subject: [PATCH 017/155] 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 86badd36d4..b012c00a3f 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:10:52 -0300
Subject: [PATCH 018/155] 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 b012c00a3f..0406ed9ee2 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:12:24 -0300
Subject: [PATCH 019/155] 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 0406ed9ee2..f6458a4525 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 <fedemoletta@hotmail.com>
Date: Wed, 23 Oct 2024 18:21:41 -0300
Subject: [PATCH 020/155] 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 f6458a4525..3ba9fbaed3 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 82a88a3c95..145f00b74d 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<Vec<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 10:40:11 -0300
Subject: [PATCH 021/155] 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 3ba9fbaed3..2816c998dc 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<u8>)> = 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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 10:43:50 -0300
Subject: [PATCH 022/155] 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 2816c998dc..6b4b172c10 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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 12:49:24 -0300
Subject: [PATCH 023/155] 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 9ed737ed24..62f86cacb8 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<u8>)>,
     pub proof: Vec<Vec<u8>>,
 }
 
diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs
index 6b4b172c10..0601c9fcab 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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 13:06:28 -0300
Subject: [PATCH 024/155] 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 0601c9fcab..93cc432340 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<AccountRange, StoreError> {
     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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 16:26:46 -0300
Subject: [PATCH 025/155] 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 93cc432340..6ef8c5c105 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 <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 16:34:12 -0300
Subject: [PATCH 026/155] 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 d69914db23..f47d4e3422 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<aes::Aes256>;

From 68dcef63fe134613936ec76c0fbcb6660bc8cd93 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 24 Oct 2024 16:39:18 -0300
Subject: [PATCH 027/155] 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 62f86cacb8..c1ab07374f 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<u8>)>,
     pub proof: Vec<Vec<u8>>,
 }

From f23077eb328731cf3bfdcf07e2f6a07de523b5d6 Mon Sep 17 00:00:00 2001
From: Esteban Dimitroff Hodi <esteban.dimitroff@lambdaclass.com>
Date: Thu, 24 Oct 2024 16:46:41 -0300
Subject: [PATCH 028/155] 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 af94e955f6..0ae4204693 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 7f6e59fd15..fd512c1380 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<TcpStream>, table: Arc<Mutex<KademliaTable>>) {
     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 69a9af5d51..bb678c7c7e 100644
--- a/crates/networking/p2p/rlpx/connection.rs
+++ b/crates/networking/p2p/rlpx/connection.rs
@@ -76,7 +76,70 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                         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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         }
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         };
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         };
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         };
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         };
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         }
     }
 
-    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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
             _ => 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 52387e6aa8..84480d6c76 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 679a0f8d83..c88941ca59 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<aes::Aes128>;
 
@@ -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<u8> {
-    // 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 <esteban.dimitroff@lambdaclass.com>
Date: Fri, 25 Oct 2024 11:35:59 -0300
Subject: [PATCH 029/155] 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 0ae4204693..32ad9e3b34 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 84480d6c76..7474a661a7 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<Self, StoreError> {
         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 <esteban.dimitroff@lambdaclass.com>
Date: Fri, 25 Oct 2024 11:53:13 -0300
Subject: [PATCH 030/155] 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 7474a661a7..2074ec85c7 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 <esteban.dimitroff@lambdaclass.com>
Date: Fri, 25 Oct 2024 11:58:55 -0300
Subject: [PATCH 031/155] 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 bb678c7c7e..9310e44ca3 100644
--- a/crates/networking/p2p/rlpx/connection.rs
+++ b/crates/networking/p2p/rlpx/connection.rs
@@ -145,13 +145,12 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
             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 <esteban.dimitroff@lambdaclass.com>
Date: Fri, 25 Oct 2024 12:52:52 -0300
Subject: [PATCH 032/155] 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 9310e44ca3..0ce82a83c5 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
     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 2074ec85c7..a03ff256ca 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<Self, StoreError> {
-        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<Self, RLPDecodeError> {
-        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 0000000000..1af62214a0
--- /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<StatusMessage, StoreError> {
+    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 496273341f..5d76a2f277 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 0000000000..9050ac433e
--- /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<Self, RLPDecodeError> {
+        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 4923e3b197..d84e9c228e 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 480e180bf8..8f06159bed 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 <esteban.dimitroff@lambdaclass.com>
Date: Fri, 25 Oct 2024 15:42:38 -0300
Subject: [PATCH 033/155] 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 0ce82a83c5..06de21c3c1 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<aes::Aes256>;
 
@@ -33,7 +37,7 @@ pub(crate) struct RLPxConnection<S> {
     state: RLPxConnectionState,
     stream: S,
     storage: Store,
-    capabilities: Vec<(String, u8)>,
+    capabilities: Vec<(Capability, u8)>,
 }
 
 impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
@@ -98,12 +102,8 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
     }
 
     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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
 
                 // 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
 
     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 852e665459..7795f5fb52 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 <fedemoletta@hotmail.com>
Date: Fri, 25 Oct 2024 18:24:13 -0300
Subject: [PATCH 034/155] 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 bb34a2bc7b..222bd0e2fa 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                         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 4177ea10fe..83b1587626 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 d2b0077040..9c5c8d266f 100644
--- a/crates/networking/p2p/rlpx/frame.rs
+++ b/crates/networking/p2p/rlpx/frame.rs
@@ -65,7 +65,6 @@ pub(crate) async fn write<S: AsyncWrite + std::marker::Unpin>(
     };
     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 8f06159bed..9ed648118b 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<Message, RLPDecodeError> {
-        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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 09:29:23 -0300
Subject: [PATCH 035/155] 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 e503240288..743f10df1e 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 32ad9e3b34..538389ec4e 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 7795f5fb52..4521c5bef4 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 09:40:31 -0300
Subject: [PATCH 036/155] 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 538389ec4e..eb59ef27bd 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 09:50:24 -0300
Subject: [PATCH 037/155] 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 eb59ef27bd..d06c485fe7 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 e2e7c57ecf..9446795b4b 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 11:56:17 -0300
Subject: [PATCH 038/155] 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 743f10df1e..0b58411fd8 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 12:07:28 -0300
Subject: [PATCH 039/155] 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 222bd0e2fa..da6eccb31c 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                         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 9ed648118b..5ae609cbd4 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 c1ab07374f..0dfbf37504 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<AccountState> 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<u8>)>,
-    pub proof: Vec<Vec<u8>>,
+    pub accounts: Vec<AccountRangeUnit>,
+    pub proof: Vec<Bytes>,
 }
 
 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 6ef8c5c105..881abbe1ad 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<AccountRange, StoreError> {
     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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 13:01:03 -0300
Subject: [PATCH 040/155] 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 881abbe1ad..549bf11f2e 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 145f00b74d..93ab11cfea 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 13:19:00 -0300
Subject: [PATCH 041/155] 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 93ab11cfea..897c5177bc 100644
--- a/crates/storage/store/storage.rs
+++ b/crates/storage/store/storage.rs
@@ -699,8 +699,8 @@ impl Store {
         limit_hash: H256,
     ) -> Result<Vec<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 15:07:36 -0300
Subject: [PATCH 042/155] 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 549bf11f2e..5c985fea7b 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 897c5177bc..f14d57d6f7 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<H256>,
     ) -> Result<Vec<Vec<u8>>, 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 15:39:24 -0300
Subject: [PATCH 043/155] 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 d06c485fe7..ea463f3209 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 16:07:49 -0300
Subject: [PATCH 044/155] 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 da6eccb31c..86d547a68e 100644
--- a/crates/networking/p2p/rlpx/connection.rs
+++ b/crates/networking/p2p/rlpx/connection.rs
@@ -151,7 +151,7 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                         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 5ae609cbd4..9eaaecdff0 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<Message, RLPDecodeError> {
-        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 0dfbf37504..a2995bae84 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 bbeb2f9b73..a5e8c8f130 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 f7cdf60e94..b451d83243 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 46113c4e5d..e90f420b16 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 4ddd2e8eee..7aedbc9231 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 16:12:08 -0300
Subject: [PATCH 045/155] 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 ea463f3209..5acad2428e 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 16:13:04 -0300
Subject: [PATCH 046/155] 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 b451d83243..f7cdf60e94 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 e90f420b16..46113c4e5d 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 16:15:13 -0300
Subject: [PATCH 047/155] 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 86d547a68e..b3c1deafb6 100644
--- a/crates/networking/p2p/rlpx/connection.rs
+++ b/crates/networking/p2p/rlpx/connection.rs
@@ -151,7 +151,7 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                         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 a2995bae84..82a03cdc44 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 5c985fea7b..4fe66c8c31 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 f14d57d6f7..e3611f8e12 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 46113c4e5d..45b098f035 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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 16:22:42 -0300
Subject: [PATCH 048/155] 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 5acad2428e..4ed1ebfcc6 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::<String>("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 <esteban.dimitroff@lambdaclass.com>
Date: Mon, 28 Oct 2024 16:24:54 -0300
Subject: [PATCH 049/155] 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 4ed1ebfcc6..ce4903ec0d 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 16:27:46 -0300
Subject: [PATCH 050/155] 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 4fe66c8c31..8efa3f5f1f 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 18:21:42 -0300
Subject: [PATCH 051/155] Update hive revision

---
 Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/Makefile b/Makefile
index 0b58411fd8..95f8b62ccc 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 <fedemoletta@hotmail.com>
Date: Mon, 28 Oct 2024 18:51:06 -0300
Subject: [PATCH 052/155] 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 82a03cdc44..12007ac827 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<AccountState> 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<AccountStateSlim> 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<AccountRangeUnit>,
+    pub proof: Vec<Bytes>,
 }
 
 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<AccountRangeUnit>,
-    pub proof: Vec<Bytes>,
-}
-
 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<AccountState> 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<AccountStateSlim> 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 10:41:49 -0300
Subject: [PATCH 053/155] Update hive revision

---
 Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/Makefile b/Makefile
index 95f8b62ccc..a34ba23947 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 10:53:30 -0300
Subject: [PATCH 054/155] 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 7d3f908500..1f0fe044b1 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 <esteban.dimitroff@lambdaclass.com>
Date: Tue, 29 Oct 2024 14:32:02 -0300
Subject: [PATCH 055/155] Reverting hive version

---
 Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/Makefile b/Makefile
index 0b58411fd8..e503240288 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 <esteban.dimitroff@lambdaclass.com>
Date: Tue, 29 Oct 2024 14:37:57 -0300
Subject: [PATCH 056/155] 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 06de21c3c1..6425359040 100644
--- a/crates/networking/p2p/rlpx/connection.rs
+++ b/crates/networking/p2p/rlpx/connection.rs
@@ -97,7 +97,6 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         info!("Completed handshake!");
 
         self.exchange_hello_messages().await?;
-        info!("Completed Hello roundtrip!");
         Ok(())
     }
 
@@ -108,12 +107,10 @@ impl<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         ));
 
         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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
         // 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
 
                 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
 
                 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 15:18:34 -0300
Subject: [PATCH 057/155] 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 9eaaecdff0..766e66cdcc 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 12007ac827..be61512738 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<Bytes>,
 }
 
+#[derive(Debug)]
+pub(crate) struct GetStorageRanges {
+    pub id: u64,
+    pub root_hash: H256,
+    pub account_hashes: Vec<H256>,
+    pub starting_hash: H256,
+    pub limit_hash: H256,
+    pub response_bytes: u64,
+}
+
+#[derive(Debug)]
+pub(crate) struct StorageRanges {
+    pub id: u64,
+    pub slots: Vec<StorageSlot>,
+    pub proof: Vec<Bytes>,
+}
+
 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<Self, RLPDecodeError> {
+        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<Self, RLPDecodeError> {
+        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<AccountStateSlim> 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 16:45:12 -0300
Subject: [PATCH 058/155] 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 90076c4296..52330863b9 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<Option<impl Iterator<Item = (H256, AccountState)>>, 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 16:46:07 -0300
Subject: [PATCH 059/155] Bump hive version

---
 Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/Makefile b/Makefile
index 3c7cff1b38..39440f5343 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 17:39:51 -0300
Subject: [PATCH 060/155] 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 8efa3f5f1f..2c40620cd6 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 17:40:36 -0300
Subject: [PATCH 061/155] Restore Makefile

---
 Makefile | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/Makefile b/Makefile
index 39eb36b523..39440f5343 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 <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 18:40:33 -0300
Subject: [PATCH 062/155] 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 d67385de45..5582d1f987 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 46039ea3c0..7dcab8eca7 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<H256, BlobsBundle>,
     receipts: HashMap<BlockHash, HashMap<Index, Receipt>>,
     state_trie_nodes: NodeMap,
-    storage_trie_nodes: HashMap<Address, NodeMap>,
+    // A storage trie for each hashed account address
+    storage_trie_nodes: HashMap<[u8; 32], NodeMap>,
     // TODO (#307): Remove TotalDifficulty.
     block_total_difficulties: HashMap<BlockHash, U256>,
     // 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 0d2f755156..25cefdf450 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::<StorageTriesNodes, [u8; 20]>::new(
+    fn open_storage_trie(&self, hashed_address: [u8; 32], storage_root: H256) -> Trie {
+        let db = Box::new(LibmdbxDupsortTrieDB::<StorageTriesNodes, [u8; 32]>::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<u8>
+    /// 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<u8>
 );
 
 dupsort!(
diff --git a/crates/storage/store/storage.rs b/crates/storage/store/storage.rs
index 52330863b9..553f325de1 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<H256, StoreError> {
         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<Vec<Vec<u8>>, 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<Option<impl Iterator<Item = (H256, AccountState)>>, StoreError> {
+    pub fn iter_storage(
+        &self,
+        state_root: H256,
+        hashed_address: H256,
+    ) -> Result<Option<impl Iterator<Item = (H256, U256)>>, 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<u8> {
         .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<u8> {
     Keccak256::new_with_prefix(key.to_fixed_bytes())

From abb28cfa78d69d121e62768bf6d39d49b9f5148f Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Tue, 29 Oct 2024 18:52:53 -0300
Subject: [PATCH 063/155] 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 be61512738..b63a88c7f0 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<StorageSlot>,
+    pub slots: Vec<Vec<StorageSlot>>,
     pub proof: Vec<Bytes>,
 }
 
@@ -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 2c40620cd6..1590922f92 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<StorageRanges, StoreError> {
+    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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 13:01:39 -0300
Subject: [PATCH 064/155] 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 22b7389d44..a6c65c20c4 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                                 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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 15:54:34 -0300
Subject: [PATCH 065/155] 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 1590922f92..7ee6e93329 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<StorageRanges, StoreError> {
     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 553f325de1..b14e951819 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<H256>,
+    ) -> Result<Option<Vec<Vec<u8>>>, 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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 15:58:21 -0300
Subject: [PATCH 066/155] 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 7ee6e93329..256a348cc8 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 b14e951819..cd56e10f75 100644
--- a/crates/storage/store/storage.rs
+++ b/crates/storage/store/storage.rs
@@ -778,8 +778,7 @@ fn hash_address(address: &Address) -> Vec<u8> {
 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<u8> {

From c791a890a493a9f2d989bf3570abf3f91cb902c9 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 16:02:41 -0300
Subject: [PATCH 067/155] 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 5582d1f987..2824176e3c 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 7dcab8eca7..15e5ca0805 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<BlockHash, HashMap<Index, Receipt>>,
     state_trie_nodes: NodeMap,
     // A storage trie for each hashed account address
-    storage_trie_nodes: HashMap<[u8; 32], NodeMap>,
+    storage_trie_nodes: HashMap<H256, NodeMap>,
     // TODO (#307): Remove TotalDifficulty.
     block_total_difficulties: HashMap<BlockHash, U256>,
     // 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 25cefdf450..9001a7007a 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::<StorageTriesNodes, [u8; 32]>::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 cd56e10f75..4a211db859 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<u8> {
         .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<u8> {

From 9fd88d322e8005b3c660bb357994e3093164f0ad Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 16:06:01 -0300
Subject: [PATCH 068/155] 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 1f0fe044b1..86e934eb3a 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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 17:55:41 -0300
Subject: [PATCH 069/155] 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 766e66cdcc..cbd1449d1a 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 b63a88c7f0..84506c09a9 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<Bytes>,
 }
 
+#[derive(Debug)]
+pub(crate) struct GetByteCodes {
+    pub id: u64,
+    pub hashes: Vec<H256>,
+    pub bytes: u64,
+}
+
+#[derive(Debug)]
+pub(crate) struct ByteCodes {
+    pub id: u64,
+    pub codes: Vec<Bytes>,
+}
+
 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<Self, RLPDecodeError> {
+        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<Self, RLPDecodeError> {
+        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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 18:25:30 -0300
Subject: [PATCH 070/155] 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 86e934eb3a..041ed4355e 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 <fedemoletta@hotmail.com>
Date: Wed, 30 Oct 2024 18:26:22 -0300
Subject: [PATCH 071/155] 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 a6c65c20c4..9b945d37e7 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                                 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 cbd1449d1a..9a3cc5d760 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 84506c09a9..6126db6dc6 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 256a348cc8..d32562d37b 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<ByteCodes, StoreError> {
+    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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 10:36:48 -0300
Subject: [PATCH 072/155] 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 9a3cc5d760..0e42ceb356 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 6126db6dc6..8da84ca16f 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<Bytes>,
 }
 
+#[derive(Debug)]
+pub(crate) struct GetTrieNodes {
+    pub id: u64,
+    pub root_hash: H256,
+    // [[acc_path, slot_path_1, slot_path_2,...]...]
+    pub paths: Vec<Vec<H256>>,
+    pub bytes: u64,
+}
+
+#[derive(Debug)]
+pub(crate) struct TrieNodes {
+    pub id: u64,
+    pub nodes: Vec<Bytes>,
+}
+
 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<Self, RLPDecodeError> {
+        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<Self, RLPDecodeError> {
+        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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 11:49:51 -0300
Subject: [PATCH 073/155] 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 b19a0111b8..cd8bacc400 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<Option<Vec<u8>>, 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 a5279bc2e0..2ee12f38cc 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<Option<Vec<u8>>, 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 4e0c0d0e40..ecdc4b276b 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<Option<Vec<u8>>, 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 b6684e61d6..05064a4a94 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<Option<Vec<u8>>, 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 45b098f035..bff2db163e 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<Option<Vec<u8>>, 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/155] 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 9eaaecdff0..2d5edffa34 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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 12:30:03 -0300
Subject: [PATCH 075/155] 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 4a211db859..b016cd65ca 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<H256>,
+    ) -> Result<Vec<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 12:33:42 -0300
Subject: [PATCH 076/155] 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 b016cd65ca..98246dc5bc 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<H256>,
+        byte_limit: u64,
     ) -> Result<Vec<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 14:42:32 -0300
Subject: [PATCH 077/155] 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 9b945d37e7..bfbb89a413 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<S: AsyncWrite + AsyncRead + std::marker::Unpin> RLPxConnection<S> {
                             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 d32562d37b..b50c4481d3 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<TrieNodes, StoreError> {
+    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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 16:43:47 -0300
Subject: [PATCH 078/155] 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 5c093188a5..b63a88c7f0 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<AccountRangeUnit>,
     pub proof: Vec<Bytes>,
-<<<<<<< HEAD
 }
 
 #[derive(Debug)]
@@ -49,8 +48,6 @@ pub(crate) struct StorageRanges {
     pub id: u64,
     pub slots: Vec<Vec<StorageSlot>>,
     pub proof: Vec<Bytes>,
-=======
->>>>>>> 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<AccountStateSlim> 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 <fedemoletta@hotmail.com>
Date: Thu, 31 Oct 2024 19:11:21 -0300
Subject: [PATCH 079/155] 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 39440f5343..6468711f07 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 ff1e498036..bd4441092f 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 8da84ca16f..dbfe1d5875 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<Vec<H256>>,
+    // The paths can be either full paths (hash) or only the partial path (bytes)
+    pub paths: Vec<Vec<Bytes>>,
     pub bytes: u64,
 }
 
diff --git a/crates/networking/p2p/snap.rs b/crates/networking/p2p/snap.rs
index b50c4481d3..b4ab972aa4 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<u8> {
+//     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<u8> {
+//     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 98246dc5bc..026bc42df3 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<H256>,
+        paths: Vec<Vec<u8>>,
         byte_limit: u64,
     ) -> Result<Vec<Vec<u8>>, 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 2ee12f38cc..3b9c1962fa 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 05064a4a94..0cb86dd8f8 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<Option<Vec<u8>>, 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 bff2db163e..0de9df58ce 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<Option<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 11:22:38 -0300
Subject: [PATCH 080/155] 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 b4ab972aa4..409e26703e 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<u8> {
-//     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<u8> {
-//     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<u8> {
+    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<u8> {
+    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 026bc42df3..c70a3d6e46 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 cd8bacc400..604ea4cc4a 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<u8> {
+        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 0de9df58ce..0c2626106f 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<Option<Vec<u8>>, 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<u8>, pos: usize) -> Result<Option<Vec<u8>>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 11:49:03 -0300
Subject: [PATCH 081/155] 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 409e26703e..49ffc4ce67 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<TrieNodes, StoreError> {
+    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 c70a3d6e46..30fbc02bd9 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 604ea4cc4a..55084ce582 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<u8> {
+    pub fn encode_raw(self, path_offset: usize) -> Vec<u8> {
         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 0c2626106f..a833644801 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<Option<Vec<u8>>, TrieError> {
+    pub fn get_node_partial(&self, partial_path: &PathRLP) -> Result<Vec<u8>, 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<u8>, pos: usize) -> Result<Option<Vec<u8>>, TrieError> {
+    fn get_node_inner(
+        &self,
+        node: Node,
+        partial_path: &Vec<u8>,
+        pos: usize,
+    ) -> Result<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 11:51:06 -0300
Subject: [PATCH 082/155] 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 55084ce582..0c64684785 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<Option<Vec<u8>>, 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<u8> {
         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 3b9c1962fa..a5279bc2e0 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<Option<Vec<u8>>, 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 ecdc4b276b..4e0c0d0e40 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<Option<Vec<u8>>, 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 0cb86dd8f8..b6684e61d6 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<Option<Vec<u8>>, 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 a833644801..6c69399f5d 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<Option<Vec<u8>>, 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<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 12:13:56 -0300
Subject: [PATCH 083/155] 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 83b1587626..cd31aecdb0 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 49ffc4ce67..7ec8d2c3bf 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<TrieNodes, StoreError> {
-    println!("PROCESSING REQUEST");
+) -> Result<TrieNodes, RLPxError> {
     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::<Result<_, _>>()?,
             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<u8> {
-    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<Vec<u8>, 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<u8> {
-    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<u8> {
     base[chop..].to_vec()
 }
 
+fn keybytes_to_hex(keybytes: Bytes) -> Vec<u8> {
+    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 6c69399f5d..194e944683 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<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 14:29:56 -0300
Subject: [PATCH 084/155] 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 537f9d29ae..dd9f939466 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<Item = Nibble>,
diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs
index 194e944683..e33a33ff58 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 15:14:34 -0300
Subject: [PATCH 085/155] 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 7ec8d2c3bf..abb295e846 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::<Result<_, _>>()?,
+                .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<Vec<u8>, RLPxError> {
+fn process_path_input(bytes: Bytes) -> Vec<u8> {
     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<u8> {
-    // 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 e33a33ff58..bcb449e687 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<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 15:16:29 -0300
Subject: [PATCH 086/155] 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 abb295e846..c43bdbaeab 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 30fbc02bd9..5c59fa122c 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 bcb449e687..88af9513ea 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 15:17:16 -0300
Subject: [PATCH 087/155] 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 041ed4355e..82614c56dc 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 15:41:09 -0300
Subject: [PATCH 088/155] 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 c43bdbaeab..9cf9cca7e2 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 5c59fa122c..b9f295d564 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 88af9513ea..e55ec18255 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<Vec<u8>, TrieError> {
+    pub fn get_node(&self, partial_path: &PathRLP) -> Result<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 16:56:10 -0300
Subject: [PATCH 089/155] 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 9cf9cca7e2..885ddab503 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<u8> {
     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<u8>) -> Vec<u8> {
+    nibbles.chunks(2).map(|chunk| match chunk.len() {
+        1 => chunk[0] << 4,
+        _ /* 2 */ => chunk[0] << 4 | chunk[1]
+    }).collect::<Vec<_>>()
+}
+
 fn compact_to_hex(compact: Bytes) -> Vec<u8> {
     if compact.is_empty() {
         return vec![];
diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs
index e55ec18255..262c1ccfc4 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<u8>,
-        pos: usize,
+        mut partial_path: NibbleSlice,
     ) -> Result<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 17:17:29 -0300
Subject: [PATCH 090/155] :(

---
 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 885ddab503..97eaf63d12 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<TrieNodes, RLPxError> {
+    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 262c1ccfc4..fcb061ee07 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<Vec<u8>, 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<Vec<u8>, 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:02:03 -0300
Subject: [PATCH 091/155] 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 97eaf63d12..ab08e076fc 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<u8> {
-    match bytes.len() {
-        n if n < 32 => nibbles_to_bytes(compact_to_hex(bytes)),
-        _ => bytes.to_vec(),
-    }
-}
-
-fn nibbles_to_bytes(nibbles: Vec<u8>) -> Vec<u8> {
-    nibbles.chunks(2).map(|chunk| match chunk.len() {
-        1 => chunk[0] << 4,
-        _ /* 2 */ => chunk[0] << 4 | chunk[1]
-    }).collect::<Vec<_>>()
-}
-
-fn compact_to_hex(compact: Bytes) -> Vec<u8> {
-    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<u8> {
-    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 fcb061ee07..09b18168ea 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<Vec<u8>, 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<u8>),
+    // Partial path expressed as compact nibbles
+    Compact(Vec<u8>),
+}
+
+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<u8>, 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::<Vec<_>>();
+                (bytes, last_is_half)
+            }
+        }
+    }
+    pub fn new(maybe_compact: Vec<u8>) -> Self {
+        match maybe_compact.len() {
+            n if n < 32 => Self::Compact(maybe_compact),
+            _ => Self::Bytes(maybe_compact),
+        }
+    }
+}
+
+fn compact_to_hex(compact: Vec<u8>) -> Vec<u8> {
+    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<u8>) -> Vec<u8> {
+    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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:13:19 -0300
Subject: [PATCH 092/155] 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 09b18168ea..ea0b027c0d 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<Vec<u8>, 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<Vec<u8>, 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<u8>),
-    // Partial path expressed as compact nibbles
-    Compact(Vec<u8>),
-}
-
-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<u8>, 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::<Vec<_>>();
-                (bytes, last_is_half)
-            }
-        }
-    }
-    pub fn new(maybe_compact: Vec<u8>) -> 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<u8>) -> (Cow<Vec<u8>>, 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::<Vec<_>>();
+                (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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:30:27 -0300
Subject: [PATCH 093/155] 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 ea0b027c0d..b7416bbab8 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<Vec<u8>, 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<u8>) -> (Cow<Vec<u8>>, 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::<Vec<_>>();
-                (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<u8>) -> (Vec<u8>, 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::<Vec<_>>();
+    (bytes, last_is_half)
 }
 
-fn compact_to_hex(compact: Vec<u8>) -> Vec<u8> {
+fn compact_to_hex(compact: &Vec<u8>) -> Vec<u8> {
     if compact.is_empty() {
         return vec![];
     }
@@ -321,7 +323,7 @@ fn compact_to_hex(compact: Vec<u8>) -> Vec<u8> {
     base[chop..].to_vec()
 }
 
-fn keybytes_to_hex(keybytes: Vec<u8>) -> Vec<u8> {
+fn keybytes_to_hex(keybytes: &Vec<u8>) -> Vec<u8> {
     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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:33:35 -0300
Subject: [PATCH 094/155] 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 b7416bbab8..7e923a6414 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:36:59 -0300
Subject: [PATCH 095/155] 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 ab08e076fc..fefd7374b0 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<TrieNodes, RLPxError> {
-    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 b9f295d564..96a10cabd6 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:48:05 -0300
Subject: [PATCH 096/155] Remove debug command

---
 Makefile | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/Makefile b/Makefile
index 6468711f07..39440f5343 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:48:48 -0300
Subject: [PATCH 097/155] 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 bd4441092f..ff1e498036 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:49:57 -0300
Subject: [PATCH 098/155] 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 dbfe1d5875..10597994df 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<Vec<Bytes>>,
     pub bytes: u64,
 }

From b307261f0881f7803fec6d7931bf715f2c8b8657 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:53:59 -0300
Subject: [PATCH 099/155] 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 96a10cabd6..62447c720c 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 <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 18:57:41 -0300
Subject: [PATCH 100/155] 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 7e923a6414..6757021305 100644
--- a/crates/storage/trie/trie.rs
+++ b/crates/storage/trie/trie.rs
@@ -306,6 +306,7 @@ fn compact_nibbles_to_bytes(compact: &Vec<u8>) -> (Vec<u8>, 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<u8>) -> Vec<u8> {
     if compact.is_empty() {
         return vec![];
@@ -320,6 +321,7 @@ fn compact_to_hex(compact: &Vec<u8>) -> Vec<u8> {
     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<u8>) -> Vec<u8> {
     let l = keybytes.len() * 2 + 1;
     let mut nibbles = vec![0; l];

From 8a0fbbbae23d22adaf79fce24014653c9287885a Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Fri, 1 Nov 2024 19:01:20 -0300
Subject: [PATCH 101/155] 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 0000000000..ff3dba7b60
--- /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<u8>, 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::<Vec<_>>();
+    (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<u8> {
+    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<u8> {
+    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 6757021305..da498f7981 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<u8>) -> (Vec<u8>, 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::<Vec<_>>();
-    (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<u8>) -> Vec<u8> {
-    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<u8>) -> Vec<u8> {
-    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 <fedemoletta@hotmail.com>
Date: Mon, 4 Nov 2024 16:42:35 -0300
Subject: [PATCH 102/155] 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 041ed4355e..0000000000
--- 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 998440206a..64bcd36658 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 <fedemoletta@hotmail.com>
Date: Mon, 4 Nov 2024 16:43:28 -0300
Subject: [PATCH 103/155] 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 86e934eb3a..0000000000
--- 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 09f688b67a6982fe7547182a8fdcf795e0f85b12 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 10:20:18 -0300
Subject: [PATCH 104/155] 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 48a1e4ff06..ce1c898b9f 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 a4a57600b4..afaff9cfaf 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 20c58f8afb..5b4bfa8174 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 19c0bf09ff264fd61a1a725135eb46bbe3ed0e9c Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 12:54:10 -0300
Subject: [PATCH 105/155] 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 a5279bc2e0..3fb91b6ae8 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 4e0c0d0e40..33903d0148 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 b6684e61d6..c3c495c468 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 a19a70d3c3..da121526ba 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 ab0a211fb8..26459ebad9 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 e60d548dde..f8d6c2ef02 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<Item = (PathRLP, ValueRLP)> {
         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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 14:47:13 -0300
Subject: [PATCH 106/155] 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 0000000000..4221e486ab
--- /dev/null
+++ b/crates/storage/trie/dumb_nibbles.rs
@@ -0,0 +1,81 @@
+pub struct DumbNibbles {
+    data: Vec<u8>,
+}
+
+impl DumbNibbles {
+    pub fn from_hex(hex: Vec<u8>) -> Self {
+        Self { data: hex }
+    }
+
+    pub fn from_bytes(bytes: Vec<u8>) -> 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 14:58:34 -0300
Subject: [PATCH 107/155] 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 4221e486ab..9de800555f 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 45b098f035..d359287465 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 15:00:01 -0300
Subject: [PATCH 108/155] 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 9de800555f..8b9fe4fcc3 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 15:02:25 -0300
Subject: [PATCH 109/155] 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 8b9fe4fcc3..b9c01cbbe1 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<u8> {
+        (!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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 15:56:10 -0300
Subject: [PATCH 110/155] 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 b9c01cbbe1..468c2ec50d 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<u8>,
 }
@@ -48,6 +49,22 @@ impl DumbNibbles {
     pub fn next(&mut self) -> Option<u8> {
         (!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 33903d0148..9d5f8e600f 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 c3c495c468..95b3161cf3 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<Option<ValueRLP>, TrieError> {
-        if path.cmp_rest(&self.path) {
+    pub fn get(&self, path: DumbNibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         /* 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<NodeHash, TrieError> {
-        let hash = self.compute_hash(path_offset);
+    pub fn insert_self(self, state: &mut TrieState) -> Result<NodeHash, TrieError> {
+        // 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 16:48:39 -0300
Subject: [PATCH 111/155] 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 468c2ec50d..053a705d0b 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 b19a0111b8..3bd2673f41 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<LeafNode> 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<Option<ValueRLP>, TrieError> {
+    pub fn get(&self, state: &TrieState, path: DumbNibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         match self {
@@ -92,13 +92,12 @@ impl Node {
 
     pub fn insert_self(
         self,
-        path_offset: usize,
         state: &mut TrieState,
     ) -> Result<NodeHash, TrieError> {
         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 9d5f8e600f..b5ff31f1b4 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<Option<ValueRLP>, 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<Node, TrieError> {
+        // 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 95b3161cf3..d447245787 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<Node>, Option<ValueRLP>), TrieError> {
-        Ok(if path.cmp_rest(&self.path) {
+    pub fn remove(self, path: DumbNibbles) -> Result<(Option<Node>, Option<ValueRLP>), 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<u8> {
         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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 17:38:45 -0300
Subject: [PATCH 112/155] 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 053a705d0b..554aeed0d6 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<u8>,
@@ -8,7 +15,7 @@ impl DumbNibbles {
         Self { data: hex }
     }
 
-    pub fn from_bytes(bytes: Vec<u8>) -> 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 3bd2673f41..2a07729d1f 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<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> Result<(), TrieError> {
         match self {
@@ -90,10 +90,7 @@ impl Node {
         }
     }
 
-    pub fn insert_self(
-        self,
-        state: &mut TrieState,
-    ) -> Result<NodeHash, TrieError> {
+    pub fn insert_self(self, state: &mut TrieState) -> Result<NodeHash, TrieError> {
         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 3fb91b6ae8..6990b9006e 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<Option<ValueRLP>, 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<Node, TrieError> {
         // 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<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> 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 b5ff31f1b4..51105542b9 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<Node, TrieError> {
         // 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<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> 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 d447245787..78f722df3b 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<u8> {
         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<Vec<u8>>,
     ) -> 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 da121526ba..13eff2282e 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 26459ebad9..5fb6b2191b 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 d359287465..6c4922e924 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 f8d6c2ef02..488fbbdd7f 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 18:06:34 -0300
Subject: [PATCH 113/155] 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 554aeed0d6..7de3296145 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<u8> = 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<u8> {
+        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 2a07729d1f..5d21d76ef7 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 51105542b9..14ce365719 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 78f722df3b..032fdafe95 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<u8> {
-        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<u8> {
+        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<NodeHash, TrieError> {
         // 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<Vec<u8>>,
-    ) -> Result<(), TrieError> {
-        let encoded = self.encode_raw(0);
+    pub fn get_path(&self, node_path: &mut Vec<Vec<u8>>) -> 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 868b3479b8..81b28b5be8 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 <fedemoletta@hotmail.com>
Date: Tue, 5 Nov 2024 18:46:19 -0300
Subject: [PATCH 114/155] 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 7de3296145..6aa599f61d 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<u8> = 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 14ce365719..05eecc637b 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 11:47:18 -0300
Subject: [PATCH 115/155] 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 032fdafe95..4d12d33b25 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 12:36:12 -0300
Subject: [PATCH 116/155] 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 05eecc637b..8239637162 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 12:39:28 -0300
Subject: [PATCH 117/155] 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 6990b9006e..01c9088b3e 100644
--- a/crates/storage/trie/node/branch.rs
+++ b/crates/storage/trie/node/branch.rs
@@ -64,20 +64,23 @@ impl BranchNode {
     ) -> Result<Option<ValueRLP>, 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 12:54:45 -0300
Subject: [PATCH 118/155] 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 01c9088b3e..c5f8aee45f 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 13:09:42 -0300
Subject: [PATCH 119/155] 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 c5f8aee45f..170afb19af 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 8239637162..011717cb83 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 15:20:47 -0300
Subject: [PATCH 120/155] 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 170afb19af..40bd69eff0 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 16:13:36 -0300
Subject: [PATCH 121/155] [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 6c4922e924..82c9f31c49 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<Option<ValueRLP>, 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<Option<ValueRLP>, 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 488fbbdd7f..e7a5bc0532 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<NodeHash>, 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 8ff338c97b1409ee78b70260448b1e37f892602d Mon Sep 17 00:00:00 2001
From: fmoletta <99273364+fmoletta@users.noreply.github.com>
Date: Wed, 6 Nov 2024 16:14:25 -0300
Subject: [PATCH 122/155] Update crates/storage/store/storage.rs

Co-authored-by: ElFantasma <estebandh@gmail.com>
---
 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 2d0a83ffd4..cea9d6d02b 100644
--- a/crates/storage/store/storage.rs
+++ b/crates/storage/store/storage.rs
@@ -812,7 +812,7 @@ 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)
+    /// (aka a hashed account address) and the following paths will be paths in the account'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
     /// For more information check out snap capability message [`GetTrieNodes`](https://github.com/ethereum/devp2p/blob/master/caps/snap.md#gettrienodes-0x06)

From fc3302202724752aa1e1c906f28922eab3f18ae0 Mon Sep 17 00:00:00 2001
From: fmoletta <99273364+fmoletta@users.noreply.github.com>
Date: Wed, 6 Nov 2024 16:16:08 -0300
Subject: [PATCH 123/155] Uncomment feature gate

---
 crates/storage/trie/nibble.rs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/crates/storage/trie/nibble.rs b/crates/storage/trie/nibble.rs
index dd9f939466..537f9d29ae 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<Item = Nibble>,

From 167c59102f5f9d6c469dde641fec7678659dfa4c Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 16:28:23 -0300
Subject: [PATCH 124/155] [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 82c9f31c49..5ce6eefcb3 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 e7a5bc0532..293c7e9850 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<NodeHash>, 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 16:58:47 -0300
Subject: [PATCH 125/155] 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 40bd69eff0..a7b7f37eae 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 011717cb83..a5b8db21b3 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 16:59:04 -0300
Subject: [PATCH 126/155] [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 293c7e9850..108a7100c4 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 <fedemoletta@hotmail.com>
Date: Wed, 6 Nov 2024 17:13:13 -0300
Subject: [PATCH 127/155] 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 6aa599f61d..c331e35936 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<usize> {
+        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 a7b7f37eae..754e262567 100644
--- a/crates/storage/trie/node/branch.rs
+++ b/crates/storage/trie/node/branch.rs
@@ -64,23 +64,20 @@ impl BranchNode {
     ) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         // 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 10:36:22 -0300
Subject: [PATCH 128/155] 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 4d12d33b25..2118d49928 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<u8> {
-        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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 12:26:48 -0300
Subject: [PATCH 129/155] 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 01e228e51d..bfb1016ead 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<T> and Vec<u8>
+    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 a5b8db21b3..e50047a6da 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 2118d49928..e564c29994 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<u8> {
         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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 12:29:28 -0300
Subject: [PATCH 130/155] 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 e50047a6da..6505ba7f50 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<u8> {
-        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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 12:40:50 -0300
Subject: [PATCH 131/155] 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 754e262567..c45c8bf1c4 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<u8> {
-        let hash_choice = |node_hash: &NodeHash| -> (Vec<u8>, 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::<Vec<_>>();
-        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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 12:44:05 -0300
Subject: [PATCH 132/155] 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 6505ba7f50..d7571dfa58 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 e564c29994..ec9473f472 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 81b28b5be8..f477b5e799 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<u8>,
-}
-
-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::<usize>() - 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<u8> {
-        self.encoded
-    }
-}

From 856224dd8227216c5df4ef11d05473475d77ed10 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 12:45:13 -0300
Subject: [PATCH 133/155] 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 c45c8bf1c4..51cc2de8da 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 f477b5e799..655e986472 100644
--- a/crates/storage/trie/node_hash.rs
+++ b/crates/storage/trie/node_hash.rs
@@ -14,11 +14,6 @@ pub enum NodeHash {
     Inline(Vec<u8>),
 }
 
-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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 13:51:32 -0300
Subject: [PATCH 134/155] 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 c331e35936..602532bcfe 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<u8> {
         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<u8> {
+        // 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::<Vec<_>>()
+    }
 }
 
 impl AsRef<[u8]> for DumbNibbles {
diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs
index 5ce6eefcb3..2ba05e2ed8 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 108a7100c4..ffc3dec608 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<NodeHash>,
+    // 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<Self::Item> {
         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<Item = (PathRLP, ValueRLP)> {
-        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::<Vec<_>>();
+        assert_eq!(content, expected_content);
+    }
+}

From 30ba82fe0cdc4c6c167c157e9ad934bb14f80ca1 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 15:36:00 -0300
Subject: [PATCH 135/155] 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 ffc3dec608..b40f52b8a9 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::<Vec<_>>();
         assert_eq!(content, expected_content);
     }
+    proptest! {
+
+        #[test]
+        fn proptest_trie_iter_content(data in btree_map(vec(any::<u8>(), 5..100), vec(any::<u8>(), 5..100), 5..100)) {
+            let expected_content = data.clone().into_iter().collect::<Vec<_>>();
+            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::<Vec<_>>();
+            assert_eq!(content, expected_content);
+        }
+    }
 }

From 21537d41c794964794bcc51b19f6e259934be911 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 15:38:40 -0300
Subject: [PATCH 136/155] 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 537f9d29ae..0000000000
--- 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<u8> for Nibble {
-    type Error = u8;
-
-    fn try_from(value: u8) -> Result<Self, Self::Error> {
-        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<Nibble> for u8 {
-    fn from(value: Nibble) -> Self {
-        value as u8
-    }
-}
-
-impl From<Nibble> 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::Item> {
-        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<Item = Nibble>,
-        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::Item> {
-        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 2ba05e2ed8..faf639e810 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 15:41:23 -0300
Subject: [PATCH 137/155] 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 602532bcfe..5181f7247c 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<u8>,
 }
 
-impl DumbNibbles {
+impl Nibbles {
     pub fn from_hex(hex: Vec<u8>) -> 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 5d21d76ef7..71c0a0892a 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<LeafNode> 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<Option<ValueRLP>, TrieError> {
+    pub fn get(&self, state: &TrieState, path: Nibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         match self {
@@ -65,7 +65,7 @@ impl Node {
     pub fn remove(
         self,
         state: &mut TrieState,
-        path: DumbNibbles,
+        path: Nibbles,
     ) -> Result<(Option<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> Result<(), TrieError> {
         match self {
diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs
index 51cc2de8da..6d7e737701 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<Option<ValueRLP>, TrieError> {
+    pub fn get(&self, state: &TrieState, mut path: Nibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         // 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<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> 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 d7571dfa58..ddb65fbf34 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<Option<ValueRLP>, TrieError> {
+    pub fn get(&self, state: &TrieState, mut path: Nibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         // OUTDATED
@@ -100,7 +96,7 @@ impl ExtensionNode {
     pub fn remove(
         mut self,
         state: &mut TrieState,
-        mut path: DumbNibbles,
+        mut path: Nibbles,
     ) -> Result<(Option<Node>, Option<ValueRLP>), 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<Vec<u8>>,
     ) -> 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 ec9473f472..10f4f39ea3 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<Option<ValueRLP>, TrieError> {
+    pub fn get(&self, path: Nibbles) -> Result<Option<ValueRLP>, 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<Node, TrieError> {
         /* 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<Node>, Option<ValueRLP>), TrieError> {
+    pub fn remove(self, path: Nibbles) -> Result<(Option<Node>, Option<ValueRLP>), 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 5fb6b2191b..ceefe3cdb5 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 faf639e810..c415a83434 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<Option<ValueRLP>, 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<Option<ValueRLP>, 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 b40f52b8a9..ca4a640acd 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<Self::Item> {
         if self.stack.is_empty() {

From c56ad04f9a30534ac82e3a60bbb09a73576330cb Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 15:49:05 -0300
Subject: [PATCH 138/155] 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 6d7e737701..e211664ce7 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<Node>, Option<ValueRLP>), 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 10f4f39ea3..59ddbea484 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<NodeHash, TrieError> {
-        // 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 16:45:36 -0300
Subject: [PATCH 139/155] 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 e211664ce7..e42d9ed661 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::<Vec<_>>();
+        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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 16:46:56 -0300
Subject: [PATCH 140/155] 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 e42d9ed661..52fcfd1699 100644
--- a/crates/storage/trie/node/branch.rs
+++ b/crates/storage/trie/node/branch.rs
@@ -174,15 +174,14 @@ impl BranchNode {
             .collect::<Vec<_>>();
         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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 17:08:58 -0300
Subject: [PATCH 141/155] 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 ddb65fbf34..269d6d85c5 100644
--- a/crates/storage/trie/node/extension.rs
+++ b/crates/storage/trie/node/extension.rs
@@ -46,14 +46,15 @@ impl ExtensionNode {
         value: ValueRLP,
     ) -> Result<Node, TrieError> {
         // 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 17:20:06 -0300
Subject: [PATCH 142/155] 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 52fcfd1699..68fc0d56c3 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 17:25:00 -0300
Subject: [PATCH 143/155] 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 5181f7247c..fd3d848623 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<u8> {
diff --git a/crates/storage/trie/node/branch.rs b/crates/storage/trie/node/branch.rs
index 68fc0d56c3..39d88368db 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 17:36:57 -0300
Subject: [PATCH 144/155] 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 39d88368db..6f047ebb1d 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 59ddbea484..cf63450970 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 ceefe3cdb5..b49ec25880 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 17:43:23 -0300
Subject: [PATCH 145/155] 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 fd3d848623..34c9757f7a 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<u8> {
-        (!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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:19:04 -0300
Subject: [PATCH 146/155] 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 6f047ebb1d..5e204c4b16 100644
--- a/crates/storage/trie/node/branch.rs
+++ b/crates/storage/trie/node/branch.rs
@@ -174,7 +174,7 @@ impl BranchNode {
             .collect::<Vec<_>>();
         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 269d6d85c5..14a2c0d029 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<Node, TrieError> {
-        // 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:23:24 -0300
Subject: [PATCH 147/155] 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 14a2c0d029..d8c78af2e2 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 cf63450970..72d01bc349 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:24:04 -0300
Subject: [PATCH 148/155] 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 c415a83434..f869f384d7 100644
--- a/crates/storage/trie/trie.rs
+++ b/crates/storage/trie/trie.rs
@@ -44,7 +44,6 @@ pub type PathRLP = Vec<u8>;
 pub type ValueRLP = Vec<u8>;
 
 /// 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<NodeHash>,

From e1be0c65de0811950973886c1bf94e3e876cf0d3 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:26:12 -0300
Subject: [PATCH 149/155] [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 f869f384d7..fdbde2da74 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<Option<ValueRLP>, 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<Option<ValueRLP>, 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 ca4a640acd..9c0f127b67 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:28:41 -0300
Subject: [PATCH 150/155] 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 fdbde2da74..5d6812d5b4 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:38:42 -0300
Subject: [PATCH 151/155] 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 34c9757f7a..8036b10904 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<u8>,
 }
 
 impl Nibbles {
+    /// Create `Nibbles` from  hex-encoded nibbles
     pub fn from_hex(hex: Vec<u8>) -> 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<u8> = 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<u8> {
         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<u8> {
         // Trim leaf flag
         let data = if !self.is_empty() && self.is_leaf() {

From 9050b0cb44f4e5990cba49fc0c40d4087224646d Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:49:24 -0300
Subject: [PATCH 152/155] 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 d8c78af2e2..0c608ff515 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:50:47 -0300
Subject: [PATCH 153/155] 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 8036b10904..b36d3b1a54 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 0c608ff515..626c78b5c6 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 <fedemoletta@hotmail.com>
Date: Thu, 7 Nov 2024 18:57:27 -0300
Subject: [PATCH 154/155] 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 9c0f127b67..5b90ddd13c 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 76640ef41a9450fb8a7fd170d371c6ee33d93c60 Mon Sep 17 00:00:00 2001
From: fmoletta <fedemoletta@hotmail.com>
Date: Fri, 8 Nov 2024 15:27:59 -0300
Subject: [PATCH 155/155] Update trie code with refactored nibble paths

---
 crates/storage/trie/encoding.rs | 47 ---------------------------------
 crates/storage/trie/nibbles.rs  | 32 ++++++++++++++++++++++
 crates/storage/trie/node.rs     |  4 +--
 crates/storage/trie/trie.rs     | 38 ++++++++------------------
 4 files changed, 45 insertions(+), 76 deletions(-)
 delete mode 100644 crates/storage/trie/encoding.rs

diff --git a/crates/storage/trie/encoding.rs b/crates/storage/trie/encoding.rs
deleted file mode 100644
index ff3dba7b60..0000000000
--- a/crates/storage/trie/encoding.rs
+++ /dev/null
@@ -1,47 +0,0 @@
-/// 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<u8>, 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::<Vec<_>>();
-    (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<u8> {
-    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<u8> {
-    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/nibbles.rs b/crates/storage/trie/nibbles.rs
index b36d3b1a54..8c7f4b33b7 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<u8> {
+    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<u8> {
+    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.rs b/crates/storage/trie/node.rs
index dc13cc9605..78326dfe1c 100644
--- a/crates/storage/trie/node.rs
+++ b/crates/storage/trie/node.rs
@@ -98,11 +98,11 @@ impl Node {
         }
     }
 
-    pub fn encode_raw(self, path_offset: usize) -> Vec<u8> {
+    pub fn encode_raw(self) -> Vec<u8> {
         match self {
             Node::Branch(n) => n.encode_raw(),
             Node::Extension(n) => n.encode_raw(),
-            Node::Leaf(n) => n.encode_raw(path_offset),
+            Node::Leaf(n) => n.encode_raw(),
         }
     }
 }
diff --git a/crates/storage/trie/trie.rs b/crates/storage/trie/trie.rs
index a00535e769..a6e631c6f2 100644
--- a/crates/storage/trie/trie.rs
+++ b/crates/storage/trie/trie.rs
@@ -1,5 +1,4 @@
 mod db;
-mod encoding;
 mod error;
 mod node;
 mod node_hash;
@@ -11,9 +10,6 @@ mod nibbles;
 #[cfg(test)]
 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 nibbles::Nibbles;
@@ -190,14 +186,11 @@ impl Trie {
     /// 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<Vec<u8>, TrieError> {
         // Convert compact-encoded nibbles into a byte slice if necessary
-        let (partial_path, last_byte_is_half) = match partial_path.len() {
+        let partial_path = match partial_path.len() {
             // Compact-encoded nibbles
-            n if n < 32 => {
-                let (p, l) = compact_nibbles_to_bytes(partial_path);
-                (Cow::Owned(p), l)
-            }
+            n if n < 32 => Nibbles::decode_compact(partial_path),
             // Full path (No conversion needed)
-            32 => (Cow::Borrowed(partial_path), false),
+            32 => Nibbles::from_bytes(partial_path),
             // We won't handle paths with length over 32
             _ => return Ok(vec![]),
         };
@@ -212,33 +205,24 @@ impl Trie {
         else {
             return Ok(vec![]);
         };
-        self.get_node_inner(
-            root_node,
-            NibbleSlice::new(&partial_path),
-            last_byte_is_half,
-        )
+        self.get_node_inner(root_node, partial_path)
     }
 
-    fn get_node_inner(
-        &self,
-        node: Node,
-        mut partial_path: NibbleSlice,
-        last_byte_is_half: bool,
-    ) -> Result<Vec<u8>, TrieError> {
+    fn get_node_inner(&self, node: Node, mut partial_path: Nibbles) -> Result<Vec<u8>, TrieError> {
         // 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()));
+        if partial_path.is_empty() {
+            return Ok(node.encode_raw());
         }
         match node {
-            Node::Branch(branch_node) => match partial_path.next().map(usize::from) {
-                Some(idx) if idx < 16 => {
+            Node::Branch(branch_node) => match partial_path.next_choice() {
+                Some(idx) => {
                     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)
+                        self.get_node_inner(child_node, partial_path)
                     } else {
                         Ok(vec![])
                     }
@@ -253,7 +237,7 @@ impl Trie {
                         .state
                         .get_node(extension_node.child.clone())?
                         .expect("inconsistent internal tree structure");
-                    self.get_node_inner(child_node, partial_path, last_byte_is_half)
+                    self.get_node_inner(child_node, partial_path)
                 } else {
                     Ok(vec![])
                 }