fix: save last finalized block (#8473)

Co-authored-by: Roman Krasiuk <rokrassyuk@gmail.com>
This commit is contained in:
Federico Gimenez
2024-06-06 17:31:56 +02:00
committed by GitHub
parent 37f598ab94
commit a583defad7
14 changed files with 216 additions and 40 deletions

View File

@ -8,10 +8,10 @@ use reth_db_api::database::Database;
use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader}; use reth_downloaders::{bodies::noop::NoopBodiesDownloader, headers::noop::NoopHeaderDownloader};
use reth_exex::ExExManagerHandle; use reth_exex::ExExManagerHandle;
use reth_node_core::args::NetworkArgs; use reth_node_core::args::NetworkArgs;
use reth_primitives::{BlockHashOrNumber, PruneModes, B256}; use reth_primitives::{BlockHashOrNumber, BlockNumber, PruneModes, B256};
use reth_provider::{ use reth_provider::{
BlockExecutionWriter, BlockNumReader, ChainSpecProvider, HeaderSyncMode, ProviderFactory, BlockExecutionWriter, BlockNumReader, ChainSpecProvider, FinalizedBlockReader,
StaticFileProviderFactory, FinalizedBlockWriter, HeaderSyncMode, ProviderFactory, StaticFileProviderFactory,
}; };
use reth_stages::{ use reth_stages::{
sets::DefaultStages, sets::DefaultStages,
@ -75,6 +75,14 @@ impl Command {
.take_block_and_execution_range(range.clone()) .take_block_and_execution_range(range.clone())
.map_err(|err| eyre::eyre!("Transaction error on unwind: {err}"))?; .map_err(|err| eyre::eyre!("Transaction error on unwind: {err}"))?;
// update finalized block if needed
let last_saved_finalized_block_number = provider.last_finalized_block_number()?;
let range_min =
range.clone().min().ok_or(eyre::eyre!("Could not fetch lower range end"))?;
if range_min < last_saved_finalized_block_number {
provider.save_finalized_block_number(BlockNumber::from(range_min))?;
}
provider.commit()?; provider.commit()?;
} }

View File

@ -13,7 +13,7 @@ use reth_primitives::{
BlockHash, BlockNumHash, BlockNumber, Receipt, SealedBlock, SealedBlockWithSenders, BlockHash, BlockNumHash, BlockNumber, Receipt, SealedBlock, SealedBlockWithSenders,
SealedHeader, SealedHeader,
}; };
use reth_storage_errors::provider::ProviderError; use reth_storage_errors::provider::{ProviderError, ProviderResult};
use std::collections::BTreeMap; use std::collections::BTreeMap;
pub mod error; pub mod error;
@ -69,7 +69,7 @@ pub trait BlockchainTreeEngine: BlockchainTreeViewer + Send + Sync {
) -> Result<InsertPayloadOk, InsertBlockError>; ) -> Result<InsertPayloadOk, InsertBlockError>;
/// Finalize blocks up until and including `finalized_block`, and remove them from the tree. /// Finalize blocks up until and including `finalized_block`, and remove them from the tree.
fn finalize_block(&self, finalized_block: BlockNumber); fn finalize_block(&self, finalized_block: BlockNumber) -> ProviderResult<()>;
/// Reads the last `N` canonical hashes from the database and updates the block indices of the /// Reads the last `N` canonical hashes from the database and updates the block indices of the
/// tree by attempting to connect the buffered blocks to canonical hashes. /// tree by attempting to connect the buffered blocks to canonical hashes.

View File

@ -45,6 +45,7 @@ reth-db = { workspace = true, features = ["test-utils"] }
reth-primitives = { workspace = true, features = ["test-utils"] } reth-primitives = { workspace = true, features = ["test-utils"] }
reth-provider = { workspace = true, features = ["test-utils"] } reth-provider = { workspace = true, features = ["test-utils"] }
reth-evm = { workspace = true, features = ["test-utils"] } reth-evm = { workspace = true, features = ["test-utils"] }
reth-consensus = { workspace = true, features = ["test-utils"] }
reth-testing-utils.workspace = true reth-testing-utils.workspace = true
reth-revm.workspace = true reth-revm.workspace = true
reth-evm-ethereum.workspace = true reth-evm-ethereum.workspace = true

View File

