mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
refactor: rename HeaderLocked to SealedHeader (#173)
This commit is contained in:
@ -2,7 +2,7 @@
|
||||
|
||||
use crate::{verification, Config};
|
||||
use reth_interfaces::consensus::{Consensus, Error, ForkchoiceState};
|
||||
use reth_primitives::{HeaderLocked, H256};
|
||||
use reth_primitives::{SealedHeader, H256};
|
||||
use tokio::sync::watch;
|
||||
|
||||
/// Ethereum consensus
|
||||
@ -32,7 +32,7 @@ impl Consensus for EthConsensus {
|
||||
self.channel.1.clone()
|
||||
}
|
||||
|
||||
fn validate_header(&self, header: &HeaderLocked, parent: &HeaderLocked) -> Result<(), Error> {
|
||||
fn validate_header(&self, header: &SealedHeader, parent: &SealedHeader) -> Result<(), Error> {
|
||||
verification::validate_header_standalone(header, &self.config)?;
|
||||
verification::validate_header_regarding_parent(parent, header, &self.config)
|
||||
|
||||
|
||||
@ -1,12 +1,12 @@
|
||||
//! ALl functions for verification of block
|
||||
use crate::{config, Config};
|
||||
use reth_interfaces::{consensus::Error, provider::HeaderProvider, Result as RethResult};
|
||||
use reth_primitives::{BlockLocked, HeaderLocked, TransactionSigned};
|
||||
use reth_primitives::{BlockLocked, SealedHeader, TransactionSigned};
|
||||
use std::time::SystemTime;
|
||||
|
||||
/// Validate header standalone
|
||||
pub fn validate_header_standalone(
|
||||
header: &HeaderLocked,
|
||||
header: &SealedHeader,
|
||||
config: &config::Config,
|
||||
) -> Result<(), Error> {
|
||||
// Gas used needs to be less then gas limit. Gas used is going to be check after execution.
|
||||
@ -106,8 +106,8 @@ pub fn calculate_next_block_base_fee(gas_used: u64, gas_limit: u64, base_fee: u6
|
||||
|
||||
/// Validate block in regards to parent
|
||||
pub fn validate_header_regarding_parent(
|
||||
parent: &HeaderLocked,
|
||||
child: &HeaderLocked,
|
||||
parent: &SealedHeader,
|
||||
child: &SealedHeader,
|
||||
config: &config::Config,
|
||||
) -> Result<(), Error> {
|
||||
// Parent number is consistent.
|
||||
@ -186,7 +186,7 @@ pub fn validate_header_regarding_parent(
|
||||
pub fn validate_block_regarding_chain<PROV: HeaderProvider>(
|
||||
block: &BlockLocked,
|
||||
provider: &PROV,
|
||||
) -> RethResult<HeaderLocked> {
|
||||
) -> RethResult<SealedHeader> {
|
||||
let hash = block.header.hash();
|
||||
|
||||
// Check if block is known.
|
||||
@ -200,7 +200,7 @@ pub fn validate_block_regarding_chain<PROV: HeaderProvider>(
|
||||
.ok_or(Error::ParentUnknown { hash: block.parent_hash })?;
|
||||
|
||||
// Return parent header.
|
||||
Ok(parent.lock())
|
||||
Ok(parent.seal())
|
||||
}
|
||||
|
||||
/// Full validation of block before execution.
|
||||
@ -310,7 +310,7 @@ mod tests {
|
||||
let receipts = Vec::new();
|
||||
let body = Vec::new();
|
||||
|
||||
(BlockLocked { header: header.lock(), body, receipts, ommers }, parent)
|
||||
(BlockLocked { header: header.seal(), body, receipts, ommers }, parent)
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@ -1,5 +1,5 @@
|
||||
use async_trait::async_trait;
|
||||
use reth_primitives::{BlockHash, BlockNumber, HeaderLocked, H256};
|
||||
use reth_primitives::{BlockHash, BlockNumber, SealedHeader, H256};
|
||||
use tokio::sync::watch::Receiver;
|
||||
|
||||
/// Re-export forkchoice state
|
||||
@ -14,7 +14,7 @@ pub trait Consensus: Send + Sync {
|
||||
fn fork_choice_state(&self) -> Receiver<ForkchoiceState>;
|
||||
|
||||
/// Validate if header is correct and follows consensus specification
|
||||
fn validate_header(&self, header: &HeaderLocked, parent: &HeaderLocked) -> Result<(), Error>;
|
||||
fn validate_header(&self, header: &SealedHeader, parent: &SealedHeader) -> Result<(), Error>;
|
||||
}
|
||||
|
||||
/// Consensus Errors
|
||||
|
||||
@ -4,7 +4,7 @@ use crate::consensus::Consensus;
|
||||
use async_trait::async_trait;
|
||||
use reth_primitives::{
|
||||
rpc::{BlockId, BlockNumber},
|
||||
Header, HeaderLocked, H256,
|
||||
Header, SealedHeader, H256,
|
||||
};
|
||||
use reth_rpc_types::engine::ForkchoiceState;
|
||||
use std::{fmt::Debug, time::Duration};
|
||||
@ -74,9 +74,9 @@ pub trait Downloader: Sync + Send {
|
||||
/// Download the headers
|
||||
async fn download(
|
||||
&self,
|
||||
head: &HeaderLocked,
|
||||
head: &SealedHeader,
|
||||
forkchoice: &ForkchoiceState,
|
||||
) -> Result<Vec<HeaderLocked>, DownloadError>;
|
||||
) -> Result<Vec<SealedHeader>, DownloadError>;
|
||||
|
||||
/// Perform a header request and returns the headers.
|
||||
// TODO: Isn't this effectively blocking per request per downloader?
|
||||
@ -108,7 +108,7 @@ pub trait Downloader: Sync + Send {
|
||||
/// Validate whether the header is valid in relation to it's parent
|
||||
///
|
||||
/// Returns Ok(false) if the
|
||||
fn validate(&self, header: &HeaderLocked, parent: &HeaderLocked) -> Result<(), DownloadError> {
|
||||
fn validate(&self, header: &SealedHeader, parent: &SealedHeader) -> Result<(), DownloadError> {
|
||||
if !(parent.hash() == header.parent_hash && parent.number + 1 == header.number) {
|
||||
return Err(DownloadError::MismatchedHeaders {
|
||||
header_number: header.number.into(),
|
||||
|
||||
@ -7,7 +7,7 @@ use crate::{
|
||||
};
|
||||
use std::{collections::HashSet, sync::Arc, time::Duration};
|
||||
|
||||
use reth_primitives::{Header, HeaderLocked, H256, H512, U256};
|
||||
use reth_primitives::{Header, SealedHeader, H256, H512, U256};
|
||||
use reth_rpc_types::engine::ForkchoiceState;
|
||||
|
||||
use tokio::sync::{broadcast, mpsc, watch};
|
||||
@ -16,12 +16,12 @@ use tokio_stream::{wrappers::BroadcastStream, StreamExt};
|
||||
#[derive(Debug)]
|
||||
/// A test downloader which just returns the values that have been pushed to it.
|
||||
pub struct TestDownloader {
|
||||
result: Result<Vec<HeaderLocked>, DownloadError>,
|
||||
result: Result<Vec<SealedHeader>, DownloadError>,
|
||||
}
|
||||
|
||||
impl TestDownloader {
|
||||
/// Instantiates the downloader with the mock responses
|
||||
pub fn new(result: Result<Vec<HeaderLocked>, DownloadError>) -> Self {
|
||||
pub fn new(result: Result<Vec<SealedHeader>, DownloadError>) -> Self {
|
||||
Self { result }
|
||||
}
|
||||
}
|
||||
@ -45,9 +45,9 @@ impl Downloader for TestDownloader {
|
||||
|
||||
async fn download(
|
||||
&self,
|
||||
_: &HeaderLocked,
|
||||
_: &SealedHeader,
|
||||
_: &ForkchoiceState,
|
||||
) -> Result<Vec<HeaderLocked>, DownloadError> {
|
||||
) -> Result<Vec<SealedHeader>, DownloadError> {
|
||||
self.result.clone()
|
||||
}
|
||||
}
|
||||
@ -157,8 +157,8 @@ impl Consensus for TestConsensus {
|
||||
|
||||
fn validate_header(
|
||||
&self,
|
||||
_header: &HeaderLocked,
|
||||
_parent: &HeaderLocked,
|
||||
_header: &SealedHeader,
|
||||
_parent: &SealedHeader,
|
||||
) -> Result<(), consensus::Error> {
|
||||
if self.fail_validation {
|
||||
Err(consensus::Error::BaseFeeMissing)
|
||||
@ -170,19 +170,19 @@ impl Consensus for TestConsensus {
|
||||
|
||||
/// Generate a range of random header. The parent hash of the first header
|
||||
/// in the result will be equal to head
|
||||
pub fn gen_random_header_range(rng: std::ops::Range<u64>, head: H256) -> Vec<HeaderLocked> {
|
||||
pub fn gen_random_header_range(rng: std::ops::Range<u64>, head: H256) -> Vec<SealedHeader> {
|
||||
let mut headers = Vec::with_capacity(rng.end.saturating_sub(rng.start) as usize);
|
||||
for idx in rng {
|
||||
headers.push(gen_random_header(
|
||||
idx,
|
||||
Some(headers.last().map(|h: &HeaderLocked| h.hash()).unwrap_or(head)),
|
||||
Some(headers.last().map(|h: &SealedHeader| h.hash()).unwrap_or(head)),
|
||||
));
|
||||
}
|
||||
headers
|
||||
}
|
||||
|
||||
/// Generate a random header
|
||||
pub fn gen_random_header(number: u64, parent: Option<H256>) -> HeaderLocked {
|
||||
pub fn gen_random_header(number: u64, parent: Option<H256>) -> SealedHeader {
|
||||
let header = reth_primitives::Header {
|
||||
number,
|
||||
nonce: rand::random(),
|
||||
@ -190,5 +190,5 @@ pub fn gen_random_header(number: u64, parent: Option<H256>) -> HeaderLocked {
|
||||
parent_hash: parent.unwrap_or_default(),
|
||||
..Default::default()
|
||||
};
|
||||
header.lock()
|
||||
header.seal()
|
||||
}
|
||||
|
||||
@ -8,7 +8,7 @@ use reth_interfaces::{
|
||||
downloader::{DownloadError, Downloader},
|
||||
},
|
||||
};
|
||||
use reth_primitives::{rpc::BlockId, HeaderLocked};
|
||||
use reth_primitives::{rpc::BlockId, SealedHeader};
|
||||
use reth_rpc_types::engine::ForkchoiceState;
|
||||
|
||||
/// Download headers in batches
|
||||
@ -49,9 +49,9 @@ impl<C: Consensus, H: HeadersClient> Downloader for LinearDownloader<C, H> {
|
||||
/// order from chain tip to local head
|
||||
async fn download(
|
||||
&self,
|
||||
head: &HeaderLocked,
|
||||
head: &SealedHeader,
|
||||
forkchoice: &ForkchoiceState,
|
||||
) -> Result<Vec<HeaderLocked>, DownloadError> {
|
||||
) -> Result<Vec<SealedHeader>, DownloadError> {
|
||||
let mut stream = self.client().stream_headers().await;
|
||||
let mut retries = self.request_retries;
|
||||
|
||||
@ -83,9 +83,9 @@ impl<C: Consensus, H: HeadersClient> Downloader for LinearDownloader<C, H> {
|
||||
#[derive(Debug)]
|
||||
pub enum LinearDownloadResult {
|
||||
/// Downloaded last batch up to tip
|
||||
Finished(Vec<HeaderLocked>),
|
||||
Finished(Vec<SealedHeader>),
|
||||
/// Downloaded batch
|
||||
Batch(Vec<HeaderLocked>),
|
||||
Batch(Vec<SealedHeader>),
|
||||
/// Ignore this batch
|
||||
Ignore,
|
||||
}
|
||||
@ -95,8 +95,8 @@ impl<C: Consensus, H: HeadersClient> LinearDownloader<C, H> {
|
||||
&self,
|
||||
stream: &mut HeadersStream,
|
||||
forkchoice: &ForkchoiceState,
|
||||
head: &HeaderLocked,
|
||||
earliest: Option<&HeaderLocked>,
|
||||
head: &SealedHeader,
|
||||
earliest: Option<&SealedHeader>,
|
||||
) -> Result<LinearDownloadResult, DownloadError> {
|
||||
// Request headers starting from tip or earliest cached
|
||||
let start = earliest.map_or(forkchoice.head_block_hash, |h| h.parent_hash);
|
||||
@ -107,7 +107,7 @@ impl<C: Consensus, H: HeadersClient> LinearDownloader<C, H> {
|
||||
let mut out = Vec::with_capacity(headers.len());
|
||||
// Iterate headers in reverse
|
||||
for parent in headers.into_iter().rev() {
|
||||
let parent = parent.lock();
|
||||
let parent = parent.seal();
|
||||
|
||||
if head.hash() == parent.hash() {
|
||||
// We've reached the target
|
||||
@ -210,7 +210,7 @@ mod tests {
|
||||
gen_random_header, gen_random_header_range, TestConsensus, TestHeadersClient,
|
||||
},
|
||||
};
|
||||
use reth_primitives::{rpc::BlockId, HeaderLocked};
|
||||
use reth_primitives::{rpc::BlockId, SealedHeader};
|
||||
|
||||
use assert_matches::assert_matches;
|
||||
use once_cell::sync::Lazy;
|
||||
@ -237,7 +237,7 @@ mod tests {
|
||||
.retries(retries)
|
||||
.build(CONSENSUS.clone(), CLIENT.clone());
|
||||
let result =
|
||||
downloader.download(&HeaderLocked::default(), &ForkchoiceState::default()).await;
|
||||
downloader.download(&SealedHeader::default(), &ForkchoiceState::default()).await;
|
||||
tx.send(result).expect("failed to forward download response");
|
||||
});
|
||||
|
||||
@ -261,7 +261,7 @@ mod tests {
|
||||
.retries(retries)
|
||||
.build(CONSENSUS.clone(), CLIENT.clone());
|
||||
let result =
|
||||
downloader.download(&HeaderLocked::default(), &ForkchoiceState::default()).await;
|
||||
downloader.download(&SealedHeader::default(), &ForkchoiceState::default()).await;
|
||||
tx.send(result).expect("failed to forward download response");
|
||||
});
|
||||
|
||||
@ -295,7 +295,7 @@ mod tests {
|
||||
let downloader =
|
||||
LinearDownloadBuilder::new().build(CONSENSUS_FAIL.clone(), CLIENT.clone());
|
||||
let forkchoice = ForkchoiceState { head_block_hash: tip_hash, ..Default::default() };
|
||||
let result = downloader.download(&HeaderLocked::default(), &forkchoice).await;
|
||||
let result = downloader.download(&SealedHeader::default(), &forkchoice).await;
|
||||
tx.send(result).expect("failed to forward download response");
|
||||
});
|
||||
|
||||
@ -310,7 +310,7 @@ mod tests {
|
||||
let request = request.unwrap();
|
||||
CLIENT.send_header_response(
|
||||
request.0,
|
||||
vec![tip_parent.clone().unlock(), tip.clone().unlock()],
|
||||
vec![tip_parent.clone().unseal(), tip.clone().unseal()],
|
||||
);
|
||||
|
||||
assert_matches!(
|
||||
@ -337,16 +337,16 @@ mod tests {
|
||||
|
||||
CLIENT
|
||||
.on_header_request(1, |id, _req| {
|
||||
let mut corrupted_tip = tip.clone().unlock();
|
||||
let mut corrupted_tip = tip.clone().unseal();
|
||||
corrupted_tip.nonce = rand::random();
|
||||
CLIENT.send_header_response(id, vec![corrupted_tip, head.clone().unlock()])
|
||||
CLIENT.send_header_response(id, vec![corrupted_tip, head.clone().unseal()])
|
||||
})
|
||||
.await;
|
||||
assert_matches!(rx.try_recv(), Err(TryRecvError::Empty));
|
||||
|
||||
CLIENT
|
||||
.on_header_request(1, |id, _req| {
|
||||
CLIENT.send_header_response(id, vec![tip.clone().unlock(), head.clone().unlock()])
|
||||
CLIENT.send_header_response(id, vec![tip.clone().unseal(), head.clone().unseal()])
|
||||
})
|
||||
.await;
|
||||
|
||||
@ -383,10 +383,10 @@ mod tests {
|
||||
headers.iter().skip(chunk_size * idx).take(chunk_size).cloned().peekable();
|
||||
idx += 1;
|
||||
if chunk.peek().is_some() {
|
||||
let headers: Vec<_> = chunk.map(|h| h.unlock()).collect();
|
||||
let headers: Vec<_> = chunk.map(|h| h.unseal()).collect();
|
||||
CLIENT.send_header_response(id, headers);
|
||||
} else {
|
||||
CLIENT.send_header_response(id, vec![head.clone().unlock()])
|
||||
CLIENT.send_header_response(id, vec![head.clone().unseal()])
|
||||
}
|
||||
})
|
||||
.await;
|
||||
|
||||
@ -1,4 +1,4 @@
|
||||
use crate::{Header, HeaderLocked, Receipt, Transaction, TransactionSigned, H256};
|
||||
use crate::{Header, Receipt, SealedHeader, Transaction, TransactionSigned, H256};
|
||||
use std::ops::Deref;
|
||||
|
||||
/// Ethereum full block.
|
||||
@ -11,7 +11,7 @@ pub struct Block {
|
||||
/// Block receipts.
|
||||
pub receipts: Vec<Receipt>,
|
||||
/// Ommers/uncles header
|
||||
pub ommers: Vec<HeaderLocked>,
|
||||
pub ommers: Vec<SealedHeader>,
|
||||
}
|
||||
|
||||
impl Deref for Block {
|
||||
@ -25,13 +25,13 @@ impl Deref for Block {
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Default)]
|
||||
pub struct BlockLocked {
|
||||
/// Locked block header.
|
||||
pub header: HeaderLocked,
|
||||
pub header: SealedHeader,
|
||||
/// Transactions with signatures.
|
||||
pub body: Vec<TransactionSigned>,
|
||||
/// Block receipts.
|
||||
pub receipts: Vec<Receipt>,
|
||||
/// Omners/uncles header
|
||||
pub ommers: Vec<HeaderLocked>,
|
||||
pub ommers: Vec<SealedHeader>,
|
||||
}
|
||||
|
||||
impl BlockLocked {
|
||||
|
||||
@ -66,17 +66,17 @@ pub struct Header {
|
||||
|
||||
impl Header {
|
||||
/// Heavy function that will calculate hash of data and will *not* save the change to metadata.
|
||||
/// Use lock, HeaderLocked and unlock if you need hash to be persistent.
|
||||
/// Use [`Header::seal`], [`SealedHeader`] and unlock if you need hash to be persistent.
|
||||
pub fn hash_slow(&self) -> H256 {
|
||||
let mut out = BytesMut::new();
|
||||
self.encode(&mut out);
|
||||
H256::from_slice(keccak256(&out).as_slice())
|
||||
}
|
||||
|
||||
/// Calculate hash and lock the Header so that it can't be changed.
|
||||
pub fn lock(self) -> HeaderLocked {
|
||||
/// Calculate hash and seal the Header so that it can't be changed.
|
||||
pub fn seal(self) -> SealedHeader {
|
||||
let hash = self.hash_slow();
|
||||
HeaderLocked { header: self, hash }
|
||||
SealedHeader { header: self, hash }
|
||||
}
|
||||
|
||||
fn header_payload_length(&self) -> usize {
|
||||
@ -174,22 +174,23 @@ impl Decodable for Header {
|
||||
}
|
||||
}
|
||||
|
||||
/// A [`Header`] that is sealed at a precalculated hash, use [`SealedHeader::unseal()`] if you want
|
||||
/// to modify header.
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Default, Hash)]
|
||||
/// HeaderLocked that has precalculated hash, use unlock if you want to modify header.
|
||||
pub struct HeaderLocked {
|
||||
pub struct SealedHeader {
|
||||
/// Locked Header fields.
|
||||
header: Header,
|
||||
/// Locked Header hash.
|
||||
hash: BlockHash,
|
||||
}
|
||||
|
||||
impl AsRef<Header> for HeaderLocked {
|
||||
impl AsRef<Header> for SealedHeader {
|
||||
fn as_ref(&self) -> &Header {
|
||||
&self.header
|
||||
}
|
||||
}
|
||||
|
||||
impl Deref for HeaderLocked {
|
||||
impl Deref for SealedHeader {
|
||||
type Target = Header;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
@ -197,16 +198,16 @@ impl Deref for HeaderLocked {
|
||||
}
|
||||
}
|
||||
|
||||
impl HeaderLocked {
|
||||
/// Construct a new locked header.
|
||||
/// Applicable when hash is known from
|
||||
/// the database provided it's not corrupted.
|
||||
impl SealedHeader {
|
||||
/// Construct a new sealed header.
|
||||
///
|
||||
/// Applicable when hash is known from the database provided it's not corrupted.
|
||||
pub fn new(header: Header, hash: H256) -> Self {
|
||||
Self { header, hash }
|
||||
}
|
||||
|
||||
/// Extract raw header that can be modified.
|
||||
pub fn unlock(self) -> Header {
|
||||
pub fn unseal(self) -> Header {
|
||||
self.header
|
||||
}
|
||||
|
||||
|
||||
@ -23,7 +23,7 @@ mod transaction;
|
||||
pub use account::Account;
|
||||
pub use block::{Block, BlockLocked};
|
||||
pub use chain::Chain;
|
||||
pub use header::{Header, HeaderLocked};
|
||||
pub use header::{Header, SealedHeader};
|
||||
pub use hex_bytes::Bytes;
|
||||
pub use integer_list::IntegerList;
|
||||
pub use jsonu256::JsonU256;
|
||||
|
||||
@ -14,7 +14,7 @@ use reth_interfaces::{
|
||||
downloader::{DownloadError, Downloader},
|
||||
},
|
||||
};
|
||||
use reth_primitives::{rpc::BigEndianHash, BlockNumber, HeaderLocked, H256, U256};
|
||||
use reth_primitives::{rpc::BigEndianHash, BlockNumber, SealedHeader, H256, U256};
|
||||
use std::{fmt::Debug, sync::Arc};
|
||||
use tracing::*;
|
||||
|
||||
@ -60,7 +60,7 @@ impl<DB: Database, D: Downloader, C: Consensus, H: HeadersClient> Stage<DB>
|
||||
tx.get::<tables::Headers>((last_block_num, last_hash).into())?.ok_or({
|
||||
DatabaseIntegrityError::Header { number: last_block_num, hash: last_hash }
|
||||
})?;
|
||||
let head = HeaderLocked::new(last_header, last_hash);
|
||||
let head = SealedHeader::new(last_header, last_hash);
|
||||
|
||||
let forkchoice = self.next_fork_choice_state(&head.hash()).await;
|
||||
// The stage relies on the downloader to return the headers
|
||||
@ -141,7 +141,7 @@ impl<D: Downloader, C: Consensus, H: HeadersClient> HeaderStage<D, C, H> {
|
||||
async fn write_headers<DB: Database>(
|
||||
&self,
|
||||
tx: &mut <DB as DatabaseGAT<'_>>::TXMut,
|
||||
headers: Vec<HeaderLocked>,
|
||||
headers: Vec<SealedHeader>,
|
||||
) -> Result<Option<BlockNumber>, StageError> {
|
||||
let mut cursor_header_number = tx.cursor_mut::<tables::HeaderNumbers>()?;
|
||||
let mut cursor_header = tx.cursor_mut::<tables::Headers>()?;
|
||||
@ -158,7 +158,7 @@ impl<D: Downloader, C: Consensus, H: HeadersClient> HeaderStage<D, C, H> {
|
||||
}
|
||||
|
||||
let key: BlockNumHash = (header.number, header.hash()).into();
|
||||
let header = header.unlock();
|
||||
let header = header.unseal();
|
||||
latest = Some(header.number);
|
||||
|
||||
td += header.difficulty;
|
||||
@ -300,7 +300,7 @@ mod tests {
|
||||
.on_header_request(1, |id, _| {
|
||||
runner.client.send_header_response(
|
||||
id,
|
||||
download_result.clone().into_iter().map(|h| h.unlock()).collect(),
|
||||
download_result.clone().into_iter().map(|h| h.unseal()).collect(),
|
||||
)
|
||||
})
|
||||
.await;
|
||||
@ -377,7 +377,7 @@ mod tests {
|
||||
p2p::headers::downloader::{DownloadError, Downloader},
|
||||
test_utils::{TestConsensus, TestHeadersClient},
|
||||
};
|
||||
use reth_primitives::{rpc::BigEndianHash, HeaderLocked, H256, U256};
|
||||
use reth_primitives::{rpc::BigEndianHash, SealedHeader, H256, U256};
|
||||
use std::{ops::Deref, sync::Arc, time::Duration};
|
||||
|
||||
pub(crate) struct HeadersTestRunner<D: Downloader> {
|
||||
@ -435,21 +435,21 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Insert header into tables
|
||||
pub(crate) fn insert_header(&self, header: &HeaderLocked) -> Result<(), db::Error> {
|
||||
pub(crate) fn insert_header(&self, header: &SealedHeader) -> Result<(), db::Error> {
|
||||
self.insert_headers(std::iter::once(header))
|
||||
}
|
||||
|
||||
/// Insert headers into tables
|
||||
pub(crate) fn insert_headers<'a, I>(&self, headers: I) -> Result<(), db::Error>
|
||||
where
|
||||
I: Iterator<Item = &'a HeaderLocked>,
|
||||
I: Iterator<Item = &'a SealedHeader>,
|
||||
{
|
||||
let headers = headers.collect::<Vec<_>>();
|
||||
self.db.map_put::<tables::HeaderNumbers, _, _>(&headers, |h| {
|
||||
(BlockNumHash((h.number, h.hash())), h.number)
|
||||
})?;
|
||||
self.db.map_put::<tables::Headers, _, _>(&headers, |h| {
|
||||
(BlockNumHash((h.number, h.hash())), h.deref().clone().unlock())
|
||||
(BlockNumHash((h.number, h.hash())), h.deref().clone().unseal())
|
||||
})?;
|
||||
self.db.map_put::<tables::CanonicalHeaders, _, _>(&headers, |h| {
|
||||
(h.number, h.hash())
|
||||
@ -469,7 +469,7 @@ mod tests {
|
||||
/// Validate stored header against provided
|
||||
pub(crate) fn validate_db_header(
|
||||
&self,
|
||||
header: &HeaderLocked,
|
||||
header: &SealedHeader,
|
||||
) -> Result<(), db::Error> {
|
||||
let db = self.db.container();
|
||||
let tx = db.get();
|
||||
@ -479,7 +479,7 @@ mod tests {
|
||||
assert_eq!(db_number, Some(header.number));
|
||||
|
||||
let db_header = tx.get::<tables::Headers>(key)?;
|
||||
assert_eq!(db_header, Some(header.clone().unlock()));
|
||||
assert_eq!(db_header, Some(header.clone().unseal()));
|
||||
|
||||
let db_canonical_header = tx.get::<tables::CanonicalHeaders>(header.number)?;
|
||||
assert_eq!(db_canonical_header, Some(header.hash()));
|
||||
@ -500,11 +500,11 @@ mod tests {
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct TestDownloader {
|
||||
result: Result<Vec<HeaderLocked>, DownloadError>,
|
||||
result: Result<Vec<SealedHeader>, DownloadError>,
|
||||
}
|
||||
|
||||
impl TestDownloader {
|
||||
pub(crate) fn new(result: Result<Vec<HeaderLocked>, DownloadError>) -> Self {
|
||||
pub(crate) fn new(result: Result<Vec<SealedHeader>, DownloadError>) -> Self {
|
||||
Self { result }
|
||||
}
|
||||
}
|
||||
@ -528,9 +528,9 @@ mod tests {
|
||||
|
||||
async fn download(
|
||||
&self,
|
||||
_: &HeaderLocked,
|
||||
_: &SealedHeader,
|
||||
_: &ForkchoiceState,
|
||||
) -> Result<Vec<HeaderLocked>, DownloadError> {
|
||||
) -> Result<Vec<SealedHeader>, DownloadError> {
|
||||
self.result.clone()
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user