Re-arrange coordinator/

coordinator/tributary was tributary-chain. This crate has been renamed
tributary-sdk and moved to coordinator/tributary-sdk.

coordinator/src/tributary was our instantion of a Tributary, the Transaction
type and scan task. This has been moved to coordinator/tributary.

The main reason for this was due to coordinator/main.rs becoming untidy. There
is now a collection of clean, independent APIs present in the codebase.
coordinator/main.rs is to compose them. Sometimes, these compositions are a bit
silly (reading from a channel just to forward the message to a distinct
channel). That's more than fine as the code is still readable and the value
from the cleanliness of the APIs composed far exceeds the nits from having
these odd compositions.

This breaks down a bit as we now define a global database, and have some APIs
interact with multiple other APIs.

coordinator/src/tributary was a self-contained, clean API. The recently added
task present in coordinator/tributary/mod.rs, which bound it to the rest of the
Coordinator, wasn't.

Now, coordinator/src is solely the API compositions, and all self-contained
APIs are their own crates.
This commit is contained in:
Luke Parker
2025-01-11 04:14:21 -05:00
parent c05b0c9eba
commit 3c664ff05f
56 changed files with 1719 additions and 1570 deletions

View File

@@ -1,388 +1,513 @@
use core::{marker::PhantomData, fmt::Debug, future::Future};
use std::{sync::Arc, io};
#![cfg_attr(docsrs, feature(doc_auto_cfg))]
#![doc = include_str!("../README.md")]
#![deny(missing_docs)]
use zeroize::Zeroizing;
use core::{marker::PhantomData, future::Future};
use std::collections::HashMap;
use ciphersuite::{Ciphersuite, Ristretto};
use ciphersuite::group::GroupEncoding;
use scale::Decode;
use futures_channel::mpsc::UnboundedReceiver;
use futures_util::{StreamExt, SinkExt};
use ::tendermint::{
ext::{BlockNumber, Commit, Block as BlockTrait, Network},
SignedMessageFor, SyncedBlock, SyncedBlockSender, SyncedBlockResultReceiver, MessageSender,
TendermintMachine, TendermintHandle,
use serai_client::{
primitives::SeraiAddress,
validator_sets::primitives::{ValidatorSet, Slash},
};
pub use ::tendermint::Evidence;
use serai_db::*;
use serai_task::ContinuallyRan;
use serai_db::Db;
use tributary_sdk::{
tendermint::{
tx::{TendermintTx, Evidence, decode_signed_message},
TendermintNetwork,
},
Signed as TributarySigned, TransactionKind, TransactionTrait,
Transaction as TributaryTransaction, Block, TributaryReader, P2p,
};
use tokio::sync::RwLock;
use serai_cosign::Cosigning;
use serai_coordinator_substrate::NewSetInformation;
mod merkle;
pub(crate) use merkle::*;
use messages::sign::VariantSignId;
pub mod transaction;
pub use transaction::{TransactionError, Signed, TransactionKind, Transaction as TransactionTrait};
mod transaction;
pub(crate) use transaction::{SigningProtocolRound, Signed};
pub use transaction::Transaction;
use crate::tendermint::tx::TendermintTx;
mod db;
use db::*;
mod provided;
pub(crate) use provided::*;
pub use provided::ProvidedError;
mod block;
pub use block::*;
mod blockchain;
pub(crate) use blockchain::*;
mod mempool;
pub(crate) use mempool::*;
pub mod tendermint;
pub(crate) use crate::tendermint::*;
#[cfg(any(test, feature = "tests"))]
pub mod tests;
/// Size limit for an individual transaction.
// This needs to be big enough to participate in a 101-of-150 eVRF DKG with each element taking
// `MAX_KEY_LEN`. This also needs to be big enough to pariticpate in signing 520 Bitcoin inputs
// with 49 key shares, and signing 120 Monero inputs with 49 key shares.
// TODO: Add a test for these properties
pub const TRANSACTION_SIZE_LIMIT: usize = 2_000_000;
/// Amount of transactions a single account may have in the mempool.
pub const ACCOUNT_MEMPOOL_LIMIT: u32 = 50;
/// Block size limit.
// This targets a growth limit of roughly 30 GB a day, under load, in order to prevent a malicious
// participant from flooding disks and causing out of space errors in order processes.
pub const BLOCK_SIZE_LIMIT: usize = 2_001_000;
pub(crate) const TENDERMINT_MESSAGE: u8 = 0;
pub(crate) const TRANSACTION_MESSAGE: u8 = 1;
#[allow(clippy::large_enum_variant)]
#[derive(Clone, PartialEq, Eq, Debug)]
pub enum Transaction<T: TransactionTrait> {
Tendermint(TendermintTx),
Application(T),
/// Messages to send to the Processors.
pub struct ProcessorMessages;
impl ProcessorMessages {
/// Try to receive a message to send to a Processor.
pub fn try_recv(txn: &mut impl DbTxn, set: ValidatorSet) -> Option<messages::CoordinatorMessage> {
db::ProcessorMessages::try_recv(txn, set)
}
}
impl<T: TransactionTrait> ReadWrite for Transaction<T> {
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let mut kind = [0];
reader.read_exact(&mut kind)?;
match kind[0] {
0 => {
let tx = TendermintTx::read(reader)?;
Ok(Transaction::Tendermint(tx))
}
1 => {
let tx = T::read(reader)?;
Ok(Transaction::Application(tx))
}
_ => Err(io::Error::other("invalid transaction type")),
struct ScanBlock<'a, CD: Db, TD: Db, TDT: DbTxn, P: P2p> {
_td: PhantomData<TD>,
_p2p: PhantomData<P>,
cosign_db: &'a CD,
tributary_txn: &'a mut TDT,
set: ValidatorSet,
validators: &'a [SeraiAddress],
total_weight: u64,
validator_weights: &'a HashMap<SeraiAddress, u64>,
}
impl<'a, CD: Db, TD: Db, TDT: DbTxn, P: P2p> ScanBlock<'a, CD, TD, TDT, P> {
fn potentially_start_cosign(&mut self) {
// Don't start a new cosigning instance if we're actively running one
if TributaryDb::actively_cosigning(self.tributary_txn, self.set).is_some() {
return;
}
}
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
match self {
Transaction::Tendermint(tx) => {
writer.write_all(&[0])?;
tx.write(writer)
}
Transaction::Application(tx) => {
writer.write_all(&[1])?;
tx.write(writer)
}
}
}
}
impl<T: TransactionTrait> Transaction<T> {
pub fn hash(&self) -> [u8; 32] {
match self {
Transaction::Tendermint(tx) => tx.hash(),
Transaction::Application(tx) => tx.hash(),
}
}
pub fn kind(&self) -> TransactionKind {
match self {
Transaction::Tendermint(tx) => tx.kind(),
Transaction::Application(tx) => tx.kind(),
}
}
}
/// An item which can be read and written.
pub trait ReadWrite: Sized {
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self>;
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()>;
fn serialize(&self) -> Vec<u8> {
// BlockHeader is 64 bytes and likely the smallest item in this system
let mut buf = Vec::with_capacity(64);
self.write(&mut buf).unwrap();
buf
}
}
pub trait P2p: 'static + Send + Sync + Clone {
/// Broadcast a message to all other members of the Tributary with the specified genesis.
///
/// The Tributary will re-broadcast consensus messages on a fixed interval to ensure they aren't
/// prematurely dropped from the P2P layer. THe P2P layer SHOULD perform content-based
/// deduplication to ensure a sane amount of load.
fn broadcast(&self, genesis: [u8; 32], msg: Vec<u8>) -> impl Send + Future<Output = ()>;
}
impl<P: P2p> P2p for Arc<P> {
fn broadcast(&self, genesis: [u8; 32], msg: Vec<u8>) -> impl Send + Future<Output = ()> {
P::broadcast(self, genesis, msg)
}
}
#[derive(Clone)]
pub struct Tributary<D: Db, T: TransactionTrait, P: P2p> {
db: D,
genesis: [u8; 32],
network: TendermintNetwork<D, T, P>,
synced_block: Arc<RwLock<SyncedBlockSender<TendermintNetwork<D, T, P>>>>,
synced_block_result: Arc<RwLock<SyncedBlockResultReceiver>>,
messages: Arc<RwLock<MessageSender<TendermintNetwork<D, T, P>>>>,
}
impl<D: Db, T: TransactionTrait, P: P2p> Tributary<D, T, P> {
pub async fn new(
db: D,
genesis: [u8; 32],
start_time: u64,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
validators: Vec<(<Ristretto as Ciphersuite>::G, u64)>,
p2p: P,
) -> Option<Self> {
log::info!("new Tributary with genesis {}", hex::encode(genesis));
let validators_vec = validators.iter().map(|validator| validator.0).collect::<Vec<_>>();
let signer = Arc::new(Signer::new(genesis, key));
let validators = Arc::new(Validators::new(genesis, validators)?);
let mut blockchain = Blockchain::new(db.clone(), genesis, &validators_vec);
let block_number = BlockNumber(blockchain.block_number());
let start_time = if let Some(commit) = blockchain.commit(&blockchain.tip()) {
Commit::<Validators>::decode(&mut commit.as_ref()).unwrap().end_time
} else {
start_time
// Fetch the latest intended-to-be-cosigned block
let Some(latest_substrate_block_to_cosign) =
TributaryDb::latest_substrate_block_to_cosign(self.tributary_txn, self.set)
else {
return;
};
let proposal = TendermintBlock(
blockchain.build_block::<TendermintNetwork<D, T, P>>(&validators).serialize(),
// If it was already cosigned, return
if TributaryDb::cosigned(self.tributary_txn, self.set, latest_substrate_block_to_cosign) {
return;
}
let Some(substrate_block_number) =
Cosigning::<CD>::finalized_block_number(self.cosign_db, latest_substrate_block_to_cosign)
else {
// This is a valid panic as we shouldn't be scanning this block if we didn't provide all
// Provided transactions within it, and the block to cosign is a Provided transaction
panic!("cosigning a block our cosigner didn't index")
};
// Mark us as actively cosigning
TributaryDb::start_cosigning(
self.tributary_txn,
self.set,
latest_substrate_block_to_cosign,
substrate_block_number,
);
let blockchain = Arc::new(RwLock::new(blockchain));
let network = TendermintNetwork { genesis, signer, validators, blockchain, p2p };
let TendermintHandle { synced_block, synced_block_result, messages, machine } =
TendermintMachine::new(
db.clone(),
network.clone(),
genesis,
block_number,
start_time,
proposal,
)
.await;
tokio::spawn(machine.run());
Some(Self {
db,
genesis,
network,
synced_block: Arc::new(RwLock::new(synced_block)),
synced_block_result: Arc::new(RwLock::new(synced_block_result)),
messages: Arc::new(RwLock::new(messages)),
})
}
pub fn block_time() -> u32 {
TendermintNetwork::<D, T, P>::block_time()
}
pub fn genesis(&self) -> [u8; 32] {
self.genesis
}
pub async fn block_number(&self) -> u64 {
self.network.blockchain.read().await.block_number()
}
pub async fn tip(&self) -> [u8; 32] {
self.network.blockchain.read().await.tip()
}
pub fn reader(&self) -> TributaryReader<D, T> {
TributaryReader(self.db.clone(), self.genesis, PhantomData)
}
pub async fn provide_transaction(&self, tx: T) -> Result<(), ProvidedError> {
self.network.blockchain.write().await.provide_transaction(tx)
}
pub async fn next_nonce(
&self,
signer: &<Ristretto as Ciphersuite>::G,
order: &[u8],
) -> Option<u32> {
self.network.blockchain.read().await.next_nonce(signer, order)
}
// Returns Ok(true) if new, Ok(false) if an already present unsigned, or the error.
// Safe to be &self since the only meaningful usage of self is self.network.blockchain which
// successfully acquires its own write lock
pub async fn add_transaction(&self, tx: T) -> Result<bool, TransactionError> {
let tx = Transaction::Application(tx);
let mut to_broadcast = vec![TRANSACTION_MESSAGE];
tx.write(&mut to_broadcast).unwrap();
let res = self.network.blockchain.write().await.add_transaction::<TendermintNetwork<D, T, P>>(
true,
tx,
&self.network.signature_scheme(),
// Send the message for the processor to start signing
TributaryDb::send_message(
self.tributary_txn,
self.set,
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
session: self.set.session,
block_number: substrate_block_number,
block: latest_substrate_block_to_cosign,
},
);
if res == Ok(true) {
self.network.p2p.broadcast(self.genesis, to_broadcast).await;
}
res
}
fn handle_application_tx(&mut self, block_number: u64, tx: Transaction) {
let signer = |signed: Signed| SeraiAddress(signed.signer().to_bytes());
async fn sync_block_internal(
&self,
block: Block<T>,
commit: Vec<u8>,
result: &mut UnboundedReceiver<bool>,
) -> bool {
let (tip, block_number) = {
let blockchain = self.network.blockchain.read().await;
(blockchain.tip(), blockchain.block_number())
};
if block.header.parent != tip {
log::debug!("told to sync a block whose parent wasn't our tip");
return false;
if let TransactionKind::Signed(_, TributarySigned { signer, .. }) = tx.kind() {
// Don't handle transactions from those fatally slashed
// TODO: The fact they can publish these TXs makes this a notable spam vector
if TributaryDb::is_fatally_slashed(
self.tributary_txn,
self.set,
SeraiAddress(signer.to_bytes()),
) {
return;
}
}
let block = TendermintBlock(block.serialize());
let mut commit_ref = commit.as_ref();
let Ok(commit) = Commit::<Arc<Validators>>::decode(&mut commit_ref) else {
log::error!("sent an invalidly serialized commit");
return false;
};
// Storage DoS vector. We *could* truncate to solely the relevant portion, trying to save this,
// yet then we'd have to test the truncation was performed correctly.
if !commit_ref.is_empty() {
log::error!("sent an commit with additional data after it");
return false;
}
if !self.network.verify_commit(block.id(), &commit) {
log::error!("sent an invalid commit");
return false;
}
match tx {
// Accumulate this vote and fatally slash the participant if past the threshold
Transaction::RemoveParticipant { participant, signed } => {
let signer = signer(signed);
let number = BlockNumber(block_number + 1);
self.synced_block.write().await.send(SyncedBlock { number, block, commit }).await.unwrap();
result.next().await.unwrap()
}
// Sync a block.
// TODO: Since we have a static validator set, we should only need the tail commit?
pub async fn sync_block(&self, block: Block<T>, commit: Vec<u8>) -> bool {
let mut result = self.synced_block_result.write().await;
self.sync_block_internal(block, commit, &mut result).await
}
// Return true if the message should be rebroadcasted.
pub async fn handle_message(&self, msg: &[u8]) -> bool {
match msg.first() {
Some(&TRANSACTION_MESSAGE) => {
let Ok(tx) = Transaction::read::<&[u8]>(&mut &msg[1 ..]) else {
log::error!("received invalid transaction message");
return false;
};
// TODO: Sync mempools with fellow peers
// Can we just rebroadcast transactions not included for at least two blocks?
let res =
self.network.blockchain.write().await.add_transaction::<TendermintNetwork<D, T, P>>(
false,
tx,
&self.network.signature_scheme(),
// Check the participant voted to be removed actually exists
if !self.validators.iter().any(|validator| *validator == participant) {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set,
signer,
"voted to remove non-existent participant",
);
log::debug!("received transaction message. valid new transaction: {res:?}");
res == Ok(true)
}
return;
}
Some(&TENDERMINT_MESSAGE) => {
let Ok(msg) =
SignedMessageFor::<TendermintNetwork<D, T, P>>::decode::<&[u8]>(&mut &msg[1 ..])
else {
log::error!("received invalid tendermint message");
return false;
match TributaryDb::accumulate(
self.tributary_txn,
self.set,
self.validators,
self.total_weight,
block_number,
Topic::RemoveParticipant { participant },
signer,
self.validator_weights[&signer],
&(),
) {
DataSet::None => {}
DataSet::Participating(_) => {
TributaryDb::fatal_slash(self.tributary_txn, self.set, participant, "voted to remove");
}
};
self.messages.write().await.send(msg).await.unwrap();
false
}
_ => false,
// Send the participation to the processor
Transaction::DkgParticipation { participation, signed } => {
TributaryDb::send_message(
self.tributary_txn,
self.set,
messages::key_gen::CoordinatorMessage::Participation {
session: self.set.session,
participant: todo!("TODO"),
participation,
},
);
}
Transaction::DkgConfirmationPreprocess { attempt, preprocess, signed } => {
// Accumulate the preprocesses into our own FROST attempt manager
todo!("TODO")
}
Transaction::DkgConfirmationShare { attempt, share, signed } => {
// Accumulate the shares into our own FROST attempt manager
todo!("TODO")
}
Transaction::Cosign { substrate_block_hash } => {
// Update the latest intended-to-be-cosigned Substrate block
TributaryDb::set_latest_substrate_block_to_cosign(
self.tributary_txn,
self.set,
substrate_block_hash,
);
// Start a new cosign if we aren't already working on one
self.potentially_start_cosign();
}
Transaction::Cosigned { substrate_block_hash } => {
/*
We provide one Cosigned per Cosign transaction, but they have independent orders. This
means we may receive Cosigned before Cosign. In order to ensure we only start work on
not-yet-Cosigned cosigns, we flag all cosigned blocks as cosigned. Then, when we choose
the next block to work on, we won't if it's already been cosigned.
*/
TributaryDb::mark_cosigned(self.tributary_txn, self.set, substrate_block_hash);
// If we aren't actively cosigning this block, return
// This occurs when we have Cosign TXs A, B, C, we received Cosigned for A and start on C,
// and then receive Cosigned for B
if TributaryDb::actively_cosigning(self.tributary_txn, self.set) !=
Some(substrate_block_hash)
{
return;
}
// Since this is the block we were cosigning, mark us as having finished cosigning
TributaryDb::finish_cosigning(self.tributary_txn, self.set);
// Start working on the next cosign
self.potentially_start_cosign();
}
Transaction::SubstrateBlock { hash } => {
// Whitelist all of the IDs this Substrate block causes to be signed
todo!("TODO")
}
Transaction::Batch { hash } => {
// Whitelist the signing of this batch, publishing our own preprocess
todo!("TODO")
}
Transaction::SlashReport { slash_points, signed } => {
let signer = signer(signed);
if slash_points.len() != self.validators.len() {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set,
signer,
"slash report was for a distinct amount of signers",
);
return;
}
// Accumulate, and if past the threshold, calculate *the* slash report and start signing it
match TributaryDb::accumulate(
self.tributary_txn,
self.set,
self.validators,
self.total_weight,
block_number,
Topic::SlashReport,
signer,
self.validator_weights[&signer],
&slash_points,
) {
DataSet::None => {}
DataSet::Participating(data_set) => {
// Find the median reported slashes for this validator
/*
TODO: This lets 34% perform a fatal slash. That shouldn't be allowed. We need
to accept slash reports for a period past the threshold, and only fatally slash if we
have a supermajority agree the slash should be fatal. If there isn't a supermajority,
but the median believe the slash should be fatal, we need to fallback to a large
constant.
Also, TODO, each slash point should probably be considered as
`MAX_KEY_SHARES_PER_SET * BLOCK_TIME` seconds of downtime. As this time crosses
various thresholds (1 day, 3 days, etc), a multiplier should be attached.
*/
let mut median_slash_report = Vec::with_capacity(self.validators.len());
for i in 0 .. self.validators.len() {
let mut this_validator =
data_set.values().map(|report| report[i]).collect::<Vec<_>>();
this_validator.sort_unstable();
// Choose the median, where if there are two median values, the lower one is chosen
let median_index = if (this_validator.len() % 2) == 1 {
this_validator.len() / 2
} else {
(this_validator.len() / 2) - 1
};
median_slash_report.push(this_validator[median_index]);
}
// We only publish slashes for the `f` worst performers to:
// 1) Effect amnesty if there were network disruptions which affected everyone
// 2) Ensure the signing threshold doesn't have a disincentive to do their job
// Find the worst performer within the signing threshold's slash points
let f = (self.validators.len() - 1) / 3;
let worst_validator_in_supermajority_slash_points = {
let mut sorted_slash_points = median_slash_report.clone();
sorted_slash_points.sort_unstable();
// This won't be a valid index if `f == 0`, which means we don't have any validators
// to slash
let index_of_first_validator_to_slash = self.validators.len() - f;
let index_of_worst_validator_in_supermajority = index_of_first_validator_to_slash - 1;
sorted_slash_points[index_of_worst_validator_in_supermajority]
};
// Perform the amortization
for slash_points in &mut median_slash_report {
*slash_points =
slash_points.saturating_sub(worst_validator_in_supermajority_slash_points)
}
let amortized_slash_report = median_slash_report;
// Create the resulting slash report
let mut slash_report = vec![];
for (validator, points) in self.validators.iter().copied().zip(amortized_slash_report) {
if points != 0 {
slash_report.push(Slash { key: validator.into(), points });
}
}
assert!(slash_report.len() <= f);
// Recognize the topic for signing the slash report
TributaryDb::recognize_topic(
self.tributary_txn,
self.set,
Topic::Sign {
id: VariantSignId::SlashReport,
attempt: 0,
round: SigningProtocolRound::Preprocess,
},
);
// Send the message for the processor to start signing
TributaryDb::send_message(
self.tributary_txn,
self.set,
messages::coordinator::CoordinatorMessage::SignSlashReport {
session: self.set.session,
report: slash_report,
},
);
}
};
}
Transaction::Sign { id, attempt, round, data, signed } => {
let topic = Topic::Sign { id, attempt, round };
let signer = signer(signed);
if u64::try_from(data.len()).unwrap() != self.validator_weights[&signer] {
TributaryDb::fatal_slash(
self.tributary_txn,
self.set,
signer,
"signer signed with a distinct amount of key shares than they had key shares",
);
return;
}
match TributaryDb::accumulate(
self.tributary_txn,
self.set,
self.validators,
self.total_weight,
block_number,
topic,
signer,
self.validator_weights[&signer],
&data,
) {
DataSet::None => {}
DataSet::Participating(data_set) => {
let id = topic.sign_id(self.set).expect("Topic::Sign didn't have SignId");
let flatten_data_set = |data_set| todo!("TODO");
let data_set = flatten_data_set(data_set);
TributaryDb::send_message(
self.tributary_txn,
self.set,
match round {
SigningProtocolRound::Preprocess => {
messages::sign::CoordinatorMessage::Preprocesses { id, preprocesses: data_set }
}
SigningProtocolRound::Share => {
messages::sign::CoordinatorMessage::Shares { id, shares: data_set }
}
},
)
}
};
}
}
}
/// Get a Future which will resolve once the next block has been added.
pub async fn next_block_notification(
&self,
) -> impl Send + Sync + core::future::Future<Output = Result<(), impl Send + Sync>> {
let (tx, rx) = tokio::sync::oneshot::channel();
self.network.blockchain.write().await.next_block_notifications.push_back(tx);
rx
fn handle_block(mut self, block_number: u64, block: Block<Transaction>) {
TributaryDb::start_of_block(self.tributary_txn, self.set, block_number);
for tx in block.transactions {
match tx {
TributaryTransaction::Tendermint(TendermintTx::SlashEvidence(ev)) => {
// Since the evidence is on the chain, it will have already been validated
// We can just punish the signer
let data = match ev {
Evidence::ConflictingMessages(first, second) => (first, Some(second)),
Evidence::InvalidPrecommit(first) | Evidence::InvalidValidRound(first) => (first, None),
};
let msgs = (
decode_signed_message::<TendermintNetwork<TD, Transaction, P>>(&data.0).unwrap(),
if data.1.is_some() {
Some(
decode_signed_message::<TendermintNetwork<TD, Transaction, P>>(&data.1.unwrap())
.unwrap(),
)
} else {
None
},
);
// Since anything with evidence is fundamentally faulty behavior, not just temporal
// errors, mark the node as fatally slashed
TributaryDb::fatal_slash(
self.tributary_txn,
self.set,
SeraiAddress(msgs.0.msg.sender),
&format!("invalid tendermint messages: {msgs:?}"),
);
}
TributaryTransaction::Application(tx) => {
self.handle_application_tx(block_number, tx);
}
}
}
}
}
#[derive(Clone)]
pub struct TributaryReader<D: Db, T: TransactionTrait>(D, [u8; 32], PhantomData<T>);
impl<D: Db, T: TransactionTrait> TributaryReader<D, T> {
pub fn genesis(&self) -> [u8; 32] {
self.1
}
/// The task to scan the Tributary, populating `ProcessorMessages`.
pub struct ScanTributaryTask<CD: Db, TD: Db, P: P2p> {
cosign_db: CD,
tributary_db: TD,
set: ValidatorSet,
validators: Vec<SeraiAddress>,
total_weight: u64,
validator_weights: HashMap<SeraiAddress, u64>,
tributary: TributaryReader<TD, Transaction>,
_p2p: PhantomData<P>,
}
// Since these values are static once set, they can be safely read from the database without lock
// acquisition
pub fn block(&self, hash: &[u8; 32]) -> Option<Block<T>> {
Blockchain::<D, T>::block_from_db(&self.0, self.1, hash)
}
pub fn commit(&self, hash: &[u8; 32]) -> Option<Vec<u8>> {
Blockchain::<D, T>::commit_from_db(&self.0, self.1, hash)
}
pub fn parsed_commit(&self, hash: &[u8; 32]) -> Option<Commit<Validators>> {
self.commit(hash).map(|commit| Commit::<Validators>::decode(&mut commit.as_ref()).unwrap())
}
pub fn block_after(&self, hash: &[u8; 32]) -> Option<[u8; 32]> {
Blockchain::<D, T>::block_after(&self.0, self.1, hash)
}
pub fn time_of_block(&self, hash: &[u8; 32]) -> Option<u64> {
self
.commit(hash)
.map(|commit| Commit::<Validators>::decode(&mut commit.as_ref()).unwrap().end_time)
}
impl<CD: Db, TD: Db, P: P2p> ScanTributaryTask<CD, TD, P> {
/// Create a new instance of this task.
pub fn new(
cosign_db: CD,
tributary_db: TD,
new_set: &NewSetInformation,
tributary: TributaryReader<TD, Transaction>,
) -> Self {
let mut validators = Vec::with_capacity(new_set.validators.len());
let mut total_weight = 0;
let mut validator_weights = HashMap::with_capacity(new_set.validators.len());
for (validator, weight) in new_set.validators.iter().copied() {
let validator = SeraiAddress::from(validator);
let weight = u64::from(weight);
validators.push(validator);
total_weight += weight;
validator_weights.insert(validator, weight);
}
pub fn locally_provided_txs_in_block(&self, hash: &[u8; 32], order: &str) -> bool {
Blockchain::<D, T>::locally_provided_txs_in_block(&self.0, &self.1, hash, order)
}
// This isn't static, yet can be read with only minor discrepancy risks
pub fn tip(&self) -> [u8; 32] {
Blockchain::<D, T>::tip_from_db(&self.0, self.1)
ScanTributaryTask {
cosign_db,
tributary_db,
set: new_set.set,
validators,
total_weight,
validator_weights,
tributary,
_p2p: PhantomData,
}
}
}
impl<CD: Db, TD: Db, P: P2p> ContinuallyRan for ScanTributaryTask<CD, TD, P> {
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
async move {
let (mut last_block_number, mut last_block_hash) =
TributaryDb::last_handled_tributary_block(&self.tributary_db, self.set)
.unwrap_or((0, self.tributary.genesis()));
let mut made_progress = false;
while let Some(next) = self.tributary.block_after(&last_block_hash) {
let block = self.tributary.block(&next).unwrap();
let block_number = last_block_number + 1;
let block_hash = block.hash();
// Make sure we have all of the provided transactions for this block
for tx in &block.transactions {
let TransactionKind::Provided(order) = tx.kind() else {
continue;
};
// make sure we have all the provided txs in this block locally
if !self.tributary.locally_provided_txs_in_block(&block_hash, order) {
return Err(format!(
"didn't have the provided Transactions on-chain for set (ephemeral error): {:?}",
self.set
));
}
}
let mut tributary_txn = self.tributary_db.txn();
(ScanBlock {
_td: PhantomData::<TD>,
_p2p: PhantomData::<P>,
cosign_db: &self.cosign_db,
tributary_txn: &mut tributary_txn,
set: self.set,
validators: &self.validators,
total_weight: self.total_weight,
validator_weights: &self.validator_weights,
})
.handle_block(block_number, block);
TributaryDb::set_last_handled_tributary_block(
&mut tributary_txn,
self.set,
block_number,
block_hash,
);
last_block_number = block_number;
last_block_hash = block_hash;
tributary_txn.commit();
made_progress = true;
}
Ok(made_progress)
}
}
}