@ -127,18 +127,7 @@ where
let last_canonical_hashes = let last_canonical_hashes =
externals.fetch_latest_canonical_hashes(config.num_of_canonical_hashes() as usize)?; externals.fetch_latest_canonical_hashes(config.num_of_canonical_hashes() as usize)?;
// TODO(rakita) save last finalized block inside database but for now just take let last_finalized_block_number = externals.fetch_latest_finalized_block_number()?;
// `tip - max_reorg_depth`
// https://github.com/paradigmxyz/reth/issues/1712
let last_finalized_block_number = if last_canonical_hashes.len() > max_reorg_depth {
// we pick `Highest - max_reorg_depth` block as last finalized block.
last_canonical_hashes.keys().nth_back(max_reorg_depth)
} else {
// we pick the lowest block as last finalized block.
last_canonical_hashes.keys().next()
}
.copied()
.unwrap_or_default();
Ok(Self { Ok(Self {
externals, externals,
@ -803,7 +792,7 @@ where
} }
/// Finalize blocks up until and including `finalized_block`, and remove them from the tree. /// Finalize blocks up until and including `finalized_block`, and remove them from the tree.
pub fn finalize_block(&mut self, finalized_block: BlockNumber) { pub fn finalize_block(&mut self, finalized_block: BlockNumber) -> ProviderResult<()> {
// remove blocks // remove blocks
let mut remove_chains = self.state.block_indices.finalize_canonical_blocks( let mut remove_chains = self.state.block_indices.finalize_canonical_blocks(
finalized_block, finalized_block,
@ -817,6 +806,11 @@ where
} }
// clean block buffer. // clean block buffer.
self.remove_old_blocks(finalized_block); self.remove_old_blocks(finalized_block);
// save finalized block in db.
self.externals.save_finalized_block_number(finalized_block)?;
Ok(())
} }
/// Reads the last `N` canonical hashes from the database and updates the block indices of the /// Reads the last `N` canonical hashes from the database and updates the block indices of the
@ -834,7 +828,7 @@ where
&mut self, &mut self,
last_finalized_block: BlockNumber, last_finalized_block: BlockNumber,
) -> ProviderResult<()> { ) -> ProviderResult<()> {
self.finalize_block(last_finalized_block); self.finalize_block(last_finalized_block)?;
let last_canonical_hashes = self.update_block_hashes()?; let last_canonical_hashes = self.update_block_hashes()?;
@ -1738,7 +1732,7 @@ mod tests {
tree.make_canonical(B256::ZERO).unwrap(); tree.make_canonical(B256::ZERO).unwrap();
// make genesis block 10 as finalized // make genesis block 10 as finalized
tree.finalize_block(10); tree.finalize_block(10).unwrap();
assert_eq!( assert_eq!(
tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(), tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(),
@ -1814,7 +1808,7 @@ mod tests {
tree.make_canonical(B256::ZERO).unwrap(); tree.make_canonical(B256::ZERO).unwrap();
// make genesis block 10 as finalized // make genesis block 10 as finalized
tree.finalize_block(10); tree.finalize_block(10).unwrap();
assert_eq!( assert_eq!(
tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(), tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(),
@ -1899,7 +1893,7 @@ mod tests {
tree.make_canonical(B256::ZERO).unwrap(); tree.make_canonical(B256::ZERO).unwrap();
// make genesis block 10 as finalized // make genesis block 10 as finalized
tree.finalize_block(10); tree.finalize_block(10).unwrap();
assert_eq!( assert_eq!(
tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(), tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(),
@ -2003,7 +1997,7 @@ mod tests {
tree.is_block_hash_canonical(&B256::ZERO).unwrap(); tree.is_block_hash_canonical(&B256::ZERO).unwrap();
// make genesis block 10 as finalized // make genesis block 10 as finalized
tree.finalize_block(head.number); tree.finalize_block(head.number).unwrap();
// block 2 parent is not known, block2 is buffered. // block 2 parent is not known, block2 is buffered.
assert_eq!( assert_eq!(
@ -2253,7 +2247,7 @@ mod tests {
assert!(tree.is_block_hash_canonical(&block2.hash()).unwrap()); assert!(tree.is_block_hash_canonical(&block2.hash()).unwrap());
// finalize b1 that would make b1a removed from tree // finalize b1 that would make b1a removed from tree
tree.finalize_block(11); tree.finalize_block(11).unwrap();
// Trie state: // Trie state:
// b2 b2a (side chain) // b2 b2a (side chain)
// | / // | /
@ -2357,4 +2351,74 @@ mod tests {
.with_buffered_blocks(HashMap::default()) .with_buffered_blocks(HashMap::default())
.assert(&tree); .assert(&tree);
} }
#[test]
fn last_finalized_block_initialization() {
let data = BlockchainTestData::default_from_number(11);
let (block1, exec1) = data.blocks[0].clone();
let (block2, exec2) = data.blocks[1].clone();
let (block3, exec3) = data.blocks[2].clone();
let genesis = data.genesis;
// test pops execution results from vector, so order is from last to first.
let externals =
setup_externals(vec![exec3.clone(), exec2.clone(), exec1.clone(), exec3, exec2, exec1]);
let cloned_externals_1 = TreeExternals {
provider_factory: externals.provider_factory.clone(),
executor_factory: externals.executor_factory.clone(),
consensus: externals.consensus.clone(),
};
let cloned_externals_2 = TreeExternals {
provider_factory: externals.provider_factory.clone(),
executor_factory: externals.executor_factory.clone(),
consensus: externals.consensus.clone(),
};
// last finalized block would be number 9.
setup_genesis(&externals.provider_factory, genesis);
// make tree
let config = BlockchainTreeConfig::new(1, 2, 3, 2);
let mut tree = BlockchainTree::new(externals, config, None).expect("failed to create tree");
assert_eq!(
tree.insert_block(block1.clone(), BlockValidationKind::Exhaustive).unwrap(),
InsertPayloadOk::Inserted(BlockStatus::Valid(BlockAttachment::Canonical))
);
assert_eq!(
tree.insert_block(block2.clone(), BlockValidationKind::Exhaustive).unwrap(),
InsertPayloadOk::Inserted(BlockStatus::Valid(BlockAttachment::Canonical))
);
assert_eq!(
tree.insert_block(block3, BlockValidationKind::Exhaustive).unwrap(),
InsertPayloadOk::Inserted(BlockStatus::Valid(BlockAttachment::Canonical))
);
tree.make_canonical(block2.hash()).unwrap();
// restart
let mut tree =
BlockchainTree::new(cloned_externals_1, config, None).expect("failed to create tree");
assert_eq!(tree.block_indices().last_finalized_block(), 0);
let mut block1a = block1;
let block1a_hash = B256::new([0x33; 32]);
block1a.set_hash(block1a_hash);
assert_eq!(
tree.insert_block(block1a.clone(), BlockValidationKind::Exhaustive).unwrap(),
InsertPayloadOk::Inserted(BlockStatus::Valid(BlockAttachment::HistoricalFork))
);
tree.make_canonical(block1a.hash()).unwrap();
tree.finalize_block(block1a.number).unwrap();
// restart
let tree =
BlockchainTree::new(cloned_externals_2, config, None).expect("failed to create tree");
assert_eq!(tree.block_indices().last_finalized_block(), block1a.number);
}
} }

View File

@ -4,7 +4,10 @@ use reth_consensus::Consensus;
use reth_db::{static_file::HeaderMask, tables}; use reth_db::{static_file::HeaderMask, tables};
use reth_db_api::{cursor::DbCursorRO, database::Database, transaction::DbTx}; use reth_db_api::{cursor::DbCursorRO, database::Database, transaction::DbTx};
use reth_primitives::{BlockHash, BlockNumber, StaticFileSegment}; use reth_primitives::{BlockHash, BlockNumber, StaticFileSegment};
use reth_provider::{ProviderFactory, StaticFileProviderFactory, StatsReader}; use reth_provider::{
FinalizedBlockReader, FinalizedBlockWriter, ProviderFactory, StaticFileProviderFactory,
StatsReader,
};
use reth_storage_errors::provider::ProviderResult; use reth_storage_errors::provider::ProviderResult;
use std::{collections::BTreeMap, sync::Arc}; use std::{collections::BTreeMap, sync::Arc};
@ -81,4 +84,18 @@ impl<DB: Database, E> TreeExternals<DB, E> {
let hashes = hashes.into_iter().rev().take(num_hashes).collect(); let hashes = hashes.into_iter().rev().take(num_hashes).collect();
Ok(hashes) Ok(hashes)
} }
pub(crate) fn fetch_latest_finalized_block_number(&self) -> ProviderResult<BlockNumber> {
self.provider_factory.provider()?.last_finalized_block_number()
}
pub(crate) fn save_finalized_block_number(
&self,
block_number: BlockNumber,
) -> ProviderResult<()> {
let provider_rw = self.provider_factory.provider_rw()?;
provider_rw.save_finalized_block_number(block_number)?;
provider_rw.commit()?;
Ok(())
}
} }

View File

@ -12,6 +12,7 @@ use reth_provider::{
BlockchainTreePendingStateProvider, CanonStateNotificationSender, CanonStateNotifications, BlockchainTreePendingStateProvider, CanonStateNotificationSender, CanonStateNotifications,
CanonStateSubscriptions, FullBundleStateDataProvider, CanonStateSubscriptions, FullBundleStateDataProvider,
}; };
use reth_storage_errors::provider::ProviderResult;
use std::collections::BTreeMap; use std::collections::BTreeMap;
/// A `BlockchainTree` that does nothing. /// A `BlockchainTree` that does nothing.
@ -49,7 +50,9 @@ impl BlockchainTreeEngine for NoopBlockchainTree {
)) ))
} }
fn finalize_block(&self, _finalized_block: BlockNumber) {} fn finalize_block(&self, _finalized_block: BlockNumber) -> ProviderResult<()> {
Ok(())
}
fn connect_buffered_blocks_to_canonical_hashes_and_finalize( fn connect_buffered_blocks_to_canonical_hashes_and_finalize(
&self, &self,

View File

@ -17,6 +17,7 @@ use reth_provider::{
BlockchainTreePendingStateProvider, CanonStateSubscriptions, FullBundleStateDataProvider, BlockchainTreePendingStateProvider, CanonStateSubscriptions, FullBundleStateDataProvider,
ProviderError, ProviderError,
}; };
use reth_storage_errors::provider::ProviderResult;
use std::{collections::BTreeMap, sync::Arc}; use std::{collections::BTreeMap, sync::Arc};
use tracing::trace; use tracing::trace;
@ -58,11 +59,13 @@ where
res res
} }
fn finalize_block(&self, finalized_block: BlockNumber) { fn finalize_block(&self, finalized_block: BlockNumber) -> ProviderResult<()> {
trace!(target: "blockchain_tree", finalized_block, "Finalizing block"); trace!(target: "blockchain_tree", finalized_block, "Finalizing block");
let mut tree = self.tree.write(); let mut tree = self.tree.write();
tree.finalize_block(finalized_block); tree.finalize_block(finalized_block)?;
tree.update_chains_metrics(); tree.update_chains_metrics();
Ok(())
} }
fn connect_buffered_blocks_to_canonical_hashes_and_finalize( fn connect_buffered_blocks_to_canonical_hashes_and_finalize(

View File

@ -960,7 +960,7 @@ where
.blockchain .blockchain
.find_block_by_hash(finalized_block_hash, BlockSource::Any)? .find_block_by_hash(finalized_block_hash, BlockSource::Any)?
.ok_or_else(|| ProviderError::UnknownBlockHash(finalized_block_hash))?; .ok_or_else(|| ProviderError::UnknownBlockHash(finalized_block_hash))?;
self.blockchain.finalize_block(finalized.number); self.blockchain.finalize_block(finalized.number)?;
self.blockchain.set_finalized(finalized.header.seal(finalized_block_hash)); self.blockchain.set_finalized(finalized.header.seal(finalized_block_hash));
} }
Ok(()) Ok(())

View File

@ -11,8 +11,8 @@ use reth_primitives::{
BlockNumber, B256, BlockNumber, B256,
}; };
use reth_provider::{ use reth_provider::{
providers::StaticFileWriter, ProviderFactory, StageCheckpointReader, StageCheckpointWriter, providers::StaticFileWriter, FinalizedBlockReader, FinalizedBlockWriter, ProviderFactory,
StaticFileProviderFactory, StageCheckpointReader, StageCheckpointWriter, StaticFileProviderFactory,
}; };
use reth_prune::PrunerBuilder; use reth_prune::PrunerBuilder;
use reth_static_file::StaticFileProducer; use reth_static_file::StaticFileProducer;
@ -353,6 +353,15 @@ where
self.event_sender self.event_sender
.notify(PipelineEvent::Unwound { stage_id, result: unwind_output }); .notify(PipelineEvent::Unwound { stage_id, result: unwind_output });
// update finalized block if needed
let last_saved_finalized_block_number =
provider_rw.last_finalized_block_number()?;
if checkpoint.block_number < last_saved_finalized_block_number {
provider_rw.save_finalized_block_number(BlockNumber::from(
checkpoint.block_number,
))?;
}
// For unwinding it makes more sense to commit the database first, since if // For unwinding it makes more sense to commit the database first, since if
// this function is interrupted before the static files commit, we can just // this function is interrupted before the static files commit, we can just
// truncate the static files according to the // truncate the static files according to the

View File

@ -4,8 +4,7 @@
//! //!
//! This module defines the tables in reth, as well as some table-related abstractions: //! This module defines the tables in reth, as well as some table-related abstractions:
//! //!
//! - [`codecs`] integrates different codecs into [`Encode`](reth_db_api::table::Encode) and //! - [`codecs`] integrates different codecs into [`Encode`] and [`Decode`]
//! [`Decode`](reth_db_api::table::Decode)
//! - [`models`](reth_db_api::models) defines the values written to tables //! - [`models`](reth_db_api::models) defines the values written to tables
//! //!
//! # Database Tour //! # Database Tour
@ -30,7 +29,7 @@ use reth_db_api::{
storage_sharded_key::StorageShardedKey, storage_sharded_key::StorageShardedKey,
CompactU256, ShardedKey, StoredBlockBodyIndices, StoredBlockWithdrawals, CompactU256, ShardedKey, StoredBlockBodyIndices, StoredBlockWithdrawals,
}, },
table::{DupSort, Table}, table::{Decode, DupSort, Encode, Table},
}; };
use reth_primitives::{ use reth_primitives::{
stage::StageCheckpoint, stage::StageCheckpoint,
@ -38,6 +37,7 @@ use reth_primitives::{
Account, Address, BlockHash, BlockNumber, Bytecode, Header, IntegerList, PruneCheckpoint, Account, Address, BlockHash, BlockNumber, Bytecode, Header, IntegerList, PruneCheckpoint,
PruneSegment, Receipt, Requests, StorageEntry, TransactionSignedNoHash, TxHash, TxNumber, B256, PruneSegment, Receipt, Requests, StorageEntry, TransactionSignedNoHash, TxHash, TxNumber, B256,
}; };
use serde::{Deserialize, Serialize};
use std::fmt; use std::fmt;
/// Enum for the types of tables present in libmdbx. /// Enum for the types of tables present in libmdbx.
@ -406,6 +406,36 @@ tables! {
/// Stores EIP-7685 EL -> CL requests, indexed by block number. /// Stores EIP-7685 EL -> CL requests, indexed by block number.
table BlockRequests<Key = BlockNumber, Value = Requests>; table BlockRequests<Key = BlockNumber, Value = Requests>;
/// Stores generic chain state info, like the last finalized block.
table ChainState<Key = ChainStateKey, Value = BlockNumber>;
}
/// Keys for the `ChainState` table.
#[derive(Ord, Clone, Eq, PartialOrd, PartialEq, Debug, Deserialize, Serialize, Hash)]
pub enum ChainStateKey {
/// Last finalized block key
LastFinalizedBlock,
}
impl Encode for ChainStateKey {
type Encoded = [u8; 1];
fn encode(self) -> Self::Encoded {
match self {
Self::LastFinalizedBlock => [0],
}
}
}
impl Decode for ChainStateKey {
fn decode<B: AsRef<[u8]>>(value: B) -> Result<Self, reth_db_api::DatabaseError> {
if value.as_ref() == [0] {
Ok(Self::LastFinalizedBlock)
} else {
Err(reth_db_api::DatabaseError::Decode)
}
}
} }
// Alias types. // Alias types.

View File

@ -6,11 +6,11 @@ use crate::{
AccountExtReader, BlockSource, ChangeSetReader, ReceiptProvider, StageCheckpointWriter, AccountExtReader, BlockSource, ChangeSetReader, ReceiptProvider, StageCheckpointWriter,
}, },
AccountReader, BlockExecutionWriter, BlockHashReader, BlockNumReader, BlockReader, BlockWriter, AccountReader, BlockExecutionWriter, BlockHashReader, BlockNumReader, BlockReader, BlockWriter,
Chain, EvmEnvProvider, HashingWriter, HeaderProvider, HeaderSyncGap, HeaderSyncGapProvider, Chain, EvmEnvProvider, FinalizedBlockReader, FinalizedBlockWriter, HashingWriter,
HeaderSyncMode, HistoricalStateProvider, HistoryWriter, LatestStateProvider, HeaderProvider, HeaderSyncGap, HeaderSyncGapProvider, HeaderSyncMode, HistoricalStateProvider,
OriginalValuesKnown, ProviderError, PruneCheckpointReader, PruneCheckpointWriter, HistoryWriter, LatestStateProvider, OriginalValuesKnown, ProviderError, PruneCheckpointReader,
RequestsProvider, StageCheckpointReader, StateProviderBox, StateWriter, StatsReader, PruneCheckpointWriter, RequestsProvider, StageCheckpointReader, StateProviderBox, StateWriter,
StorageReader, TransactionVariant, TransactionsProvider, TransactionsProviderExt, StatsReader, StorageReader, TransactionVariant, TransactionsProvider, TransactionsProviderExt,
WithdrawalsProvider, WithdrawalsProvider,
}; };
use itertools::{izip, Itertools}; use itertools::{izip, Itertools};
@ -2753,6 +2753,30 @@ impl<TX: DbTx> StatsReader for DatabaseProvider<TX> {
} }
} }
impl<TX: DbTx> FinalizedBlockReader for DatabaseProvider<TX> {
fn last_finalized_block_number(&self) -> ProviderResult<BlockNumber> {
let mut finalized_blocks = self
.tx
.cursor_read::<tables::ChainState>()?
.walk(Some(tables::ChainStateKey::LastFinalizedBlock))?
.take(1)
.collect::<Result<BTreeMap<tables::ChainStateKey, BlockNumber>, _>>()?;
let last_finalized_block_number = finalized_blocks
.pop_first()
.unwrap_or((tables::ChainStateKey::LastFinalizedBlock, 0_u64));
Ok(last_finalized_block_number.1)
}
}
impl<TX: DbTxMut> FinalizedBlockWriter for DatabaseProvider<TX> {
fn save_finalized_block_number(&self, block_number: BlockNumber) -> ProviderResult<()> {
Ok(self
.tx
.put::<tables::ChainState>(tables::ChainStateKey::LastFinalizedBlock, block_number)?)
}
}
fn range_size_hint(range: &impl RangeBounds<TxNumber>) -> Option<usize> { fn range_size_hint(range: &impl RangeBounds<TxNumber>) -> Option<usize> {
let start = match range.start_bound().cloned() { let start = match range.start_bound().cloned() {
Bound::Included(start) => start, Bound::Included(start) => start,

View File

@ -684,7 +684,7 @@ where
self.tree.insert_block(block, validation_kind) self.tree.insert_block(block, validation_kind)
} }
fn finalize_block(&self, finalized_block: BlockNumber) { fn finalize_block(&self, finalized_block: BlockNumber) -> ProviderResult<()> {
self.tree.finalize_block(finalized_block) self.tree.finalize_block(finalized_block)
} }

View File

@ -0,0 +1,14 @@
use reth_errors::ProviderResult;
use reth_primitives::BlockNumber;
/// Functionality to read the last known finalized block from the database.
pub trait FinalizedBlockReader: Send + Sync {
/// Returns the last finalized block number.
fn last_finalized_block_number(&self) -> ProviderResult<BlockNumber>;
}
/// Functionality to write the last known finalized block to the database.
pub trait FinalizedBlockWriter: Send + Sync {
/// Saves the given finalized block number in the DB.
fn save_finalized_block_number(&self, block_number: BlockNumber) -> ProviderResult<()>;
}

View File

@ -47,3 +47,6 @@ pub use full::FullProvider;
mod tree_viewer; mod tree_viewer;
pub use tree_viewer::TreeViewer; pub use tree_viewer::TreeViewer;
mod finalized_block;
pub use finalized_block::{FinalizedBlockReader, FinalizedBlockWriter};